KAFKA-17370 Move LeaderAndIsr to metadata module (#16943)

isrWithBrokerEpoch = addBrokerEpochToIsr(isrToSend.toL
This commit is contained in:
Dmitry Werner 2024-08-22 12:47:09 +05:00 committed by GitHub
parent 0300025097
commit 6cad2c0d67
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
48 changed files with 587 additions and 513 deletions

View File

@ -1,104 +0,0 @@
/**
* 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.api
import org.apache.kafka.common.message.AlterPartitionRequestData.BrokerState
import org.apache.kafka.metadata.LeaderRecoveryState
object LeaderAndIsr {
val InitialLeaderEpoch: Int = 0
val InitialPartitionEpoch: Int = 0
val NoLeader: Int = -1
val NoEpoch: Int = -1
val LeaderDuringDelete: Int = -2
val EpochDuringDelete: Int = -2
def apply(leader: Int, isr: List[Int]): LeaderAndIsr = {
LeaderAndIsr(leader, InitialLeaderEpoch, isr, LeaderRecoveryState.RECOVERED, InitialPartitionEpoch)
}
def apply(leader: Int,
leaderEpoch: Int,
isr: List[Int],
leaderRecoveryState: LeaderRecoveryState,
partitionEpoch: Int): LeaderAndIsr = {
LeaderAndIsr(
leader,
leaderEpoch,
leaderRecoveryState,
isr.map(brokerId => new BrokerState().setBrokerId(brokerId)),
partitionEpoch)
}
def duringDelete(isr: List[Int]): LeaderAndIsr = LeaderAndIsr(LeaderDuringDelete, isr)
}
case class LeaderAndIsr(
leader: Int,
leaderEpoch: Int,
leaderRecoveryState: LeaderRecoveryState,
isrWithBrokerEpoch: List[BrokerState],
// The current epoch for the partition for KRaft controllers. The current ZK version for the
// legacy controllers. The epoch is a monotonically increasing value which is incremented
// after every partition change.
partitionEpoch: Int
) {
def withPartitionEpoch(partitionEpoch: Int): LeaderAndIsr = copy(partitionEpoch = partitionEpoch)
def newLeader(leader: Int): LeaderAndIsr = newLeaderAndIsrWithBrokerEpoch(leader, isrWithBrokerEpoch)
def newLeaderAndIsr(leader: Int, isr: List[Int]): LeaderAndIsr = {
LeaderAndIsr(leader, leaderEpoch + 1, isr, leaderRecoveryState, partitionEpoch)
}
private def newLeaderAndIsrWithBrokerEpoch(leader: Int, isrWithBrokerEpoch: List[BrokerState]): LeaderAndIsr = {
LeaderAndIsr(leader, leaderEpoch + 1, leaderRecoveryState, isrWithBrokerEpoch, partitionEpoch)
}
def newRecoveringLeaderAndIsr(leader: Int, isr: List[Int]): LeaderAndIsr = {
LeaderAndIsr(leader, leaderEpoch + 1, isr, LeaderRecoveryState.RECOVERING, partitionEpoch)
}
def newEpoch: LeaderAndIsr = newLeaderAndIsrWithBrokerEpoch(leader, isrWithBrokerEpoch)
def leaderOpt: Option[Int] = {
if (leader == LeaderAndIsr.NoLeader) None else Some(leader)
}
def isr: List[Int] = {
isrWithBrokerEpoch.map(_.brokerId())
}
def equalsAllowStalePartitionEpoch(other: LeaderAndIsr): Boolean = {
if (this == other) {
true
} else if (other == null) {
false
} else {
leader == other.leader &&
leaderEpoch == other.leaderEpoch &&
isrWithBrokerEpoch.equals(other.isrWithBrokerEpoch) &&
leaderRecoveryState == other.leaderRecoveryState &&
partitionEpoch <= other.partitionEpoch
}
}
override def toString: String = {
s"LeaderAndIsr(leader=$leader, leaderEpoch=$leaderEpoch, isrWithBrokerEpoch=$isrWithBrokerEpoch, leaderRecoveryState=$leaderRecoveryState, partitionEpoch=$partitionEpoch)"
}
}

View File

@ -19,7 +19,6 @@ package kafka.cluster
import java.util.concurrent.locks.ReentrantReadWriteLock import java.util.concurrent.locks.ReentrantReadWriteLock
import java.util.Optional import java.util.Optional
import java.util.concurrent.{CompletableFuture, CopyOnWriteArrayList} import java.util.concurrent.{CompletableFuture, CopyOnWriteArrayList}
import kafka.api.LeaderAndIsr
import kafka.common.UnexpectedAppendOffsetException import kafka.common.UnexpectedAppendOffsetException
import kafka.controller.{KafkaController, StateChangeLogger} import kafka.controller.{KafkaController, StateChangeLogger}
import kafka.log._ import kafka.log._
@ -41,7 +40,7 @@ import org.apache.kafka.common.record.{MemoryRecords, RecordBatch}
import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests._
import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET} import org.apache.kafka.common.requests.OffsetsForLeaderEpochResponse.{UNDEFINED_EPOCH, UNDEFINED_EPOCH_OFFSET}
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState}
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchDataInfo, FetchIsolation, FetchParams, LeaderHwChange, LogAppendInfo, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogReadInfo, LogStartOffsetIncrementReason, VerificationGuard} import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchDataInfo, FetchIsolation, FetchParams, LeaderHwChange, LogAppendInfo, LogOffsetMetadata, LogOffsetSnapshot, LogOffsetsListener, LogReadInfo, LogStartOffsetIncrementReason, VerificationGuard}
import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.metrics.KafkaMetricsGroup
@ -317,8 +316,8 @@ class Partition(val topicPartition: TopicPartition,
// lock to prevent the follower replica log update while checking if the log dir could be replaced with future log. // lock to prevent the follower replica log update while checking if the log dir could be replaced with future log.
private val futureLogLock = new Object() private val futureLogLock = new Object()
// The current epoch for the partition for KRaft controllers. The current ZK version for the legacy controllers. // The current epoch for the partition for KRaft controllers. The current ZK version for the legacy controllers.
@volatile private var partitionEpoch: Int = LeaderAndIsr.InitialPartitionEpoch @volatile private var partitionEpoch: Int = LeaderAndIsr.INITIAL_PARTITION_EPOCH
@volatile private var leaderEpoch: Int = LeaderAndIsr.InitialLeaderEpoch - 1 @volatile private var leaderEpoch: Int = LeaderAndIsr.INITIAL_LEADER_EPOCH - 1
// start offset for 'leaderEpoch' above (leader epoch of the current leader for this partition), // start offset for 'leaderEpoch' above (leader epoch of the current leader for this partition),
// defined when this broker is leader for partition // defined when this broker is leader for partition
@volatile private[cluster] var leaderEpochStartOffsetOpt: Option[Long] = None @volatile private[cluster] var leaderEpochStartOffsetOpt: Option[Long] = None
@ -1775,8 +1774,8 @@ class Partition(val topicPartition: TopicPartition,
// Alternatively, if the update fails, no harm is done since the expanded ISR puts // Alternatively, if the update fails, no harm is done since the expanded ISR puts
// a stricter requirement for advancement of the HW. // a stricter requirement for advancement of the HW.
val isrToSend = partitionState.isr + newInSyncReplicaId val isrToSend = partitionState.isr + newInSyncReplicaId
val isrWithBrokerEpoch = addBrokerEpochToIsr(isrToSend.toList) val isrWithBrokerEpoch = addBrokerEpochToIsr(isrToSend.toList).asJava
val newLeaderAndIsr = LeaderAndIsr( val newLeaderAndIsr = new LeaderAndIsr(
localBrokerId, localBrokerId,
leaderEpoch, leaderEpoch,
partitionState.leaderRecoveryState, partitionState.leaderRecoveryState,
@ -1800,8 +1799,8 @@ class Partition(val topicPartition: TopicPartition,
// erroneously advance the HW if the `AlterPartition` were to fail. Hence the "maximal ISR" // erroneously advance the HW if the `AlterPartition` were to fail. Hence the "maximal ISR"
// for `PendingShrinkIsr` is the the current ISR. // for `PendingShrinkIsr` is the the current ISR.
val isrToSend = partitionState.isr -- outOfSyncReplicaIds val isrToSend = partitionState.isr -- outOfSyncReplicaIds
val isrWithBrokerEpoch = addBrokerEpochToIsr(isrToSend.toList) val isrWithBrokerEpoch = addBrokerEpochToIsr(isrToSend.toList).asJava
val newLeaderAndIsr = LeaderAndIsr( val newLeaderAndIsr = new LeaderAndIsr(
localBrokerId, localBrokerId,
leaderEpoch, leaderEpoch,
partitionState.leaderRecoveryState, partitionState.leaderRecoveryState,
@ -1972,7 +1971,7 @@ class Partition(val topicPartition: TopicPartition,
// 2) leaderAndIsr.partitionEpoch == partitionEpoch: No update was performed since proposed and actual state are the same. // 2) leaderAndIsr.partitionEpoch == partitionEpoch: No update was performed since proposed and actual state are the same.
// In both cases, we want to move from Pending to Committed state to ensure new updates are processed. // In both cases, we want to move from Pending to Committed state to ensure new updates are processed.
partitionState = CommittedPartitionState(leaderAndIsr.isr.toSet, leaderAndIsr.leaderRecoveryState) partitionState = CommittedPartitionState(leaderAndIsr.isr.asScala.map(_.toInt).toSet, leaderAndIsr.leaderRecoveryState)
partitionEpoch = leaderAndIsr.partitionEpoch partitionEpoch = leaderAndIsr.partitionEpoch
info(s"ISR updated to ${partitionState.isr.mkString(",")} ${if (isUnderMinIsr) "(under-min-isr)" else ""} " + info(s"ISR updated to ${partitionState.isr.mkString(",")} ${if (isUnderMinIsr) "(under-min-isr)" else ""} " +
s"and version updated to $partitionEpoch") s"and version updated to $partitionEpoch")

View File

@ -17,7 +17,6 @@
package kafka.controller package kafka.controller
import com.yammer.metrics.core.{Gauge, Timer} import com.yammer.metrics.core.{Gauge, Timer}
import kafka.api._
import kafka.cluster.Broker import kafka.cluster.Broker
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.Implicits._ import kafka.utils.Implicits._
@ -34,6 +33,7 @@ import org.apache.kafka.common.requests._
import org.apache.kafka.common.security.JaasContext import org.apache.kafka.common.security.JaasContext
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.common.utils.{LogContext, Time}
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.common.MetadataVersion._ import org.apache.kafka.server.common.MetadataVersion._
import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.metrics.KafkaMetricsGroup
@ -429,7 +429,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
.setControllerEpoch(leaderIsrAndControllerEpoch.controllerEpoch) .setControllerEpoch(leaderIsrAndControllerEpoch.controllerEpoch)
.setLeader(leaderAndIsr.leader) .setLeader(leaderAndIsr.leader)
.setLeaderEpoch(leaderAndIsr.leaderEpoch) .setLeaderEpoch(leaderAndIsr.leaderEpoch)
.setIsr(leaderAndIsr.isr.map(Integer.valueOf).asJava) .setIsr(leaderAndIsr.isr)
.setPartitionEpoch(leaderAndIsr.partitionEpoch) .setPartitionEpoch(leaderAndIsr.partitionEpoch)
.setReplicas(replicaAssignment.replicas.map(Integer.valueOf).asJava) .setReplicas(replicaAssignment.replicas.map(Integer.valueOf).asJava)
.setAddingReplicas(replicaAssignment.addingReplicas.map(Integer.valueOf).asJava) .setAddingReplicas(replicaAssignment.addingReplicas.map(Integer.valueOf).asJava)
@ -478,7 +478,7 @@ abstract class AbstractControllerBrokerRequestBatch(config: KafkaConfig,
else leaderAndIsr else leaderAndIsr
addUpdateMetadataRequestForBrokers(brokerIds, controllerEpoch, partition, addUpdateMetadataRequestForBrokers(brokerIds, controllerEpoch, partition,
updatedLeaderAndIsr.leader, updatedLeaderAndIsr.leaderEpoch, updatedLeaderAndIsr.partitionEpoch, updatedLeaderAndIsr.leader, updatedLeaderAndIsr.leaderEpoch, updatedLeaderAndIsr.partitionEpoch,
updatedLeaderAndIsr.isr, replicas, offlineReplicas) updatedLeaderAndIsr.isr.asScala.map(_.toInt).toList, replicas, offlineReplicas)
case None => case None =>
info(s"Leader not yet assigned for partition $partition. Skip sending UpdateMetadataRequest.") info(s"Leader not yet assigned for partition $partition. Skip sending UpdateMetadataRequest.")
} }

View File

@ -17,10 +17,10 @@
package kafka.controller package kafka.controller
import kafka.api.LeaderAndIsr
import kafka.cluster.Broker import kafka.cluster.Broker
import kafka.utils.Implicits._ import kafka.utils.Implicits._
import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.{TopicPartition, Uuid}
import org.apache.kafka.metadata.LeaderAndIsr
import scala.collection.{Map, Seq, Set, mutable} import scala.collection.{Map, Seq, Set, mutable}
@ -458,11 +458,11 @@ class ControllerContext extends ControllerChannelContext {
// A sentinel (-2) is used as an epoch if the topic is queued for deletion. It overrides // A sentinel (-2) is used as an epoch if the topic is queued for deletion. It overrides
// any existing epoch. // any existing epoch.
if (isTopicQueuedUpForDeletion(partition.topic)) { if (isTopicQueuedUpForDeletion(partition.topic)) {
LeaderAndIsr.EpochDuringDelete LeaderAndIsr.EPOCH_DURING_DELETE
} else { } else {
partitionLeadershipInfo.get(partition) partitionLeadershipInfo.get(partition)
.map(_.leaderAndIsr.leaderEpoch) .map(_.leaderAndIsr.leaderEpoch)
.getOrElse(LeaderAndIsr.NoEpoch) .getOrElse(LeaderAndIsr.NO_EPOCH)
} }
} }

View File

@ -16,10 +16,11 @@
*/ */
package kafka.controller package kafka.controller
import kafka.api.LeaderAndIsr
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.metadata.LeaderAndIsr
import scala.collection.Seq import scala.collection.Seq
import scala.jdk.CollectionConverters._
case class ElectionResult(topicPartition: TopicPartition, leaderAndIsr: Option[LeaderAndIsr], liveReplicas: Seq[Int]) case class ElectionResult(topicPartition: TopicPartition, leaderAndIsr: Option[LeaderAndIsr], liveReplicas: Seq[Int])
@ -35,19 +36,19 @@ object Election {
val liveReplicas = assignment.filter(replica => controllerContext.isReplicaOnline(replica, partition)) val liveReplicas = assignment.filter(replica => controllerContext.isReplicaOnline(replica, partition))
leaderAndIsrOpt match { leaderAndIsrOpt match {
case Some(leaderAndIsr) => case Some(leaderAndIsr) =>
val isr = leaderAndIsr.isr val isr = leaderAndIsr.isr.asScala.map(_.toInt)
val leaderOpt = PartitionLeaderElectionAlgorithms.offlinePartitionLeaderElection( val leaderOpt = PartitionLeaderElectionAlgorithms.offlinePartitionLeaderElection(
assignment, isr, liveReplicas.toSet, uncleanLeaderElectionEnabled, controllerContext) assignment, isr, liveReplicas.toSet, uncleanLeaderElectionEnabled, controllerContext)
val newLeaderAndIsrOpt = leaderOpt.map { leader => val newLeaderAndIsrOpt = leaderOpt.map { leader =>
val newIsr = if (isr.contains(leader)) isr.filter(replica => controllerContext.isReplicaOnline(replica, partition)) val newIsr = if (isr.contains(leader)) isr.filter(replica => controllerContext.isReplicaOnline(replica, partition))
else List(leader) else List(leader)
val newIsrAsJava = newIsr.map(Integer.valueOf).asJava
if (!isr.contains(leader) && isLeaderRecoverySupported) { if (!isr.contains(leader) && isLeaderRecoverySupported) {
// The new leader is not in the old ISR so mark the partition a RECOVERING // The new leader is not in the old ISR so mark the partition a RECOVERING
leaderAndIsr.newRecoveringLeaderAndIsr(leader, newIsr) leaderAndIsr.newRecoveringLeaderAndIsr(leader, newIsrAsJava)
} else { } else {
// Elect a new leader but keep the previous leader recovery state // Elect a new leader but keep the previous leader recovery state
leaderAndIsr.newLeaderAndIsr(leader, newIsr) leaderAndIsr.newLeaderAndIsr(leader, newIsrAsJava)
} }
} }
ElectionResult(partition, newLeaderAndIsrOpt, liveReplicas) ElectionResult(partition, newLeaderAndIsrOpt, liveReplicas)
@ -85,7 +86,7 @@ object Election {
val targetReplicas = controllerContext.partitionFullReplicaAssignment(partition).targetReplicas val targetReplicas = controllerContext.partitionFullReplicaAssignment(partition).targetReplicas
val liveReplicas = targetReplicas.filter(replica => controllerContext.isReplicaOnline(replica, partition)) val liveReplicas = targetReplicas.filter(replica => controllerContext.isReplicaOnline(replica, partition))
val isr = leaderAndIsr.isr val isr = leaderAndIsr.isr
val leaderOpt = PartitionLeaderElectionAlgorithms.reassignPartitionLeaderElection(targetReplicas, isr, liveReplicas.toSet) val leaderOpt = PartitionLeaderElectionAlgorithms.reassignPartitionLeaderElection(targetReplicas, isr.asScala.map(_.toInt), liveReplicas.toSet)
val newLeaderAndIsrOpt = leaderOpt.map(leader => leaderAndIsr.newLeader(leader)) val newLeaderAndIsrOpt = leaderOpt.map(leader => leaderAndIsr.newLeader(leader))
ElectionResult(partition, newLeaderAndIsrOpt, targetReplicas) ElectionResult(partition, newLeaderAndIsrOpt, targetReplicas)
} }
@ -112,7 +113,7 @@ object Election {
val assignment = controllerContext.partitionReplicaAssignment(partition) val assignment = controllerContext.partitionReplicaAssignment(partition)
val liveReplicas = assignment.filter(replica => controllerContext.isReplicaOnline(replica, partition)) val liveReplicas = assignment.filter(replica => controllerContext.isReplicaOnline(replica, partition))
val isr = leaderAndIsr.isr val isr = leaderAndIsr.isr
val leaderOpt = PartitionLeaderElectionAlgorithms.preferredReplicaPartitionLeaderElection(assignment, isr, liveReplicas.toSet) val leaderOpt = PartitionLeaderElectionAlgorithms.preferredReplicaPartitionLeaderElection(assignment, isr.asScala.map(_.toInt), liveReplicas.toSet)
val newLeaderAndIsrOpt = leaderOpt.map(leader => leaderAndIsr.newLeader(leader)) val newLeaderAndIsrOpt = leaderOpt.map(leader => leaderAndIsr.newLeader(leader))
ElectionResult(partition, newLeaderAndIsrOpt, assignment) ElectionResult(partition, newLeaderAndIsrOpt, assignment)
} }
@ -140,10 +141,10 @@ object Election {
val assignment = controllerContext.partitionReplicaAssignment(partition) val assignment = controllerContext.partitionReplicaAssignment(partition)
val liveOrShuttingDownReplicas = assignment.filter(replica => val liveOrShuttingDownReplicas = assignment.filter(replica =>
controllerContext.isReplicaOnline(replica, partition, includeShuttingDownBrokers = true)) controllerContext.isReplicaOnline(replica, partition, includeShuttingDownBrokers = true))
val isr = leaderAndIsr.isr val isr = leaderAndIsr.isr.asScala.map(_.toInt)
val leaderOpt = PartitionLeaderElectionAlgorithms.controlledShutdownPartitionLeaderElection(assignment, isr, val leaderOpt = PartitionLeaderElectionAlgorithms.controlledShutdownPartitionLeaderElection(assignment, isr,
liveOrShuttingDownReplicas.toSet, shuttingDownBrokerIds) liveOrShuttingDownReplicas.toSet, shuttingDownBrokerIds)
val newIsr = isr.filter(replica => !shuttingDownBrokerIds.contains(replica)) val newIsr = isr.filter(replica => !shuttingDownBrokerIds.contains(replica)).map(Integer.valueOf).asJava
val newLeaderAndIsrOpt = leaderOpt.map(leader => leaderAndIsr.newLeaderAndIsr(leader, newIsr)) val newLeaderAndIsrOpt = leaderOpt.map(leader => leaderAndIsr.newLeaderAndIsr(leader, newIsr))
ElectionResult(partition, newLeaderAndIsrOpt, liveOrShuttingDownReplicas) ElectionResult(partition, newLeaderAndIsrOpt, liveOrShuttingDownReplicas)
} }

View File

@ -19,7 +19,6 @@ package kafka.controller
import com.yammer.metrics.core.{Meter, Timer} import com.yammer.metrics.core.{Meter, Timer}
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import kafka.api._
import kafka.common._ import kafka.common._
import kafka.cluster.Broker import kafka.cluster.Broker
import kafka.controller.KafkaController.{ActiveBrokerCountMetricName, ActiveControllerCountMetricName, AlterReassignmentsCallback, ControllerStateMetricName, ElectLeadersCallback, FencedBrokerCountMetricName, GlobalPartitionCountMetricName, GlobalTopicCountMetricName, ListReassignmentsCallback, OfflinePartitionsCountMetricName, PreferredReplicaImbalanceCountMetricName, ReplicasIneligibleToDeleteCountMetricName, ReplicasToDeleteCountMetricName, TopicsIneligibleToDeleteCountMetricName, TopicsToDeleteCountMetricName, UpdateFeaturesCallback, ZkMigrationStateMetricName} import kafka.controller.KafkaController.{ActiveBrokerCountMetricName, ActiveControllerCountMetricName, AlterReassignmentsCallback, ControllerStateMetricName, ElectLeadersCallback, FencedBrokerCountMetricName, GlobalPartitionCountMetricName, GlobalTopicCountMetricName, ListReassignmentsCallback, OfflinePartitionsCountMetricName, PreferredReplicaImbalanceCountMetricName, ReplicasIneligibleToDeleteCountMetricName, ReplicasToDeleteCountMetricName, TopicsIneligibleToDeleteCountMetricName, TopicsToDeleteCountMetricName, UpdateFeaturesCallback, ZkMigrationStateMetricName}
@ -43,7 +42,7 @@ import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.{AbstractControlRequest, ApiError, LeaderAndIsrResponse, UpdateFeaturesRequest, UpdateMetadataResponse} import org.apache.kafka.common.requests.{AbstractControlRequest, ApiError, LeaderAndIsrResponse, UpdateFeaturesRequest, UpdateMetadataResponse}
import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.utils.{Time, Utils}
import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState}
import org.apache.kafka.metadata.migration.ZkMigrationState import org.apache.kafka.metadata.migration.ZkMigrationState
import org.apache.kafka.server.common.{AdminOperationException, ProducerIdsBlock} import org.apache.kafka.server.common.{AdminOperationException, ProducerIdsBlock}
import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.metrics.KafkaMetricsGroup
@ -1041,7 +1040,7 @@ class KafkaController(val config: KafkaConfig,
true true
} else { } else {
zkClient.getTopicPartitionStates(Seq(partition)).get(partition).exists { leaderIsrAndControllerEpoch => zkClient.getTopicPartitionStates(Seq(partition)).get(partition).exists { leaderIsrAndControllerEpoch =>
val isr = leaderIsrAndControllerEpoch.leaderAndIsr.isr.toSet val isr = leaderIsrAndControllerEpoch.leaderAndIsr.isr.asScala.toSet.map(Int.unbox)
val targetReplicas = assignment.targetReplicas.toSet val targetReplicas = assignment.targetReplicas.toSet
targetReplicas.subsetOf(isr) targetReplicas.subsetOf(isr)
} }
@ -1324,7 +1323,7 @@ class KafkaController(val config: KafkaConfig,
private def canPreferredReplicaBeLeader(tp: TopicPartition): Boolean = { private def canPreferredReplicaBeLeader(tp: TopicPartition): Boolean = {
val assignment = controllerContext.partitionReplicaAssignment(tp) val assignment = controllerContext.partitionReplicaAssignment(tp)
val liveReplicas = assignment.filter(replica => controllerContext.isReplicaOnline(replica, tp)) val liveReplicas = assignment.filter(replica => controllerContext.isReplicaOnline(replica, tp))
val isr = controllerContext.partitionLeadershipInfo(tp).get.leaderAndIsr.isr val isr = controllerContext.partitionLeadershipInfo(tp).get.leaderAndIsr.isr.asScala.toSeq.map(_.toInt)
PartitionLeaderElectionAlgorithms PartitionLeaderElectionAlgorithms
.preferredReplicaPartitionLeaderElection(assignment, isr, liveReplicas.toSet) .preferredReplicaPartitionLeaderElection(assignment, isr, liveReplicas.toSet)
.nonEmpty .nonEmpty
@ -2241,7 +2240,7 @@ class KafkaController(val config: KafkaConfig,
case ElectionType.UNCLEAN => case ElectionType.UNCLEAN =>
val currentLeader = controllerContext.partitionLeadershipInfo(partition).get.leaderAndIsr.leader val currentLeader = controllerContext.partitionLeadershipInfo(partition).get.leaderAndIsr.leader
currentLeader == LeaderAndIsr.NoLeader || !controllerContext.isLiveBroker(currentLeader) currentLeader == LeaderAndIsr.NO_LEADER || !controllerContext.isLiveBroker(currentLeader)
} }
} }
@ -2367,13 +2366,13 @@ class KafkaController(val config: KafkaConfig,
case Some(topicName) => case Some(topicName) =>
topicReq.partitions.forEach { partitionReq => topicReq.partitions.forEach { partitionReq =>
val isr = if (alterPartitionRequestVersion >= 3) { val isr = if (alterPartitionRequestVersion >= 3) {
partitionReq.newIsrWithEpochs.asScala.toList.map(brokerState => brokerState.brokerId()) partitionReq.newIsrWithEpochs.asScala.toList.map(brokerState => Integer.valueOf(brokerState.brokerId())).asJava
} else { } else {
partitionReq.newIsr.asScala.toList.map(_.toInt) partitionReq.newIsr
} }
partitionsToAlter.put( partitionsToAlter.put(
new TopicPartition(topicName, partitionReq.partitionIndex), new TopicPartition(topicName, partitionReq.partitionIndex),
LeaderAndIsr( new LeaderAndIsr(
alterPartitionRequest.brokerId, alterPartitionRequest.brokerId,
partitionReq.leaderEpoch, partitionReq.leaderEpoch,
isr, isr,
@ -2409,7 +2408,7 @@ class KafkaController(val config: KafkaConfig,
} else if (newLeaderAndIsr.partitionEpoch < currentLeaderAndIsr.partitionEpoch) { } else if (newLeaderAndIsr.partitionEpoch < currentLeaderAndIsr.partitionEpoch) {
partitionResponses(tp) = Left(Errors.INVALID_UPDATE_VERSION) partitionResponses(tp) = Left(Errors.INVALID_UPDATE_VERSION)
None None
} else if (newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.length > 1) { } else if (newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.size() > 1) {
partitionResponses(tp) = Left(Errors.INVALID_REQUEST) partitionResponses(tp) = Left(Errors.INVALID_REQUEST)
info( info(
s"Rejecting AlterPartition from node $brokerId for $tp because leader is recovering and ISR is greater than 1: " + s"Rejecting AlterPartition from node $brokerId for $tp because leader is recovering and ISR is greater than 1: " +
@ -2428,7 +2427,7 @@ class KafkaController(val config: KafkaConfig,
} else { } else {
// Pull out replicas being added to ISR and verify they are all online. // Pull out replicas being added to ISR and verify they are all online.
// If a replica is not online, reject the update as specified in KIP-841. // If a replica is not online, reject the update as specified in KIP-841.
val ineligibleReplicas = newLeaderAndIsr.isr.toSet -- controllerContext.liveBrokerIds val ineligibleReplicas = newLeaderAndIsr.isr.asScala.toSet.map(Int.unbox) -- controllerContext.liveBrokerIds
if (ineligibleReplicas.nonEmpty) { if (ineligibleReplicas.nonEmpty) {
info(s"Rejecting AlterPartition request from node $brokerId for $tp because " + info(s"Rejecting AlterPartition request from node $brokerId for $tp because " +
s"it specified ineligible replicas $ineligibleReplicas in the new ISR ${newLeaderAndIsr.isr}." s"it specified ineligible replicas $ineligibleReplicas in the new ISR ${newLeaderAndIsr.isr}."
@ -2511,7 +2510,7 @@ class KafkaController(val config: KafkaConfig,
.setPartitionIndex(tp.partition) .setPartitionIndex(tp.partition)
.setLeaderId(leaderAndIsr.leader) .setLeaderId(leaderAndIsr.leader)
.setLeaderEpoch(leaderAndIsr.leaderEpoch) .setLeaderEpoch(leaderAndIsr.leaderEpoch)
.setIsr(leaderAndIsr.isr.map(Integer.valueOf).asJava) .setIsr(leaderAndIsr.isr)
.setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value) .setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value)
.setPartitionEpoch(leaderAndIsr.partitionEpoch) .setPartitionEpoch(leaderAndIsr.partitionEpoch)
) )
@ -2767,7 +2766,7 @@ case class LeaderIsrAndControllerEpoch(leaderAndIsr: LeaderAndIsr, controllerEpo
override def toString: String = { override def toString: String = {
val leaderAndIsrInfo = new StringBuilder val leaderAndIsrInfo = new StringBuilder
leaderAndIsrInfo.append("(Leader:" + leaderAndIsr.leader) leaderAndIsrInfo.append("(Leader:" + leaderAndIsr.leader)
leaderAndIsrInfo.append(",ISR:" + leaderAndIsr.isr.mkString(",")) leaderAndIsrInfo.append(",ISR:" + leaderAndIsr.isr.asScala.mkString(","))
leaderAndIsrInfo.append(",LeaderRecoveryState:" + leaderAndIsr.leaderRecoveryState) leaderAndIsrInfo.append(",LeaderRecoveryState:" + leaderAndIsr.leaderRecoveryState)
leaderAndIsrInfo.append(",LeaderEpoch:" + leaderAndIsr.leaderEpoch) leaderAndIsrInfo.append(",LeaderEpoch:" + leaderAndIsr.leaderEpoch)
leaderAndIsrInfo.append(",ZkVersion:" + leaderAndIsr.partitionEpoch) leaderAndIsrInfo.append(",ZkVersion:" + leaderAndIsr.partitionEpoch)

View File

@ -16,7 +16,6 @@
*/ */
package kafka.controller package kafka.controller
import kafka.api.LeaderAndIsr
import kafka.common.StateChangeFailedException import kafka.common.StateChangeFailedException
import kafka.controller.Election._ import kafka.controller.Election._
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
@ -27,11 +26,13 @@ import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult
import kafka.zk.TopicPartitionStateZNode import kafka.zk.TopicPartitionStateZNode
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors.ControllerMovedException import org.apache.kafka.common.errors.ControllerMovedException
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.server.common.MetadataVersion.IBP_3_2_IV0 import org.apache.kafka.server.common.MetadataVersion.IBP_3_2_IV0
import org.apache.zookeeper.KeeperException import org.apache.zookeeper.KeeperException
import org.apache.zookeeper.KeeperException.Code import org.apache.zookeeper.KeeperException.Code
import scala.collection.{Map, Seq, mutable} import scala.collection.{Map, Seq, mutable}
import scala.jdk.CollectionConverters._
abstract class PartitionStateMachine(controllerContext: ControllerContext) extends Logging { abstract class PartitionStateMachine(controllerContext: ControllerContext) extends Logging {
/** /**
@ -288,7 +289,7 @@ class ZkPartitionStateMachine(config: KafkaConfig,
logFailedStateChange(partition, NewPartition, OnlinePartition, new StateChangeFailedException(failMsg)) logFailedStateChange(partition, NewPartition, OnlinePartition, new StateChangeFailedException(failMsg))
} }
val leaderIsrAndControllerEpochs = partitionsWithLiveReplicas.map { case (partition, liveReplicas) => val leaderIsrAndControllerEpochs = partitionsWithLiveReplicas.map { case (partition, liveReplicas) =>
val leaderAndIsr = LeaderAndIsr(liveReplicas.head, liveReplicas.toList) val leaderAndIsr = new LeaderAndIsr(liveReplicas.head, liveReplicas.toList.map(Integer.valueOf).asJava)
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerContext.epoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerContext.epoch)
partition -> leaderIsrAndControllerEpoch partition -> leaderIsrAndControllerEpoch
}.toMap }.toMap
@ -308,7 +309,7 @@ class ZkPartitionStateMachine(config: KafkaConfig,
val leaderIsrAndControllerEpoch = leaderIsrAndControllerEpochs(partition) val leaderIsrAndControllerEpoch = leaderIsrAndControllerEpochs(partition)
if (code == Code.OK) { if (code == Code.OK) {
controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch)
controllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(leaderIsrAndControllerEpoch.leaderAndIsr.isr, controllerBrokerRequestBatch.addLeaderAndIsrRequestForBrokers(leaderIsrAndControllerEpoch.leaderAndIsr.isr.asScala.map(_.toInt),
partition, leaderIsrAndControllerEpoch, controllerContext.partitionFullReplicaAssignment(partition), isNew = true) partition, leaderIsrAndControllerEpoch, controllerContext.partitionFullReplicaAssignment(partition), isNew = true)
successfulInitializations += partition successfulInitializations += partition
} else { } else {
@ -475,7 +476,7 @@ class ZkPartitionStateMachine(config: KafkaConfig,
): Seq[(TopicPartition, Option[LeaderAndIsr], Boolean)] = { ): Seq[(TopicPartition, Option[LeaderAndIsr], Boolean)] = {
val (partitionsWithNoLiveInSyncReplicas, partitionsWithLiveInSyncReplicas) = leaderAndIsrs.partition { val (partitionsWithNoLiveInSyncReplicas, partitionsWithLiveInSyncReplicas) = leaderAndIsrs.partition {
case (partition, leaderAndIsr) => case (partition, leaderAndIsr) =>
val liveInSyncReplicas = leaderAndIsr.isr.filter(controllerContext.isReplicaOnline(_, partition)) val liveInSyncReplicas = leaderAndIsr.isr.asScala.filter(controllerContext.isReplicaOnline(_, partition))
liveInSyncReplicas.isEmpty liveInSyncReplicas.isEmpty
} }

View File

@ -16,7 +16,6 @@
*/ */
package kafka.controller package kafka.controller
import kafka.api.LeaderAndIsr
import kafka.common.StateChangeFailedException import kafka.common.StateChangeFailedException
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.Implicits._ import kafka.utils.Implicits._
@ -26,7 +25,10 @@ import kafka.zk.KafkaZkClient.UpdateLeaderAndIsrResult
import kafka.zk.TopicPartitionStateZNode import kafka.zk.TopicPartitionStateZNode
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors.ControllerMovedException import org.apache.kafka.common.errors.ControllerMovedException
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.zookeeper.KeeperException.Code import org.apache.zookeeper.KeeperException.Code
import java.util.stream.Collectors
import scala.collection.{Seq, mutable} import scala.collection.{Seq, mutable}
abstract class ReplicaStateMachine(controllerContext: ControllerContext) extends Logging { abstract class ReplicaStateMachine(controllerContext: ControllerContext) extends Logging {
@ -341,8 +343,10 @@ class ZkReplicaStateMachine(config: KafkaConfig,
val adjustedLeaderAndIsrs: Map[TopicPartition, LeaderAndIsr] = leaderAndIsrsWithReplica.flatMap { val adjustedLeaderAndIsrs: Map[TopicPartition, LeaderAndIsr] = leaderAndIsrsWithReplica.flatMap {
case (partition, result) => case (partition, result) =>
result.toOption.map { leaderAndIsr => result.toOption.map { leaderAndIsr =>
val newLeader = if (replicaId == leaderAndIsr.leader) LeaderAndIsr.NoLeader else leaderAndIsr.leader val newLeader = if (replicaId == leaderAndIsr.leader) LeaderAndIsr.NO_LEADER else leaderAndIsr.leader
val adjustedIsr = if (leaderAndIsr.isr.size == 1) leaderAndIsr.isr else leaderAndIsr.isr.filter(_ != replicaId) val adjustedIsr =
if (leaderAndIsr.isr.size == 1) leaderAndIsr.isr
else leaderAndIsr.isr.stream().filter(_ != replicaId).collect(Collectors.toList[Integer])
partition -> leaderAndIsr.newLeaderAndIsr(newLeader, adjustedIsr) partition -> leaderAndIsr.newLeaderAndIsr(newLeader, adjustedIsr)
} }
} }

View File

@ -250,7 +250,7 @@ class TopicDeletionManager(config: KafkaConfig,
* Invoked with the list of topics to be deleted * Invoked with the list of topics to be deleted
* It invokes onPartitionDeletion for all partitions of a topic. * It invokes onPartitionDeletion for all partitions of a topic.
* The updateMetadataRequest is also going to set the leader for the topics being deleted to * The updateMetadataRequest is also going to set the leader for the topics being deleted to
* [[kafka.api.LeaderAndIsr#LeaderDuringDelete]]. This lets each broker know that this topic is being deleted and can be * [[org.apache.kafka.metadata.LeaderAndIsr#LeaderDuringDelete]]. This lets each broker know that this topic is being deleted and can be
* removed from their caches. * removed from their caches.
*/ */
private def onTopicDeletion(topics: Set[String]): Unit = { private def onTopicDeletion(topics: Set[String]): Unit = {

View File

@ -16,11 +16,11 @@
*/ */
package kafka.migration package kafka.migration
import kafka.api.LeaderAndIsr
import kafka.cluster.Broker import kafka.cluster.Broker
import kafka.controller.{ControllerChannelContext, LeaderIsrAndControllerEpoch} import kafka.controller.{ControllerChannelContext, LeaderIsrAndControllerEpoch}
import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.{TopicPartition, Uuid}
import org.apache.kafka.image.MetadataImage import org.apache.kafka.image.MetadataImage
import org.apache.kafka.metadata.LeaderAndIsr
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
@ -44,7 +44,7 @@ object MigrationControllerChannelContext {
image.topics().topicsByName().asScala.get(topicPartition.topic()) match { image.topics().topicsByName().asScala.get(topicPartition.topic()) match {
case Some(topic) => topic.partitions().asScala.get(topicPartition.partition()) match { case Some(topic) => topic.partitions().asScala.get(topicPartition.partition()) match {
case Some(partition) => case Some(partition) =>
val leaderAndIsr = LeaderAndIsr(partition.leader, partition.leaderEpoch, partition.isr.toList, val leaderAndIsr = new LeaderAndIsr(partition.leader, partition.leaderEpoch, partition.isr.toList.map(Integer.valueOf).asJava,
partition.leaderRecoveryState, partition.partitionEpoch) partition.leaderRecoveryState, partition.partitionEpoch)
Some(LeaderIsrAndControllerEpoch(leaderAndIsr, image.highestOffsetAndEpoch().epoch())) Some(LeaderIsrAndControllerEpoch(leaderAndIsr, image.highestOffsetAndEpoch().epoch()))
case None => None case None => None
@ -95,14 +95,14 @@ sealed class MigrationControllerChannelContext(
override def leaderEpoch(topicPartition: TopicPartition): Int = { override def leaderEpoch(topicPartition: TopicPartition): Int = {
// Topic is deleted use a special sentinel -2 to the indicate the same. // Topic is deleted use a special sentinel -2 to the indicate the same.
if (isTopicQueuedUpForDeletion(topicPartition.topic())) { if (isTopicQueuedUpForDeletion(topicPartition.topic())) {
LeaderAndIsr.EpochDuringDelete LeaderAndIsr.EPOCH_DURING_DELETE
} else { } else {
image.topics().topicsByName.asScala.get(topicPartition.topic()) match { image.topics().topicsByName.asScala.get(topicPartition.topic()) match {
case Some(topic) => topic.partitions().asScala.get(topicPartition.partition()) match { case Some(topic) => topic.partitions().asScala.get(topicPartition.partition()) match {
case Some(partition) => partition.leaderEpoch case Some(partition) => partition.leaderEpoch
case None => LeaderAndIsr.NoEpoch case None => LeaderAndIsr.NO_EPOCH
} }
case None => LeaderAndIsr.NoEpoch case None => LeaderAndIsr.NO_EPOCH
} }
} }
} }

View File

@ -16,7 +16,6 @@
*/ */
package kafka.migration package kafka.migration
import kafka.api.LeaderAndIsr
import kafka.cluster.Broker import kafka.cluster.Broker
import kafka.controller.{ControllerChannelContext, ControllerChannelManager, ReplicaAssignment, StateChangeLogger} import kafka.controller.{ControllerChannelContext, ControllerChannelManager, ReplicaAssignment, StateChangeLogger}
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
@ -25,7 +24,7 @@ import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.requests.AbstractControlRequest import org.apache.kafka.common.requests.AbstractControlRequest
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.image.{ClusterImage, MetadataDelta, MetadataImage, TopicsImage} import org.apache.kafka.image.{ClusterImage, MetadataDelta, MetadataImage, TopicsImage}
import org.apache.kafka.metadata.PartitionRegistration import org.apache.kafka.metadata.{LeaderAndIsr, PartitionRegistration}
import org.apache.kafka.metadata.migration.LegacyPropagator import org.apache.kafka.metadata.migration.LegacyPropagator
import java.util import java.util
@ -174,11 +173,11 @@ class MigrationPropagator(
val offlineReplicas = partitionRegistration.replicas.filter { val offlineReplicas = partitionRegistration.replicas.filter {
MigrationControllerChannelContext.isReplicaOnline(image, _, partitionRegistration.replicas.toSet) MigrationControllerChannelContext.isReplicaOnline(image, _, partitionRegistration.replicas.toSet)
} }
val deletedLeaderAndIsr = LeaderAndIsr.duringDelete(partitionRegistration.isr.toList) val deletedLeaderAndIsr = LeaderAndIsr.duringDelete(partitionRegistration.isr.toList.map(Integer.valueOf).asJava)
requestBatch.addStopReplicaRequestForBrokers(partitionRegistration.replicas, tp, deletePartition = true) requestBatch.addStopReplicaRequestForBrokers(partitionRegistration.replicas, tp, deletePartition = true)
requestBatch.addUpdateMetadataRequestForBrokers( requestBatch.addUpdateMetadataRequestForBrokers(
oldZkBrokers.toSeq, zkControllerEpoch, tp, deletedLeaderAndIsr.leader, deletedLeaderAndIsr.leaderEpoch, oldZkBrokers.toSeq, zkControllerEpoch, tp, deletedLeaderAndIsr.leader, deletedLeaderAndIsr.leaderEpoch,
deletedLeaderAndIsr.partitionEpoch, deletedLeaderAndIsr.isr, partitionRegistration.replicas, offlineReplicas) deletedLeaderAndIsr.partitionEpoch, deletedLeaderAndIsr.isr.asScala.map(_.intValue()).toList, partitionRegistration.replicas, offlineReplicas)
} }
} }

View File

@ -19,7 +19,6 @@ package kafka.server
import java.util import java.util
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.{CompletableFuture, ConcurrentHashMap} import java.util.concurrent.{CompletableFuture, ConcurrentHashMap}
import kafka.api.LeaderAndIsr
import kafka.utils.Logging import kafka.utils.Logging
import kafka.zk.KafkaZkClient import kafka.zk.KafkaZkClient
import org.apache.kafka.clients.ClientResponse import org.apache.kafka.clients.ClientResponse
@ -33,7 +32,7 @@ import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.RequestHeader import org.apache.kafka.common.requests.RequestHeader
import org.apache.kafka.common.requests.{AlterPartitionRequest, AlterPartitionResponse} import org.apache.kafka.common.requests.{AlterPartitionRequest, AlterPartitionResponse}
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState}
import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager}
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.util.Scheduler import org.apache.kafka.server.util.Scheduler
@ -41,7 +40,6 @@ import org.apache.kafka.server.util.Scheduler
import scala.collection.mutable import scala.collection.mutable
import scala.collection.mutable.ListBuffer import scala.collection.mutable.ListBuffer
import scala.compat.java8.OptionConverters._ import scala.compat.java8.OptionConverters._
import scala.jdk.CollectionConverters._
/** /**
* Handles updating the ISR by sending AlterPartition requests to the controller (as of 2.7) or by updating ZK directly * Handles updating the ISR by sending AlterPartition requests to the controller (as of 2.7) or by updating ZK directly
@ -284,7 +282,7 @@ class DefaultAlterPartitionManager(
val partitionData = new AlterPartitionRequestData.PartitionData() val partitionData = new AlterPartitionRequestData.PartitionData()
.setPartitionIndex(item.topicIdPartition.partition) .setPartitionIndex(item.topicIdPartition.partition)
.setLeaderEpoch(item.leaderAndIsr.leaderEpoch) .setLeaderEpoch(item.leaderAndIsr.leaderEpoch)
.setNewIsrWithEpochs(item.leaderAndIsr.isrWithBrokerEpoch.asJava) .setNewIsrWithEpochs(item.leaderAndIsr.isrWithBrokerEpoch)
.setPartitionEpoch(item.leaderAndIsr.partitionEpoch) .setPartitionEpoch(item.leaderAndIsr.partitionEpoch)
if (metadataVersion.isLeaderRecoverySupported) { if (metadataVersion.isLeaderRecoverySupported) {
@ -333,10 +331,10 @@ class DefaultAlterPartitionManager(
LeaderRecoveryState.optionalOf(partition.leaderRecoveryState).asScala match { LeaderRecoveryState.optionalOf(partition.leaderRecoveryState).asScala match {
case Some(leaderRecoveryState) => case Some(leaderRecoveryState) =>
partitionResponses(tp) = Right( partitionResponses(tp) = Right(
LeaderAndIsr( new LeaderAndIsr(
partition.leaderId, partition.leaderId,
partition.leaderEpoch, partition.leaderEpoch,
partition.isr.asScala.toList.map(_.toInt), partition.isr,
leaderRecoveryState, leaderRecoveryState,
partition.partitionEpoch partition.partitionEpoch
) )

View File

@ -17,9 +17,9 @@
package kafka.server package kafka.server
import kafka.api.LeaderAndIsr
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.ApiError import org.apache.kafka.common.requests.ApiError
import org.apache.kafka.metadata.LeaderAndIsr
import scala.collection._ import scala.collection._
@ -97,6 +97,6 @@ class DelayedCreatePartitions(delayMs: Long,
private def isMissingLeader(topic: String, partition: Int): Boolean = { private def isMissingLeader(topic: String, partition: Int): Boolean = {
val partitionInfo = adminManager.metadataCache.getPartitionInfo(topic, partition) val partitionInfo = adminManager.metadataCache.getPartitionInfo(topic, partition)
partitionInfo.forall(_.leader == LeaderAndIsr.NoLeader) partitionInfo.forall(_.leader == LeaderAndIsr.NO_LEADER)
} }
} }

View File

@ -17,7 +17,6 @@
package kafka.server package kafka.server
import com.yammer.metrics.core.Meter import com.yammer.metrics.core.Meter
import kafka.api._
import kafka.cluster.{BrokerEndPoint, Partition, PartitionListener} import kafka.cluster.{BrokerEndPoint, Partition, PartitionListener}
import kafka.controller.{KafkaController, StateChangeLogger} import kafka.controller.{KafkaController, StateChangeLogger}
import kafka.log.remote.RemoteLogManager import kafka.log.remote.RemoteLogManager
@ -53,6 +52,7 @@ import org.apache.kafka.common.requests._
import org.apache.kafka.common.utils.{Exit, Time} import org.apache.kafka.common.utils.{Exit, Time}
import org.apache.kafka.common.{ElectionType, IsolationLevel, Node, TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.{ElectionType, IsolationLevel, Node, TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta} import org.apache.kafka.image.{LocalReplicaChanges, MetadataImage, TopicsDelta}
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.metadata.LeaderConstants.NO_LEADER import org.apache.kafka.metadata.LeaderConstants.NO_LEADER
import org.apache.kafka.server.common import org.apache.kafka.server.common
import org.apache.kafka.server.common.DirectoryEventHandler import org.apache.kafka.server.common.DirectoryEventHandler
@ -525,11 +525,11 @@ class ReplicaManager(val config: KafkaConfig,
// a sentinel value (EpochDuringDelete) overwriting any previous epoch is used. // a sentinel value (EpochDuringDelete) overwriting any previous epoch is used.
// When an older version of the StopReplica request which does not contain the leader // When an older version of the StopReplica request which does not contain the leader
// epoch, a sentinel value (NoEpoch) is used and bypass the epoch validation. // epoch, a sentinel value (NoEpoch) is used and bypass the epoch validation.
if (requestLeaderEpoch == LeaderAndIsr.EpochDuringDelete || if (requestLeaderEpoch == LeaderAndIsr.EPOCH_DURING_DELETE ||
requestLeaderEpoch == LeaderAndIsr.NoEpoch || requestLeaderEpoch == LeaderAndIsr.NO_EPOCH ||
requestLeaderEpoch >= currentLeaderEpoch) { requestLeaderEpoch >= currentLeaderEpoch) {
stoppedPartitions += StopPartition(topicPartition, deletePartition, stoppedPartitions += StopPartition(topicPartition, deletePartition,
deletePartition && partition.isLeader && requestLeaderEpoch == LeaderAndIsr.EpochDuringDelete) deletePartition && partition.isLeader && requestLeaderEpoch == LeaderAndIsr.EPOCH_DURING_DELETE)
// Assume that everything will go right. It is overwritten in case of an error. // Assume that everything will go right. It is overwritten in case of an error.
responseMap.put(topicPartition, Errors.NONE) responseMap.put(topicPartition, Errors.NONE)
} else { } else {

View File

@ -22,10 +22,10 @@ import org.apache.kafka.common.TopicPartition
import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.atomic.AtomicLong
import java.util.concurrent.CompletableFuture import java.util.concurrent.CompletableFuture
import kafka.api.LeaderAndIsr
import org.apache.kafka.common.TopicIdPartition import org.apache.kafka.common.TopicIdPartition
import org.apache.kafka.common.errors.InvalidUpdateVersionException import org.apache.kafka.common.errors.InvalidUpdateVersionException
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.server.util.Scheduler import org.apache.kafka.server.util.Scheduler
import scala.collection.mutable import scala.collection.mutable

View File

@ -23,7 +23,6 @@ import java.util.concurrent.locks.{ReentrantLock, ReentrantReadWriteLock}
import scala.collection.{Seq, Set, mutable} import scala.collection.{Seq, Set, mutable}
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import kafka.cluster.{Broker, EndPoint} import kafka.cluster.{Broker, EndPoint}
import kafka.api._
import kafka.controller.StateChangeLogger import kafka.controller.StateChangeLogger
import kafka.server.{BrokerFeatures, CachedControllerId, KRaftCachedControllerId, MetadataCache, ZkCachedControllerId} import kafka.server.{BrokerFeatures, CachedControllerId, KRaftCachedControllerId, MetadataCache, ZkCachedControllerId}
import kafka.utils.CoreUtils._ import kafka.utils.CoreUtils._
@ -40,6 +39,7 @@ import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.{AbstractControlRequest, ApiVersionsResponse, MetadataResponse, UpdateMetadataRequest} import org.apache.kafka.common.requests.{AbstractControlRequest, ApiVersionsResponse, MetadataResponse, UpdateMetadataRequest}
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion} import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion}
import java.util.concurrent.{ThreadLocalRandom, TimeUnit} import java.util.concurrent.{ThreadLocalRandom, TimeUnit}
@ -134,7 +134,7 @@ object ZkMetadataCache {
.setTopicName(topicName) .setTopicName(topicName)
.setPartitionStates(new util.ArrayList()) .setPartitionStates(new util.ArrayList())
partitions.foreach(partition => { partitions.foreach(partition => {
val lisr = LeaderAndIsr.duringDelete(partition.isr().asScala.map(_.intValue()).toList) val lisr = LeaderAndIsr.duringDelete(partition.isr())
val newPartitionState = new UpdateMetadataPartitionState() val newPartitionState = new UpdateMetadataPartitionState()
.setPartitionIndex(partition.partitionIndex()) .setPartitionIndex(partition.partitionIndex())
.setTopicName(topicName) .setTopicName(topicName)
@ -143,7 +143,7 @@ object ZkMetadataCache {
.setControllerEpoch(requestControllerEpoch) .setControllerEpoch(requestControllerEpoch)
.setReplicas(partition.replicas()) .setReplicas(partition.replicas())
.setZkVersion(lisr.partitionEpoch) .setZkVersion(lisr.partitionEpoch)
.setIsr(lisr.isr.map(Integer.valueOf).asJava) .setIsr(lisr.isr)
topicState.partitionStates().add(newPartitionState) topicState.partitionStates().add(newPartitionState)
}) })
topicState topicState
@ -443,7 +443,7 @@ class ZkMetadataCache(
} }
val partitions = getAllPartitions(snapshot) val partitions = getAllPartitions(snapshot)
.filter { case (_, state) => state.leader != LeaderAndIsr.LeaderDuringDelete } .filter { case (_, state) => state.leader != LeaderAndIsr.LEADER_DURING_DELETE }
.map { case (tp, state) => .map { case (tp, state) =>
new PartitionInfo(tp.topic, tp.partition, node(state.leader), new PartitionInfo(tp.topic, tp.partition, node(state.leader),
state.replicas.asScala.map(node).toArray, state.replicas.asScala.map(node).toArray,
@ -573,7 +573,7 @@ class ZkMetadataCache(
newStates.foreach { state => newStates.foreach { state =>
// per-partition logging here can be very expensive due going through all partitions in the cluster // per-partition logging here can be very expensive due going through all partitions in the cluster
val tp = new TopicPartition(state.topicName, state.partitionIndex) val tp = new TopicPartition(state.topicName, state.partitionIndex)
if (state.leader == LeaderAndIsr.LeaderDuringDelete) { if (state.leader == LeaderAndIsr.LEADER_DURING_DELETE) {
removePartitionInfo(partitionStates, topicIds, tp.topic, tp.partition) removePartitionInfo(partitionStates, topicIds, tp.topic, tp.partition)
if (traceEnabled) if (traceEnabled)
stateChangeLogger.trace(s"Deleted partition $tp from metadata cache in response to UpdateMetadata " + stateChangeLogger.trace(s"Deleted partition $tp from metadata cache in response to UpdateMetadata " +

View File

@ -17,16 +17,18 @@
package kafka.utils package kafka.utils
import kafka.api.LeaderAndIsr
import kafka.controller.LeaderIsrAndControllerEpoch import kafka.controller.LeaderIsrAndControllerEpoch
import kafka.zk._ import kafka.zk._
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.metadata.LeaderAndIsr
import scala.jdk.CollectionConverters._
object ReplicationUtils extends Logging { object ReplicationUtils extends Logging {
def updateLeaderAndIsr(zkClient: KafkaZkClient, partition: TopicPartition, newLeaderAndIsr: LeaderAndIsr, def updateLeaderAndIsr(zkClient: KafkaZkClient, partition: TopicPartition, newLeaderAndIsr: LeaderAndIsr,
controllerEpoch: Int): (Boolean, Int) = { controllerEpoch: Int): (Boolean, Int) = {
debug(s"Updated ISR for $partition to ${newLeaderAndIsr.isr.mkString(",")}") debug(s"Updated ISR for $partition to ${newLeaderAndIsr.isr.asScala.mkString(",")}")
val path = TopicPartitionStateZNode.path(partition) val path = TopicPartitionStateZNode.path(partition)
val newLeaderData = TopicPartitionStateZNode.encode(LeaderIsrAndControllerEpoch(newLeaderAndIsr, controllerEpoch)) val newLeaderData = TopicPartitionStateZNode.encode(LeaderIsrAndControllerEpoch(newLeaderAndIsr, controllerEpoch))
// use the epoch of the controller that made the leadership decision, instead of the current controller epoch // use the epoch of the controller that made the leadership decision, instead of the current controller epoch

View File

@ -17,7 +17,6 @@
package kafka.zk package kafka.zk
import java.util.Properties import java.util.Properties
import kafka.api.LeaderAndIsr
import kafka.cluster.Broker import kafka.cluster.Broker
import kafka.controller.{KafkaController, LeaderIsrAndControllerEpoch, ReplicaAssignment} import kafka.controller.{KafkaController, LeaderIsrAndControllerEpoch, ReplicaAssignment}
import kafka.security.authorizer.AclAuthorizer.{NoAcls, VersionedAcls} import kafka.security.authorizer.AclAuthorizer.{NoAcls, VersionedAcls}
@ -31,6 +30,7 @@ import org.apache.kafka.common.security.JaasUtils
import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation}
import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.utils.{Time, Utils}
import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid}
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState
import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.security.authorizer.AclEntry
import org.apache.kafka.server.config.{ConfigType, ZkConfigs} import org.apache.kafka.server.config.{ConfigType, ZkConfigs}
@ -45,6 +45,7 @@ import org.apache.zookeeper.{CreateMode, KeeperException, OpResult, ZooKeeper}
import java.lang.{Long => JLong} import java.lang.{Long => JLong}
import scala.collection.{Map, Seq, mutable} import scala.collection.{Map, Seq, mutable}
import scala.jdk.CollectionConverters._
sealed trait KRaftRegistrationResult sealed trait KRaftRegistrationResult
case class FailedRegistrationResult() extends KRaftRegistrationResult case class FailedRegistrationResult() extends KRaftRegistrationResult
@ -1156,7 +1157,7 @@ class KafkaZkClient private[zk] (
* @return optional ISR if exists and None otherwise * @return optional ISR if exists and None otherwise
*/ */
def getInSyncReplicasForPartition(partition: TopicPartition): Option[Seq[Int]] = def getInSyncReplicasForPartition(partition: TopicPartition): Option[Seq[Int]] =
getTopicPartitionState(partition).map(_.leaderAndIsr.isr) getTopicPartitionState(partition).map(_.leaderAndIsr.isr.asScala.map(_.toInt))
/** /**

View File

@ -21,7 +21,6 @@ import java.util
import java.util.Properties import java.util.Properties
import com.fasterxml.jackson.annotation.JsonProperty import com.fasterxml.jackson.annotation.JsonProperty
import com.fasterxml.jackson.core.JsonProcessingException import com.fasterxml.jackson.core.JsonProcessingException
import kafka.api.LeaderAndIsr
import kafka.cluster.{Broker, EndPoint} import kafka.cluster.{Broker, EndPoint}
import kafka.common.{NotificationHandler, ZkNodeChangeNotificationListener} import kafka.common.{NotificationHandler, ZkNodeChangeNotificationListener}
import kafka.controller.{IsrChangeNotificationHandler, LeaderIsrAndControllerEpoch, ReplicaAssignment} import kafka.controller.{IsrChangeNotificationHandler, LeaderIsrAndControllerEpoch, ReplicaAssignment}
@ -38,7 +37,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.security.token.delegation.TokenInformation import org.apache.kafka.common.security.token.delegation.TokenInformation
import org.apache.kafka.common.utils.{SecurityUtils, Time} import org.apache.kafka.common.utils.{SecurityUtils, Time}
import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid}
import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState}
import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.security.authorizer.AclEntry
@ -387,7 +386,7 @@ object TopicPartitionStateZNode {
"leader" -> leaderAndIsr.leader, "leader" -> leaderAndIsr.leader,
"leader_epoch" -> leaderAndIsr.leaderEpoch, "leader_epoch" -> leaderAndIsr.leaderEpoch,
"controller_epoch" -> controllerEpoch, "controller_epoch" -> controllerEpoch,
"isr" -> leaderAndIsr.isr.asJava "isr" -> leaderAndIsr.isr
) )
if (leaderAndIsr.leaderRecoveryState != LeaderRecoveryState.RECOVERED) { if (leaderAndIsr.leaderRecoveryState != LeaderRecoveryState.RECOVERED) {
@ -410,7 +409,7 @@ object TopicPartitionStateZNode {
val controllerEpoch = leaderIsrAndEpochInfo("controller_epoch").to[Int] val controllerEpoch = leaderIsrAndEpochInfo("controller_epoch").to[Int]
val zkPathVersion = stat.getVersion val zkPathVersion = stat.getVersion
LeaderIsrAndControllerEpoch(LeaderAndIsr(leader, epoch, isr, recovery, zkPathVersion), controllerEpoch) LeaderIsrAndControllerEpoch(new LeaderAndIsr(leader, epoch, isr.map(Int.box).asJava, recovery, zkPathVersion), controllerEpoch)
} }
} }
} }

View File

@ -17,7 +17,6 @@
package kafka.zk.migration package kafka.zk.migration
import kafka.api.LeaderAndIsr
import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment} import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment}
import kafka.utils.Logging import kafka.utils.Logging
import kafka.zk.TopicZNode.TopicIdReplicaAssignment import kafka.zk.TopicZNode.TopicIdReplicaAssignment
@ -28,7 +27,7 @@ import org.apache.kafka.common.metadata.PartitionRecord
import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.metadata.migration.TopicMigrationClient.TopicVisitorInterest import org.apache.kafka.metadata.migration.TopicMigrationClient.TopicVisitorInterest
import org.apache.kafka.metadata.migration.{MigrationClientException, TopicMigrationClient, ZkMigrationLeadershipState} import org.apache.kafka.metadata.migration.{MigrationClientException, TopicMigrationClient, ZkMigrationLeadershipState}
import org.apache.kafka.metadata.{LeaderRecoveryState, PartitionRegistration} import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState, PartitionRegistration}
import org.apache.kafka.server.config.ConfigType import org.apache.kafka.server.config.ConfigType
import org.apache.zookeeper.CreateMode import org.apache.zookeeper.CreateMode
import org.apache.zookeeper.KeeperException.Code import org.apache.zookeeper.KeeperException.Code
@ -80,7 +79,7 @@ class ZkTopicMigrationClient(zkClient: KafkaZkClient) extends TopicMigrationClie
leaderIsrAndControllerEpochs.get(topicPartition) match { leaderIsrAndControllerEpochs.get(topicPartition) match {
case Some(leaderIsrAndEpoch) => case Some(leaderIsrAndEpoch) =>
record record
.setIsr(leaderIsrAndEpoch.leaderAndIsr.isr.map(Integer.valueOf).asJava) .setIsr(leaderIsrAndEpoch.leaderAndIsr.isr)
.setLeader(leaderIsrAndEpoch.leaderAndIsr.leader) .setLeader(leaderIsrAndEpoch.leaderAndIsr.leader)
.setLeaderEpoch(leaderIsrAndEpoch.leaderAndIsr.leaderEpoch) .setLeaderEpoch(leaderIsrAndEpoch.leaderAndIsr.leaderEpoch)
.setPartitionEpoch(leaderIsrAndEpoch.leaderAndIsr.partitionEpoch) .setPartitionEpoch(leaderIsrAndEpoch.leaderAndIsr.partitionEpoch)
@ -309,10 +308,10 @@ class ZkTopicMigrationClient(zkClient: KafkaZkClient) extends TopicMigrationClie
controllerEpoch: Int controllerEpoch: Int
): (String, Array[Byte]) = { ): (String, Array[Byte]) = {
val path = TopicPartitionStateZNode.path(topicPartition) val path = TopicPartitionStateZNode.path(topicPartition)
val data = TopicPartitionStateZNode.encode(LeaderIsrAndControllerEpoch(LeaderAndIsr( val data = TopicPartitionStateZNode.encode(LeaderIsrAndControllerEpoch(new LeaderAndIsr(
partitionRegistration.leader, partitionRegistration.leader,
partitionRegistration.leaderEpoch, partitionRegistration.leaderEpoch,
partitionRegistration.isr.toList, partitionRegistration.isr.toList.map(Integer.valueOf).asJava,
partitionRegistration.leaderRecoveryState, partitionRegistration.leaderRecoveryState,
partitionRegistration.partitionEpoch), controllerEpoch)) partitionRegistration.partitionEpoch), controllerEpoch))
(path, data) (path, data)

View File

@ -63,6 +63,7 @@ import java.util.Collections.singletonList
import org.apache.kafka.common.message.MetadataRequestData.MetadataRequestTopic import org.apache.kafka.common.message.MetadataRequestData.MetadataRequestTopic
import org.apache.kafka.common.message.WriteTxnMarkersRequestData.{WritableTxnMarker, WritableTxnMarkerTopic} import org.apache.kafka.common.message.WriteTxnMarkersRequestData.{WritableTxnMarker, WritableTxnMarkerTopic}
import org.apache.kafka.coordinator.group.GroupConfig import org.apache.kafka.coordinator.group.GroupConfig
import org.apache.kafka.metadata.LeaderAndIsr
import org.junit.jupiter.api.function.Executable import org.junit.jupiter.api.function.Executable
import scala.annotation.nowarn import scala.annotation.nowarn
@ -470,7 +471,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
.setTopicName(tp.topic) .setTopicName(tp.topic)
.setPartitionStates(Seq(new StopReplicaPartitionState() .setPartitionStates(Seq(new StopReplicaPartitionState()
.setPartitionIndex(tp.partition) .setPartitionIndex(tp.partition)
.setLeaderEpoch(LeaderAndIsr.InitialLeaderEpoch + 2) .setLeaderEpoch(LeaderAndIsr.INITIAL_LEADER_EPOCH + 2)
.setDeletePartition(true)).asJava) .setDeletePartition(true)).asJava)
).asJava ).asJava
new StopReplicaRequest.Builder(ApiKeys.STOP_REPLICA.latestVersion, brokerId, Int.MaxValue, new StopReplicaRequest.Builder(ApiKeys.STOP_REPLICA.latestVersion, brokerId, Int.MaxValue,

View File

@ -982,9 +982,9 @@ class ZkMigrationIntegrationTest {
val lisrMap = zkClient.getTopicPartitionStates(partitions) val lisrMap = zkClient.getTopicPartitionStates(partitions)
lisrMap.size == partitions.size && lisrMap.size == partitions.size &&
lisrMap.forall { case (tp, lisr) => lisrMap.forall { case (tp, lisr) =>
lisr.leaderAndIsr.isr.size == replicationFactor && lisr.leaderAndIsr.isr.size() == replicationFactor &&
lisr.leaderAndIsr.leader >= 0 && lisr.leaderAndIsr.leader >= 0 &&
topicIdReplicaAssignment.exists(_.assignment(tp).replicas == lisr.leaderAndIsr.isr) topicIdReplicaAssignment.exists(_.assignment(tp).replicas == lisr.leaderAndIsr.isr.asScala.map(_.toInt))
} }
}, "Unable to find topic partition metadata") }, "Unable to find topic partition metadata")
} }

View File

@ -1,75 +0,0 @@
/**
* 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.api
import org.apache.kafka.metadata.LeaderRecoveryState
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.Test
final class LeaderAndIsrTest {
@Test
def testRecoveringLeaderAndIsr(): Unit = {
val leaderAndIsr = LeaderAndIsr(1, List(1, 2))
val recoveringLeaderAndIsr = leaderAndIsr.newRecoveringLeaderAndIsr(3, List(3))
assertEquals(3, recoveringLeaderAndIsr.leader)
assertEquals(List(3), recoveringLeaderAndIsr.isr)
assertEquals(LeaderRecoveryState.RECOVERING, recoveringLeaderAndIsr.leaderRecoveryState)
}
@Test
def testNewLeaderAndIsr(): Unit = {
val leaderAndIsr = LeaderAndIsr(1, List(1, 2))
val newLeaderAndIsr = leaderAndIsr.newLeaderAndIsr(2, List(1, 2))
assertEquals(2, newLeaderAndIsr.leader)
assertEquals(List(1, 2), newLeaderAndIsr.isr)
assertEquals(LeaderRecoveryState.RECOVERED, newLeaderAndIsr.leaderRecoveryState)
}
@Test
def testNewLeader(): Unit = {
val leaderAndIsr = LeaderAndIsr(2, List(1, 2, 3))
assertEquals(2, leaderAndIsr.leader)
assertEquals(List(1, 2, 3), leaderAndIsr.isr)
val newLeaderAndIsr = leaderAndIsr.newLeader(3)
assertEquals(3, newLeaderAndIsr.leader)
assertEquals(List(1, 2, 3), newLeaderAndIsr.isr)
}
@Test
def testNewEpoch() : Unit = {
val leaderAndIsr = LeaderAndIsr(3, List(1, 2, 3))
assertEquals(0, leaderAndIsr.leaderEpoch)
val leaderWithNewEpoch = leaderAndIsr.newEpoch
assertEquals(1, leaderWithNewEpoch.leaderEpoch)
}
@Test
def testLeaderOpt() : Unit = {
val leaderAndIsr = LeaderAndIsr(2, List(1, 2, 3))
assertEquals(2, leaderAndIsr.leaderOpt.get)
}
}

View File

@ -18,10 +18,9 @@ package kafka.zk
import TopicPartitionStateZNode.decode import TopicPartitionStateZNode.decode
import TopicPartitionStateZNode.encode import TopicPartitionStateZNode.encode
import kafka.api.LeaderAndIsr
import kafka.controller.LeaderIsrAndControllerEpoch import kafka.controller.LeaderIsrAndControllerEpoch
import kafka.utils.Json import kafka.utils.Json
import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState}
import org.apache.zookeeper.data.Stat import org.apache.zookeeper.data.Stat
import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
@ -37,7 +36,7 @@ final class TopicPartitionStateZNodeTest {
val stat = mock(classOf[Stat]) val stat = mock(classOf[Stat])
when(stat.getVersion).thenReturn(zkVersion) when(stat.getVersion).thenReturn(zkVersion)
val expected = LeaderIsrAndControllerEpoch(LeaderAndIsr(1, 6, List(1), LeaderRecoveryState.RECOVERING, zkVersion), 10) val expected = LeaderIsrAndControllerEpoch(new LeaderAndIsr(1, 6, List(1).map(Int.box).asJava, LeaderRecoveryState.RECOVERING, zkVersion), 10)
assertEquals(Some(expected), decode(encode(expected), stat)) assertEquals(Some(expected), decode(encode(expected), stat))
} }
@ -48,7 +47,7 @@ final class TopicPartitionStateZNodeTest {
val stat = mock(classOf[Stat]) val stat = mock(classOf[Stat])
when(stat.getVersion).thenReturn(zkVersion) when(stat.getVersion).thenReturn(zkVersion)
val expected = LeaderIsrAndControllerEpoch(LeaderAndIsr(1, 6, List(1), LeaderRecoveryState.RECOVERED, zkVersion), 10) val expected = LeaderIsrAndControllerEpoch(new LeaderAndIsr(1, 6, List(1).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, zkVersion), 10)
assertEquals(Some(expected), decode(encode(expected), stat)) assertEquals(Some(expected), decode(encode(expected), stat))
} }
@ -59,14 +58,14 @@ final class TopicPartitionStateZNodeTest {
val stat = mock(classOf[Stat]) val stat = mock(classOf[Stat])
when(stat.getVersion).thenReturn(zkVersion) when(stat.getVersion).thenReturn(zkVersion)
val expected = LeaderIsrAndControllerEpoch(LeaderAndIsr(1, 6, List(1), LeaderRecoveryState.RECOVERED, zkVersion), 10) val expected = LeaderIsrAndControllerEpoch(new LeaderAndIsr(1, 6, List(1).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, zkVersion), 10)
val partitionState = Map( val partitionState = Map(
"version" -> 1, "version" -> 1,
"leader" -> expected.leaderAndIsr.leader, "leader" -> expected.leaderAndIsr.leader,
"leader_epoch" -> expected.leaderAndIsr.leaderEpoch, "leader_epoch" -> expected.leaderAndIsr.leaderEpoch,
"controller_epoch" -> expected.controllerEpoch, "controller_epoch" -> expected.controllerEpoch,
"isr" -> expected.leaderAndIsr.isr.asJava "isr" -> expected.leaderAndIsr.isr
) )
assertEquals(Some(expected), decode(Json.encodeAsBytes(partitionState.asJava), stat)) assertEquals(Some(expected), decode(Json.encodeAsBytes(partitionState.asJava), stat))

View File

@ -21,7 +21,6 @@ import java.lang.{Long => JLong}
import java.util.{Optional, Properties} import java.util.{Optional, Properties}
import java.util.concurrent._ import java.util.concurrent._
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import kafka.api.LeaderAndIsr
import kafka.log._ import kafka.log._
import kafka.server._ import kafka.server._
import kafka.server.metadata.MockConfigRepository import kafka.server.metadata.MockConfigRepository
@ -35,6 +34,7 @@ import org.apache.kafka.common.requests.FetchRequest
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.{TopicPartition, Uuid}
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.ReplicationConfigs
import org.apache.kafka.server.util.MockTime import org.apache.kafka.server.util.MockTime

View File

@ -1180,7 +1180,7 @@ class PartitionTest extends AbstractPartitionTest {
// Expansion does not affect the ISR // Expansion does not affect the ISR
assertEquals(Set[Integer](leader, follower2), partition.partitionState.isr, "ISR") assertEquals(Set[Integer](leader, follower2), partition.partitionState.isr, "ISR")
assertEquals(Set[Integer](leader, follower1, follower2), partition.partitionState.maximalIsr, "ISR") assertEquals(Set[Integer](leader, follower1, follower2), partition.partitionState.maximalIsr, "ISR")
assertEquals(alterPartitionManager.isrUpdates.head.leaderAndIsr.isr.toSet, assertEquals(alterPartitionManager.isrUpdates.head.leaderAndIsr.isr.asScala.toSet,
Set(leader, follower1, follower2), "AlterIsr") Set(leader, follower1, follower2), "AlterIsr")
} }
@ -1425,7 +1425,7 @@ class PartitionTest extends AbstractPartitionTest {
assertEquals(Set(brokerId), partition.inSyncReplicaIds) assertEquals(Set(brokerId), partition.inSyncReplicaIds)
assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.maximalIsr) assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.maximalIsr)
assertEquals(1, alterPartitionManager.isrUpdates.size) assertEquals(1, alterPartitionManager.isrUpdates.size)
assertEquals(Set(brokerId, remoteBrokerId), alterPartitionManager.isrUpdates.head.leaderAndIsr.isr.toSet) assertEquals(Set(brokerId, remoteBrokerId), alterPartitionManager.isrUpdates.head.leaderAndIsr.isr.asScala.toSet)
// Simulate invalid request failure // Simulate invalid request failure
alterPartitionManager.failIsrUpdate(Errors.INVALID_REQUEST) alterPartitionManager.failIsrUpdate(Errors.INVALID_REQUEST)
@ -1481,8 +1481,8 @@ class PartitionTest extends AbstractPartitionTest {
fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = 10L) fetchFollower(partition, replicaId = remoteBrokerId, fetchOffset = 10L)
assertEquals(alterPartitionManager.isrUpdates.size, 1) assertEquals(alterPartitionManager.isrUpdates.size, 1)
val isrItem = alterPartitionManager.isrUpdates.head val isrItem = alterPartitionManager.isrUpdates.head
assertEquals(isrItem.leaderAndIsr.isr, List(brokerId, remoteBrokerId)) assertEquals(isrItem.leaderAndIsr.isr, List(brokerId, remoteBrokerId).map(Int.box).asJava)
isrItem.leaderAndIsr.isrWithBrokerEpoch.foreach { brokerState => isrItem.leaderAndIsr.isrWithBrokerEpoch.asScala.foreach { brokerState =>
// In ZK mode, the broker epochs in the leaderAndIsr should be -1. // In ZK mode, the broker epochs in the leaderAndIsr should be -1.
assertEquals(-1, brokerState.brokerEpoch()) assertEquals(-1, brokerState.brokerEpoch())
} }
@ -1865,7 +1865,7 @@ class PartitionTest extends AbstractPartitionTest {
assertEquals(replicas.toSet, partition.partitionState.maximalIsr) assertEquals(replicas.toSet, partition.partitionState.maximalIsr)
assertEquals(1, alterPartitionManager.isrUpdates.size) assertEquals(1, alterPartitionManager.isrUpdates.size)
val isrUpdate = alterPartitionManager.isrUpdates.head val isrUpdate = alterPartitionManager.isrUpdates.head
isrUpdate.leaderAndIsr.isrWithBrokerEpoch.foreach { brokerState => isrUpdate.leaderAndIsr.isrWithBrokerEpoch.asScala.foreach { brokerState =>
if (brokerState.brokerId() == remoteBrokerId2) { if (brokerState.brokerId() == remoteBrokerId2) {
// remoteBrokerId2 has not received any fetch request yet, it does not have broker epoch. // remoteBrokerId2 has not received any fetch request yet, it does not have broker epoch.
assertEquals(-1, brokerState.brokerEpoch()) assertEquals(-1, brokerState.brokerEpoch())
@ -2070,7 +2070,7 @@ class PartitionTest extends AbstractPartitionTest {
// Try to shrink the ISR // Try to shrink the ISR
partition.maybeShrinkIsr() partition.maybeShrinkIsr()
assertEquals(alterPartitionManager.isrUpdates.size, 1) assertEquals(alterPartitionManager.isrUpdates.size, 1)
assertEquals(alterPartitionManager.isrUpdates.head.leaderAndIsr.isr, List(brokerId)) assertEquals(alterPartitionManager.isrUpdates.head.leaderAndIsr.isr, List(brokerId).map(Int.box).asJava)
assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.isr) assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.isr)
assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.maximalIsr) assertEquals(Set(brokerId, remoteBrokerId), partition.partitionState.maximalIsr)
@ -2157,9 +2157,9 @@ class PartitionTest extends AbstractPartitionTest {
partition.maybeShrinkIsr() partition.maybeShrinkIsr()
assertEquals(0, alterPartitionListener.shrinks.get) assertEquals(0, alterPartitionListener.shrinks.get)
assertEquals(alterPartitionManager.isrUpdates.size, 1) assertEquals(alterPartitionManager.isrUpdates.size, 1)
assertEquals(alterPartitionManager.isrUpdates.head.leaderAndIsr.isr, List(brokerId, remoteBrokerId1)) assertEquals(alterPartitionManager.isrUpdates.head.leaderAndIsr.isr, List(brokerId, remoteBrokerId1).map(Int.box).asJava)
val isrUpdate = alterPartitionManager.isrUpdates.head val isrUpdate = alterPartitionManager.isrUpdates.head
isrUpdate.leaderAndIsr.isrWithBrokerEpoch.foreach { brokerState => isrUpdate.leaderAndIsr.isrWithBrokerEpoch.asScala.foreach { brokerState =>
assertEquals(defaultBrokerEpoch(brokerState.brokerId()), brokerState.brokerEpoch()) assertEquals(defaultBrokerEpoch(brokerState.brokerId()), brokerState.brokerEpoch())
} }
assertEquals(Set(brokerId, remoteBrokerId1, remoteBrokerId2), partition.partitionState.isr) assertEquals(Set(brokerId, remoteBrokerId1, remoteBrokerId2), partition.partitionState.isr)

View File

@ -17,7 +17,6 @@
package kafka.controller package kafka.controller
import java.util.Properties import java.util.Properties
import kafka.api.LeaderAndIsr
import kafka.cluster.{Broker, EndPoint} import kafka.cluster.{Broker, EndPoint}
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.TestUtils import kafka.utils.TestUtils
@ -31,7 +30,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.{AbstractControlRequest, AbstractResponse, LeaderAndIsrRequest, LeaderAndIsrResponse, StopReplicaRequest, StopReplicaResponse, UpdateMetadataRequest, UpdateMetadataResponse} import org.apache.kafka.common.requests.{AbstractControlRequest, AbstractResponse, LeaderAndIsrRequest, LeaderAndIsrResponse, StopReplicaRequest, StopReplicaResponse, UpdateMetadataRequest, UpdateMetadataResponse}
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.{TopicPartition, Uuid}
import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState}
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_0_IV1, IBP_0_10_2_IV0, IBP_0_9_0, IBP_1_0_IV0, IBP_2_2_IV0, IBP_2_4_IV0, IBP_2_4_IV1, IBP_2_6_IV0, IBP_2_8_IV1, IBP_3_2_IV0, IBP_3_4_IV0} import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_0_IV1, IBP_0_10_2_IV0, IBP_0_9_0, IBP_1_0_IV0, IBP_2_2_IV0, IBP_2_4_IV0, IBP_2_4_IV1, IBP_2_6_IV0, IBP_2_8_IV1, IBP_3_2_IV0, IBP_3_4_IV0}
import org.apache.kafka.server.config.{ServerConfigs, ZkConfigs} import org.apache.kafka.server.config.{ServerConfigs, ZkConfigs}
@ -56,9 +55,9 @@ class ControllerChannelManagerTest {
val batch = new MockControllerBrokerRequestBatch(context) val batch = new MockControllerBrokerRequestBatch(context)
val partitions = Map( val partitions = Map(
new TopicPartition("foo", 0) -> LeaderAndIsr(1, List(1, 2)), new TopicPartition("foo", 0) -> new LeaderAndIsr(1, List(1, 2).map(Int.box).asJava),
new TopicPartition("foo", 1) -> LeaderAndIsr(2, List(2, 3)), new TopicPartition("foo", 1) -> new LeaderAndIsr(2, List(2, 3).map(Int.box).asJava),
new TopicPartition("bar", 1) -> LeaderAndIsr(3, List(1, 3)) new TopicPartition("bar", 1) -> new LeaderAndIsr(3, List(1, 3).map(Int.box).asJava)
) )
batch.newBatch() batch.newBatch()
@ -84,7 +83,7 @@ class ControllerChannelManagerTest {
assertEquals(partitions.map { case (k, v) => (k, v.leader) }, assertEquals(partitions.map { case (k, v) => (k, v.leader) },
leaderAndIsrRequest.partitionStates.asScala.map(p => new TopicPartition(p.topicName, p.partitionIndex) -> p.leader).toMap) leaderAndIsrRequest.partitionStates.asScala.map(p => new TopicPartition(p.topicName, p.partitionIndex) -> p.leader).toMap)
assertEquals(partitions.map { case (k, v) => (k, v.isr) }, assertEquals(partitions.map { case (k, v) => (k, v.isr) },
leaderAndIsrRequest.partitionStates.asScala.map(p => new TopicPartition(p.topicName, p.partitionIndex) -> p.isr.asScala).toMap) leaderAndIsrRequest.partitionStates.asScala.map(p => new TopicPartition(p.topicName, p.partitionIndex) -> p.isr).toMap)
applyLeaderAndIsrResponseCallbacks(Errors.NONE, batch.sentRequests(2).toList) applyLeaderAndIsrResponseCallbacks(Errors.NONE, batch.sentRequests(2).toList)
assertEquals(1, batch.sentEvents.size) assertEquals(1, batch.sentEvents.size)
@ -104,7 +103,7 @@ class ControllerChannelManagerTest {
val batch = new MockControllerBrokerRequestBatch(context) val batch = new MockControllerBrokerRequestBatch(context)
val partition = new TopicPartition("foo", 0) val partition = new TopicPartition("foo", 0)
val leaderAndIsr = LeaderAndIsr(1, List(1, 2)) val leaderAndIsr = new LeaderAndIsr(1, List(1, 2).map(Int.box).asJava)
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)
context.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) context.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch)
@ -136,7 +135,7 @@ class ControllerChannelManagerTest {
context.removeLiveBrokers(Set(3)) context.removeLiveBrokers(Set(3))
val partition = new TopicPartition("foo", 0) val partition = new TopicPartition("foo", 0)
val leaderAndIsr = LeaderAndIsr(1, List(1, 2)) val leaderAndIsr = new LeaderAndIsr(1, List(1, 2).map(Int.box).asJava)
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)
context.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) context.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch)
@ -186,9 +185,15 @@ class ControllerChannelManagerTest {
val batch = new MockControllerBrokerRequestBatch(context, config) val batch = new MockControllerBrokerRequestBatch(context, config)
val partition = new TopicPartition("foo", 0) val partition = new TopicPartition("foo", 0)
var leaderAndIsr = LeaderAndIsr(1, List(1, 2)) var leaderAndIsr = new LeaderAndIsr(1, List(1, 2).map(Int.box).asJava)
if (interBrokerProtocolVersion.isAtLeast(IBP_3_2_IV0)) { if (interBrokerProtocolVersion.isAtLeast(IBP_3_2_IV0)) {
leaderAndIsr = leaderAndIsr.copy(leaderRecoveryState = LeaderRecoveryState.RECOVERING) leaderAndIsr = new LeaderAndIsr(
leaderAndIsr.leader(),
leaderAndIsr.leaderEpoch(),
leaderAndIsr.isr(),
LeaderRecoveryState.RECOVERING,
leaderAndIsr.partitionEpoch()
)
} }
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)
@ -239,9 +244,9 @@ class ControllerChannelManagerTest {
val batch = new MockControllerBrokerRequestBatch(context) val batch = new MockControllerBrokerRequestBatch(context)
val partitions = Map( val partitions = Map(
new TopicPartition("foo", 0) -> LeaderAndIsr(1, List(1, 2)), new TopicPartition("foo", 0) -> new LeaderAndIsr(1, List(1, 2).map(Int.box).asJava),
new TopicPartition("foo", 1) -> LeaderAndIsr(2, List(2, 3)), new TopicPartition("foo", 1) -> new LeaderAndIsr(2, List(2, 3).map(Int.box).asJava),
new TopicPartition("bar", 1) -> LeaderAndIsr(3, List(1, 3)) new TopicPartition("bar", 1) -> new LeaderAndIsr(3, List(1, 3).map(Int.box).asJava)
) )
partitions.foreach { case (partition, leaderAndIsr) => partitions.foreach { case (partition, leaderAndIsr) =>
@ -261,7 +266,7 @@ class ControllerChannelManagerTest {
assertEquals(partitions.map { case (k, v) => (k, v.leader) }, assertEquals(partitions.map { case (k, v) => (k, v.leader) },
partitionStates.map(ps => (new TopicPartition(ps.topicName, ps.partitionIndex), ps.leader)).toMap) partitionStates.map(ps => (new TopicPartition(ps.topicName, ps.partitionIndex), ps.leader)).toMap)
assertEquals(partitions.map { case (k, v) => (k, v.isr) }, assertEquals(partitions.map { case (k, v) => (k, v.isr) },
partitionStates.map(ps => (new TopicPartition(ps.topicName, ps.partitionIndex), ps.isr.asScala)).toMap) partitionStates.map(ps => (new TopicPartition(ps.topicName, ps.partitionIndex), ps.isr)).toMap)
val topicStates = updateMetadataRequest.topicStates() val topicStates = updateMetadataRequest.topicStates()
assertEquals(2, topicStates.size) assertEquals(2, topicStates.size)
@ -316,9 +321,9 @@ class ControllerChannelManagerTest {
val batch = new MockControllerBrokerRequestBatch(context) val batch = new MockControllerBrokerRequestBatch(context)
val partitions = Map( val partitions = Map(
new TopicPartition("foo", 0) -> LeaderAndIsr(1, List(1, 2)), new TopicPartition("foo", 0) -> new LeaderAndIsr(1, List(1, 2).map(Int.box).asJava),
new TopicPartition("foo", 1) -> LeaderAndIsr(2, List(2, 3)), new TopicPartition("foo", 1) -> new LeaderAndIsr(2, List(2, 3).map(Int.box).asJava),
new TopicPartition("bar", 1) -> LeaderAndIsr(3, List(1, 3)) new TopicPartition("bar", 1) -> new LeaderAndIsr(3, List(1, 3).map(Int.box).asJava)
) )
partitions.foreach { case (partition, leaderAndIsr) => partitions.foreach { case (partition, leaderAndIsr) =>
@ -340,13 +345,13 @@ class ControllerChannelManagerTest {
assertTrue(updateMetadataRequest.partitionStates.asScala assertTrue(updateMetadataRequest.partitionStates.asScala
.filter(_.topicName == "foo") .filter(_.topicName == "foo")
.map(_.leader) .map(_.leader)
.forall(leaderId => leaderId == LeaderAndIsr.LeaderDuringDelete)) .forall(leaderId => leaderId == LeaderAndIsr.LEADER_DURING_DELETE))
assertEquals(partitions.filter { case (k, _) => k.topic == "bar" }.map { case (k, v) => (k, v.leader) }, assertEquals(partitions.filter { case (k, _) => k.topic == "bar" }.map { case (k, v) => (k, v.leader) },
updateMetadataRequest.partitionStates.asScala.filter(ps => ps.topicName == "bar").map { ps => updateMetadataRequest.partitionStates.asScala.filter(ps => ps.topicName == "bar").map { ps =>
(new TopicPartition(ps.topicName, ps.partitionIndex), ps.leader) }.toMap) (new TopicPartition(ps.topicName, ps.partitionIndex), ps.leader) }.toMap)
assertEquals(partitions.map { case (k, v) => (k, v.isr) }, assertEquals(partitions.map { case (k, v) => (k, v.isr) },
updateMetadataRequest.partitionStates.asScala.map(ps => (new TopicPartition(ps.topicName, ps.partitionIndex), ps.isr.asScala)).toMap) updateMetadataRequest.partitionStates.asScala.map(ps => (new TopicPartition(ps.topicName, ps.partitionIndex), ps.isr)).toMap)
assertEquals(3, updateMetadataRequest.liveBrokers.size) assertEquals(3, updateMetadataRequest.liveBrokers.size)
assertEquals(Set(1, 2, 3), updateMetadataRequest.liveBrokers.asScala.map(_.id).toSet) assertEquals(Set(1, 2, 3), updateMetadataRequest.liveBrokers.asScala.map(_.id).toSet)
@ -459,7 +464,7 @@ class ControllerChannelManagerTest {
val batch = new MockControllerBrokerRequestBatch(context) val batch = new MockControllerBrokerRequestBatch(context)
val partition = new TopicPartition("foo", 0) val partition = new TopicPartition("foo", 0)
val leaderAndIsr = LeaderAndIsr(1, List(1, 2)) val leaderAndIsr = new LeaderAndIsr(1, List(1, 2).map(Int.box).asJava)
context.putPartitionLeadershipInfo(partition, LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)) context.putPartitionLeadershipInfo(partition, LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch))
batch.newBatch() batch.newBatch()
@ -471,7 +476,7 @@ class ControllerChannelManagerTest {
assertEquals(1, sentStopReplicaRequests.size) assertEquals(1, sentStopReplicaRequests.size)
val stopReplicaRequest = sentStopReplicaRequests.head val stopReplicaRequest = sentStopReplicaRequests.head
assertEquals(partitionStates(Map(partition -> LeaderAndDelete(leaderAndIsr, true))), assertEquals(partitionStates(Map(partition -> new LeaderAndDelete(leaderAndIsr, true))),
stopReplicaRequest.partitionStates().asScala) stopReplicaRequest.partitionStates().asScala)
} }
@ -799,7 +804,7 @@ class ControllerChannelManagerTest {
val batch = new MockControllerBrokerRequestBatch(context, config) val batch = new MockControllerBrokerRequestBatch(context, config)
val partition = new TopicPartition("foo", 0) val partition = new TopicPartition("foo", 0)
val leaderAndIsr = LeaderAndIsr(1, List(1, 2)) val leaderAndIsr = new LeaderAndIsr(1, List(1, 2).map(Int.box).asJava)
context.putPartitionLeadershipInfo(partition, LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)) context.putPartitionLeadershipInfo(partition, LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch))
@ -822,7 +827,7 @@ class ControllerChannelManagerTest {
private object LeaderAndDelete { private object LeaderAndDelete {
def apply(leader: Int, deletePartition: Boolean): LeaderAndDelete = def apply(leader: Int, deletePartition: Boolean): LeaderAndDelete =
new LeaderAndDelete(LeaderAndIsr(leader, List()), deletePartition) new LeaderAndDelete(new LeaderAndIsr(leader, List().map(Int.box).asJava), deletePartition)
} }
private def partitionStates(partitions: Map[TopicPartition, LeaderAndDelete], private def partitionStates(partitions: Map[TopicPartition, LeaderAndDelete],
@ -836,7 +841,7 @@ class ControllerChannelManagerTest {
if (version >= 3) { if (version >= 3) {
partitionState.setLeaderEpoch(if (topicsQueuedForDeletion.contains(topicPartition.topic)) partitionState.setLeaderEpoch(if (topicsQueuedForDeletion.contains(topicPartition.topic))
LeaderAndIsr.EpochDuringDelete LeaderAndIsr.EPOCH_DURING_DELETE
else else
leaderAndIsr.leaderEpoch) leaderAndIsr.leaderEpoch)
} }

View File

@ -17,14 +17,15 @@
package kafka.controller package kafka.controller
import kafka.api.LeaderAndIsr
import kafka.cluster.{Broker, EndPoint} import kafka.cluster.{Broker, EndPoint}
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.metadata.LeaderAndIsr
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
import org.junit.jupiter.api.{BeforeEach, Test} import org.junit.jupiter.api.{BeforeEach, Test}
import scala.jdk.CollectionConverters._
class ControllerContextTest { class ControllerContextTest {
@ -179,13 +180,13 @@ class ControllerContextTest {
context.updatePartitionFullReplicaAssignment(tp3, ReplicaAssignment(Seq(1, 2, 3))) context.updatePartitionFullReplicaAssignment(tp3, ReplicaAssignment(Seq(1, 2, 3)))
assertEquals(0, context.preferredReplicaImbalanceCount) assertEquals(0, context.preferredReplicaImbalanceCount)
context.putPartitionLeadershipInfo(tp1, LeaderIsrAndControllerEpoch(LeaderAndIsr(1, List(1, 2, 3)), 0)) context.putPartitionLeadershipInfo(tp1, LeaderIsrAndControllerEpoch(new LeaderAndIsr(1, List(1, 2, 3).map(Int.box).asJava), 0))
assertEquals(0, context.preferredReplicaImbalanceCount) assertEquals(0, context.preferredReplicaImbalanceCount)
context.putPartitionLeadershipInfo(tp2, LeaderIsrAndControllerEpoch(LeaderAndIsr(2, List(2, 3, 1)), 0)) context.putPartitionLeadershipInfo(tp2, LeaderIsrAndControllerEpoch(new LeaderAndIsr(2, List(2, 3, 1).map(Int.box).asJava), 0))
assertEquals(1, context.preferredReplicaImbalanceCount) assertEquals(1, context.preferredReplicaImbalanceCount)
context.putPartitionLeadershipInfo(tp3, LeaderIsrAndControllerEpoch(LeaderAndIsr(3, List(3, 1, 2)), 0)) context.putPartitionLeadershipInfo(tp3, LeaderIsrAndControllerEpoch(new LeaderAndIsr(3, List(3, 1, 2).map(Int.box).asJava), 0))
assertEquals(2, context.preferredReplicaImbalanceCount) assertEquals(2, context.preferredReplicaImbalanceCount)
context.updatePartitionFullReplicaAssignment(tp1, ReplicaAssignment(Seq(2, 3, 1))) context.updatePartitionFullReplicaAssignment(tp1, ReplicaAssignment(Seq(2, 3, 1)))
@ -195,7 +196,7 @@ class ControllerContextTest {
context.queueTopicDeletion(Set(tp3.topic)) context.queueTopicDeletion(Set(tp3.topic))
assertEquals(1, context.preferredReplicaImbalanceCount) assertEquals(1, context.preferredReplicaImbalanceCount)
context.putPartitionLeadershipInfo(tp3, LeaderIsrAndControllerEpoch(LeaderAndIsr(1, List(3, 1, 2)), 0)) context.putPartitionLeadershipInfo(tp3, LeaderIsrAndControllerEpoch(new LeaderAndIsr(1, List(3, 1, 2).map(Int.box).asJava), 0))
assertEquals(1, context.preferredReplicaImbalanceCount) assertEquals(1, context.preferredReplicaImbalanceCount)
context.removeTopic(tp1.topic) context.removeTopic(tp1.topic)
@ -216,7 +217,7 @@ class ControllerContextTest {
context.queueTopicDeletion(Set(topicA)) context.queueTopicDeletion(Set(topicA))
// All partitions in topic will be marked as Offline during deletion procedure // All partitions in topic will be marked as Offline during deletion procedure
context.putPartitionLeadershipInfo(tpA, LeaderIsrAndControllerEpoch(LeaderAndIsr(LeaderAndIsr.NoLeader, List(1, 2, 3)), 0)) context.putPartitionLeadershipInfo(tpA, LeaderIsrAndControllerEpoch(new LeaderAndIsr(LeaderAndIsr.NO_LEADER, List(1, 2, 3).map(Int.box).asJava), 0))
assertEquals(0, context.preferredReplicaImbalanceCount) assertEquals(0, context.preferredReplicaImbalanceCount)
// Initiate topicB's topic deletion before topicA's deletion completes. // Initiate topicB's topic deletion before topicA's deletion completes.

View File

@ -21,7 +21,6 @@ import java.util.Properties
import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingQueue, TimeUnit} import java.util.concurrent.{CompletableFuture, CountDownLatch, LinkedBlockingQueue, TimeUnit}
import java.util.stream.{Stream => JStream} import java.util.stream.{Stream => JStream}
import com.yammer.metrics.core.Timer import com.yammer.metrics.core.Timer
import kafka.api.LeaderAndIsr
import kafka.server.{KafkaConfig, KafkaServer, QuorumTestHarness} import kafka.server.{KafkaConfig, KafkaServer, QuorumTestHarness}
import kafka.utils.TestUtils import kafka.utils.TestUtils
import kafka.zk._ import kafka.zk._
@ -34,7 +33,7 @@ import org.apache.kafka.common.requests.AlterPartitionRequest
import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource
import org.apache.kafka.common.utils.LogCaptureAppender import org.apache.kafka.common.utils.LogCaptureAppender
import org.apache.kafka.common.{ElectionType, TopicPartition, Uuid} import org.apache.kafka.common.{ElectionType, TopicPartition, Uuid}
import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState}
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.ReplicationConfigs
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
@ -276,7 +275,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val tp = new TopicPartition("t", 0) val tp = new TopicPartition("t", 0)
val assignment = Map(tp.partition -> Seq(0)) val assignment = Map(tp.partition -> Seq(0))
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
waitForPartitionState(tp, firstControllerEpoch, 0, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp, firstControllerEpoch, 0, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
} }
@ -290,7 +289,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val tp = new TopicPartition("t", 0) val tp = new TopicPartition("t", 0)
val assignment = Map(tp.partition -> Seq(otherBrokerId, controllerId)) val assignment = Map(tp.partition -> Seq(otherBrokerId, controllerId))
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers.take(1)) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers.take(1))
waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
} }
@ -305,7 +304,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
tp1 -> ReplicaAssignment(Seq(0), Seq(), Seq())) tp1 -> ReplicaAssignment(Seq(0), Seq(), Seq()))
TestUtils.createTopic(zkClient, tp0.topic, partitionReplicaAssignment = assignment, servers = servers) TestUtils.createTopic(zkClient, tp0.topic, partitionReplicaAssignment = assignment, servers = servers)
zkClient.setTopicAssignment(tp0.topic, Some(Uuid.randomUuid()), expandedAssignment, firstControllerEpochZkVersion) zkClient.setTopicAssignment(tp0.topic, Some(Uuid.randomUuid()), expandedAssignment, firstControllerEpochZkVersion)
waitForPartitionState(tp1, firstControllerEpoch, 0, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp1, firstControllerEpoch, 0, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic partition expansion") "failed to get expected partition state upon topic partition expansion")
TestUtils.waitForPartitionMetadata(servers, tp1.topic, tp1.partition) TestUtils.waitForPartitionMetadata(servers, tp1.topic, tp1.partition)
} }
@ -325,7 +324,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
servers(otherBrokerId).shutdown() servers(otherBrokerId).shutdown()
servers(otherBrokerId).awaitShutdown() servers(otherBrokerId).awaitShutdown()
zkClient.setTopicAssignment(tp0.topic, Some(Uuid.randomUuid()), expandedAssignment, firstControllerEpochZkVersion) zkClient.setTopicAssignment(tp0.topic, Some(Uuid.randomUuid()), expandedAssignment, firstControllerEpochZkVersion)
waitForPartitionState(tp1, firstControllerEpoch, controllerId, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp1, firstControllerEpoch, controllerId, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic partition expansion") "failed to get expected partition state upon topic partition expansion")
TestUtils.waitForPartitionMetadata(Seq(servers(controllerId)), tp1.topic, tp1.partition) TestUtils.waitForPartitionMetadata(Seq(servers(controllerId)), tp1.topic, tp1.partition)
} }
@ -344,7 +343,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val reassignment = Map(tp -> ReplicaAssignment(Seq(otherBrokerId), List(), List())) val reassignment = Map(tp -> ReplicaAssignment(Seq(otherBrokerId), List(), List()))
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
zkClient.createPartitionReassignment(reassignment.map { case (k, v) => k -> v.replicas }) zkClient.createPartitionReassignment(reassignment.map { case (k, v) => k -> v.replicas })
waitForPartitionState(tp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.InitialLeaderEpoch + 3, waitForPartitionState(tp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.INITIAL_LEADER_EPOCH + 3,
"failed to get expected partition state after partition reassignment") "failed to get expected partition state after partition reassignment")
TestUtils.waitUntilTrue(() => zkClient.getFullReplicaAssignmentForTopics(Set(tp.topic)) == reassignment, TestUtils.waitUntilTrue(() => zkClient.getFullReplicaAssignmentForTopics(Set(tp.topic)) == reassignment,
"failed to get updated partition assignment on topic znode after partition reassignment") "failed to get updated partition assignment on topic znode after partition reassignment")
@ -383,7 +382,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val reassignment = Map(tp -> ReplicaAssignment(Seq(otherBrokerId), List(), List())) val reassignment = Map(tp -> ReplicaAssignment(Seq(otherBrokerId), List(), List()))
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
zkClient.createPartitionReassignment(reassignment.map { case (k, v) => k -> v.replicas }) zkClient.createPartitionReassignment(reassignment.map { case (k, v) => k -> v.replicas })
waitForPartitionState(tp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.InitialLeaderEpoch + 3, waitForPartitionState(tp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.INITIAL_LEADER_EPOCH + 3,
"with an offline log directory on the target broker, the partition reassignment stalls") "with an offline log directory on the target broker, the partition reassignment stalls")
TestUtils.waitUntilTrue(() => zkClient.getFullReplicaAssignmentForTopics(Set(tp.topic)) == reassignment, TestUtils.waitUntilTrue(() => zkClient.getFullReplicaAssignmentForTopics(Set(tp.topic)) == reassignment,
"failed to get updated partition assignment on topic znode after partition reassignment") "failed to get updated partition assignment on topic znode after partition reassignment")
@ -408,7 +407,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
servers(otherBrokerId).awaitShutdown() servers(otherBrokerId).awaitShutdown()
val controller = getController() val controller = getController()
zkClient.setOrCreatePartitionReassignment(reassignment, controller.kafkaController.controllerContext.epochZkVersion) zkClient.setOrCreatePartitionReassignment(reassignment, controller.kafkaController.controllerContext.epochZkVersion)
waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.InitialLeaderEpoch + 1, waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.INITIAL_LEADER_EPOCH + 1,
"failed to get expected partition state during partition reassignment with offline replica") "failed to get expected partition state during partition reassignment with offline replica")
TestUtils.waitUntilTrue(() => zkClient.reassignPartitionsInProgress, TestUtils.waitUntilTrue(() => zkClient.reassignPartitionsInProgress,
"partition reassignment path should remain while reassignment in progress") "partition reassignment path should remain while reassignment in progress")
@ -426,10 +425,10 @@ class ControllerIntegrationTest extends QuorumTestHarness {
servers(otherBrokerId).shutdown() servers(otherBrokerId).shutdown()
servers(otherBrokerId).awaitShutdown() servers(otherBrokerId).awaitShutdown()
zkClient.createPartitionReassignment(reassignment.map { case (k, v) => k -> v.replicas }) zkClient.createPartitionReassignment(reassignment.map { case (k, v) => k -> v.replicas })
waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.InitialLeaderEpoch + 1, waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.INITIAL_LEADER_EPOCH + 1,
"failed to get expected partition state during partition reassignment with offline replica") "failed to get expected partition state during partition reassignment with offline replica")
servers(otherBrokerId).startup() servers(otherBrokerId).startup()
waitForPartitionState(tp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.InitialLeaderEpoch + 4, waitForPartitionState(tp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.INITIAL_LEADER_EPOCH + 4,
"failed to get expected partition state after partition reassignment") "failed to get expected partition state after partition reassignment")
TestUtils.waitUntilTrue(() => zkClient.getFullReplicaAssignmentForTopics(Set(tp.topic)) == reassignment, TestUtils.waitUntilTrue(() => zkClient.getFullReplicaAssignmentForTopics(Set(tp.topic)) == reassignment,
"failed to get updated partition assignment on topic znode after partition reassignment") "failed to get updated partition assignment on topic znode after partition reassignment")
@ -445,7 +444,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val tp = new TopicPartition("t", 0) val tp = new TopicPartition("t", 0)
val assignment = Map(tp.partition -> Seq(otherBroker.config.brokerId, controllerId)) val assignment = Map(tp.partition -> Seq(otherBroker.config.brokerId, controllerId))
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
preferredReplicaLeaderElection(controllerId, otherBroker, tp, assignment(tp.partition).toSet, LeaderAndIsr.InitialLeaderEpoch) preferredReplicaLeaderElection(controllerId, otherBroker, tp, assignment(tp.partition).toSet, LeaderAndIsr.INITIAL_LEADER_EPOCH)
} }
@Test @Test
@ -456,8 +455,8 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val tp = new TopicPartition("t", 0) val tp = new TopicPartition("t", 0)
val assignment = Map(tp.partition -> Seq(otherBroker.config.brokerId, controllerId)) val assignment = Map(tp.partition -> Seq(otherBroker.config.brokerId, controllerId))
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
preferredReplicaLeaderElection(controllerId, otherBroker, tp, assignment(tp.partition).toSet, LeaderAndIsr.InitialLeaderEpoch) preferredReplicaLeaderElection(controllerId, otherBroker, tp, assignment(tp.partition).toSet, LeaderAndIsr.INITIAL_LEADER_EPOCH)
preferredReplicaLeaderElection(controllerId, otherBroker, tp, assignment(tp.partition).toSet, LeaderAndIsr.InitialLeaderEpoch + 2) preferredReplicaLeaderElection(controllerId, otherBroker, tp, assignment(tp.partition).toSet, LeaderAndIsr.INITIAL_LEADER_EPOCH + 2)
} }
@Test @Test
@ -473,7 +472,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
zkClient.createPreferredReplicaElection(Set(tp)) zkClient.createPreferredReplicaElection(Set(tp))
TestUtils.waitUntilTrue(() => !zkClient.pathExists(PreferredReplicaElectionZNode.path), TestUtils.waitUntilTrue(() => !zkClient.pathExists(PreferredReplicaElectionZNode.path),
"failed to remove preferred replica leader election path after giving up") "failed to remove preferred replica leader election path after giving up")
waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.InitialLeaderEpoch + 1, waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.INITIAL_LEADER_EPOCH + 1,
"failed to get expected partition state upon broker shutdown") "failed to get expected partition state upon broker shutdown")
} }
@ -487,10 +486,10 @@ class ControllerIntegrationTest extends QuorumTestHarness {
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
servers(otherBrokerId).shutdown() servers(otherBrokerId).shutdown()
servers(otherBrokerId).awaitShutdown() servers(otherBrokerId).awaitShutdown()
waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.InitialLeaderEpoch + 1, waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.INITIAL_LEADER_EPOCH + 1,
"failed to get expected partition state upon broker shutdown") "failed to get expected partition state upon broker shutdown")
servers(otherBrokerId).startup() servers(otherBrokerId).startup()
waitForPartitionState(tp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.InitialLeaderEpoch + 2, waitForPartitionState(tp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.INITIAL_LEADER_EPOCH + 2,
"failed to get expected partition state upon broker startup") "failed to get expected partition state upon broker startup")
} }
@ -513,7 +512,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
// Shutdown broker leaderBrokerId so that broker controllerId will be elected as leader for partition tp // Shutdown broker leaderBrokerId so that broker controllerId will be elected as leader for partition tp
servers(leaderBrokerId).shutdown() servers(leaderBrokerId).shutdown()
servers(leaderBrokerId).awaitShutdown() servers(leaderBrokerId).awaitShutdown()
waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.InitialLeaderEpoch + 1, waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.INITIAL_LEADER_EPOCH + 1,
"failed to get expected partition state upon broker shutdown") "failed to get expected partition state upon broker shutdown")
// Shutdown broker otherBrokerId and reassign partition reassigningTp from [controllerId] to [otherBrokerId] // Shutdown broker otherBrokerId and reassign partition reassigningTp from [controllerId] to [otherBrokerId]
@ -522,18 +521,18 @@ class ControllerIntegrationTest extends QuorumTestHarness {
servers(otherBrokerId).awaitShutdown() servers(otherBrokerId).awaitShutdown()
val reassignment = Map(reassigningTp -> ReplicaAssignment(Seq(otherBrokerId), List(), List())) val reassignment = Map(reassigningTp -> ReplicaAssignment(Seq(otherBrokerId), List(), List()))
zkClient.createPartitionReassignment(reassignment.map { case (k, v) => k -> v.replicas }) zkClient.createPartitionReassignment(reassignment.map { case (k, v) => k -> v.replicas })
waitForPartitionState(reassigningTp, firstControllerEpoch, controllerId, LeaderAndIsr.InitialLeaderEpoch + 1, waitForPartitionState(reassigningTp, firstControllerEpoch, controllerId, LeaderAndIsr.INITIAL_LEADER_EPOCH + 1,
"failed to get expected partition state during partition reassignment with offline replica") "failed to get expected partition state during partition reassignment with offline replica")
// Start broker leaderBrokerId and make sure it is elected as leader (preferred) of partition tp automatically // Start broker leaderBrokerId and make sure it is elected as leader (preferred) of partition tp automatically
// even though there is some other ongoing reassignment. // even though there is some other ongoing reassignment.
servers(leaderBrokerId).startup() servers(leaderBrokerId).startup()
waitForPartitionState(tp, firstControllerEpoch, leaderBrokerId, LeaderAndIsr.InitialLeaderEpoch + 2, waitForPartitionState(tp, firstControllerEpoch, leaderBrokerId, LeaderAndIsr.INITIAL_LEADER_EPOCH + 2,
"failed to get expected partition state upon leader broker startup") "failed to get expected partition state upon leader broker startup")
// Start broker otherBrokerId and make sure the reassignment which was stuck can be fulfilled. // Start broker otherBrokerId and make sure the reassignment which was stuck can be fulfilled.
servers(otherBrokerId).startup() servers(otherBrokerId).startup()
waitForPartitionState(reassigningTp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.InitialLeaderEpoch + 4, waitForPartitionState(reassigningTp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.INITIAL_LEADER_EPOCH + 4,
"failed to get expected partition state upon other broker startup") "failed to get expected partition state upon other broker startup")
TestUtils.waitUntilTrue(() => zkClient.getFullReplicaAssignmentForTopics(Set(reassigningTp.topic)) == reassignment, TestUtils.waitUntilTrue(() => zkClient.getFullReplicaAssignmentForTopics(Set(reassigningTp.topic)) == reassignment,
"failed to get updated partition assignment on topic znode after partition reassignment") "failed to get updated partition assignment on topic znode after partition reassignment")
@ -562,12 +561,12 @@ class ControllerIntegrationTest extends QuorumTestHarness {
//Make sure broker leaderBrokerId is elected as leader (preferred) of partition tp automatically //Make sure broker leaderBrokerId is elected as leader (preferred) of partition tp automatically
// even though the reassignment is still ongoing. // even though the reassignment is still ongoing.
waitForPartitionState(tp, firstControllerEpoch, leaderBrokerId, LeaderAndIsr.InitialLeaderEpoch + 2, waitForPartitionState(tp, firstControllerEpoch, leaderBrokerId, LeaderAndIsr.INITIAL_LEADER_EPOCH + 2,
"failed to get expected partition state after auto preferred replica leader election") "failed to get expected partition state after auto preferred replica leader election")
// Start broker otherBrokerId and make sure the reassignment which was stuck can be fulfilled. // Start broker otherBrokerId and make sure the reassignment which was stuck can be fulfilled.
servers(otherBrokerId).startup() servers(otherBrokerId).startup()
waitForPartitionState(tp, firstControllerEpoch, leaderBrokerId, LeaderAndIsr.InitialLeaderEpoch + 3, waitForPartitionState(tp, firstControllerEpoch, leaderBrokerId, LeaderAndIsr.INITIAL_LEADER_EPOCH + 3,
"failed to get expected partition state upon broker startup") "failed to get expected partition state upon broker startup")
TestUtils.waitUntilTrue(() => zkClient.getFullReplicaAssignmentForTopics(Set(tp.topic)) == reassignment, TestUtils.waitUntilTrue(() => zkClient.getFullReplicaAssignmentForTopics(Set(tp.topic)) == reassignment,
"failed to get updated partition assignment on topic znode after partition reassignment") "failed to get updated partition assignment on topic znode after partition reassignment")
@ -583,15 +582,15 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val tp = new TopicPartition("t", 0) val tp = new TopicPartition("t", 0)
val assignment = Map(tp.partition -> Seq(otherBrokerId)) val assignment = Map(tp.partition -> Seq(otherBrokerId))
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
waitForPartitionState(tp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
servers(otherBrokerId).shutdown() servers(otherBrokerId).shutdown()
servers(otherBrokerId).awaitShutdown() servers(otherBrokerId).awaitShutdown()
TestUtils.waitUntilTrue(() => { TestUtils.waitUntilTrue(() => {
val leaderIsrAndControllerEpochMap = zkClient.getTopicPartitionStates(Seq(tp)) val leaderIsrAndControllerEpochMap = zkClient.getTopicPartitionStates(Seq(tp))
leaderIsrAndControllerEpochMap.contains(tp) && leaderIsrAndControllerEpochMap.contains(tp) &&
isExpectedPartitionState(leaderIsrAndControllerEpochMap(tp), firstControllerEpoch, LeaderAndIsr.NoLeader, LeaderAndIsr.InitialLeaderEpoch + 1) && isExpectedPartitionState(leaderIsrAndControllerEpochMap(tp), firstControllerEpoch, LeaderAndIsr.NO_LEADER, LeaderAndIsr.INITIAL_LEADER_EPOCH + 1) &&
leaderIsrAndControllerEpochMap(tp).leaderAndIsr.isr == List(otherBrokerId) leaderIsrAndControllerEpochMap(tp).leaderAndIsr.isr.asScala == List(otherBrokerId)
}, "failed to get expected partition state after entire isr went offline") }, "failed to get expected partition state after entire isr went offline")
} }
@ -603,15 +602,15 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val tp = new TopicPartition("t", 0) val tp = new TopicPartition("t", 0)
val assignment = Map(tp.partition -> Seq(otherBrokerId)) val assignment = Map(tp.partition -> Seq(otherBrokerId))
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
waitForPartitionState(tp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp, firstControllerEpoch, otherBrokerId, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
servers(otherBrokerId).shutdown() servers(otherBrokerId).shutdown()
servers(otherBrokerId).awaitShutdown() servers(otherBrokerId).awaitShutdown()
TestUtils.waitUntilTrue(() => { TestUtils.waitUntilTrue(() => {
val leaderIsrAndControllerEpochMap = zkClient.getTopicPartitionStates(Seq(tp)) val leaderIsrAndControllerEpochMap = zkClient.getTopicPartitionStates(Seq(tp))
leaderIsrAndControllerEpochMap.contains(tp) && leaderIsrAndControllerEpochMap.contains(tp) &&
isExpectedPartitionState(leaderIsrAndControllerEpochMap(tp), firstControllerEpoch, LeaderAndIsr.NoLeader, LeaderAndIsr.InitialLeaderEpoch + 1) && isExpectedPartitionState(leaderIsrAndControllerEpochMap(tp), firstControllerEpoch, LeaderAndIsr.NO_LEADER, LeaderAndIsr.INITIAL_LEADER_EPOCH + 1) &&
leaderIsrAndControllerEpochMap(tp).leaderAndIsr.isr == List(otherBrokerId) leaderIsrAndControllerEpochMap(tp).leaderAndIsr.isr.asScala == List(otherBrokerId)
}, "failed to get expected partition state after entire isr went offline") }, "failed to get expected partition state after entire isr went offline")
} }
@ -769,7 +768,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val assignment = Map(tp.partition -> Seq(0, 1)) val assignment = Map(tp.partition -> Seq(0, 1))
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
waitForPartitionState(tp, firstControllerEpoch, 0, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp, firstControllerEpoch, 0, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
// Wait until the event thread is idle // Wait until the event thread is idle
@ -983,7 +982,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
.setPartitionIndex(tp.partition) .setPartitionIndex(tp.partition)
.setLeaderEpoch(newLeaderAndIsr.leaderEpoch) .setLeaderEpoch(newLeaderAndIsr.leaderEpoch)
.setPartitionEpoch(newLeaderAndIsr.partitionEpoch) .setPartitionEpoch(newLeaderAndIsr.partitionEpoch)
.setNewIsrWithEpochs(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(newLeaderAndIsr.isr.map(Int.box).asJava)) .setNewIsrWithEpochs(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(newLeaderAndIsr.isr))
.setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value) .setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value)
).asJava) ).asJava)
).asJava), alterPartitionVersion > 1).build(alterPartitionVersion).data() ).asJava), alterPartitionVersion > 1).build(alterPartitionVersion).data()
@ -999,7 +998,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
.setLeaderId(brokerId) .setLeaderId(brokerId)
.setLeaderEpoch(newLeaderAndIsr.leaderEpoch) .setLeaderEpoch(newLeaderAndIsr.leaderEpoch)
.setPartitionEpoch(newLeaderAndIsr.partitionEpoch) .setPartitionEpoch(newLeaderAndIsr.partitionEpoch)
.setIsr(newLeaderAndIsr.isr.map(Int.box).asJava) .setIsr(newLeaderAndIsr.isr)
.setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value) .setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value)
).asJava) ).asJava)
).asJava) ).asJava)
@ -1042,7 +1041,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
.setPartitionIndex(tp.partition) .setPartitionIndex(tp.partition)
.setLeaderEpoch(newLeaderAndIsr.leaderEpoch) .setLeaderEpoch(newLeaderAndIsr.leaderEpoch)
.setPartitionEpoch(newLeaderAndIsr.partitionEpoch) .setPartitionEpoch(newLeaderAndIsr.partitionEpoch)
.setNewIsrWithEpochs(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(newLeaderAndIsr.isr.map(Int.box).asJava)) .setNewIsrWithEpochs(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(newLeaderAndIsr.isr))
.setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value) .setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value)
).asJava) ).asJava)
).asJava) ).asJava)
@ -1057,7 +1056,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
.setLeaderId(brokerId) .setLeaderId(brokerId)
.setLeaderEpoch(newLeaderAndIsr.leaderEpoch) .setLeaderEpoch(newLeaderAndIsr.leaderEpoch)
.setPartitionEpoch(newLeaderAndIsr.partitionEpoch) .setPartitionEpoch(newLeaderAndIsr.partitionEpoch)
.setIsr(newLeaderAndIsr.isr.map(Int.box).asJava) .setIsr(newLeaderAndIsr.isr)
.setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value) .setLeaderRecoveryState(newLeaderAndIsr.leaderRecoveryState.value)
).asJava) ).asJava)
).asJava) ).asJava)
@ -1134,7 +1133,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val brokerId = otherBroker.config.brokerId val brokerId = otherBroker.config.brokerId
val tp = new TopicPartition("t", 0) val tp = new TopicPartition("t", 0)
val assignment = Map(tp.partition -> Seq(controllerId, brokerId)) val assignment = Map(tp.partition -> Seq(controllerId, brokerId))
val fullIsr = List(controllerId, brokerId) val fullIsr = List(controllerId, brokerId).map(Int.box).asJava
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
// Shut down follower. // Shut down follower.
@ -1148,14 +1147,14 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val controllerEpoch = controller.controllerContext.liveBrokerIdAndEpochs(controllerId) val controllerEpoch = controller.controllerContext.liveBrokerIdAndEpochs(controllerId)
// We expect only the controller (online broker) to be in ISR // We expect only the controller (online broker) to be in ISR
assertEquals(List(controllerId), leaderAndIsr.isr) assertEquals(List(controllerId).map(Int.box).asJava, leaderAndIsr.isr)
val requestTopic = new AlterPartitionRequestData.TopicData() val requestTopic = new AlterPartitionRequestData.TopicData()
.setPartitions(Seq(new AlterPartitionRequestData.PartitionData() .setPartitions(Seq(new AlterPartitionRequestData.PartitionData()
.setPartitionIndex(tp.partition) .setPartitionIndex(tp.partition)
.setLeaderEpoch(leaderAndIsr.leaderEpoch) .setLeaderEpoch(leaderAndIsr.leaderEpoch)
.setPartitionEpoch(leaderAndIsr.partitionEpoch) .setPartitionEpoch(leaderAndIsr.partitionEpoch)
.setNewIsrWithEpochs(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(fullIsr.map(Int.box).asJava)) .setNewIsrWithEpochs(AlterPartitionRequest.newIsrToSimpleNewIsrWithBrokerEpochs(fullIsr))
.setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value)).asJava) .setLeaderRecoveryState(leaderAndIsr.leaderRecoveryState.value)).asJava)
if (alterPartitionVersion > 1) requestTopic.setTopicId(topicId) else requestTopic.setTopicName(tp.topic) if (alterPartitionVersion > 1) requestTopic.setTopicId(topicId) else requestTopic.setTopicName(tp.topic)
@ -1183,7 +1182,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val newLeaderIsrAndControllerEpochMap = controller.controllerContext.partitionsLeadershipInfo val newLeaderIsrAndControllerEpochMap = controller.controllerContext.partitionsLeadershipInfo
val newLeaderAndIsr = newLeaderIsrAndControllerEpochMap(tp).leaderAndIsr val newLeaderAndIsr = newLeaderIsrAndControllerEpochMap(tp).leaderAndIsr
assertEquals(expectedAlterPartitionResponse, future.get(10, TimeUnit.SECONDS)) assertEquals(expectedAlterPartitionResponse, future.get(10, TimeUnit.SECONDS))
assertEquals(List(controllerId), newLeaderAndIsr.isr) assertEquals(List(controllerId).map(Int.box).asJava, newLeaderAndIsr.isr)
// Bring replica back online. // Bring replica back online.
servers(brokerId).startup() servers(brokerId).startup()
@ -1355,21 +1354,21 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val partitionStateAfterFirstShutdown = controller.controllerContext.partitionLeadershipInfo(tp).get val partitionStateAfterFirstShutdown = controller.controllerContext.partitionLeadershipInfo(tp).get
assertEquals(replica2, partitionStateAfterFirstShutdown.leaderAndIsr.leader) assertEquals(replica2, partitionStateAfterFirstShutdown.leaderAndIsr.leader)
assertEquals(Set(replica2), partitionStateAfterFirstShutdown.leaderAndIsr.isr.toSet) assertEquals(Set(replica2), partitionStateAfterFirstShutdown.leaderAndIsr.isr.asScala.map(_.toInt).toSet)
servers(replica2).shutdown() servers(replica2).shutdown()
servers(replica2).awaitShutdown() servers(replica2).awaitShutdown()
val partitionStateAfterSecondShutdown = controller.controllerContext.partitionLeadershipInfo(tp).get val partitionStateAfterSecondShutdown = controller.controllerContext.partitionLeadershipInfo(tp).get
assertEquals(-1, partitionStateAfterSecondShutdown.leaderAndIsr.leader) assertEquals(-1, partitionStateAfterSecondShutdown.leaderAndIsr.leader)
assertEquals(Set(replica2), partitionStateAfterSecondShutdown.leaderAndIsr.isr.toSet) assertEquals(Set(replica2), partitionStateAfterSecondShutdown.leaderAndIsr.isr.asScala.map(_.toInt).toSet)
servers(replica1).startup() servers(replica1).startup()
TestUtils.waitUntilLeaderIsKnown(servers, tp) TestUtils.waitUntilLeaderIsKnown(servers, tp)
val partitionStateAfterRestart = controller.controllerContext.partitionLeadershipInfo(tp).get val partitionStateAfterRestart = controller.controllerContext.partitionLeadershipInfo(tp).get
assertEquals(replica1, partitionStateAfterRestart.leaderAndIsr.leader) assertEquals(replica1, partitionStateAfterRestart.leaderAndIsr.leader)
assertEquals(Set(replica1), partitionStateAfterRestart.leaderAndIsr.isr.toSet) assertEquals(Set(replica1), partitionStateAfterRestart.leaderAndIsr.isr.asScala.map(_.toInt).toSet)
assertEquals(LeaderRecoveryState.RECOVERING, partitionStateAfterRestart.leaderAndIsr.leaderRecoveryState) assertEquals(LeaderRecoveryState.RECOVERING, partitionStateAfterRestart.leaderAndIsr.leaderRecoveryState)
val leaderId = replica1 val leaderId = replica1
@ -1522,7 +1521,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
TestUtils.createTopic(zkClient, tp1.topic(), assignment1, servers) TestUtils.createTopic(zkClient, tp1.topic(), assignment1, servers)
// Test that the first topic has its ID added correctly // Test that the first topic has its ID added correctly
waitForPartitionState(tp1, firstControllerEpoch, 0, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp1, firstControllerEpoch, 0, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
assertNotEquals(None, controller.controllerContext.topicIds.get("t1")) assertNotEquals(None, controller.controllerContext.topicIds.get("t1"))
val topicId1 = controller.controllerContext.topicIds("t1") val topicId1 = controller.controllerContext.topicIds("t1")
@ -1533,7 +1532,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
TestUtils.createTopic(zkClient, tp2.topic(), assignment2, servers) TestUtils.createTopic(zkClient, tp2.topic(), assignment2, servers)
// Test that the second topic has its ID added correctly // Test that the second topic has its ID added correctly
waitForPartitionState(tp2, firstControllerEpoch, 0, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp2, firstControllerEpoch, 0, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
assertNotEquals(None, controller.controllerContext.topicIds.get("t2")) assertNotEquals(None, controller.controllerContext.topicIds.get("t2"))
val topicId2 = controller.controllerContext.topicIds("t2") val topicId2 = controller.controllerContext.topicIds("t2")
@ -1558,7 +1557,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
TestUtils.createTopic(zkClient, tp1.topic(), assignment1, servers) TestUtils.createTopic(zkClient, tp1.topic(), assignment1, servers)
// Test that the first topic has no topic ID added. // Test that the first topic has no topic ID added.
waitForPartitionState(tp1, firstControllerEpoch, 0, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp1, firstControllerEpoch, 0, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
assertEquals(None, controller.controllerContext.topicIds.get("t1")) assertEquals(None, controller.controllerContext.topicIds.get("t1"))
@ -1567,7 +1566,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
TestUtils.createTopic(zkClient, tp2.topic(), assignment2, servers) TestUtils.createTopic(zkClient, tp2.topic(), assignment2, servers)
// Test that the second topic has no topic ID added. // Test that the second topic has no topic ID added.
waitForPartitionState(tp2, firstControllerEpoch, 0, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp2, firstControllerEpoch, 0, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
assertEquals(None, controller.controllerContext.topicIds.get("t2")) assertEquals(None, controller.controllerContext.topicIds.get("t2"))
@ -1584,7 +1583,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
servers = makeServers(1) servers = makeServers(1)
adminZkClient.createTopic(tp.topic, 1, 1) adminZkClient.createTopic(tp.topic, 1, 1)
waitForPartitionState(tp, firstControllerEpoch, 0, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp, firstControllerEpoch, 0, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
val (topicIdAfterCreate, _) = TestUtils.computeUntilTrue(zkClient.getTopicIdsForTopics(Set(tp.topic)).get(tp.topic))(_.nonEmpty) val (topicIdAfterCreate, _) = TestUtils.computeUntilTrue(zkClient.getTopicIdsForTopics(Set(tp.topic)).get(tp.topic))(_.nonEmpty)
assertTrue(topicIdAfterCreate.isDefined) assertTrue(topicIdAfterCreate.isDefined)
@ -1610,7 +1609,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
servers = makeServers(1, interBrokerProtocolVersion = Some(IBP_2_7_IV0)) servers = makeServers(1, interBrokerProtocolVersion = Some(IBP_2_7_IV0))
adminZkClient.createTopic(tp.topic, 1, 1) adminZkClient.createTopic(tp.topic, 1, 1)
waitForPartitionState(tp, firstControllerEpoch, 0, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp, firstControllerEpoch, 0, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
val (topicIdAfterCreate, _) = TestUtils.computeUntilTrue(zkClient.getTopicIdsForTopics(Set(tp.topic)).get(tp.topic))(_.nonEmpty) val (topicIdAfterCreate, _) = TestUtils.computeUntilTrue(zkClient.getTopicIdsForTopics(Set(tp.topic)).get(tp.topic))(_.nonEmpty)
assertEquals(None, topicIdAfterCreate) assertEquals(None, topicIdAfterCreate)
@ -1636,7 +1635,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val tp = new TopicPartition("t", 0) val tp = new TopicPartition("t", 0)
val assignment = Map(tp.partition -> Seq(controllerId)) val assignment = Map(tp.partition -> Seq(controllerId))
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
val topicId = controller.controllerContext.topicIds("t") val topicId = controller.controllerContext.topicIds("t")
@ -1655,7 +1654,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val tp = new TopicPartition("t", 0) val tp = new TopicPartition("t", 0)
val assignment = Map(tp.partition -> Seq(controllerId)) val assignment = Map(tp.partition -> Seq(controllerId))
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
val emptyTopicId = controller.controllerContext.topicIds.get("t") val emptyTopicId = controller.controllerContext.topicIds.get("t")
assertEquals(None, emptyTopicId) assertEquals(None, emptyTopicId)
@ -1675,7 +1674,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val tp = new TopicPartition("t", 0) val tp = new TopicPartition("t", 0)
val assignment = Map(tp.partition -> Seq(controllerId)) val assignment = Map(tp.partition -> Seq(controllerId))
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
val topicId = controller.controllerContext.topicIds("t") val topicId = controller.controllerContext.topicIds("t")
@ -1695,7 +1694,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
val tp = new TopicPartition("t", 0) val tp = new TopicPartition("t", 0)
val assignment = Map(tp.partition -> Seq(controllerId)) val assignment = Map(tp.partition -> Seq(controllerId))
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp, firstControllerEpoch, controllerId, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
assertEquals(None, zkClient.getTopicIdsForTopics(Set(tp.topic)).get(tp.topic)) assertEquals(None, zkClient.getTopicIdsForTopics(Set(tp.topic)).get(tp.topic))
assertEquals(None, controller.controllerContext.topicIds.get(tp.topic)) assertEquals(None, controller.controllerContext.topicIds.get(tp.topic))
@ -1737,7 +1736,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
// Only the remaining brokers will have the replicas for the partition // Only the remaining brokers will have the replicas for the partition
val assignment = Map(tp.partition -> remainingBrokers.map(_.config.brokerId)) val assignment = Map(tp.partition -> remainingBrokers.map(_.config.brokerId))
TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers) TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
waitForPartitionState(tp, firstControllerEpoch, remainingBrokers(0).config.brokerId, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp, firstControllerEpoch, remainingBrokers(0).config.brokerId, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
val (topicIdAfterCreate, _) = TestUtils.computeUntilTrue(zkClient.getTopicIdsForTopics(Set(tp.topic)).get(tp.topic))(_.nonEmpty) val (topicIdAfterCreate, _) = TestUtils.computeUntilTrue(zkClient.getTopicIdsForTopics(Set(tp.topic)).get(tp.topic))(_.nonEmpty)
assertEquals(None, topicIdAfterCreate) assertEquals(None, topicIdAfterCreate)
@ -1788,7 +1787,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
servers = makeServers(1, interBrokerProtocolVersion = Some(IBP_2_7_IV0)) servers = makeServers(1, interBrokerProtocolVersion = Some(IBP_2_7_IV0))
// use create topic with ZK client directly, without topic ID // use create topic with ZK client directly, without topic ID
adminZkClient.createTopic(tp.topic, 1, 1) adminZkClient.createTopic(tp.topic, 1, 1)
waitForPartitionState(tp, firstControllerEpoch, 0, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp, firstControllerEpoch, 0, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
val topicIdAfterCreate = zkClient.getTopicIdsForTopics(Set(tp.topic())).get(tp.topic()) val topicIdAfterCreate = zkClient.getTopicIdsForTopics(Set(tp.topic())).get(tp.topic())
val id = servers.head.kafkaController.controllerContext.topicIds.get(tp.topic) val id = servers.head.kafkaController.controllerContext.topicIds.get(tp.topic)
@ -1826,7 +1825,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
servers(0).shutdown() servers(0).shutdown()
servers(0).awaitShutdown() servers(0).awaitShutdown()
servers = makeServers(1, interBrokerProtocolVersion = Some(IBP_2_7_IV0)) servers = makeServers(1, interBrokerProtocolVersion = Some(IBP_2_7_IV0))
waitForPartitionState(tp, firstControllerEpoch, 0, LeaderAndIsr.InitialLeaderEpoch, waitForPartitionState(tp, firstControllerEpoch, 0, LeaderAndIsr.INITIAL_LEADER_EPOCH,
"failed to get expected partition state upon topic creation") "failed to get expected partition state upon topic creation")
assertEquals(topicId, awaitTopicId()) assertEquals(topicId, awaitTopicId())

View File

@ -16,10 +16,10 @@
*/ */
package kafka.controller package kafka.controller
import kafka.api.LeaderAndIsr
import kafka.common.StateChangeFailedException import kafka.common.StateChangeFailedException
import kafka.controller.Election._ import kafka.controller.Election._
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.metadata.LeaderAndIsr
import scala.collection.{Seq, mutable} import scala.collection.{Seq, mutable}

View File

@ -16,7 +16,6 @@
*/ */
package kafka.controller package kafka.controller
import kafka.api.LeaderAndIsr
import kafka.cluster.Broker import kafka.cluster.Broker
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.TestUtils import kafka.utils.TestUtils
@ -26,6 +25,7 @@ import kafka.zookeeper._
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.server.common.MetadataVersion.{IBP_3_1_IV0, IBP_3_2_IV0} import org.apache.kafka.server.common.MetadataVersion.{IBP_3_1_IV0, IBP_3_2_IV0}
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.ReplicationConfigs
import org.apache.kafka.storage.internals.log.LogConfig import org.apache.kafka.storage.internals.log.LogConfig
@ -39,6 +39,7 @@ import org.mockito.ArgumentMatchers.{any, anyInt}
import org.mockito.Mockito.{mock, verify, when} import org.mockito.Mockito.{mock, verify, when}
import java.util.Properties import java.util.Properties
import scala.jdk.CollectionConverters._
class PartitionStateMachineTest { class PartitionStateMachineTest {
private var controllerContext: ControllerContext = _ private var controllerContext: ControllerContext = _
@ -93,7 +94,7 @@ class PartitionStateMachineTest {
controllerContext.setLiveBrokers(Map(createBrokerAndEpoch(brokerId, "host", 0))) controllerContext.setLiveBrokers(Map(createBrokerAndEpoch(brokerId, "host", 0)))
controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId)))
controllerContext.putPartitionState(partition, NewPartition) controllerContext.putPartitionState(partition, NewPartition)
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(LeaderAndIsr(brokerId, List(brokerId)), controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(new LeaderAndIsr(brokerId, List(brokerId).map(Int.box).asJava), controllerEpoch)
when(mockZkClient.createTopicPartitionStatesRaw(Map(partition -> leaderIsrAndControllerEpoch), controllerContext.epochZkVersion)) when(mockZkClient.createTopicPartitionStatesRaw(Map(partition -> leaderIsrAndControllerEpoch), controllerContext.epochZkVersion))
.thenReturn(Seq(CreateResponse(Code.OK, null, Some(partition), null, ResponseMetadata(0, 0)))) .thenReturn(Seq(CreateResponse(Code.OK, null, Some(partition), null, ResponseMetadata(0, 0))))
partitionStateMachine.handleStateChanges( partitionStateMachine.handleStateChanges(
@ -114,7 +115,7 @@ class PartitionStateMachineTest {
controllerContext.setLiveBrokers(Map(createBrokerAndEpoch(brokerId, "host", 0))) controllerContext.setLiveBrokers(Map(createBrokerAndEpoch(brokerId, "host", 0)))
controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId)))
controllerContext.putPartitionState(partition, NewPartition) controllerContext.putPartitionState(partition, NewPartition)
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(LeaderAndIsr(brokerId, List(brokerId)), controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(new LeaderAndIsr(brokerId, List(brokerId).map(Int.box).asJava), controllerEpoch)
when(mockZkClient.createTopicPartitionStatesRaw(Map(partition -> leaderIsrAndControllerEpoch), controllerContext.epochZkVersion)) when(mockZkClient.createTopicPartitionStatesRaw(Map(partition -> leaderIsrAndControllerEpoch), controllerContext.epochZkVersion))
.thenThrow(new ZooKeeperClientException("test")) .thenThrow(new ZooKeeperClientException("test"))
partitionStateMachine.handleStateChanges( partitionStateMachine.handleStateChanges(
@ -133,7 +134,7 @@ class PartitionStateMachineTest {
controllerContext.setLiveBrokers(Map(createBrokerAndEpoch(brokerId, "host", 0))) controllerContext.setLiveBrokers(Map(createBrokerAndEpoch(brokerId, "host", 0)))
controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId)))
controllerContext.putPartitionState(partition, NewPartition) controllerContext.putPartitionState(partition, NewPartition)
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(LeaderAndIsr(brokerId, List(brokerId)), controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(new LeaderAndIsr(brokerId, List(brokerId).map(Int.box).asJava), controllerEpoch)
when(mockZkClient.createTopicPartitionStatesRaw(Map(partition -> leaderIsrAndControllerEpoch), controllerContext.epochZkVersion)) when(mockZkClient.createTopicPartitionStatesRaw(Map(partition -> leaderIsrAndControllerEpoch), controllerContext.epochZkVersion))
.thenReturn(Seq(CreateResponse(Code.NODEEXISTS, null, Some(partition), null, ResponseMetadata(0, 0)))) .thenReturn(Seq(CreateResponse(Code.NODEEXISTS, null, Some(partition), null, ResponseMetadata(0, 0))))
partitionStateMachine.handleStateChanges( partitionStateMachine.handleStateChanges(
@ -166,7 +167,7 @@ class PartitionStateMachineTest {
controllerContext.setLiveBrokers(Map(createBrokerAndEpoch(brokerId, "host", 0))) controllerContext.setLiveBrokers(Map(createBrokerAndEpoch(brokerId, "host", 0)))
controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId)))
controllerContext.putPartitionState(partition, OnlinePartition) controllerContext.putPartitionState(partition, OnlinePartition)
val leaderAndIsr = LeaderAndIsr(brokerId, List(brokerId)) val leaderAndIsr = new LeaderAndIsr(brokerId, List(brokerId).map(Int.box).asJava)
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)
controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch)
@ -202,7 +203,7 @@ class PartitionStateMachineTest {
ReplicaAssignment(Seq(brokerId, otherBrokerId)) ReplicaAssignment(Seq(brokerId, otherBrokerId))
) )
controllerContext.putPartitionState(partition, OnlinePartition) controllerContext.putPartitionState(partition, OnlinePartition)
val leaderAndIsr = LeaderAndIsr(brokerId, List(brokerId, otherBrokerId)) val leaderAndIsr = new LeaderAndIsr(brokerId, List(brokerId, otherBrokerId).map(Int.box).asJava)
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)
controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch)
@ -210,7 +211,7 @@ class PartitionStateMachineTest {
when(mockZkClient.getTopicPartitionStatesRaw(partitions)) when(mockZkClient.getTopicPartitionStatesRaw(partitions))
.thenReturn(Seq(GetDataResponse(Code.OK, null, Some(partition), .thenReturn(Seq(GetDataResponse(Code.OK, null, Some(partition),
TopicPartitionStateZNode.encode(leaderIsrAndControllerEpoch), stat, ResponseMetadata(0, 0)))) TopicPartitionStateZNode.encode(leaderIsrAndControllerEpoch), stat, ResponseMetadata(0, 0))))
val leaderAndIsrAfterElection = leaderAndIsr.newLeaderAndIsr(otherBrokerId, List(otherBrokerId)) val leaderAndIsrAfterElection = leaderAndIsr.newLeaderAndIsr(otherBrokerId, List(otherBrokerId).map(Int.box).asJava)
val updatedLeaderAndIsr = leaderAndIsrAfterElection.withPartitionEpoch(2) val updatedLeaderAndIsr = leaderAndIsrAfterElection.withPartitionEpoch(2)
when(mockZkClient.updateLeaderAndIsr(Map(partition -> leaderAndIsrAfterElection), controllerEpoch, controllerContext.epochZkVersion)) when(mockZkClient.updateLeaderAndIsr(Map(partition -> leaderAndIsrAfterElection), controllerEpoch, controllerContext.epochZkVersion))
.thenReturn(UpdateLeaderAndIsrResult(Map(partition -> Right(updatedLeaderAndIsr)), Seq.empty)) .thenReturn(UpdateLeaderAndIsrResult(Map(partition -> Right(updatedLeaderAndIsr)), Seq.empty))
@ -253,7 +254,7 @@ class PartitionStateMachineTest {
controllerContext.setLiveBrokers(Map(createBrokerAndEpoch(brokerId, "host", 0))) controllerContext.setLiveBrokers(Map(createBrokerAndEpoch(brokerId, "host", 0)))
controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId)))
controllerContext.putPartitionState(partition, OfflinePartition) controllerContext.putPartitionState(partition, OfflinePartition)
val leaderAndIsr = LeaderAndIsr(LeaderAndIsr.NoLeader, List(brokerId)) val leaderAndIsr = new LeaderAndIsr(LeaderAndIsr.NO_LEADER, List(brokerId).map(Int.box).asJava)
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)
controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch)
@ -320,7 +321,7 @@ class PartitionStateMachineTest {
) )
controllerContext.putPartitionState(partition, OfflinePartition) controllerContext.putPartitionState(partition, OfflinePartition)
val leaderAndIsr = LeaderAndIsr(leaderBrokerId, List(leaderBrokerId)) val leaderAndIsr = new LeaderAndIsr(leaderBrokerId, List(leaderBrokerId).map(Int.box).asJava)
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)
controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch)
@ -338,9 +339,9 @@ class PartitionStateMachineTest {
) )
val leaderAndIsrAfterElection = if (isLeaderRecoverySupported) { val leaderAndIsrAfterElection = if (isLeaderRecoverySupported) {
leaderAndIsr.newRecoveringLeaderAndIsr(brokerId, List(brokerId)) leaderAndIsr.newRecoveringLeaderAndIsr(brokerId, List(brokerId).map(Int.box).asJava)
} else { } else {
leaderAndIsr.newLeaderAndIsr(brokerId, List(brokerId)) leaderAndIsr.newLeaderAndIsr(brokerId, List(brokerId).map(Int.box).asJava)
} }
val updatedLeaderAndIsr = leaderAndIsrAfterElection.withPartitionEpoch(2) val updatedLeaderAndIsr = leaderAndIsrAfterElection.withPartitionEpoch(2)
when(mockZkClient.updateLeaderAndIsr(Map(partition -> leaderAndIsrAfterElection), controllerEpoch, controllerContext.epochZkVersion)) when(mockZkClient.updateLeaderAndIsr(Map(partition -> leaderAndIsrAfterElection), controllerEpoch, controllerContext.epochZkVersion))
@ -368,7 +369,7 @@ class PartitionStateMachineTest {
controllerContext.setLiveBrokers(Map(createBrokerAndEpoch(brokerId, "host", 0))) controllerContext.setLiveBrokers(Map(createBrokerAndEpoch(brokerId, "host", 0)))
controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId)))
controllerContext.putPartitionState(partition, OfflinePartition) controllerContext.putPartitionState(partition, OfflinePartition)
val leaderAndIsr = LeaderAndIsr(LeaderAndIsr.NoLeader, List(brokerId)) val leaderAndIsr = new LeaderAndIsr(LeaderAndIsr.NO_LEADER, List(brokerId).map(Int.box).asJava)
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)
controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch)
@ -391,7 +392,7 @@ class PartitionStateMachineTest {
controllerContext.setLiveBrokers(Map(createBrokerAndEpoch(brokerId, "host", 0))) controllerContext.setLiveBrokers(Map(createBrokerAndEpoch(brokerId, "host", 0)))
controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId)))
controllerContext.putPartitionState(partition, OfflinePartition) controllerContext.putPartitionState(partition, OfflinePartition)
val leaderAndIsr = LeaderAndIsr(LeaderAndIsr.NoLeader, List(brokerId)) val leaderAndIsr = new LeaderAndIsr(LeaderAndIsr.NO_LEADER, List(brokerId).map(Int.box).asJava)
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)
controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch)
@ -426,7 +427,7 @@ class PartitionStateMachineTest {
} }
private def prepareMockToElectLeaderForPartitions(partitions: Seq[TopicPartition]): Unit = { private def prepareMockToElectLeaderForPartitions(partitions: Seq[TopicPartition]): Unit = {
val leaderAndIsr = LeaderAndIsr(brokerId, List(brokerId)) val leaderAndIsr = new LeaderAndIsr(brokerId, List(brokerId).map(Int.box).asJava)
def prepareMockToGetTopicPartitionsStatesRaw(): Unit = { def prepareMockToGetTopicPartitionsStatesRaw(): Unit = {
val stat = new Stat(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0) val stat = new Stat(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0)
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)
@ -443,7 +444,7 @@ class PartitionStateMachineTest {
def prepareMockToUpdateLeaderAndIsr(): Unit = { def prepareMockToUpdateLeaderAndIsr(): Unit = {
val updatedLeaderAndIsr: Map[TopicPartition, LeaderAndIsr] = partitions.map { partition => val updatedLeaderAndIsr: Map[TopicPartition, LeaderAndIsr] = partitions.map { partition =>
partition -> leaderAndIsr.newLeaderAndIsr(brokerId, List(brokerId)) partition -> leaderAndIsr.newLeaderAndIsr(brokerId, List(brokerId).map(Int.box).asJava)
}.toMap }.toMap
when(mockZkClient.updateLeaderAndIsr(updatedLeaderAndIsr, controllerEpoch, controllerContext.epochZkVersion)) when(mockZkClient.updateLeaderAndIsr(updatedLeaderAndIsr, controllerEpoch, controllerContext.epochZkVersion))
.thenReturn(UpdateLeaderAndIsrResult(updatedLeaderAndIsr.map { case (k, v) => k -> Right(v) }, Seq.empty)) .thenReturn(UpdateLeaderAndIsrResult(updatedLeaderAndIsr.map { case (k, v) => k -> Right(v) }, Seq.empty))

View File

@ -16,7 +16,6 @@
*/ */
package kafka.controller package kafka.controller
import kafka.api.LeaderAndIsr
import kafka.cluster.{Broker, EndPoint} import kafka.cluster.{Broker, EndPoint}
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import kafka.utils.TestUtils import kafka.utils.TestUtils
@ -26,6 +25,7 @@ import kafka.zookeeper.{GetDataResponse, ResponseMetadata}
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.zookeeper.KeeperException.Code import org.apache.zookeeper.KeeperException.Code
import org.apache.zookeeper.data.Stat import org.apache.zookeeper.data.Stat
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
@ -34,6 +34,8 @@ import org.mockito.ArgumentMatchers
import org.mockito.ArgumentMatchers.{any, anyInt} import org.mockito.ArgumentMatchers.{any, anyInt}
import org.mockito.Mockito.{mock, verify, when} import org.mockito.Mockito.{mock, verify, when}
import scala.jdk.CollectionConverters._
class ReplicaStateMachineTest { class ReplicaStateMachineTest {
private var controllerContext: ControllerContext = _ private var controllerContext: ControllerContext = _
private var mockZkClient: KafkaZkClient = _ private var mockZkClient: KafkaZkClient = _
@ -190,7 +192,7 @@ class ReplicaStateMachineTest {
def testOnlineReplicaToOnlineReplicaTransition(): Unit = { def testOnlineReplicaToOnlineReplicaTransition(): Unit = {
controllerContext.putReplicaState(replica, OnlineReplica) controllerContext.putReplicaState(replica, OnlineReplica)
controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId)))
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(LeaderAndIsr(brokerId, List(brokerId)), controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(new LeaderAndIsr(brokerId, List(brokerId).map(Int.box).asJava), controllerEpoch)
controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch)
replicaStateMachine.handleStateChanges(replicas, OnlineReplica) replicaStateMachine.handleStateChanges(replicas, OnlineReplica)
@ -207,12 +209,12 @@ class ReplicaStateMachineTest {
val replicaIds = List(brokerId, otherBrokerId) val replicaIds = List(brokerId, otherBrokerId)
controllerContext.putReplicaState(replica, OnlineReplica) controllerContext.putReplicaState(replica, OnlineReplica)
controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(replicaIds)) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(replicaIds))
val leaderAndIsr = LeaderAndIsr(brokerId, replicaIds) val leaderAndIsr = new LeaderAndIsr(brokerId, replicaIds.map(Int.box).asJava)
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)
controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch)
val stat = new Stat(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0) val stat = new Stat(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0)
val adjustedLeaderAndIsr = leaderAndIsr.newLeaderAndIsr(LeaderAndIsr.NoLeader, List(otherBrokerId)) val adjustedLeaderAndIsr = leaderAndIsr.newLeaderAndIsr(LeaderAndIsr.NO_LEADER, List(otherBrokerId).map(Int.box).asJava)
val updatedLeaderAndIsr = adjustedLeaderAndIsr.withPartitionEpoch(adjustedLeaderAndIsr.partitionEpoch + 1) val updatedLeaderAndIsr = adjustedLeaderAndIsr.withPartitionEpoch(adjustedLeaderAndIsr.partitionEpoch + 1)
val updatedLeaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(updatedLeaderAndIsr, controllerEpoch) val updatedLeaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(updatedLeaderAndIsr, controllerEpoch)
when(mockZkClient.getTopicPartitionStatesRaw(partitions)).thenReturn( when(mockZkClient.getTopicPartitionStatesRaw(partitions)).thenReturn(
@ -262,7 +264,7 @@ class ReplicaStateMachineTest {
def testOfflineReplicaToOnlineReplicaTransition(): Unit = { def testOfflineReplicaToOnlineReplicaTransition(): Unit = {
controllerContext.putReplicaState(replica, OfflineReplica) controllerContext.putReplicaState(replica, OfflineReplica)
controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId)))
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(LeaderAndIsr(brokerId, List(brokerId)), controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(new LeaderAndIsr(brokerId, List(brokerId).map(Int.box).asJava), controllerEpoch)
controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch)
replicaStateMachine.handleStateChanges(replicas, OnlineReplica) replicaStateMachine.handleStateChanges(replicas, OnlineReplica)
verify(mockControllerBrokerRequestBatch).newBatch() verify(mockControllerBrokerRequestBatch).newBatch()
@ -376,7 +378,7 @@ class ReplicaStateMachineTest {
def testReplicaDeletionIneligibleToOnlineReplicaTransition(): Unit = { def testReplicaDeletionIneligibleToOnlineReplicaTransition(): Unit = {
controllerContext.putReplicaState(replica, ReplicaDeletionIneligible) controllerContext.putReplicaState(replica, ReplicaDeletionIneligible)
controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId))) controllerContext.updatePartitionFullReplicaAssignment(partition, ReplicaAssignment(Seq(brokerId)))
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(LeaderAndIsr(brokerId, List(brokerId)), controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(new LeaderAndIsr(brokerId, List(brokerId).map(Int.box).asJava), controllerEpoch)
controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch) controllerContext.putPartitionLeadershipInfo(partition, leaderIsrAndControllerEpoch)
replicaStateMachine.handleStateChanges(replicas, OnlineReplica) replicaStateMachine.handleStateChanges(replicas, OnlineReplica)
verify(mockControllerBrokerRequestBatch).newBatch() verify(mockControllerBrokerRequestBatch).newBatch()

View File

@ -19,7 +19,6 @@ package kafka.server
import java.util.Collections import java.util.Collections
import java.util.stream.{Stream => JStream} import java.util.stream.{Stream => JStream}
import kafka.api.LeaderAndIsr
import kafka.zk.KafkaZkClient import kafka.zk.KafkaZkClient
import org.apache.kafka.clients.ClientResponse import org.apache.kafka.clients.ClientResponse
import org.apache.kafka.common.TopicIdPartition import org.apache.kafka.common.TopicIdPartition
@ -32,7 +31,7 @@ import org.apache.kafka.common.protocol.MessageUtil
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.RequestHeader import org.apache.kafka.common.requests.RequestHeader
import org.apache.kafka.common.requests.{AbstractRequest, AlterPartitionRequest, AlterPartitionResponse} import org.apache.kafka.common.requests.{AbstractRequest, AlterPartitionRequest, AlterPartitionResponse}
import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState}
import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager}
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.common.MetadataVersion.{IBP_2_7_IV2, IBP_3_2_IV0, IBP_3_5_IV1} import org.apache.kafka.server.common.MetadataVersion.{IBP_2_7_IV2, IBP_3_2_IV0, IBP_3_5_IV1}
@ -78,7 +77,7 @@ class AlterPartitionManagerTest {
val scheduler = new MockScheduler(time) val scheduler = new MockScheduler(time)
val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion)
alterPartitionManager.start() alterPartitionManager.start()
alterPartitionManager.submit(tp0, LeaderAndIsr(1, 1, List(1, 2, 3), LeaderRecoveryState.RECOVERED, 10), 0) alterPartitionManager.submit(tp0, new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10), 0)
verify(brokerToController).start() verify(brokerToController).start()
verify(brokerToController).sendRequest(any(), any()) verify(brokerToController).sendRequest(any(), any())
} }
@ -93,7 +92,7 @@ class AlterPartitionManagerTest {
for (ii <- 1 to 3) { for (ii <- 1 to 3) {
isrWithBrokerEpoch += new BrokerState().setBrokerId(ii).setBrokerEpoch(100 + ii) isrWithBrokerEpoch += new BrokerState().setBrokerId(ii).setBrokerEpoch(100 + ii)
} }
alterPartitionManager.submit(tp0, LeaderAndIsr(1, 1, LeaderRecoveryState.RECOVERED, isrWithBrokerEpoch.toList, 10), 0) alterPartitionManager.submit(tp0, new LeaderAndIsr(1, 1, LeaderRecoveryState.RECOVERED, isrWithBrokerEpoch.toList.asJava, 10), 0)
val expectedAlterPartitionData = new AlterPartitionRequestData() val expectedAlterPartitionData = new AlterPartitionRequestData()
.setBrokerId(brokerId) .setBrokerId(brokerId)
@ -139,7 +138,7 @@ class AlterPartitionManagerTest {
val scheduler = new MockScheduler(time) val scheduler = new MockScheduler(time)
val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion) val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => metadataVersion)
alterPartitionManager.start() alterPartitionManager.start()
alterPartitionManager.submit(tp0, LeaderAndIsr(1, 1, List(1), leaderRecoveryState, 10), 0) alterPartitionManager.submit(tp0, new LeaderAndIsr(1, 1, List(1).map(Int.box).asJava, leaderRecoveryState, 10), 0)
verify(brokerToController).start() verify(brokerToController).start()
verify(brokerToController).sendRequest(requestCapture.capture(), any()) verify(brokerToController).sendRequest(requestCapture.capture(), any())
@ -160,10 +159,10 @@ class AlterPartitionManagerTest {
alterPartitionManager.start() alterPartitionManager.start()
// Only send one ISR update for a given topic+partition // Only send one ISR update for a given topic+partition
val firstSubmitFuture = alterPartitionManager.submit(tp0, LeaderAndIsr(1, 1, List(1, 2, 3), LeaderRecoveryState.RECOVERED, 10), 0) val firstSubmitFuture = alterPartitionManager.submit(tp0, new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10), 0)
assertFalse(firstSubmitFuture.isDone) assertFalse(firstSubmitFuture.isDone)
val failedSubmitFuture = alterPartitionManager.submit(tp0, LeaderAndIsr(1, 1, List(1, 2), LeaderRecoveryState.RECOVERED, 10), 0) val failedSubmitFuture = alterPartitionManager.submit(tp0, new LeaderAndIsr(1, 1, List(1, 2).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10), 0)
assertTrue(failedSubmitFuture.isCompletedExceptionally) assertTrue(failedSubmitFuture.isCompletedExceptionally)
assertFutureThrows(failedSubmitFuture, classOf[OperationNotAttemptedException]) assertFutureThrows(failedSubmitFuture, classOf[OperationNotAttemptedException])
@ -177,7 +176,7 @@ class AlterPartitionManagerTest {
callbackCapture.getValue.onComplete(resp) callbackCapture.getValue.onComplete(resp)
// Now we can submit this partition again // Now we can submit this partition again
val newSubmitFuture = alterPartitionManager.submit(tp0, LeaderAndIsr(1, 1, List(1), LeaderRecoveryState.RECOVERED, 10), 0) val newSubmitFuture = alterPartitionManager.submit(tp0, new LeaderAndIsr(1, 1, List(1).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10), 0)
assertFalse(newSubmitFuture.isDone) assertFalse(newSubmitFuture.isDone)
verify(brokerToController).start() verify(brokerToController).start()
@ -205,12 +204,12 @@ class AlterPartitionManagerTest {
// First request will send batch of one // First request will send batch of one
alterPartitionManager.submit(new TopicIdPartition(topicId, 0, topic), alterPartitionManager.submit(new TopicIdPartition(topicId, 0, topic),
LeaderAndIsr(1, 1, List(1, 2, 3), LeaderRecoveryState.RECOVERED, 10), 0) new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10), 0)
// Other submissions will queue up until a response // Other submissions will queue up until a response
for (i <- 1 to 9) { for (i <- 1 to 9) {
alterPartitionManager.submit(new TopicIdPartition(topicId, i, topic), alterPartitionManager.submit(new TopicIdPartition(topicId, i, topic),
LeaderAndIsr(1, 1, List(1, 2, 3), LeaderRecoveryState.RECOVERED, 10), 0) new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10), 0)
} }
// Simulate response, omitting partition 0 will allow it to stay in unsent queue // Simulate response, omitting partition 0 will allow it to stay in unsent queue
@ -241,7 +240,7 @@ class AlterPartitionManagerTest {
val leaderEpoch = 1 val leaderEpoch = 1
val partitionEpoch = 10 val partitionEpoch = 10
val isr = List(1, 2, 3) val isr = List(1, 2, 3)
val leaderAndIsr = LeaderAndIsr(leaderId, leaderEpoch, isr, LeaderRecoveryState.RECOVERED, partitionEpoch) val leaderAndIsr = new LeaderAndIsr(leaderId, leaderEpoch, isr.map(Int.box).asJava, LeaderRecoveryState.RECOVERED, partitionEpoch)
val callbackCapture = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val callbackCapture = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler])
val scheduler = new MockScheduler(time) val scheduler = new MockScheduler(time)
@ -315,7 +314,7 @@ class AlterPartitionManagerTest {
} }
private def testRetryOnErrorResponse(response: ClientResponse): Unit = { private def testRetryOnErrorResponse(response: ClientResponse): Unit = {
val leaderAndIsr = LeaderAndIsr(1, 1, List(1, 2, 3), LeaderRecoveryState.RECOVERED, 10) val leaderAndIsr = new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10)
val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler]) val callbackCapture: ArgumentCaptor[ControllerRequestCompletionHandler] = ArgumentCaptor.forClass(classOf[ControllerRequestCompletionHandler])
val scheduler = new MockScheduler(time) val scheduler = new MockScheduler(time)
@ -369,7 +368,7 @@ class AlterPartitionManagerTest {
private def checkPartitionError(error: Errors): Unit = { private def checkPartitionError(error: Errors): Unit = {
val alterPartitionManager = testPartitionError(tp0, error) val alterPartitionManager = testPartitionError(tp0, error)
// Any partition-level error should clear the item from the pending queue allowing for future updates // Any partition-level error should clear the item from the pending queue allowing for future updates
val future = alterPartitionManager.submit(tp0, LeaderAndIsr(1, 1, List(1, 2, 3), LeaderRecoveryState.RECOVERED, 10), 0) val future = alterPartitionManager.submit(tp0, new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10), 0)
assertFalse(future.isDone) assertFalse(future.isDone)
} }
@ -381,7 +380,7 @@ class AlterPartitionManagerTest {
val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => IBP_3_2_IV0) val alterPartitionManager = new DefaultAlterPartitionManager(brokerToController, scheduler, time, brokerId, () => 2, () => IBP_3_2_IV0)
alterPartitionManager.start() alterPartitionManager.start()
val future = alterPartitionManager.submit(tp, LeaderAndIsr(1, 1, List(1, 2, 3), LeaderRecoveryState.RECOVERED, 10), 0) val future = alterPartitionManager.submit(tp, new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10), 0)
verify(brokerToController).start() verify(brokerToController).start()
verify(brokerToController).sendRequest(any(), callbackCapture.capture()) verify(brokerToController).sendRequest(any(), callbackCapture.capture())
@ -405,11 +404,11 @@ class AlterPartitionManagerTest {
alterPartitionManager.start() alterPartitionManager.start()
// First submit will send the request // First submit will send the request
alterPartitionManager.submit(tp0, LeaderAndIsr(1, 1, List(1, 2, 3), LeaderRecoveryState.RECOVERED, 10), 0) alterPartitionManager.submit(tp0, new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10), 0)
// These will become pending unsent items // These will become pending unsent items
alterPartitionManager.submit(tp1, LeaderAndIsr(1, 1, List(1, 2, 3), LeaderRecoveryState.RECOVERED, 10), 0) alterPartitionManager.submit(tp1, new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10), 0)
alterPartitionManager.submit(tp2, LeaderAndIsr(1, 1, List(1, 2, 3), LeaderRecoveryState.RECOVERED, 10), 0) alterPartitionManager.submit(tp2, new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10), 0)
verify(brokerToController).start() verify(brokerToController).start()
verify(brokerToController).sendRequest(any(), callbackCapture.capture()) verify(brokerToController).sendRequest(any(), callbackCapture.capture())
@ -430,7 +429,7 @@ class AlterPartitionManagerTest {
} else { } else {
1.toShort 1.toShort
} }
val leaderAndIsr = LeaderAndIsr(1, 1, List(1, 2, 3), LeaderRecoveryState.RECOVERED, 10) val leaderAndIsr = new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10)
val controlledEpoch = 0 val controlledEpoch = 0
val brokerEpoch = 2 val brokerEpoch = 2
val scheduler = new MockScheduler(time) val scheduler = new MockScheduler(time)
@ -497,7 +496,7 @@ class AlterPartitionManagerTest {
val bar = new TopicIdPartition(Uuid.randomUuid(), 0, "bar") val bar = new TopicIdPartition(Uuid.randomUuid(), 0, "bar")
val zar = new TopicIdPartition(Uuid.randomUuid(), 0, "zar") val zar = new TopicIdPartition(Uuid.randomUuid(), 0, "zar")
val leaderAndIsr = LeaderAndIsr(1, 1, List(1, 2, 3), LeaderRecoveryState.RECOVERED, 10) val leaderAndIsr = new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 10)
val controlledEpoch = 0 val controlledEpoch = 0
val brokerEpoch = 2 val brokerEpoch = 2
val scheduler = new MockScheduler(time) val scheduler = new MockScheduler(time)
@ -648,12 +647,12 @@ class AlterPartitionManagerTest {
zkIsrManager.start() zkIsrManager.start()
// Correct ZK version // Correct ZK version
val future1 = zkIsrManager.submit(tp0, LeaderAndIsr(1, 1, List(1, 2, 3), LeaderRecoveryState.RECOVERED, 1), 0) val future1 = zkIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 1), 0)
assertTrue(future1.isDone) assertTrue(future1.isDone)
assertEquals(LeaderAndIsr(1, 1, List(1, 2, 3), LeaderRecoveryState.RECOVERED, 2), future1.get) assertEquals(new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 2), future1.get)
// Wrong ZK version // Wrong ZK version
val future2 = zkIsrManager.submit(tp0, LeaderAndIsr(1, 1, List(1, 2, 3), LeaderRecoveryState.RECOVERED, 3), 0) val future2 = zkIsrManager.submit(tp0, new LeaderAndIsr(1, 1, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 3), 0)
assertTrue(future2.isCompletedExceptionally) assertTrue(future2.isCompletedExceptionally)
assertFutureThrows(future2, classOf[InvalidUpdateVersionException]) assertFutureThrows(future2, classOf[InvalidUpdateVersionException])
} }

