mirror of https://github.com/apache/kafka.git
KAFKA-17370 Move LeaderAndIsr to metadata module (#16943)
isrWithBrokerEpoch = addBrokerEpochToIsr(isrToSend.toL
This commit is contained in:
parent
0300025097
commit
6cad2c0d67
|
@ -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)"
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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")
|
||||||
|
|
|
@ -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.")
|
||||||
}
|
}
|
||||||
|
|
|
@ -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)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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 = {
|
||||||
|
|
|
@ -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
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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
|
||||||
)
|
)
|
||||||
|
|
|
@ -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)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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 {
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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 " +
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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))
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -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)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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")
|
||||||
}
|
}
|
||||||
|
|
|
@ -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)
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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))
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
|
@ -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.
|
||||||
|
|
|
@ -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())
|
||||||
|
|
||||||
|
|
|
@ -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}
|
||||||
|
|
||||||
|
|
|
@ -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))
|
||||||
|
|
|
@ -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()
|
||||||
|
|
|
@ -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])
|
||||||
}
|
}
|
||||||
|
|
|
@ -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(
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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)
|
||||||
)
|
)
|
||||||
|
|
|
@ -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().
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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, (_, _) => ())
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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
|
||||||
|
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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 },
|
||||||
|
|
|
@ -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)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
|
@ -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));
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue