Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

MINOR: Change AlterPartition validation order in KafkaController #12032

Merged
merged 4 commits into from
Apr 25, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
19 changes: 11 additions & 8 deletions core/src/main/scala/kafka/controller/KafkaController.scala
Original file line number Diff line number Diff line change
Expand Up @@ -2347,7 +2347,17 @@ class KafkaController(val config: KafkaConfig,
controllerContext.partitionLeadershipInfo(tp) match {
case Some(leaderIsrAndControllerEpoch) =>
val currentLeaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr
if (newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.length > 1) {
if (newLeaderAndIsr.leaderEpoch != currentLeaderAndIsr.leaderEpoch) {
partitionResponses(tp) = Left(Errors.FENCED_LEADER_EPOCH)
None
} else if (newLeaderAndIsr.partitionEpoch < currentLeaderAndIsr.partitionEpoch) {
partitionResponses(tp) = Left(Errors.INVALID_UPDATE_VERSION)
None
} else if (newLeaderAndIsr.equalsIgnorePartitionEpoch(currentLeaderAndIsr)) {
// If a partition is already in the desired state, just return it
partitionResponses(tp) = Right(currentLeaderAndIsr)
None
} else if (newLeaderAndIsr.leaderRecoveryState == LeaderRecoveryState.RECOVERING && newLeaderAndIsr.isr.length > 1) {
partitionResponses(tp) = Left(Errors.INVALID_REQUEST)
info(
s"Rejecting AlterPartition from node $brokerId for $tp because leader is recovering and ISR is greater than 1: " +
Expand All @@ -2363,13 +2373,6 @@ class KafkaController(val config: KafkaConfig,
s"RECOVERED to RECOVERING: $newLeaderAndIsr"
)
None
} else if (newLeaderAndIsr.leaderEpoch < currentLeaderAndIsr.leaderEpoch) {
partitionResponses(tp) = Left(Errors.FENCED_LEADER_EPOCH)
None
} else if (newLeaderAndIsr.equalsIgnorePartitionEpoch(currentLeaderAndIsr)) {
// If a partition is already in the desired state, just return it
partitionResponses(tp) = Right(currentLeaderAndIsr)
None
} else {
Some(tp -> newLeaderAndIsr)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -847,7 +847,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
}

@Test
def testIdempotentAlterIsr(): Unit = {
def testIdempotentAlterPartition(): Unit = {
servers = makeServers(2)
val controllerId = TestUtils.waitUntilControllerElected(zkClient)
val otherBroker = servers.find(_.config.brokerId != controllerId).get
Expand Down Expand Up @@ -881,7 +881,7 @@ class ControllerIntegrationTest extends QuorumTestHarness {
}

@Test
def testAlterIsrErrors(): Unit = {
def testAlterPartitionErrors(): Unit = {
servers = makeServers(2)
val controllerId = TestUtils.waitUntilControllerElected(zkClient)
val tp = new TopicPartition("t", 0)
Expand All @@ -890,65 +890,288 @@ class ControllerIntegrationTest extends QuorumTestHarness {

TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
val controller = getController().kafkaController
var future = captureAlterIsrError(controllerId, controller.brokerEpoch - 1,
Map(tp -> LeaderAndIsr(controllerId, replicas)))
var capturedError = future.get(5, TimeUnit.SECONDS)
assertEquals(Errors.STALE_BROKER_EPOCH, capturedError)

future = captureAlterIsrError(99, controller.brokerEpoch,
Map(tp -> LeaderAndIsr(controllerId, replicas)))
capturedError = future.get(5, TimeUnit.SECONDS)
assertEquals(Errors.STALE_BROKER_EPOCH, capturedError)

val unknownTopicPartition = new TopicPartition("unknown", 99)
future = captureAlterIsrPartitionError(controllerId, controller.brokerEpoch,
Map(unknownTopicPartition -> LeaderAndIsr(controllerId, replicas)), unknownTopicPartition)
capturedError = future.get(5, TimeUnit.SECONDS)
assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, capturedError)

future = captureAlterIsrPartitionError(controllerId, controller.brokerEpoch,
Map(tp -> LeaderAndIsr(controllerId, 1, replicas, LeaderRecoveryState.RECOVERED, 99)), tp)
capturedError = future.get(5, TimeUnit.SECONDS)
assertEquals(Errors.INVALID_UPDATE_VERSION, capturedError)

future = captureAlterIsrPartitionError(controllerId, controller.brokerEpoch,
Map(tp -> LeaderAndIsr(controllerId, 1, replicas, LeaderRecoveryState.RECOVERING, 1)), tp)
capturedError = future.get(5, TimeUnit.SECONDS)
assertEquals(Errors.INVALID_REQUEST, capturedError)

future = captureAlterIsrPartitionError(controllerId, controller.brokerEpoch,
Map(tp -> LeaderAndIsr(controllerId, 1, List(controllerId), LeaderRecoveryState.RECOVERING, 1)), tp)
capturedError = future.get(5, TimeUnit.SECONDS)
assertEquals(Errors.INVALID_REQUEST, capturedError)
val partitionState = controller.controllerContext.partitionLeadershipInfo(tp).get
val leaderId = partitionState.leaderAndIsr.leader
val leaderBrokerEpoch = servers(leaderId).kafkaController.brokerEpoch
val leaderEpoch = partitionState.leaderAndIsr.leaderEpoch
val partitionEpoch = partitionState.leaderAndIsr.partitionEpoch

def assertAlterPartition(
topLevelError: Errors = Errors.NONE,
partitionError: Errors = Errors.NONE,
topicPartition: TopicPartition = tp,
leaderId: Int = leaderId,
brokerEpoch: Long = leaderBrokerEpoch,
leaderEpoch: Int = leaderEpoch,
partitionEpoch: Int = partitionEpoch,
isr: Set[Int] = replicas.toSet,
leaderRecoveryState: LeaderRecoveryState = LeaderRecoveryState.RECOVERED
): Unit = {
assertAlterPartitionError(
topicPartition = topicPartition,
leaderId = leaderId,
brokerEpoch = brokerEpoch,
leaderEpoch = leaderEpoch,
partitionEpoch = partitionEpoch,
isr = isr,
leaderRecoveryState = leaderRecoveryState,
topLevelError = topLevelError,
partitionError = partitionError
)
}

assertAlterPartition(
topLevelError = Errors.STALE_BROKER_EPOCH,
brokerEpoch = leaderBrokerEpoch - 1
)

assertAlterPartition(
topLevelError = Errors.STALE_BROKER_EPOCH,
leaderId = 99,
)

assertAlterPartition(
partitionError = Errors.UNKNOWN_TOPIC_OR_PARTITION,
topicPartition = new TopicPartition("unknown", 0)
)

assertAlterPartition(
partitionError = Errors.UNKNOWN_TOPIC_OR_PARTITION,
topicPartition = new TopicPartition(tp.topic, 1)
)

assertAlterPartition(
partitionError = Errors.INVALID_UPDATE_VERSION,
partitionEpoch = partitionEpoch - 1
)

assertAlterPartition(
partitionError = Errors.FENCED_LEADER_EPOCH,
leaderEpoch = leaderEpoch - 1
)

assertAlterPartition(
partitionError = Errors.FENCED_LEADER_EPOCH,
leaderEpoch = leaderEpoch + 1
)

assertAlterPartition(
partitionError = Errors.INVALID_REQUEST,
leaderRecoveryState = LeaderRecoveryState.RECOVERING
)

assertAlterPartition(
partitionError = Errors.INVALID_REQUEST,
leaderRecoveryState = LeaderRecoveryState.RECOVERING,
isr = Set(controllerId)
)

// Version/epoch errors take precedence over other validations since
// the leader may be working with outdated state.

assertAlterPartition(
partitionError = Errors.INVALID_UPDATE_VERSION,
leaderRecoveryState = LeaderRecoveryState.RECOVERING,
partitionEpoch = partitionEpoch - 1
)

assertAlterPartition(
partitionError = Errors.FENCED_LEADER_EPOCH,
leaderRecoveryState = LeaderRecoveryState.RECOVERING,
leaderEpoch = leaderEpoch - 1
)

assertAlterPartition(
partitionError = Errors.FENCED_LEADER_EPOCH,
leaderRecoveryState = LeaderRecoveryState.RECOVERING,
leaderEpoch = leaderEpoch + 1
)
}

def captureAlterIsrError(brokerId: Int, brokerEpoch: Long, isrsToAlter: Map[TopicPartition, LeaderAndIsr]): CompletableFuture[Errors] = {
val future = new CompletableFuture[Errors]()
@Test
def testAlterPartitionErrorsAfterUncleanElection(): Unit = {
// - Start 3 brokers with unclean election enabled
// - Create a topic with two non-controller replicas: A and B
// - Shutdown A to bring ISR to [B]
// - Shutdown B to make partition offline
// - Restart A to force unclean election with ISR [A]
// - Verify AlterPartition handling in this state

servers = makeServers(numConfigs = 3, uncleanLeaderElectionEnable = true)
val controllerId = TestUtils.waitUntilControllerElected(zkClient)
val controller = getController().kafkaController
val callback: AlterPartitionCallback = {
case Left(_: Map[TopicPartition, Either[Errors, LeaderAndIsr]]) =>
future.completeExceptionally(new AssertionError(s"Should have seen top-level error"))
case Right(error: Errors) =>
future.complete(error)

val tp = new TopicPartition("t", 0)
val replicas = servers.map(_.config.nodeId).filter(_ != controllerId).take(2).toList
val assignment = Map(tp.partition -> replicas)

val replica1 :: replica2 :: Nil = replicas

TestUtils.createTopic(zkClient, tp.topic, partitionReplicaAssignment = assignment, servers = servers)
servers(replica1).shutdown()
servers(replica1).awaitShutdown()

val partitionStateAfterFirstShutdown = controller.controllerContext.partitionLeadershipInfo(tp).get
assertEquals(replica2, partitionStateAfterFirstShutdown.leaderAndIsr.leader)
assertEquals(Set(replica2), partitionStateAfterFirstShutdown.leaderAndIsr.isr.toSet)

servers(replica2).shutdown()
servers(replica2).awaitShutdown()

val partitionStateAfterSecondShutdown = controller.controllerContext.partitionLeadershipInfo(tp).get
assertEquals(-1, partitionStateAfterSecondShutdown.leaderAndIsr.leader)
assertEquals(Set(replica2), partitionStateAfterSecondShutdown.leaderAndIsr.isr.toSet)

servers(replica1).startup()
TestUtils.waitUntilLeaderIsKnown(servers, tp)

val partitionStateAfterRestart = controller.controllerContext.partitionLeadershipInfo(tp).get
assertEquals(replica1, partitionStateAfterRestart.leaderAndIsr.leader)
assertEquals(Set(replica1), partitionStateAfterRestart.leaderAndIsr.isr.toSet)
assertEquals(LeaderRecoveryState.RECOVERING, partitionStateAfterRestart.leaderAndIsr.leaderRecoveryState)

val leaderId = replica1
val leaderBrokerEpoch = servers(replica1).kafkaController.brokerEpoch
val leaderEpoch = partitionStateAfterRestart.leaderAndIsr.leaderEpoch
val partitionEpoch = partitionStateAfterRestart.leaderAndIsr.partitionEpoch

def assertAlterPartition(
topLevelError: Errors = Errors.NONE,
partitionError: Errors = Errors.NONE,
leaderId: Int = leaderId,
brokerEpoch: Long = leaderBrokerEpoch,
leaderEpoch: Int = leaderEpoch,
partitionEpoch: Int = partitionEpoch,
leaderRecoveryState: LeaderRecoveryState = LeaderRecoveryState.RECOVERED
): Unit = {
assertAlterPartitionError(
topicPartition = tp,
leaderId = leaderId,
brokerEpoch = brokerEpoch,
leaderEpoch = leaderEpoch,
partitionEpoch = partitionEpoch,
isr = replicas.toSet,
leaderRecoveryState = leaderRecoveryState,
topLevelError = topLevelError,
partitionError = partitionError
)
}
controller.eventManager.put(AlterPartitionReceived(brokerId, brokerEpoch, isrsToAlter, callback))
future

assertAlterPartition(
topLevelError = Errors.STALE_BROKER_EPOCH,
brokerEpoch = leaderBrokerEpoch - 1
)

assertAlterPartition(
topLevelError = Errors.STALE_BROKER_EPOCH,
leaderId = 99
)

assertAlterPartition(
partitionError = Errors.INVALID_UPDATE_VERSION,
partitionEpoch = partitionEpoch - 1
)

assertAlterPartition(
partitionError = Errors.FENCED_LEADER_EPOCH,
leaderEpoch = leaderEpoch - 1
)

assertAlterPartition(
partitionError = Errors.FENCED_LEADER_EPOCH,
leaderEpoch = leaderEpoch + 1
)

assertAlterPartition(
partitionError = Errors.INVALID_REQUEST,
leaderRecoveryState = LeaderRecoveryState.RECOVERING
)

// Version/epoch errors take precedence over other validations since
// the leader may be working with outdated state.

assertAlterPartition(
partitionError = Errors.INVALID_UPDATE_VERSION,
partitionEpoch = partitionEpoch - 1,
leaderRecoveryState = LeaderRecoveryState.RECOVERING
)

assertAlterPartition(
partitionError = Errors.FENCED_LEADER_EPOCH,
leaderEpoch = leaderEpoch - 1,
leaderRecoveryState = LeaderRecoveryState.RECOVERING
)

assertAlterPartition(
partitionError = Errors.FENCED_LEADER_EPOCH,
leaderEpoch = leaderEpoch + 1,
leaderRecoveryState = LeaderRecoveryState.RECOVERING
)
}

def captureAlterIsrPartitionError(brokerId: Int, brokerEpoch: Long, isrsToAlter: Map[TopicPartition, LeaderAndIsr], tp: TopicPartition): CompletableFuture[Errors] = {
val future = new CompletableFuture[Errors]()
val controller = getController().kafkaController
def assertAlterPartitionError(
topicPartition: TopicPartition,
leaderId: Int,
brokerEpoch: Long,
leaderEpoch: Int,
partitionEpoch: Int,
isr: Set[Int],
leaderRecoveryState: LeaderRecoveryState,
topLevelError: Errors,
partitionError: Errors,
): Unit = {
val leaderAndIsr = LeaderAndIsr(
leader = leaderId,
leaderEpoch = leaderEpoch,
isr = isr.toList,
partitionEpoch = partitionEpoch,
leaderRecoveryState = leaderRecoveryState
)

val future = captureAlterPartitionError(
brokerId = leaderId,
brokerEpoch = brokerEpoch,
topicPartition = topicPartition,
leaderAndIsr = leaderAndIsr
)

val errors = future.get(10, TimeUnit.SECONDS)
assertEquals(topLevelError, errors.topLevelError)

if (topLevelError == Errors.NONE) {
assertEquals(Some(partitionError), errors.partitionError)
}
}

private case class AlterPartitionError(topLevelError: Errors, partitionError: Option[Errors])

private def captureAlterPartitionError(
brokerId: Int,
brokerEpoch: Long,
topicPartition: TopicPartition,
leaderAndIsr: LeaderAndIsr
): CompletableFuture[AlterPartitionError] = {
val future = new CompletableFuture[AlterPartitionError]()
val callback: AlterPartitionCallback = {
case Left(partitionResults: Map[TopicPartition, Either[Errors, LeaderAndIsr]]) =>
partitionResults.get(tp) match {
case Some(Left(error: Errors)) => future.complete(error)
case Some(Right(_: LeaderAndIsr)) => future.completeExceptionally(new AssertionError(s"Should have seen an error for $tp in result"))
case None => future.completeExceptionally(new AssertionError(s"Should have seen $tp in result"))
partitionResults.get(topicPartition) match {
case Some(Left(error: Errors)) =>
future.complete(AlterPartitionError(topLevelError = Errors.NONE, partitionError = Some(error)))

case Some(Right(_: LeaderAndIsr)) =>
future.complete(AlterPartitionError(topLevelError = Errors.NONE, partitionError = Some(Errors.NONE)))

case None =>
future.completeExceptionally(new AssertionError(s"Should have seen $topicPartition in result"))
}
case Right(_: Errors) =>
future.completeExceptionally(new AssertionError(s"Should not seen top-level error"))

case Right(error: Errors) =>
future.complete(AlterPartitionError(topLevelError = error, partitionError = None))
}
controller.eventManager.put(AlterPartitionReceived(brokerId, brokerEpoch, isrsToAlter, callback))

val partitionsToAlter = Map(topicPartition -> leaderAndIsr)
val controller = getController().kafkaController
controller.eventManager.put(AlterPartitionReceived(brokerId, brokerEpoch, partitionsToAlter, callback))
future
}

Expand Down