View File

@ -18,13 +18,10 @@
package kafka.server package kafka.server
import java.util.Collections import java.util.Collections
import kafka.api.LeaderAndIsr
import kafka.cluster.Broker import kafka.cluster.Broker
import kafka.controller.{ControllerChannelManager, ControllerContext, StateChangeLogger} import kafka.controller.{ControllerChannelManager, ControllerContext, StateChangeLogger}
import kafka.utils.TestUtils import kafka.utils.TestUtils
import kafka.utils.TestUtils.createTopic import kafka.utils.TestUtils.createTopic
import kafka.server.QuorumTestHarness
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.message.StopReplicaRequestData.{StopReplicaPartitionState, StopReplicaTopicState} import org.apache.kafka.common.message.StopReplicaRequestData.{StopReplicaPartitionState, StopReplicaTopicState}
@ -35,6 +32,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests._
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.ReplicationConfigs
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
@ -153,9 +151,9 @@ class BrokerEpochIntegrationTest extends QuorumTestHarness {
.setPartitionIndex(tp.partition) .setPartitionIndex(tp.partition)
.setControllerEpoch(controllerEpoch) .setControllerEpoch(controllerEpoch)
.setLeader(brokerId2) .setLeader(brokerId2)
.setLeaderEpoch(LeaderAndIsr.InitialLeaderEpoch + 1) .setLeaderEpoch(LeaderAndIsr.INITIAL_LEADER_EPOCH + 1)
.setIsr(Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava) .setIsr(Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava)
.setPartitionEpoch(LeaderAndIsr.InitialPartitionEpoch) .setPartitionEpoch(LeaderAndIsr.INITIAL_PARTITION_EPOCH)
.setReplicas(Seq(0, 1).map(Integer.valueOf).asJava) .setReplicas(Seq(0, 1).map(Integer.valueOf).asJava)
.setIsNew(false) .setIsNew(false)
) )
@ -183,9 +181,9 @@ class BrokerEpochIntegrationTest extends QuorumTestHarness {
.setPartitionIndex(tp.partition) .setPartitionIndex(tp.partition)
.setControllerEpoch(controllerEpoch) .setControllerEpoch(controllerEpoch)
.setLeader(brokerId2) .setLeader(brokerId2)
.setLeaderEpoch(LeaderAndIsr.InitialLeaderEpoch + 1) .setLeaderEpoch(LeaderAndIsr.INITIAL_LEADER_EPOCH + 1)
.setIsr(Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava) .setIsr(Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava)
.setZkVersion(LeaderAndIsr.InitialPartitionEpoch) .setZkVersion(LeaderAndIsr.INITIAL_PARTITION_EPOCH)
.setReplicas(Seq(0, 1).map(Integer.valueOf).asJava)) .setReplicas(Seq(0, 1).map(Integer.valueOf).asJava))
val liveBrokers = brokerAndEpochs.map { case (broker, _) => val liveBrokers = brokerAndEpochs.map { case (broker, _) =>
val securityProtocol = SecurityProtocol.PLAINTEXT val securityProtocol = SecurityProtocol.PLAINTEXT
@ -226,7 +224,7 @@ class BrokerEpochIntegrationTest extends QuorumTestHarness {
.setTopicName(tp.topic()) .setTopicName(tp.topic())
.setPartitionStates(Seq(new StopReplicaPartitionState() .setPartitionStates(Seq(new StopReplicaPartitionState()
.setPartitionIndex(tp.partition()) .setPartitionIndex(tp.partition())
.setLeaderEpoch(LeaderAndIsr.InitialLeaderEpoch + 2) .setLeaderEpoch(LeaderAndIsr.INITIAL_LEADER_EPOCH + 2)
.setDeletePartition(true)).asJava) .setDeletePartition(true)).asJava)
).asJava ).asJava
val requestBuilder = new StopReplicaRequest.Builder( val requestBuilder = new StopReplicaRequest.Builder(

View File

@ -17,7 +17,6 @@
package kafka.server package kafka.server
import kafka.api.LeaderAndIsr
import kafka.cluster.{Broker, Partition} import kafka.cluster.{Broker, Partition}
import kafka.controller.{ControllerContext, KafkaController} import kafka.controller.{ControllerContext, KafkaController}
import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator} import kafka.coordinator.transaction.{InitProducerIdResult, TransactionCoordinator}
@ -78,6 +77,7 @@ import org.apache.kafka.common.utils.{ImplicitLinkedHashCollection, ProducerIdAn
import org.apache.kafka.coordinator.group.GroupConfig.{CONSUMER_HEARTBEAT_INTERVAL_MS_CONFIG, CONSUMER_SESSION_TIMEOUT_MS_CONFIG} import org.apache.kafka.coordinator.group.GroupConfig.{CONSUMER_HEARTBEAT_INTERVAL_MS_CONFIG, CONSUMER_SESSION_TIMEOUT_MS_CONFIG}
import org.apache.kafka.coordinator.group.{GroupCoordinator, GroupCoordinatorConfig} import org.apache.kafka.coordinator.group.{GroupCoordinator, GroupCoordinatorConfig}
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.security.authorizer.AclEntry
import org.apache.kafka.server.ClientMetricsManager import org.apache.kafka.server.ClientMetricsManager
@ -3061,25 +3061,25 @@ class KafkaApisTest extends Logging {
@Test @Test
def shouldResignCoordinatorsIfStopReplicaReceivedWithDeleteFlagAndLeaderEpoch(): Unit = { def shouldResignCoordinatorsIfStopReplicaReceivedWithDeleteFlagAndLeaderEpoch(): Unit = {
shouldResignCoordinatorsIfStopReplicaReceivedWithDeleteFlag( shouldResignCoordinatorsIfStopReplicaReceivedWithDeleteFlag(
LeaderAndIsr.InitialLeaderEpoch + 2, deletePartition = true) LeaderAndIsr.INITIAL_LEADER_EPOCH + 2, deletePartition = true)
} }
@Test @Test
def shouldResignCoordinatorsIfStopReplicaReceivedWithDeleteFlagAndDeleteSentinel(): Unit = { def shouldResignCoordinatorsIfStopReplicaReceivedWithDeleteFlagAndDeleteSentinel(): Unit = {
shouldResignCoordinatorsIfStopReplicaReceivedWithDeleteFlag( shouldResignCoordinatorsIfStopReplicaReceivedWithDeleteFlag(
LeaderAndIsr.EpochDuringDelete, deletePartition = true) LeaderAndIsr.EPOCH_DURING_DELETE, deletePartition = true)
} }
@Test @Test
def shouldResignCoordinatorsIfStopReplicaReceivedWithDeleteFlagAndNoEpochSentinel(): Unit = { def shouldResignCoordinatorsIfStopReplicaReceivedWithDeleteFlagAndNoEpochSentinel(): Unit = {
shouldResignCoordinatorsIfStopReplicaReceivedWithDeleteFlag( shouldResignCoordinatorsIfStopReplicaReceivedWithDeleteFlag(
LeaderAndIsr.NoEpoch, deletePartition = true) LeaderAndIsr.NO_EPOCH, deletePartition = true)
} }
@Test @Test
def shouldNotResignCoordinatorsIfStopReplicaReceivedWithoutDeleteFlag(): Unit = { def shouldNotResignCoordinatorsIfStopReplicaReceivedWithoutDeleteFlag(): Unit = {
shouldResignCoordinatorsIfStopReplicaReceivedWithDeleteFlag( shouldResignCoordinatorsIfStopReplicaReceivedWithDeleteFlag(
LeaderAndIsr.InitialLeaderEpoch + 2, deletePartition = false) LeaderAndIsr.INITIAL_LEADER_EPOCH + 2, deletePartition = false)
} }
def shouldResignCoordinatorsIfStopReplicaReceivedWithDeleteFlag(leaderEpoch: Int, def shouldResignCoordinatorsIfStopReplicaReceivedWithDeleteFlag(leaderEpoch: Int,

View File

@ -18,24 +18,22 @@
package kafka.server package kafka.server
import java.util.Collections import java.util.Collections
import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.{TopicPartition, Uuid}
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import kafka.api.LeaderAndIsr
import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests._
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import kafka.utils.TestUtils import kafka.utils.TestUtils
import kafka.cluster.Broker import kafka.cluster.Broker
import kafka.controller.{ControllerChannelManager, ControllerContext, StateChangeLogger} import kafka.controller.{ControllerChannelManager, ControllerContext, StateChangeLogger}
import kafka.utils.TestUtils._ import kafka.utils.TestUtils._
import kafka.server.QuorumTestHarness
import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.metadata.LeaderAndIsr
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
class LeaderElectionTest extends QuorumTestHarness { class LeaderElectionTest extends QuorumTestHarness {
@ -155,9 +153,9 @@ class LeaderElectionTest extends QuorumTestHarness {
.setPartitionIndex(partitionId) .setPartitionIndex(partitionId)
.setControllerEpoch(2) .setControllerEpoch(2)
.setLeader(brokerId2) .setLeader(brokerId2)
.setLeaderEpoch(LeaderAndIsr.InitialLeaderEpoch) .setLeaderEpoch(LeaderAndIsr.INITIAL_LEADER_EPOCH)
.setIsr(Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava) .setIsr(Seq(brokerId1, brokerId2).map(Integer.valueOf).asJava)
.setPartitionEpoch(LeaderAndIsr.InitialPartitionEpoch) .setPartitionEpoch(LeaderAndIsr.INITIAL_PARTITION_EPOCH)
.setReplicas(Seq(0, 1).map(Integer.valueOf).asJava) .setReplicas(Seq(0, 1).map(Integer.valueOf).asJava)
.setIsNew(false) .setIsNew(false)
) )

View File

@ -16,7 +16,6 @@
*/ */
package kafka.server package kafka.server
import kafka.api.LeaderAndIsr
import kafka.cluster.Broker import kafka.cluster.Broker
import kafka.server.metadata.{KRaftMetadataCache, MetadataSnapshot, ZkMetadataCache} import kafka.server.metadata.{KRaftMetadataCache, MetadataSnapshot, ZkMetadataCache}
import org.apache.kafka.common.message.DescribeTopicPartitionsResponseData.DescribeTopicPartitionsResponsePartition import org.apache.kafka.common.message.DescribeTopicPartitionsResponseData.DescribeTopicPartitionsResponsePartition
@ -31,7 +30,7 @@ import org.apache.kafka.common.requests.{AbstractControlRequest, UpdateMetadataR
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.{DirectoryId, Node, TopicPartition, Uuid} import org.apache.kafka.common.{DirectoryId, Node, TopicPartition, Uuid}
import org.apache.kafka.image.{ClusterImage, MetadataDelta, MetadataImage, MetadataProvenance} import org.apache.kafka.image.{ClusterImage, MetadataDelta, MetadataImage, MetadataProvenance}
import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState}
import org.apache.kafka.server.common.{KRaftVersion, MetadataVersion} import org.apache.kafka.server.common.{KRaftVersion, MetadataVersion}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
@ -109,7 +108,7 @@ object MetadataCacheTest {
val results = new mutable.ArrayBuffer[ApiMessage]() val results = new mutable.ArrayBuffer[ApiMessage]()
results += new TopicRecord().setName(topic.topicName()).setTopicId(topic.topicId()) results += new TopicRecord().setName(topic.topicName()).setTopicId(topic.topicId())
topic.partitionStates().forEach { partition => topic.partitionStates().forEach { partition =>
if (partition.leader() == LeaderAndIsr.LeaderDuringDelete) { if (partition.leader() == LeaderAndIsr.LEADER_DURING_DELETE) {
results += new RemoveTopicRecord().setTopicId(topic.topicId()) results += new RemoveTopicRecord().setTopicId(topic.topicId())
} else { } else {
results += new PartitionRecord(). results += new PartitionRecord().

View File

@ -20,7 +20,6 @@ import java.net.InetAddress
import java.util import java.util
import java.util.concurrent.{CompletableFuture, Executors, LinkedBlockingQueue, TimeUnit} import java.util.concurrent.{CompletableFuture, Executors, LinkedBlockingQueue, TimeUnit}
import java.util.{Optional, Properties} import java.util.{Optional, Properties}
import kafka.api.LeaderAndIsr
import kafka.server.QuotaFactory.QuotaManagers import kafka.server.QuotaFactory.QuotaManagers
import kafka.server.metadata.KRaftMetadataCache import kafka.server.metadata.KRaftMetadataCache
import kafka.server.metadata.MockConfigRepository import kafka.server.metadata.MockConfigRepository
@ -37,7 +36,7 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.common.{DirectoryId, IsolationLevel, TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.{DirectoryId, IsolationLevel, TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.image.{MetadataDelta, MetadataImage} import org.apache.kafka.image.{MetadataDelta, MetadataImage}
import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState}
import org.apache.kafka.metadata.PartitionRegistration import org.apache.kafka.metadata.PartitionRegistration
import org.apache.kafka.metadata.storage.Formatter import org.apache.kafka.metadata.storage.Formatter
import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.raft.QuorumConfig
@ -439,7 +438,7 @@ class ReplicaManagerConcurrencyTest extends Logging {
delta.replay(new PartitionChangeRecord() delta.replay(new PartitionChangeRecord()
.setTopicId(topic.topicId) .setTopicId(topic.topicId)
.setPartitionId(partitionId) .setPartitionId(partitionId)
.setIsr(leaderAndIsr.isr.map(Int.box).asJava) .setIsr(leaderAndIsr.isr)
.setLeader(leaderAndIsr.leader) .setLeader(leaderAndIsr.leader)
) )
this.registration = delta.topicsDelta this.registration = delta.topicsDelta

View File

@ -18,7 +18,6 @@
package kafka.server package kafka.server
import com.yammer.metrics.core.{Gauge, Meter, Timer} import com.yammer.metrics.core.{Gauge, Meter, Timer}
import kafka.api._
import kafka.cluster.PartitionTest.MockPartitionListener import kafka.cluster.PartitionTest.MockPartitionListener
import kafka.cluster.{BrokerEndPoint, Partition} import kafka.cluster.{BrokerEndPoint, Partition}
import kafka.log._ import kafka.log._
@ -55,7 +54,7 @@ import org.apache.kafka.common.utils.{Exit, LogContext, Time, Utils}
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.image._ import org.apache.kafka.image._
import org.apache.kafka.metadata.LeaderConstants.NO_LEADER import org.apache.kafka.metadata.LeaderConstants.NO_LEADER
import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState}
import org.apache.kafka.metadata.migration.ZkMigrationState import org.apache.kafka.metadata.migration.ZkMigrationState
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
@ -2197,7 +2196,7 @@ class ReplicaManagerTest {
mutable.Map(tp0 -> new StopReplicaPartitionState() mutable.Map(tp0 -> new StopReplicaPartitionState()
.setPartitionIndex(tp0.partition) .setPartitionIndex(tp0.partition)
.setDeletePartition(true) .setDeletePartition(true)
.setLeaderEpoch(LeaderAndIsr.EpochDuringDelete))) .setLeaderEpoch(LeaderAndIsr.EPOCH_DURING_DELETE)))
assertEquals(Errors.NOT_LEADER_OR_FOLLOWER, fetchResult.assertFired.error) assertEquals(Errors.NOT_LEADER_OR_FOLLOWER, fetchResult.assertFired.error)
} finally { } finally {
@ -2240,7 +2239,7 @@ class ReplicaManagerTest {
mutable.Map(tp0 -> new StopReplicaPartitionState() mutable.Map(tp0 -> new StopReplicaPartitionState()
.setPartitionIndex(tp0.partition) .setPartitionIndex(tp0.partition)
.setDeletePartition(true) .setDeletePartition(true)
.setLeaderEpoch(LeaderAndIsr.EpochDuringDelete))) .setLeaderEpoch(LeaderAndIsr.EPOCH_DURING_DELETE)))
assertNotNull(produceResult.get) assertNotNull(produceResult.get)
assertEquals(Errors.NOT_LEADER_OR_FOLLOWER, produceResult.get.error) assertEquals(Errors.NOT_LEADER_OR_FOLLOWER, produceResult.get.error)
@ -2261,11 +2260,11 @@ class ReplicaManagerTest {
val replicaManager = setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager, List(tp0, tp1)) val replicaManager = setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager, List(tp0, tp1))
try { try {
replicaManager.becomeLeaderOrFollower(1, replicaManager.becomeLeaderOrFollower(1,
makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), LeaderAndIsr(1, List(0, 1))), makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), new LeaderAndIsr(1, List(0, 1).map(Int.box).asJava)),
(_, _) => ()) (_, _) => ())
replicaManager.becomeLeaderOrFollower(1, replicaManager.becomeLeaderOrFollower(1,
makeLeaderAndIsrRequest(topicIds(tp1.topic), tp1, Seq(0, 1), LeaderAndIsr(1, List(0, 1))), makeLeaderAndIsrRequest(topicIds(tp1.topic), tp1, Seq(0, 1), new LeaderAndIsr(1, List(0, 1).map(Int.box).asJava)),
(_, _) => ()) (_, _) => ())
// If we supply no transactional ID and idempotent records, we do not verify. // If we supply no transactional ID and idempotent records, we do not verify.
@ -2309,7 +2308,7 @@ class ReplicaManagerTest {
val replicaManager = setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager, List(tp0)) val replicaManager = setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager, List(tp0))
try { try {
replicaManager.becomeLeaderOrFollower(1, replicaManager.becomeLeaderOrFollower(1,
makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), LeaderAndIsr(1, List(0, 1))), makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), new LeaderAndIsr(1, List(0, 1).map(Int.box).asJava)),
(_, _) => ()) (_, _) => ())
// Append some transactional records. // Append some transactional records.
@ -2369,7 +2368,7 @@ class ReplicaManagerTest {
val replicaManager = setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager, List(tp0)) val replicaManager = setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager, List(tp0))
try { try {
replicaManager.becomeLeaderOrFollower(1, replicaManager.becomeLeaderOrFollower(1,
makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), LeaderAndIsr(1, List(0, 1))), makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), new LeaderAndIsr(1, List(0, 1).map(Int.box).asJava)),
(_, _) => ()) (_, _) => ())
// Start with sequence 6 // Start with sequence 6
@ -2434,11 +2433,11 @@ class ReplicaManagerTest {
val replicaManager = setupReplicaManagerWithMockedPurgatories(mockTimer) val replicaManager = setupReplicaManagerWithMockedPurgatories(mockTimer)
try { try {
replicaManager.becomeLeaderOrFollower(1, replicaManager.becomeLeaderOrFollower(1,
makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), LeaderAndIsr(0, List(0, 1))), makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), new LeaderAndIsr(0, List(0, 1).map(Int.box).asJava)),
(_, _) => ()) (_, _) => ())
replicaManager.becomeLeaderOrFollower(1, replicaManager.becomeLeaderOrFollower(1,
makeLeaderAndIsrRequest(topicIds(tp1.topic), tp1, Seq(0, 1), LeaderAndIsr(0, List(0, 1))), makeLeaderAndIsrRequest(topicIds(tp1.topic), tp1, Seq(0, 1), new LeaderAndIsr(0, List(0, 1).map(Int.box).asJava)),
(_, _) => ()) (_, _) => ())
val transactionalRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence, val transactionalRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence,
@ -2469,11 +2468,11 @@ class ReplicaManagerTest {
try { try {
replicaManager.becomeLeaderOrFollower(1, replicaManager.becomeLeaderOrFollower(1,
makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), LeaderAndIsr(1, List(0, 1))), makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), new LeaderAndIsr(1, List(0, 1).map(Int.box).asJava)),
(_, _) => ()) (_, _) => ())
replicaManager.becomeLeaderOrFollower(1, replicaManager.becomeLeaderOrFollower(1,
makeLeaderAndIsrRequest(topicIds(tp1.topic), tp1, Seq(0, 1), LeaderAndIsr(1, List(0, 1))), makeLeaderAndIsrRequest(topicIds(tp1.topic), tp1, Seq(0, 1), new LeaderAndIsr(1, List(0, 1).map(Int.box).asJava)),
(_, _) => ()) (_, _) => ())
// Append some transactional records with different producer IDs // Append some transactional records with different producer IDs
@ -2533,7 +2532,7 @@ class ReplicaManagerTest {
val replicaManager = setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager, List(tp), config = config) val replicaManager = setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager, List(tp), config = config)
try { try {
val becomeLeaderRequest = makeLeaderAndIsrRequest(topicIds(tp.topic), tp, Seq(0, 1), LeaderAndIsr(0, List(0, 1))) val becomeLeaderRequest = makeLeaderAndIsrRequest(topicIds(tp.topic), tp, Seq(0, 1), new LeaderAndIsr(0, List(0, 1).map(Int.box).asJava))
replicaManager.becomeLeaderOrFollower(1, becomeLeaderRequest, (_, _) => ()) replicaManager.becomeLeaderOrFollower(1, becomeLeaderRequest, (_, _) => ())
val transactionalRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence, val transactionalRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence,
@ -2577,7 +2576,7 @@ class ReplicaManagerTest {
val replicaManager = setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager, List(tp0)) val replicaManager = setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager, List(tp0))
try { try {
replicaManager.becomeLeaderOrFollower(1, replicaManager.becomeLeaderOrFollower(1,
makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), LeaderAndIsr(1, List(0, 1))), makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), new LeaderAndIsr(1, List(0, 1).map(Int.box).asJava)),
(_, _) => ()) (_, _) => ())
// Append some transactional records. // Append some transactional records.
@ -2642,7 +2641,7 @@ class ReplicaManagerTest {
val replicaManager = setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager, List(tp0)) val replicaManager = setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager, List(tp0))
try { try {
replicaManager.becomeLeaderOrFollower(1, replicaManager.becomeLeaderOrFollower(1,
makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), LeaderAndIsr(1, List(0, 1))), makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), new LeaderAndIsr(1, List(0, 1).map(Int.box).asJava)),
(_, _) => ()) (_, _) => ())
val transactionalRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence, val transactionalRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence,
@ -3936,13 +3935,13 @@ class ReplicaManagerTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(booleans = Array(true, false)) @ValueSource(booleans = Array(true, false))
def testStopReplicaWithExistingPartitionAndDeleteSentinel(enableRemoteStorage: Boolean): Unit = { def testStopReplicaWithExistingPartitionAndDeleteSentinel(enableRemoteStorage: Boolean): Unit = {
testStopReplicaWithExistingPartition(LeaderAndIsr.EpochDuringDelete, false, false, Errors.NONE, enableRemoteStorage) testStopReplicaWithExistingPartition(LeaderAndIsr.EPOCH_DURING_DELETE, false, false, Errors.NONE, enableRemoteStorage)
} }
@ParameterizedTest @ParameterizedTest
@ValueSource(booleans = Array(true, false)) @ValueSource(booleans = Array(true, false))
def testStopReplicaWithExistingPartitionAndLeaderEpochNotProvided(enableRemoteStorage: Boolean): Unit = { def testStopReplicaWithExistingPartitionAndLeaderEpochNotProvided(enableRemoteStorage: Boolean): Unit = {
testStopReplicaWithExistingPartition(LeaderAndIsr.NoEpoch, false, false, Errors.NONE, enableRemoteStorage) testStopReplicaWithExistingPartition(LeaderAndIsr.NO_EPOCH, false, false, Errors.NONE, enableRemoteStorage)
} }
@ParameterizedTest @ParameterizedTest
@ -3972,13 +3971,13 @@ class ReplicaManagerTest {
@ParameterizedTest @ParameterizedTest
@ValueSource(booleans = Array(true, false)) @ValueSource(booleans = Array(true, false))
def testStopReplicaWithDeletePartitionAndExistingPartitionAndDeleteSentinel(enableRemoteStorage: Boolean): Unit = { def testStopReplicaWithDeletePartitionAndExistingPartitionAndDeleteSentinel(enableRemoteStorage: Boolean): Unit = {
testStopReplicaWithExistingPartition(LeaderAndIsr.EpochDuringDelete, true, false, Errors.NONE, enableRemoteStorage) testStopReplicaWithExistingPartition(LeaderAndIsr.EPOCH_DURING_DELETE, true, false, Errors.NONE, enableRemoteStorage)
} }
@ParameterizedTest @ParameterizedTest
@ValueSource(booleans = Array(true, false)) @ValueSource(booleans = Array(true, false))
def testStopReplicaWithDeletePartitionAndExistingPartitionAndLeaderEpochNotProvided(enableRemoteStorage: Boolean): Unit = { def testStopReplicaWithDeletePartitionAndExistingPartitionAndLeaderEpochNotProvided(enableRemoteStorage: Boolean): Unit = {
testStopReplicaWithExistingPartition(LeaderAndIsr.NoEpoch, true, false, Errors.NONE, enableRemoteStorage) testStopReplicaWithExistingPartition(LeaderAndIsr.NO_EPOCH, true, false, Errors.NONE, enableRemoteStorage)
} }
@ParameterizedTest @ParameterizedTest
@ -4592,7 +4591,7 @@ class ReplicaManagerTest {
if (enableRemoteStorage) { if (enableRemoteStorage) {
val stopPartition = StopPartition(tp0, val stopPartition = StopPartition(tp0,
deleteLocalLog = deletePartition, deleteLocalLog = deletePartition,
deleteRemoteLog = leaderEpoch == LeaderAndIsr.EpochDuringDelete) deleteRemoteLog = leaderEpoch == LeaderAndIsr.EPOCH_DURING_DELETE)
verify(mockRemoteLogManager) verify(mockRemoteLogManager)
.stopPartitions(ArgumentMatchers.eq(Collections.singleton(stopPartition)), any()) .stopPartitions(ArgumentMatchers.eq(Collections.singleton(stopPartition)), any())
} }
@ -4909,7 +4908,7 @@ class ReplicaManagerTest {
topicId = Uuid.randomUuid(), topicId = Uuid.randomUuid(),
topicPartition = topicPartition, topicPartition = topicPartition,
replicas = Seq(0, 1), replicas = Seq(0, 1),
leaderAndIsr = LeaderAndIsr(if (becomeLeader) 0 else 1, List(0, 1)) leaderAndIsr = new LeaderAndIsr(if (becomeLeader) 0 else 1, List(0, 1).map(Int.box).asJava)
) )
replicaManager.becomeLeaderOrFollower(0, request, (_, _) => ()) replicaManager.becomeLeaderOrFollower(0, request, (_, _) => ())
@ -4944,7 +4943,7 @@ class ReplicaManagerTest {
.setControllerEpoch(controllerEpoch) .setControllerEpoch(controllerEpoch)
.setLeader(leaderAndIsr.leader) .setLeader(leaderAndIsr.leader)
.setLeaderEpoch(leaderAndIsr.leaderEpoch) .setLeaderEpoch(leaderAndIsr.leaderEpoch)
.setIsr(leaderAndIsr.isr.map(Int.box).asJava) .setIsr(leaderAndIsr.isr)
.setPartitionEpoch(leaderAndIsr.partitionEpoch) .setPartitionEpoch(leaderAndIsr.partitionEpoch)
.setReplicas(replicas.map(Int.box).asJava) .setReplicas(replicas.map(Int.box).asJava)
.setIsNew(isNew) .setIsNew(isNew)
@ -4988,7 +4987,7 @@ class ReplicaManagerTest {
topicId = Uuid.randomUuid(), topicId = Uuid.randomUuid(),
topicPartition = barPartition, topicPartition = barPartition,
replicas = Seq(brokerId), replicas = Seq(brokerId),
leaderAndIsr = LeaderAndIsr(brokerId, List(brokerId)) leaderAndIsr = new LeaderAndIsr(brokerId, List(brokerId).map(Int.box).asJava)
) )
replicaManager.becomeLeaderOrFollower(0, barLeaderAndIsrRequest, (_, _) => ()) replicaManager.becomeLeaderOrFollower(0, barLeaderAndIsrRequest, (_, _) => ())
val barProducerState = replicaManager.activeProducerState(barPartition) val barProducerState = replicaManager.activeProducerState(barPartition)
@ -5000,7 +4999,7 @@ class ReplicaManagerTest {
topicId = Uuid.randomUuid(), topicId = Uuid.randomUuid(),
topicPartition = bazPartition, topicPartition = bazPartition,
replicas = Seq(brokerId, otherBrokerId), replicas = Seq(brokerId, otherBrokerId),
leaderAndIsr = LeaderAndIsr(otherBrokerId, List(brokerId, otherBrokerId)) leaderAndIsr = new LeaderAndIsr(otherBrokerId, List(brokerId, otherBrokerId).map(Int.box).asJava)
) )
replicaManager.becomeLeaderOrFollower(0, bazLeaderAndIsrRequest, (_, _) => ()) replicaManager.becomeLeaderOrFollower(0, bazLeaderAndIsrRequest, (_, _) => ())
val bazProducerState = replicaManager.activeProducerState(bazPartition) val bazProducerState = replicaManager.activeProducerState(bazPartition)
@ -5857,12 +5856,12 @@ class ReplicaManagerTest {
topicId = FOO_UUID, topicId = FOO_UUID,
topicPartition = topicPartition, topicPartition = topicPartition,
replicas = Seq(localId, localId + 1), replicas = Seq(localId, localId + 1),
leaderAndIsr = LeaderAndIsr( leaderAndIsr = new LeaderAndIsr(
leader = localId + 1, localId + 1,
leaderEpoch = 0, 0,
isr = List(localId, localId + 1), List(localId, localId + 1).map(Int.box).asJava,
leaderRecoveryState = LeaderRecoveryState.RECOVERED, LeaderRecoveryState.RECOVERED,
partitionEpoch = 0 0
) )
) )
@ -5890,12 +5889,12 @@ class ReplicaManagerTest {
topicId = FOO_UUID, topicId = FOO_UUID,
topicPartition = topicPartition, topicPartition = topicPartition,
replicas = Seq(localId, localId + 1, localId + 2), replicas = Seq(localId, localId + 1, localId + 2),
leaderAndIsr = LeaderAndIsr( leaderAndIsr = new LeaderAndIsr(
leader = localId + 1, localId + 1,
leaderEpoch = 0, 0,
isr = List(localId, localId + 1), List(localId, localId + 1).map(Int.box).asJava,
leaderRecoveryState = LeaderRecoveryState.RECOVERED, LeaderRecoveryState.RECOVERED,
partitionEpoch = 1 1
) )
) )
@ -5915,12 +5914,12 @@ class ReplicaManagerTest {
topicId = FOO_UUID, topicId = FOO_UUID,
topicPartition = topicPartition, topicPartition = topicPartition,
replicas = Seq(localId, localId + 1, localId + 2), replicas = Seq(localId, localId + 1, localId + 2),
leaderAndIsr = LeaderAndIsr( leaderAndIsr = new LeaderAndIsr(
leader = localId + 2, localId + 2,
leaderEpoch = 1, 1,
isr = List(localId, localId + 1, localId + 2), List(localId, localId + 1, localId + 2).map(Int.box).asJava,
leaderRecoveryState = LeaderRecoveryState.RECOVERED, LeaderRecoveryState.RECOVERED,
partitionEpoch = 2 2
) )
) )
@ -6379,7 +6378,7 @@ class ReplicaManagerTest {
brokerId = 0, aliveBrokersIds) brokerId = 0, aliveBrokersIds)
try { try {
val tp = new TopicPartition(topic, 0) val tp = new TopicPartition(topic, 0)
val leaderAndIsr = LeaderAndIsr(1, aliveBrokersIds.toList) val leaderAndIsr = new LeaderAndIsr(1, aliveBrokersIds.toList.map(Int.box).asJava)
// This test either starts with a topic ID in the PartitionFetchState and removes it on the next request (startsWithTopicId) // This test either starts with a topic ID in the PartitionFetchState and removes it on the next request (startsWithTopicId)
// or does not start with a topic ID in the PartitionFetchState and adds one on the next request (!startsWithTopicId) // or does not start with a topic ID in the PartitionFetchState and adds one on the next request (!startsWithTopicId)
@ -6429,7 +6428,7 @@ class ReplicaManagerTest {
topicId = topicId, topicId = topicId,
topicPartition = tp, topicPartition = tp,
replicas = Seq(0, 1), replicas = Seq(0, 1),
leaderAndIsr = LeaderAndIsr(0, List(0, 1)), leaderAndIsr = new LeaderAndIsr(0, List(0, 1).map(Int.box).asJava),
version = version version = version
) )
replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ()) replicaManager.becomeLeaderOrFollower(0, leaderAndIsrRequest, (_, _) => ())
@ -6550,7 +6549,7 @@ class ReplicaManagerTest {
// The unified log created is not tiered because `defaultTopicRemoteLogStorageEnable` is set to false // The unified log created is not tiered because `defaultTopicRemoteLogStorageEnable` is set to false
partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None) partition.createLogIfNotExists(isNew = false, isFutureReplica = false, offsetCheckpoints, None)
val leaderAndIsr = LeaderAndIsr(0, 1, List(0, 1), LeaderRecoveryState.RECOVERED, LeaderAndIsr.InitialPartitionEpoch) val leaderAndIsr = new LeaderAndIsr(0, 1, List(0, 1).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, LeaderAndIsr.INITIAL_PARTITION_EPOCH)
val becomeLeaderRequest = makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), leaderAndIsr) val becomeLeaderRequest = makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), leaderAndIsr)
replicaManager.becomeLeaderOrFollower(1, becomeLeaderRequest, (_, _) => ()) replicaManager.becomeLeaderOrFollower(1, becomeLeaderRequest, (_, _) => ())

