forked from apache/kafka
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
KAFKA-3670; ControlledShutdownLeaderSelector should pick the preferre…
…d replica as the new leader, if possible Author: Ismael Juma <[email protected]> Reviewers: Jun Rao <[email protected]> Closes apache#1338 from ijuma/kafka-3670-controlled-shutdown-leader-selector-preferred-replica
- Loading branch information
Showing
2 changed files
with
76 additions
and
6 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
73 changes: 73 additions & 0 deletions
73
core/src/test/scala/unit/kafka/server/ControlledShutdownLeaderSelectorTest.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,73 @@ | ||
/** | ||
* Licensed to the Apache Software Foundation (ASF) under one or more | ||
* contributor license agreements. See the NOTICE file distributed with | ||
* this work for additional information regarding copyright ownership. | ||
* The ASF licenses this file to You under the Apache License, Version 2.0 | ||
* (the "License"); you may not use this file except in compliance with | ||
* the License. You may obtain a copy of the License at | ||
* | ||
* http://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
package kafka.server | ||
|
||
import kafka.api.LeaderAndIsr | ||
import kafka.common.TopicAndPartition | ||
import kafka.controller.{ControlledShutdownLeaderSelector, ControllerContext} | ||
import org.easymock.EasyMock | ||
import org.junit.{Assert, Test} | ||
import Assert._ | ||
import kafka.cluster.Broker | ||
import kafka.utils.ZkUtils | ||
|
||
import scala.collection.mutable | ||
|
||
class ControlledShutdownLeaderSelectorTest { | ||
|
||
@Test | ||
def testSelectLeader() { | ||
val topicPartition = TopicAndPartition("topic", 1) | ||
val assignment = Seq(6, 5, 4, 3, 2, 1) | ||
val preferredReplicaId = assignment.head | ||
|
||
val firstIsr = List(1, 3, 6) | ||
val firstLeader = 1 | ||
|
||
val zkUtils = EasyMock.mock(classOf[ZkUtils]) | ||
val controllerContext = new ControllerContext(zkUtils, zkSessionTimeout = 1000) | ||
controllerContext.liveBrokers = assignment.map(Broker(_, Map.empty, None)).toSet | ||
controllerContext.shuttingDownBrokerIds = mutable.Set(2, 3) | ||
controllerContext.partitionReplicaAssignment = mutable.Map(topicPartition -> assignment) | ||
|
||
val leaderSelector = new ControlledShutdownLeaderSelector(controllerContext) | ||
val firstLeaderAndIsr = new LeaderAndIsr(firstLeader, firstIsr) | ||
val (secondLeaderAndIsr, secondReplicas) = leaderSelector.selectLeader(topicPartition, firstLeaderAndIsr) | ||
|
||
assertEquals(preferredReplicaId, secondLeaderAndIsr.leader) | ||
assertEquals(Seq(1, 6), secondLeaderAndIsr.isr) | ||
assertEquals(1, secondLeaderAndIsr.zkVersion) | ||
assertEquals(1, secondLeaderAndIsr.leaderEpoch) | ||
assertEquals(assignment, secondReplicas) | ||
|
||
controllerContext.shuttingDownBrokerIds += preferredReplicaId | ||
|
||
val deadBrokerId = 2 | ||
controllerContext.liveBrokers = controllerContext.liveOrShuttingDownBrokers.filter(_.id != deadBrokerId) | ||
controllerContext.shuttingDownBrokerIds -= deadBrokerId | ||
|
||
val (thirdLeaderAndIsr, thirdReplicas) = leaderSelector.selectLeader(topicPartition, secondLeaderAndIsr) | ||
|
||
assertEquals(1, thirdLeaderAndIsr.leader) | ||
assertEquals(Seq(1), thirdLeaderAndIsr.isr) | ||
assertEquals(2, thirdLeaderAndIsr.zkVersion) | ||
assertEquals(2, thirdLeaderAndIsr.leaderEpoch) | ||
assertEquals(Seq(6, 5, 4, 3, 1), thirdReplicas) | ||
|
||
} | ||
|
||
} |