Browse Source

KAFKA-4623; Default unclean.leader.election.enabled to false (KIP-106)

Author: sharad.develop <sharad.develop@gmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #2625 from sharad-develop/KAFKA-4623
pull/2047/merge
sharad-develop 8 years ago committed by Ismael Juma
parent
commit
46da01a4a7
  1. 2
      core/src/main/scala/kafka/server/KafkaConfig.scala
  2. 1
      core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
  3. 27
      core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala
  4. 2
      core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
  5. 11
      core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala
  6. 2
      docs/upgrade.html

2
core/src/main/scala/kafka/server/KafkaConfig.scala

@ -131,7 +131,7 @@ object Defaults { @@ -131,7 +131,7 @@ object Defaults {
val AutoLeaderRebalanceEnable = true
val LeaderImbalancePerBrokerPercentage = 10
val LeaderImbalanceCheckIntervalSeconds = 300
val UncleanLeaderElectionEnable = true
val UncleanLeaderElectionEnable = false
val InterBrokerSecurityProtocol = SecurityProtocol.PLAINTEXT.toString
val InterBrokerProtocolVersion = ApiVersion.latestVersion.toString

1
core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala

@ -49,6 +49,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { @@ -49,6 +49,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging {
this.serverConfig.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset
this.serverConfig.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1")
this.serverConfig.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, "10") // set small enough session timeout
this.serverConfig.setProperty(KafkaConfig.UncleanLeaderElectionEnableProp, "true")
this.serverConfig.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, "false")
this.producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "all")
this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test")

27
core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala

@ -26,7 +26,6 @@ import java.util.Properties @@ -26,7 +26,6 @@ import java.util.Properties
import java.util.concurrent.ExecutionException
import kafka.admin.AdminUtils
import kafka.common.FailedToSendMessageException
import kafka.consumer.{Consumer, ConsumerConfig}
import kafka.serializer.StringDecoder
import kafka.server.{KafkaConfig, KafkaServer}
@ -67,9 +66,9 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { @@ -67,9 +66,9 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
configProps2 = createBrokerConfig(brokerId2, zkConnect)
for (configProps <- List(configProps1, configProps2)) {
configProps.put("controlled.shutdown.enable", String.valueOf(enableControlledShutdown))
configProps.put("controlled.shutdown.max.retries", String.valueOf(1))
configProps.put("controlled.shutdown.retry.backoff.ms", String.valueOf(1000))
configProps.put("controlled.shutdown.enable", enableControlledShutdown.toString)
configProps.put("controlled.shutdown.max.retries", "1")
configProps.put("controlled.shutdown.retry.backoff.ms", "1000")
}
// temporarily set loggers to a higher level so that tests run quietly
@ -104,7 +103,9 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { @@ -104,7 +103,9 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
@Test
def testUncleanLeaderElectionEnabled {
// unclean leader election is enabled by default
// enable unclean leader election
configProps1.put("unclean.leader.election.enable", "true")
configProps2.put("unclean.leader.election.enable", "true")
startBrokers(Seq(configProps1, configProps2))
// create topic with 1 partition, 2 replicas, one on each broker
@ -115,9 +116,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { @@ -115,9 +116,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
@Test
def testUncleanLeaderElectionDisabled {
// disable unclean leader election
configProps1.put("unclean.leader.election.enable", String.valueOf(false))
configProps2.put("unclean.leader.election.enable", String.valueOf(false))
// unclean leader election is disabled by default
startBrokers(Seq(configProps1, configProps2))
// create topic with 1 partition, 2 replicas, one on each broker
@ -129,13 +128,13 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { @@ -129,13 +128,13 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
@Test
def testUncleanLeaderElectionEnabledByTopicOverride {
// disable unclean leader election globally, but enable for our specific test topic
configProps1.put("unclean.leader.election.enable", String.valueOf(false))
configProps2.put("unclean.leader.election.enable", String.valueOf(false))
configProps1.put("unclean.leader.election.enable", "false")
configProps2.put("unclean.leader.election.enable", "false")
startBrokers(Seq(configProps1, configProps2))
// create topic with 1 partition, 2 replicas, one on each broker, and unclean leader election enabled
val topicProps = new Properties()
topicProps.put("unclean.leader.election.enable", String.valueOf(true))
topicProps.put("unclean.leader.election.enable", "true")
AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, Map(partitionId -> Seq(brokerId1, brokerId2)),
topicProps)
@ -145,13 +144,13 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { @@ -145,13 +144,13 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
@Test
def testCleanLeaderElectionDisabledByTopicOverride {
// enable unclean leader election globally, but disable for our specific test topic
configProps1.put("unclean.leader.election.enable", String.valueOf(true))
configProps2.put("unclean.leader.election.enable", String.valueOf(true))
configProps1.put("unclean.leader.election.enable", "true")
configProps2.put("unclean.leader.election.enable", "true")
startBrokers(Seq(configProps1, configProps2))
// create topic with 1 partition, 2 replicas, one on each broker, and unclean leader election disabled
val topicProps = new Properties()
topicProps.put("unclean.leader.election.enable", String.valueOf(false))
topicProps.put("unclean.leader.election.enable", "false")
AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, Map(partitionId -> Seq(brokerId1, brokerId2)),
topicProps)

2
core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala

@ -410,7 +410,7 @@ class KafkaConfigTest { @@ -410,7 +410,7 @@ class KafkaConfigTest {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
val serverConfig = KafkaConfig.fromProps(props)
assertEquals(serverConfig.uncleanLeaderElectionEnable, true)
assertEquals(serverConfig.uncleanLeaderElectionEnable, false)
}
@Test

11
core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala

@ -49,6 +49,9 @@ class LeaderElectionTest extends ZooKeeperTestHarness { @@ -49,6 +49,9 @@ class LeaderElectionTest extends ZooKeeperTestHarness {
val configProps1 = TestUtils.createBrokerConfig(brokerId1, zkConnect, enableControlledShutdown = false)
val configProps2 = TestUtils.createBrokerConfig(brokerId2, zkConnect, enableControlledShutdown = false)
configProps1.put("unclean.leader.election.enable", "true")
configProps2.put("unclean.leader.election.enable", "true")
// start both servers
val server1 = TestUtils.createServer(KafkaConfig.fromProps(configProps1))
val server2 = TestUtils.createServer(KafkaConfig.fromProps(configProps2))
@ -72,7 +75,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness { @@ -72,7 +75,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness {
val leader1 = createTopic(zkUtils, topic, partitionReplicaAssignment = Map(0 -> Seq(0, 1)), servers = servers)(0)
val leaderEpoch1 = zkUtils.getEpochForPartition(topic, partitionId)
debug("leader Epoc: " + leaderEpoch1)
debug("leader Epoch: " + leaderEpoch1)
debug("Leader is elected to be: %s".format(leader1.getOrElse(-1)))
assertTrue("Leader should get elected", leader1.isDefined)
// NOTE: this is to avoid transient test failures
@ -86,7 +89,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness { @@ -86,7 +89,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness {
oldLeaderOpt = if(leader1.get == 0) None else leader1)
val leaderEpoch2 = zkUtils.getEpochForPartition(topic, partitionId)
debug("Leader is elected to be: %s".format(leader1.getOrElse(-1)))
debug("leader Epoc: " + leaderEpoch2)
debug("leader Epoch: " + leaderEpoch2)
assertEquals("Leader must move to broker 0", 0, leader2.getOrElse(-1))
if(leader1.get == leader2.get)
assertEquals("Second epoch value should be " + leaderEpoch1+1, leaderEpoch1+1, leaderEpoch2)
@ -99,7 +102,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness { @@ -99,7 +102,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness {
val leader3 = waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId,
oldLeaderOpt = if(leader2.get == 1) None else leader2)
val leaderEpoch3 = zkUtils.getEpochForPartition(topic, partitionId)
debug("leader Epoc: " + leaderEpoch3)
debug("leader Epoch: " + leaderEpoch3)
debug("Leader is elected to be: %s".format(leader3.getOrElse(-1)))
assertEquals("Leader must return to 1", 1, leader3.getOrElse(-1))
if(leader2.get == leader3.get)
@ -118,7 +121,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness { @@ -118,7 +121,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness {
val leader1 = createTopic(zkUtils, topic, partitionReplicaAssignment = Map(0 -> Seq(0, 1)), servers = servers)(0)
val leaderEpoch1 = zkUtils.getEpochForPartition(topic, partitionId)
debug("leader Epoc: " + leaderEpoch1)
debug("leader Epoch: " + leaderEpoch1)
debug("Leader is elected to be: %s".format(leader1.getOrElse(-1)))
assertTrue("Leader should get elected", leader1.isDefined)
// NOTE: this is to avoid transient test failures

2
docs/upgrade.html

@ -53,6 +53,8 @@ @@ -53,6 +53,8 @@
<h5><a id="upgrade_1100_notable" href="#upgrade_1100_notable">Notable changes in 0.11.0.0</a></h5>
<ul>
<li>Unclean leader election is now disabled by default. The new default favors durability over availability. Users who wish to
to retain the previous behavior should set the broker config <code>unclean.leader.election.enabled</code> to <code>false</code>.</li>
<li>The <code>offsets.topic.replication.factor</code> broker config is now enforced upon auto topic creation. Internal
auto topic creation will fail with a GROUP_COORDINATOR_NOT_AVAILABLE error until the cluster size meets this
replication factor requirement.</li>

Loading…
Cancel
Save