View File

@ -14,7 +14,6 @@
package kafka.server package kafka.server
import kafka.api.LeaderAndIsr
import kafka.security.authorizer.AclAuthorizer import kafka.security.authorizer.AclAuthorizer
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.common._ import org.apache.kafka.common._
@ -43,10 +42,11 @@ import org.apache.kafka.common.resource.{PatternType, ResourceType => AdminResou
import org.apache.kafka.common.security.auth._ import org.apache.kafka.common.security.auth._
import org.apache.kafka.common.utils.{Sanitizer, SecurityUtils} import org.apache.kafka.common.utils.{Sanitizer, SecurityUtils}
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.metadata.authorizer.StandardAuthorizer import org.apache.kafka.metadata.authorizer.StandardAuthorizer
import org.apache.kafka.network.Session import org.apache.kafka.network.Session
import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult} import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult}
import org.apache.kafka.server.config.{ServerConfigs, QuotaConfigs} import org.apache.kafka.server.config.{QuotaConfigs, ServerConfigs}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -312,7 +312,7 @@ class RequestQuotaTest extends BaseRequestTest {
.setTopicName(tp.topic()) .setTopicName(tp.topic())
.setPartitionStates(Seq(new StopReplicaPartitionState() .setPartitionStates(Seq(new StopReplicaPartitionState()
.setPartitionIndex(tp.partition()) .setPartitionIndex(tp.partition())
.setLeaderEpoch(LeaderAndIsr.InitialLeaderEpoch + 2) .setLeaderEpoch(LeaderAndIsr.INITIAL_LEADER_EPOCH + 2)
.setDeletePartition(true)).asJava) .setDeletePartition(true)).asJava)
).asJava ).asJava
new StopReplicaRequest.Builder(ApiKeys.STOP_REPLICA.latestVersion, brokerId, new StopReplicaRequest.Builder(ApiKeys.STOP_REPLICA.latestVersion, brokerId,

View File

@ -17,13 +17,13 @@
package kafka.server package kafka.server
import kafka.api.LeaderAndIsr
import kafka.utils._ import kafka.utils._
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.message.StopReplicaRequestData.{StopReplicaPartitionState, StopReplicaTopicState} import org.apache.kafka.common.message.StopReplicaRequestData.{StopReplicaPartitionState, StopReplicaTopicState}
import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.protocol.ApiKeys
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests._
import org.apache.kafka.metadata.LeaderAndIsr
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
@ -53,13 +53,13 @@ class StopReplicaRequestTest extends BaseRequestTest {
.setTopicName(tp0.topic()) .setTopicName(tp0.topic())
.setPartitionStates(Seq(new StopReplicaPartitionState() .setPartitionStates(Seq(new StopReplicaPartitionState()
.setPartitionIndex(tp0.partition()) .setPartitionIndex(tp0.partition())
.setLeaderEpoch(LeaderAndIsr.InitialLeaderEpoch + 2) .setLeaderEpoch(LeaderAndIsr.INITIAL_LEADER_EPOCH + 2)
.setDeletePartition(true)).asJava), .setDeletePartition(true)).asJava),
new StopReplicaTopicState() new StopReplicaTopicState()
.setTopicName(tp1.topic()) .setTopicName(tp1.topic())
.setPartitionStates(Seq(new StopReplicaPartitionState() .setPartitionStates(Seq(new StopReplicaPartitionState()
.setPartitionIndex(tp1.partition()) .setPartitionIndex(tp1.partition())
.setLeaderEpoch(LeaderAndIsr.InitialLeaderEpoch + 2) .setLeaderEpoch(LeaderAndIsr.INITIAL_LEADER_EPOCH + 2)
.setDeletePartition(true)).asJava) .setDeletePartition(true)).asJava)
).asJava ).asJava

View File

@ -17,15 +17,16 @@
package kafka.utils package kafka.utils
import kafka.api.LeaderAndIsr
import kafka.controller.LeaderIsrAndControllerEpoch import kafka.controller.LeaderIsrAndControllerEpoch
import kafka.server.QuorumTestHarness import kafka.server.QuorumTestHarness
import kafka.zk._ import kafka.zk._
import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.TopicPartition
import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
import java.util
class ReplicationUtilsTest extends QuorumTestHarness { class ReplicationUtilsTest extends QuorumTestHarness {
private val zkVersion = 1 private val zkVersion = 1
private val topic = "my-topic-test" private val topic = "my-topic-test"
@ -33,14 +34,14 @@ class ReplicationUtilsTest extends QuorumTestHarness {
private val leader = 1 private val leader = 1
private val leaderEpoch = 1 private val leaderEpoch = 1
private val controllerEpoch = 1 private val controllerEpoch = 1
private val isr = List(1, 2) private val isr: util.List[Integer] = util.Arrays.asList(1, 2)
@BeforeEach @BeforeEach
override def setUp(testInfo: TestInfo): Unit = { override def setUp(testInfo: TestInfo): Unit = {
super.setUp(testInfo) super.setUp(testInfo)
zkClient.makeSurePersistentPathExists(TopicZNode.path(topic)) zkClient.makeSurePersistentPathExists(TopicZNode.path(topic))
val topicPartition = new TopicPartition(topic, partition) val topicPartition = new TopicPartition(topic, partition)
val leaderAndIsr = LeaderAndIsr(leader, leaderEpoch, isr, LeaderRecoveryState.RECOVERED, 1) val leaderAndIsr = new LeaderAndIsr(leader, leaderEpoch, isr, LeaderRecoveryState.RECOVERED, 1)
val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch) val leaderIsrAndControllerEpoch = LeaderIsrAndControllerEpoch(leaderAndIsr, controllerEpoch)
zkClient.createTopicPartitionStatesRaw(Map(topicPartition -> leaderIsrAndControllerEpoch), ZkVersion.MatchAnyVersion) zkClient.createTopicPartitionStatesRaw(Map(topicPartition -> leaderIsrAndControllerEpoch), ZkVersion.MatchAnyVersion)
} }
@ -49,17 +50,17 @@ class ReplicationUtilsTest extends QuorumTestHarness {
def testUpdateLeaderAndIsr(): Unit = { def testUpdateLeaderAndIsr(): Unit = {
zkClient.makeSurePersistentPathExists(IsrChangeNotificationZNode.path) zkClient.makeSurePersistentPathExists(IsrChangeNotificationZNode.path)
val replicas = List(0, 1) val replicas: util.List[Integer] = util.Arrays.asList(0, 1)
// regular update // regular update
val newLeaderAndIsr1 = LeaderAndIsr(leader, leaderEpoch, replicas, LeaderRecoveryState.RECOVERED, 0) val newLeaderAndIsr1 = new LeaderAndIsr(leader, leaderEpoch, replicas, LeaderRecoveryState.RECOVERED, 0)
val (updateSucceeded1, newZkVersion1) = ReplicationUtils.updateLeaderAndIsr(zkClient, val (updateSucceeded1, newZkVersion1) = ReplicationUtils.updateLeaderAndIsr(zkClient,
new TopicPartition(topic, partition), newLeaderAndIsr1, controllerEpoch) new TopicPartition(topic, partition), newLeaderAndIsr1, controllerEpoch)
assertTrue(updateSucceeded1) assertTrue(updateSucceeded1)
assertEquals(newZkVersion1, 1) assertEquals(newZkVersion1, 1)
// mismatched zkVersion with the same data // mismatched zkVersion with the same data
val newLeaderAndIsr2 = LeaderAndIsr(leader, leaderEpoch, replicas, LeaderRecoveryState.RECOVERED, zkVersion + 1) val newLeaderAndIsr2 = new LeaderAndIsr(leader, leaderEpoch, replicas, LeaderRecoveryState.RECOVERED, zkVersion + 1)
val (updateSucceeded2, newZkVersion2) = ReplicationUtils.updateLeaderAndIsr(zkClient, val (updateSucceeded2, newZkVersion2) = ReplicationUtils.updateLeaderAndIsr(zkClient,
new TopicPartition(topic, partition), newLeaderAndIsr2, controllerEpoch) new TopicPartition(topic, partition), newLeaderAndIsr2, controllerEpoch)
assertTrue(updateSucceeded2) assertTrue(updateSucceeded2)
@ -67,7 +68,7 @@ class ReplicationUtilsTest extends QuorumTestHarness {
assertEquals(newZkVersion2, 1) assertEquals(newZkVersion2, 1)
// mismatched zkVersion and leaderEpoch // mismatched zkVersion and leaderEpoch
val newLeaderAndIsr3 = LeaderAndIsr(leader, leaderEpoch + 1, replicas, LeaderRecoveryState.RECOVERED, zkVersion + 1) val newLeaderAndIsr3 = new LeaderAndIsr(leader, leaderEpoch + 1, replicas, LeaderRecoveryState.RECOVERED, zkVersion + 1)
val (updateSucceeded3, newZkVersion3) = ReplicationUtils.updateLeaderAndIsr(zkClient, val (updateSucceeded3, newZkVersion3) = ReplicationUtils.updateLeaderAndIsr(zkClient,
new TopicPartition(topic, partition), newLeaderAndIsr3, controllerEpoch) new TopicPartition(topic, partition), newLeaderAndIsr3, controllerEpoch)
assertFalse(updateSucceeded3) assertFalse(updateSucceeded3)

View File

@ -17,7 +17,6 @@
package kafka.utils package kafka.utils
import com.yammer.metrics.core.{Histogram, Meter} import com.yammer.metrics.core.{Histogram, Meter}
import kafka.api._
import kafka.controller.ControllerEventManager import kafka.controller.ControllerEventManager
import kafka.log._ import kafka.log._
import kafka.network.RequestChannel import kafka.network.RequestChannel
@ -53,6 +52,7 @@ import org.apache.kafka.common.utils.Utils.formatAddress
import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.utils.{Time, Utils}
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.queue.KafkaEventQueue import org.apache.kafka.queue.KafkaEventQueue
import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.raft.QuorumConfig
@ -778,7 +778,7 @@ object TestUtils extends Logging {
): Int = { ): Int = {
def getPartitionLeader(topic: String, partition: Int): Option[Int] = { def getPartitionLeader(topic: String, partition: Int): Option[Int] = {
zkClient.getLeaderForPartition(new TopicPartition(topic, partition)) zkClient.getLeaderForPartition(new TopicPartition(topic, partition))
.filter(p => !ignoreNoLeader || p != LeaderAndIsr.NoLeader) .filter(p => !ignoreNoLeader || p != LeaderAndIsr.NO_LEADER)
} }
doWaitUntilLeaderIsElectedOrChanged(getPartitionLeader, topic, partition, timeoutMs, oldLeaderOpt, newLeaderOpt) doWaitUntilLeaderIsElectedOrChanged(getPartitionLeader, topic, partition, timeoutMs, oldLeaderOpt, newLeaderOpt)
} }

View File

@ -16,7 +16,6 @@
*/ */
package kafka.admin package kafka.admin
import kafka.api.LeaderAndIsr
import kafka.cluster.{Broker, EndPoint} import kafka.cluster.{Broker, EndPoint}
import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment} import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment}
import kafka.server.KafkaConfig._ import kafka.server.KafkaConfig._
@ -32,6 +31,7 @@ import org.apache.kafka.common.errors.{InvalidReplicaAssignmentException, Invali
import org.apache.kafka.common.metrics.Quota import org.apache.kafka.common.metrics.Quota
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.metadata.LeaderAndIsr
import org.apache.kafka.server.common.{AdminOperationException, MetadataVersion} import org.apache.kafka.server.common.{AdminOperationException, MetadataVersion}
import org.apache.kafka.server.config.{ConfigType, QuotaConfigs} import org.apache.kafka.server.config.{ConfigType, QuotaConfigs}
import org.apache.kafka.storage.internals.log.LogConfig import org.apache.kafka.storage.internals.log.LogConfig
@ -450,7 +450,7 @@ class AdminZkClientTest extends QuorumTestHarness with Logging with RackAwareTes
val topicPartition = new TopicPartition(topic, partition) val topicPartition = new TopicPartition(topic, partition)
val newLeaderAndIsr = zkClient.getTopicPartitionState(topicPartition) val newLeaderAndIsr = zkClient.getTopicPartitionState(topicPartition)
.map(_.leaderAndIsr.newLeader(leader)) .map(_.leaderAndIsr.newLeader(leader))
.getOrElse(LeaderAndIsr(leader, List(leader))) .getOrElse(new LeaderAndIsr(leader, List(leader).map(Integer.valueOf).asJava))
topicPartition -> LeaderIsrAndControllerEpoch(newLeaderAndIsr, 1) topicPartition -> LeaderIsrAndControllerEpoch(newLeaderAndIsr, 1)
} }
zkClient.setTopicPartitionStatesRaw(newLeaderIsrAndControllerEpochs, ZkVersion.MatchAnyVersion) zkClient.setTopicPartitionStatesRaw(newLeaderIsrAndControllerEpochs, ZkVersion.MatchAnyVersion)

View File

@ -19,7 +19,6 @@ package kafka.zk
import java.nio.charset.StandardCharsets.UTF_8 import java.nio.charset.StandardCharsets.UTF_8
import java.util.concurrent.{CountDownLatch, Executors, TimeUnit} import java.util.concurrent.{CountDownLatch, Executors, TimeUnit}
import java.util.{Collections, Properties} import java.util.{Collections, Properties}
import kafka.api.LeaderAndIsr
import kafka.cluster.{Broker, EndPoint} import kafka.cluster.{Broker, EndPoint}
import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment} import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment}
import kafka.server.{KafkaConfig, QuorumTestHarness} import kafka.server.{KafkaConfig, QuorumTestHarness}
@ -41,7 +40,7 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.common.security.token.delegation.TokenInformation import org.apache.kafka.common.security.token.delegation.TokenInformation
import org.apache.kafka.common.utils.{SecurityUtils, Time} import org.apache.kafka.common.utils.{SecurityUtils, Time}
import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.{TopicPartition, Uuid}
import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState}
import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState import org.apache.kafka.metadata.migration.ZkMigrationLeadershipState
import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.security.authorizer.AclEntry
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
@ -928,10 +927,10 @@ class KafkaZkClientTest extends QuorumTestHarness {
private def leaderIsrAndControllerEpochs(state: Int, partitionEpoch: Int): Map[TopicPartition, LeaderIsrAndControllerEpoch] = private def leaderIsrAndControllerEpochs(state: Int, partitionEpoch: Int): Map[TopicPartition, LeaderIsrAndControllerEpoch] =
Map( Map(
topicPartition10 -> LeaderIsrAndControllerEpoch( topicPartition10 -> LeaderIsrAndControllerEpoch(
LeaderAndIsr(leader = 1, leaderEpoch = state, isr = List(2 + state, 3 + state), LeaderRecoveryState.RECOVERED, partitionEpoch = partitionEpoch), new LeaderAndIsr(1, state, List(2 + state, 3 + state).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, partitionEpoch),
controllerEpoch = 4), controllerEpoch = 4),
topicPartition11 -> LeaderIsrAndControllerEpoch( topicPartition11 -> LeaderIsrAndControllerEpoch(
LeaderAndIsr(leader = 0, leaderEpoch = state + 1, isr = List(1 + state, 2 + state), LeaderRecoveryState.RECOVERED, partitionEpoch = partitionEpoch), new LeaderAndIsr(0, state + 1, List(1 + state, 2 + state).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, partitionEpoch),
controllerEpoch = 4)) controllerEpoch = 4))
val initialLeaderIsrAndControllerEpochs: Map[TopicPartition, LeaderIsrAndControllerEpoch] = val initialLeaderIsrAndControllerEpochs: Map[TopicPartition, LeaderIsrAndControllerEpoch] =
@ -1020,9 +1019,9 @@ class KafkaZkClientTest extends QuorumTestHarness {
// Trigger successful, to be retried and failed partitions in same call // Trigger successful, to be retried and failed partitions in same call
val mixedState = Map( val mixedState = Map(
topicPartition10 -> LeaderAndIsr(leader = 1, leaderEpoch = 2, isr = List(4, 5), LeaderRecoveryState.RECOVERED, partitionEpoch = 1), topicPartition10 -> new LeaderAndIsr(1, 2, List(4, 5).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 1),
topicPartition11 -> LeaderAndIsr(leader = 0, leaderEpoch = 2, isr = List(3, 4), LeaderRecoveryState.RECOVERED, partitionEpoch = 0), topicPartition11 -> new LeaderAndIsr(0, 2, List(3, 4).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 0),
topicPartition20 -> LeaderAndIsr(leader = 0, leaderEpoch = 2, isr = List(3, 4), LeaderRecoveryState.RECOVERED, partitionEpoch = 0)) topicPartition20 -> new LeaderAndIsr(0, 2, List(3, 4).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, 0))
checkUpdateLeaderAndIsrResult( checkUpdateLeaderAndIsrResult(
leaderIsrs(state = 2, partitionEpoch = 2).filter { case (tp, _) => tp == topicPartition10 }, leaderIsrs(state = 2, partitionEpoch = 2).filter { case (tp, _) => tp == topicPartition10 },

View File

@ -16,7 +16,6 @@
*/ */
package kafka.zk.migration package kafka.zk.migration
import kafka.api.LeaderAndIsr
import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment} import kafka.controller.{LeaderIsrAndControllerEpoch, ReplicaAssignment}
import kafka.coordinator.transaction.{ProducerIdManager, ZkProducerIdManager} import kafka.coordinator.transaction.{ProducerIdManager, ZkProducerIdManager}
import org.apache.kafka.common.config.{ConfigResource, SslConfigs, TopicConfig} import org.apache.kafka.common.config.{ConfigResource, SslConfigs, TopicConfig}
@ -25,7 +24,7 @@ import org.apache.kafka.common.metadata.{ConfigRecord, MetadataRecordType, Parti
import org.apache.kafka.common.{DirectoryId, TopicPartition, Uuid} import org.apache.kafka.common.{DirectoryId, TopicPartition, Uuid}
import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance} import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance}
import org.apache.kafka.metadata.migration.{KRaftMigrationZkWriter, ZkMigrationLeadershipState} import org.apache.kafka.metadata.migration.{KRaftMigrationZkWriter, ZkMigrationLeadershipState}
import org.apache.kafka.metadata.{LeaderRecoveryState, PartitionRegistration} import org.apache.kafka.metadata.{LeaderAndIsr, LeaderRecoveryState, PartitionRegistration}
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.ReplicationConfigs
import org.apache.kafka.server.common.ApiMessageAndVersion import org.apache.kafka.server.common.ApiMessageAndVersion
import org.apache.kafka.server.config.ConfigType import org.apache.kafka.server.config.ConfigType
@ -70,9 +69,9 @@ class ZkMigrationClientTest extends ZkMigrationTestHarness {
val leaderAndIsrs = Map( val leaderAndIsrs = Map(
new TopicPartition("test", 0) -> LeaderIsrAndControllerEpoch( new TopicPartition("test", 0) -> LeaderIsrAndControllerEpoch(
LeaderAndIsr(0, 5, List(0, 1, 2), LeaderRecoveryState.RECOVERED, -1), 1), new LeaderAndIsr(0, 5, List(0, 1, 2).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, -1), 1),
new TopicPartition("test", 1) -> LeaderIsrAndControllerEpoch( new TopicPartition("test", 1) -> LeaderIsrAndControllerEpoch(
LeaderAndIsr(1, 5, List(1, 2, 3), LeaderRecoveryState.RECOVERED, -1), 1) new LeaderAndIsr(1, 5, List(1, 2, 3).map(Int.box).asJava, LeaderRecoveryState.RECOVERED, -1), 1)
) )
zkClient.createTopicPartitionStatesRaw(leaderAndIsrs, 0) zkClient.createTopicPartitionStatesRaw(leaderAndIsrs, 0)
@ -106,12 +105,12 @@ class ZkMigrationClientTest extends ZkMigrationTestHarness {
val partition0 = zkClient.getTopicPartitionState(new TopicPartition("test", 0)).get.leaderAndIsr val partition0 = zkClient.getTopicPartitionState(new TopicPartition("test", 0)).get.leaderAndIsr
assertEquals(1, partition0.leader) assertEquals(1, partition0.leader)
assertEquals(6, partition0.leaderEpoch) assertEquals(6, partition0.leaderEpoch)
assertEquals(List(1, 2), partition0.isr) assertEquals(List(1, 2).map(Int.box).asJava, partition0.isr)
val partition1 = zkClient.getTopicPartitionState(new TopicPartition("test", 1)).get.leaderAndIsr val partition1 = zkClient.getTopicPartitionState(new TopicPartition("test", 1)).get.leaderAndIsr
assertEquals(3, partition1.leader) assertEquals(3, partition1.leader)
assertEquals(7, partition1.leaderEpoch) assertEquals(7, partition1.leaderEpoch)
assertEquals(List(3), partition1.isr) assertEquals(List(3).map(Int.box).asJava, partition1.isr)
// Delete whole topic // Delete whole topic
migrationState = migrationClient.topicClient().deleteTopic("test", migrationState) migrationState = migrationClient.topicClient().deleteTopic("test", migrationState)
@ -149,12 +148,12 @@ class ZkMigrationClientTest extends ZkMigrationTestHarness {
val partition0 = zkClient.getTopicPartitionState(new TopicPartition("test", 0)).get.leaderAndIsr val partition0 = zkClient.getTopicPartitionState(new TopicPartition("test", 0)).get.leaderAndIsr
assertEquals(0, partition0.leader) assertEquals(0, partition0.leader)
assertEquals(0, partition0.leaderEpoch) assertEquals(0, partition0.leaderEpoch)
assertEquals(List(0, 1, 2), partition0.isr) assertEquals(List(0, 1, 2).map(Int.box).asJava, partition0.isr)
val partition1 = zkClient.getTopicPartitionState(new TopicPartition("test", 1)).get.leaderAndIsr val partition1 = zkClient.getTopicPartitionState(new TopicPartition("test", 1)).get.leaderAndIsr
assertEquals(1, partition1.leader) assertEquals(1, partition1.leader)
assertEquals(0, partition1.leaderEpoch) assertEquals(0, partition1.leaderEpoch)
assertEquals(List(1, 2, 3), partition1.isr) assertEquals(List(1, 2, 3).map(Int.box).asJava, partition1.isr)
} }
@Test @Test
@ -393,7 +392,7 @@ class ZkMigrationClientTest extends ZkMigrationTestHarness {
assertEquals(leaderPartition, state.leaderAndIsr.leader) assertEquals(leaderPartition, state.leaderAndIsr.leader)
assertEquals(leaderEpoch, state.leaderAndIsr.leaderEpoch) assertEquals(leaderEpoch, state.leaderAndIsr.leaderEpoch)
assertEquals(LeaderRecoveryState.RECOVERED, state.leaderAndIsr.leaderRecoveryState) assertEquals(LeaderRecoveryState.RECOVERED, state.leaderAndIsr.leaderRecoveryState)
assertEquals(replicas.asScala.map(Integer2int).toList, state.leaderAndIsr.isr) assertEquals(replicas, state.leaderAndIsr.isr)
} }
// Verify the broker and topic configs (including sensitive configs). // Verify the broker and topic configs (including sensitive configs).
@ -510,7 +509,7 @@ class ZkMigrationClientTest extends ZkMigrationTestHarness {
assertEquals(expectedPartition.partitionEpoch + 1, part.leaderAndIsr.partitionEpoch) assertEquals(expectedPartition.partitionEpoch + 1, part.leaderAndIsr.partitionEpoch)
assertEquals(expectedPartition.leaderEpoch, part.leaderAndIsr.leaderEpoch) assertEquals(expectedPartition.leaderEpoch, part.leaderAndIsr.leaderEpoch)
assertEquals(expectedPartition.leaderRecoveryState, part.leaderAndIsr.leaderRecoveryState) assertEquals(expectedPartition.leaderRecoveryState, part.leaderAndIsr.leaderRecoveryState)
assertEquals(expectedPartition.isr.toList, part.leaderAndIsr.isr) assertEquals(expectedPartition.isr.toList.map(Int.box).asJava, part.leaderAndIsr.isr)
} }
} }
} }

View File

@ -0,0 +1,174 @@
/*
* 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 org.apache.kafka.metadata;
import org.apache.kafka.common.message.AlterPartitionRequestData.BrokerState;
import java.util.List;
import java.util.Objects;
import java.util.Optional;
import java.util.stream.Collectors;
public class LeaderAndIsr {
public static final int INITIAL_LEADER_EPOCH = 0;
public static final int INITIAL_PARTITION_EPOCH = 0;
public static final int NO_LEADER = -1;
public static final int NO_EPOCH = -1;
public static final int LEADER_DURING_DELETE = -2;
public static final int EPOCH_DURING_DELETE = -2;
private final int leader;
private final int leaderEpoch;
private final LeaderRecoveryState leaderRecoveryState;
private final List<BrokerState> isrWithBrokerEpoch;
// The current epoch for the partition for KRaft controllers. The current ZK version for the
// legacy controllers. The epoch is a monotonically increasing value which is incremented
// after every partition change.
private final int partitionEpoch;
public LeaderAndIsr(int leader, List<Integer> isr) {
this(leader, INITIAL_LEADER_EPOCH, isr, LeaderRecoveryState.RECOVERED, INITIAL_PARTITION_EPOCH);
}
public LeaderAndIsr(
int leader,
int leaderEpoch,
List<Integer> isr,
LeaderRecoveryState leaderRecoveryState,
int partitionEpoch
) {
this(
leader,
leaderEpoch,
leaderRecoveryState,
isr.stream().map(brokerId -> new BrokerState().setBrokerId(brokerId)).collect(Collectors.toList()),
partitionEpoch
);
}
public LeaderAndIsr(
int leader,
int leaderEpoch,
LeaderRecoveryState leaderRecoveryState,
List<BrokerState> isrWithBrokerEpoch,
int partitionEpoch
) {
this.leader = leader;
this.leaderEpoch = leaderEpoch;
this.leaderRecoveryState = leaderRecoveryState;
this.isrWithBrokerEpoch = isrWithBrokerEpoch;
this.partitionEpoch = partitionEpoch;
}
public static LeaderAndIsr duringDelete(List<Integer> isr) {
return new LeaderAndIsr(LEADER_DURING_DELETE, isr);
}
public int leader() {
return leader;
}
public int leaderEpoch() {
return leaderEpoch;
}
public List<BrokerState> isrWithBrokerEpoch() {
return isrWithBrokerEpoch;
}
public LeaderRecoveryState leaderRecoveryState() {
return leaderRecoveryState;
}
public int partitionEpoch() {
return partitionEpoch;
}
public LeaderAndIsr withPartitionEpoch(int partitionEpoch) {
return new LeaderAndIsr(leader, leaderEpoch, leaderRecoveryState, isrWithBrokerEpoch, partitionEpoch);
}
public LeaderAndIsr newLeader(int leader) {
return newLeaderAndIsrWithBrokerEpoch(leader, isrWithBrokerEpoch);
}
public LeaderAndIsr newLeaderAndIsr(int leader, List<Integer> isr) {
return new LeaderAndIsr(leader, leaderEpoch + 1, isr, leaderRecoveryState, partitionEpoch);
}
private LeaderAndIsr newLeaderAndIsrWithBrokerEpoch(int leader, List<BrokerState> isrWithBrokerEpoch) {
return new LeaderAndIsr(leader, leaderEpoch + 1, leaderRecoveryState, isrWithBrokerEpoch, partitionEpoch);
}
public LeaderAndIsr newRecoveringLeaderAndIsr(int leader, List<Integer> isr) {
return new LeaderAndIsr(leader, leaderEpoch + 1, isr, LeaderRecoveryState.RECOVERING, partitionEpoch);
}
public LeaderAndIsr newEpoch() {
return newLeaderAndIsrWithBrokerEpoch(leader, isrWithBrokerEpoch);
}
public Optional<Integer> leaderOpt() {
return leader == LeaderAndIsr.NO_LEADER ? Optional.empty() : Optional.of(leader);
}
public List<Integer> isr() {
return isrWithBrokerEpoch.stream()
.map(BrokerState::brokerId)
.collect(Collectors.toList());
}
public boolean equalsAllowStalePartitionEpoch(LeaderAndIsr other) {
if (this == other) {
return true;
} else if (other == null) {
return false;
} else {
return leader == other.leader &&
leaderEpoch == other.leaderEpoch &&
isrWithBrokerEpoch.equals(other.isrWithBrokerEpoch) &&
leaderRecoveryState == other.leaderRecoveryState &&
partitionEpoch <= other.partitionEpoch;
}
}
@Override
public String toString() {
return "LeaderAndIsr(" +
"leader=" + leader +
", leaderEpoch=" + leaderEpoch +
", isrWithBrokerEpoch=" + isrWithBrokerEpoch +
", leaderRecoveryState=" + leaderRecoveryState +
", partitionEpoch=" + partitionEpoch +
')';
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
LeaderAndIsr that = (LeaderAndIsr) o;
return leader == that.leader && leaderEpoch == that.leaderEpoch && partitionEpoch == that.partitionEpoch &&
leaderRecoveryState == that.leaderRecoveryState && Objects.equals(isrWithBrokerEpoch, that.isrWithBrokerEpoch);
}
@Override
public int hashCode() {
return Objects.hash(leader, leaderEpoch, leaderRecoveryState, isrWithBrokerEpoch, partitionEpoch);
}
}

View File

@ -0,0 +1,78 @@
/*
* 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 org.apache.kafka.metadata;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
import java.util.Collections;
import static org.junit.jupiter.api.Assertions.assertEquals;
public final class LeaderAndIsrTest {
@Test
public void testRecoveringLeaderAndIsr() {
LeaderAndIsr leaderAndIsr = new LeaderAndIsr(1, Arrays.asList(1, 2));
LeaderAndIsr recoveringLeaderAndIsr = leaderAndIsr.newRecoveringLeaderAndIsr(3, Collections.singletonList(3));
assertEquals(3, recoveringLeaderAndIsr.leader());
assertEquals(Collections.singletonList(3), recoveringLeaderAndIsr.isr());
assertEquals(LeaderRecoveryState.RECOVERING, recoveringLeaderAndIsr.leaderRecoveryState());
}
@Test
public void testNewLeaderAndIsr() {
LeaderAndIsr leaderAndIsr = new LeaderAndIsr(1, Arrays.asList(1, 2));
LeaderAndIsr newLeaderAndIsr = leaderAndIsr.newLeaderAndIsr(2, Arrays.asList(1, 2));
assertEquals(2, newLeaderAndIsr.leader());
assertEquals(Arrays.asList(1, 2), newLeaderAndIsr.isr());
assertEquals(LeaderRecoveryState.RECOVERED, newLeaderAndIsr.leaderRecoveryState());
}
@Test
public void testNewLeader() {
LeaderAndIsr leaderAndIsr = new LeaderAndIsr(2, Arrays.asList(1, 2, 3));
assertEquals(2, leaderAndIsr.leader());
assertEquals(Arrays.asList(1, 2, 3), leaderAndIsr.isr());
LeaderAndIsr newLeaderAndIsr = leaderAndIsr.newLeader(3);
assertEquals(3, newLeaderAndIsr.leader());
assertEquals(Arrays.asList(1, 2, 3), newLeaderAndIsr.isr());
}
@Test
public void testNewEpoch() {
LeaderAndIsr leaderAndIsr = new LeaderAndIsr(3, Arrays.asList(1, 2, 3));
assertEquals(0, leaderAndIsr.leaderEpoch());
LeaderAndIsr leaderWithNewEpoch = leaderAndIsr.newEpoch();
assertEquals(1, leaderWithNewEpoch.leaderEpoch());
}
@Test
public void testLeaderOpt() {
LeaderAndIsr leaderAndIsr = new LeaderAndIsr(2, Arrays.asList(1, 2, 3));
assertEquals(2, leaderAndIsr.leaderOpt().orElse(0));
}
}