You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ju...@apache.org on 2015/10/19 00:24:06 UTC
[1/5] kafka git commit: KAFKA-2639: Refactoring of ZkUtils
Repository: kafka
Updated Branches:
refs/heads/trunk 78a2e2f8f -> ce306ba4e
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala
index e4f3576..8e72ad3 100755
--- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala
+++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala
@@ -105,7 +105,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
startBrokers(Seq(configProps1, configProps2))
// create topic with 1 partition, 2 replicas, one on each broker
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2)))
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, Map(partitionId -> Seq(brokerId1, brokerId2)))
verifyUncleanLeaderElectionEnabled
}
@@ -118,7 +118,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
startBrokers(Seq(configProps1, configProps2))
// create topic with 1 partition, 2 replicas, one on each broker
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2)))
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, Map(partitionId -> Seq(brokerId1, brokerId2)))
verifyUncleanLeaderElectionDisabled
}
@@ -133,7 +133,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
// 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))
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2)),
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, Map(partitionId -> Seq(brokerId1, brokerId2)),
topicProps)
verifyUncleanLeaderElectionEnabled
@@ -149,7 +149,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
// 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))
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1, brokerId2)),
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, Map(partitionId -> Seq(brokerId1, brokerId2)),
topicProps)
verifyUncleanLeaderElectionDisabled
@@ -164,13 +164,13 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
topicProps.put("unclean.leader.election.enable", "invalid")
intercept[ConfigException] {
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, Map(partitionId -> Seq(brokerId1)), topicProps)
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, Map(partitionId -> Seq(brokerId1)), topicProps)
}
}
def verifyUncleanLeaderElectionEnabled {
// wait until leader is elected
- val leaderIdOpt = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId)
+ val leaderIdOpt = waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId)
assertTrue("Leader should get elected", leaderIdOpt.isDefined)
val leaderId = leaderIdOpt.get
debug("Leader for " + topic + " is elected to be: %s".format(leaderId))
@@ -195,7 +195,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
servers.filter(server => server.config.brokerId == followerId).map(server => server.startup())
// wait until new leader is (uncleanly) elected
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, newLeaderOpt = Some(followerId))
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId, newLeaderOpt = Some(followerId))
sendMessage(servers, topic, "third")
@@ -205,7 +205,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
def verifyUncleanLeaderElectionDisabled {
// wait until leader is elected
- val leaderIdOpt = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId)
+ val leaderIdOpt = waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId)
assertTrue("Leader should get elected", leaderIdOpt.isDefined)
val leaderId = leaderIdOpt.get
debug("Leader for " + topic + " is elected to be: %s".format(leaderId))
@@ -230,7 +230,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
servers.filter(server => server.config.brokerId == followerId).map(server => server.startup())
// verify that unclean election to non-ISR follower does not occur
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, newLeaderOpt = Some(-1))
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId, newLeaderOpt = Some(-1))
// message production and consumption should both fail while leader is down
intercept[FailedToSendMessageException] {
@@ -240,14 +240,14 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness {
// restart leader temporarily to send a successfully replicated message
servers.filter(server => server.config.brokerId == leaderId).map(server => server.startup())
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, newLeaderOpt = Some(leaderId))
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId, newLeaderOpt = Some(leaderId))
sendMessage(servers, topic, "third")
waitUntilMetadataIsPropagated(servers, topic, partitionId)
servers.filter(server => server.config.brokerId == leaderId).map(server => shutdownServer(server))
// verify clean leader transition to ISR follower
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, newLeaderOpt = Some(followerId))
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId, newLeaderOpt = Some(followerId))
// verify messages can be consumed from ISR follower that was just promoted to leader
assertEquals(List("first", "second", "third"), consumeAllMessages(topic))
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala
index f4e0127..88d95e8 100644
--- a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala
+++ b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala
@@ -57,7 +57,7 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with ZooKeep
requestHandlerLogger.setLevel(Level.FATAL)
// create the topic
- TestUtils.createTopic(zkClient, topic, numParts, 1, servers)
+ TestUtils.createTopic(zkUtils, topic, numParts, 1, servers)
// send some messages to each broker
val sentMessages1 = sendMessages(servers, nMessages, "batch1")
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala
index 6ceb17b..d699386 100644
--- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala
+++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala
@@ -54,7 +54,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
@Test
def testMetricsLeak() {
// create topic topic1 with 1 partition on broker 0
- createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = servers)
+ createTopic(zkUtils, topic, numPartitions = 1, replicationFactor = 1, servers = servers)
// force creation not client's specific metrics.
createAndShutdownStep("group0", "consumer0", "producer0")
@@ -69,9 +69,9 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
@Test
def testMetricsReporterAfterDeletingTopic() {
val topic = "test-topic-metric"
- AdminUtils.createTopic(zkClient, topic, 1, 1)
- AdminUtils.deleteTopic(zkClient, topic)
- TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers)
+ AdminUtils.createTopic(zkUtils, topic, 1, 1)
+ AdminUtils.deleteTopic(zkUtils, topic)
+ TestUtils.verifyTopicDeletion(zkUtils, topic, 1, servers)
assertFalse("Topic metrics exists after deleteTopic", checkTopicMetricsExists(topic))
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/producer/ProducerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala
index dd96d29..18a0cd5 100755
--- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala
+++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala
@@ -100,7 +100,7 @@ class ProducerTest extends ZooKeeperTestHarness with Logging{
@Test
def testUpdateBrokerPartitionInfo() {
val topic = "new-topic"
- TestUtils.createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 2, servers = servers)
+ TestUtils.createTopic(zkUtils, topic, numPartitions = 1, replicationFactor = 2, servers = servers)
val props = new Properties()
// no need to retry since the send will always fail
@@ -155,7 +155,7 @@ class ProducerTest extends ZooKeeperTestHarness with Logging{
val topic = "new-topic"
// create topic with 1 partition and await leadership
- TestUtils.createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 2, servers = servers)
+ TestUtils.createTopic(zkUtils, topic, numPartitions = 1, replicationFactor = 2, servers = servers)
val producer1 = TestUtils.createProducer[String, String](
brokerList = TestUtils.getBrokerListStrFromServers(Seq(server1, server2)),
@@ -168,7 +168,7 @@ class ProducerTest extends ZooKeeperTestHarness with Logging{
producer1.send(new KeyedMessage[String, String](topic, "test", "test1"))
producer1.send(new KeyedMessage[String, String](topic, "test", "test2"))
// get the leader
- val leaderOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0)
+ val leaderOpt = zkUtils.getLeaderForPartition(topic, 0)
assertTrue("Leader for topic new-topic partition 0 should exist", leaderOpt.isDefined)
val leader = leaderOpt.get
@@ -217,7 +217,7 @@ class ProducerTest extends ZooKeeperTestHarness with Logging{
val topic = "new-topic"
// create topic
- TestUtils.createTopic(zkClient, topic, partitionReplicaAssignment = Map(0->Seq(0), 1->Seq(0), 2->Seq(0), 3->Seq(0)),
+ TestUtils.createTopic(zkUtils, topic, partitionReplicaAssignment = Map(0->Seq(0), 1->Seq(0), 2->Seq(0), 3->Seq(0)),
servers = servers)
val producer = TestUtils.createProducer[String, String](
@@ -250,7 +250,7 @@ class ProducerTest extends ZooKeeperTestHarness with Logging{
// restart server 1
server1.startup()
- TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0)
+ TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 0)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0)
try {
@@ -283,7 +283,7 @@ class ProducerTest extends ZooKeeperTestHarness with Logging{
val topic = "new-topic"
// create topics in ZK
- TestUtils.createTopic(zkClient, topic, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers)
+ TestUtils.createTopic(zkUtils, topic, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers)
// do a simple test to make sure plumbing is okay
try {
@@ -330,12 +330,12 @@ class ProducerTest extends ZooKeeperTestHarness with Logging{
try {
// create topic
- AdminUtils.createTopic(zkClient, "new-topic", 2, 1)
+ AdminUtils.createTopic(zkUtils, "new-topic", 2, 1)
TestUtils.waitUntilTrue(() =>
- AdminUtils.fetchTopicMetadataFromZk("new-topic", zkClient).errorCode != ErrorMapping.UnknownTopicOrPartitionCode,
+ AdminUtils.fetchTopicMetadataFromZk("new-topic", zkUtils).errorCode != ErrorMapping.UnknownTopicOrPartitionCode,
"Topic new-topic not created after timeout",
waitTime = zookeeper.tickTime)
- TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "new-topic", 0)
+ TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, "new-topic", 0)
producer.send(new KeyedMessage[String, String]("new-topic", "key", null))
} finally {
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala
index 90689f6..8e15ef8 100644
--- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala
+++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala
@@ -97,7 +97,7 @@ class SyncProducerTest extends KafkaServerTestHarness {
val props = TestUtils.getSyncProducerConfig(server.socketServer.boundPort())
val producer = new SyncProducer(new SyncProducerConfig(props))
- TestUtils.createTopic(zkClient, "test", numPartitions = 1, replicationFactor = 1, servers = servers)
+ TestUtils.createTopic(zkUtils, "test", numPartitions = 1, replicationFactor = 1, servers = servers)
val message1 = new Message(new Array[Byte](configs(0).messageMaxBytes + 1))
val messageSet1 = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = message1)
@@ -126,8 +126,8 @@ class SyncProducerTest extends KafkaServerTestHarness {
props.put("request.required.acks", "0")
val producer = new SyncProducer(new SyncProducerConfig(props))
- AdminUtils.createTopic(zkClient, "test", 1, 1)
- TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "test", 0)
+ AdminUtils.createTopic(zkUtils, "test", 1, 1)
+ TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, "test", 0)
// This message will be dropped silently since message size too large.
producer.send(TestUtils.produceRequest("test", 0,
@@ -167,10 +167,10 @@ class SyncProducerTest extends KafkaServerTestHarness {
}
// #2 - test that we get correct offsets when partition is owned by broker
- AdminUtils.createTopic(zkClient, "topic1", 1, 1)
- TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "topic1", 0)
- AdminUtils.createTopic(zkClient, "topic3", 1, 1)
- TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "topic3", 0)
+ AdminUtils.createTopic(zkUtils, "topic1", 1, 1)
+ TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, "topic1", 0)
+ AdminUtils.createTopic(zkUtils, "topic3", 1, 1)
+ TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, "topic3", 0)
val response2 = producer.send(request)
Assert.assertNotNull(response2)
@@ -244,8 +244,8 @@ class SyncProducerTest extends KafkaServerTestHarness {
val producer = new SyncProducer(new SyncProducerConfig(props))
val topicProps = new Properties()
topicProps.put("min.insync.replicas","2")
- AdminUtils.createTopic(zkClient, topicName, 1, 1,topicProps)
- TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topicName, 0)
+ AdminUtils.createTopic(zkUtils, topicName, 1, 1,topicProps)
+ TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, topicName, 0)
val response = producer.send(TestUtils.produceRequest(topicName, 0,
new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(messageBytes)),-1))
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala
index 655bc20..85252d0 100644
--- a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala
+++ b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala
@@ -219,12 +219,12 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness {
//test remove all acls for resource
simpleAclAuthorizer.removeAcls(resource)
TestUtils.waitAndVerifyAcls(Set.empty[Acl], simpleAclAuthorizer, resource)
- assertTrue(!ZkUtils.pathExists(zkClient, simpleAclAuthorizer.toResourcePath(resource)))
+ assertTrue(!zkUtils.pathExists(simpleAclAuthorizer.toResourcePath(resource)))
//test removing last acl also deletes zookeeper path
acls = changeAclAndVerify(Set.empty[Acl], Set(acl1), Set.empty[Acl])
changeAclAndVerify(acls, Set.empty[Acl], acls)
- assertTrue(!ZkUtils.pathExists(zkClient, simpleAclAuthorizer.toResourcePath(resource)))
+ assertTrue(!zkUtils.pathExists(simpleAclAuthorizer.toResourcePath(resource)))
}
@Test
@@ -240,7 +240,7 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness {
val acls1 = Set[Acl](acl2)
simpleAclAuthorizer.addAcls(acls1, resource1)
- ZkUtils.deletePathRecursive(zkClient, SimpleAclAuthorizer.AclChangedZkPath)
+ zkUtils.deletePathRecursive(SimpleAclAuthorizer.AclChangedZkPath)
val authorizer = new SimpleAclAuthorizer
authorizer.configure(config.originals)
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala
index 7b55f79..75fa664 100755
--- a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala
@@ -49,7 +49,7 @@ class AdvertiseBrokerTest extends ZooKeeperTestHarness {
@Test
def testBrokerAdvertiseToZK {
- val brokerInfo = ZkUtils.getBrokerInfo(zkClient, brokerId)
+ val brokerInfo = zkUtils.getBrokerInfo(brokerId)
val endpoint = brokerInfo.get.endPoints.get(SecurityProtocol.PLAINTEXT).get
assertEquals(advertisedHostName, endpoint.host)
assertEquals(advertisedPort, endpoint.port)
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/server/BaseReplicaFetchTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/BaseReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/BaseReplicaFetchTest.scala
index b744b94..ade110d 100644
--- a/core/src/test/scala/unit/kafka/server/BaseReplicaFetchTest.scala
+++ b/core/src/test/scala/unit/kafka/server/BaseReplicaFetchTest.scala
@@ -57,7 +57,7 @@ abstract class BaseReplicaFetchTest extends ZooKeeperTestHarness {
// create a topic and partition and await leadership
for (topic <- List(topic1,topic2)) {
- createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 2, servers = brokers)
+ createTopic(zkUtils, topic, numPartitions = 1, replicationFactor = 2, servers = brokers)
}
// send test messages to leader
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala
index 16ac40d..6061e66 100644
--- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala
@@ -39,14 +39,14 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
val tp = TopicAndPartition("test", 0)
val logProps = new Properties()
logProps.put(LogConfig.FlushMessagesProp, oldVal.toString)
- AdminUtils.createTopic(zkClient, tp.topic, 1, 1, logProps)
+ AdminUtils.createTopic(zkUtils, tp.topic, 1, 1, logProps)
TestUtils.retry(10000) {
val logOpt = this.servers(0).logManager.getLog(tp)
assertTrue(logOpt.isDefined)
assertEquals(oldVal, logOpt.get.config.flushInterval)
}
logProps.put(LogConfig.FlushMessagesProp, newVal.toString)
- AdminUtils.changeTopicConfig(zkClient, tp.topic, logProps)
+ AdminUtils.changeTopicConfig(zkUtils, tp.topic, logProps)
TestUtils.retry(10000) {
assertEquals(newVal, this.servers(0).logManager.getLog(tp).get.config.flushInterval)
}
@@ -61,7 +61,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
val props = new Properties()
props.put("a.b", "c")
props.put("x.y", "z")
- AdminUtils.changeClientIdConfig(zkClient, clientId, props)
+ AdminUtils.changeClientIdConfig(zkUtils, clientId, props)
TestUtils.retry(10000) {
val configHandler = this.servers(0).dynamicConfigHandlers(ConfigType.Client).asInstanceOf[ClientIdConfigHandler]
assertTrue("ClientId testClient must exist", configHandler.configPool.contains(clientId))
@@ -77,7 +77,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
try {
val logProps = new Properties()
logProps.put(LogConfig.FlushMessagesProp, 10000: java.lang.Integer)
- AdminUtils.changeTopicConfig(zkClient, topic, logProps)
+ AdminUtils.changeTopicConfig(zkUtils, topic, logProps)
fail("Should fail with AdminOperationException for topic doesn't exist")
} catch {
case e: AdminOperationException => // expected
@@ -99,7 +99,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
EasyMock.expectLastCall().once()
EasyMock.replay(handler)
- val configManager = new DynamicConfigManager(zkClient, Map(ConfigType.Topic -> handler))
+ val configManager = new DynamicConfigManager(zkUtils, Map(ConfigType.Topic -> handler))
// Notifications created using the old TopicConfigManager are ignored.
configManager.processNotification(Some("not json"))
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
index c288e56..2e66601 100755
--- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
+++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
@@ -25,7 +25,7 @@ import org.junit._
import org.junit.Assert._
import kafka.common._
import kafka.cluster.Replica
-import kafka.utils.{SystemTime, KafkaScheduler, TestUtils, MockTime, CoreUtils}
+import kafka.utils.{ZkUtils, SystemTime, KafkaScheduler, TestUtils, MockTime, CoreUtils}
import java.util.concurrent.atomic.AtomicBoolean
import org.apache.kafka.common.utils.{MockTime => JMockTime}
@@ -48,15 +48,15 @@ class HighwatermarkPersistenceTest {
@Test
def testHighWatermarkPersistenceSinglePartition() {
// mock zkclient
- val zkClient = EasyMock.createMock(classOf[ZkClient])
- EasyMock.replay(zkClient)
+ val zkUtils = EasyMock.createMock(classOf[ZkUtils])
+ EasyMock.replay(zkUtils)
// create kafka scheduler
val scheduler = new KafkaScheduler(2)
scheduler.startup
val metrics = new Metrics
// create replica manager
- val replicaManager = new ReplicaManager(configs.head, metrics, new MockTime, new JMockTime, zkClient, scheduler,
+ val replicaManager = new ReplicaManager(configs.head, metrics, new MockTime, new JMockTime, zkUtils, scheduler,
logManagers(0), new AtomicBoolean(false))
replicaManager.startup()
try {
@@ -78,7 +78,7 @@ class HighwatermarkPersistenceTest {
replicaManager.checkpointHighWatermarks()
fooPartition0Hw = hwmFor(replicaManager, topic, 0)
assertEquals(leaderReplicaPartition0.highWatermark.messageOffset, fooPartition0Hw)
- EasyMock.verify(zkClient)
+ EasyMock.verify(zkUtils)
} finally {
// shutdown the replica manager upon test completion
replicaManager.shutdown(false)
@@ -92,14 +92,14 @@ class HighwatermarkPersistenceTest {
val topic1 = "foo1"
val topic2 = "foo2"
// mock zkclient
- val zkClient = EasyMock.createMock(classOf[ZkClient])
- EasyMock.replay(zkClient)
+ val zkUtils = EasyMock.createMock(classOf[ZkUtils])
+ EasyMock.replay(zkUtils)
// create kafka scheduler
val scheduler = new KafkaScheduler(2)
scheduler.startup
val metrics = new Metrics
// create replica manager
- val replicaManager = new ReplicaManager(configs.head, metrics, new MockTime(), new JMockTime, zkClient,
+ val replicaManager = new ReplicaManager(configs.head, metrics, new MockTime(), new JMockTime, zkUtils,
scheduler, logManagers(0), new AtomicBoolean(false))
replicaManager.startup()
try {
@@ -144,7 +144,7 @@ class HighwatermarkPersistenceTest {
// verify checkpointed hw for topic 1
topic1Partition0Hw = hwmFor(replicaManager, topic1, 0)
assertEquals(10L, topic1Partition0Hw)
- EasyMock.verify(zkClient)
+ EasyMock.verify(zkUtils)
} finally {
// shutdown the replica manager upon test completion
replicaManager.shutdown(false)
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
index 4637972..42c1199 100755
--- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
@@ -407,7 +407,7 @@ class KafkaConfigTest {
case KafkaConfig.RequestTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.AuthorizerClassNameProp => //ignore string
-
+
case KafkaConfig.PortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
case KafkaConfig.HostNameProp => // ignore string
case KafkaConfig.AdvertisedHostNameProp => //ignore string
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala
index 0efaa6a..1d741f2 100755
--- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala
@@ -70,9 +70,9 @@ class LeaderElectionTest extends ZooKeeperTestHarness {
val partitionId = 0
// create topic with 1 partition, 2 replicas, one on each broker
- val leader1 = createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(0, 1)), servers = servers)(0)
+ val leader1 = createTopic(zkUtils, topic, partitionReplicaAssignment = Map(0 -> Seq(0, 1)), servers = servers)(0)
- val leaderEpoch1 = ZkUtils.getEpochForPartition(zkClient, topic, partitionId)
+ val leaderEpoch1 = zkUtils.getEpochForPartition(topic, partitionId)
debug("leader Epoc: " + leaderEpoch1)
debug("Leader is elected to be: %s".format(leader1.getOrElse(-1)))
assertTrue("Leader should get elected", leader1.isDefined)
@@ -83,9 +83,9 @@ class LeaderElectionTest extends ZooKeeperTestHarness {
// kill the server hosting the preferred replica
servers.last.shutdown()
// check if leader moves to the other server
- val leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId,
+ val leader2 = waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId,
oldLeaderOpt = if(leader1.get == 0) None else leader1)
- val leaderEpoch2 = ZkUtils.getEpochForPartition(zkClient, topic, partitionId)
+ val leaderEpoch2 = zkUtils.getEpochForPartition(topic, partitionId)
debug("Leader is elected to be: %s".format(leader1.getOrElse(-1)))
debug("leader Epoc: " + leaderEpoch2)
assertEquals("Leader must move to broker 0", 0, leader2.getOrElse(-1))
@@ -97,9 +97,9 @@ class LeaderElectionTest extends ZooKeeperTestHarness {
servers.last.startup()
servers.head.shutdown()
Thread.sleep(zookeeper.tickTime)
- val leader3 = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId,
+ val leader3 = waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId,
oldLeaderOpt = if(leader2.get == 1) None else leader2)
- val leaderEpoch3 = ZkUtils.getEpochForPartition(zkClient, topic, partitionId)
+ val leaderEpoch3 = zkUtils.getEpochForPartition(topic, partitionId)
debug("leader Epoc: " + leaderEpoch3)
debug("Leader is elected to be: %s".format(leader3.getOrElse(-1)))
assertEquals("Leader must return to 1", 1, leader3.getOrElse(-1))
@@ -116,9 +116,9 @@ class LeaderElectionTest extends ZooKeeperTestHarness {
val partitionId = 0
// create topic with 1 partition, 2 replicas, one on each broker
- val leader1 = createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(0, 1)), servers = servers)(0)
+ val leader1 = createTopic(zkUtils, topic, partitionReplicaAssignment = Map(0 -> Seq(0, 1)), servers = servers)(0)
- val leaderEpoch1 = ZkUtils.getEpochForPartition(zkClient, topic, partitionId)
+ val leaderEpoch1 = zkUtils.getEpochForPartition(topic, partitionId)
debug("leader Epoc: " + leaderEpoch1)
debug("Leader is elected to be: %s".format(leader1.getOrElse(-1)))
assertTrue("Leader should get elected", leader1.isDefined)
@@ -136,7 +136,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness {
new LeaderAndIsrRequest.EndPoint(brokerEndPoint.id, brokerEndPoint.host, brokerEndPoint.port)
}
- val controllerContext = new ControllerContext(zkClient, zkConnection, 6000)
+ val controllerContext = new ControllerContext(zkUtils, 6000)
controllerContext.liveBrokers = brokers.toSet
val metrics = new Metrics
val controllerChannelManager = new ControllerChannelManager(controllerContext, controllerConfig, new SystemTime, metrics)
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
index 344001d..babf6fb 100755
--- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
@@ -77,7 +77,7 @@ class LogOffsetTest extends ZooKeeperTestHarness {
val part = Integer.valueOf(topicPartition.split("-").last).intValue
// setup brokers in zookeeper as owners of partitions for this test
- AdminUtils.createTopic(zkClient, topic, 1, 1)
+ AdminUtils.createTopic(zkUtils, topic, 1, 1)
val logManager = server.getLogManager
waitUntilTrue(() => logManager.getLog(TopicAndPartition(topic, part)).isDefined,
@@ -117,7 +117,7 @@ class LogOffsetTest extends ZooKeeperTestHarness {
val topic = topicPartition.split("-").head
// setup brokers in zookeeper as owners of partitions for this test
- createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = Seq(server))
+ createTopic(zkUtils, topic, numPartitions = 1, replicationFactor = 1, servers = Seq(server))
var offsetChanged = false
for(i <- 1 to 14) {
@@ -141,7 +141,7 @@ class LogOffsetTest extends ZooKeeperTestHarness {
val part = Integer.valueOf(topicPartition.split("-").last).intValue
// setup brokers in zookeeper as owners of partitions for this test
- AdminUtils.createTopic(zkClient, topic, 3, 1)
+ AdminUtils.createTopic(zkUtils, topic, 3, 1)
val logManager = server.getLogManager
val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig)
@@ -170,7 +170,7 @@ class LogOffsetTest extends ZooKeeperTestHarness {
val part = Integer.valueOf(topicPartition.split("-").last).intValue
// setup brokers in zookeeper as owners of partitions for this test
- AdminUtils.createTopic(zkClient, topic, 3, 1)
+ AdminUtils.createTopic(zkUtils, topic, 3, 1)
val logManager = server.getLogManager
val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig)
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala
index 46829b8..7a434aa 100755
--- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala
@@ -81,7 +81,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness {
servers = List(server1, server2)
// create topic with 1 partition, 2 replicas, one on each broker
- createTopic(zkClient, topic, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers)
+ createTopic(zkUtils, topic, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers)
// create the producer
updateProducer()
@@ -116,7 +116,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness {
@Test
def testHWCheckpointWithFailuresSingleLogSegment {
- var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId)
+ var leader = waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId)
assertEquals(0L, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L))
@@ -129,7 +129,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness {
assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L))
// check if leader moves to the other server
- leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, oldLeaderOpt = leader)
+ leader = waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId, oldLeaderOpt = leader)
assertEquals("Leader must move to broker 1", 1, leader.getOrElse(-1))
// bring the preferred replica back
@@ -137,7 +137,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness {
// Update producer with new server settings
updateProducer()
- leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId)
+ leader = waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId)
assertTrue("Leader must remain on broker 1, in case of zookeeper session expiration it can move to broker 0",
leader.isDefined && (leader.get == 0 || leader.get == 1))
@@ -148,7 +148,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness {
server2.startup()
updateProducer()
- leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, oldLeaderOpt = leader)
+ leader = waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId, oldLeaderOpt = leader)
assertTrue("Leader must remain on broker 0, in case of zookeeper session expiration it can move to broker 1",
leader.isDefined && (leader.get == 0 || leader.get == 1))
@@ -183,7 +183,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness {
@Test
def testHWCheckpointWithFailuresMultipleLogSegments {
- var leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId)
+ var leader = waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId)
sendMessages(2)
var hw = 2L
@@ -201,7 +201,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness {
server2.startup()
updateProducer()
// check if leader moves to the other server
- leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, oldLeaderOpt = leader)
+ leader = waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId, oldLeaderOpt = leader)
assertEquals("Leader must move to broker 1", 1, leader.getOrElse(-1))
assertEquals(hw, hwFile1.read.getOrElse(TopicAndPartition(topic, 0), 0L))
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
index f846698..7440500 100755
--- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
+++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
@@ -82,7 +82,7 @@ class OffsetCommitTest extends ZooKeeperTestHarness {
val topicAndPartition = TopicAndPartition(topic, 0)
val expectedReplicaAssignment = Map(0 -> List(1))
// create the topic
- createTopic(zkClient, topic, partitionReplicaAssignment = expectedReplicaAssignment, servers = Seq(server))
+ createTopic(zkUtils, topic, partitionReplicaAssignment = expectedReplicaAssignment, servers = Seq(server))
val commitRequest = OffsetCommitRequest(group, immutable.Map(topicAndPartition -> OffsetAndMetadata(offset = 42L)))
val commitResponse = simpleConsumer.commitOffsets(commitRequest)
@@ -131,10 +131,10 @@ class OffsetCommitTest extends ZooKeeperTestHarness {
val topic4 = "topic-4" // Topic that group never consumes
val topic5 = "topic-5" // Non-existent topic
- createTopic(zkClient, topic1, servers = Seq(server), numPartitions = 1)
- createTopic(zkClient, topic2, servers = Seq(server), numPartitions = 2)
- createTopic(zkClient, topic3, servers = Seq(server), numPartitions = 1)
- createTopic(zkClient, topic4, servers = Seq(server), numPartitions = 1)
+ createTopic(zkUtils, topic1, servers = Seq(server), numPartitions = 1)
+ createTopic(zkUtils, topic2, servers = Seq(server), numPartitions = 2)
+ createTopic(zkUtils, topic3, servers = Seq(server), numPartitions = 1)
+ createTopic(zkUtils, topic4, servers = Seq(server), numPartitions = 1)
val commitRequest = OffsetCommitRequest("test-group", immutable.Map(
TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="metadata one"),
@@ -197,7 +197,7 @@ class OffsetCommitTest extends ZooKeeperTestHarness {
def testLargeMetadataPayload() {
val topicAndPartition = TopicAndPartition("large-metadata", 0)
val expectedReplicaAssignment = Map(0 -> List(1))
- createTopic(zkClient, topicAndPartition.topic, partitionReplicaAssignment = expectedReplicaAssignment,
+ createTopic(zkUtils, topicAndPartition.topic, partitionReplicaAssignment = expectedReplicaAssignment,
servers = Seq(server))
val commitRequest = OffsetCommitRequest("test-group", immutable.Map(topicAndPartition -> OffsetAndMetadata(
@@ -222,7 +222,7 @@ class OffsetCommitTest extends ZooKeeperTestHarness {
// set up topic partition
val topic = "topic"
val topicPartition = TopicAndPartition(topic, 0)
- createTopic(zkClient, topic, servers = Seq(server), numPartitions = 1)
+ createTopic(zkUtils, topic, servers = Seq(server), numPartitions = 1)
val fetchRequest = OffsetFetchRequest(group, Seq(TopicAndPartition(topic, 0)))
@@ -293,7 +293,7 @@ class OffsetCommitTest extends ZooKeeperTestHarness {
val topic1 = "topicDoesNotExists"
val topic2 = "topic-2"
- createTopic(zkClient, topic2, servers = Seq(server), numPartitions = 1)
+ createTopic(zkUtils, topic2, servers = Seq(server), numPartitions = 1)
// Commit an offset
val expectedReplicaAssignment = Map(0 -> List(1))
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 1813349..724d4ac 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -19,7 +19,7 @@ package kafka.server
import kafka.api.{ProducerResponseStatus, SerializationTestUtils, ProducerRequest}
import kafka.common.TopicAndPartition
-import kafka.utils.{MockScheduler, MockTime, TestUtils}
+import kafka.utils.{ZkUtils, MockScheduler, MockTime, TestUtils}
import java.util.concurrent.atomic.AtomicBoolean
import java.io.File
@@ -42,11 +42,12 @@ class ReplicaManagerTest {
val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect)
val config = KafkaConfig.fromProps(props)
val zkClient = EasyMock.createMock(classOf[ZkClient])
+ val zkUtils = ZkUtils(zkClient, false)
val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray)
val time: MockTime = new MockTime()
val jTime = new JMockTime
val metrics = new Metrics
- val rm = new ReplicaManager(config, metrics, time, jTime, zkClient, new MockScheduler(time), mockLogMgr,
+ val rm = new ReplicaManager(config, metrics, time, jTime, zkUtils, new MockScheduler(time), mockLogMgr,
new AtomicBoolean(false))
try {
val partition = rm.getOrCreatePartition(topic, 1)
@@ -65,11 +66,12 @@ class ReplicaManagerTest {
props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath)
val config = KafkaConfig.fromProps(props)
val zkClient = EasyMock.createMock(classOf[ZkClient])
+ val zkUtils = ZkUtils(zkClient, false)
val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray)
val time: MockTime = new MockTime()
val jTime = new JMockTime
val metrics = new Metrics
- val rm = new ReplicaManager(config, metrics, time, jTime, zkClient, new MockScheduler(time), mockLogMgr,
+ val rm = new ReplicaManager(config, metrics, time, jTime, zkUtils, new MockScheduler(time), mockLogMgr,
new AtomicBoolean(false))
try {
val partition = rm.getOrCreatePartition(topic, 1)
@@ -87,11 +89,12 @@ class ReplicaManagerTest {
val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect)
val config = KafkaConfig.fromProps(props)
val zkClient = EasyMock.createMock(classOf[ZkClient])
+ val zkUtils = ZkUtils(zkClient, false)
val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray)
val time: MockTime = new MockTime()
val jTime = new JMockTime
val metrics = new Metrics
- val rm = new ReplicaManager(config, metrics, time, jTime, zkClient, new MockScheduler(time), mockLogMgr,
+ val rm = new ReplicaManager(config, metrics, time, jTime, zkUtils, new MockScheduler(time), mockLogMgr,
new AtomicBoolean(false), Option(this.getClass.getName))
try {
val produceRequest = new ProducerRequest(1, "client 1", 3, 1000, SerializationTestUtils.topicDataProducerRequest)
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala
index b6d5697..8f081b9 100755
--- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala
@@ -53,7 +53,7 @@ class ServerShutdownTest extends ZooKeeperTestHarness {
keyEncoder = classOf[IntEncoder].getName)
// create topic
- createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = Seq(server))
+ createTopic(zkUtils, topic, numPartitions = 1, replicationFactor = 1, servers = Seq(server))
// send some messages
producer.send(sent1.map(m => new KeyedMessage[Int, String](topic, 0, m)):_*)
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala
index 0adc0aa..145f00a 100755
--- a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala
@@ -36,7 +36,7 @@ class ServerStartupTest extends ZooKeeperTestHarness {
props.put("zookeeper.connect", zooKeeperConnect + zookeeperChroot)
val server = TestUtils.createServer(KafkaConfig.fromProps(props))
- val pathExists = ZkUtils.pathExists(zkClient, zookeeperChroot)
+ val pathExists = zkUtils.pathExists(zookeeperChroot)
assertTrue(pathExists)
server.shutdown()
@@ -51,7 +51,7 @@ class ServerStartupTest extends ZooKeeperTestHarness {
val brokerId = 0
val props1 = TestUtils.createBrokerConfig(brokerId, zkConnect)
val server1 = TestUtils.createServer(KafkaConfig.fromProps(props1))
- val brokerRegistration = ZkUtils.readData(zkClient, ZkUtils.BrokerIdsPath + "/" + brokerId)._1
+ val brokerRegistration = zkUtils.readData(ZkUtils.BrokerIdsPath + "/" + brokerId)._1
val props2 = TestUtils.createBrokerConfig(brokerId, zkConnect)
try {
@@ -63,7 +63,7 @@ class ServerStartupTest extends ZooKeeperTestHarness {
}
// broker registration shouldn't change
- assertEquals(brokerRegistration, ZkUtils.readData(zkClient, ZkUtils.BrokerIdsPath + "/" + brokerId)._1)
+ assertEquals(brokerRegistration, zkUtils.readData(ZkUtils.BrokerIdsPath + "/" + brokerId)._1)
server1.shutdown()
CoreUtils.rm(server1.config.logDirs)
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
index 0485f7b..689b70b 100644
--- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
+++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
@@ -69,8 +69,8 @@ class SimpleFetchTest {
@Before
def setUp() {
// create nice mock since we don't particularly care about zkclient calls
- val zkClient = EasyMock.createNiceMock(classOf[ZkClient])
- EasyMock.replay(zkClient)
+ val zkUtils = EasyMock.createNiceMock(classOf[ZkUtils])
+ EasyMock.replay(zkUtils)
// create nice mock since we don't particularly care about scheduler calls
val scheduler = EasyMock.createNiceMock(classOf[KafkaScheduler])
@@ -98,7 +98,7 @@ class SimpleFetchTest {
EasyMock.replay(logManager)
// create the replica manager
- replicaManager = new ReplicaManager(configs.head, metrics, time, jTime, zkClient, scheduler, logManager,
+ replicaManager = new ReplicaManager(configs.head, metrics, time, jTime, zkUtils, scheduler, logManager,
new AtomicBoolean(false))
// add the partition with two replicas, both in ISR
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala
index b3835f0..d4ddb2f 100644
--- a/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala
+++ b/core/src/test/scala/unit/kafka/utils/ReplicationUtilsTest.scala
@@ -47,7 +47,7 @@ class ReplicationUtilsTest extends ZooKeeperTestHarness {
@Before
override def setUp() {
super.setUp()
- ZkUtils.createPersistentPath(zkClient,topicPath,topicData)
+ zkUtils.createPersistentPath(topicPath, topicData)
}
@Test
@@ -66,23 +66,23 @@ class ReplicationUtilsTest extends ZooKeeperTestHarness {
EasyMock.expect(replicaManager.config).andReturn(configs.head)
EasyMock.expect(replicaManager.logManager).andReturn(logManager)
EasyMock.expect(replicaManager.replicaFetcherManager).andReturn(EasyMock.createMock(classOf[ReplicaFetcherManager]))
- EasyMock.expect(replicaManager.zkClient).andReturn(zkClient)
+ EasyMock.expect(replicaManager.zkUtils).andReturn(zkUtils)
EasyMock.replay(replicaManager)
- ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.IsrChangeNotificationPath)
+ zkUtils.makeSurePersistentPathExists(ZkUtils.IsrChangeNotificationPath)
val replicas = List(0,1)
// regular update
val newLeaderAndIsr1 = new LeaderAndIsr(brokerId, leaderEpoch, replicas, 0)
- val (updateSucceeded1,newZkVersion1) = ReplicationUtils.updateLeaderAndIsr(zkClient,
+ val (updateSucceeded1,newZkVersion1) = ReplicationUtils.updateLeaderAndIsr(zkUtils,
"my-topic-test", partitionId, newLeaderAndIsr1, controllerEpoch, 0)
assertTrue(updateSucceeded1)
assertEquals(newZkVersion1, 1)
// mismatched zkVersion with the same data
val newLeaderAndIsr2 = new LeaderAndIsr(brokerId, leaderEpoch, replicas, zkVersion + 1)
- val (updateSucceeded2,newZkVersion2) = ReplicationUtils.updateLeaderAndIsr(zkClient,
+ val (updateSucceeded2,newZkVersion2) = ReplicationUtils.updateLeaderAndIsr(zkUtils,
"my-topic-test", partitionId, newLeaderAndIsr2, controllerEpoch, zkVersion + 1)
assertTrue(updateSucceeded2)
// returns true with existing zkVersion
@@ -90,7 +90,7 @@ class ReplicationUtilsTest extends ZooKeeperTestHarness {
// mismatched zkVersion and leaderEpoch
val newLeaderAndIsr3 = new LeaderAndIsr(brokerId, leaderEpoch + 1, replicas, zkVersion + 1)
- val (updateSucceeded3,newZkVersion3) = ReplicationUtils.updateLeaderAndIsr(zkClient,
+ val (updateSucceeded3,newZkVersion3) = ReplicationUtils.updateLeaderAndIsr(zkUtils,
"my-topic-test", partitionId, newLeaderAndIsr3, controllerEpoch, zkVersion + 1)
assertFalse(updateSucceeded3)
assertEquals(newZkVersion3,-1)
@@ -98,9 +98,9 @@ class ReplicationUtilsTest extends ZooKeeperTestHarness {
@Test
def testGetLeaderIsrAndEpochForPartition() {
- val leaderIsrAndControllerEpoch = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partitionId)
+ val leaderIsrAndControllerEpoch = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkUtils, topic, partitionId)
assertEquals(topicDataLeaderIsrAndControllerEpoch, leaderIsrAndControllerEpoch.get)
- assertEquals(None, ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partitionId + 1))
+ assertEquals(None, ReplicationUtils.getLeaderIsrAndEpochForPartition(zkUtils, topic, partitionId + 1))
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/utils/TestUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index 4a53e11..1a0a7dc 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -45,6 +45,7 @@ import kafka.common.TopicAndPartition
import kafka.admin.AdminUtils
import kafka.producer.ProducerConfig
import kafka.log._
+import kafka.utils.ZkUtils._
import org.junit.Assert._
import org.apache.kafka.clients.producer.KafkaProducer
@@ -187,18 +188,18 @@ object TestUtils extends Logging {
* Wait until the leader is elected and the metadata is propagated to all brokers.
* Return the leader for each partition.
*/
- def createTopic(zkClient: ZkClient,
+ def createTopic(zkUtils: ZkUtils,
topic: String,
numPartitions: Int = 1,
replicationFactor: Int = 1,
servers: Seq[KafkaServer],
topicConfig: Properties = new Properties) : scala.collection.immutable.Map[Int, Option[Int]] = {
// create topic
- AdminUtils.createTopic(zkClient, topic, numPartitions, replicationFactor, topicConfig)
+ AdminUtils.createTopic(zkUtils, topic, numPartitions, replicationFactor, topicConfig)
// wait until the update metadata request for new topic reaches all servers
(0 until numPartitions).map { case i =>
TestUtils.waitUntilMetadataIsPropagated(servers, topic, i)
- i -> TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, i)
+ i -> TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, topic, i)
}.toMap
}
@@ -207,14 +208,14 @@ object TestUtils extends Logging {
* Wait until the leader is elected and the metadata is propagated to all brokers.
* Return the leader for each partition.
*/
- def createTopic(zkClient: ZkClient, topic: String, partitionReplicaAssignment: collection.Map[Int, Seq[Int]],
+ def createTopic(zkUtils: ZkUtils, topic: String, partitionReplicaAssignment: collection.Map[Int, Seq[Int]],
servers: Seq[KafkaServer]) : scala.collection.immutable.Map[Int, Option[Int]] = {
// create topic
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaAssignment)
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, partitionReplicaAssignment)
// wait until the update metadata request for new topic reaches all servers
partitionReplicaAssignment.keySet.map { case i =>
TestUtils.waitUntilMetadataIsPropagated(servers, topic, i)
- i -> TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, i)
+ i -> TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, topic, i)
}.toMap
}
@@ -484,8 +485,8 @@ object TestUtils extends Logging {
}
def updateConsumerOffset(config : ConsumerConfig, path : String, offset : Long) = {
- val zkClient = ZkUtils.createZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs)
- ZkUtils.updatePersistentPath(zkClient, path, offset.toString)
+ val zkUtils = ZkUtils(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false)
+ zkUtils.updatePersistentPath(path, offset.toString)
}
@@ -500,15 +501,15 @@ object TestUtils extends Logging {
}
}
- def createBrokersInZk(zkClient: ZkClient, zkConnection: ZkConnection, ids: Seq[Int]): Seq[Broker] = {
+ def createBrokersInZk(zkUtils: ZkUtils, ids: Seq[Int]): Seq[Broker] = {
val brokers = ids.map(id => new Broker(id, "localhost", 6667, SecurityProtocol.PLAINTEXT))
- brokers.foreach(b => ZkUtils.registerBrokerInZk(zkClient, zkConnection, b.id, "localhost", 6667, b.endPoints, jmxPort = -1))
+ brokers.foreach(b => zkUtils.registerBrokerInZk(b.id, "localhost", 6667, b.endPoints, jmxPort = -1))
brokers
}
- def deleteBrokersInZk(zkClient: ZkClient, ids: Seq[Int]): Seq[Broker] = {
+ def deleteBrokersInZk(zkUtils: ZkUtils, ids: Seq[Int]): Seq[Broker] = {
val brokers = ids.map(id => new Broker(id, "localhost", 6667, SecurityProtocol.PLAINTEXT))
- brokers.foreach(b => ZkUtils.deletePath(zkClient, ZkUtils.BrokerIdsPath + "/" + b))
+ brokers.foreach(b => zkUtils.deletePath(ZkUtils.BrokerIdsPath + "/" + b))
brokers
}
@@ -545,7 +546,7 @@ object TestUtils extends Logging {
new ProducerRequest(correlationId, clientId, acks.toShort, timeout, collection.mutable.Map(data:_*))
}
- def makeLeaderForPartition(zkClient: ZkClient, topic: String,
+ def makeLeaderForPartition(zkUtils: ZkUtils, topic: String,
leaderPerPartitionMap: scala.collection.immutable.Map[Int, Int],
controllerEpoch: Int) {
leaderPerPartitionMap.foreach
@@ -554,7 +555,7 @@ object TestUtils extends Logging {
val partition = leaderForPartition._1
val leader = leaderForPartition._2
try{
- val currentLeaderAndIsrOpt = ZkUtils.getLeaderAndIsrForPartition(zkClient, topic, partition)
+ val currentLeaderAndIsrOpt = zkUtils.getLeaderAndIsrForPartition(topic, partition)
var newLeaderAndIsr: LeaderAndIsr = null
if(currentLeaderAndIsrOpt == None)
newLeaderAndIsr = new LeaderAndIsr(leader, List(leader))
@@ -564,8 +565,8 @@ object TestUtils extends Logging {
newLeaderAndIsr.leaderEpoch += 1
newLeaderAndIsr.zkVersion += 1
}
- ZkUtils.updatePersistentPath(zkClient, ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partition),
- ZkUtils.leaderAndIsrZkData(newLeaderAndIsr, controllerEpoch))
+ zkUtils.updatePersistentPath(getTopicPartitionLeaderAndIsrPath(topic, partition),
+ zkUtils.leaderAndIsrZkData(newLeaderAndIsr, controllerEpoch))
} catch {
case oe: Throwable => error("Error while electing leader for partition [%s,%d]".format(topic, partition), oe)
}
@@ -579,7 +580,7 @@ object TestUtils extends Logging {
* If newLeaderOpt is defined, it waits until the new leader becomes the expected new leader.
* @return The new leader or assertion failure if timeout is reached.
*/
- def waitUntilLeaderIsElectedOrChanged(zkClient: ZkClient, topic: String, partition: Int, timeoutMs: Long = 5000L,
+ def waitUntilLeaderIsElectedOrChanged(zkUtils: ZkUtils, topic: String, partition: Int, timeoutMs: Long = 5000L,
oldLeaderOpt: Option[Int] = None, newLeaderOpt: Option[Int] = None): Option[Int] = {
require(!(oldLeaderOpt.isDefined && newLeaderOpt.isDefined), "Can't define both the old and the new leader")
val startTime = System.currentTimeMillis()
@@ -591,7 +592,7 @@ object TestUtils extends Logging {
var leader: Option[Int] = None
while (!isLeaderElectedOrChanged && System.currentTimeMillis() < startTime + timeoutMs) {
// check if leader is elected
- leader = ZkUtils.getLeaderForPartition(zkClient, topic, partition)
+ leader = zkUtils.getLeaderForPartition(topic, partition)
leader match {
case Some(l) =>
if (newLeaderOpt.isDefined && newLeaderOpt.get == l) {
@@ -724,24 +725,24 @@ object TestUtils extends Logging {
file.close()
}
- def checkForPhantomInSyncReplicas(zkClient: ZkClient, topic: String, partitionToBeReassigned: Int, assignedReplicas: Seq[Int]) {
- val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic, partitionToBeReassigned)
+ def checkForPhantomInSyncReplicas(zkUtils: ZkUtils, topic: String, partitionToBeReassigned: Int, assignedReplicas: Seq[Int]) {
+ val inSyncReplicas = zkUtils.getInSyncReplicasForPartition(topic, partitionToBeReassigned)
// in sync replicas should not have any replica that is not in the new assigned replicas
val phantomInSyncReplicas = inSyncReplicas.toSet -- assignedReplicas.toSet
assertTrue("All in sync replicas %s must be in the assigned replica list %s".format(inSyncReplicas, assignedReplicas),
phantomInSyncReplicas.size == 0)
}
- def ensureNoUnderReplicatedPartitions(zkClient: ZkClient, topic: String, partitionToBeReassigned: Int, assignedReplicas: Seq[Int],
+ def ensureNoUnderReplicatedPartitions(zkUtils: ZkUtils, topic: String, partitionToBeReassigned: Int, assignedReplicas: Seq[Int],
servers: Seq[KafkaServer]) {
TestUtils.waitUntilTrue(() => {
- val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic, partitionToBeReassigned)
+ val inSyncReplicas = zkUtils.getInSyncReplicasForPartition(topic, partitionToBeReassigned)
inSyncReplicas.size == assignedReplicas.size
},
"Reassigned partition [%s,%d] is under replicated".format(topic, partitionToBeReassigned))
var leader: Option[Int] = None
TestUtils.waitUntilTrue(() => {
- leader = ZkUtils.getLeaderForPartition(zkClient, topic, partitionToBeReassigned)
+ leader = zkUtils.getLeaderForPartition(topic, partitionToBeReassigned)
leader.isDefined
},
"Reassigned partition [%s,%d] is unavailable".format(topic, partitionToBeReassigned))
@@ -752,8 +753,8 @@ object TestUtils extends Logging {
"Reassigned partition [%s,%d] is under-replicated as reported by the leader %d".format(topic, partitionToBeReassigned, leader.get))
}
- def checkIfReassignPartitionPathExists(zkClient: ZkClient): Boolean = {
- ZkUtils.pathExists(zkClient, ZkUtils.ReassignPartitionsPath)
+ def checkIfReassignPartitionPathExists(zkUtils: ZkUtils): Boolean = {
+ zkUtils.pathExists(ZkUtils.ReassignPartitionsPath)
}
def verifyNonDaemonThreadsStatus(threadNamePrefix: String) {
@@ -875,12 +876,12 @@ object TestUtils extends Logging {
messages.reverse
}
- def verifyTopicDeletion(zkClient: ZkClient, topic: String, numPartitions: Int, servers: Seq[KafkaServer]) {
+ def verifyTopicDeletion(zkUtils: ZkUtils, topic: String, numPartitions: Int, servers: Seq[KafkaServer]) {
val topicAndPartitions = (0 until numPartitions).map(TopicAndPartition(topic, _))
// wait until admin path for delete topic is deleted, signaling completion of topic deletion
- TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)),
+ TestUtils.waitUntilTrue(() => !zkUtils.pathExists(getDeleteTopicPath(topic)),
"Admin path /admin/delete_topic/%s path not deleted even after a replica is restarted".format(topic))
- TestUtils.waitUntilTrue(() => !ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic)),
+ TestUtils.waitUntilTrue(() => !zkUtils.pathExists(getTopicPath(topic)),
"Topic path /brokers/topics/%s not deleted after /admin/delete_topic/%s path is deleted".format(topic, topic))
// ensure that the topic-partition has been deleted from all brokers' replica managers
TestUtils.waitUntilTrue(() =>
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala
index 2bf658c..eb08f58 100644
--- a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala
+++ b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala
@@ -34,20 +34,20 @@ class ZKEphemeralTest extends ZooKeeperTestHarness {
@Test
def testEphemeralNodeCleanup = {
val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1"))
- var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs)
+ var zkUtils = ZkUtils(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false)
try {
- ZkUtils.createEphemeralPathExpectConflict(zkClient, "/tmp/zktest", "node created")
+ zkUtils.createEphemeralPathExpectConflict("/tmp/zktest", "node created")
} catch {
case e: Exception =>
}
var testData: String = null
- testData = ZkUtils.readData(zkClient, "/tmp/zktest")._1
+ testData = zkUtils.readData("/tmp/zktest")._1
Assert.assertNotNull(testData)
- zkClient.close
- zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs)
- val nodeExists = ZkUtils.pathExists(zkClient, "/tmp/zktest")
+ zkUtils.close
+ zkUtils = ZkUtils(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false)
+ val nodeExists = zkUtils.pathExists("/tmp/zktest")
Assert.assertFalse(nodeExists)
}
@@ -74,8 +74,8 @@ class ZKEphemeralTest extends ZooKeeperTestHarness {
}
private def testCreation(path: String) {
- val zk = zkConnection.getZookeeper
- val zwe = new ZKCheckedEphemeral(path, "", zk)
+ val zk = zkUtils.zkConnection.getZookeeper
+ val zwe = new ZKCheckedEphemeral(path, "", zk, false)
var created = false
var counter = 10
@@ -88,7 +88,7 @@ class ZKEphemeralTest extends ZooKeeperTestHarness {
})
zwe.create()
// Waits until the znode is created
- TestUtils.waitUntilTrue(() => ZkUtils.pathExists(zkClient, path),
+ TestUtils.waitUntilTrue(() => zkUtils.pathExists(path),
"Znode %s wasn't created".format(path))
}
@@ -99,12 +99,12 @@ class ZKEphemeralTest extends ZooKeeperTestHarness {
@Test
def testOverlappingSessions = {
val path = "/zwe-test"
- val zk1 = zkConnection.getZookeeper
+ val zk1 = zkUtils.zkConnection.getZookeeper
//Creates a second session
val (_, zkConnection2) = ZkUtils.createZkClientAndConnection(zkConnect, zkSessionTimeoutMs, zkConnectionTimeout)
val zk2 = zkConnection2.getZookeeper
- var zwe = new ZKCheckedEphemeral(path, "", zk2)
+ var zwe = new ZKCheckedEphemeral(path, "", zk2, false)
// Creates znode for path in the first session
zk1.create(path, Array[Byte](), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL)
@@ -127,11 +127,11 @@ class ZKEphemeralTest extends ZooKeeperTestHarness {
@Test
def testSameSession = {
val path = "/zwe-test"
- val zk = zkConnection.getZookeeper
+ val zk = zkUtils.zkConnection.getZookeeper
// Creates znode for path in the first session
zk.create(path, Array[Byte](), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL)
- var zwe = new ZKCheckedEphemeral(path, "", zk)
+ var zwe = new ZKCheckedEphemeral(path, "", zk, false)
//Bootstraps the ZKWatchedEphemeral object
var gotException = false;
try {
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala
index 241eea5..65dd589 100644
--- a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala
+++ b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala
@@ -33,11 +33,11 @@ class ZKPathTest extends ZooKeeperTestHarness {
def testCreatePersistentPathThrowsException {
val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot,
"test", "1"))
- var zkClient = ZkUtils.createZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs,
- config.zkConnectionTimeoutMs)
+ var zkUtils = ZkUtils(zkConnectWithInvalidRoot, zkSessionTimeoutMs,
+ config.zkConnectionTimeoutMs, false)
try {
ZkPath.resetNamespaceCheckedState
- ZkUtils.createPersistentPath(zkClient, path)
+ zkUtils.createPersistentPath(path)
fail("Failed to throw ConfigException for missing zookeeper root node")
} catch {
case configException: ConfigException =>
@@ -48,26 +48,26 @@ class ZKPathTest extends ZooKeeperTestHarness {
@Test
def testCreatePersistentPath {
val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1"))
- var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs)
+ var zkUtils = ZkUtils(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false)
try {
ZkPath.resetNamespaceCheckedState
- ZkUtils.createPersistentPath(zkClient, path)
+ zkUtils.createPersistentPath(path)
} catch {
case exception: Throwable => fail("Failed to create persistent path")
}
- assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, path))
+ assertTrue("Failed to create persistent path", zkUtils.pathExists(path))
}
@Test
def testMakeSurePersistsPathExistsThrowsException {
val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot,
"test", "1"))
- var zkClient = ZkUtils.createZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs,
- config.zkConnectionTimeoutMs)
+ var zkUtils = ZkUtils(zkConnectWithInvalidRoot, zkSessionTimeoutMs,
+ config.zkConnectionTimeoutMs, false)
try {
ZkPath.resetNamespaceCheckedState
- ZkUtils.makeSurePersistentPathExists(zkClient, path)
+ zkUtils.makeSurePersistentPathExists(path)
fail("Failed to throw ConfigException for missing zookeeper root node")
} catch {
case configException: ConfigException =>
@@ -78,26 +78,26 @@ class ZKPathTest extends ZooKeeperTestHarness {
@Test
def testMakeSurePersistsPathExists {
val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1"))
- var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs)
+ var zkUtils = ZkUtils(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false)
try {
ZkPath.resetNamespaceCheckedState
- ZkUtils.makeSurePersistentPathExists(zkClient, path)
+ zkUtils.makeSurePersistentPathExists(path)
} catch {
case exception: Throwable => fail("Failed to create persistent path")
}
- assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, path))
+ assertTrue("Failed to create persistent path", zkUtils.pathExists(path))
}
@Test
def testCreateEphemeralPathThrowsException {
val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot,
"test", "1"))
- var zkClient = ZkUtils.createZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs,
- config.zkConnectionTimeoutMs)
+ var zkUtils = ZkUtils(zkConnectWithInvalidRoot, zkSessionTimeoutMs,
+ config.zkConnectionTimeoutMs, false)
try {
ZkPath.resetNamespaceCheckedState
- ZkUtils.createEphemeralPathExpectConflict(zkClient, path, "somedata")
+ zkUtils.createEphemeralPathExpectConflict(path, "somedata")
fail("Failed to throw ConfigException for missing zookeeper root node")
} catch {
case configException: ConfigException =>
@@ -108,26 +108,26 @@ class ZKPathTest extends ZooKeeperTestHarness {
@Test
def testCreateEphemeralPathExists {
val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1"))
- var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs)
+ var zkUtils = ZkUtils(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false)
try {
ZkPath.resetNamespaceCheckedState
- ZkUtils.createEphemeralPathExpectConflict(zkClient, path, "somedata")
+ zkUtils.createEphemeralPathExpectConflict(path, "somedata")
} catch {
case exception: Throwable => fail("Failed to create ephemeral path")
}
- assertTrue("Failed to create ephemeral path", ZkUtils.pathExists(zkClient, path))
+ assertTrue("Failed to create ephemeral path", zkUtils.pathExists(path))
}
@Test
def testCreatePersistentSequentialThrowsException {
val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnectWithInvalidRoot,
"test", "1"))
- var zkClient = ZkUtils.createZkClient(zkConnectWithInvalidRoot, zkSessionTimeoutMs,
- config.zkConnectionTimeoutMs)
+ var zkUtils = ZkUtils(zkConnectWithInvalidRoot, zkSessionTimeoutMs,
+ config.zkConnectionTimeoutMs, false)
try {
ZkPath.resetNamespaceCheckedState
- ZkUtils.createSequentialPersistentPath(zkClient, path)
+ zkUtils.createSequentialPersistentPath(path)
fail("Failed to throw ConfigException for missing zookeeper root node")
} catch {
case configException: ConfigException =>
@@ -138,16 +138,16 @@ class ZKPathTest extends ZooKeeperTestHarness {
@Test
def testCreatePersistentSequentialExists {
val config = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, "test", "1"))
- var zkClient = ZkUtils.createZkClient(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs)
+ var zkUtils = ZkUtils(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false)
var actualPath: String = ""
try {
ZkPath.resetNamespaceCheckedState
- actualPath = ZkUtils.createSequentialPersistentPath(zkClient, path)
+ actualPath = zkUtils.createSequentialPersistentPath(path)
} catch {
case exception: Throwable => fail("Failed to create persistent path")
}
- assertTrue("Failed to create persistent path", ZkUtils.pathExists(zkClient, actualPath))
+ assertTrue("Failed to create persistent path", zkUtils.pathExists(actualPath))
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala
index 3e1c6e0..38a0765 100755
--- a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala
+++ b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala
@@ -25,25 +25,21 @@ import org.scalatest.junit.JUnitSuite
trait ZooKeeperTestHarness extends JUnitSuite {
var zkPort: Int = -1
var zookeeper: EmbeddedZookeeper = null
- var zkClient: ZkClient = null
- var zkConnection : ZkConnection = null
+ var zkUtils: ZkUtils = null
val zkConnectionTimeout = 6000
val zkSessionTimeout = 6000
-
def zkConnect: String = "127.0.0.1:" + zkPort
@Before
def setUp() {
zookeeper = new EmbeddedZookeeper()
zkPort = zookeeper.port
- val (client, connection) = ZkUtils.createZkClientAndConnection(zkConnect, zkSessionTimeout, zkConnectionTimeout)
- zkClient = client
- zkConnection = connection
+ zkUtils = ZkUtils.apply(zkConnect, zkSessionTimeout, zkConnectionTimeout, false)
}
@After
def tearDown() {
- CoreUtils.swallow(zkClient.close())
+ CoreUtils.swallow(zkUtils.close())
CoreUtils.swallow(zookeeper.shutdown())
}
[2/5] kafka git commit: KAFKA-2639: Refactoring of ZkUtils
Posted by ju...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/other/kafka/TestOffsetManager.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/other/kafka/TestOffsetManager.scala b/core/src/test/scala/other/kafka/TestOffsetManager.scala
index e0e46c8..549a96b 100644
--- a/core/src/test/scala/other/kafka/TestOffsetManager.scala
+++ b/core/src/test/scala/other/kafka/TestOffsetManager.scala
@@ -63,13 +63,13 @@ object TestOffsetManager {
}
- class CommitThread(id: Int, partitionCount: Int, commitIntervalMs: Long, zkClient: ZkClient)
+ class CommitThread(id: Int, partitionCount: Int, commitIntervalMs: Long, zkUtils: ZkUtils)
extends ShutdownableThread("commit-thread")
with KafkaMetricsGroup {
private val groupId = "group-" + id
private val metadata = "Metadata from commit thread " + id
- private var offsetsChannel = ClientUtils.channelToOffsetManager(groupId, zkClient, SocketTimeoutMs)
+ private var offsetsChannel = ClientUtils.channelToOffsetManager(groupId, zkUtils, SocketTimeoutMs)
private var offset = 0L
val numErrors = new AtomicInteger(0)
val numCommits = new AtomicInteger(0)
@@ -79,7 +79,7 @@ object TestOffsetManager {
private def ensureConnected() {
if (!offsetsChannel.isConnected)
- offsetsChannel = ClientUtils.channelToOffsetManager(groupId, zkClient, SocketTimeoutMs)
+ offsetsChannel = ClientUtils.channelToOffsetManager(groupId, zkUtils, SocketTimeoutMs)
}
override def doWork() {
@@ -119,7 +119,7 @@ object TestOffsetManager {
}
}
- class FetchThread(numGroups: Int, fetchIntervalMs: Long, zkClient: ZkClient)
+ class FetchThread(numGroups: Int, fetchIntervalMs: Long, zkUtils: ZkUtils)
extends ShutdownableThread("fetch-thread")
with KafkaMetricsGroup {
@@ -127,7 +127,7 @@ object TestOffsetManager {
private val fetchTimer = new KafkaTimer(timer)
private val channels = mutable.Map[Int, BlockingChannel]()
- private var metadataChannel = ClientUtils.channelToAnyBroker(zkClient, SocketTimeoutMs)
+ private var metadataChannel = ClientUtils.channelToAnyBroker(zkUtils, SocketTimeoutMs)
private val numErrors = new AtomicInteger(0)
@@ -141,7 +141,7 @@ object TestOffsetManager {
val channel = if (channels.contains(coordinatorId))
channels(coordinatorId)
else {
- val newChannel = ClientUtils.channelToOffsetManager(group, zkClient, SocketTimeoutMs)
+ val newChannel = ClientUtils.channelToOffsetManager(group, zkUtils, SocketTimeoutMs)
channels.put(coordinatorId, newChannel)
newChannel
}
@@ -173,7 +173,7 @@ object TestOffsetManager {
println("Error while querying %s:%d - shutting down query channel.".format(metadataChannel.host, metadataChannel.port))
metadataChannel.disconnect()
println("Creating new query channel.")
- metadataChannel = ClientUtils.channelToAnyBroker(zkClient, SocketTimeoutMs)
+ metadataChannel = ClientUtils.channelToAnyBroker(zkUtils, SocketTimeoutMs)
}
finally {
Thread.sleep(fetchIntervalMs)
@@ -250,17 +250,17 @@ object TestOffsetManager {
println("Commit thread count: %d; Partition count: %d, Commit interval: %d ms; Fetch interval: %d ms; Reporting interval: %d ms"
.format(threadCount, partitionCount, commitIntervalMs, fetchIntervalMs, reportingIntervalMs))
- var zkClient: ZkClient = null
+ var zkUtils: ZkUtils = null
var commitThreads: Seq[CommitThread] = Seq()
var fetchThread: FetchThread = null
var statsThread: StatsThread = null
try {
- zkClient = ZkUtils.createZkClient(zookeeper, 6000, 2000)
+ zkUtils = ZkUtils(zookeeper, 6000, 2000, false)
commitThreads = (0 to (threadCount-1)).map { threadId =>
- new CommitThread(threadId, partitionCount, commitIntervalMs, zkClient)
+ new CommitThread(threadId, partitionCount, commitIntervalMs, zkUtils)
}
- fetchThread = new FetchThread(threadCount, fetchIntervalMs, zkClient)
+ fetchThread = new FetchThread(threadCount, fetchIntervalMs, zkUtils)
val statsThread = new StatsThread(reportingIntervalMs, commitThreads, fetchThread)
@@ -300,7 +300,7 @@ object TestOffsetManager {
statsThread.shutdown()
statsThread.join()
}
- zkClient.close()
+ zkUtils.close()
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala
index ed94039..0fce611 100755
--- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala
@@ -49,10 +49,10 @@ class AddPartitionsTest extends ZooKeeperTestHarness {
brokers = servers.map(s => new Broker(s.config.brokerId, s.config.hostName, s.boundPort()))
// create topics first
- createTopic(zkClient, topic1, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers)
- createTopic(zkClient, topic2, partitionReplicaAssignment = Map(0->Seq(1,2)), servers = servers)
- createTopic(zkClient, topic3, partitionReplicaAssignment = Map(0->Seq(2,3,0,1)), servers = servers)
- createTopic(zkClient, topic4, partitionReplicaAssignment = Map(0->Seq(0,3)), servers = servers)
+ createTopic(zkUtils, topic1, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers)
+ createTopic(zkUtils, topic2, partitionReplicaAssignment = Map(0->Seq(1,2)), servers = servers)
+ createTopic(zkUtils, topic3, partitionReplicaAssignment = Map(0->Seq(2,3,0,1)), servers = servers)
+ createTopic(zkUtils, topic4, partitionReplicaAssignment = Map(0->Seq(0,3)), servers = servers)
}
@After
@@ -65,7 +65,7 @@ class AddPartitionsTest extends ZooKeeperTestHarness {
@Test
def testTopicDoesNotExist {
try {
- AdminUtils.addPartitions(zkClient, "Blah", 1)
+ AdminUtils.addPartitions(zkUtils, "Blah", 1)
fail("Topic should not exist")
} catch {
case e: AdminOperationException => //this is good
@@ -76,7 +76,7 @@ class AddPartitionsTest extends ZooKeeperTestHarness {
@Test
def testWrongReplicaCount {
try {
- AdminUtils.addPartitions(zkClient, topic1, 2, "0:1,0:1:2")
+ AdminUtils.addPartitions(zkUtils, topic1, 2, "0:1,0:1:2")
fail("Add partitions should fail")
} catch {
case e: AdminOperationException => //this is good
@@ -86,12 +86,12 @@ class AddPartitionsTest extends ZooKeeperTestHarness {
@Test
def testIncrementPartitions {
- AdminUtils.addPartitions(zkClient, topic1, 3)
+ AdminUtils.addPartitions(zkUtils, topic1, 3)
// wait until leader is elected
- var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 1)
- var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 2)
- val leader1FromZk = ZkUtils.getLeaderForPartition(zkClient, topic1, 1).get
- val leader2FromZk = ZkUtils.getLeaderForPartition(zkClient, topic1, 2).get
+ var leader1 = waitUntilLeaderIsElectedOrChanged(zkUtils, topic1, 1)
+ var leader2 = waitUntilLeaderIsElectedOrChanged(zkUtils, topic1, 2)
+ val leader1FromZk = zkUtils.getLeaderForPartition(topic1, 1).get
+ val leader2FromZk = zkUtils.getLeaderForPartition(topic1, 2).get
assertEquals(leader1.get, leader1FromZk)
assertEquals(leader2.get, leader2FromZk)
@@ -112,12 +112,12 @@ class AddPartitionsTest extends ZooKeeperTestHarness {
@Test
def testManualAssignmentOfReplicas {
- AdminUtils.addPartitions(zkClient, topic2, 3, "1:2,0:1,2:3")
+ AdminUtils.addPartitions(zkUtils, topic2, 3, "1:2,0:1,2:3")
// wait until leader is elected
- var leader1 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 1)
- var leader2 = waitUntilLeaderIsElectedOrChanged(zkClient, topic2, 2)
- val leader1FromZk = ZkUtils.getLeaderForPartition(zkClient, topic2, 1).get
- val leader2FromZk = ZkUtils.getLeaderForPartition(zkClient, topic2, 2).get
+ var leader1 = waitUntilLeaderIsElectedOrChanged(zkUtils, topic2, 1)
+ var leader2 = waitUntilLeaderIsElectedOrChanged(zkUtils, topic2, 2)
+ val leader1FromZk = zkUtils.getLeaderForPartition(topic2, 1).get
+ val leader2FromZk = zkUtils.getLeaderForPartition(topic2, 2).get
assertEquals(leader1.get, leader1FromZk)
assertEquals(leader2.get, leader2FromZk)
@@ -139,7 +139,7 @@ class AddPartitionsTest extends ZooKeeperTestHarness {
@Test
def testReplicaPlacement {
- AdminUtils.addPartitions(zkClient, topic3, 7)
+ AdminUtils.addPartitions(zkUtils, topic3, 7)
// read metadata from a broker and verify the new topic partitions exist
TestUtils.waitUntilMetadataIsPropagated(servers, topic3, 1)
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/admin/AdminTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala
index 2d18069..52ea580 100755
--- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala
@@ -66,23 +66,23 @@ class AdminTest extends ZooKeeperTestHarness with Logging {
@Test
def testManualReplicaAssignment() {
val brokers = List(0, 1, 2, 3, 4)
- TestUtils.createBrokersInZk(zkClient, zkConnection, brokers)
+ TestUtils.createBrokersInZk(zkUtils, brokers)
// duplicate brokers
intercept[IllegalArgumentException] {
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", Map(0->Seq(0,0)))
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, "test", Map(0->Seq(0,0)))
}
// inconsistent replication factor
intercept[IllegalArgumentException] {
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", Map(0->Seq(0,1), 1->Seq(0)))
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, "test", Map(0->Seq(0,1), 1->Seq(0)))
}
// good assignment
val assignment = Map(0 -> List(0, 1, 2),
1 -> List(1, 2, 3))
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, "test", assignment)
- val found = ZkUtils.getPartitionAssignmentForTopics(zkClient, Seq("test"))
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, "test", assignment)
+ val found = zkUtils.getPartitionAssignmentForTopics(Seq("test"))
assertEquals(assignment, found("test"))
}
@@ -117,19 +117,19 @@ class AdminTest extends ZooKeeperTestHarness with Logging {
11 -> 1
)
val topic = "test"
- TestUtils.createBrokersInZk(zkClient, zkConnection, List(0, 1, 2, 3, 4))
+ TestUtils.createBrokersInZk(zkUtils, List(0, 1, 2, 3, 4))
// create the topic
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment)
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, expectedReplicaAssignment)
// create leaders for all partitions
- TestUtils.makeLeaderForPartition(zkClient, topic, leaderForPartitionMap, 1)
- val actualReplicaList = leaderForPartitionMap.keys.toArray.map(p => (p -> ZkUtils.getReplicasForPartition(zkClient, topic, p))).toMap
+ TestUtils.makeLeaderForPartition(zkUtils, topic, leaderForPartitionMap, 1)
+ val actualReplicaList = leaderForPartitionMap.keys.toArray.map(p => (p -> zkUtils.getReplicasForPartition(topic, p))).toMap
assertEquals(expectedReplicaAssignment.size, actualReplicaList.size)
for(i <- 0 until actualReplicaList.size)
assertEquals(expectedReplicaAssignment.get(i).get, actualReplicaList(i))
intercept[TopicExistsException] {
// shouldn't be able to create a topic that already exists
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment)
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, expectedReplicaAssignment)
}
}
@@ -137,13 +137,13 @@ class AdminTest extends ZooKeeperTestHarness with Logging {
def testTopicCreationWithCollision() {
val topic = "test.topic"
val collidingTopic = "test_topic"
- TestUtils.createBrokersInZk(zkClient, zkConnection, List(0, 1, 2, 3, 4))
+ TestUtils.createBrokersInZk(zkUtils, List(0, 1, 2, 3, 4))
// create the topic
- AdminUtils.createTopic(zkClient, topic, 3, 1)
+ AdminUtils.createTopic(zkUtils, topic, 3, 1)
intercept[InvalidTopicException] {
// shouldn't be able to create a topic that collides
- AdminUtils.createTopic(zkClient, collidingTopic, 3, 1)
+ AdminUtils.createTopic(zkUtils, collidingTopic, 3, 1)
}
}
@@ -160,25 +160,25 @@ class AdminTest extends ZooKeeperTestHarness with Logging {
// create brokers
val servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b)))
// create the topic
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment)
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, expectedReplicaAssignment)
// reassign partition 0
val newReplicas = Seq(0, 2, 3)
val partitionToBeReassigned = 0
val topicAndPartition = TopicAndPartition(topic, partitionToBeReassigned)
- val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas))
+ val reassignPartitionsCommand = new ReassignPartitionsCommand(zkUtils, Map(topicAndPartition -> newReplicas))
assertTrue("Partition reassignment attempt failed for [test, 0]", reassignPartitionsCommand.reassignPartitions())
// wait until reassignment is completed
TestUtils.waitUntilTrue(() => {
- val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas);
- ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition, newReplicas,
+ val partitionsBeingReassigned = zkUtils.getPartitionsBeingReassigned().mapValues(_.newReplicas);
+ ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkUtils, topicAndPartition, newReplicas,
Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentCompleted;
},
"Partition reassignment should complete")
- val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned)
+ val assignedReplicas = zkUtils.getReplicasForPartition(topic, partitionToBeReassigned)
// in sync replicas should not have any replica that is not in the new assigned replicas
- checkForPhantomInSyncReplicas(zkClient, topic, partitionToBeReassigned, assignedReplicas)
+ checkForPhantomInSyncReplicas(zkUtils, topic, partitionToBeReassigned, assignedReplicas)
assertEquals("Partition should have been reassigned to 0, 2, 3", newReplicas, assignedReplicas)
- ensureNoUnderReplicatedPartitions(zkClient, topic, partitionToBeReassigned, assignedReplicas, servers)
+ ensureNoUnderReplicatedPartitions(zkUtils, topic, partitionToBeReassigned, assignedReplicas, servers)
TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet,
"New replicas should exist on brokers")
servers.foreach(_.shutdown())
@@ -191,24 +191,24 @@ class AdminTest extends ZooKeeperTestHarness with Logging {
// create brokers
val servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b)))
// create the topic
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment)
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, expectedReplicaAssignment)
// reassign partition 0
val newReplicas = Seq(1, 2, 3)
val partitionToBeReassigned = 0
val topicAndPartition = TopicAndPartition(topic, partitionToBeReassigned)
- val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas))
+ val reassignPartitionsCommand = new ReassignPartitionsCommand(zkUtils, Map(topicAndPartition -> newReplicas))
assertTrue("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions())
// wait until reassignment is completed
TestUtils.waitUntilTrue(() => {
- val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas);
- ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition, newReplicas,
+ val partitionsBeingReassigned = zkUtils.getPartitionsBeingReassigned().mapValues(_.newReplicas);
+ ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkUtils, topicAndPartition, newReplicas,
Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentCompleted;
},
"Partition reassignment should complete")
- val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned)
+ val assignedReplicas = zkUtils.getReplicasForPartition(topic, partitionToBeReassigned)
assertEquals("Partition should have been reassigned to 0, 2, 3", newReplicas, assignedReplicas)
- checkForPhantomInSyncReplicas(zkClient, topic, partitionToBeReassigned, assignedReplicas)
- ensureNoUnderReplicatedPartitions(zkClient, topic, partitionToBeReassigned, assignedReplicas, servers)
+ checkForPhantomInSyncReplicas(zkUtils, topic, partitionToBeReassigned, assignedReplicas)
+ ensureNoUnderReplicatedPartitions(zkUtils, topic, partitionToBeReassigned, assignedReplicas, servers)
TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet,
"New replicas should exist on brokers")
@@ -222,24 +222,24 @@ class AdminTest extends ZooKeeperTestHarness with Logging {
// create brokers
val servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b)))
// create the topic
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment)
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, expectedReplicaAssignment)
// reassign partition 0
val newReplicas = Seq(2, 3)
val partitionToBeReassigned = 0
val topicAndPartition = TopicAndPartition(topic, partitionToBeReassigned)
- val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas))
+ val reassignPartitionsCommand = new ReassignPartitionsCommand(zkUtils, Map(topicAndPartition -> newReplicas))
assertTrue("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions())
// wait until reassignment is completed
TestUtils.waitUntilTrue(() => {
- val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas);
- ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition, newReplicas,
+ val partitionsBeingReassigned = zkUtils.getPartitionsBeingReassigned().mapValues(_.newReplicas);
+ ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkUtils, topicAndPartition, newReplicas,
Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentCompleted;
},
"Partition reassignment should complete")
- val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned)
+ val assignedReplicas = zkUtils.getReplicasForPartition(topic, partitionToBeReassigned)
assertEquals("Partition should have been reassigned to 2, 3", newReplicas, assignedReplicas)
- checkForPhantomInSyncReplicas(zkClient, topic, partitionToBeReassigned, assignedReplicas)
- ensureNoUnderReplicatedPartitions(zkClient, topic, partitionToBeReassigned, assignedReplicas, servers)
+ checkForPhantomInSyncReplicas(zkUtils, topic, partitionToBeReassigned, assignedReplicas)
+ ensureNoUnderReplicatedPartitions(zkUtils, topic, partitionToBeReassigned, assignedReplicas, servers)
TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet,
"New replicas should exist on brokers")
servers.foreach(_.shutdown())
@@ -254,9 +254,9 @@ class AdminTest extends ZooKeeperTestHarness with Logging {
val newReplicas = Seq(2, 3)
val partitionToBeReassigned = 0
val topicAndPartition = TopicAndPartition(topic, partitionToBeReassigned)
- val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas))
+ val reassignPartitionsCommand = new ReassignPartitionsCommand(zkUtils, Map(topicAndPartition -> newReplicas))
assertFalse("Partition reassignment failed for test, 0", reassignPartitionsCommand.reassignPartitions())
- val reassignedPartitions = ZkUtils.getPartitionsBeingReassigned(zkClient)
+ val reassignedPartitions = zkUtils.getPartitionsBeingReassigned()
assertFalse("Partition should not be reassigned", reassignedPartitions.contains(topicAndPartition))
servers.foreach(_.shutdown())
}
@@ -266,25 +266,25 @@ class AdminTest extends ZooKeeperTestHarness with Logging {
val expectedReplicaAssignment = Map(0 -> List(0, 1))
val topic = "test"
// create the topic
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment)
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, expectedReplicaAssignment)
// put the partition in the reassigned path as well
// reassign partition 0
val newReplicas = Seq(0, 1)
val partitionToBeReassigned = 0
val topicAndPartition = TopicAndPartition(topic, partitionToBeReassigned)
- val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas))
+ val reassignPartitionsCommand = new ReassignPartitionsCommand(zkUtils, Map(topicAndPartition -> newReplicas))
reassignPartitionsCommand.reassignPartitions
// create brokers
val servers = TestUtils.createBrokerConfigs(2, zkConnect, false).map(b => TestUtils.createServer(KafkaConfig.fromProps(b)))
// wait until reassignment completes
- TestUtils.waitUntilTrue(() => !checkIfReassignPartitionPathExists(zkClient),
+ TestUtils.waitUntilTrue(() => !checkIfReassignPartitionPathExists(zkUtils),
"Partition reassignment should complete")
- val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, partitionToBeReassigned)
+ val assignedReplicas = zkUtils.getReplicasForPartition(topic, partitionToBeReassigned)
assertEquals("Partition should have been reassigned to 0, 1", newReplicas, assignedReplicas)
- checkForPhantomInSyncReplicas(zkClient, topic, partitionToBeReassigned, assignedReplicas)
+ checkForPhantomInSyncReplicas(zkUtils, topic, partitionToBeReassigned, assignedReplicas)
// ensure that there are no under replicated partitions
- ensureNoUnderReplicatedPartitions(zkClient, topic, partitionToBeReassigned, assignedReplicas, servers)
+ ensureNoUnderReplicatedPartitions(zkUtils, topic, partitionToBeReassigned, assignedReplicas, servers)
TestUtils.waitUntilTrue(() => getBrokersWithPartitionDir(servers, topic, 0) == newReplicas.toSet,
"New replicas should exist on brokers")
servers.foreach(_.shutdown())
@@ -294,10 +294,9 @@ class AdminTest extends ZooKeeperTestHarness with Logging {
def testPreferredReplicaJsonData() {
// write preferred replica json data to zk path
val partitionsForPreferredReplicaElection = Set(TopicAndPartition("test", 1), TopicAndPartition("test2", 1))
- PreferredReplicaLeaderElectionCommand.writePreferredReplicaElectionData(zkClient, partitionsForPreferredReplicaElection)
+ PreferredReplicaLeaderElectionCommand.writePreferredReplicaElectionData(zkUtils, partitionsForPreferredReplicaElection)
// try to read it back and compare with what was written
- val preferredReplicaElectionZkData = ZkUtils.readData(zkClient,
- ZkUtils.PreferredReplicaLeaderElectionPath)._1
+ val preferredReplicaElectionZkData = zkUtils.readData(ZkUtils.PreferredReplicaLeaderElectionPath)._1
val partitionsUndergoingPreferredReplicaElection =
PreferredReplicaLeaderElectionCommand.parsePreferredReplicaElectionData(preferredReplicaElectionZkData)
assertEquals("Preferred replica election ser-de failed", partitionsForPreferredReplicaElection,
@@ -313,14 +312,14 @@ class AdminTest extends ZooKeeperTestHarness with Logging {
// create brokers
val serverConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false).map(KafkaConfig.fromProps)
// create the topic
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment)
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, expectedReplicaAssignment)
val servers = serverConfigs.reverseMap(s => TestUtils.createServer(s))
// broker 2 should be the leader since it was started first
- val currentLeader = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, partition, oldLeaderOpt = None).get
+ val currentLeader = TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partition, oldLeaderOpt = None).get
// trigger preferred replica election
- val preferredReplicaElection = new PreferredReplicaLeaderElectionCommand(zkClient, Set(TopicAndPartition(topic, partition)))
+ val preferredReplicaElection = new PreferredReplicaLeaderElectionCommand(zkUtils, Set(TopicAndPartition(topic, partition)))
preferredReplicaElection.moveLeaderToPreferredReplica()
- val newLeader = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, partition, oldLeaderOpt = Some(currentLeader)).get
+ val newLeader = TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partition, oldLeaderOpt = Some(currentLeader)).get
assertEquals("Preferred replica election failed", preferredReplica, newLeader)
servers.foreach(_.shutdown())
}
@@ -334,9 +333,9 @@ class AdminTest extends ZooKeeperTestHarness with Logging {
val serverConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false).map(KafkaConfig.fromProps)
val servers = serverConfigs.reverseMap(s => TestUtils.createServer(s))
// create the topic
- TestUtils.createTopic(zkClient, topic, partitionReplicaAssignment = expectedReplicaAssignment, servers = servers)
+ TestUtils.createTopic(zkUtils, topic, partitionReplicaAssignment = expectedReplicaAssignment, servers = servers)
- val controllerId = ZkUtils.getController(zkClient)
+ val controllerId = zkUtils.getController()
val controller = servers.find(p => p.config.brokerId == controllerId).get.kafkaController
var partitionsRemaining = controller.shutdownBroker(2)
var activeServers = servers.filter(s => s.config.brokerId != 2)
@@ -402,16 +401,16 @@ class AdminTest extends ZooKeeperTestHarness with Logging {
// create a topic with a few config overrides and check that they are applied
val maxMessageSize = 1024
val retentionMs = 1000*1000
- AdminUtils.createTopic(server.zkClient, topic, partitions, 1, makeConfig(maxMessageSize, retentionMs))
+ AdminUtils.createTopic(server.zkUtils, topic, partitions, 1, makeConfig(maxMessageSize, retentionMs))
checkConfig(maxMessageSize, retentionMs)
// now double the config values for the topic and check that it is applied
val newConfig: Properties = makeConfig(2*maxMessageSize, 2 * retentionMs)
- AdminUtils.changeTopicConfig(server.zkClient, topic, makeConfig(2*maxMessageSize, 2 * retentionMs))
+ AdminUtils.changeTopicConfig(server.zkUtils, topic, makeConfig(2*maxMessageSize, 2 * retentionMs))
checkConfig(2*maxMessageSize, 2 * retentionMs)
// Verify that the same config can be read from ZK
- val configInZk = AdminUtils.fetchEntityConfig(server.zkClient, ConfigType.Topic, topic)
+ val configInZk = AdminUtils.fetchEntityConfig(server.zkUtils, ConfigType.Topic, topic)
assertEquals(newConfig, configInZk)
} finally {
server.shutdown()
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala
index d3abf08..c19127e 100644
--- a/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/DeleteConsumerGroupTest.scala
@@ -33,11 +33,11 @@ class DeleteConsumerGroupTest extends KafkaServerTestHarness {
val groupToDelete = "groupToDelete"
val otherGroup = "otherGroup"
- TestUtils.createTopic(zkClient, topic, 1, 3, servers)
+ TestUtils.createTopic(zkUtils, topic, 1, 3, servers)
fillInConsumerGroupInfo(topic, groupToDelete, "consumer", 0, 10, false)
fillInConsumerGroupInfo(topic, otherGroup, "consumer", 0, 10, false)
- AdminUtils.deleteConsumerGroupInZK(zkClient, groupToDelete)
+ AdminUtils.deleteConsumerGroupInZK(zkUtils, groupToDelete)
TestUtils.waitUntilTrue(() => !groupDirExists(new ZKGroupDirs(groupToDelete)),
"DeleteConsumerGroupInZK should delete the provided consumer group's directory")
@@ -51,11 +51,11 @@ class DeleteConsumerGroupTest extends KafkaServerTestHarness {
val groupToDelete = "groupToDelete"
val otherGroup = "otherGroup"
- TestUtils.createTopic(zkClient, topic, 1, 3, servers)
+ TestUtils.createTopic(zkUtils, topic, 1, 3, servers)
fillInConsumerGroupInfo(topic, groupToDelete, "consumer", 0, 10, true)
fillInConsumerGroupInfo(topic, otherGroup, "consumer", 0, 10, false)
- AdminUtils.deleteConsumerGroupInZK(zkClient, groupToDelete)
+ AdminUtils.deleteConsumerGroupInZK(zkUtils, groupToDelete)
TestUtils.waitUntilTrue(() => groupDirExists(new ZKGroupDirs(groupToDelete)),
"DeleteConsumerGroupInZK should not delete the provided consumer group's directory if the consumer group is still active")
@@ -68,11 +68,11 @@ class DeleteConsumerGroupTest extends KafkaServerTestHarness {
val topic = "test"
val groupToDelete = "groupToDelete"
val otherGroup = "otherGroup"
- TestUtils.createTopic(zkClient, topic, 1, 3, servers)
+ TestUtils.createTopic(zkUtils, topic, 1, 3, servers)
fillInConsumerGroupInfo(topic, groupToDelete, "consumer", 0, 10, false)
fillInConsumerGroupInfo(topic, otherGroup, "consumer", 0, 10, false)
- AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkClient, groupToDelete, topic)
+ AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkUtils, groupToDelete, topic)
TestUtils.waitUntilTrue(() => !groupDirExists(new ZKGroupDirs(groupToDelete)),
"DeleteConsumerGroupInfoForTopicInZK should delete the provided consumer group's directory if it just consumes from one topic")
@@ -86,14 +86,14 @@ class DeleteConsumerGroupTest extends KafkaServerTestHarness {
val otherTopic = "otherTopic"
val groupToDelete = "groupToDelete"
val otherGroup = "otherGroup"
- TestUtils.createTopic(zkClient, topicToDelete, 1, 3, servers)
- TestUtils.createTopic(zkClient, otherTopic, 1, 3, servers)
+ TestUtils.createTopic(zkUtils, topicToDelete, 1, 3, servers)
+ TestUtils.createTopic(zkUtils, otherTopic, 1, 3, servers)
fillInConsumerGroupInfo(topicToDelete, groupToDelete, "consumer", 0, 10, false)
fillInConsumerGroupInfo(otherTopic, groupToDelete, "consumer", 0, 10, false)
fillInConsumerGroupInfo(topicToDelete, otherGroup, "consumer", 0, 10, false)
- AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkClient, groupToDelete, topicToDelete)
+ AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkUtils, groupToDelete, topicToDelete)
TestUtils.waitUntilTrue(() => !groupTopicOffsetAndOwnerDirsExist(new ZKGroupTopicDirs(groupToDelete, topicToDelete)),
"DeleteConsumerGroupInfoForTopicInZK should delete the provided consumer group's owner and offset directories for the given topic")
@@ -108,13 +108,13 @@ class DeleteConsumerGroupTest extends KafkaServerTestHarness {
val topicToDelete = "topicToDelete"
val otherTopic = "otherTopic"
val group = "group"
- TestUtils.createTopic(zkClient, topicToDelete, 1, 3, servers)
- TestUtils.createTopic(zkClient, otherTopic, 1, 3, servers)
+ TestUtils.createTopic(zkUtils, topicToDelete, 1, 3, servers)
+ TestUtils.createTopic(zkUtils, otherTopic, 1, 3, servers)
fillInConsumerGroupInfo(topicToDelete, group, "consumer", 0, 10, true)
fillInConsumerGroupInfo(otherTopic, group, "consumer", 0, 10, true)
- AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkClient, group, topicToDelete)
+ AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkUtils, group, topicToDelete)
TestUtils.waitUntilTrue(() => groupTopicOffsetAndOwnerDirsExist(new ZKGroupTopicDirs(group, topicToDelete)),
"DeleteConsumerGroupInfoForTopicInZK should not delete the provided consumer group's owner and offset directories for the given topic if the consumer group is still active")
@@ -128,14 +128,14 @@ class DeleteConsumerGroupTest extends KafkaServerTestHarness {
val otherTopic = "otherTopic"
val groups = Seq("group1", "group2")
- TestUtils.createTopic(zkClient, topicToDelete, 1, 3, servers)
- TestUtils.createTopic(zkClient, otherTopic, 1, 3, servers)
+ TestUtils.createTopic(zkUtils, topicToDelete, 1, 3, servers)
+ TestUtils.createTopic(zkUtils, otherTopic, 1, 3, servers)
val groupTopicDirsForTopicToDelete = groups.map(group => new ZKGroupTopicDirs(group, topicToDelete))
val groupTopicDirsForOtherTopic = groups.map(group => new ZKGroupTopicDirs(group, otherTopic))
groupTopicDirsForTopicToDelete.foreach(dir => fillInConsumerGroupInfo(topicToDelete, dir.group, "consumer", 0, 10, false))
groupTopicDirsForOtherTopic.foreach(dir => fillInConsumerGroupInfo(otherTopic, dir.group, "consumer", 0, 10, false))
- AdminUtils.deleteAllConsumerGroupInfoForTopicInZK(zkClient, topicToDelete)
+ AdminUtils.deleteAllConsumerGroupInfoForTopicInZK(zkUtils, topicToDelete)
TestUtils.waitUntilTrue(() => !groupTopicDirsForTopicToDelete.exists(groupTopicOffsetAndOwnerDirsExist),
"Consumer group info on deleted topic topic should be deleted by DeleteAllConsumerGroupInfoForTopicInZK")
@@ -148,13 +148,13 @@ class DeleteConsumerGroupTest extends KafkaServerTestHarness {
val topic = "topic"
val group = "group"
- TestUtils.createTopic(zkClient, topic, 1, 3, servers)
+ TestUtils.createTopic(zkUtils, topic, 1, 3, servers)
val dir = new ZKGroupTopicDirs(group, topic)
fillInConsumerGroupInfo(topic, dir.group, "consumer", 0, 10, false)
- AdminUtils.deleteTopic(zkClient, topic)
- TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers)
- AdminUtils.deleteAllConsumerGroupInfoForTopicInZK(zkClient, topic)
+ AdminUtils.deleteTopic(zkUtils, topic)
+ TestUtils.verifyTopicDeletion(zkUtils, topic, 1, servers)
+ AdminUtils.deleteAllConsumerGroupInfoForTopicInZK(zkUtils, topic)
TestUtils.waitUntilTrue(() => !groupDirExists(dir),
"Consumer group info on related topics should be deleted by DeleteAllConsumerGroupInfoForTopicInZK")
@@ -185,19 +185,19 @@ class DeleteConsumerGroupTest extends KafkaServerTestHarness {
val consumerConfig = new ConsumerConfig(consumerProps)
val dir = new ZKGroupTopicDirs(group, topic)
TestUtils.updateConsumerOffset(consumerConfig, dir.consumerOffsetDir + "/" + partition, offset)
- ZkUtils.createEphemeralPathExpectConflict(zkClient, ZkUtils.getConsumerPartitionOwnerPath(group, topic, partition), "")
- ZkUtils.makeSurePersistentPathExists(zkClient, dir.consumerRegistryDir)
+ zkUtils.createEphemeralPathExpectConflict(zkUtils.getConsumerPartitionOwnerPath(group, topic, partition), "")
+ zkUtils.makeSurePersistentPathExists(dir.consumerRegistryDir)
if (registerConsumer) {
- ZkUtils.createEphemeralPathExpectConflict(zkClient, dir.consumerRegistryDir + "/" + consumerId, "")
+ zkUtils.createEphemeralPathExpectConflict(dir.consumerRegistryDir + "/" + consumerId, "")
}
}
private def groupDirExists(dir: ZKGroupDirs) = {
- ZkUtils.pathExists(zkClient, dir.consumerGroupDir)
+ zkUtils.pathExists(dir.consumerGroupDir)
}
private def groupTopicOffsetAndOwnerDirsExist(dir: ZKGroupTopicDirs) = {
- ZkUtils.pathExists(zkClient, dir.consumerOffsetDir) && ZkUtils.pathExists(zkClient, dir.consumerOwnerDir)
+ zkUtils.pathExists(dir.consumerOffsetDir) && zkUtils.pathExists(dir.consumerOwnerDir)
}
private def produceEvents(producer: KafkaProducer[Array[Byte], Array[Byte]], topic: String, messages: List[String]) {
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
index fbae398..383cb44 100644
--- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
@@ -20,6 +20,7 @@ import kafka.log.Log
import kafka.zk.ZooKeeperTestHarness
import junit.framework.Assert._
import kafka.utils.{ZkUtils, TestUtils}
+import kafka.utils.ZkUtils._
import kafka.server.{KafkaServer, KafkaConfig}
import org.junit.Test
import java.util.Properties
@@ -33,8 +34,8 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
val topic = topicAndPartition.topic
val servers = createTestTopicAndCluster(topic)
// start topic deletion
- AdminUtils.deleteTopic(zkClient, topic)
- TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers)
+ AdminUtils.deleteTopic(zkUtils, topic)
+ TestUtils.verifyTopicDeletion(zkUtils, topic, 1, servers)
servers.foreach(_.shutdown())
}
@@ -44,22 +45,22 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
val topic = topicAndPartition.topic
val servers = createTestTopicAndCluster(topic)
// shut down one follower replica
- val leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0)
+ val leaderIdOpt = zkUtils.getLeaderForPartition(topic, 0)
assertTrue("Leader should exist for partition [test,0]", leaderIdOpt.isDefined)
val follower = servers.filter(s => s.config.brokerId != leaderIdOpt.get).last
follower.shutdown()
// start topic deletion
- AdminUtils.deleteTopic(zkClient, topic)
+ AdminUtils.deleteTopic(zkUtils, topic)
// check if all replicas but the one that is shut down has deleted the log
TestUtils.waitUntilTrue(() =>
servers.filter(s => s.config.brokerId != follower.config.brokerId)
.forall(_.getLogManager().getLog(topicAndPartition).isEmpty), "Replicas 0,1 have not deleted log.")
// ensure topic deletion is halted
- TestUtils.waitUntilTrue(() => ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)),
+ TestUtils.waitUntilTrue(() => zkUtils.pathExists(getDeleteTopicPath(topic)),
"Admin path /admin/delete_topic/test path deleted even when a follower replica is down")
// restart follower replica
follower.startup()
- TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers)
+ TestUtils.verifyTopicDeletion(zkUtils, topic, 1, servers)
servers.foreach(_.shutdown())
}
@@ -68,25 +69,25 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
val topicAndPartition = TopicAndPartition("test", 0)
val topic = topicAndPartition.topic
val servers = createTestTopicAndCluster(topic)
- val controllerId = ZkUtils.getController(zkClient)
+ val controllerId = zkUtils.getController()
val controller = servers.filter(s => s.config.brokerId == controllerId).head
- val leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0)
+ val leaderIdOpt = zkUtils.getLeaderForPartition(topic, 0)
val follower = servers.filter(s => s.config.brokerId != leaderIdOpt.get && s.config.brokerId != controllerId).last
follower.shutdown()
// start topic deletion
- AdminUtils.deleteTopic(zkClient, topic)
+ AdminUtils.deleteTopic(zkUtils, topic)
// shut down the controller to trigger controller failover during delete topic
controller.shutdown()
// ensure topic deletion is halted
- TestUtils.waitUntilTrue(() => ZkUtils.pathExists(zkClient, ZkUtils.getDeleteTopicPath(topic)),
+ TestUtils.waitUntilTrue(() => zkUtils.pathExists(getDeleteTopicPath(topic)),
"Admin path /admin/delete_topic/test path deleted even when a replica is down")
controller.startup()
follower.startup()
- TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers)
+ TestUtils.verifyTopicDeletion(zkUtils, topic, 1, servers)
servers.foreach(_.shutdown())
}
@@ -101,37 +102,37 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
val allServers = brokerConfigs.map(b => TestUtils.createServer(KafkaConfig.fromProps(b)))
val servers = allServers.filter(s => expectedReplicaAssignment(0).contains(s.config.brokerId))
// create the topic
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment)
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, expectedReplicaAssignment)
// wait until replica log is created on every broker
TestUtils.waitUntilTrue(() => servers.forall(_.getLogManager().getLog(topicAndPartition).isDefined),
"Replicas for topic test not created.")
- val leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0)
+ val leaderIdOpt = zkUtils.getLeaderForPartition(topic, 0)
assertTrue("Leader should exist for partition [test,0]", leaderIdOpt.isDefined)
val follower = servers.filter(s => s.config.brokerId != leaderIdOpt.get).last
follower.shutdown()
// start topic deletion
- AdminUtils.deleteTopic(zkClient, topic)
+ AdminUtils.deleteTopic(zkUtils, topic)
// start partition reassignment at the same time right after delete topic. In this case, reassignment will fail since
// the topic is being deleted
// reassign partition 0
- val oldAssignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, 0)
+ val oldAssignedReplicas = zkUtils.getReplicasForPartition(topic, 0)
val newReplicas = Seq(1, 2, 3)
- val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, Map(topicAndPartition -> newReplicas))
+ val reassignPartitionsCommand = new ReassignPartitionsCommand(zkUtils, Map(topicAndPartition -> newReplicas))
assertTrue("Partition reassignment should fail for [test,0]", reassignPartitionsCommand.reassignPartitions())
// wait until reassignment is completed
TestUtils.waitUntilTrue(() => {
- val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas);
- ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition, newReplicas,
+ val partitionsBeingReassigned = zkUtils.getPartitionsBeingReassigned().mapValues(_.newReplicas);
+ ReassignPartitionsCommand.checkIfPartitionReassignmentSucceeded(zkUtils, topicAndPartition, newReplicas,
Map(topicAndPartition -> newReplicas), partitionsBeingReassigned) == ReassignmentFailed;
}, "Partition reassignment shouldn't complete.")
- val controllerId = ZkUtils.getController(zkClient)
+ val controllerId = zkUtils.getController()
val controller = servers.filter(s => s.config.brokerId == controllerId).head
assertFalse("Partition reassignment should fail",
controller.kafkaController.controllerContext.partitionsBeingReassigned.contains(topicAndPartition))
- val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topic, 0)
+ val assignedReplicas = zkUtils.getReplicasForPartition(topic, 0)
assertEquals("Partition should not be reassigned to 0, 1, 2", oldAssignedReplicas, assignedReplicas)
follower.startup()
- TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers)
+ TestUtils.verifyTopicDeletion(zkUtils, topic, 1, servers)
allServers.foreach(_.shutdown())
}
@@ -139,18 +140,18 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
def testDeleteTopicDuringAddPartition() {
val topic = "test"
val servers = createTestTopicAndCluster(topic)
- val leaderIdOpt = ZkUtils.getLeaderForPartition(zkClient, topic, 0)
+ val leaderIdOpt = zkUtils.getLeaderForPartition(topic, 0)
assertTrue("Leader should exist for partition [test,0]", leaderIdOpt.isDefined)
val follower = servers.filter(s => s.config.brokerId != leaderIdOpt.get).last
val newPartition = TopicAndPartition(topic, 1)
follower.shutdown()
// add partitions to topic
- AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2", false)
+ AdminUtils.addPartitions(zkUtils, topic, 2, "0:1:2,0:1:2", false)
// start topic deletion
- AdminUtils.deleteTopic(zkClient, topic)
+ AdminUtils.deleteTopic(zkUtils, topic)
follower.startup()
// test if topic deletion is resumed
- TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers)
+ TestUtils.verifyTopicDeletion(zkUtils, topic, 1, servers)
// verify that new partition doesn't exist on any broker either
TestUtils.waitUntilTrue(() =>
servers.forall(_.getLogManager().getLog(newPartition).isEmpty),
@@ -164,11 +165,11 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
val topicAndPartition = TopicAndPartition(topic, 0)
val servers = createTestTopicAndCluster(topic)
// start topic deletion
- AdminUtils.deleteTopic(zkClient, topic)
+ AdminUtils.deleteTopic(zkUtils, topic)
// add partitions to topic
val newPartition = TopicAndPartition(topic, 1)
- AdminUtils.addPartitions(zkClient, topic, 2, "0:1:2,0:1:2")
- TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers)
+ AdminUtils.addPartitions(zkUtils, topic, 2, "0:1:2,0:1:2")
+ TestUtils.verifyTopicDeletion(zkUtils, topic, 1, servers)
// verify that new partition doesn't exist on any broker either
assertTrue("Replica logs not deleted after delete topic is complete",
servers.forall(_.getLogManager().getLog(newPartition).isEmpty))
@@ -182,12 +183,12 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
val topicAndPartition = TopicAndPartition(topic, 0)
val servers = createTestTopicAndCluster(topic)
// start topic deletion
- AdminUtils.deleteTopic(zkClient, topic)
- TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers)
+ AdminUtils.deleteTopic(zkUtils, topic)
+ TestUtils.verifyTopicDeletion(zkUtils, topic, 1, servers)
// re-create topic on same replicas
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment)
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, expectedReplicaAssignment)
// wait until leader is elected
- val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000)
+ val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 0, 1000)
assertTrue("New leader should be elected after re-creating topic test", leaderIdOpt.isDefined)
// check if all replica logs are created
TestUtils.waitUntilTrue(() => servers.forall(_.getLogManager().getLog(topicAndPartition).isDefined),
@@ -201,16 +202,16 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
val topic = topicAndPartition.topic
val servers = createTestTopicAndCluster(topic)
// start topic deletion
- AdminUtils.deleteTopic(zkClient, "test2")
+ AdminUtils.deleteTopic(zkUtils, "test2")
// verify delete topic path for test2 is removed from zookeeper
- TestUtils.verifyTopicDeletion(zkClient, "test2", 1, servers)
+ TestUtils.verifyTopicDeletion(zkUtils, "test2", 1, servers)
// verify that topic test is untouched
TestUtils.waitUntilTrue(() => servers.forall(_.getLogManager().getLog(topicAndPartition).isDefined),
"Replicas for topic test not created")
// test the topic path exists
- assertTrue("Topic test mistakenly deleted", ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic)))
+ assertTrue("Topic test mistakenly deleted", zkUtils.pathExists(getTopicPath(topic)))
// topic test should have a leader
- val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000)
+ val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 0, 1000)
assertTrue("Leader should exist for topic test", leaderIdOpt.isDefined)
servers.foreach(_.shutdown())
}
@@ -242,8 +243,8 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
server.logManager.cleaner.awaitCleaned(topicName,0,0)
// delete topic
- AdminUtils.deleteTopic(zkClient, "test")
- TestUtils.verifyTopicDeletion(zkClient, "test", 1, servers)
+ AdminUtils.deleteTopic(zkUtils, "test")
+ TestUtils.verifyTopicDeletion(zkUtils, "test", 1, servers)
servers.foreach(_.shutdown())
}
@@ -256,16 +257,16 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
try {
// start topic deletion
- AdminUtils.deleteTopic(zkClient, topic)
+ AdminUtils.deleteTopic(zkUtils, topic)
// try to delete topic marked as deleted
- AdminUtils.deleteTopic(zkClient, topic)
+ AdminUtils.deleteTopic(zkUtils, topic)
fail("Expected TopicAlreadyMarkedForDeletionException")
}
catch {
case e: TopicAlreadyMarkedForDeletionException => // expected exception
}
- TestUtils.verifyTopicDeletion(zkClient, topic, 1, servers)
+ TestUtils.verifyTopicDeletion(zkUtils, topic, 1, servers)
servers.foreach(_.shutdown())
}
@@ -283,7 +284,7 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
// create brokers
val servers = brokerConfigs.map(b => TestUtils.createServer(KafkaConfig.fromProps(b)))
// create the topic
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, expectedReplicaAssignment)
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, expectedReplicaAssignment)
// wait until replica log is created on every broker
TestUtils.waitUntilTrue(() => servers.forall(_.getLogManager().getLog(topicAndPartition).isDefined),
"Replicas for topic test not created")
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
index d4fa0d5..cab4813 100644
--- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
@@ -23,7 +23,7 @@ import kafka.utils.TestUtils
import kafka.zk.ZooKeeperTestHarness
import kafka.server.ConfigType
import kafka.admin.TopicCommand.TopicCommandOptions
-import kafka.utils.ZkUtils
+import kafka.utils.ZkUtils._
import kafka.coordinator.ConsumerCoordinator
class TopicCommandTest extends ZooKeeperTestHarness with Logging {
@@ -36,25 +36,25 @@ class TopicCommandTest extends ZooKeeperTestHarness with Logging {
val cleanupVal = "compact"
// create brokers
val brokers = List(0, 1, 2)
- TestUtils.createBrokersInZk(zkClient, zkConnection, brokers)
+ TestUtils.createBrokersInZk(zkUtils, brokers)
// create the topic
val createOpts = new TopicCommandOptions(Array("--partitions", numPartitionsOriginal.toString,
"--replication-factor", "1",
"--config", cleanupKey + "=" + cleanupVal,
"--topic", topic))
- TopicCommand.createTopic(zkClient, createOpts)
- val props = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topic, topic)
+ TopicCommand.createTopic(zkUtils, createOpts)
+ val props = AdminUtils.fetchEntityConfig(zkUtils, ConfigType.Topic, topic)
assertTrue("Properties after creation don't contain " + cleanupKey, props.containsKey(cleanupKey))
assertTrue("Properties after creation have incorrect value", props.getProperty(cleanupKey).equals(cleanupVal))
// pre-create the topic config changes path to avoid a NoNodeException
- ZkUtils.createPersistentPath(zkClient, ZkUtils.EntityConfigChangesPath)
+ zkUtils.createPersistentPath(EntityConfigChangesPath)
// modify the topic to add new partitions
val numPartitionsModified = 3
val alterOpts = new TopicCommandOptions(Array("--partitions", numPartitionsModified.toString, "--topic", topic))
- TopicCommand.alterTopic(zkClient, alterOpts)
- val newProps = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topic, topic)
+ TopicCommand.alterTopic(zkUtils, alterOpts)
+ val newProps = AdminUtils.fetchEntityConfig(zkUtils, ConfigType.Topic, topic)
assertTrue("Updated properties do not contain " + cleanupKey, newProps.containsKey(cleanupKey))
assertTrue("Updated properties have incorrect value", newProps.getProperty(cleanupKey).equals(cleanupVal))
}
@@ -67,34 +67,34 @@ class TopicCommandTest extends ZooKeeperTestHarness with Logging {
// create brokers
val brokers = List(0, 1, 2)
- TestUtils.createBrokersInZk(zkClient, zkConnection, brokers)
+ TestUtils.createBrokersInZk(zkUtils, brokers)
// create the NormalTopic
val createOpts = new TopicCommandOptions(Array("--partitions", numPartitionsOriginal.toString,
"--replication-factor", "1",
"--topic", normalTopic))
- TopicCommand.createTopic(zkClient, createOpts)
+ TopicCommand.createTopic(zkUtils, createOpts)
// delete the NormalTopic
val deleteOpts = new TopicCommandOptions(Array("--topic", normalTopic))
- val deletePath = ZkUtils.getDeleteTopicPath(normalTopic)
- assertFalse("Delete path for topic shouldn't exist before deletion.", zkClient.exists(deletePath))
- TopicCommand.deleteTopic(zkClient, deleteOpts)
- assertTrue("Delete path for topic should exist after deletion.", zkClient.exists(deletePath))
+ val deletePath = getDeleteTopicPath(normalTopic)
+ assertFalse("Delete path for topic shouldn't exist before deletion.", zkUtils.zkClient.exists(deletePath))
+ TopicCommand.deleteTopic(zkUtils, deleteOpts)
+ assertTrue("Delete path for topic should exist after deletion.", zkUtils.zkClient.exists(deletePath))
// create the offset topic
val createOffsetTopicOpts = new TopicCommandOptions(Array("--partitions", numPartitionsOriginal.toString,
"--replication-factor", "1",
"--topic", ConsumerCoordinator.OffsetsTopicName))
- TopicCommand.createTopic(zkClient, createOffsetTopicOpts)
+ TopicCommand.createTopic(zkUtils, createOffsetTopicOpts)
// try to delete the OffsetManager.OffsetsTopicName and make sure it doesn't
val deleteOffsetTopicOpts = new TopicCommandOptions(Array("--topic", ConsumerCoordinator.OffsetsTopicName))
- val deleteOffsetTopicPath = ZkUtils.getDeleteTopicPath(ConsumerCoordinator.OffsetsTopicName)
- assertFalse("Delete path for topic shouldn't exist before deletion.", zkClient.exists(deleteOffsetTopicPath))
+ val deleteOffsetTopicPath = getDeleteTopicPath(ConsumerCoordinator.OffsetsTopicName)
+ assertFalse("Delete path for topic shouldn't exist before deletion.", zkUtils.zkClient.exists(deleteOffsetTopicPath))
intercept[AdminOperationException] {
- TopicCommand.deleteTopic(zkClient, deleteOffsetTopicOpts)
+ TopicCommand.deleteTopic(zkUtils, deleteOffsetTopicOpts)
}
- assertFalse("Delete path for topic shouldn't exist after deletion.", zkClient.exists(deleteOffsetTopicPath))
+ assertFalse("Delete path for topic shouldn't exist after deletion.", zkUtils.zkClient.exists(deleteOffsetTopicPath))
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala b/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala
index d43778e..50496f0 100644
--- a/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala
+++ b/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala
@@ -43,10 +43,10 @@ class ZkNodeChangeNotificationListenerTest extends KafkaServerTestHarness {
val notificationMessage2 = "message2"
val changeExpirationMs = 100
- val notificationListener = new ZkNodeChangeNotificationListener(zkClient, seqNodeRoot, seqNodePrefix, notificationHandler, changeExpirationMs)
+ val notificationListener = new ZkNodeChangeNotificationListener(zkUtils, seqNodeRoot, seqNodePrefix, notificationHandler, changeExpirationMs)
notificationListener.init()
- ZkUtils.createSequentialPersistentPath(zkClient, seqNodePath, notificationMessage1)
+ zkUtils.createSequentialPersistentPath(seqNodePath, notificationMessage1)
TestUtils.waitUntilTrue(() => notificationHandler.invocationCount == 1 && notificationHandler.notification == notificationMessage1, "failed to send/process notification message in the timeout period.")
@@ -55,7 +55,7 @@ class ZkNodeChangeNotificationListenerTest extends KafkaServerTestHarness {
Assert.assertEquals(1, ZkUtils.getChildren(zkClient, seqNodeRoot).size) however even after that the assertion can fail as the second node it self can be deleted
depending on how threads get scheduled.*/
- ZkUtils.createSequentialPersistentPath(zkClient, seqNodePath, notificationMessage2)
+ zkUtils.createSequentialPersistentPath(seqNodePath, notificationMessage2)
TestUtils.waitUntilTrue(() => notificationHandler.invocationCount == 2 && notificationHandler.notification == notificationMessage2, "failed to send/process notification message in the timeout period.")
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala
index ca63c80..b8ad15b 100755
--- a/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala
+++ b/core/src/test/scala/unit/kafka/consumer/ConsumerIteratorTest.scala
@@ -57,7 +57,7 @@ class ConsumerIteratorTest extends KafkaServerTestHarness {
new AtomicLong(0),
new AtomicInteger(0),
""))
- createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId)), servers = servers)
+ createTopic(zkUtils, topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId)), servers = servers)
}
@Test
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala
index 6c22e8b..818229c 100644
--- a/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala
+++ b/core/src/test/scala/unit/kafka/consumer/PartitionAssignorTest.scala
@@ -48,9 +48,9 @@ class PartitionAssignorTest extends Logging {
("g1c" + consumer, WildcardSubscriptionInfo(streamCount, ".*", isWhitelist = true))
}).toSeq:_*)
val scenario = Scenario("g1", topicPartitionCounts, subscriptions)
- val zkClient = PartitionAssignorTest.setupZkClientMock(scenario)
- EasyMock.replay(zkClient)
- PartitionAssignorTest.assignAndVerify(scenario, assignor, zkClient, verifyAssignmentIsUniform = true)
+ val zkUtils = PartitionAssignorTest.setupZkClientMock(scenario)
+ EasyMock.replay(zkUtils.zkClient)
+ PartitionAssignorTest.assignAndVerify(scenario, assignor, zkUtils, verifyAssignmentIsUniform = true)
})
}
@@ -73,10 +73,10 @@ class PartitionAssignorTest extends Logging {
("g1c" + consumer, StaticSubscriptionInfo(streamCounts))
}).toSeq:_*)
val scenario = Scenario("g1", topicPartitionCounts, subscriptions)
- val zkClient = PartitionAssignorTest.setupZkClientMock(scenario)
- EasyMock.replay(zkClient)
+ val zkUtils = PartitionAssignorTest.setupZkClientMock(scenario)
+ EasyMock.replay(zkUtils.zkClient)
- PartitionAssignorTest.assignAndVerify(scenario, assignor, zkClient)
+ PartitionAssignorTest.assignAndVerify(scenario, assignor, zkUtils)
})
}
}
@@ -135,6 +135,7 @@ private object PartitionAssignorTest extends Logging {
val consumers = java.util.Arrays.asList(scenario.subscriptions.keys.toSeq:_*)
val zkClient = EasyMock.createStrictMock(classOf[ZkClient])
+ val zkUtils = ZkUtils(zkClient, false)
EasyMock.checkOrder(zkClient, false)
EasyMock.expect(zkClient.getChildren("/consumers/%s/ids".format(scenario.group))).andReturn(consumers)
@@ -149,21 +150,21 @@ private object PartitionAssignorTest extends Logging {
scenario.topicPartitionCounts.foreach { case(topic, partitionCount) =>
val replicaAssignment = Map((0 until partitionCount).map(partition => (partition.toString, Seq(0))):_*)
EasyMock.expect(zkClient.readData("/brokers/topics/%s".format(topic), new Stat()))
- .andReturn(ZkUtils.replicaAssignmentZkData(replicaAssignment))
+ .andReturn(zkUtils.replicaAssignmentZkData(replicaAssignment))
EasyMock.expectLastCall().anyTimes()
}
- EasyMock.expect(zkClient.getChildren("/brokers/topics")).andReturn(
+ EasyMock.expect(zkUtils.zkClient.getChildren("/brokers/topics")).andReturn(
java.util.Arrays.asList(scenario.topicPartitionCounts.keys.toSeq:_*))
EasyMock.expectLastCall().anyTimes()
- zkClient
+ zkUtils
}
- private def assignAndVerify(scenario: Scenario, assignor: PartitionAssignor, zkClient: ZkClient,
+ private def assignAndVerify(scenario: Scenario, assignor: PartitionAssignor, zkUtils: ZkUtils,
verifyAssignmentIsUniform: Boolean = false) {
val assignments = scenario.subscriptions.map{ case(consumer, subscription) =>
- val ctx = new AssignmentContext("g1", consumer, excludeInternalTopics = true, zkClient)
+ val ctx = new AssignmentContext("g1", consumer, excludeInternalTopics = true, zkUtils)
assignor.assign(ctx).get(consumer)
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala
index cb59542..28b1dd5 100644
--- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala
+++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala
@@ -97,8 +97,8 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging
sendMessages(servers, topic, nMessages, 1)
// wait to make sure the topic and partition have a leader for the successful case
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0)
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1)
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 0)
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 1)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1)
@@ -130,8 +130,8 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging
val sentMessages2 = sendMessages(servers, topic, nMessages, 0) ++
sendMessages(servers, topic, nMessages, 1)
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0)
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1)
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 0)
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 1)
val receivedMessages2 = getMessages(topicMessageStreams1, nMessages) ++ getMessages(topicMessageStreams2, nMessages)
assertEquals(sentMessages2.sorted, receivedMessages2.sorted)
@@ -151,8 +151,8 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging
val sentMessages3 = sendMessages(servers, topic, nMessages, 0) ++
sendMessages(servers, topic, nMessages, 1)
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0)
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1)
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 0)
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 1)
val receivedMessages3 = getMessages(topicMessageStreams1, nMessages) ++ getMessages(topicMessageStreams2, nMessages)
assertEquals(sentMessages3.sorted, receivedMessages3.sorted)
@@ -185,8 +185,8 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging
val sentMessages1 = sendMessages(servers, topic, nMessages, 0, GZIPCompressionCodec) ++
sendMessages(servers, topic, nMessages, 1, GZIPCompressionCodec)
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0)
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1)
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 0)
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 1)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 0)
TestUtils.waitUntilMetadataIsPropagated(servers, topic, 1)
@@ -218,8 +218,8 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging
val sentMessages2 = sendMessages(servers, topic, nMessages, 0, GZIPCompressionCodec) ++
sendMessages(servers, topic, nMessages, 1, GZIPCompressionCodec)
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0)
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1)
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 0)
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 1)
val receivedMessages2 = getMessages(topicMessageStreams1, nMessages) ++ getMessages(topicMessageStreams2, nMessages)
assertEquals(sentMessages2.sorted, receivedMessages2.sorted)
@@ -239,8 +239,8 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging
val sentMessages3 = sendMessages(servers, topic, nMessages, 0, GZIPCompressionCodec) ++
sendMessages(servers, topic, nMessages, 1, GZIPCompressionCodec)
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0)
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1)
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 0)
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 1)
val receivedMessages3 = getMessages(topicMessageStreams1, nMessages) ++ getMessages(topicMessageStreams2, nMessages)
assertEquals(sentMessages3.sorted, receivedMessages3.sorted)
@@ -294,8 +294,8 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging
val consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1))
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0)
- waitUntilLeaderIsElectedOrChanged(zkClient, topic, 1)
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 0)
+ waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 1)
val zkConsumerConnector =
new ZookeeperConsumerConnector(consumerConfig, true)
@@ -322,10 +322,10 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging
@Test
def testLeaderSelectionForPartition() {
- val zkClient = ZkUtils.createZkClient(zkConnect, 6000, 30000)
+ val zkUtils = ZkUtils(zkConnect, 6000, 30000, false)
// create topic topic1 with 1 partition on broker 0
- createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = servers)
+ createTopic(zkUtils, topic, numPartitions = 1, replicationFactor = 1, servers = servers)
// send some messages to each broker
val sentMessages1 = sendMessages(servers, topic, nMessages)
@@ -349,7 +349,7 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging
val receivedMessages1 = getMessages(topicMessageStreams1, nMessages)
assertEquals(sentMessages1, receivedMessages1)
zkConsumerConnector1.shutdown()
- zkClient.close()
+ zkUtils.close()
}
@Test
@@ -416,14 +416,14 @@ class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging
}
def getZKChildrenValues(path : String) : Seq[Tuple2[String,String]] = {
- val children = zkClient.getChildren(path)
+ val children = zkUtils.zkClient.getChildren(path)
Collections.sort(children)
val childrenAsSeq : Seq[java.lang.String] = {
import scala.collection.JavaConversions._
children.toSeq
}
childrenAsSeq.map(partition =>
- (partition, zkClient.readData(path + "/" + partition).asInstanceOf[String]))
+ (partition, zkUtils.zkClient.readData(path + "/" + partition).asInstanceOf[String]))
}
private class TestConsumerRebalanceListener extends ConsumerRebalanceListener {
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala
index c93eca5..91ac1f6 100644
--- a/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala
+++ b/core/src/test/scala/unit/kafka/controller/ControllerFailoverTest.scala
@@ -75,7 +75,7 @@ class ControllerFailoverTest extends KafkaServerTestHarness with Logging {
}
}
// Create topic with one partition
- kafka.admin.AdminUtils.createTopic(controller.zkClient, topic, 1, 1)
+ kafka.admin.AdminUtils.createTopic(controller.zkUtils, topic, 1, 1)
val topicPartition = TopicAndPartition("topic1", 0)
var partitions = controller.kafkaController.partitionStateMachine.partitionsInState(OnlinePartition)
while (!partitions.contains(topicPartition)) {
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala
index d8a7948..3bc37e5 100644
--- a/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala
@@ -18,7 +18,8 @@
package kafka.coordinator
import kafka.server.KafkaConfig
-import kafka.utils.{ZkUtils, TestUtils}
+import kafka.utils.{TestUtils, ZkUtils}
+import kafka.utils.ZkUtils._
import org.junit.Assert._
import org.I0Itec.zkclient.{IZkDataListener, ZkClient}
@@ -33,14 +34,15 @@ import org.scalatest.junit.JUnitSuite
class CoordinatorMetadataTest extends JUnitSuite {
val DefaultNumPartitions = 8
val DefaultNumReplicas = 2
- var zkClient: ZkClient = null
+ var zkUtils: ZkUtils = null
var coordinatorMetadata: CoordinatorMetadata = null
@Before
def setUp() {
val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
- zkClient = EasyMock.createStrictMock(classOf[ZkClient])
- coordinatorMetadata = new CoordinatorMetadata(KafkaConfig.fromProps(props).brokerId, zkClient, null)
+ val zkClient = EasyMock.createStrictMock(classOf[ZkClient])
+ zkUtils = ZkUtils(zkClient, false)
+ coordinatorMetadata = new CoordinatorMetadata(KafkaConfig.fromProps(props).brokerId, zkUtils, null)
}
@Test
@@ -77,8 +79,8 @@ class CoordinatorMetadataTest extends JUnitSuite {
val topics = Set("a")
coordinatorMetadata.addGroup(groupId, "range")
- expectZkClientSubscribeDataChanges(zkClient, topics)
- EasyMock.replay(zkClient)
+ expectZkClientSubscribeDataChanges(zkUtils, topics)
+ EasyMock.replay(zkUtils.zkClient)
coordinatorMetadata.bindGroupToTopics(groupId, topics)
assertEquals(Map("a" -> DefaultNumPartitions), coordinatorMetadata.partitionsPerTopic)
}
@@ -91,8 +93,8 @@ class CoordinatorMetadataTest extends JUnitSuite {
coordinatorMetadata.addGroup(group1, "range")
coordinatorMetadata.addGroup(group2, "range")
- expectZkClientSubscribeDataChanges(zkClient, topics)
- EasyMock.replay(zkClient)
+ expectZkClientSubscribeDataChanges(zkUtils, topics)
+ EasyMock.replay(zkUtils.zkClient)
coordinatorMetadata.bindGroupToTopics(group1, topics)
coordinatorMetadata.bindGroupToTopics(group2, topics)
assertEquals(Map("a" -> DefaultNumPartitions), coordinatorMetadata.partitionsPerTopic)
@@ -111,8 +113,8 @@ class CoordinatorMetadataTest extends JUnitSuite {
val topics = Set("a")
coordinatorMetadata.addGroup(groupId, "range")
- expectZkClientSubscribeDataChanges(zkClient, topics)
- EasyMock.replay(zkClient)
+ expectZkClientSubscribeDataChanges(zkUtils, topics)
+ EasyMock.replay(zkUtils.zkClient)
coordinatorMetadata.bindGroupToTopics(groupId, topics)
coordinatorMetadata.unbindGroupFromTopics(groupId, Set("b"))
assertEquals(Map("a" -> DefaultNumPartitions), coordinatorMetadata.partitionsPerTopic)
@@ -126,8 +128,8 @@ class CoordinatorMetadataTest extends JUnitSuite {
coordinatorMetadata.addGroup(group1, "range")
coordinatorMetadata.addGroup(group2, "range")
- expectZkClientSubscribeDataChanges(zkClient, topics)
- EasyMock.replay(zkClient)
+ expectZkClientSubscribeDataChanges(zkUtils, topics)
+ EasyMock.replay(zkUtils.zkClient)
coordinatorMetadata.bindGroupToTopics(group1, topics)
coordinatorMetadata.bindGroupToTopics(group2, topics)
coordinatorMetadata.unbindGroupFromTopics(group1, topics)
@@ -140,9 +142,9 @@ class CoordinatorMetadataTest extends JUnitSuite {
val topics = Set("a")
coordinatorMetadata.addGroup(groupId, "range")
- expectZkClientSubscribeDataChanges(zkClient, topics)
- expectZkClientUnsubscribeDataChanges(zkClient, topics)
- EasyMock.replay(zkClient)
+ expectZkClientSubscribeDataChanges(zkUtils, topics)
+ expectZkClientUnsubscribeDataChanges(zkUtils.zkClient, topics)
+ EasyMock.replay(zkUtils.zkClient)
coordinatorMetadata.bindGroupToTopics(groupId, topics)
coordinatorMetadata.unbindGroupFromTopics(groupId, topics)
assertEquals(Map.empty[String, Int], coordinatorMetadata.partitionsPerTopic)
@@ -163,8 +165,8 @@ class CoordinatorMetadataTest extends JUnitSuite {
coordinatorMetadata.addGroup(group1, "range")
coordinatorMetadata.addGroup(group2, "range")
- expectZkClientSubscribeDataChanges(zkClient, topics)
- EasyMock.replay(zkClient)
+ expectZkClientSubscribeDataChanges(zkUtils, topics)
+ EasyMock.replay(zkUtils.zkClient)
coordinatorMetadata.bindGroupToTopics(group1, topics)
coordinatorMetadata.bindGroupToTopics(group2, topics)
coordinatorMetadata.removeGroup(group1, topics)
@@ -179,17 +181,17 @@ class CoordinatorMetadataTest extends JUnitSuite {
val topics = Set("a")
coordinatorMetadata.addGroup(groupId, "range")
- expectZkClientSubscribeDataChanges(zkClient, topics)
- expectZkClientUnsubscribeDataChanges(zkClient, topics)
- EasyMock.replay(zkClient)
+ expectZkClientSubscribeDataChanges(zkUtils, topics)
+ expectZkClientUnsubscribeDataChanges(zkUtils.zkClient, topics)
+ EasyMock.replay(zkUtils.zkClient)
coordinatorMetadata.bindGroupToTopics(groupId, topics)
coordinatorMetadata.removeGroup(groupId, topics)
assertNull(coordinatorMetadata.getGroup(groupId))
assertEquals(Map.empty[String, Int], coordinatorMetadata.partitionsPerTopic)
}
- private def expectZkClientSubscribeDataChanges(zkClient: ZkClient, topics: Set[String]) {
- topics.foreach(topic => expectZkClientSubscribeDataChange(zkClient, topic))
+ private def expectZkClientSubscribeDataChanges(zkUtils: ZkUtils, topics: Set[String]) {
+ topics.foreach(topic => expectZkClientSubscribeDataChange(zkUtils.zkClient, topic))
}
private def expectZkClientUnsubscribeDataChanges(zkClient: ZkClient, topics: Set[String]) {
@@ -200,14 +202,14 @@ class CoordinatorMetadataTest extends JUnitSuite {
val replicaAssignment =
(0 until DefaultNumPartitions)
.map(partition => partition.toString -> (0 until DefaultNumReplicas).toSeq).toMap
- val topicPath = ZkUtils.getTopicPath(topic)
+ val topicPath = getTopicPath(topic)
EasyMock.expect(zkClient.readData(topicPath, new Stat()))
- .andReturn(ZkUtils.replicaAssignmentZkData(replicaAssignment))
+ .andReturn(zkUtils.replicaAssignmentZkData(replicaAssignment))
zkClient.subscribeDataChanges(EasyMock.eq(topicPath), EasyMock.isA(classOf[IZkDataListener]))
}
private def expectZkClientUnsubscribeDataChange(zkClient: ZkClient, topic: String) {
- val topicPath = ZkUtils.getTopicPath(topic)
+ val topicPath = getTopicPath(topic)
zkClient.unsubscribeDataChanges(EasyMock.eq(topicPath), EasyMock.isA(classOf[IZkDataListener]))
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala
index 818673f..a71ddf1 100644
--- a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala
+++ b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala
@@ -76,7 +76,7 @@ class AutoOffsetResetTest extends KafkaServerTestHarness with Logging {
* Returns the count of messages received.
*/
def resetAndConsume(numMessages: Int, resetTo: String, offset: Long): Int = {
- TestUtils.createTopic(zkClient, topic, 1, 1, servers)
+ TestUtils.createTopic(zkUtils, topic, 1, 1, servers)
val producer: Producer[String, Array[Byte]] = TestUtils.createProducer(
TestUtils.getBrokerListStrFromServers(servers),
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala
index 20a4068..3cf4dae 100644
--- a/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala
+++ b/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala
@@ -62,7 +62,7 @@ abstract class BaseTopicMetadataTest extends ZooKeeperTestHarness {
def testTopicMetadataRequest {
// create topic
val topic = "test"
- AdminUtils.createTopic(zkClient, topic, 1, 1)
+ AdminUtils.createTopic(zkUtils, topic, 1, 1)
// create a topic metadata request
val topicMetadataRequest = new TopicMetadataRequest(List(topic), 0)
@@ -79,7 +79,7 @@ abstract class BaseTopicMetadataTest extends ZooKeeperTestHarness {
def testBasicTopicMetadata {
// create topic
val topic = "test"
- createTopic(zkClient, topic, numPartitions = 1, replicationFactor = 1, servers = Seq(server1))
+ createTopic(zkUtils, topic, numPartitions = 1, replicationFactor = 1, servers = Seq(server1))
val topicsMetadata = ClientUtils.fetchTopicMetadata(Set(topic), brokerEndPoints, "TopicMetadataTest-testBasicTopicMetadata",
2000,0).topicsMetadata
@@ -98,8 +98,8 @@ abstract class BaseTopicMetadataTest extends ZooKeeperTestHarness {
// create topic
val topic1 = "testGetAllTopicMetadata1"
val topic2 = "testGetAllTopicMetadata2"
- createTopic(zkClient, topic1, numPartitions = 1, replicationFactor = 1, servers = Seq(server1))
- createTopic(zkClient, topic2, numPartitions = 1, replicationFactor = 1, servers = Seq(server1))
+ createTopic(zkUtils, topic1, numPartitions = 1, replicationFactor = 1, servers = Seq(server1))
+ createTopic(zkUtils, topic2, numPartitions = 1, replicationFactor = 1, servers = Seq(server1))
// issue metadata request with empty list of topics
val topicsMetadata = ClientUtils.fetchTopicMetadata(Set.empty, brokerEndPoints, "TopicMetadataTest-testGetAllTopicMetadata",
@@ -130,7 +130,7 @@ abstract class BaseTopicMetadataTest extends ZooKeeperTestHarness {
assertEquals(0, topicsMetadata.head.partitionsMetadata.size)
// wait for leader to be elected
- TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0)
+ TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 0)
TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic, 0)
// retry the metadata for the auto created topic
@@ -159,7 +159,7 @@ abstract class BaseTopicMetadataTest extends ZooKeeperTestHarness {
assertEquals("Expecting InvalidTopicCode for topic2 metadata", ErrorMapping.InvalidTopicCode, topicsMetadata(1).errorCode)
// wait for leader to be elected
- TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic1, 0)
+ TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, topic1, 0)
TestUtils.waitUntilMetadataIsPropagated(Seq(server1), topic1, 0)
// retry the metadata for the first auto created topic
@@ -218,7 +218,7 @@ abstract class BaseTopicMetadataTest extends ZooKeeperTestHarness {
// create topic
val topic: String = "test"
- AdminUtils.createTopic(zkClient, topic, 1, numBrokers)
+ AdminUtils.createTopic(zkUtils, topic, 1, numBrokers)
// shutdown a broker
adHocServers.last.shutdown()
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/integration/FetcherTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala
index c061597..5af5d1a 100644
--- a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala
+++ b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala
@@ -42,11 +42,11 @@ class FetcherTest extends KafkaServerTestHarness {
@Before
override def setUp() {
super.setUp
- TestUtils.createTopic(zkClient, topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId)), servers = servers)
+ TestUtils.createTopic(zkUtils, topic, partitionReplicaAssignment = Map(0 -> Seq(configs.head.brokerId)), servers = servers)
val cluster = new Cluster(servers.map(s => new Broker(s.config.brokerId, "localhost", s.boundPort())))
- fetcher = new ConsumerFetcherManager("consumer1", new ConsumerConfig(TestUtils.createConsumerProperties("", "", "")), zkClient)
+ fetcher = new ConsumerFetcherManager("consumer1", new ConsumerConfig(TestUtils.createConsumerProperties("", "", "")), zkUtils)
fetcher.stopConnections()
val topicInfos = configs.map(c =>
new PartitionTopicInfo(topic,
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala
index e6f0c54..df752db 100755
--- a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala
+++ b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala
@@ -113,7 +113,7 @@ class PrimitiveApiTest extends ProducerConsumerTestHarness with ZooKeeperTestHar
private def produceAndMultiFetch(producer: Producer[String, String]) {
for(topic <- List("test1", "test2", "test3", "test4"))
- TestUtils.createTopic(zkClient, topic, servers = servers)
+ TestUtils.createTopic(zkUtils, topic, servers = servers)
// send some messages
val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
@@ -182,7 +182,7 @@ class PrimitiveApiTest extends ProducerConsumerTestHarness with ZooKeeperTestHar
private def multiProduce(producer: Producer[String, String]) {
val topics = Map("test4" -> 0, "test1" -> 0, "test2" -> 0, "test3" -> 0)
- topics.keys.map(topic => TestUtils.createTopic(zkClient, topic, servers = servers))
+ topics.keys.map(topic => TestUtils.createTopic(zkUtils, topic, servers = servers))
val messages = new mutable.HashMap[String, Seq[String]]
val builder = new FetchRequestBuilder()
@@ -210,7 +210,7 @@ class PrimitiveApiTest extends ProducerConsumerTestHarness with ZooKeeperTestHar
@Test
def testConsumerEmptyTopic() {
val newTopic = "new-topic"
- TestUtils.createTopic(zkClient, newTopic, numPartitions = 1, replicationFactor = 1, servers = servers)
+ TestUtils.createTopic(zkUtils, newTopic, numPartitions = 1, replicationFactor = 1, servers = servers)
val fetchResponse = consumer.fetch(new FetchRequestBuilder().addFetch(newTopic, 0, 0, 10000).build())
assertFalse(fetchResponse.messageSet(newTopic, 0).iterator.hasNext)
@@ -219,7 +219,7 @@ class PrimitiveApiTest extends ProducerConsumerTestHarness with ZooKeeperTestHar
@Test
def testPipelinedProduceRequests() {
val topics = Map("test4" -> 0, "test1" -> 0, "test2" -> 0, "test3" -> 0)
- topics.keys.map(topic => TestUtils.createTopic(zkClient, topic, servers = servers))
+ topics.keys.map(topic => TestUtils.createTopic(zkUtils, topic, servers = servers))
val props = new Properties()
props.put("request.required.acks", "0")
val pipelinedProducer: Producer[String, String] =
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala
index 4d73be1..b931568 100755
--- a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala
+++ b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala
@@ -56,10 +56,10 @@ class RollingBounceTest extends ZooKeeperTestHarness {
val topic4 = "new-topic4"
// create topics with 1 partition, 2 replicas, one on each broker
- createTopic(zkClient, topic1, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers)
- createTopic(zkClient, topic2, partitionReplicaAssignment = Map(0->Seq(1,2)), servers = servers)
- createTopic(zkClient, topic3, partitionReplicaAssignment = Map(0->Seq(2,3)), servers = servers)
- createTopic(zkClient, topic4, partitionReplicaAssignment = Map(0->Seq(0,3)), servers = servers)
+ createTopic(zkUtils, topic1, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers)
+ createTopic(zkUtils, topic2, partitionReplicaAssignment = Map(0->Seq(1,2)), servers = servers)
+ createTopic(zkUtils, topic3, partitionReplicaAssignment = Map(0->Seq(2,3)), servers = servers)
+ createTopic(zkUtils, topic4, partitionReplicaAssignment = Map(0->Seq(0,3)), servers = servers)
// Do a rolling bounce and check if leader transitions happen correctly
@@ -86,7 +86,7 @@ class RollingBounceTest extends ZooKeeperTestHarness {
servers((startIndex + 1) % 4).shutdown()
prevLeader = (startIndex + 1) % 4
}
- var newleader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId)
+ var newleader = waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId)
// Ensure the new leader is different from the old
assertTrue("Leader transition did not happen for " + topic, newleader.getOrElse(-1) != -1 && (newleader.getOrElse(-1) != prevLeader))
// Start the server back up again
[3/5] kafka git commit: KAFKA-2639: Refactoring of ZkUtils
Posted by ju...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala b/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala
index b283e0a..e4e1e9c 100755
--- a/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala
+++ b/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala
@@ -23,6 +23,7 @@ import org.I0Itec.zkclient.exception.ZkNodeExistsException
import org.I0Itec.zkclient.IZkDataListener
import kafka.controller.ControllerContext
import kafka.controller.KafkaController
+import org.apache.kafka.common.security.JaasUtils
/**
* This class handles zookeeper based leader election based on an ephemeral path. The election module does not handle
@@ -40,18 +41,18 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext,
// create the election path in ZK, if one does not exist
val index = electionPath.lastIndexOf("/")
if (index > 0)
- makeSurePersistentPathExists(controllerContext.zkClient, electionPath.substring(0, index))
+ controllerContext.zkUtils.makeSurePersistentPathExists(electionPath.substring(0, index))
val leaderChangeListener = new LeaderChangeListener
def startup {
inLock(controllerContext.controllerLock) {
- controllerContext.zkClient.subscribeDataChanges(electionPath, leaderChangeListener)
+ controllerContext.zkUtils.zkClient.subscribeDataChanges(electionPath, leaderChangeListener)
elect
}
}
private def getControllerID(): Int = {
- readDataMaybeNull(controllerContext.zkClient, electionPath)._1 match {
+ controllerContext.zkUtils.readDataMaybeNull(electionPath)._1 match {
case Some(controller) => KafkaController.parseControllerId(controller)
case None => -1
}
@@ -75,7 +76,8 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext,
try {
val zkCheckedEphemeral = new ZKCheckedEphemeral(electionPath,
electString,
- controllerContext.zkConnection.getZookeeper)
+ controllerContext.zkUtils.zkConnection.getZookeeper,
+ JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)))
zkCheckedEphemeral.create()
info(brokerId + " successfully elected as leader")
leaderId = brokerId
@@ -105,7 +107,7 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext,
def resign() = {
leaderId = -1
- deletePath(controllerContext.zkClient, electionPath)
+ controllerContext.zkUtils.deletePath(electionPath)
}
/**
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala
index c39fbfe..87d9fb7 100644
--- a/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala
+++ b/core/src/main/scala/kafka/tools/ConsumerOffsetChecker.scala
@@ -25,6 +25,7 @@ import kafka.consumer.SimpleConsumer
import kafka.api.{OffsetFetchResponse, OffsetFetchRequest, OffsetRequest}
import kafka.common.{OffsetMetadataAndError, ErrorMapping, BrokerNotAvailableException, TopicAndPartition}
import org.apache.kafka.common.protocol.SecurityProtocol
+import org.apache.kafka.common.security.JaasUtils
import scala.collection._
import kafka.client.ClientUtils
import kafka.network.BlockingChannel
@@ -37,9 +38,9 @@ object ConsumerOffsetChecker extends Logging {
private val offsetMap: mutable.Map[TopicAndPartition, Long] = mutable.Map()
private var topicPidMap: immutable.Map[String, Seq[Int]] = immutable.Map()
- private def getConsumer(zkClient: ZkClient, bid: Int): Option[SimpleConsumer] = {
+ private def getConsumer(zkUtils: ZkUtils, bid: Int): Option[SimpleConsumer] = {
try {
- ZkUtils.readDataMaybeNull(zkClient, ZkUtils.BrokerIdsPath + "/" + bid)._1 match {
+ zkUtils.readDataMaybeNull(ZkUtils.BrokerIdsPath + "/" + bid)._1 match {
case Some(brokerInfoString) =>
Json.parseFull(brokerInfoString) match {
case Some(m) =>
@@ -60,15 +61,15 @@ object ConsumerOffsetChecker extends Logging {
}
}
- private def processPartition(zkClient: ZkClient,
+ private def processPartition(zkUtils: ZkUtils,
group: String, topic: String, pid: Int) {
val topicPartition = TopicAndPartition(topic, pid)
val offsetOpt = offsetMap.get(topicPartition)
val groupDirs = new ZKGroupTopicDirs(group, topic)
- val owner = ZkUtils.readDataMaybeNull(zkClient, groupDirs.consumerOwnerDir + "/%s".format(pid))._1
- ZkUtils.getLeaderForPartition(zkClient, topic, pid) match {
+ val owner = zkUtils.readDataMaybeNull(groupDirs.consumerOwnerDir + "/%s".format(pid))._1
+ zkUtils.getLeaderForPartition(topic, pid) match {
case Some(bid) =>
- val consumerOpt = consumerMap.getOrElseUpdate(bid, getConsumer(zkClient, bid))
+ val consumerOpt = consumerMap.getOrElseUpdate(bid, getConsumer(zkUtils, bid))
consumerOpt match {
case Some(consumer) =>
val topicAndPartition = TopicAndPartition(topic, pid)
@@ -86,11 +87,11 @@ object ConsumerOffsetChecker extends Logging {
}
}
- private def processTopic(zkClient: ZkClient, group: String, topic: String) {
+ private def processTopic(zkUtils: ZkUtils, group: String, topic: String) {
topicPidMap.get(topic) match {
case Some(pids) =>
pids.sorted.foreach {
- pid => processPartition(zkClient, group, topic, pid)
+ pid => processPartition(zkUtils, group, topic, pid)
}
case None => // ignore
}
@@ -148,19 +149,22 @@ object ConsumerOffsetChecker extends Logging {
val topics = if (options.has(topicsOpt)) Some(options.valueOf(topicsOpt)) else None
- var zkClient: ZkClient = null
+ var zkUtils: ZkUtils = null
var channel: BlockingChannel = null
try {
- zkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000)
+ zkUtils = ZkUtils(zkConnect,
+ 30000,
+ 30000,
+ JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)))
val topicList = topics match {
case Some(x) => x.split(",").view.toList
- case None => ZkUtils.getChildren(zkClient, groupDirs.consumerGroupDir + "/owners").toList
+ case None => zkUtils.getChildren(groupDirs.consumerGroupDir + "/owners").toList
}
- topicPidMap = immutable.Map(ZkUtils.getPartitionsForTopics(zkClient, topicList).toSeq:_*)
+ topicPidMap = immutable.Map(zkUtils.getPartitionsForTopics(topicList).toSeq:_*)
val topicPartitions = topicPidMap.flatMap { case(topic, partitionSeq) => partitionSeq.map(TopicAndPartition(topic, _)) }.toSeq
- val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs)
+ val channel = ClientUtils.channelToOffsetManager(group, zkUtils, channelSocketTimeoutMs, channelRetryBackoffMs)
debug("Sending offset fetch request to coordinator %s:%d.".format(channel.host, channel.port))
channel.send(OffsetFetchRequest(group, topicPartitions))
@@ -173,11 +177,11 @@ object ConsumerOffsetChecker extends Logging {
// this group may not have migrated off zookeeper for offsets storage (we don't expose the dual-commit option in this tool
// (meaning the lag may be off until all the consumers in the group have the same setting for offsets storage)
try {
- val offset = ZkUtils.readData(zkClient, topicDirs.consumerOffsetDir + "/%d".format(topicAndPartition.partition))._1.toLong
+ val offset = zkUtils.readData(topicDirs.consumerOffsetDir + "/%d".format(topicAndPartition.partition))._1.toLong
offsetMap.put(topicAndPartition, offset)
} catch {
case z: ZkNoNodeException =>
- if(ZkUtils.pathExists(zkClient,topicDirs.consumerOffsetDir))
+ if(zkUtils.pathExists(topicDirs.consumerOffsetDir))
offsetMap.put(topicAndPartition,-1)
else
throw z
@@ -193,7 +197,7 @@ object ConsumerOffsetChecker extends Logging {
println("%-15s %-30s %-3s %-15s %-15s %-15s %s".format("Group", "Topic", "Pid", "Offset", "logSize", "Lag", "Owner"))
topicList.sorted.foreach {
- topic => processTopic(zkClient, group, topic)
+ topic => processTopic(zkUtils, group, topic)
}
if (options.has("broker-info"))
@@ -216,8 +220,8 @@ object ConsumerOffsetChecker extends Logging {
case None => // ignore
}
}
- if (zkClient != null)
- zkClient.close()
+ if (zkUtils != null)
+ zkUtils.close()
if (channel != null)
channel.disconnect()
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/tools/ExportZkOffsets.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/ExportZkOffsets.scala b/core/src/main/scala/kafka/tools/ExportZkOffsets.scala
index 7b52fe4..75d4fd1 100644
--- a/core/src/main/scala/kafka/tools/ExportZkOffsets.scala
+++ b/core/src/main/scala/kafka/tools/ExportZkOffsets.scala
@@ -21,6 +21,7 @@ import java.io.FileWriter
import joptsimple._
import kafka.utils.{Logging, ZkUtils, ZKGroupTopicDirs, CommandLineUtils}
import org.I0Itec.zkclient.ZkClient
+import org.apache.kafka.common.security.JaasUtils
/**
@@ -72,16 +73,19 @@ object ExportZkOffsets extends Logging {
val groups = options.valuesOf(groupOpt)
val outfile = options.valueOf(outFileOpt)
- var zkClient : ZkClient = null
+ var zkUtils : ZkUtils = null
val fileWriter : FileWriter = new FileWriter(outfile)
try {
- zkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000)
+ zkUtils = ZkUtils(zkConnect,
+ 30000,
+ 30000,
+ JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)))
var consumerGroups: Seq[String] = null
if (groups.size == 0) {
- consumerGroups = ZkUtils.getChildren(zkClient, ZkUtils.ConsumersPath).toList
+ consumerGroups = zkUtils.getChildren(ZkUtils.ConsumersPath).toList
}
else {
import scala.collection.JavaConversions._
@@ -89,15 +93,15 @@ object ExportZkOffsets extends Logging {
}
for (consumerGrp <- consumerGroups) {
- val topicsList = getTopicsList(zkClient, consumerGrp)
+ val topicsList = getTopicsList(zkUtils, consumerGrp)
for (topic <- topicsList) {
- val bidPidList = getBrokeridPartition(zkClient, consumerGrp, topic)
+ val bidPidList = getBrokeridPartition(zkUtils, consumerGrp, topic)
for (bidPid <- bidPidList) {
val zkGrpTpDir = new ZKGroupTopicDirs(consumerGrp,topic)
val offsetPath = zkGrpTpDir.consumerOffsetDir + "/" + bidPid
- ZkUtils.readDataMaybeNull(zkClient, offsetPath)._1 match {
+ zkUtils.readDataMaybeNull(offsetPath)._1 match {
case Some(offsetVal) =>
fileWriter.write(offsetPath + ":" + offsetVal + "\n")
debug(offsetPath + " => " + offsetVal)
@@ -114,10 +118,10 @@ object ExportZkOffsets extends Logging {
}
}
- private def getBrokeridPartition(zkClient: ZkClient, consumerGroup: String, topic: String): List[String] =
- ZkUtils.getChildrenParentMayNotExist(zkClient, "/consumers/%s/offsets/%s".format(consumerGroup, topic)).toList
+ private def getBrokeridPartition(zkUtils: ZkUtils, consumerGroup: String, topic: String): List[String] =
+ zkUtils.getChildrenParentMayNotExist("/consumers/%s/offsets/%s".format(consumerGroup, topic)).toList
- private def getTopicsList(zkClient: ZkClient, consumerGroup: String): List[String] =
- ZkUtils.getChildren(zkClient, "/consumers/%s/offsets".format(consumerGroup)).toList
+ private def getTopicsList(zkUtils: ZkUtils, consumerGroup: String): List[String] =
+ zkUtils.getChildren("/consumers/%s/offsets".format(consumerGroup)).toList
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/tools/ImportZkOffsets.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/ImportZkOffsets.scala b/core/src/main/scala/kafka/tools/ImportZkOffsets.scala
index b56f587..38a71ae 100644
--- a/core/src/main/scala/kafka/tools/ImportZkOffsets.scala
+++ b/core/src/main/scala/kafka/tools/ImportZkOffsets.scala
@@ -22,6 +22,7 @@ import java.io.FileReader
import joptsimple._
import kafka.utils.{Logging, ZkUtils, CommandLineUtils}
import org.I0Itec.zkclient.ZkClient
+import org.apache.kafka.common.security.JaasUtils
/**
@@ -68,10 +69,10 @@ object ImportZkOffsets extends Logging {
val zkConnect = options.valueOf(zkConnectOpt)
val partitionOffsetFile = options.valueOf(inFileOpt)
- val zkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000)
+ val zkUtils = ZkUtils(zkConnect, 30000, 30000, JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)))
val partitionOffsets: Map[String,String] = getPartitionOffsetsFromFile(partitionOffsetFile)
- updateZkOffsets(zkClient, partitionOffsets)
+ updateZkOffsets(zkUtils, partitionOffsets)
}
private def getPartitionOffsetsFromFile(filename: String):Map[String,String] = {
@@ -92,12 +93,12 @@ object ImportZkOffsets extends Logging {
partOffsetsMap
}
- private def updateZkOffsets(zkClient: ZkClient, partitionOffsets: Map[String,String]): Unit = {
+ private def updateZkOffsets(zkUtils: ZkUtils, partitionOffsets: Map[String,String]): Unit = {
for ((partition, offset) <- partitionOffsets) {
debug("updating [" + partition + "] with offset [" + offset + "]")
try {
- ZkUtils.updatePersistentPath(zkClient, partition, offset.toString)
+ zkUtils.updatePersistentPath(partition, offset.toString)
} catch {
case e: Throwable => e.printStackTrace()
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala
index 9942686..95dd2a6 100755
--- a/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala
+++ b/core/src/main/scala/kafka/tools/UpdateOffsetsInZK.scala
@@ -23,6 +23,7 @@ import kafka.api.{PartitionOffsetRequestInfo, OffsetRequest}
import kafka.common.{TopicAndPartition, KafkaException}
import kafka.utils.{ZKGroupTopicDirs, ZkUtils, CoreUtils}
import org.apache.kafka.common.protocol.SecurityProtocol
+import org.apache.kafka.common.security.JaasUtils
import org.apache.kafka.common.utils.Utils
/**
@@ -36,17 +37,17 @@ object UpdateOffsetsInZK {
if(args.length < 3)
usage
val config = new ConsumerConfig(Utils.loadProps(args(1)))
- val zkClient = ZkUtils.createZkClient(config.zkConnect, config.zkSessionTimeoutMs,
- config.zkConnectionTimeoutMs)
+ val zkUtils = ZkUtils(config.zkConnect, config.zkSessionTimeoutMs,
+ config.zkConnectionTimeoutMs, JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)))
args(0) match {
- case Earliest => getAndSetOffsets(zkClient, OffsetRequest.EarliestTime, config, args(2))
- case Latest => getAndSetOffsets(zkClient, OffsetRequest.LatestTime, config, args(2))
+ case Earliest => getAndSetOffsets(zkUtils, OffsetRequest.EarliestTime, config, args(2))
+ case Latest => getAndSetOffsets(zkUtils, OffsetRequest.LatestTime, config, args(2))
case _ => usage
}
}
- private def getAndSetOffsets(zkClient: ZkClient, offsetOption: Long, config: ConsumerConfig, topic: String): Unit = {
- val partitionsPerTopicMap = ZkUtils.getPartitionsForTopics(zkClient, List(topic))
+ private def getAndSetOffsets(zkUtils: ZkUtils, offsetOption: Long, config: ConsumerConfig, topic: String): Unit = {
+ val partitionsPerTopicMap = zkUtils.getPartitionsForTopics(List(topic))
var partitions: Seq[Int] = Nil
partitionsPerTopicMap.get(topic) match {
@@ -56,7 +57,7 @@ object UpdateOffsetsInZK {
var numParts = 0
for (partition <- partitions) {
- val brokerHostingPartition = ZkUtils.getLeaderForPartition(zkClient, topic, partition)
+ val brokerHostingPartition = zkUtils.getLeaderForPartition(topic, partition)
val broker = brokerHostingPartition match {
case Some(b) => b
@@ -64,7 +65,7 @@ object UpdateOffsetsInZK {
"getOffsetsBefore request")
}
- ZkUtils.getBrokerInfo(zkClient, broker) match {
+ zkUtils.getBrokerInfo(broker) match {
case Some(brokerInfo) =>
val consumer = new SimpleConsumer(brokerInfo.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).host,
brokerInfo.getBrokerEndPoint(SecurityProtocol.PLAINTEXT).port,
@@ -75,7 +76,7 @@ object UpdateOffsetsInZK {
val topicDirs = new ZKGroupTopicDirs(config.groupId, topic)
println("updating partition " + partition + " with new offset: " + offset)
- ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + partition, offset.toString)
+ zkUtils.updatePersistentPath(topicDirs.consumerOffsetDir + "/" + partition, offset.toString)
numParts += 1
case None => throw new KafkaException("Broker information for broker id %d does not exist in ZK".format(broker))
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala
index db2721f..5a505c6 100644
--- a/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala
+++ b/core/src/main/scala/kafka/tools/VerifyConsumerRebalance.scala
@@ -19,6 +19,8 @@ package kafka.tools
import joptsimple.OptionParser
import org.I0Itec.zkclient.ZkClient
+import org.apache.kafka.common.security._
+
import kafka.utils.{Logging, ZKGroupTopicDirs, ZkUtils, CommandLineUtils}
object VerifyConsumerRebalance extends Logging {
@@ -46,15 +48,18 @@ object VerifyConsumerRebalance extends Logging {
val zkConnect = options.valueOf(zkConnectOpt)
val group = options.valueOf(groupOpt)
- var zkClient: ZkClient = null
+ var zkUtils: ZkUtils = null
try {
- zkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000)
+ zkUtils = ZkUtils(zkConnect,
+ 30000,
+ 30000,
+ JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)))
debug("zkConnect = %s; group = %s".format(zkConnect, group))
// check if the rebalancing operation succeeded.
try {
- if(validateRebalancingOperation(zkClient, group))
+ if(validateRebalancingOperation(zkUtils, group))
println("Rebalance operation successful !")
else
println("Rebalance operation failed !")
@@ -63,12 +68,12 @@ object VerifyConsumerRebalance extends Logging {
}
}
finally {
- if (zkClient != null)
- zkClient.close()
+ if (zkUtils != null)
+ zkUtils.close()
}
}
- private def validateRebalancingOperation(zkClient: ZkClient, group: String): Boolean = {
+ private def validateRebalancingOperation(zkUtils: ZkUtils, group: String): Boolean = {
info("Verifying rebalancing operation for consumer group " + group)
var rebalanceSucceeded: Boolean = true
/**
@@ -76,14 +81,14 @@ object VerifyConsumerRebalance extends Logging {
* This means that for each partition registered under /brokers/topics/[topic]/[broker-id], an owner exists
* under /consumers/[consumer_group]/owners/[topic]/[broker_id-partition_id]
*/
- val consumersPerTopicMap = ZkUtils.getConsumersPerTopic(zkClient, group, excludeInternalTopics = false)
- val partitionsPerTopicMap = ZkUtils.getPartitionsForTopics(zkClient, consumersPerTopicMap.keySet.toSeq)
+ val consumersPerTopicMap = zkUtils.getConsumersPerTopic(group, excludeInternalTopics = false)
+ val partitionsPerTopicMap = zkUtils.getPartitionsForTopics(consumersPerTopicMap.keySet.toSeq)
partitionsPerTopicMap.foreach { case (topic, partitions) =>
val topicDirs = new ZKGroupTopicDirs(group, topic)
info("Alive partitions for topic %s are %s ".format(topic, partitions.toString))
info("Alive consumers for topic %s => %s ".format(topic, consumersPerTopicMap.get(topic)))
- val partitionsWithOwners = ZkUtils.getChildrenParentMayNotExist(zkClient, topicDirs.consumerOwnerDir)
+ val partitionsWithOwners = zkUtils.getChildrenParentMayNotExist(topicDirs.consumerOwnerDir)
if(partitionsWithOwners.size == 0) {
error("No owners for any partitions for topic " + topic)
rebalanceSucceeded = false
@@ -100,7 +105,7 @@ object VerifyConsumerRebalance extends Logging {
}
// try reading the partition owner path for see if a valid consumer id exists there
val partitionOwnerPath = topicDirs.consumerOwnerDir + "/" + partition
- val partitionOwner = ZkUtils.readDataMaybeNull(zkClient, partitionOwnerPath)._1 match {
+ val partitionOwner = zkUtils.readDataMaybeNull(partitionOwnerPath)._1 match {
case Some(m) => m
case None => null
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/utils/ReplicationUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/ReplicationUtils.scala b/core/src/main/scala/kafka/utils/ReplicationUtils.scala
index d99629a..4074c0f 100644
--- a/core/src/main/scala/kafka/utils/ReplicationUtils.scala
+++ b/core/src/main/scala/kafka/utils/ReplicationUtils.scala
@@ -20,6 +20,7 @@ package kafka.utils
import kafka.api.LeaderAndIsr
import kafka.common.TopicAndPartition
import kafka.controller.{IsrChangeNotificationListener, LeaderIsrAndControllerEpoch}
+import kafka.utils.ZkUtils._
import org.I0Itec.zkclient.ZkClient
import org.apache.zookeeper.data.Stat
@@ -29,26 +30,26 @@ object ReplicationUtils extends Logging {
private val IsrChangeNotificationPrefix = "isr_change_"
- def updateLeaderAndIsr(zkClient: ZkClient, topic: String, partitionId: Int, newLeaderAndIsr: LeaderAndIsr, controllerEpoch: Int,
+ def updateLeaderAndIsr(zkUtils: ZkUtils, topic: String, partitionId: Int, newLeaderAndIsr: LeaderAndIsr, controllerEpoch: Int,
zkVersion: Int): (Boolean,Int) = {
debug("Updated ISR for partition [%s,%d] to %s".format(topic, partitionId, newLeaderAndIsr.isr.mkString(",")))
- val path = ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partitionId)
- val newLeaderData = ZkUtils.leaderAndIsrZkData(newLeaderAndIsr, controllerEpoch)
+ val path = getTopicPartitionLeaderAndIsrPath(topic, partitionId)
+ val newLeaderData = zkUtils.leaderAndIsrZkData(newLeaderAndIsr, controllerEpoch)
// use the epoch of the controller that made the leadership decision, instead of the current controller epoch
- val updatePersistentPath: (Boolean, Int) = ZkUtils.conditionalUpdatePersistentPath(zkClient, path, newLeaderData, zkVersion, Some(checkLeaderAndIsrZkData))
+ val updatePersistentPath: (Boolean, Int) = zkUtils.conditionalUpdatePersistentPath(path, newLeaderData, zkVersion, Some(checkLeaderAndIsrZkData))
updatePersistentPath
}
- def propagateIsrChanges(zkClient: ZkClient, isrChangeSet: Set[TopicAndPartition]): Unit = {
- val isrChangeNotificationPath: String = ZkUtils.createSequentialPersistentPath(
- zkClient, ZkUtils.IsrChangeNotificationPath + "/" + IsrChangeNotificationPrefix,
+ def propagateIsrChanges(zkUtils: ZkUtils, isrChangeSet: Set[TopicAndPartition]): Unit = {
+ val isrChangeNotificationPath: String = zkUtils.createSequentialPersistentPath(
+ ZkUtils.IsrChangeNotificationPath + "/" + IsrChangeNotificationPrefix,
generateIsrChangeJson(isrChangeSet))
debug("Added " + isrChangeNotificationPath + " for " + isrChangeSet)
}
- def checkLeaderAndIsrZkData(zkClient: ZkClient, path: String, expectedLeaderAndIsrInfo: String): (Boolean,Int) = {
+ def checkLeaderAndIsrZkData(zkUtils: ZkUtils, path: String, expectedLeaderAndIsrInfo: String): (Boolean,Int) = {
try {
- val writtenLeaderAndIsrInfo = ZkUtils.readDataMaybeNull(zkClient, path)
+ val writtenLeaderAndIsrInfo = zkUtils.readDataMaybeNull(path)
val writtenLeaderOpt = writtenLeaderAndIsrInfo._1
val writtenStat = writtenLeaderAndIsrInfo._2
val expectedLeader = parseLeaderAndIsr(expectedLeaderAndIsrInfo, path, writtenStat)
@@ -69,9 +70,9 @@ object ReplicationUtils extends Logging {
(false,-1)
}
- def getLeaderIsrAndEpochForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderIsrAndControllerEpoch] = {
- val leaderAndIsrPath = ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partition)
- val (leaderAndIsrOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, leaderAndIsrPath)
+ def getLeaderIsrAndEpochForPartition(zkUtils: ZkUtils, topic: String, partition: Int):Option[LeaderIsrAndControllerEpoch] = {
+ val leaderAndIsrPath = getTopicPartitionLeaderAndIsrPath(topic, partition)
+ val (leaderAndIsrOpt, stat) = zkUtils.readDataMaybeNull(leaderAndIsrPath)
leaderAndIsrOpt.flatMap(leaderAndIsrStr => parseLeaderAndIsr(leaderAndIsrStr, leaderAndIsrPath, stat))
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/utils/ZkUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala
index e1cfa2e..17e63e2 100644
--- a/core/src/main/scala/kafka/utils/ZkUtils.scala
+++ b/core/src/main/scala/kafka/utils/ZkUtils.scala
@@ -17,7 +17,12 @@
package kafka.utils
+import java.io.File
+import java.net.URI
+import java.security.URIParameter
+import javax.security.auth.login.Configuration
import java.util.concurrent.CountDownLatch
+
import kafka.cluster._
import kafka.consumer.{ConsumerThreadId, TopicCount}
import kafka.server.ConfigType
@@ -27,15 +32,19 @@ import org.I0Itec.zkclient.exception.{ZkException, ZkNodeExistsException, ZkNoNo
import org.I0Itec.zkclient.serialize.ZkSerializer
import org.apache.kafka.common.config.ConfigException
import org.apache.kafka.common.protocol.SecurityProtocol
-import collection._
+
+import org.apache.zookeeper.ZooDefs
+import scala.collection.JavaConverters._
+import scala.collection._
import kafka.api.LeaderAndIsr
-import org.apache.zookeeper.data.Stat
+import org.apache.zookeeper.data.{ACL, Stat}
import kafka.admin._
import kafka.common.{KafkaException, NoEpochForPartitionException}
import kafka.controller.ReassignedPartitionsContext
import kafka.controller.KafkaController
import kafka.controller.LeaderIsrAndControllerEpoch
import kafka.common.TopicAndPartition
+import kafka.utils.ZkUtils._
import org.apache.zookeeper.AsyncCallback.{DataCallback,StringCallback}
import org.apache.zookeeper.CreateMode
@@ -44,8 +53,7 @@ import org.apache.zookeeper.KeeperException.Code
import org.apache.zookeeper.ZooDefs.Ids
import org.apache.zookeeper.ZooKeeper
-
-object ZkUtils extends Logging {
+object ZkUtils {
val ConsumersPath = "/consumers"
val BrokerIdsPath = "/brokers/ids"
val BrokerTopicsPath = "/brokers/topics"
@@ -58,17 +66,51 @@ object ZkUtils extends Logging {
val IsrChangeNotificationPath = "/isr_change_notification"
val EntityConfigPath = "/config"
val EntityConfigChangesPath = "/config/changes"
- // These are persistent ZK paths that should exist on kafka broker startup.
- val persistentZkPaths = Seq(ConsumersPath,
- BrokerIdsPath,
- BrokerTopicsPath,
- EntityConfigChangesPath,
- ZkUtils.getEntityConfigRootPath(ConfigType.Topic),
- ZkUtils.getEntityConfigRootPath(ConfigType.Client),
- DeleteTopicsPath,
- BrokerSequenceIdPath,
- IsrChangeNotificationPath)
+
+ def apply(zkUrl: String, sessionTimeout: Int, connectionTimeout: Int, isZkSecurityEnabled: Boolean): ZkUtils = {
+ val (zkClient, zkConnection) = createZkClientAndConnection(zkUrl, sessionTimeout, connectionTimeout)
+ new ZkUtils(zkClient, zkConnection, isZkSecurityEnabled)
+ }
+
+ /*
+ * Used in tests
+ */
+ def apply(zkClient: ZkClient, isZkSecurityEnabled: Boolean): ZkUtils = {
+ new ZkUtils(zkClient, null, isZkSecurityEnabled)
+ }
+
+ def createZkClient(zkUrl: String, sessionTimeout: Int, connectionTimeout: Int): ZkClient = {
+ val zkClient = new ZkClient(zkUrl, sessionTimeout, connectionTimeout, ZKStringSerializer)
+ zkClient
+ }
+ def createZkClientAndConnection(zkUrl: String, sessionTimeout: Int, connectionTimeout: Int): (ZkClient, ZkConnection) = {
+ val zkConnection = new ZkConnection(zkUrl, sessionTimeout)
+ val zkClient = new ZkClient(zkConnection, connectionTimeout, ZKStringSerializer)
+ (zkClient, zkConnection)
+ }
+
+ def DefaultAcls(isSecure: Boolean): java.util.List[ACL] = if (isSecure) {
+ val list = ZooDefs.Ids.CREATOR_ALL_ACL
+ list.addAll(ZooDefs.Ids.READ_ACL_UNSAFE)
+ list
+ } else {
+ ZooDefs.Ids.OPEN_ACL_UNSAFE
+ }
+
+ def maybeDeletePath(zkUrl: String, dir: String) {
+ try {
+ val zk = createZkClient(zkUrl, 30*1000, 30*1000)
+ zk.deleteRecursive(dir)
+ zk.close()
+ } catch {
+ case _: Throwable => // swallow
+ }
+ }
+
+ /*
+ * Get calls that only depend on static paths
+ */
def getTopicPath(topic: String): String = {
BrokerTopicsPath + "/" + topic
}
@@ -77,6 +119,12 @@ object ZkUtils extends Logging {
getTopicPath(topic) + "/partitions"
}
+ def getTopicPartitionPath(topic: String, partitionId: Int): String =
+ getTopicPartitionsPath(topic) + "/" + partitionId
+
+ def getTopicPartitionLeaderAndIsrPath(topic: String, partitionId: Int): String =
+ getTopicPartitionPath(topic, partitionId) + "/" + "state"
+
def getEntityConfigRootPath(entityType: String): String =
EntityConfigPath + "/" + entityType
@@ -85,43 +133,54 @@ object ZkUtils extends Logging {
def getDeleteTopicPath(topic: String): String =
DeleteTopicsPath + "/" + topic
+}
+
+class ZkUtils(val zkClient: ZkClient,
+ val zkConnection: ZkConnection,
+ val isSecure: Boolean) extends Logging {
+ // These are persistent ZK paths that should exist on kafka broker startup.
+ val persistentZkPaths = Seq(ConsumersPath,
+ BrokerIdsPath,
+ BrokerTopicsPath,
+ EntityConfigChangesPath,
+ getEntityConfigRootPath(ConfigType.Topic),
+ getEntityConfigRootPath(ConfigType.Client),
+ DeleteTopicsPath,
+ BrokerSequenceIdPath,
+ IsrChangeNotificationPath)
- def getController(zkClient: ZkClient): Int = {
- readDataMaybeNull(zkClient, ControllerPath)._1 match {
+ val DefaultAcls: java.util.List[ACL] = ZkUtils.DefaultAcls(isSecure)
+
+ def getController(): Int = {
+ readDataMaybeNull(ControllerPath)._1 match {
case Some(controller) => KafkaController.parseControllerId(controller)
case None => throw new KafkaException("Controller doesn't exist")
}
}
- def getTopicPartitionPath(topic: String, partitionId: Int): String =
- getTopicPartitionsPath(topic) + "/" + partitionId
-
- def getTopicPartitionLeaderAndIsrPath(topic: String, partitionId: Int): String =
- getTopicPartitionPath(topic, partitionId) + "/" + "state"
-
- def getSortedBrokerList(zkClient: ZkClient): Seq[Int] =
- ZkUtils.getChildren(zkClient, BrokerIdsPath).map(_.toInt).sorted
+ def getSortedBrokerList(): Seq[Int] =
+ getChildren(BrokerIdsPath).map(_.toInt).sorted
- def getAllBrokersInCluster(zkClient: ZkClient): Seq[Broker] = {
- val brokerIds = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerIdsPath).sorted
- brokerIds.map(_.toInt).map(getBrokerInfo(zkClient, _)).filter(_.isDefined).map(_.get)
+ def getAllBrokersInCluster(): Seq[Broker] = {
+ val brokerIds = getChildrenParentMayNotExist(BrokerIdsPath).sorted
+ brokerIds.map(_.toInt).map(getBrokerInfo(_)).filter(_.isDefined).map(_.get)
}
- def getAllBrokerEndPointsForChannel(zkClient: ZkClient, protocolType: SecurityProtocol): Seq[BrokerEndPoint] = {
- getAllBrokersInCluster(zkClient).map(_.getBrokerEndPoint(protocolType))
+ def getAllBrokerEndPointsForChannel(protocolType: SecurityProtocol): Seq[BrokerEndPoint] = {
+ getAllBrokersInCluster().map(_.getBrokerEndPoint(protocolType))
}
- def getLeaderAndIsrForPartition(zkClient: ZkClient, topic: String, partition: Int):Option[LeaderAndIsr] = {
- ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition).map(_.leaderAndIsr)
+ def getLeaderAndIsrForPartition(topic: String, partition: Int):Option[LeaderAndIsr] = {
+ ReplicationUtils.getLeaderIsrAndEpochForPartition(this, topic, partition).map(_.leaderAndIsr)
}
- def setupCommonPaths(zkClient: ZkClient) {
+ def setupCommonPaths() {
for(path <- persistentZkPaths)
- makeSurePersistentPathExists(zkClient, path)
+ makeSurePersistentPathExists(path)
}
- def getLeaderForPartition(zkClient: ZkClient, topic: String, partition: Int): Option[Int] = {
- val leaderAndIsrOpt = readDataMaybeNull(zkClient, getTopicPartitionLeaderAndIsrPath(topic, partition))._1
+ def getLeaderForPartition(topic: String, partition: Int): Option[Int] = {
+ val leaderAndIsrOpt = readDataMaybeNull(getTopicPartitionLeaderAndIsrPath(topic, partition))._1
leaderAndIsrOpt match {
case Some(leaderAndIsr) =>
Json.parseFull(leaderAndIsr) match {
@@ -138,8 +197,8 @@ object ZkUtils extends Logging {
* leader fails after updating epoch in the leader path and before updating epoch in the ISR path, effectively some
* other broker will retry becoming leader with the same new epoch value.
*/
- def getEpochForPartition(zkClient: ZkClient, topic: String, partition: Int): Int = {
- val leaderAndIsrOpt = readDataMaybeNull(zkClient, getTopicPartitionLeaderAndIsrPath(topic, partition))._1
+ def getEpochForPartition(topic: String, partition: Int): Int = {
+ val leaderAndIsrOpt = readDataMaybeNull(getTopicPartitionLeaderAndIsrPath(topic, partition))._1
leaderAndIsrOpt match {
case Some(leaderAndIsr) =>
Json.parseFull(leaderAndIsr) match {
@@ -155,15 +214,15 @@ object ZkUtils extends Logging {
* users can provide brokerId in the config , inorder to avoid conflicts between zk generated
* seqId and config.brokerId we increment zk seqId by KafkaConfig.MaxReservedBrokerId.
*/
- def getBrokerSequenceId(zkClient: ZkClient, MaxReservedBrokerId: Int): Int = {
- getSequenceId(zkClient, BrokerSequenceIdPath) + MaxReservedBrokerId
+ def getBrokerSequenceId(MaxReservedBrokerId: Int): Int = {
+ getSequenceId(BrokerSequenceIdPath) + MaxReservedBrokerId
}
/**
* Gets the in-sync replicas (ISR) for a specific topic and partition
*/
- def getInSyncReplicasForPartition(zkClient: ZkClient, topic: String, partition: Int): Seq[Int] = {
- val leaderAndIsrOpt = readDataMaybeNull(zkClient, getTopicPartitionLeaderAndIsrPath(topic, partition))._1
+ def getInSyncReplicasForPartition(topic: String, partition: Int): Seq[Int] = {
+ val leaderAndIsrOpt = readDataMaybeNull(getTopicPartitionLeaderAndIsrPath(topic, partition))._1
leaderAndIsrOpt match {
case Some(leaderAndIsr) =>
Json.parseFull(leaderAndIsr) match {
@@ -177,8 +236,8 @@ object ZkUtils extends Logging {
/**
* Gets the assigned replicas (AR) for a specific topic and partition
*/
- def getReplicasForPartition(zkClient: ZkClient, topic: String, partition: Int): Seq[Int] = {
- val jsonPartitionMapOpt = readDataMaybeNull(zkClient, getTopicPath(topic))._1
+ def getReplicasForPartition(topic: String, partition: Int): Seq[Int] = {
+ val jsonPartitionMapOpt = readDataMaybeNull(getTopicPath(topic))._1
jsonPartitionMapOpt match {
case Some(jsonPartitionMap) =>
Json.parseFull(jsonPartitionMap) match {
@@ -204,21 +263,22 @@ object ZkUtils extends Logging {
* @param timeout
* @param jmxPort
*/
- def registerBrokerInZk(zkClient: ZkClient, zkConnection: ZkConnection, id: Int, host: String, port: Int, advertisedEndpoints: immutable.Map[SecurityProtocol, EndPoint], jmxPort: Int) {
- val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id
+ def registerBrokerInZk(id: Int, host: String, port: Int, advertisedEndpoints: immutable.Map[SecurityProtocol, EndPoint], jmxPort: Int) {
+ val brokerIdPath = BrokerIdsPath + "/" + id
val timestamp = SystemTime.milliseconds.toString
val brokerInfo = Json.encode(Map("version" -> 2, "host" -> host, "port" -> port, "endpoints"->advertisedEndpoints.values.map(_.connectionString).toArray, "jmx_port" -> jmxPort, "timestamp" -> timestamp))
- registerBrokerInZk(zkClient, zkConnection, brokerIdPath, brokerInfo)
+ registerBrokerInZk(brokerIdPath, brokerInfo)
info("Registered broker %d at path %s with addresses: %s".format(id, brokerIdPath, advertisedEndpoints.mkString(",")))
}
- private def registerBrokerInZk(zkClient: ZkClient, zkConnection: ZkConnection, brokerIdPath: String, brokerInfo: String) {
+ private def registerBrokerInZk(brokerIdPath: String, brokerInfo: String) {
try {
val zkCheckedEphemeral = new ZKCheckedEphemeral(brokerIdPath,
brokerInfo,
- zkConnection.getZookeeper)
+ zkConnection.getZookeeper,
+ isSecure)
zkCheckedEphemeral.create()
} catch {
case e: ZkNodeExistsException =>
@@ -250,31 +310,36 @@ object ZkUtils extends Logging {
/**
* make sure a persistent path exists in ZK. Create the path if not exist.
*/
- def makeSurePersistentPathExists(client: ZkClient, path: String) {
- if (!client.exists(path))
- ZkPath.createPersistent(client, path, true) //won't throw NoNodeException or NodeExistsException
+ def makeSurePersistentPathExists(path: String, acls: java.util.List[ACL] = DefaultAcls) {
+ //Consumer path is kept open as different consumers will write under this node.
+ val acl = if (path == null || path.isEmpty || path.equals(ConsumersPath)) {
+ ZooDefs.Ids.OPEN_ACL_UNSAFE
+ } else acls
+
+ if (!zkClient.exists(path))
+ ZkPath.createPersistent(zkClient, path, true, acl) //won't throw NoNodeException or NodeExistsException
}
/**
* create the parent path
*/
- private def createParentPath(client: ZkClient, path: String): Unit = {
+ private def createParentPath(path: String, acls: java.util.List[ACL] = DefaultAcls): Unit = {
val parentDir = path.substring(0, path.lastIndexOf('/'))
if (parentDir.length != 0) {
- ZkPath.createPersistent(client, parentDir, true)
+ ZkPath.createPersistent(zkClient, parentDir, true, acls)
}
}
/**
* Create an ephemeral node with the given path and data. Create parents if necessary.
*/
- private def createEphemeralPath(client: ZkClient, path: String, data: String): Unit = {
+ private def createEphemeralPath(path: String, data: String, acls: java.util.List[ACL] = DefaultAcls): Unit = {
try {
- ZkPath.createEphemeral(client, path, data)
+ ZkPath.createEphemeral(zkClient, path, data, acls)
} catch {
case e: ZkNoNodeException => {
- createParentPath(client, path)
- ZkPath.createEphemeral(client, path, data)
+ createParentPath(path)
+ ZkPath.createEphemeral(zkClient, path, data, acls)
}
}
}
@@ -283,15 +348,15 @@ object ZkUtils extends Logging {
* Create an ephemeral node with the given path and data.
* Throw NodeExistException if node already exists.
*/
- def createEphemeralPathExpectConflict(client: ZkClient, path: String, data: String): Unit = {
+ def createEphemeralPathExpectConflict(path: String, data: String, acls: java.util.List[ACL] = DefaultAcls): Unit = {
try {
- createEphemeralPath(client, path, data)
+ createEphemeralPath(path, data, acls)
} catch {
case e: ZkNodeExistsException => {
// this can happen when there is connection loss; make sure the data is what we intend to write
var storedData: String = null
try {
- storedData = readData(client, path)._1
+ storedData = readData(path)._1
} catch {
case e1: ZkNoNodeException => // the node disappeared; treat as if node existed and let caller handles this
case e2: Throwable => throw e2
@@ -311,19 +376,19 @@ object ZkUtils extends Logging {
/**
* Create an persistent node with the given path and data. Create parents if necessary.
*/
- def createPersistentPath(client: ZkClient, path: String, data: String = ""): Unit = {
+ def createPersistentPath(path: String, data: String = "", acls: java.util.List[ACL] = DefaultAcls): Unit = {
try {
- ZkPath.createPersistent(client, path, data)
+ ZkPath.createPersistent(zkClient, path, data, acls)
} catch {
case e: ZkNoNodeException => {
- createParentPath(client, path)
- ZkPath.createPersistent(client, path, data)
+ createParentPath(path)
+ ZkPath.createPersistent(zkClient, path, data, acls)
}
}
}
- def createSequentialPersistentPath(client: ZkClient, path: String, data: String = ""): String = {
- ZkPath.createPersistentSequential(client, path, data)
+ def createSequentialPersistentPath(path: String, data: String = "", acls: java.util.List[ACL] = DefaultAcls): String = {
+ ZkPath.createPersistentSequential(zkClient, path, data, acls)
}
/**
@@ -331,17 +396,17 @@ object ZkUtils extends Logging {
* create parrent directory if necessary. Never throw NodeExistException.
* Return the updated path zkVersion
*/
- def updatePersistentPath(client: ZkClient, path: String, data: String) = {
+ def updatePersistentPath(path: String, data: String, acls: java.util.List[ACL] = DefaultAcls) = {
try {
- client.writeData(path, data)
+ zkClient.writeData(path, data)
} catch {
case e: ZkNoNodeException => {
- createParentPath(client, path)
+ createParentPath(path)
try {
- ZkPath.createPersistent(client, path, data)
+ ZkPath.createPersistent(zkClient, path, data, acls)
} catch {
case e: ZkNodeExistsException =>
- client.writeData(path, data)
+ zkClient.writeData(path, data)
case e2: Throwable => throw e2
}
}
@@ -357,17 +422,17 @@ object ZkUtils extends Logging {
* since the previous update may have succeeded (but the stored zkVersion no longer matches the expected one).
* In this case, we will run the optionalChecker to further check if the previous write did indeed succeeded.
*/
- def conditionalUpdatePersistentPath(client: ZkClient, path: String, data: String, expectVersion: Int,
- optionalChecker:Option[(ZkClient, String, String) => (Boolean,Int)] = None): (Boolean, Int) = {
+ def conditionalUpdatePersistentPath(path: String, data: String, expectVersion: Int,
+ optionalChecker:Option[(ZkUtils, String, String) => (Boolean,Int)] = None): (Boolean, Int) = {
try {
- val stat = client.writeDataReturnStat(path, data, expectVersion)
+ val stat = zkClient.writeDataReturnStat(path, data, expectVersion)
debug("Conditional update of path %s with value %s and expected version %d succeeded, returning the new version: %d"
.format(path, data, expectVersion, stat.getVersion))
(true, stat.getVersion)
} catch {
case e1: ZkBadVersionException =>
optionalChecker match {
- case Some(checker) => return checker(client, path, data)
+ case Some(checker) => return checker(this, path, data)
case _ => debug("Checker method is not passed skipping zkData match")
}
warn("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, data,
@@ -384,9 +449,9 @@ object ZkUtils extends Logging {
* Conditional update the persistent path data, return (true, newVersion) if it succeeds, otherwise (the current
* version is not the expected version, etc.) return (false, -1). If path doesn't exist, throws ZkNoNodeException
*/
- def conditionalUpdatePersistentPathIfExists(client: ZkClient, path: String, data: String, expectVersion: Int): (Boolean, Int) = {
+ def conditionalUpdatePersistentPathIfExists(path: String, data: String, expectVersion: Int): (Boolean, Int) = {
try {
- val stat = client.writeDataReturnStat(path, data, expectVersion)
+ val stat = zkClient.writeDataReturnStat(path, data, expectVersion)
debug("Conditional update of path %s with value %s and expected version %d succeeded, returning the new version: %d"
.format(path, data, expectVersion, stat.getVersion))
(true, stat.getVersion)
@@ -403,21 +468,21 @@ object ZkUtils extends Logging {
* Update the value of a persistent node with the given path and data.
* create parrent directory if necessary. Never throw NodeExistException.
*/
- def updateEphemeralPath(client: ZkClient, path: String, data: String): Unit = {
+ def updateEphemeralPath(path: String, data: String, acls: java.util.List[ACL] = DefaultAcls): Unit = {
try {
- client.writeData(path, data)
+ zkClient.writeData(path, data)
} catch {
case e: ZkNoNodeException => {
- createParentPath(client, path)
- ZkPath.createEphemeral(client, path, data)
+ createParentPath(path)
+ ZkPath.createEphemeral(zkClient, path, data, acls)
}
case e2: Throwable => throw e2
}
}
- def deletePath(client: ZkClient, path: String): Boolean = {
+ def deletePath(path: String): Boolean = {
try {
- client.delete(path)
+ zkClient.delete(path)
} catch {
case e: ZkNoNodeException =>
// this can happen during a connection loss event, return normally
@@ -427,9 +492,9 @@ object ZkUtils extends Logging {
}
}
- def deletePathRecursive(client: ZkClient, path: String) {
+ def deletePathRecursive(path: String) {
try {
- client.deleteRecursive(path)
+ zkClient.deleteRecursive(path)
} catch {
case e: ZkNoNodeException =>
// this can happen during a connection loss event, return normally
@@ -438,26 +503,16 @@ object ZkUtils extends Logging {
}
}
- def maybeDeletePath(zkUrl: String, dir: String) {
- try {
- val zk = createZkClient(zkUrl, 30*1000, 30*1000)
- zk.deleteRecursive(dir)
- zk.close()
- } catch {
- case _: Throwable => // swallow
- }
- }
-
- def readData(client: ZkClient, path: String): (String, Stat) = {
+ def readData(path: String): (String, Stat) = {
val stat: Stat = new Stat()
- val dataStr: String = client.readData(path, stat)
+ val dataStr: String = zkClient.readData(path, stat)
(dataStr, stat)
}
- def readDataMaybeNull(client: ZkClient, path: String): (Option[String], Stat) = {
+ def readDataMaybeNull(path: String): (Option[String], Stat) = {
val stat: Stat = new Stat()
val dataAndStat = try {
- (Some(client.readData(path, stat)), stat)
+ (Some(zkClient.readData(path, stat)), stat)
} catch {
case e: ZkNoNodeException =>
(None, stat)
@@ -466,17 +521,17 @@ object ZkUtils extends Logging {
dataAndStat
}
- def getChildren(client: ZkClient, path: String): Seq[String] = {
+ def getChildren(path: String): Seq[String] = {
import scala.collection.JavaConversions._
// triggers implicit conversion from java list to scala Seq
- client.getChildren(path)
+ zkClient.getChildren(path)
}
- def getChildrenParentMayNotExist(client: ZkClient, path: String): Seq[String] = {
+ def getChildrenParentMayNotExist(path: String): Seq[String] = {
import scala.collection.JavaConversions._
// triggers implicit conversion from java list to scala Seq
try {
- client.getChildren(path)
+ zkClient.getChildren(path)
} catch {
case e: ZkNoNodeException => Nil
case e2: Throwable => throw e2
@@ -486,15 +541,15 @@ object ZkUtils extends Logging {
/**
* Check if the given path exists
*/
- def pathExists(client: ZkClient, path: String): Boolean = {
- client.exists(path)
+ def pathExists(path: String): Boolean = {
+ zkClient.exists(path)
}
- def getCluster(zkClient: ZkClient) : Cluster = {
+ def getCluster() : Cluster = {
val cluster = new Cluster
- val nodes = getChildrenParentMayNotExist(zkClient, BrokerIdsPath)
+ val nodes = getChildrenParentMayNotExist(BrokerIdsPath)
for (node <- nodes) {
- val brokerZKString = readData(zkClient, BrokerIdsPath + "/" + node)._1
+ val brokerZKString = readData(BrokerIdsPath + "/" + node)._1
cluster.add(Broker.createBroker(node.toInt, brokerZKString))
}
cluster
@@ -504,7 +559,7 @@ object ZkUtils extends Logging {
: mutable.Map[TopicAndPartition, LeaderIsrAndControllerEpoch] = {
val ret = new mutable.HashMap[TopicAndPartition, LeaderIsrAndControllerEpoch]
for(topicAndPartition <- topicAndPartitions) {
- ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic, topicAndPartition.partition) match {
+ ReplicationUtils.getLeaderIsrAndEpochForPartition(this, topicAndPartition.topic, topicAndPartition.partition) match {
case Some(leaderIsrAndControllerEpoch) => ret.put(topicAndPartition, leaderIsrAndControllerEpoch)
case None =>
}
@@ -512,10 +567,10 @@ object ZkUtils extends Logging {
ret
}
- def getReplicaAssignmentForTopics(zkClient: ZkClient, topics: Seq[String]): mutable.Map[TopicAndPartition, Seq[Int]] = {
+ def getReplicaAssignmentForTopics(topics: Seq[String]): mutable.Map[TopicAndPartition, Seq[Int]] = {
val ret = new mutable.HashMap[TopicAndPartition, Seq[Int]]
topics.foreach { topic =>
- val jsonPartitionMapOpt = readDataMaybeNull(zkClient, getTopicPath(topic))._1
+ val jsonPartitionMapOpt = readDataMaybeNull(getTopicPath(topic))._1
jsonPartitionMapOpt match {
case Some(jsonPartitionMap) =>
Json.parseFull(jsonPartitionMap) match {
@@ -536,10 +591,10 @@ object ZkUtils extends Logging {
ret
}
- def getPartitionAssignmentForTopics(zkClient: ZkClient, topics: Seq[String]): mutable.Map[String, collection.Map[Int, Seq[Int]]] = {
+ def getPartitionAssignmentForTopics(topics: Seq[String]): mutable.Map[String, collection.Map[Int, Seq[Int]]] = {
val ret = new mutable.HashMap[String, Map[Int, Seq[Int]]]()
topics.foreach{ topic =>
- val jsonPartitionMapOpt = readDataMaybeNull(zkClient, getTopicPath(topic))._1
+ val jsonPartitionMapOpt = readDataMaybeNull(getTopicPath(topic))._1
val partitionMap = jsonPartitionMapOpt match {
case Some(jsonPartitionMap) =>
Json.parseFull(jsonPartitionMap) match {
@@ -559,8 +614,8 @@ object ZkUtils extends Logging {
ret
}
- def getPartitionsForTopics(zkClient: ZkClient, topics: Seq[String]): mutable.Map[String, Seq[Int]] = {
- getPartitionAssignmentForTopics(zkClient, topics).map { topicAndPartitionMap =>
+ def getPartitionsForTopics(topics: Seq[String]): mutable.Map[String, Seq[Int]] = {
+ getPartitionAssignmentForTopics(topics).map { topicAndPartitionMap =>
val topic = topicAndPartitionMap._1
val partitionMap = topicAndPartitionMap._2
debug("partition assignment of /brokers/topics/%s is %s".format(topic, partitionMap))
@@ -568,9 +623,9 @@ object ZkUtils extends Logging {
}
}
- def getPartitionsBeingReassigned(zkClient: ZkClient): Map[TopicAndPartition, ReassignedPartitionsContext] = {
+ def getPartitionsBeingReassigned(): Map[TopicAndPartition, ReassignedPartitionsContext] = {
// read the partitions and their new replica list
- val jsonPartitionMapOpt = readDataMaybeNull(zkClient, ReassignPartitionsPath)._1
+ val jsonPartitionMapOpt = readDataMaybeNull(ReassignPartitionsPath)._1
jsonPartitionMapOpt match {
case Some(jsonPartitionMap) =>
val reassignedPartitions = parsePartitionReassignmentData(jsonPartitionMap)
@@ -626,53 +681,53 @@ object ZkUtils extends Logging {
"replicas" -> e._2))))
}
- def updatePartitionReassignmentData(zkClient: ZkClient, partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]]) {
- val zkPath = ZkUtils.ReassignPartitionsPath
+ def updatePartitionReassignmentData(partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]]) {
+ val zkPath = ReassignPartitionsPath
partitionsToBeReassigned.size match {
case 0 => // need to delete the /admin/reassign_partitions path
- deletePath(zkClient, zkPath)
+ deletePath(zkPath)
info("No more partitions need to be reassigned. Deleting zk path %s".format(zkPath))
case _ =>
val jsonData = getPartitionReassignmentZkData(partitionsToBeReassigned)
try {
- updatePersistentPath(zkClient, zkPath, jsonData)
+ updatePersistentPath(zkPath, jsonData)
debug("Updated partition reassignment path with %s".format(jsonData))
} catch {
case nne: ZkNoNodeException =>
- ZkUtils.createPersistentPath(zkClient, zkPath, jsonData)
+ createPersistentPath(zkPath, jsonData)
debug("Created path %s with %s for partition reassignment".format(zkPath, jsonData))
case e2: Throwable => throw new AdminOperationException(e2.toString)
}
}
}
- def getPartitionsUndergoingPreferredReplicaElection(zkClient: ZkClient): Set[TopicAndPartition] = {
+ def getPartitionsUndergoingPreferredReplicaElection(): Set[TopicAndPartition] = {
// read the partitions and their new replica list
- val jsonPartitionListOpt = readDataMaybeNull(zkClient, PreferredReplicaLeaderElectionPath)._1
+ val jsonPartitionListOpt = readDataMaybeNull(PreferredReplicaLeaderElectionPath)._1
jsonPartitionListOpt match {
case Some(jsonPartitionList) => PreferredReplicaLeaderElectionCommand.parsePreferredReplicaElectionData(jsonPartitionList)
case None => Set.empty[TopicAndPartition]
}
}
- def deletePartition(zkClient : ZkClient, brokerId: Int, topic: String) {
+ def deletePartition(brokerId: Int, topic: String) {
val brokerIdPath = BrokerIdsPath + "/" + brokerId
zkClient.delete(brokerIdPath)
val brokerPartTopicPath = BrokerTopicsPath + "/" + topic + "/" + brokerId
zkClient.delete(brokerPartTopicPath)
}
- def getConsumersInGroup(zkClient: ZkClient, group: String): Seq[String] = {
+ def getConsumersInGroup(group: String): Seq[String] = {
val dirs = new ZKGroupDirs(group)
- getChildren(zkClient, dirs.consumerRegistryDir)
+ getChildren(dirs.consumerRegistryDir)
}
- def getConsumersPerTopic(zkClient: ZkClient, group: String, excludeInternalTopics: Boolean) : mutable.Map[String, List[ConsumerThreadId]] = {
+ def getConsumersPerTopic(group: String, excludeInternalTopics: Boolean) : mutable.Map[String, List[ConsumerThreadId]] = {
val dirs = new ZKGroupDirs(group)
- val consumers = getChildrenParentMayNotExist(zkClient, dirs.consumerRegistryDir)
+ val consumers = getChildrenParentMayNotExist(dirs.consumerRegistryDir)
val consumersPerTopicMap = new mutable.HashMap[String, List[ConsumerThreadId]]
for (consumer <- consumers) {
- val topicCount = TopicCount.constructTopicCount(group, consumer, zkClient, excludeInternalTopics)
+ val topicCount = TopicCount.constructTopicCount(group, consumer, this, excludeInternalTopics)
for ((topic, consumerThreadIdSet) <- topicCount.getConsumerThreadIdsPerTopic) {
for (consumerThreadId <- consumerThreadIdSet)
consumersPerTopicMap.get(topic) match {
@@ -693,8 +748,8 @@ object ZkUtils extends Logging {
* @param zkClient The zookeeper client connection
* @return An optional Broker object encapsulating the broker metadata
*/
- def getBrokerInfo(zkClient: ZkClient, brokerId: Int): Option[Broker] = {
- ZkUtils.readDataMaybeNull(zkClient, ZkUtils.BrokerIdsPath + "/" + brokerId)._1 match {
+ def getBrokerInfo(brokerId: Int): Option[Broker] = {
+ readDataMaybeNull(BrokerIdsPath + "/" + brokerId)._1 match {
case Some(brokerInfo) => Some(Broker.createBroker(brokerId, brokerInfo))
case None => None
}
@@ -705,27 +760,28 @@ object ZkUtils extends Logging {
* It uses the stat returned by the zookeeper and return the version. Every time
* client updates the path stat.version gets incremented
*/
- def getSequenceId(client: ZkClient, path: String): Int = {
+ def getSequenceId(path: String, acls: java.util.List[ACL] = DefaultAcls): Int = {
try {
- val stat = client.writeDataReturnStat(path, "", -1)
+ val stat = zkClient.writeDataReturnStat(path, "", -1)
stat.getVersion
} catch {
case e: ZkNoNodeException => {
- createParentPath(client, BrokerSequenceIdPath)
+ createParentPath(BrokerSequenceIdPath, acls)
try {
- client.createPersistent(BrokerSequenceIdPath, "")
+ import scala.collection.JavaConversions._
+ zkClient.createPersistent(BrokerSequenceIdPath, "", acls)
0
} catch {
case e: ZkNodeExistsException =>
- val stat = client.writeDataReturnStat(BrokerSequenceIdPath, "", -1)
+ val stat = zkClient.writeDataReturnStat(BrokerSequenceIdPath, "", -1)
stat.getVersion
}
}
}
}
- def getAllTopics(zkClient: ZkClient): Seq[String] = {
- val topics = ZkUtils.getChildrenParentMayNotExist(zkClient, BrokerTopicsPath)
+ def getAllTopics(): Seq[String] = {
+ val topics = getChildrenParentMayNotExist(BrokerTopicsPath)
if(topics == null)
Seq.empty[String]
else
@@ -735,53 +791,48 @@ object ZkUtils extends Logging {
/**
* Returns all the entities whose configs have been overridden.
*/
- def getAllEntitiesWithConfig(zkClient: ZkClient, entityType: String): Seq[String] = {
- val entities = ZkUtils.getChildrenParentMayNotExist(zkClient, getEntityConfigRootPath(entityType))
+ def getAllEntitiesWithConfig(entityType: String): Seq[String] = {
+ val entities = getChildrenParentMayNotExist(getEntityConfigRootPath(entityType))
if(entities == null)
Seq.empty[String]
else
entities
}
- def getAllPartitions(zkClient: ZkClient): Set[TopicAndPartition] = {
- val topics = ZkUtils.getChildrenParentMayNotExist(zkClient, BrokerTopicsPath)
+ def getAllPartitions(): Set[TopicAndPartition] = {
+ val topics = getChildrenParentMayNotExist(BrokerTopicsPath)
if(topics == null) Set.empty[TopicAndPartition]
else {
topics.map { topic =>
- getChildren(zkClient, getTopicPartitionsPath(topic)).map(_.toInt).map(TopicAndPartition(topic, _))
+ getChildren(getTopicPartitionsPath(topic)).map(_.toInt).map(TopicAndPartition(topic, _))
}.flatten.toSet
}
}
- def getConsumerGroups(zkClient: ZkClient) = {
- ZkUtils.getChildren(zkClient, ConsumersPath)
+ def getConsumerGroups() = {
+ getChildren(ConsumersPath)
}
- def getTopicsByConsumerGroup(zkClient: ZkClient,consumerGroup:String) = {
- ZkUtils.getChildrenParentMayNotExist(zkClient, new ZKGroupDirs(consumerGroup).consumerGroupOwnersDir)
+ def getTopicsByConsumerGroup(consumerGroup:String) = {
+ getChildrenParentMayNotExist(new ZKGroupDirs(consumerGroup).consumerGroupOwnersDir)
}
- def getAllConsumerGroupsForTopic(zkClient: ZkClient, topic: String): Set[String] = {
- val groups = ZkUtils.getChildrenParentMayNotExist(zkClient, ConsumersPath)
+ def getAllConsumerGroupsForTopic(topic: String): Set[String] = {
+ val groups = getChildrenParentMayNotExist(ConsumersPath)
if (groups == null) Set.empty
else {
groups.foldLeft(Set.empty[String]) {(consumerGroupsForTopic, group) =>
- val topics = getChildren(zkClient, new ZKGroupDirs(group).consumerGroupOffsetsDir)
+ val topics = getChildren(new ZKGroupDirs(group).consumerGroupOffsetsDir)
if (topics.contains(topic)) consumerGroupsForTopic + group
else consumerGroupsForTopic
}
}
}
-
- def createZkClient(zkUrl: String, sessionTimeout: Int, connectionTimeout: Int): ZkClient = {
- val zkClient = new ZkClient(zkUrl, sessionTimeout, connectionTimeout, ZKStringSerializer)
- zkClient
- }
-
- def createZkClientAndConnection(zkUrl: String, sessionTimeout: Int, connectionTimeout: Int): (ZkClient, ZkConnection) = {
- val zkConnection = new ZkConnection(zkUrl, sessionTimeout)
- val zkClient = new ZkClient(zkConnection, connectionTimeout, ZKStringSerializer)
- (zkClient, zkConnection)
+
+ def close() {
+ if(zkClient != null) {
+ zkClient.close()
+ }
}
}
@@ -800,7 +851,7 @@ private object ZKStringSerializer extends ZkSerializer {
}
class ZKGroupDirs(val group: String) {
- def consumerDir = ZkUtils.ConsumersPath
+ def consumerDir = ConsumersPath
def consumerGroupDir = consumerDir + "/" + group
def consumerRegistryDir = consumerGroupDir + "/ids"
def consumerGroupOffsetsDir = consumerGroupDir + "/offsets"
@@ -829,6 +880,7 @@ class ZKConfig(props: VerifiableProperties) {
object ZkPath {
@volatile private var isNamespacePresent: Boolean = false
+ import scala.collection.JavaConversions._
def checkNamespace(client: ZkClient) {
if(isNamespacePresent)
@@ -844,24 +896,24 @@ object ZkPath {
isNamespacePresent = false
}
- def createPersistent(client: ZkClient, path: String, data: Object) {
+ def createPersistent(client: ZkClient, path: String, data: Object, acls: java.util.List[ACL]) {
checkNamespace(client)
- client.createPersistent(path, data)
+ client.createPersistent(path, data, acls)
}
- def createPersistent(client: ZkClient, path: String, createParents: Boolean) {
+ def createPersistent(client: ZkClient, path: String, createParents: Boolean, acls: java.util.List[ACL]) {
checkNamespace(client)
- client.createPersistent(path, createParents)
+ client.createPersistent(path, createParents, acls)
}
- def createEphemeral(client: ZkClient, path: String, data: Object) {
+ def createEphemeral(client: ZkClient, path: String, data: Object, acls: java.util.List[ACL]) {
checkNamespace(client)
- client.createEphemeral(path, data)
+ client.createEphemeral(path, data, acls)
}
- def createPersistentSequential(client: ZkClient, path: String, data: Object): String = {
+ def createPersistentSequential(client: ZkClient, path: String, data: Object, acls: java.util.List[ACL]): String = {
checkNamespace(client)
- client.createPersistentSequential(path, data)
+ client.createPersistentSequential(path, data, acls)
}
}
@@ -876,7 +928,8 @@ object ZkPath {
class ZKCheckedEphemeral(path: String,
data: String,
- zkHandle: ZooKeeper) extends Logging {
+ zkHandle: ZooKeeper,
+ isSecure: Boolean) extends Logging {
private val createCallback = new CreateCallback
private val getDataCallback = new GetDataCallback
val latch: CountDownLatch = new CountDownLatch(1)
@@ -936,7 +989,7 @@ class ZKCheckedEphemeral(path: String,
private def createEphemeral() {
zkHandle.create(path,
ZKStringSerializer.serialize(data),
- Ids.OPEN_ACL_UNSAFE,
+ DefaultAcls(isSecure),
CreateMode.EPHEMERAL,
createCallback,
null)
@@ -949,7 +1002,7 @@ class ZKCheckedEphemeral(path: String,
} else {
zkHandle.create(prefix,
new Array[Byte](0),
- Ids.OPEN_ACL_UNSAFE,
+ ZkUtils. DefaultAcls(isSecure),
CreateMode.PERSISTENT,
new StringCallback() {
def processResult(rc : Int,
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
index 16d7c26..db610c1 100644
--- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
@@ -61,7 +61,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging {
super.setUp()
// create the test topic with all the brokers as replicas
- TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers)
+ TestUtils.createTopic(this.zkUtils, topic, 1, serverCount, this.servers)
}
@Test
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/integration/kafka/api/ConsumerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala
index a64c2f3..d973d9a 100644
--- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala
@@ -63,7 +63,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
super.setUp()
// create the test topic with all the brokers as replicas
- TestUtils.createTopic(this.zkClient, topic, 2, serverCount, this.servers)
+ TestUtils.createTopic(this.zkUtils, topic, 2, serverCount, this.servers)
}
@Test
@@ -147,7 +147,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
@Test
def testAutoCommitOnRebalance() {
val topic2 = "topic2"
- TestUtils.createTopic(this.zkClient, topic2, 2, serverCount, this.servers)
+ TestUtils.createTopic(this.zkUtils, topic2, 2, serverCount, this.servers)
this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true")
val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
@@ -186,17 +186,17 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
sendRecords(numRecords)
val topic1: String = "tblablac" // matches subscribed pattern
- TestUtils.createTopic(this.zkClient, topic1, 2, serverCount, this.servers)
+ TestUtils.createTopic(this.zkUtils, topic1, 2, serverCount, this.servers)
sendRecords(1000, new TopicPartition(topic1, 0))
sendRecords(1000, new TopicPartition(topic1, 1))
val topic2: String = "tblablak" // does not match subscribed pattern
- TestUtils.createTopic(this.zkClient, topic2, 2, serverCount, this.servers)
+ TestUtils.createTopic(this.zkUtils, topic2, 2, serverCount, this.servers)
sendRecords(1000, new TopicPartition(topic2, 0))
sendRecords(1000, new TopicPartition(topic2, 1))
val topic3: String = "tblab1" // does not match subscribed pattern
- TestUtils.createTopic(this.zkClient, topic3, 2, serverCount, this.servers)
+ TestUtils.createTopic(this.zkUtils, topic3, 2, serverCount, this.servers)
sendRecords(1000, new TopicPartition(topic3, 0))
sendRecords(1000, new TopicPartition(topic3, 1))
@@ -218,7 +218,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
}, s"Expected partitions ${subscriptions.asJava} but actually got ${this.consumers(0).assignment()}")
val topic4: String = "tsomec" // matches subscribed pattern
- TestUtils.createTopic(this.zkClient, topic4, 2, serverCount, this.servers)
+ TestUtils.createTopic(this.zkUtils, topic4, 2, serverCount, this.servers)
sendRecords(1000, new TopicPartition(topic4, 0))
sendRecords(1000, new TopicPartition(topic4, 1))
@@ -242,7 +242,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
sendRecords(numRecords)
val topic1: String = "tblablac" // matches subscribed pattern
- TestUtils.createTopic(this.zkClient, topic1, 2, serverCount, this.servers)
+ TestUtils.createTopic(this.zkUtils, topic1, 2, serverCount, this.servers)
sendRecords(1000, new TopicPartition(topic1, 0))
sendRecords(1000, new TopicPartition(topic1, 1))
@@ -383,7 +383,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
@Test
def testPartitionsFor() {
val numParts = 2
- TestUtils.createTopic(this.zkClient, "part-test", numParts, 1, this.servers)
+ TestUtils.createTopic(this.zkUtils, "part-test", numParts, 1, this.servers)
val parts = this.consumers(0).partitionsFor("part-test")
assertNotNull(parts)
assertEquals(2, parts.size)
@@ -396,9 +396,9 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
val topic1: String = "part-test-topic-1"
val topic2: String = "part-test-topic-2"
val topic3: String = "part-test-topic-3"
- TestUtils.createTopic(this.zkClient, topic1, numParts, 1, this.servers)
- TestUtils.createTopic(this.zkClient, topic2, numParts, 1, this.servers)
- TestUtils.createTopic(this.zkClient, topic3, numParts, 1, this.servers)
+ TestUtils.createTopic(this.zkUtils, topic1, numParts, 1, this.servers)
+ TestUtils.createTopic(this.zkUtils, topic2, numParts, 1, this.servers)
+ TestUtils.createTopic(this.zkUtils, topic3, numParts, 1, this.servers)
val topics = this.consumers.head.listTopics()
assertNotNull(topics)
@@ -475,7 +475,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
this.consumers(0).assignment == subscriptions.asJava
}, s"Expected partitions ${subscriptions.asJava} but actually got ${this.consumers(0).assignment}")
- TestUtils.createTopic(this.zkClient, otherTopic, 2, serverCount, this.servers)
+ TestUtils.createTopic(this.zkUtils, otherTopic, 2, serverCount, this.servers)
this.consumers(0).subscribe(List(topic, otherTopic))
TestUtils.waitUntilTrue(() => {
this.consumers(0).poll(50)
@@ -486,7 +486,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
@Test
def testShrinkingTopicSubscriptions() {
val otherTopic = "other"
- TestUtils.createTopic(this.zkClient, otherTopic, 2, serverCount, this.servers)
+ TestUtils.createTopic(this.zkUtils, otherTopic, 2, serverCount, this.servers)
val subscriptions = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1), new TopicPartition(otherTopic, 0), new TopicPartition(otherTopic, 1))
val shrunkenSubscriptions = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1))
this.consumers(0).subscribe(List(topic, otherTopic))
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
index 8080b08..77fcd8b 100644
--- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
+++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
@@ -66,7 +66,7 @@ trait IntegrationTestHarness extends KafkaServerTestHarness {
consumers += new KafkaConsumer(consumerConfig)
// create the consumer offset topic
- TestUtils.createTopic(zkClient, ConsumerCoordinator.OffsetsTopicName,
+ TestUtils.createTopic(zkUtils, ConsumerCoordinator.OffsetsTopicName,
serverConfig.getProperty(KafkaConfig.OffsetsTopicPartitionsProp).toInt,
serverConfig.getProperty(KafkaConfig.OffsetsTopicReplicationFactorProp).toInt,
servers,
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala
index 2dbb9dc..29e146e 100644
--- a/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ProducerBounceTest.scala
@@ -87,7 +87,7 @@ class ProducerBounceTest extends KafkaServerTestHarness {
@Test
def testBrokerFailure() {
val numPartitions = 3
- val leaders = TestUtils.createTopic(zkClient, topic1, numPartitions, numServers, servers)
+ val leaders = TestUtils.createTopic(zkUtils, topic1, numPartitions, numServers, servers)
assertTrue("Leader of all partitions of the topic should exist", leaders.values.forall(leader => leader.isDefined))
val scheduler = new ProducerScheduler()
@@ -107,7 +107,7 @@ class ProducerBounceTest extends KafkaServerTestHarness {
assertTrue(scheduler.failed == false)
// Make sure the leader still exists after bouncing brokers
- (0 until numPartitions).foreach(partition => TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic1, partition))
+ (0 until numPartitions).foreach(partition => TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, topic1, partition))
}
scheduler.shutdown
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala
index 87db255..0d401f7 100755
--- a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala
@@ -79,7 +79,7 @@ class ProducerCompressionTest(compression: String) extends ZooKeeperTestHarness
try {
// create topic
- TestUtils.createTopic(zkClient, topic, 1, 1, List(server))
+ TestUtils.createTopic(zkUtils, topic, 1, 1, List(server))
val partition = 0
// prepare the messages
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
index e90818a..8ba7fad 100644
--- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
@@ -84,7 +84,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
@Test
def testTooLargeRecordWithAckZero() {
// create topic
- TestUtils.createTopic(zkClient, topic1, 1, numServers, servers)
+ TestUtils.createTopic(zkUtils, topic1, 1, numServers, servers)
// send a too-large record
val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1))
@@ -97,7 +97,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
@Test
def testTooLargeRecordWithAckOne() {
// create topic
- TestUtils.createTopic(zkClient, topic1, 1, numServers, servers)
+ TestUtils.createTopic(zkUtils, topic1, 1, numServers, servers)
// send a too-large record
val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1))
@@ -131,7 +131,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
@Test
def testWrongBrokerList() {
// create topic
- TestUtils.createTopic(zkClient, topic1, 1, numServers, servers)
+ TestUtils.createTopic(zkUtils, topic1, 1, numServers, servers)
// producer with incorrect broker list
producer4 = TestUtils.createNewProducer("localhost:8686,localhost:4242", acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize)
@@ -149,7 +149,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
@Test
def testInvalidPartition() {
// create topic
- TestUtils.createTopic(zkClient, topic1, 1, numServers, servers)
+ TestUtils.createTopic(zkUtils, topic1, 1, numServers, servers)
// create a record with incorrect partition id, send should fail
val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, new Integer(1), "key".getBytes, "value".getBytes)
@@ -170,7 +170,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
@Test
def testSendAfterClosed() {
// create topic
- TestUtils.createTopic(zkClient, topic1, 1, numServers, servers)
+ TestUtils.createTopic(zkUtils, topic1, 1, numServers, servers)
val record = new ProducerRecord[Array[Byte],Array[Byte]](topic1, null, "key".getBytes, "value".getBytes)
@@ -209,7 +209,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
val topicProps = new Properties()
topicProps.put("min.insync.replicas",(numServers+1).toString)
- TestUtils.createTopic(zkClient, topicName, 1, numServers, servers, topicProps)
+ TestUtils.createTopic(zkUtils, topicName, 1, numServers, servers, topicProps)
val record = new ProducerRecord[Array[Byte],Array[Byte]](topicName, null, "key".getBytes, "value".getBytes)
try {
@@ -229,7 +229,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
val topicProps = new Properties()
topicProps.put("min.insync.replicas",numServers.toString)
- TestUtils.createTopic(zkClient, topicName, 1, numServers, servers,topicProps)
+ TestUtils.createTopic(zkUtils, topicName, 1, numServers, servers,topicProps)
val record = new ProducerRecord[Array[Byte],Array[Byte]](topicName, null, "key".getBytes, "value".getBytes)
// this should work with all brokers up and running
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala
index 637d6f3..3aef172 100644
--- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala
@@ -91,7 +91,7 @@ class ProducerSendTest extends KafkaServerTestHarness {
try {
// create topic
- TestUtils.createTopic(zkClient, topic, 1, 2, servers)
+ TestUtils.createTopic(zkUtils, topic, 1, 2, servers)
// send a normal record
val record0 = new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, "key".getBytes, "value".getBytes)
@@ -194,7 +194,7 @@ class ProducerSendTest extends KafkaServerTestHarness {
try {
// create topic
- TestUtils.createTopic(zkClient, topic, 1, 2, servers)
+ TestUtils.createTopic(zkUtils, topic, 1, 2, servers)
// non-blocking send a list of records
val record0 = new ProducerRecord[Array[Byte],Array[Byte]](topic, null, "key".getBytes, "value".getBytes)
@@ -230,7 +230,7 @@ class ProducerSendTest extends KafkaServerTestHarness {
try {
// create topic
- val leaders = TestUtils.createTopic(zkClient, topic, 2, 2, servers)
+ val leaders = TestUtils.createTopic(zkUtils, topic, 2, 2, servers)
val partition = 1
// make sure leaders exist
@@ -289,7 +289,7 @@ class ProducerSendTest extends KafkaServerTestHarness {
assertEquals("Should have offset 0", 0L, producer.send(record).get.offset)
// double check that the topic is created with leader elected
- TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0)
+ TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 0)
} finally {
if (producer != null) {
@@ -306,7 +306,7 @@ class ProducerSendTest extends KafkaServerTestHarness {
def testFlush() {
var producer = TestUtils.createNewProducer(brokerList, lingerMs = Long.MaxValue)
try {
- TestUtils.createTopic(zkClient, topic, 2, 2, servers)
+ TestUtils.createTopic(zkUtils, topic, 2, 2, servers)
val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, "value".getBytes)
for(i <- 0 until 50) {
val responses = (0 until numRecords) map (i => producer.send(record))
@@ -328,7 +328,7 @@ class ProducerSendTest extends KafkaServerTestHarness {
var producer: KafkaProducer[Array[Byte],Array[Byte]] = null
try {
// create topic
- val leaders = TestUtils.createTopic(zkClient, topic, 2, 2, servers)
+ val leaders = TestUtils.createTopic(zkUtils, topic, 2, 2, servers)
val leader0 = leaders(0)
val leader1 = leaders(1)
@@ -372,7 +372,7 @@ class ProducerSendTest extends KafkaServerTestHarness {
var producer: KafkaProducer[Array[Byte],Array[Byte]] = null
try {
// create topic
- val leaders = TestUtils.createTopic(zkClient, topic, 1, 2, servers)
+ val leaders = TestUtils.createTopic(zkUtils, topic, 1, 2, servers)
val leader = leaders(0)
// create record
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/integration/kafka/api/QuotasTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala
index 38b3dbd..bdf7e49 100644
--- a/core/src/test/scala/integration/kafka/api/QuotasTest.scala
+++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala
@@ -85,7 +85,7 @@ class QuotasTest extends KafkaServerTestHarness {
producers += new KafkaProducer[Array[Byte], Array[Byte]](producerProps)
val numPartitions = 1
- val leaders = TestUtils.createTopic(zkClient, topic1, numPartitions, numServers, servers)
+ val leaders = TestUtils.createTopic(zkUtils, topic1, numPartitions, numServers, servers)
leaderNode = if (leaders(0).get == servers.head.config.brokerId) servers.head else servers(1)
followerNode = if (leaders(0).get != servers.head.config.brokerId) servers.head else servers(1)
assertTrue("Leader of all partitions of the topic should exist", leaders.values.forall(leader => leader.isDefined))
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala
index 2f72c78..5dc4cbc 100644
--- a/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala
+++ b/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala
@@ -97,14 +97,14 @@ class SSLConsumerTest extends KafkaServerTestHarness with Logging {
// create the consumer offset topic
- TestUtils.createTopic(zkClient, ConsumerCoordinator.OffsetsTopicName,
+ TestUtils.createTopic(zkUtils, ConsumerCoordinator.OffsetsTopicName,
overridingProps.getProperty(KafkaConfig.OffsetsTopicPartitionsProp).toInt,
overridingProps.getProperty(KafkaConfig.OffsetsTopicReplicationFactorProp).toInt,
servers,
servers(0).consumerCoordinator.offsetsTopicConfigs)
// create the test topic with all the brokers as replicas
- TestUtils.createTopic(this.zkClient, topic, 1, numServers, this.servers)
+ TestUtils.createTopic(zkUtils, topic, 1, numServers, this.servers)
}
@After
@@ -193,7 +193,7 @@ class SSLConsumerTest extends KafkaServerTestHarness with Logging {
@Test
def testPartitionsFor() {
val numParts = 2
- TestUtils.createTopic(this.zkClient, "part-test", numParts, 1, this.servers)
+ TestUtils.createTopic(zkUtils, "part-test", numParts, 1, this.servers)
val parts = this.consumers(0).partitionsFor("part-test")
assertNotNull(parts)
assertEquals(2, parts.length)
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala
index 0f70624..c22e57a 100644
--- a/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala
+++ b/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala
@@ -95,7 +95,7 @@ class SSLProducerSendTest extends KafkaServerTestHarness {
try {
// create topic
- TestUtils.createTopic(zkClient, topic, 1, 2, servers)
+ TestUtils.createTopic(zkUtils, topic, 1, 2, servers)
// send a normal record
val record0 = new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, "key".getBytes, "value".getBytes)
@@ -159,7 +159,7 @@ class SSLProducerSendTest extends KafkaServerTestHarness {
var producer = TestUtils.createNewProducer(TestUtils.getSSLBrokerListStrFromServers(servers), enableSSL=true, trustStoreFile=Some(trustStoreFile))
try {
// create topic
- TestUtils.createTopic(zkClient, topic, 1, 2, servers)
+ TestUtils.createTopic(zkUtils, topic, 1, 2, servers)
// non-blocking send a list of records
val record0 = new ProducerRecord[Array[Byte],Array[Byte]](topic, null, "key".getBytes, "value".getBytes)
@@ -194,7 +194,7 @@ class SSLProducerSendTest extends KafkaServerTestHarness {
var producer = TestUtils.createNewProducer(TestUtils.getSSLBrokerListStrFromServers(servers), enableSSL=true, trustStoreFile=Some(trustStoreFile))
try {
// create topic
- val leaders = TestUtils.createTopic(zkClient, topic, 2, 2, servers)
+ val leaders = TestUtils.createTopic(zkUtils, topic, 2, 2, servers)
val partition = 1
// make sure leaders exist
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/test/scala/other/kafka/DeleteZKPath.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/other/kafka/DeleteZKPath.scala b/core/src/test/scala/other/kafka/DeleteZKPath.scala
index fb8ab9f..92bde88 100755
--- a/core/src/test/scala/other/kafka/DeleteZKPath.scala
+++ b/core/src/test/scala/other/kafka/DeleteZKPath.scala
@@ -32,10 +32,10 @@ object DeleteZKPath {
val config = new ConsumerConfig(Utils.loadProps(args(0)))
val zkPath = args(1)
- val zkClient = ZkUtils.createZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs)
+ val zkUtils = ZkUtils(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false)
try {
- ZkUtils.deletePathRecursive(zkClient, zkPath);
+ zkUtils.deletePathRecursive(zkPath);
System.out.println(zkPath + " is deleted")
} catch {
case e: Exception => System.err.println("Path not deleted " + e.printStackTrace())
[4/5] kafka git commit: KAFKA-2639: Refactoring of ZkUtils
Posted by ju...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
index 2027ec8..f39b9a1 100755
--- a/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
+++ b/core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
@@ -37,6 +37,7 @@ import kafka.utils.ZkUtils._
import kafka.utils._
import org.I0Itec.zkclient.exception.ZkNodeExistsException
import org.I0Itec.zkclient.{IZkChildListener, IZkDataListener, IZkStateListener, ZkClient, ZkConnection}
+import org.apache.kafka.common.security.JaasUtils
import org.apache.zookeeper.Watcher.Event.KeeperState
import scala.collection._
@@ -88,8 +89,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
private val isShuttingDown = new AtomicBoolean(false)
private val rebalanceLock = new Object
private var fetcher: Option[ConsumerFetcherManager] = None
- private var zkClient: ZkClient = null
- private var zkConnection : ZkConnection = null
+ private var zkUtils: ZkUtils = null
private var topicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]]
private val checkpointedZkOffsets = new Pool[TopicAndPartition, Long]
private val topicThreadIdAndQueues = new Pool[(String, ConsumerThreadId), BlockingQueue[FetchedDataChunk]]
@@ -173,21 +173,22 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
private def createFetcher() {
if (enableFetcher)
- fetcher = Some(new ConsumerFetcherManager(consumerIdString, config, zkClient))
+ fetcher = Some(new ConsumerFetcherManager(consumerIdString, config, zkUtils))
}
private def connectZk() {
info("Connecting to zookeeper instance at " + config.zkConnect)
- val (client, connection) = ZkUtils.createZkClientAndConnection(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs)
- zkClient = client
- zkConnection = connection
+ zkUtils = ZkUtils(config.zkConnect,
+ config.zkSessionTimeoutMs,
+ config.zkConnectionTimeoutMs,
+ JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)))
}
// Blocks until the offset manager is located and a channel is established to it.
private def ensureOffsetManagerConnected() {
if (config.offsetsStorage == "kafka") {
if (offsetsChannel == null || !offsetsChannel.isConnected)
- offsetsChannel = ClientUtils.channelToOffsetManager(config.groupId, zkClient,
+ offsetsChannel = ClientUtils.channelToOffsetManager(config.groupId, zkUtils,
config.offsetsChannelSocketTimeoutMs, config.offsetsChannelBackoffMs)
debug("Connected to offset manager %s:%d.".format(offsetsChannel.host, offsetsChannel.port))
@@ -213,9 +214,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
sendShutdownToAllQueues()
if (config.autoCommitEnable)
commitOffsets(true)
- if (zkClient != null) {
- zkClient.close()
- zkClient = null
+ if (zkUtils != null) {
+ zkUtils.close()
+ zkUtils = null
}
if (offsetsChannel != null) offsetsChannel.disconnect()
@@ -266,7 +267,8 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
val zkWatchedEphemeral = new ZKCheckedEphemeral(dirs.
consumerRegistryDir + "/" + consumerIdString,
consumerRegistrationInfo,
- zkConnection.getZookeeper)
+ zkUtils.zkConnection.getZookeeper,
+ false)
zkWatchedEphemeral.create()
info("end registering consumer " + consumerIdString + " in ZK")
@@ -296,7 +298,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
def commitOffsetToZooKeeper(topicPartition: TopicAndPartition, offset: Long) {
if (checkpointedZkOffsets.get(topicPartition) != offset) {
val topicDirs = new ZKGroupTopicDirs(config.groupId, topicPartition.topic)
- updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + topicPartition.partition, offset.toString)
+ zkUtils.updatePersistentPath(topicDirs.consumerOffsetDir + "/" + topicPartition.partition, offset.toString)
checkpointedZkOffsets.put(topicPartition, offset)
zkCommitMeter.mark()
}
@@ -404,7 +406,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
private def fetchOffsetFromZooKeeper(topicPartition: TopicAndPartition) = {
val dirs = new ZKGroupTopicDirs(config.groupId, topicPartition.topic)
- val offsetString = readDataMaybeNull(zkClient, dirs.consumerOffsetDir + "/" + topicPartition.partition)._1
+ val offsetString = zkUtils.readDataMaybeNull(dirs.consumerOffsetDir + "/" + topicPartition.partition)._1
offsetString match {
case Some(offsetStr) => (topicPartition, OffsetMetadataAndError(offsetStr.toLong))
case None => (topicPartition, OffsetMetadataAndError.NoOffset)
@@ -599,7 +601,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
private def deletePartitionOwnershipFromZK(topic: String, partition: Int) {
val topicDirs = new ZKGroupTopicDirs(group, topic)
val znode = topicDirs.consumerOwnerDir + "/" + partition
- deletePath(zkClient, znode)
+ zkUtils.deletePath(znode)
debug("Consumer " + consumerIdString + " releasing " + znode)
}
@@ -630,7 +632,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
var done = false
var cluster: Cluster = null
try {
- cluster = getCluster(zkClient)
+ cluster = zkUtils.getCluster()
done = rebalance(cluster)
} catch {
case e: Throwable =>
@@ -660,14 +662,14 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
private def rebalance(cluster: Cluster): Boolean = {
val myTopicThreadIdsMap = TopicCount.constructTopicCount(
- group, consumerIdString, zkClient, config.excludeInternalTopics).getConsumerThreadIdsPerTopic
- val brokers = getAllBrokersInCluster(zkClient)
+ group, consumerIdString, zkUtils, config.excludeInternalTopics).getConsumerThreadIdsPerTopic
+ val brokers = zkUtils.getAllBrokersInCluster()
if (brokers.size == 0) {
// This can happen in a rare case when there are no brokers available in the cluster when the consumer is started.
// We log an warning and register for child changes on brokers/id so that rebalance can be triggered when the brokers
// are up.
warn("no brokers found when trying to rebalance.")
- zkClient.subscribeChildChanges(ZkUtils.BrokerIdsPath, loadBalancerListener)
+ zkUtils.zkClient.subscribeChildChanges(BrokerIdsPath, loadBalancerListener)
true
}
else {
@@ -690,7 +692,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
)
}
releasePartitionOwnership(topicRegistry)
- val assignmentContext = new AssignmentContext(group, consumerIdString, config.excludeInternalTopics, zkClient)
+ val assignmentContext = new AssignmentContext(group, consumerIdString, config.excludeInternalTopics, zkUtils)
val globalPartitionAssignment = partitionAssignor.assign(assignmentContext)
val partitionAssignment = globalPartitionAssignment.get(assignmentContext.consumerId)
val currentTopicRegistry = new Pool[String, Pool[Int, PartitionTopicInfo]](
@@ -713,7 +715,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
})
/**
- * move the partition ownership here, since that can be used to indicate a truly successful rebalancing attempt
+ * move the partition ownership here, since that can be used to indicate a truly successful re-balancing attempt
* A rebalancing attempt is completed successfully only after the fetchers have been started correctly
*/
if(reflectPartitionOwnershipDecision(partitionAssignment)) {
@@ -832,9 +834,9 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
val topic = partitionOwner._1.topic
val partition = partitionOwner._1.partition
val consumerThreadId = partitionOwner._2
- val partitionOwnerPath = getConsumerPartitionOwnerPath(group, topic, partition)
+ val partitionOwnerPath = zkUtils.getConsumerPartitionOwnerPath(group, topic, partition)
try {
- createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId.toString)
+ zkUtils.createEphemeralPathExpectConflict(partitionOwnerPath, consumerThreadId.toString)
info(consumerThreadId + " successfully owned partition " + partition + " for topic " + topic)
successfullyOwnedPartitions ::= (topic, partition)
true
@@ -951,14 +953,14 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
})
// listener to consumer and partition changes
- zkClient.subscribeStateChanges(sessionExpirationListener)
+ zkUtils.zkClient.subscribeStateChanges(sessionExpirationListener)
- zkClient.subscribeChildChanges(dirs.consumerRegistryDir, loadBalancerListener)
+ zkUtils.zkClient.subscribeChildChanges(dirs.consumerRegistryDir, loadBalancerListener)
topicStreamsMap.foreach { topicAndStreams =>
// register on broker partition path changes
val topicPath = BrokerTopicsPath + "/" + topicAndStreams._1
- zkClient.subscribeDataChanges(topicPath, topicPartitionChangeListener)
+ zkUtils.zkClient.subscribeDataChanges(topicPath, topicPartitionChangeListener)
}
// explicitly trigger load balancing for this consumer
@@ -988,11 +990,11 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
// bootstrap with existing topics
private var wildcardTopics =
- getChildrenParentMayNotExist(zkClient, BrokerTopicsPath)
+ zkUtils.getChildrenParentMayNotExist(BrokerTopicsPath)
.filter(topic => topicFilter.isTopicAllowed(topic, config.excludeInternalTopics))
private val wildcardTopicCount = TopicCount.constructTopicCount(
- consumerIdString, topicFilter, numStreams, zkClient, config.excludeInternalTopics)
+ consumerIdString, topicFilter, numStreams, zkUtils, config.excludeInternalTopics)
val dirs = new ZKGroupDirs(config.groupId)
registerConsumerInZK(dirs, consumerIdString, wildcardTopicCount)
@@ -1002,7 +1004,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
* Topic events will trigger subsequent synced rebalances.
*/
info("Creating topic event watcher for topics " + topicFilter)
- wildcardTopicWatcher = new ZookeeperTopicEventWatcher(zkClient, this)
+ wildcardTopicWatcher = new ZookeeperTopicEventWatcher(zkUtils, this)
def handleTopicEvent(allTopics: Seq[String]) {
debug("Handling topic event")
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala b/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala
index f74823b..0cd22f0 100644
--- a/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala
+++ b/core/src/main/scala/kafka/consumer/ZookeeperTopicEventWatcher.scala
@@ -22,7 +22,7 @@ import kafka.utils.{ZkUtils, Logging}
import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient}
import org.apache.zookeeper.Watcher.Event.KeeperState
-class ZookeeperTopicEventWatcher(val zkClient: ZkClient,
+class ZookeeperTopicEventWatcher(val zkUtils: ZkUtils,
val eventHandler: TopicEventHandler[String]) extends Logging {
val lock = new Object()
@@ -31,24 +31,24 @@ class ZookeeperTopicEventWatcher(val zkClient: ZkClient,
private def startWatchingTopicEvents() {
val topicEventListener = new ZkTopicEventListener()
- ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.BrokerTopicsPath)
+ zkUtils.makeSurePersistentPathExists(ZkUtils.BrokerTopicsPath)
- zkClient.subscribeStateChanges(
+ zkUtils.zkClient.subscribeStateChanges(
new ZkSessionExpireListener(topicEventListener))
- val topics = zkClient.subscribeChildChanges(
+ val topics = zkUtils.zkClient.subscribeChildChanges(
ZkUtils.BrokerTopicsPath, topicEventListener).toList
// call to bootstrap topic list
topicEventListener.handleChildChange(ZkUtils.BrokerTopicsPath, topics)
}
- private def stopWatchingTopicEvents() { zkClient.unsubscribeAll() }
+ private def stopWatchingTopicEvents() { zkUtils.zkClient.unsubscribeAll() }
def shutdown() {
lock.synchronized {
info("Shutting down topic event watcher.")
- if (zkClient != null) {
+ if (zkUtils != null) {
stopWatchingTopicEvents()
}
else {
@@ -63,8 +63,8 @@ class ZookeeperTopicEventWatcher(val zkClient: ZkClient,
def handleChildChange(parent: String, children: java.util.List[String]) {
lock.synchronized {
try {
- if (zkClient != null) {
- val latestTopics = zkClient.getChildren(ZkUtils.BrokerTopicsPath).toList
+ if (zkUtils != null) {
+ val latestTopics = zkUtils.zkClient.getChildren(ZkUtils.BrokerTopicsPath).toList
debug("all topics: %s".format(latestTopics))
eventHandler.handleTopicEvent(latestTopics)
}
@@ -87,9 +87,9 @@ class ZookeeperTopicEventWatcher(val zkClient: ZkClient,
@throws(classOf[Exception])
def handleNewSession() {
lock.synchronized {
- if (zkClient != null) {
+ if (zkUtils != null) {
info("ZK expired: resubscribing topic event listener to topic registry")
- zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath, topicEventListener)
+ zkUtils.zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath, topicEventListener)
}
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/controller/KafkaController.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala
index a7b44ca..0a1a684 100755
--- a/core/src/main/scala/kafka/controller/KafkaController.scala
+++ b/core/src/main/scala/kafka/controller/KafkaController.scala
@@ -43,8 +43,7 @@ import java.util.concurrent.locks.ReentrantLock
import kafka.server._
import kafka.common.TopicAndPartition
-class ControllerContext(val zkClient: ZkClient,
- val zkConnection: ZkConnection,
+class ControllerContext(val zkUtils: ZkUtils,
val zkSessionTimeout: Int) {
var controllerChannelManager: ControllerChannelManager = null
val controllerLock: ReentrantLock = new ReentrantLock()
@@ -154,11 +153,11 @@ object KafkaController extends Logging {
}
}
-class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection: ZkConnection, val brokerState: BrokerState, time: Time, metrics: Metrics, threadNamePrefix: Option[String] = None) extends Logging with KafkaMetricsGroup {
+class KafkaController(val config : KafkaConfig, zkUtils: ZkUtils, val brokerState: BrokerState, time: Time, metrics: Metrics, threadNamePrefix: Option[String] = None) extends Logging with KafkaMetricsGroup {
this.logIdent = "[Controller " + config.brokerId + "]: "
private var isRunning = true
private val stateChangeLogger = KafkaController.stateChangeLogger
- val controllerContext = new ControllerContext(zkClient, zkConnection, config.zkSessionTimeoutMs)
+ val controllerContext = new ControllerContext(zkUtils, config.zkSessionTimeoutMs)
val partitionStateMachine = new PartitionStateMachine(this)
val replicaStateMachine = new ReplicaStateMachine(this)
private val controllerElector = new ZookeeperLeaderElector(controllerContext, ZkUtils.ControllerPath, onControllerFailover,
@@ -321,7 +320,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
//read controller epoch from zk
readControllerEpochFromZookeeper()
// increment the controller epoch
- incrementControllerEpoch(zkClient)
+ incrementControllerEpoch(zkUtils.zkClient)
// before reading source of truth from zookeeper, register the listeners to get broker/topic callbacks
registerReassignedPartitionsListener()
registerIsrChangeNotificationListener()
@@ -613,7 +612,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
reassignedReplicas.toSet)
reassignedPartitionContext.isrChangeListener = isrChangeListener
// register listener on the leader and isr path to wait until they catch up with the current leader
- zkClient.subscribeDataChanges(ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partition), isrChangeListener)
+ zkUtils.zkClient.subscribeDataChanges(getTopicPartitionLeaderAndIsrPath(topic, partition), isrChangeListener)
}
def initiateReassignReplicasForTopicPartition(topicAndPartition: TopicAndPartition,
@@ -703,7 +702,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
def incrementControllerEpoch(zkClient: ZkClient) = {
try {
var newControllerEpoch = controllerContext.epoch + 1
- val (updateSucceeded, newVersion) = ZkUtils.conditionalUpdatePersistentPathIfExists(zkClient,
+ val (updateSucceeded, newVersion) = zkUtils.conditionalUpdatePersistentPathIfExists(
ZkUtils.ControllerEpochPath, newControllerEpoch.toString, controllerContext.epochZkVersion)
if(!updateSucceeded)
throw new ControllerMovedException("Controller moved to another broker. Aborting controller startup procedure")
@@ -732,14 +731,14 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
}
private def registerSessionExpirationListener() = {
- zkClient.subscribeStateChanges(new SessionExpirationListener())
+ zkUtils.zkClient.subscribeStateChanges(new SessionExpirationListener())
}
private def initializeControllerContext() {
// update controller cache with delete topic information
- controllerContext.liveBrokers = ZkUtils.getAllBrokersInCluster(zkClient).toSet
- controllerContext.allTopics = ZkUtils.getAllTopics(zkClient).toSet
- controllerContext.partitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, controllerContext.allTopics.toSeq)
+ controllerContext.liveBrokers = zkUtils.getAllBrokersInCluster().toSet
+ controllerContext.allTopics = zkUtils.getAllTopics().toSet
+ controllerContext.partitionReplicaAssignment = zkUtils.getReplicaAssignmentForTopics(controllerContext.allTopics.toSeq)
controllerContext.partitionLeadershipInfo = new mutable.HashMap[TopicAndPartition, LeaderIsrAndControllerEpoch]
controllerContext.shuttingDownBrokerIds = mutable.Set.empty[Int]
// update the leader and isr cache for all existing partitions from Zookeeper
@@ -756,7 +755,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
private def initializePreferredReplicaElection() {
// initialize preferred replica election state
- val partitionsUndergoingPreferredReplicaElection = ZkUtils.getPartitionsUndergoingPreferredReplicaElection(zkClient)
+ val partitionsUndergoingPreferredReplicaElection = zkUtils.getPartitionsUndergoingPreferredReplicaElection()
// check if they are already completed or topic was deleted
val partitionsThatCompletedPreferredReplicaElection = partitionsUndergoingPreferredReplicaElection.filter { partition =>
val replicasOpt = controllerContext.partitionReplicaAssignment.get(partition)
@@ -774,7 +773,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
private def initializePartitionReassignment() {
// read the partitions being reassigned from zookeeper path /admin/reassign_partitions
- val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient)
+ val partitionsBeingReassigned = zkUtils.getPartitionsBeingReassigned()
// check if they are already completed or topic was deleted
val reassignedPartitions = partitionsBeingReassigned.filter { partition =>
val replicasOpt = controllerContext.partitionReplicaAssignment.get(partition._1)
@@ -793,7 +792,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
}
private def initializeTopicDeletion() {
- val topicsQueuedForDeletion = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.DeleteTopicsPath).toSet
+ val topicsQueuedForDeletion = zkUtils.getChildrenParentMayNotExist(ZkUtils.DeleteTopicsPath).toSet
val topicsWithReplicasOnDeadBrokers = controllerContext.partitionReplicaAssignment.filter { case(partition, replicas) =>
replicas.exists(r => !controllerContext.liveBrokerIds.contains(r)) }.keySet.map(_.topic)
val topicsForWhichPartitionReassignmentIsInProgress = controllerContext.partitionsUndergoingPreferredReplicaElection.map(_.topic)
@@ -822,13 +821,13 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
}
def updateLeaderAndIsrCache(topicAndPartitions: Set[TopicAndPartition] = controllerContext.partitionReplicaAssignment.keySet) {
- val leaderAndIsrInfo = ZkUtils.getPartitionLeaderAndIsrForTopics(zkClient, topicAndPartitions)
+ val leaderAndIsrInfo = zkUtils.getPartitionLeaderAndIsrForTopics(zkUtils.zkClient, topicAndPartitions)
for((topicPartition, leaderIsrAndControllerEpoch) <- leaderAndIsrInfo)
controllerContext.partitionLeadershipInfo.put(topicPartition, leaderIsrAndControllerEpoch)
}
private def areReplicasInIsr(topic: String, partition: Int, replicas: Seq[Int]): Boolean = {
- getLeaderAndIsrForPartition(zkClient, topic, partition) match {
+ zkUtils.getLeaderAndIsrForPartition(topic, partition) match {
case Some(leaderAndIsr) =>
val replicasNotInIsr = replicas.filterNot(r => leaderAndIsr.isr.contains(r))
replicasNotInIsr.isEmpty
@@ -930,42 +929,42 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
private def registerIsrChangeNotificationListener() = {
debug("Registering IsrChangeNotificationListener")
- zkClient.subscribeChildChanges(ZkUtils.IsrChangeNotificationPath, isrChangeNotificationListener)
+ zkUtils.zkClient.subscribeChildChanges(ZkUtils.IsrChangeNotificationPath, isrChangeNotificationListener)
}
private def deregisterIsrChangeNotificationListener() = {
debug("De-registering IsrChangeNotificationListener")
- zkClient.unsubscribeChildChanges(ZkUtils.IsrChangeNotificationPath, isrChangeNotificationListener)
+ zkUtils.zkClient.unsubscribeChildChanges(ZkUtils.IsrChangeNotificationPath, isrChangeNotificationListener)
}
private def registerReassignedPartitionsListener() = {
- zkClient.subscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener)
+ zkUtils.zkClient.subscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener)
}
private def deregisterReassignedPartitionsListener() = {
- zkClient.unsubscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener)
+ zkUtils.zkClient.unsubscribeDataChanges(ZkUtils.ReassignPartitionsPath, partitionReassignedListener)
}
private def registerPreferredReplicaElectionListener() {
- zkClient.subscribeDataChanges(ZkUtils.PreferredReplicaLeaderElectionPath, preferredReplicaElectionListener)
+ zkUtils.zkClient.subscribeDataChanges(ZkUtils.PreferredReplicaLeaderElectionPath, preferredReplicaElectionListener)
}
private def deregisterPreferredReplicaElectionListener() {
- zkClient.unsubscribeDataChanges(ZkUtils.PreferredReplicaLeaderElectionPath, preferredReplicaElectionListener)
+ zkUtils.zkClient.unsubscribeDataChanges(ZkUtils.PreferredReplicaLeaderElectionPath, preferredReplicaElectionListener)
}
private def deregisterReassignedPartitionsIsrChangeListeners() {
controllerContext.partitionsBeingReassigned.foreach {
case (topicAndPartition, reassignedPartitionsContext) =>
- val zkPartitionPath = ZkUtils.getTopicPartitionLeaderAndIsrPath(topicAndPartition.topic, topicAndPartition.partition)
- zkClient.unsubscribeDataChanges(zkPartitionPath, reassignedPartitionsContext.isrChangeListener)
+ val zkPartitionPath = getTopicPartitionLeaderAndIsrPath(topicAndPartition.topic, topicAndPartition.partition)
+ zkUtils.zkClient.unsubscribeDataChanges(zkPartitionPath, reassignedPartitionsContext.isrChangeListener)
}
}
private def readControllerEpochFromZookeeper() {
// initialize the controller epoch and zk version by reading from zookeeper
- if(ZkUtils.pathExists(controllerContext.zkClient, ZkUtils.ControllerEpochPath)) {
- val epochData = ZkUtils.readData(controllerContext.zkClient, ZkUtils.ControllerEpochPath)
+ if(controllerContext.zkUtils.pathExists(ZkUtils.ControllerEpochPath)) {
+ val epochData = controllerContext.zkUtils.readData(ZkUtils.ControllerEpochPath)
controllerContext.epoch = epochData._1.toInt
controllerContext.epochZkVersion = epochData._2.getVersion
info("Initialized controller epoch to %d and zk version %d".format(controllerContext.epoch, controllerContext.epochZkVersion))
@@ -975,15 +974,15 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
def removePartitionFromReassignedPartitions(topicAndPartition: TopicAndPartition) {
if(controllerContext.partitionsBeingReassigned.get(topicAndPartition).isDefined) {
// stop watching the ISR changes for this partition
- zkClient.unsubscribeDataChanges(ZkUtils.getTopicPartitionLeaderAndIsrPath(topicAndPartition.topic, topicAndPartition.partition),
+ zkUtils.zkClient.unsubscribeDataChanges(getTopicPartitionLeaderAndIsrPath(topicAndPartition.topic, topicAndPartition.partition),
controllerContext.partitionsBeingReassigned(topicAndPartition).isrChangeListener)
}
// read the current list of reassigned partitions from zookeeper
- val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient)
+ val partitionsBeingReassigned = zkUtils.getPartitionsBeingReassigned()
// remove this partition from that list
val updatedPartitionsBeingReassigned = partitionsBeingReassigned - topicAndPartition
// write the new list to zookeeper
- ZkUtils.updatePartitionReassignmentData(zkClient, updatedPartitionsBeingReassigned.mapValues(_.newReplicas))
+ zkUtils.updatePartitionReassignmentData(updatedPartitionsBeingReassigned.mapValues(_.newReplicas))
// update the cache. NO-OP if the partition's reassignment was never started
controllerContext.partitionsBeingReassigned.remove(topicAndPartition)
}
@@ -991,9 +990,9 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
def updateAssignedReplicasForPartition(topicAndPartition: TopicAndPartition,
newReplicaAssignmentForTopic: Map[TopicAndPartition, Seq[Int]]) {
try {
- val zkPath = ZkUtils.getTopicPath(topicAndPartition.topic)
- val jsonPartitionMap = ZkUtils.replicaAssignmentZkData(newReplicaAssignmentForTopic.map(e => (e._1.partition.toString -> e._2)))
- ZkUtils.updatePersistentPath(zkClient, zkPath, jsonPartitionMap)
+ val zkPath = getTopicPath(topicAndPartition.topic)
+ val jsonPartitionMap = zkUtils.replicaAssignmentZkData(newReplicaAssignmentForTopic.map(e => (e._1.partition.toString -> e._2)))
+ zkUtils.updatePersistentPath(zkPath, jsonPartitionMap)
debug("Updated path %s with %s for replica assignment".format(zkPath, jsonPartitionMap))
} catch {
case e: ZkNoNodeException => throw new IllegalStateException("Topic %s doesn't exist".format(topicAndPartition.topic))
@@ -1014,7 +1013,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
}
}
if (!isTriggeredByAutoRebalance)
- ZkUtils.deletePath(zkClient, ZkUtils.PreferredReplicaLeaderElectionPath)
+ zkUtils.deletePath(ZkUtils.PreferredReplicaLeaderElectionPath)
controllerContext.partitionsUndergoingPreferredReplicaElection --= partitionsToBeRemoved
}
@@ -1057,7 +1056,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
var zkWriteCompleteOrUnnecessary = false
while (!zkWriteCompleteOrUnnecessary) {
// refresh leader and isr from zookeeper again
- val leaderIsrAndEpochOpt = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition)
+ val leaderIsrAndEpochOpt = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkUtils, topic, partition)
zkWriteCompleteOrUnnecessary = leaderIsrAndEpochOpt match {
case Some(leaderIsrAndEpoch) => // increment the leader epoch even if the ISR changes
val leaderAndIsr = leaderIsrAndEpoch.leaderAndIsr
@@ -1074,7 +1073,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
// if the replica to be removed from the ISR is the last surviving member of the ISR and unclean leader election
// is disallowed for the corresponding topic, then we must preserve the ISR membership so that the replica can
// eventually be restored as the leader.
- if (newIsr.isEmpty && !LogConfig.fromProps(config.originals, AdminUtils.fetchEntityConfig(zkClient,
+ if (newIsr.isEmpty && !LogConfig.fromProps(config.originals, AdminUtils.fetchEntityConfig(zkUtils,
ConfigType.Topic, topicAndPartition.topic)).uncleanLeaderElectionEnable) {
info("Retaining last ISR %d of partition %s since unclean leader election is disabled".format(replicaId, topicAndPartition))
newIsr = leaderAndIsr.isr
@@ -1083,7 +1082,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
val newLeaderAndIsr = new LeaderAndIsr(newLeader, leaderAndIsr.leaderEpoch + 1,
newIsr, leaderAndIsr.zkVersion + 1)
// update the new leadership decision in zookeeper or retry
- val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topic, partition,
+ val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkUtils, topic, partition,
newLeaderAndIsr, epoch, leaderAndIsr.zkVersion)
newLeaderAndIsr.zkVersion = newVersion
@@ -1120,7 +1119,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
var zkWriteCompleteOrUnnecessary = false
while (!zkWriteCompleteOrUnnecessary) {
// refresh leader and isr from zookeeper again
- val leaderIsrAndEpochOpt = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition)
+ val leaderIsrAndEpochOpt = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkUtils, topic, partition)
zkWriteCompleteOrUnnecessary = leaderIsrAndEpochOpt match {
case Some(leaderIsrAndEpoch) =>
val leaderAndIsr = leaderIsrAndEpoch.leaderAndIsr
@@ -1134,7 +1133,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
val newLeaderAndIsr = new LeaderAndIsr(leaderAndIsr.leader, leaderAndIsr.leaderEpoch + 1,
leaderAndIsr.isr, leaderAndIsr.zkVersion + 1)
// update the new leadership decision in zookeeper or retry
- val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topic,
+ val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkUtils, topic,
partition, newLeaderAndIsr, epoch, leaderAndIsr.zkVersion)
newLeaderAndIsr.zkVersion = newVersion
@@ -1245,7 +1244,7 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, zkConnection
*/
class PartitionsReassignedListener(controller: KafkaController) extends IZkDataListener with Logging {
this.logIdent = "[PartitionsReassignedListener on " + controller.config.brokerId + "]: "
- val zkClient = controller.controllerContext.zkClient
+ val zkUtils = controller.controllerContext.zkUtils
val controllerContext = controller.controllerContext
/**
@@ -1256,7 +1255,7 @@ class PartitionsReassignedListener(controller: KafkaController) extends IZkDataL
def handleDataChange(dataPath: String, data: Object) {
debug("Partitions reassigned listener fired for path %s. Record partitions to be reassigned %s"
.format(dataPath, data))
- val partitionsReassignmentData = ZkUtils.parsePartitionReassignmentData(data.toString)
+ val partitionsReassignmentData = zkUtils.parsePartitionReassignmentData(data.toString)
val partitionsToBeReassigned = inLock(controllerContext.controllerLock) {
partitionsReassignmentData.filterNot(p => controllerContext.partitionsBeingReassigned.contains(p._1))
}
@@ -1288,7 +1287,7 @@ class ReassignedPartitionsIsrChangeListener(controller: KafkaController, topic:
reassignedReplicas: Set[Int])
extends IZkDataListener with Logging {
this.logIdent = "[ReassignedPartitionsIsrChangeListener on controller " + controller.config.brokerId + "]: "
- val zkClient = controller.controllerContext.zkClient
+ val zkUtils = controller.controllerContext.zkUtils
val controllerContext = controller.controllerContext
/**
@@ -1305,7 +1304,7 @@ class ReassignedPartitionsIsrChangeListener(controller: KafkaController, topic:
controllerContext.partitionsBeingReassigned.get(topicAndPartition) match {
case Some(reassignedPartitionContext) =>
// need to re-read leader and isr from zookeeper since the zkclient callback doesn't return the Stat object
- val newLeaderAndIsrOpt = ZkUtils.getLeaderAndIsrForPartition(zkClient, topic, partition)
+ val newLeaderAndIsrOpt = zkUtils.getLeaderAndIsrForPartition(topic, partition)
newLeaderAndIsrOpt match {
case Some(leaderAndIsr) => // check if new replicas have joined ISR
val caughtUpReplicas = reassignedReplicas & leaderAndIsr.isr.toSet
@@ -1359,7 +1358,7 @@ class IsrChangeNotificationListener(controller: KafkaController) extends IZkChil
processUpdateNotifications(topicAndPartitions)
} finally {
// delete processed children
- childrenAsScala.map(x => ZkUtils.deletePath(controller.controllerContext.zkClient,
+ childrenAsScala.map(x => controller.controllerContext.zkUtils.deletePath(
ZkUtils.IsrChangeNotificationPath + "/" + x))
}
}
@@ -1373,7 +1372,7 @@ class IsrChangeNotificationListener(controller: KafkaController) extends IZkChil
private def getTopicAndPartition(child: String): Set[TopicAndPartition] = {
val changeZnode: String = ZkUtils.IsrChangeNotificationPath + "/" + child
- val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(controller.controllerContext.zkClient, changeZnode)
+ val (jsonOpt, stat) = controller.controllerContext.zkUtils.readDataMaybeNull(changeZnode)
if (jsonOpt.isDefined) {
val json = Json.parseFull(jsonOpt.get)
@@ -1410,7 +1409,7 @@ object IsrChangeNotificationListener {
*/
class PreferredReplicaElectionListener(controller: KafkaController) extends IZkDataListener with Logging {
this.logIdent = "[PreferredReplicaElectionListener on " + controller.config.brokerId + "]: "
- val zkClient = controller.controllerContext.zkClient
+ val zkUtils = controller.controllerContext.zkUtils
val controllerContext = controller.controllerContext
/**
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala
index 4ebeb5a..5eed382 100644
--- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala
+++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala
@@ -61,7 +61,7 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, confi
case true =>
// Prior to electing an unclean (i.e. non-ISR) leader, ensure that doing so is not disallowed by the configuration
// for unclean leader election.
- if (!LogConfig.fromProps(config.originals, AdminUtils.fetchEntityConfig(controllerContext.zkClient,
+ if (!LogConfig.fromProps(config.originals, AdminUtils.fetchEntityConfig(controllerContext.zkUtils,
ConfigType.Topic, topicAndPartition.topic)).uncleanLeaderElectionEnable) {
throw new NoReplicaOnlineException(("No broker in ISR for partition " +
"%s is alive. Live brokers are: [%s],".format(topicAndPartition, controllerContext.liveBrokerIds)) +
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/controller/PartitionStateMachine.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala
index 675a807..73b173e 100755
--- a/core/src/main/scala/kafka/controller/PartitionStateMachine.scala
+++ b/core/src/main/scala/kafka/controller/PartitionStateMachine.scala
@@ -22,7 +22,8 @@ import collection.mutable.Buffer
import java.util.concurrent.atomic.AtomicBoolean
import kafka.api.LeaderAndIsr
import kafka.common.{LeaderElectionNotNeededException, TopicAndPartition, StateChangeFailedException, NoReplicaOnlineException}
-import kafka.utils.{Logging, ZkUtils, ReplicationUtils}
+import kafka.utils.{Logging, ReplicationUtils}
+import kafka.utils.ZkUtils._
import org.I0Itec.zkclient.{IZkDataListener, IZkChildListener}
import org.I0Itec.zkclient.exception.ZkNodeExistsException
import kafka.controller.Callbacks.CallbackBuilder
@@ -43,7 +44,7 @@ import kafka.utils.CoreUtils._
class PartitionStateMachine(controller: KafkaController) extends Logging {
private val controllerContext = controller.controllerContext
private val controllerId = controller.config.brokerId
- private val zkClient = controllerContext.zkClient
+ private val zkUtils = controllerContext.zkUtils
private val partitionState: mutable.Map[TopicAndPartition, PartitionState] = mutable.Map.empty
private val brokerRequestBatch = new ControllerBrokerRequestBatch(controller)
private val hasStarted = new AtomicBoolean(false)
@@ -83,7 +84,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
deregisterTopicChangeListener()
addPartitionsListener.foreach {
case (topic, listener) =>
- zkClient.unsubscribeDataChanges(ZkUtils.getTopicPath(topic), listener)
+ zkUtils.zkClient.unsubscribeDataChanges(getTopicPath(topic), listener)
}
addPartitionsListener.clear()
if(controller.config.deleteTopicEnable)
@@ -289,9 +290,9 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
controller.epoch)
debug("Initializing leader and isr for partition %s to %s".format(topicAndPartition, leaderIsrAndControllerEpoch))
try {
- ZkUtils.createPersistentPath(controllerContext.zkClient,
- ZkUtils.getTopicPartitionLeaderAndIsrPath(topicAndPartition.topic, topicAndPartition.partition),
- ZkUtils.leaderAndIsrZkData(leaderIsrAndControllerEpoch.leaderAndIsr, controller.epoch))
+ zkUtils.createPersistentPath(
+ getTopicPartitionLeaderAndIsrPath(topicAndPartition.topic, topicAndPartition.partition),
+ zkUtils.leaderAndIsrZkData(leaderIsrAndControllerEpoch.leaderAndIsr, controller.epoch))
// NOTE: the above write can fail only if the current controller lost its zk session and the new controller
// took over and initialized this partition. This can happen if the current controller went into a long
// GC pause
@@ -301,7 +302,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
} catch {
case e: ZkNodeExistsException =>
// read the controller epoch
- val leaderIsrAndEpoch = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topicAndPartition.topic,
+ val leaderIsrAndEpoch = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkUtils, topicAndPartition.topic,
topicAndPartition.partition).get
val failMsg = ("encountered error while changing partition %s's state from New to Online since LeaderAndIsr path already " +
"exists with value %s and controller epoch %d")
@@ -342,7 +343,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
}
// elect new leader or throw exception
val (leaderAndIsr, replicas) = leaderSelector.selectLeader(topicAndPartition, currentLeaderAndIsr)
- val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topic, partition,
+ val (updateSucceeded, newVersion) = ReplicationUtils.updateLeaderAndIsr(zkUtils, topic, partition,
leaderAndIsr, controller.epoch, currentLeaderAndIsr.zkVersion)
newLeaderAndIsr = leaderAndIsr
newLeaderAndIsr.zkVersion = newVersion
@@ -370,34 +371,34 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
}
private def registerTopicChangeListener() = {
- zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath, topicChangeListener)
+ zkUtils.zkClient.subscribeChildChanges(BrokerTopicsPath, topicChangeListener)
}
private def deregisterTopicChangeListener() = {
- zkClient.unsubscribeChildChanges(ZkUtils.BrokerTopicsPath, topicChangeListener)
+ zkUtils.zkClient.unsubscribeChildChanges(BrokerTopicsPath, topicChangeListener)
}
def registerPartitionChangeListener(topic: String) = {
addPartitionsListener.put(topic, new AddPartitionsListener(topic))
- zkClient.subscribeDataChanges(ZkUtils.getTopicPath(topic), addPartitionsListener(topic))
+ zkUtils.zkClient.subscribeDataChanges(getTopicPath(topic), addPartitionsListener(topic))
}
def deregisterPartitionChangeListener(topic: String) = {
- zkClient.unsubscribeDataChanges(ZkUtils.getTopicPath(topic), addPartitionsListener(topic))
+ zkUtils.zkClient.unsubscribeDataChanges(getTopicPath(topic), addPartitionsListener(topic))
addPartitionsListener.remove(topic)
}
private def registerDeleteTopicListener() = {
- zkClient.subscribeChildChanges(ZkUtils.DeleteTopicsPath, deleteTopicsListener)
+ zkUtils.zkClient.subscribeChildChanges(DeleteTopicsPath, deleteTopicsListener)
}
private def deregisterDeleteTopicListener() = {
- zkClient.unsubscribeChildChanges(ZkUtils.DeleteTopicsPath, deleteTopicsListener)
+ zkUtils.zkClient.unsubscribeChildChanges(DeleteTopicsPath, deleteTopicsListener)
}
private def getLeaderIsrAndEpochOrThrowException(topic: String, partition: Int): LeaderIsrAndControllerEpoch = {
val topicAndPartition = TopicAndPartition(topic, partition)
- ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition) match {
+ ReplicationUtils.getLeaderIsrAndEpochForPartition(zkUtils, topic, partition) match {
case Some(currentLeaderIsrAndEpoch) => currentLeaderIsrAndEpoch
case None =>
val failMsg = "LeaderAndIsr information doesn't exist for partition %s in %s state"
@@ -426,7 +427,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
val deletedTopics = controllerContext.allTopics -- currentChildren
controllerContext.allTopics = currentChildren
- val addedPartitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, newTopics.toSeq)
+ val addedPartitionReplicaAssignment = zkUtils.getReplicaAssignmentForTopics(newTopics.toSeq)
controllerContext.partitionReplicaAssignment = controllerContext.partitionReplicaAssignment.filter(p =>
!deletedTopics.contains(p._1.topic))
controllerContext.partitionReplicaAssignment.++=(addedPartitionReplicaAssignment)
@@ -449,7 +450,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
*/
class DeleteTopicsListener() extends IZkChildListener with Logging {
this.logIdent = "[DeleteTopicsListener on " + controller.config.brokerId + "]: "
- val zkClient = controllerContext.zkClient
+ val zkUtils = controllerContext.zkUtils
/**
* Invoked when a topic is being deleted
@@ -466,7 +467,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
val nonExistentTopics = topicsToBeDeleted.filter(t => !controllerContext.allTopics.contains(t))
if(nonExistentTopics.size > 0) {
warn("Ignoring request to delete non-existing topics " + nonExistentTopics.mkString(","))
- nonExistentTopics.foreach(topic => ZkUtils.deletePathRecursive(zkClient, ZkUtils.getDeleteTopicPath(topic)))
+ nonExistentTopics.foreach(topic => zkUtils.deletePathRecursive(getDeleteTopicPath(topic)))
}
topicsToBeDeleted --= nonExistentTopics
if(topicsToBeDeleted.size > 0) {
@@ -505,7 +506,7 @@ class PartitionStateMachine(controller: KafkaController) extends Logging {
inLock(controllerContext.controllerLock) {
try {
info("Add Partition triggered " + data.toString + " for path " + dataPath)
- val partitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic))
+ val partitionReplicaAssignment = zkUtils.getReplicaAssignmentForTopics(List(topic))
val partitionsToBeAdded = partitionReplicaAssignment.filter(p =>
!controllerContext.partitionReplicaAssignment.contains(p._1))
if(controller.deleteTopicManager.isTopicQueuedUpForDeletion(topic))
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala
index acad83a..32ed288 100755
--- a/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala
+++ b/core/src/main/scala/kafka/controller/ReplicaStateMachine.scala
@@ -47,7 +47,7 @@ import kafka.utils.CoreUtils._
class ReplicaStateMachine(controller: KafkaController) extends Logging {
private val controllerContext = controller.controllerContext
private val controllerId = controller.config.brokerId
- private val zkClient = controllerContext.zkClient
+ private val zkUtils = controllerContext.zkUtils
private val replicaState: mutable.Map[PartitionAndReplica, ReplicaState] = mutable.Map.empty
private val brokerChangeListener = new BrokerChangeListener()
private val brokerRequestBatch = new ControllerBrokerRequestBatch(controller)
@@ -171,7 +171,7 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging {
case NewReplica =>
assertValidPreviousStates(partitionAndReplica, List(NonExistentReplica), targetState)
// start replica as a follower to the current leader for its partition
- val leaderIsrAndControllerEpochOpt = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkClient, topic, partition)
+ val leaderIsrAndControllerEpochOpt = ReplicationUtils.getLeaderIsrAndEpochForPartition(zkUtils, topic, partition)
leaderIsrAndControllerEpochOpt match {
case Some(leaderIsrAndControllerEpoch) =>
if(leaderIsrAndControllerEpoch.leaderAndIsr.leader == replicaId)
@@ -313,11 +313,11 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging {
}
private def registerBrokerChangeListener() = {
- zkClient.subscribeChildChanges(ZkUtils.BrokerIdsPath, brokerChangeListener)
+ zkUtils.zkClient.subscribeChildChanges(ZkUtils.BrokerIdsPath, brokerChangeListener)
}
private def deregisterBrokerChangeListener() = {
- zkClient.unsubscribeChildChanges(ZkUtils.BrokerIdsPath, brokerChangeListener)
+ zkUtils.zkClient.unsubscribeChildChanges(ZkUtils.BrokerIdsPath, brokerChangeListener)
}
/**
@@ -359,10 +359,10 @@ class ReplicaStateMachine(controller: KafkaController) extends Logging {
try {
val curBrokerIds = currentBrokerList.map(_.toInt).toSet
val newBrokerIds = curBrokerIds -- controllerContext.liveOrShuttingDownBrokerIds
- val newBrokerInfo = newBrokerIds.map(ZkUtils.getBrokerInfo(zkClient, _))
+ val newBrokerInfo = newBrokerIds.map(zkUtils.getBrokerInfo(_))
val newBrokers = newBrokerInfo.filter(_.isDefined).map(_.get)
val deadBrokerIds = controllerContext.liveOrShuttingDownBrokerIds -- curBrokerIds
- controllerContext.liveBrokers = curBrokerIds.map(ZkUtils.getBrokerInfo(zkClient, _)).filter(_.isDefined).map(_.get)
+ controllerContext.liveBrokers = curBrokerIds.map(zkUtils.getBrokerInfo(_)).filter(_.isDefined).map(_.get)
info("Newly added brokers: %s, deleted brokers: %s, all live brokers: %s"
.format(newBrokerIds.mkString(","), deadBrokerIds.mkString(","), controllerContext.liveBrokerIds.mkString(",")))
newBrokers.foreach(controllerContext.controllerChannelManager.addBroker(_))
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/controller/TopicDeletionManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala
index 9e39dd5..c6f80ac 100755
--- a/core/src/main/scala/kafka/controller/TopicDeletionManager.scala
+++ b/core/src/main/scala/kafka/controller/TopicDeletionManager.scala
@@ -23,8 +23,9 @@ import org.apache.kafka.common.requests.{StopReplicaResponse, AbstractRequestRes
import collection.mutable
import collection.JavaConverters._
-import kafka.utils.{ShutdownableThread, Logging, ZkUtils}
+import kafka.utils.{ShutdownableThread, Logging}
import kafka.utils.CoreUtils._
+import kafka.utils.ZkUtils._
import collection.Set
import kafka.common.TopicAndPartition
import java.util.concurrent.locks.ReentrantLock
@@ -288,9 +289,10 @@ class TopicDeletionManager(controller: KafkaController,
partitionStateMachine.handleStateChanges(partitionsForDeletedTopic, NonExistentPartition)
topicsToBeDeleted -= topic
partitionsToBeDeleted.retain(_.topic != topic)
- controllerContext.zkClient.deleteRecursive(ZkUtils.getTopicPath(topic))
- controllerContext.zkClient.deleteRecursive(ZkUtils.getEntityConfigPath(ConfigType.Topic, topic))
- controllerContext.zkClient.delete(ZkUtils.getDeleteTopicPath(topic))
+ val zkUtils = controllerContext.zkUtils
+ zkUtils.zkClient.deleteRecursive(getTopicPath(topic))
+ zkUtils.zkClient.deleteRecursive(getEntityConfigPath(ConfigType.Topic, topic))
+ zkUtils.zkClient.delete(getDeleteTopicPath(topic))
controllerContext.removeTopic(topic)
}
@@ -385,7 +387,7 @@ class TopicDeletionManager(controller: KafkaController,
}
class DeleteTopicsThread() extends ShutdownableThread(name = "delete-topics-thread-" + controller.config.brokerId, isInterruptible = false) {
- val zkClient = controllerContext.zkClient
+ val zkUtils = controllerContext.zkUtils
override def doWork() {
awaitTopicDeletionNotification()
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala
index 68ff4fc..bf23e9b 100644
--- a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala
+++ b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala
@@ -43,7 +43,7 @@ class ConsumerCoordinator(val brokerId: Int,
val groupConfig: GroupManagerConfig,
val offsetConfig: OffsetManagerConfig,
private val offsetManager: OffsetManager,
- zkClient: ZkClient) extends Logging {
+ zkUtils: ZkUtils) extends Logging {
this.logIdent = "[ConsumerCoordinator " + brokerId + "]: "
@@ -57,9 +57,9 @@ class ConsumerCoordinator(val brokerId: Int,
groupConfig: GroupManagerConfig,
offsetConfig: OffsetManagerConfig,
replicaManager: ReplicaManager,
- zkClient: ZkClient,
+ zkUtils: ZkUtils,
scheduler: KafkaScheduler) = this(brokerId, groupConfig, offsetConfig,
- new OffsetManager(offsetConfig, replicaManager, zkClient, scheduler), zkClient)
+ new OffsetManager(offsetConfig, replicaManager, zkUtils, scheduler), zkUtils)
def offsetsTopicConfigs: Properties = {
val props = new Properties
@@ -81,7 +81,7 @@ class ConsumerCoordinator(val brokerId: Int,
info("Starting up.")
heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", brokerId)
rebalancePurgatory = new DelayedOperationPurgatory[DelayedRebalance]("Rebalance", brokerId)
- coordinatorMetadata = new CoordinatorMetadata(brokerId, zkClient, maybePrepareRebalance)
+ coordinatorMetadata = new CoordinatorMetadata(brokerId, zkUtils, maybePrepareRebalance)
isActive.set(true)
info("Startup complete.")
}
@@ -499,7 +499,7 @@ object ConsumerCoordinator {
val OffsetsTopicName = "__consumer_offsets"
def create(config: KafkaConfig,
- zkClient: ZkClient,
+ zkUtils: ZkUtils,
replicaManager: ReplicaManager,
kafkaScheduler: KafkaScheduler): ConsumerCoordinator = {
val offsetConfig = OffsetManagerConfig(maxMetadataSize = config.offsetMetadataMaxSize,
@@ -513,11 +513,11 @@ object ConsumerCoordinator {
val groupConfig = GroupManagerConfig(consumerMinSessionTimeoutMs = config.consumerMinSessionTimeoutMs,
consumerMaxSessionTimeoutMs = config.consumerMaxSessionTimeoutMs)
- new ConsumerCoordinator(config.brokerId, groupConfig, offsetConfig, replicaManager, zkClient, kafkaScheduler)
+ new ConsumerCoordinator(config.brokerId, groupConfig, offsetConfig, replicaManager, zkUtils, kafkaScheduler)
}
def create(config: KafkaConfig,
- zkClient: ZkClient,
+ zkUtils: ZkUtils,
offsetManager: OffsetManager): ConsumerCoordinator = {
val offsetConfig = OffsetManagerConfig(maxMetadataSize = config.offsetMetadataMaxSize,
loadBufferSize = config.offsetsLoadBufferSize,
@@ -530,6 +530,6 @@ object ConsumerCoordinator {
val groupConfig = GroupManagerConfig(consumerMinSessionTimeoutMs = config.consumerMinSessionTimeoutMs,
consumerMaxSessionTimeoutMs = config.consumerMaxSessionTimeoutMs)
- new ConsumerCoordinator(config.brokerId, groupConfig, offsetConfig, offsetManager, zkClient)
+ new ConsumerCoordinator(config.brokerId, groupConfig, offsetConfig, offsetManager, zkUtils)
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala b/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala
index 2920320..a33231a 100644
--- a/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala
+++ b/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala
@@ -20,7 +20,7 @@ package kafka.coordinator
import kafka.server.KafkaConfig
import kafka.utils.CoreUtils.{inReadLock, inWriteLock}
import kafka.utils.{threadsafe, ZkUtils, Logging}
-
+import kafka.utils.ZkUtils._
import org.I0Itec.zkclient.{ZkClient, IZkDataListener}
import java.util.concurrent.locks.ReentrantReadWriteLock
@@ -33,7 +33,7 @@ import scala.collection.mutable
*/
@threadsafe
private[coordinator] class CoordinatorMetadata(brokerId: Int,
- zkClient: ZkClient,
+ zkUtils: ZkUtils,
maybePrepareRebalance: ConsumerGroupMetadata => Unit) {
/**
@@ -159,19 +159,19 @@ private[coordinator] class CoordinatorMetadata(brokerId: Int,
}
private def getTopicPartitionCountFromZK(topic: String) = {
- val topicData = ZkUtils.getPartitionAssignmentForTopics(zkClient, Seq(topic))
+ val topicData = zkUtils.getPartitionAssignmentForTopics(Seq(topic))
topicData(topic).size
}
private def registerTopicPartitionChangeListener(topic: String) {
val listener = new TopicPartitionChangeListener
topicPartitionChangeListeners.put(topic, listener)
- zkClient.subscribeDataChanges(ZkUtils.getTopicPath(topic), listener)
+ zkUtils.zkClient.subscribeDataChanges(getTopicPath(topic), listener)
}
private def deregisterTopicPartitionChangeListener(topic: String) {
val listener = topicPartitionChangeListeners(topic)
- zkClient.unsubscribeDataChanges(ZkUtils.getTopicPath(topic), listener)
+ zkUtils.zkClient.unsubscribeDataChanges(getTopicPath(topic), listener)
topicPartitionChangeListeners.remove(topic)
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala
index 2e5ee8d..9b4314e 100644
--- a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala
+++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala
@@ -27,6 +27,7 @@ import kafka.server.KafkaConfig
import kafka.utils.CoreUtils.{inReadLock, inWriteLock}
import kafka.utils._
import org.I0Itec.zkclient.{IZkStateListener, ZkClient}
+import org.apache.kafka.common.security.JaasUtils
import org.apache.kafka.common.security.auth.KafkaPrincipal
import scala.collection.JavaConverters._
import org.apache.log4j.Logger
@@ -59,7 +60,7 @@ object SimpleAclAuthorizer {
//notification node which gets updated with the resource name when acl on a resource is changed.
val AclChangedZkPath = "/kafka-acl-changes"
- //prefix of all the change notificiation sequence node.
+ //prefix of all the change notification sequence node.
val AclChangedPrefix = "acl_changes_"
}
@@ -67,7 +68,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging {
private val authorizerLogger = Logger.getLogger("kafka.authorizer.logger")
private var superUsers = Set.empty[KafkaPrincipal]
private var shouldAllowEveryoneIfNoAclIsFound = false
- private var zkClient: ZkClient = null
+ private var zkUtils: ZkUtils = null
private var aclChangeListener: ZkNodeChangeNotificationListener = null
private val aclCache = new scala.collection.mutable.HashMap[Resource, Set[Acl]]
@@ -92,16 +93,19 @@ class SimpleAclAuthorizer extends Authorizer with Logging {
val zkConnectionTimeoutMs = configs.getOrElse(SimpleAclAuthorizer.ZkConnectionTimeOutProp, kafkaConfig.zkConnectionTimeoutMs).toString.toInt
val zkSessionTimeOutMs = configs.getOrElse(SimpleAclAuthorizer.ZkSessionTimeOutProp, kafkaConfig.zkSessionTimeoutMs).toString.toInt
- zkClient = ZkUtils.createZkClient(zkUrl, zkConnectionTimeoutMs, zkSessionTimeOutMs)
- ZkUtils.makeSurePersistentPathExists(zkClient, SimpleAclAuthorizer.AclZkPath)
+ zkUtils = ZkUtils(zkUrl,
+ zkConnectionTimeoutMs,
+ zkSessionTimeOutMs,
+ JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)))
+ zkUtils.makeSurePersistentPathExists(SimpleAclAuthorizer.AclZkPath)
loadCache()
- ZkUtils.makeSurePersistentPathExists(zkClient, SimpleAclAuthorizer.AclChangedZkPath)
- aclChangeListener = new ZkNodeChangeNotificationListener(zkClient, SimpleAclAuthorizer.AclChangedZkPath, SimpleAclAuthorizer.AclChangedPrefix, AclChangedNotificaitonHandler)
+ zkUtils.makeSurePersistentPathExists(SimpleAclAuthorizer.AclChangedZkPath)
+ aclChangeListener = new ZkNodeChangeNotificationListener(zkUtils, SimpleAclAuthorizer.AclChangedZkPath, SimpleAclAuthorizer.AclChangedPrefix, AclChangedNotificaitonHandler)
aclChangeListener.init()
- zkClient.subscribeStateChanges(ZkStateChangeListener)
+ zkUtils.zkClient.subscribeStateChanges(ZkStateChangeListener)
}
override def authorize(session: Session, operation: Operation, resource: Resource): Boolean = {
@@ -162,17 +166,17 @@ class SimpleAclAuthorizer extends Authorizer with Logging {
val updatedAcls = getAcls(resource) ++ acls
val path = toResourcePath(resource)
- if (ZkUtils.pathExists(zkClient, path))
- ZkUtils.updatePersistentPath(zkClient, path, Json.encode(Acl.toJsonCompatibleMap(updatedAcls)))
+ if (zkUtils.pathExists(path))
+ zkUtils.updatePersistentPath(path, Json.encode(Acl.toJsonCompatibleMap(updatedAcls)))
else
- ZkUtils.createPersistentPath(zkClient, path, Json.encode(Acl.toJsonCompatibleMap(updatedAcls)))
+ zkUtils.createPersistentPath(path, Json.encode(Acl.toJsonCompatibleMap(updatedAcls)))
updateAclChangedFlag(resource)
}
}
override def removeAcls(aclsTobeRemoved: Set[Acl], resource: Resource): Boolean = {
- if (ZkUtils.pathExists(zkClient, toResourcePath(resource))) {
+ if (zkUtils.pathExists(toResourcePath(resource))) {
val existingAcls = getAcls(resource)
val filteredAcls = existingAcls.filter((acl: Acl) => !aclsTobeRemoved.contains(acl))
@@ -180,9 +184,9 @@ class SimpleAclAuthorizer extends Authorizer with Logging {
if (aclNeedsRemoval) {
val path: String = toResourcePath(resource)
if (filteredAcls.nonEmpty)
- ZkUtils.updatePersistentPath(zkClient, path, Json.encode(Acl.toJsonCompatibleMap(filteredAcls)))
+ zkUtils.updatePersistentPath(path, Json.encode(Acl.toJsonCompatibleMap(filteredAcls)))
else
- ZkUtils.deletePath(zkClient, toResourcePath(resource))
+ zkUtils.deletePath(toResourcePath(resource))
updateAclChangedFlag(resource)
}
@@ -192,8 +196,8 @@ class SimpleAclAuthorizer extends Authorizer with Logging {
}
override def removeAcls(resource: Resource): Boolean = {
- if (ZkUtils.pathExists(zkClient, toResourcePath(resource))) {
- ZkUtils.deletePath(zkClient, toResourcePath(resource))
+ if (zkUtils.pathExists(toResourcePath(resource))) {
+ zkUtils.deletePath(toResourcePath(resource))
updateAclChangedFlag(resource)
true
} else false
@@ -206,7 +210,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging {
}
private def getAclsFromZk(resource: Resource): Set[Acl] = {
- val aclJson = ZkUtils.readDataMaybeNull(zkClient, toResourcePath(resource))._1
+ val aclJson = zkUtils.readDataMaybeNull(toResourcePath(resource))._1
aclJson.map(Acl.fromJson).getOrElse(Set.empty)
}
@@ -224,11 +228,11 @@ class SimpleAclAuthorizer extends Authorizer with Logging {
private def loadCache() {
var acls = Set.empty[Acl]
- val resourceTypes = ZkUtils.getChildren(zkClient, SimpleAclAuthorizer.AclZkPath)
+ val resourceTypes = zkUtils.getChildren(SimpleAclAuthorizer.AclZkPath)
for (rType <- resourceTypes) {
val resourceType = ResourceType.fromString(rType)
val resourceTypePath = SimpleAclAuthorizer.AclZkPath + "/" + resourceType.name
- val resourceNames = ZkUtils.getChildren(zkClient, resourceTypePath)
+ val resourceNames = zkUtils.getChildren(resourceTypePath)
for (resourceName <- resourceNames) {
acls = getAclsFromZk(Resource(resourceType, resourceName.toString))
updateCache(new Resource(resourceType, resourceName), acls)
@@ -255,7 +259,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging {
}
private def updateAclChangedFlag(resource: Resource) {
- ZkUtils.createSequentialPersistentPath(zkClient, SimpleAclAuthorizer.AclChangedZkPath + "/" + SimpleAclAuthorizer.AclChangedPrefix, resource.toString)
+ zkUtils.createSequentialPersistentPath(SimpleAclAuthorizer.AclChangedZkPath + "/" + SimpleAclAuthorizer.AclChangedPrefix, resource.toString)
}
object AclChangedNotificaitonHandler extends NotificationHandler {
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/server/DynamicConfigManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/DynamicConfigManager.scala b/core/src/main/scala/kafka/server/DynamicConfigManager.scala
index 4da1833..d443a1f 100644
--- a/core/src/main/scala/kafka/server/DynamicConfigManager.scala
+++ b/core/src/main/scala/kafka/server/DynamicConfigManager.scala
@@ -70,7 +70,7 @@ object ConfigType {
* on startup where a change might be missed between the initial config load and registering for change notifications.
*
*/
-class DynamicConfigManager(private val zkClient: ZkClient,
+class DynamicConfigManager(private val zkUtils: ZkUtils,
private val configHandler : Map[String, ConfigHandler],
private val changeExpirationMs: Long = 15*60*1000,
private val time: Time = SystemTime) extends Logging {
@@ -80,8 +80,8 @@ class DynamicConfigManager(private val zkClient: ZkClient,
* Begin watching for config changes
*/
def startup() {
- ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.EntityConfigChangesPath)
- zkClient.subscribeChildChanges(ZkUtils.EntityConfigChangesPath, ConfigChangeListener)
+ zkUtils.makeSurePersistentPathExists(ZkUtils.EntityConfigChangesPath)
+ zkUtils.zkClient.subscribeChildChanges(ZkUtils.EntityConfigChangesPath, ConfigChangeListener)
processAllConfigChanges()
}
@@ -89,7 +89,7 @@ class DynamicConfigManager(private val zkClient: ZkClient,
* Process all config changes
*/
private def processAllConfigChanges() {
- val configChanges = zkClient.getChildren(ZkUtils.EntityConfigChangesPath)
+ val configChanges = zkUtils.zkClient.getChildren(ZkUtils.EntityConfigChangesPath)
import JavaConversions._
processConfigChanges((configChanges: mutable.Buffer[String]).sorted)
}
@@ -107,7 +107,7 @@ class DynamicConfigManager(private val zkClient: ZkClient,
if (changeId > lastExecutedChange) {
val changeZnode = ZkUtils.EntityConfigChangesPath + "/" + notification
- val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, changeZnode)
+ val (jsonOpt, stat) = zkUtils.readDataMaybeNull(changeZnode)
processNotification(jsonOpt)
}
lastExecutedChange = changeId
@@ -138,7 +138,7 @@ class DynamicConfigManager(private val zkClient: ZkClient,
case Some(value: String) => value
case _ => throw new IllegalArgumentException("Config change notification does not specify 'entity_name'. Received: " + json)
}
- configHandler(entityType).processConfigChanges(entity, AdminUtils.fetchEntityConfig(zkClient, entityType, entity))
+ configHandler(entityType).processConfigChanges(entity, AdminUtils.fetchEntityConfig(zkUtils, entityType, entity))
case o => throw new IllegalArgumentException("Config change notification has an unexpected value. The format is:" +
"{\"version\" : 1," +
@@ -151,12 +151,12 @@ class DynamicConfigManager(private val zkClient: ZkClient,
private def purgeObsoleteNotifications(now: Long, notifications: Seq[String]) {
for(notification <- notifications.sorted) {
- val (jsonOpt, stat) = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.EntityConfigChangesPath + "/" + notification)
+ val (jsonOpt, stat) = zkUtils.readDataMaybeNull(ZkUtils.EntityConfigChangesPath + "/" + notification)
if(jsonOpt.isDefined) {
val changeZnode = ZkUtils.EntityConfigChangesPath + "/" + notification
if (now - stat.getCtime > changeExpirationMs) {
debug("Purging config change notification " + notification)
- ZkUtils.deletePath(zkClient, changeZnode)
+ zkUtils.deletePath(changeZnode)
} else {
return
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/server/KafkaApis.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala
index 5715626..6acab8d 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -44,7 +44,7 @@ class KafkaApis(val requestChannel: RequestChannel,
val replicaManager: ReplicaManager,
val coordinator: ConsumerCoordinator,
val controller: KafkaController,
- val zkClient: ZkClient,
+ val zkUtils: ZkUtils,
val brokerId: Int,
val config: KafkaConfig,
val metadataCache: MetadataCache,
@@ -221,7 +221,7 @@ class KafkaApis(val requestChannel: RequestChannel,
} else if (metaAndError.metadata != null && metaAndError.metadata.length > config.offsetMetadataMaxSize) {
(topicAndPartition, ErrorMapping.OffsetMetadataTooLargeCode)
} else {
- ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" +
+ zkUtils.updatePersistentPath(topicDirs.consumerOffsetDir + "/" +
topicAndPartition.partition, metaAndError.offset.toString)
(topicAndPartition, ErrorMapping.NoError)
}
@@ -535,14 +535,14 @@ class KafkaApis(val requestChannel: RequestChannel,
Math.min(config.offsetsTopicReplicationFactor.toInt, aliveBrokers.length)
else
config.offsetsTopicReplicationFactor.toInt
- AdminUtils.createTopic(zkClient, topic, config.offsetsTopicPartitions,
+ AdminUtils.createTopic(zkUtils, topic, config.offsetsTopicPartitions,
offsetsTopicReplicationFactor,
coordinator.offsetsTopicConfigs)
info("Auto creation of topic %s with %d partitions and replication factor %d is successful!"
.format(topic, config.offsetsTopicPartitions, offsetsTopicReplicationFactor))
}
else {
- AdminUtils.createTopic(zkClient, topic, config.numPartitions, config.defaultReplicationFactor)
+ AdminUtils.createTopic(zkUtils, topic, config.numPartitions, config.defaultReplicationFactor)
info("Auto creation of topic %s with %d partitions and replication factor %d is successful!"
.format(topic, config.numPartitions, config.defaultReplicationFactor))
}
@@ -624,7 +624,7 @@ class KafkaApis(val requestChannel: RequestChannel,
if (metadataCache.getTopicMetadata(Set(topicAndPartition.topic), request.securityProtocol).size <= 0) {
(topicAndPartition, OffsetMetadataAndError.UnknownTopicOrPartition)
} else {
- val payloadOpt = ZkUtils.readDataMaybeNull(zkClient, topicDirs.consumerOffsetDir + "/" + topicAndPartition.partition)._1
+ val payloadOpt = zkUtils.readDataMaybeNull(topicDirs.consumerOffsetDir + "/" + topicAndPartition.partition)._1
payloadOpt match {
case Some(payload) => (topicAndPartition, OffsetMetadataAndError(payload.toLong))
case None => (topicAndPartition, OffsetMetadataAndError.UnknownTopicOrPartition)
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/server/KafkaHealthcheck.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala
index 16760d4..928ff43 100644
--- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala
+++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala
@@ -35,14 +35,13 @@ import java.net.InetAddress
*/
class KafkaHealthcheck(private val brokerId: Int,
private val advertisedEndpoints: Map[SecurityProtocol, EndPoint],
- private val zkClient: ZkClient,
- private val zkConnection: ZkConnection) extends Logging {
+ private val zkUtils: ZkUtils) extends Logging {
val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + brokerId
val sessionExpireListener = new SessionExpireListener
def startup() {
- zkClient.subscribeStateChanges(sessionExpireListener)
+ zkUtils.zkClient.subscribeStateChanges(sessionExpireListener)
register()
}
@@ -62,7 +61,7 @@ class KafkaHealthcheck(private val brokerId: Int,
// only PLAINTEXT is supported as default
// if the broker doesn't listen on PLAINTEXT protocol, an empty endpoint will be registered and older clients will be unable to connect
val plaintextEndpoint = updatedEndpoints.getOrElse(SecurityProtocol.PLAINTEXT, new EndPoint(null,-1,null))
- ZkUtils.registerBrokerInZk(zkClient, zkConnection, brokerId, plaintextEndpoint.host, plaintextEndpoint.port, updatedEndpoints, jmxPort)
+ zkUtils.registerBrokerInZk(brokerId, plaintextEndpoint.host, plaintextEndpoint.port, updatedEndpoints, jmxPort)
}
/**
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/server/KafkaServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
index 99a3c12..f50c266 100755
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -38,6 +38,7 @@ import org.apache.kafka.common.network.{Selectable, ChannelBuilders, NetworkRece
import org.apache.kafka.common.protocol.{Errors, ApiKeys, SecurityProtocol}
import org.apache.kafka.common.metrics.{JmxReporter, Metrics}
import org.apache.kafka.common.requests.{ControlledShutdownResponse, ControlledShutdownRequest, RequestSend}
+import org.apache.kafka.common.security.JaasUtils
import org.apache.kafka.common.security.ssl.SSLFactory
import org.apache.kafka.common.utils.AppInfoParser
@@ -128,8 +129,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
var kafkaHealthcheck: KafkaHealthcheck = null
val metadataCache: MetadataCache = new MetadataCache(config.brokerId)
- var zkClient: ZkClient = null
- var zkConnection: ZkConnection = null
+ var zkUtils: ZkUtils = null
val correlationId: AtomicInteger = new AtomicInteger(0)
val brokerMetaPropsFile = "meta.properties"
val brokerMetadataCheckpoints = config.logDirs.map(logDir => (logDir, new BrokerMetadataCheckpoint(new File(logDir + File.separator +brokerMetaPropsFile)))).toMap
@@ -165,12 +165,10 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
kafkaScheduler.startup()
/* setup zookeeper */
- val (client, connection) = initZk()
- zkClient = client
- zkConnection = connection
+ zkUtils = initZk()
/* start log manager */
- logManager = createLogManager(zkClient, brokerState)
+ logManager = createLogManager(zkUtils.zkClient, brokerState)
logManager.startup()
/* generate brokerId */
@@ -181,16 +179,16 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
socketServer.startup()
/* start replica manager */
- replicaManager = new ReplicaManager(config, metrics, time, kafkaMetricsTime, zkClient, kafkaScheduler, logManager,
+ replicaManager = new ReplicaManager(config, metrics, time, kafkaMetricsTime, zkUtils, kafkaScheduler, logManager,
isShuttingDown)
replicaManager.startup()
/* start kafka controller */
- kafkaController = new KafkaController(config, zkClient, zkConnection, brokerState, kafkaMetricsTime, metrics, threadNamePrefix)
+ kafkaController = new KafkaController(config, zkUtils, brokerState, kafkaMetricsTime, metrics, threadNamePrefix)
kafkaController.startup()
/* start kafka coordinator */
- consumerCoordinator = ConsumerCoordinator.create(config, zkClient, replicaManager, kafkaScheduler)
+ consumerCoordinator = ConsumerCoordinator.create(config, zkUtils, replicaManager, kafkaScheduler)
consumerCoordinator.startup()
/* Get the authorizer and initialize it if one is specified.*/
@@ -204,7 +202,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
/* start processing requests */
apis = new KafkaApis(socketServer.requestChannel, replicaManager, consumerCoordinator,
- kafkaController, zkClient, config.brokerId, config, metadataCache, metrics, authorizer)
+ kafkaController, zkUtils, config.brokerId, config, metadataCache, metrics, authorizer)
requestHandlerPool = new KafkaRequestHandlerPool(config.brokerId, socketServer.requestChannel, apis, config.numIoThreads)
brokerState.newState(RunningAsBroker)
@@ -213,7 +211,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
/* start dynamic config manager */
dynamicConfigHandlers = Map[String, ConfigHandler](ConfigType.Topic -> new TopicConfigHandler(logManager),
ConfigType.Client -> new ClientIdConfigHandler)
- dynamicConfigManager = new DynamicConfigManager(zkClient, dynamicConfigHandlers)
+ dynamicConfigManager = new DynamicConfigManager(zkUtils, dynamicConfigHandlers)
dynamicConfigManager.startup()
/* tell everyone we are alive */
@@ -223,7 +221,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
else
(protocol, endpoint)
}
- kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, listeners, zkClient, zkConnection)
+ kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, listeners, zkUtils)
kafkaHealthcheck.startup()
/* register broker metrics */
@@ -245,7 +243,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
}
}
- private def initZk(): (ZkClient, ZkConnection) = {
+ private def initZk(): ZkUtils = {
info("Connecting to zookeeper on " + config.zkConnect)
val chroot = {
@@ -257,15 +255,21 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
if (chroot.length > 1) {
val zkConnForChrootCreation = config.zkConnect.substring(0, config.zkConnect.indexOf("/"))
- val zkClientForChrootCreation = ZkUtils.createZkClient(zkConnForChrootCreation, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs)
- ZkUtils.makeSurePersistentPathExists(zkClientForChrootCreation, chroot)
+ val zkClientForChrootCreation = ZkUtils(zkConnForChrootCreation,
+ config.zkSessionTimeoutMs,
+ config.zkConnectionTimeoutMs,
+ JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)))
+ zkClientForChrootCreation.makeSurePersistentPathExists(chroot)
info("Created zookeeper path " + chroot)
- zkClientForChrootCreation.close()
+ zkClientForChrootCreation.zkClient.close()
}
- val (zkClient, zkConnection) = ZkUtils.createZkClientAndConnection(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs)
- ZkUtils.setupCommonPaths(zkClient)
- (zkClient, zkConnection)
+ val zkUtils = ZkUtils(config.zkConnect,
+ config.zkSessionTimeoutMs,
+ config.zkConnectionTimeoutMs,
+ JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)))
+ zkUtils.setupCommonPaths()
+ zkUtils
}
@@ -334,8 +338,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
// Get the current controller info. This is to ensure we use the most recent info to issue the
// controlled shutdown request
- val controllerId = ZkUtils.getController(zkClient)
- ZkUtils.getBrokerInfo(zkClient, controllerId) match {
+ val controllerId = zkUtils.getController()
+ zkUtils.getBrokerInfo(controllerId) match {
case Some(broker) =>
// if this is the first attempt, if the controller has changed or if an exception was thrown in a previous
// attempt, connect to the most recent controller
@@ -410,8 +414,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
// Get the current controller info. This is to ensure we use the most recent info to issue the
// controlled shutdown request
- val controllerId = ZkUtils.getController(zkClient)
- ZkUtils.getBrokerInfo(zkClient, controllerId) match {
+ val controllerId = zkUtils.getController()
+ zkUtils.getBrokerInfo(controllerId) match {
case Some(broker) =>
if (channel == null || prevController == null || !prevController.equals(broker)) {
// if this is the first attempt or if the controller has changed, create a channel to the most recent
@@ -524,8 +528,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
CoreUtils.swallow(consumerCoordinator.shutdown())
if(kafkaController != null)
CoreUtils.swallow(kafkaController.shutdown())
- if(zkClient != null)
- CoreUtils.swallow(zkClient.close())
+ if(zkUtils != null)
+ CoreUtils.swallow(zkUtils.close())
if (metrics != null)
CoreUtils.swallow(metrics.close())
@@ -559,7 +563,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
val defaultProps = KafkaServer.copyKafkaConfigToLog(config)
val defaultLogConfig = LogConfig(defaultProps)
- val configs = AdminUtils.fetchAllTopicConfigs(zkClient).mapValues(LogConfig.fromProps(defaultProps, _))
+ val configs = AdminUtils.fetchAllTopicConfigs(zkUtils).mapValues(LogConfig.fromProps(defaultProps, _))
// read the log configurations from zookeeper
val cleanerConfig = CleanerConfig(numThreads = config.logCleanerThreads,
dedupeBufferSize = config.logCleanerDedupeBufferSize,
@@ -626,7 +630,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
private def generateBrokerId: Int = {
try {
- ZkUtils.getBrokerSequenceId(zkClient, config.maxReservedBrokerId)
+ zkUtils.getBrokerSequenceId(config.maxReservedBrokerId)
} catch {
case e: Exception =>
error("Failed to generate broker.id due to ", e)
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/server/OffsetManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala
index 0e613e7..bdc3bb6 100755
--- a/core/src/main/scala/kafka/server/OffsetManager.scala
+++ b/core/src/main/scala/kafka/server/OffsetManager.scala
@@ -87,7 +87,7 @@ object OffsetManagerConfig {
class OffsetManager(val config: OffsetManagerConfig,
replicaManager: ReplicaManager,
- zkClient: ZkClient,
+ zkUtils: ZkUtils,
scheduler: Scheduler) extends Logging with KafkaMetricsGroup {
/* offsets and metadata cache */
@@ -449,7 +449,7 @@ class OffsetManager(val config: OffsetManagerConfig,
*/
private def getOffsetsTopicPartitionCount = {
val topic = ConsumerCoordinator.OffsetsTopicName
- val topicData = ZkUtils.getPartitionAssignmentForTopics(zkClient, Seq(topic))
+ val topicData = zkUtils.getPartitionAssignmentForTopics(Seq(topic))
if (topicData(topic).nonEmpty)
topicData(topic).size
else
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
index 82a6001..0a17fd0 100644
--- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
@@ -144,7 +144,7 @@ class ReplicaFetcherThread(name: String,
// Prior to truncating the follower's log, ensure that doing so is not disallowed by the configuration for unclean leader election.
// This situation could only happen if the unclean election configuration for a topic changes while a replica is down. Otherwise,
// we should never encounter this situation since a non-ISR leader cannot be elected if disallowed by the broker configuration.
- if (!LogConfig.fromProps(brokerConfig.originals, AdminUtils.fetchEntityConfig(replicaMgr.zkClient,
+ if (!LogConfig.fromProps(brokerConfig.originals, AdminUtils.fetchEntityConfig(replicaMgr.zkUtils,
ConfigType.Topic, topicAndPartition.topic)).uncleanLeaderElectionEnable) {
// Log a fatal error and shutdown the broker to ensure that data loss does not unexpectedly occur.
fatal("Halting because log truncation is not allowed for topic %s,".format(topicAndPartition.topic) +
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/server/ReplicaManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala
index c0fec67..1fc47f4 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -99,7 +99,7 @@ class ReplicaManager(val config: KafkaConfig,
metrics: Metrics,
time: Time,
jTime: JTime,
- val zkClient: ZkClient,
+ val zkUtils: ZkUtils,
scheduler: Scheduler,
val logManager: LogManager,
val isShuttingDown: AtomicBoolean,
@@ -163,7 +163,7 @@ class ReplicaManager(val config: KafkaConfig,
def maybePropagateIsrChanges() {
isrChangeSet synchronized {
if (isrChangeSet.nonEmpty) {
- ReplicationUtils.propagateIsrChanges(zkClient, isrChangeSet)
+ ReplicationUtils.propagateIsrChanges(zkUtils, isrChangeSet)
isrChangeSet.clear()
}
}
[5/5] kafka git commit: KAFKA-2639: Refactoring of ZkUtils
Posted by ju...@apache.org.
KAFKA-2639: Refactoring of ZkUtils
I've split the work of KAFKA-1695 because this refactoring touches a large number of files. Most of the changes are trivial, but I feel it will be easier to review this way.
This pull request includes the one Parth-Brahmbhatt started to address KAFKA-1695.
Author: flavio junqueira <fp...@apache.org>
Author: Flavio Junqueira <fp...@apache.org>
Reviewers: Ismael Juma <is...@juma.me.uk>, Jun Rao <ju...@gmail.com>
Closes #303 from fpj/KAFKA-2639
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/ce306ba4
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/ce306ba4
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/ce306ba4
Branch: refs/heads/trunk
Commit: ce306ba4ebc77464bf8ff4d656e1f1f44979182e
Parents: 78a2e2f
Author: flavio junqueira <fp...@apache.org>
Authored: Sun Oct 18 15:23:52 2015 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Sun Oct 18 15:23:52 2015 -0700
----------------------------------------------------------------------
checkstyle/import-control.xml | 1 +
.../apache/kafka/common/security/JaasUtils.java | 64 +++
.../src/main/scala/kafka/admin/AdminUtils.scala | 129 +++---
.../main/scala/kafka/admin/ConfigCommand.scala | 26 +-
.../kafka/admin/ConsumerGroupCommand.scala | 74 ++--
.../PreferredReplicaLeaderElectionCommand.scala | 27 +-
.../kafka/admin/ReassignPartitionsCommand.scala | 67 ++--
.../main/scala/kafka/admin/TopicCommand.scala | 65 +--
.../main/scala/kafka/client/ClientUtils.scala | 12 +-
.../main/scala/kafka/cluster/Partition.scala | 6 +-
.../ZkNodeChangeNotificationListener.scala | 14 +-
.../kafka/consumer/ConsumerFetcherManager.scala | 6 +-
.../kafka/consumer/PartitionAssignor.scala | 10 +-
.../main/scala/kafka/consumer/TopicCount.scala | 14 +-
.../consumer/ZookeeperConsumerConnector.scala | 58 +--
.../consumer/ZookeeperTopicEventWatcher.scala | 20 +-
.../kafka/controller/KafkaController.scala | 89 ++---
.../controller/PartitionLeaderSelector.scala | 2 +-
.../controller/PartitionStateMachine.scala | 39 +-
.../kafka/controller/ReplicaStateMachine.scala | 12 +-
.../kafka/controller/TopicDeletionManager.scala | 12 +-
.../kafka/coordinator/ConsumerCoordinator.scala | 16 +-
.../kafka/coordinator/CoordinatorMetadata.scala | 10 +-
.../security/auth/SimpleAclAuthorizer.scala | 42 +-
.../kafka/server/DynamicConfigManager.scala | 16 +-
.../src/main/scala/kafka/server/KafkaApis.scala | 10 +-
.../scala/kafka/server/KafkaHealthcheck.scala | 7 +-
.../main/scala/kafka/server/KafkaServer.scala | 58 +--
.../main/scala/kafka/server/OffsetManager.scala | 4 +-
.../kafka/server/ReplicaFetcherThread.scala | 2 +-
.../scala/kafka/server/ReplicaManager.scala | 4 +-
.../kafka/server/ZookeeperLeaderElector.scala | 12 +-
.../kafka/tools/ConsumerOffsetChecker.scala | 40 +-
.../scala/kafka/tools/ExportZkOffsets.scala | 24 +-
.../scala/kafka/tools/ImportZkOffsets.scala | 9 +-
.../scala/kafka/tools/UpdateOffsetsInZK.scala | 19 +-
.../kafka/tools/VerifyConsumerRebalance.scala | 25 +-
.../scala/kafka/utils/ReplicationUtils.scala | 25 +-
core/src/main/scala/kafka/utils/ZkUtils.scala | 399 +++++++++++--------
.../kafka/api/ConsumerBounceTest.scala | 2 +-
.../integration/kafka/api/ConsumerTest.scala | 26 +-
.../kafka/api/IntegrationTestHarness.scala | 2 +-
.../kafka/api/ProducerBounceTest.scala | 4 +-
.../kafka/api/ProducerCompressionTest.scala | 2 +-
.../kafka/api/ProducerFailureHandlingTest.scala | 14 +-
.../kafka/api/ProducerSendTest.scala | 14 +-
.../integration/kafka/api/QuotasTest.scala | 2 +-
.../integration/kafka/api/SSLConsumerTest.scala | 6 +-
.../kafka/api/SSLProducerSendTest.scala | 6 +-
.../test/scala/other/kafka/DeleteZKPath.scala | 4 +-
.../scala/other/kafka/TestOffsetManager.scala | 24 +-
.../unit/kafka/admin/AddPartitionsTest.scala | 34 +-
.../test/scala/unit/kafka/admin/AdminTest.scala | 107 +++--
.../kafka/admin/DeleteConsumerGroupTest.scala | 48 +--
.../unit/kafka/admin/DeleteTopicTest.scala | 85 ++--
.../unit/kafka/admin/TopicCommandTest.scala | 36 +-
.../ZkNodeChangeNotificationListenerTest.scala | 6 +-
.../kafka/consumer/ConsumerIteratorTest.scala | 2 +-
.../kafka/consumer/PartitionAssignorTest.scala | 23 +-
.../ZookeeperConsumerConnectorTest.scala | 38 +-
.../controller/ControllerFailoverTest.scala | 2 +-
.../coordinator/CoordinatorMetadataTest.scala | 52 +--
.../kafka/integration/AutoOffsetResetTest.scala | 2 +-
.../integration/BaseTopicMetadataTest.scala | 14 +-
.../unit/kafka/integration/FetcherTest.scala | 4 +-
.../kafka/integration/PrimitiveApiTest.scala | 8 +-
.../kafka/integration/RollingBounceTest.scala | 10 +-
.../integration/UncleanLeaderElectionTest.scala | 22 +-
.../ZookeeperConsumerConnectorTest.scala | 2 +-
.../scala/unit/kafka/metrics/MetricsTest.scala | 8 +-
.../unit/kafka/producer/ProducerTest.scala | 18 +-
.../unit/kafka/producer/SyncProducerTest.scala | 18 +-
.../security/auth/SimpleAclAuthorizerTest.scala | 6 +-
.../unit/kafka/server/AdvertiseBrokerTest.scala | 2 +-
.../kafka/server/BaseReplicaFetchTest.scala | 2 +-
.../kafka/server/DynamicConfigChangeTest.scala | 10 +-
.../server/HighwatermarkPersistenceTest.scala | 18 +-
.../unit/kafka/server/KafkaConfigTest.scala | 2 +-
.../unit/kafka/server/LeaderElectionTest.scala | 18 +-
.../scala/unit/kafka/server/LogOffsetTest.scala | 8 +-
.../unit/kafka/server/LogRecoveryTest.scala | 14 +-
.../unit/kafka/server/OffsetCommitTest.scala | 16 +-
.../unit/kafka/server/ReplicaManagerTest.scala | 11 +-
.../unit/kafka/server/ServerShutdownTest.scala | 2 +-
.../unit/kafka/server/ServerStartupTest.scala | 6 +-
.../unit/kafka/server/SimpleFetchTest.scala | 6 +-
.../unit/kafka/utils/ReplicationUtilsTest.scala | 16 +-
.../test/scala/unit/kafka/utils/TestUtils.scala | 57 +--
.../scala/unit/kafka/zk/ZKEphemeralTest.scala | 26 +-
.../test/scala/unit/kafka/zk/ZKPathTest.scala | 48 +--
.../unit/kafka/zk/ZooKeeperTestHarness.scala | 10 +-
91 files changed, 1283 insertions(+), 1109 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/checkstyle/import-control.xml
----------------------------------------------------------------------
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 247f556..289f1d0 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -31,6 +31,7 @@
<allow pkg="org.powermock" />
<allow pkg="javax.net.ssl" />
+ <allow pkg="javax.security.auth" />
<!-- no one depends on the server -->
<disallow pkg="kafka" />
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java b/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java
new file mode 100644
index 0000000..ce0be62
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java
@@ -0,0 +1,64 @@
+/**
+ * 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.common.security;
+
+import java.io.File;
+import java.net.URI;
+import java.security.URIParameter;
+import javax.security.auth.login.Configuration;
+import org.apache.kafka.common.KafkaException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JaasUtils {
+ private static final Logger LOG = LoggerFactory.getLogger(JaasUtils.class);
+ public static final String LOGIN_CONTEXT_SERVER = "KafkaServer";
+ public static final String LOGIN_CONTEXT_CLIENT = "KafkaClient";
+ public static final String SERVICE_NAME = "serviceName";
+ public static final String JAVA_LOGIN_CONFIG_PARAM = "java.security.auth.login.config";
+ public static final String ZK_SASL_CLIENT = "zookeeper.sasl.client";
+ public static final String ZK_LOGIN_CONTEXT_NAME_KEY = "zookeeper.sasl.clientconfig";
+
+ public static boolean isZkSecurityEnabled(String loginConfigFile) {
+ boolean isSecurityEnabled = false;
+ boolean zkSaslEnabled = Boolean.getBoolean(System.getProperty(ZK_SASL_CLIENT, "true"));
+ String zkLoginContextName = System.getProperty(ZK_LOGIN_CONTEXT_NAME_KEY, "Client");
+
+ if (loginConfigFile != null && loginConfigFile.length() > 0) {
+ File configFile = new File(loginConfigFile);
+ if (!configFile.canRead()) {
+ throw new KafkaException("File " + loginConfigFile + "cannot be read.");
+ }
+ try {
+ URI configUri = configFile.toURI();
+ Configuration loginConf = Configuration.getInstance("JavaLoginConfig", new URIParameter(configUri));
+ isSecurityEnabled = loginConf.getAppConfigurationEntry(zkLoginContextName) != null;
+ } catch (Exception e) {
+ throw new KafkaException(e);
+ }
+ if (isSecurityEnabled && !zkSaslEnabled) {
+ LOG.error("JAAS file is present, but system property " +
+ ZK_SASL_CLIENT + " is set to false, which disables " +
+ "SASL in the ZooKeeper client");
+ throw new KafkaException("Exception while determining if ZooKeeper is secure");
+ }
+ }
+
+ return isSecurityEnabled;
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/admin/AdminUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala
index 9966660..ecc5b9d 100644
--- a/core/src/main/scala/kafka/admin/AdminUtils.scala
+++ b/core/src/main/scala/kafka/admin/AdminUtils.scala
@@ -23,6 +23,7 @@ import kafka.cluster.{BrokerEndPoint, Broker}
import kafka.log.LogConfig
import kafka.server.ConfigType
import kafka.utils._
+import kafka.utils.ZkUtils._
import kafka.api.{TopicMetadata, PartitionMetadata}
import java.util.Random
@@ -103,12 +104,12 @@ object AdminUtils extends Logging {
* @param replicaAssignmentStr Manual replica assignment
* @param checkBrokerAvailable Ignore checking if assigned replica broker is available. Only used for testing
*/
- def addPartitions(zkClient: ZkClient,
+ def addPartitions(zkUtils: ZkUtils,
topic: String,
numPartitions: Int = 1,
replicaAssignmentStr: String = "",
checkBrokerAvailable: Boolean = true) {
- val existingPartitionsReplicaList = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic))
+ val existingPartitionsReplicaList = zkUtils.getReplicaAssignmentForTopics(List(topic))
if (existingPartitionsReplicaList.size == 0)
throw new AdminOperationException("The topic %s does not exist".format(topic))
@@ -118,7 +119,7 @@ object AdminUtils extends Logging {
throw new AdminOperationException("The number of partitions for a topic can only be increased")
// create the new partition replication list
- val brokerList = ZkUtils.getSortedBrokerList(zkClient)
+ val brokerList = zkUtils.getSortedBrokerList()
val newPartitionReplicaList = if (replicaAssignmentStr == null || replicaAssignmentStr == "")
AdminUtils.assignReplicasToBrokers(brokerList, partitionsToAdd, existingReplicaList.size, existingReplicaList.head, existingPartitionsReplicaList.size)
else
@@ -134,7 +135,7 @@ object AdminUtils extends Logging {
val partitionReplicaList = existingPartitionsReplicaList.map(p => p._1.partition -> p._2)
// add the new list
partitionReplicaList ++= newPartitionReplicaList
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, update = true)
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, partitionReplicaList, update = true)
}
def getManualReplicaAssignment(replicaAssignmentList: String, availableBrokerList: Set[Int], startPartitionId: Int, checkBrokerAvailable: Boolean = true): Map[Int, List[Int]] = {
@@ -159,9 +160,9 @@ object AdminUtils extends Logging {
ret.toMap
}
- def deleteTopic(zkClient: ZkClient, topic: String) {
+ def deleteTopic(zkUtils: ZkUtils, topic: String) {
try {
- ZkUtils.createPersistentPath(zkClient, ZkUtils.getDeleteTopicPath(topic))
+ zkUtils.createPersistentPath(getDeleteTopicPath(topic))
} catch {
case e1: ZkNodeExistsException => throw new TopicAlreadyMarkedForDeletionException(
"topic %s is already marked for deletion".format(topic))
@@ -169,8 +170,8 @@ object AdminUtils extends Logging {
}
}
- def isConsumerGroupActive(zkClient: ZkClient, group: String) = {
- ZkUtils.getConsumersInGroup(zkClient, group).nonEmpty
+ def isConsumerGroupActive(zkUtils: ZkUtils, group: String) = {
+ zkUtils.getConsumersInGroup(group).nonEmpty
}
/**
@@ -180,10 +181,10 @@ object AdminUtils extends Logging {
* @param group Consumer group
* @return whether or not we deleted the consumer group information
*/
- def deleteConsumerGroupInZK(zkClient: ZkClient, group: String) = {
- if (!isConsumerGroupActive(zkClient, group)) {
+ def deleteConsumerGroupInZK(zkUtils: ZkUtils, group: String) = {
+ if (!isConsumerGroupActive(zkUtils, group)) {
val dir = new ZKGroupDirs(group)
- ZkUtils.deletePathRecursive(zkClient, dir.consumerGroupDir)
+ zkUtils.deletePathRecursive(dir.consumerGroupDir)
true
}
else false
@@ -198,15 +199,15 @@ object AdminUtils extends Logging {
* @param topic Topic of the consumer group information we wish to delete
* @return whether or not we deleted the consumer group information for the given topic
*/
- def deleteConsumerGroupInfoForTopicInZK(zkClient: ZkClient, group: String, topic: String) = {
- val topics = ZkUtils.getTopicsByConsumerGroup(zkClient, group)
+ def deleteConsumerGroupInfoForTopicInZK(zkUtils: ZkUtils, group: String, topic: String) = {
+ val topics = zkUtils.getTopicsByConsumerGroup(group)
if (topics == Seq(topic)) {
- deleteConsumerGroupInZK(zkClient, group)
+ deleteConsumerGroupInZK(zkUtils, group)
}
- else if (!isConsumerGroupActive(zkClient, group)) {
+ else if (!isConsumerGroupActive(zkUtils, group)) {
val dir = new ZKGroupTopicDirs(group, topic)
- ZkUtils.deletePathRecursive(zkClient, dir.consumerOwnerDir)
- ZkUtils.deletePathRecursive(zkClient, dir.consumerOffsetDir)
+ zkUtils.deletePathRecursive(dir.consumerOwnerDir)
+ zkUtils.deletePathRecursive(dir.consumerOffsetDir)
true
}
else false
@@ -218,25 +219,25 @@ object AdminUtils extends Logging {
* @param zkClient Zookeeper client
* @param topic Topic of the consumer group information we wish to delete
*/
- def deleteAllConsumerGroupInfoForTopicInZK(zkClient: ZkClient, topic: String) {
- val groups = ZkUtils.getAllConsumerGroupsForTopic(zkClient, topic)
- groups.foreach(group => deleteConsumerGroupInfoForTopicInZK(zkClient, group, topic))
+ def deleteAllConsumerGroupInfoForTopicInZK(zkUtils: ZkUtils, topic: String) {
+ val groups = zkUtils.getAllConsumerGroupsForTopic(topic)
+ groups.foreach(group => deleteConsumerGroupInfoForTopicInZK(zkUtils, group, topic))
}
- def topicExists(zkClient: ZkClient, topic: String): Boolean =
- zkClient.exists(ZkUtils.getTopicPath(topic))
+ def topicExists(zkUtils: ZkUtils, topic: String): Boolean =
+ zkUtils.zkClient.exists(getTopicPath(topic))
- def createTopic(zkClient: ZkClient,
+ def createTopic(zkUtils: ZkUtils,
topic: String,
partitions: Int,
replicationFactor: Int,
topicConfig: Properties = new Properties) {
- val brokerList = ZkUtils.getSortedBrokerList(zkClient)
+ val brokerList = zkUtils.getSortedBrokerList()
val replicaAssignment = AdminUtils.assignReplicasToBrokers(brokerList, partitions, replicationFactor)
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, replicaAssignment, topicConfig)
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, replicaAssignment, topicConfig)
}
- def createOrUpdateTopicPartitionAssignmentPathInZK(zkClient: ZkClient,
+ def createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils: ZkUtils,
topic: String,
partitionReplicaAssignment: Map[Int, Seq[Int]],
config: Properties = new Properties,
@@ -245,13 +246,13 @@ object AdminUtils extends Logging {
Topic.validate(topic)
require(partitionReplicaAssignment.values.map(_.size).toSet.size == 1, "All partitions should have the same number of replicas.")
- val topicPath = ZkUtils.getTopicPath(topic)
+ val topicPath = getTopicPath(topic)
if (!update) {
- if (zkClient.exists(topicPath))
+ if (zkUtils.zkClient.exists(topicPath))
throw new TopicExistsException("Topic \"%s\" already exists.".format(topic))
else if (Topic.hasCollisionChars(topic)) {
- val allTopics = ZkUtils.getAllTopics(zkClient)
+ val allTopics = zkUtils.getAllTopics()
val collidingTopics = allTopics.filter(t => Topic.hasCollision(topic, t))
if (collidingTopics.nonEmpty) {
throw new InvalidTopicException("Topic \"%s\" collides with existing topics: %s".format(topic, collidingTopics.mkString(", ")))
@@ -265,24 +266,24 @@ object AdminUtils extends Logging {
if (!update) {
// write out the config if there is any, this isn't transactional with the partition assignments
LogConfig.validate(config)
- writeEntityConfig(zkClient, ConfigType.Topic, topic, config)
+ writeEntityConfig(zkUtils, ConfigType.Topic, topic, config)
}
// create the partition assignment
- writeTopicPartitionAssignment(zkClient, topic, partitionReplicaAssignment, update)
+ writeTopicPartitionAssignment(zkUtils, topic, partitionReplicaAssignment, update)
}
- private def writeTopicPartitionAssignment(zkClient: ZkClient, topic: String, replicaAssignment: Map[Int, Seq[Int]], update: Boolean) {
+ private def writeTopicPartitionAssignment(zkUtils: ZkUtils, topic: String, replicaAssignment: Map[Int, Seq[Int]], update: Boolean) {
try {
- val zkPath = ZkUtils.getTopicPath(topic)
- val jsonPartitionData = ZkUtils.replicaAssignmentZkData(replicaAssignment.map(e => (e._1.toString -> e._2)))
+ val zkPath = getTopicPath(topic)
+ val jsonPartitionData = zkUtils.replicaAssignmentZkData(replicaAssignment.map(e => (e._1.toString -> e._2)))
if (!update) {
info("Topic creation " + jsonPartitionData.toString)
- ZkUtils.createPersistentPath(zkClient, zkPath, jsonPartitionData)
+ zkUtils.createPersistentPath(zkPath, jsonPartitionData)
} else {
info("Topic update " + jsonPartitionData.toString)
- ZkUtils.updatePersistentPath(zkClient, zkPath, jsonPartitionData)
+ zkUtils.updatePersistentPath(zkPath, jsonPartitionData)
}
debug("Updated path %s with %s for replica assignment".format(zkPath, jsonPartitionData))
} catch {
@@ -299,8 +300,8 @@ object AdminUtils extends Logging {
* existing configs need to be deleted, it should be done prior to invoking this API
*
*/
- def changeClientIdConfig(zkClient: ZkClient, clientId: String, configs: Properties) {
- changeEntityConfig(zkClient, ConfigType.Client, clientId, configs)
+ def changeClientIdConfig(zkUtils: ZkUtils, clientId: String, configs: Properties) {
+ changeEntityConfig(zkUtils, ConfigType.Client, clientId, configs)
}
/**
@@ -311,22 +312,22 @@ object AdminUtils extends Logging {
* existing configs need to be deleted, it should be done prior to invoking this API
*
*/
- def changeTopicConfig(zkClient: ZkClient, topic: String, configs: Properties) {
- if(!topicExists(zkClient, topic))
+ def changeTopicConfig(zkUtils: ZkUtils, topic: String, configs: Properties) {
+ if(!topicExists(zkUtils, topic))
throw new AdminOperationException("Topic \"%s\" does not exist.".format(topic))
// remove the topic overrides
LogConfig.validate(configs)
- changeEntityConfig(zkClient, ConfigType.Topic, topic, configs)
+ changeEntityConfig(zkUtils, ConfigType.Topic, topic, configs)
}
- private def changeEntityConfig(zkClient: ZkClient, entityType: String, entityName: String, configs: Properties) {
+ private def changeEntityConfig(zkUtils: ZkUtils, entityType: String, entityName: String, configs: Properties) {
// write the new config--may not exist if there were previously no overrides
- writeEntityConfig(zkClient, entityType, entityName, configs)
+ writeEntityConfig(zkUtils, entityType, entityName, configs)
// create the change notification
val seqNode = ZkUtils.EntityConfigChangesPath + "/" + EntityConfigChangeZnodePrefix
val content = Json.encode(getConfigChangeZnodeData(entityType, entityName))
- zkClient.createPersistentSequential(seqNode, content)
+ zkUtils.zkClient.createPersistentSequential(seqNode, content)
}
def getConfigChangeZnodeData(entityType: String, entityName: String) : Map[String, Any] = {
@@ -336,20 +337,20 @@ object AdminUtils extends Logging {
/**
* Write out the topic config to zk, if there is any
*/
- private def writeEntityConfig(zkClient: ZkClient, entityType: String, entityName: String, config: Properties) {
+ private def writeEntityConfig(zkUtils: ZkUtils, entityType: String, entityName: String, config: Properties) {
val configMap: mutable.Map[String, String] = {
import JavaConversions._
config
}
val map = Map("version" -> 1, "config" -> configMap)
- ZkUtils.updatePersistentPath(zkClient, ZkUtils.getEntityConfigPath(entityType, entityName), Json.encode(map))
+ zkUtils.updatePersistentPath(getEntityConfigPath(entityType, entityName), Json.encode(map))
}
/**
* Read the entity (topic or client) config (if any) from zk
*/
- def fetchEntityConfig(zkClient: ZkClient, entityType: String, entity: String): Properties = {
- val str: String = zkClient.readData(ZkUtils.getEntityConfigPath(entityType, entity), true)
+ def fetchEntityConfig(zkUtils: ZkUtils, entityType: String, entity: String): Properties = {
+ val str: String = zkUtils.zkClient.readData(getEntityConfigPath(entityType, entity), true)
val props = new Properties()
if(str != null) {
Json.parseFull(str) match {
@@ -375,28 +376,28 @@ object AdminUtils extends Logging {
props
}
- def fetchAllTopicConfigs(zkClient: ZkClient): Map[String, Properties] =
- ZkUtils.getAllTopics(zkClient).map(topic => (topic, fetchEntityConfig(zkClient, ConfigType.Topic, topic))).toMap
+ def fetchAllTopicConfigs(zkUtils: ZkUtils): Map[String, Properties] =
+ zkUtils.getAllTopics().map(topic => (topic, fetchEntityConfig(zkUtils, ConfigType.Topic, topic))).toMap
- def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient): TopicMetadata =
- fetchTopicMetadataFromZk(topic, zkClient, new mutable.HashMap[Int, Broker])
+ def fetchTopicMetadataFromZk(topic: String, zkUtils: ZkUtils): TopicMetadata =
+ fetchTopicMetadataFromZk(topic, zkUtils, new mutable.HashMap[Int, Broker])
- def fetchTopicMetadataFromZk(topics: Set[String], zkClient: ZkClient): Set[TopicMetadata] = {
+ def fetchTopicMetadataFromZk(topics: Set[String], zkUtils: ZkUtils): Set[TopicMetadata] = {
val cachedBrokerInfo = new mutable.HashMap[Int, Broker]()
- topics.map(topic => fetchTopicMetadataFromZk(topic, zkClient, cachedBrokerInfo))
+ topics.map(topic => fetchTopicMetadataFromZk(topic, zkUtils, cachedBrokerInfo))
}
- private def fetchTopicMetadataFromZk(topic: String, zkClient: ZkClient, cachedBrokerInfo: mutable.HashMap[Int, Broker], protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): TopicMetadata = {
- if(ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic))) {
- val topicPartitionAssignment = ZkUtils.getPartitionAssignmentForTopics(zkClient, List(topic)).get(topic).get
+ private def fetchTopicMetadataFromZk(topic: String, zkUtils: ZkUtils, cachedBrokerInfo: mutable.HashMap[Int, Broker], protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): TopicMetadata = {
+ if(zkUtils.pathExists(getTopicPath(topic))) {
+ val topicPartitionAssignment = zkUtils.getPartitionAssignmentForTopics(List(topic)).get(topic).get
val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1)
val partitionMetadata = sortedPartitions.map { partitionMap =>
val partition = partitionMap._1
val replicas = partitionMap._2
- val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic, partition)
- val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partition)
+ val inSyncReplicas = zkUtils.getInSyncReplicasForPartition(topic, partition)
+ val leader = zkUtils.getLeaderForPartition(topic, partition)
debug("replicas = " + replicas + ", in sync replicas = " + inSyncReplicas + ", leader = " + leader)
var leaderInfo: Option[BrokerEndPoint] = None
@@ -406,15 +407,15 @@ object AdminUtils extends Logging {
leaderInfo = leader match {
case Some(l) =>
try {
- Some(getBrokerInfoFromCache(zkClient, cachedBrokerInfo, List(l)).head.getBrokerEndPoint(protocol))
+ Some(getBrokerInfoFromCache(zkUtils, cachedBrokerInfo, List(l)).head.getBrokerEndPoint(protocol))
} catch {
case e: Throwable => throw new LeaderNotAvailableException("Leader not available for partition [%s,%d]".format(topic, partition), e)
}
case None => throw new LeaderNotAvailableException("No leader exists for partition " + partition)
}
try {
- replicaInfo = getBrokerInfoFromCache(zkClient, cachedBrokerInfo, replicas).map(_.getBrokerEndPoint(protocol))
- isrInfo = getBrokerInfoFromCache(zkClient, cachedBrokerInfo, inSyncReplicas).map(_.getBrokerEndPoint(protocol))
+ replicaInfo = getBrokerInfoFromCache(zkUtils, cachedBrokerInfo, replicas).map(_.getBrokerEndPoint(protocol))
+ isrInfo = getBrokerInfoFromCache(zkUtils, cachedBrokerInfo, inSyncReplicas).map(_.getBrokerEndPoint(protocol))
} catch {
case e: Throwable => throw new ReplicaNotAvailableException(e)
}
@@ -439,7 +440,7 @@ object AdminUtils extends Logging {
}
}
- private def getBrokerInfoFromCache(zkClient: ZkClient,
+ private def getBrokerInfoFromCache(zkUtils: ZkUtils,
cachedBrokerInfo: scala.collection.mutable.Map[Int, Broker],
brokerIds: Seq[Int]): Seq[Broker] = {
var failedBrokerIds: ListBuffer[Int] = new ListBuffer()
@@ -448,7 +449,7 @@ object AdminUtils extends Logging {
optionalBrokerInfo match {
case Some(brokerInfo) => Some(brokerInfo) // return broker info from the cache
case None => // fetch it from zookeeper
- ZkUtils.getBrokerInfo(zkClient, id) match {
+ zkUtils.getBrokerInfo(id) match {
case Some(brokerInfo) =>
cachedBrokerInfo += (id -> brokerInfo)
Some(brokerInfo)
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/admin/ConfigCommand.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala
index f0217de..ba4c003 100644
--- a/core/src/main/scala/kafka/admin/ConfigCommand.scala
+++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala
@@ -26,6 +26,7 @@ import org.I0Itec.zkclient.ZkClient
import scala.collection._
import scala.collection.JavaConversions._
import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.common.security.JaasUtils
/**
@@ -42,52 +43,55 @@ object ConfigCommand {
opts.checkArgs()
- val zkClient = ZkUtils.createZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000)
+ val zkUtils = ZkUtils(opts.options.valueOf(opts.zkConnectOpt),
+ 30000,
+ 30000,
+ JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)))
try {
if (opts.options.has(opts.alterOpt))
- alterConfig(zkClient, opts)
+ alterConfig(zkUtils, opts)
else if (opts.options.has(opts.describeOpt))
- describeConfig(zkClient, opts)
+ describeConfig(zkUtils, opts)
} catch {
case e: Throwable =>
println("Error while executing topic command " + e.getMessage)
println(Utils.stackTrace(e))
} finally {
- zkClient.close()
+ zkUtils.close()
}
}
- private def alterConfig(zkClient: ZkClient, opts: ConfigCommandOptions) {
+ private def alterConfig(zkUtils: ZkUtils, opts: ConfigCommandOptions) {
val configsToBeAdded = parseConfigsToBeAdded(opts)
val configsToBeDeleted = parseConfigsToBeDeleted(opts)
val entityType = opts.options.valueOf(opts.entityType)
val entityName = opts.options.valueOf(opts.entityName)
// compile the final set of configs
- val configs = AdminUtils.fetchEntityConfig(zkClient, entityType, entityName)
+ val configs = AdminUtils.fetchEntityConfig(zkUtils, entityType, entityName)
configs.putAll(configsToBeAdded)
configsToBeDeleted.foreach(config => configs.remove(config))
if (entityType.equals(ConfigType.Topic)) {
- AdminUtils.changeTopicConfig(zkClient, entityName, configs)
+ AdminUtils.changeTopicConfig(zkUtils, entityName, configs)
println("Updated config for topic: \"%s\".".format(entityName))
} else {
- AdminUtils.changeClientIdConfig(zkClient, entityName, configs)
+ AdminUtils.changeClientIdConfig(zkUtils, entityName, configs)
println("Updated config for clientId: \"%s\".".format(entityName))
}
}
- private def describeConfig(zkClient: ZkClient, opts: ConfigCommandOptions) {
+ private def describeConfig(zkUtils: ZkUtils, opts: ConfigCommandOptions) {
val entityType = opts.options.valueOf(opts.entityType)
val entityNames: Seq[String] =
if (opts.options.has(opts.entityName))
Seq(opts.options.valueOf(opts.entityName))
else
- ZkUtils.getAllEntitiesWithConfig(zkClient, entityType)
+ zkUtils.getAllEntitiesWithConfig(entityType)
for (entityName <- entityNames) {
- val configs = AdminUtils.fetchEntityConfig(zkClient, entityType, entityName)
+ val configs = AdminUtils.fetchEntityConfig(zkUtils, entityType, entityName)
println("Configs for %s:%s are %s"
.format(entityType, entityName, configs.map(kv => kv._1 + "=" + kv._2).mkString(",")))
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala
index f23120e..8efbb2a 100755
--- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala
+++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala
@@ -31,6 +31,7 @@ import scala.collection.{Set, mutable}
import kafka.consumer.SimpleConsumer
import collection.JavaConversions._
import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.common.security.JaasUtils
object ConsumerGroupCommand {
@@ -48,57 +49,60 @@ object ConsumerGroupCommand {
opts.checkArgs()
- val zkClient = ZkUtils.createZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000)
+ val zkUtils = ZkUtils(opts.options.valueOf(opts.zkConnectOpt),
+ 30000,
+ 30000,
+ JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)))
try {
if (opts.options.has(opts.listOpt))
- list(zkClient)
+ list(zkUtils)
else if (opts.options.has(opts.describeOpt))
- describe(zkClient, opts)
+ describe(zkUtils, opts)
else if (opts.options.has(opts.deleteOpt))
- delete(zkClient, opts)
+ delete(zkUtils, opts)
} catch {
case e: Throwable =>
println("Error while executing consumer group command " + e.getMessage)
println(Utils.stackTrace(e))
} finally {
- zkClient.close()
+ zkUtils.close()
}
}
- def list(zkClient: ZkClient) {
- ZkUtils.getConsumerGroups(zkClient).foreach(println)
+ def list(zkUtils: ZkUtils) {
+ zkUtils.getConsumerGroups().foreach(println)
}
- def describe(zkClient: ZkClient, opts: ConsumerGroupCommandOptions) {
+ def describe(zkUtils: ZkUtils, opts: ConsumerGroupCommandOptions) {
val configs = parseConfigs(opts)
val channelSocketTimeoutMs = configs.getProperty("channelSocketTimeoutMs", "600").toInt
val channelRetryBackoffMs = configs.getProperty("channelRetryBackoffMsOpt", "300").toInt
val group = opts.options.valueOf(opts.groupOpt)
- val topics = ZkUtils.getTopicsByConsumerGroup(zkClient, group)
+ val topics = zkUtils.getTopicsByConsumerGroup(group)
if (topics.isEmpty) {
println("No topic available for consumer group provided")
}
- topics.foreach(topic => describeTopic(zkClient, group, topic, channelSocketTimeoutMs, channelRetryBackoffMs))
+ topics.foreach(topic => describeTopic(zkUtils, group, topic, channelSocketTimeoutMs, channelRetryBackoffMs))
}
- def delete(zkClient: ZkClient, opts: ConsumerGroupCommandOptions) {
+ def delete(zkUtils: ZkUtils, opts: ConsumerGroupCommandOptions) {
if (opts.options.has(opts.groupOpt) && opts.options.has(opts.topicOpt)) {
- deleteForTopic(zkClient, opts)
+ deleteForTopic(zkUtils, opts)
}
else if (opts.options.has(opts.groupOpt)) {
- deleteForGroup(zkClient, opts)
+ deleteForGroup(zkUtils, opts)
}
else if (opts.options.has(opts.topicOpt)) {
- deleteAllForTopic(zkClient, opts)
+ deleteAllForTopic(zkUtils, opts)
}
}
- private def deleteForGroup(zkClient: ZkClient, opts: ConsumerGroupCommandOptions) {
+ private def deleteForGroup(zkUtils: ZkUtils, opts: ConsumerGroupCommandOptions) {
val groups = opts.options.valuesOf(opts.groupOpt)
groups.foreach { group =>
try {
- if (AdminUtils.deleteConsumerGroupInZK(zkClient, group))
+ if (AdminUtils.deleteConsumerGroupInZK(zkUtils, group))
println("Deleted all consumer group information for group %s in zookeeper.".format(group))
else
println("Delete for group %s failed because its consumers are still active.".format(group))
@@ -110,13 +114,13 @@ object ConsumerGroupCommand {
}
}
- private def deleteForTopic(zkClient: ZkClient, opts: ConsumerGroupCommandOptions) {
+ private def deleteForTopic(zkUtils: ZkUtils, opts: ConsumerGroupCommandOptions) {
val groups = opts.options.valuesOf(opts.groupOpt)
val topic = opts.options.valueOf(opts.topicOpt)
Topic.validate(topic)
groups.foreach { group =>
try {
- if (AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkClient, group, topic))
+ if (AdminUtils.deleteConsumerGroupInfoForTopicInZK(zkUtils, group, topic))
println("Deleted consumer group information for group %s topic %s in zookeeper.".format(group, topic))
else
println("Delete for group %s topic %s failed because its consumers are still active.".format(group, topic))
@@ -128,10 +132,10 @@ object ConsumerGroupCommand {
}
}
- private def deleteAllForTopic(zkClient: ZkClient, opts: ConsumerGroupCommandOptions) {
+ private def deleteAllForTopic(zkUtils: ZkUtils, opts: ConsumerGroupCommandOptions) {
val topic = opts.options.valueOf(opts.topicOpt)
Topic.validate(topic)
- AdminUtils.deleteAllConsumerGroupInfoForTopicInZK(zkClient, topic)
+ AdminUtils.deleteAllConsumerGroupInfoForTopicInZK(zkUtils, topic)
println("Deleted consumer group information for all inactive consumer groups for topic %s in zookeeper.".format(topic))
}
@@ -144,35 +148,35 @@ object ConsumerGroupCommand {
props
}
- private def describeTopic(zkClient: ZkClient,
+ private def describeTopic(zkUtils: ZkUtils,
group: String,
topic: String,
channelSocketTimeoutMs: Int,
channelRetryBackoffMs: Int) {
- val topicPartitions = getTopicPartitions(zkClient, topic)
- val partitionOffsets = getPartitionOffsets(zkClient, group, topicPartitions, channelSocketTimeoutMs, channelRetryBackoffMs)
+ val topicPartitions = getTopicPartitions(zkUtils, topic)
+ val partitionOffsets = getPartitionOffsets(zkUtils, group, topicPartitions, channelSocketTimeoutMs, channelRetryBackoffMs)
println("%s, %s, %s, %s, %s, %s, %s"
.format("GROUP", "TOPIC", "PARTITION", "CURRENT OFFSET", "LOG END OFFSET", "LAG", "OWNER"))
topicPartitions
.sortBy { case topicPartition => topicPartition.partition }
.foreach { topicPartition =>
- describePartition(zkClient, group, topicPartition.topic, topicPartition.partition, partitionOffsets.get(topicPartition))
+ describePartition(zkUtils, group, topicPartition.topic, topicPartition.partition, partitionOffsets.get(topicPartition))
}
}
- private def getTopicPartitions(zkClient: ZkClient, topic: String) = {
- val topicPartitionMap = ZkUtils.getPartitionsForTopics(zkClient, Seq(topic))
+ private def getTopicPartitions(zkUtils: ZkUtils, topic: String) = {
+ val topicPartitionMap = zkUtils.getPartitionsForTopics(Seq(topic))
val partitions = topicPartitionMap.getOrElse(topic, Seq.empty)
partitions.map(TopicAndPartition(topic, _))
}
- private def getPartitionOffsets(zkClient: ZkClient,
+ private def getPartitionOffsets(zkUtils: ZkUtils,
group: String,
topicPartitions: Seq[TopicAndPartition],
channelSocketTimeoutMs: Int,
channelRetryBackoffMs: Int): Map[TopicAndPartition, Long] = {
val offsetMap = mutable.Map[TopicAndPartition, Long]()
- val channel = ClientUtils.channelToOffsetManager(group, zkClient, channelSocketTimeoutMs, channelRetryBackoffMs)
+ val channel = ClientUtils.channelToOffsetManager(group, zkUtils, channelSocketTimeoutMs, channelRetryBackoffMs)
channel.send(OffsetFetchRequest(group, topicPartitions))
val offsetFetchResponse = OffsetFetchResponse.readFrom(channel.receive().payload())
@@ -182,7 +186,7 @@ object ConsumerGroupCommand {
// this group may not have migrated off zookeeper for offsets storage (we don't expose the dual-commit option in this tool
// (meaning the lag may be off until all the consumers in the group have the same setting for offsets storage)
try {
- val offset = ZkUtils.readData(zkClient, topicDirs.consumerOffsetDir + "/" + topicAndPartition.partition)._1.toLong
+ val offset = zkUtils.readData(topicDirs.consumerOffsetDir + "/" + topicAndPartition.partition)._1.toLong
offsetMap.put(topicAndPartition, offset)
} catch {
case z: ZkNoNodeException =>
@@ -200,20 +204,20 @@ object ConsumerGroupCommand {
offsetMap.toMap
}
- private def describePartition(zkClient: ZkClient,
+ private def describePartition(zkUtils: ZkUtils,
group: String,
topic: String,
partition: Int,
offsetOpt: Option[Long]) {
val topicAndPartition = TopicAndPartition(topic, partition)
val groupDirs = new ZKGroupTopicDirs(group, topic)
- val owner = ZkUtils.readDataMaybeNull(zkClient, groupDirs.consumerOwnerDir + "/" + partition)._1
- ZkUtils.getLeaderForPartition(zkClient, topic, partition) match {
+ val owner = zkUtils.readDataMaybeNull(groupDirs.consumerOwnerDir + "/" + partition)._1
+ zkUtils.getLeaderForPartition(topic, partition) match {
case Some(-1) =>
println("%s, %s, %s, %s, %s, %s, %s"
.format(group, topic, partition, offsetOpt.getOrElse("unknown"), "unknown", "unknown", owner.getOrElse("none")))
case Some(brokerId) =>
- val consumerOpt = getConsumer(zkClient, brokerId)
+ val consumerOpt = getConsumer(zkUtils, brokerId)
consumerOpt match {
case Some(consumer) =>
val request =
@@ -231,9 +235,9 @@ object ConsumerGroupCommand {
}
}
- private def getConsumer(zkClient: ZkClient, brokerId: Int): Option[SimpleConsumer] = {
+ private def getConsumer(zkUtils: ZkUtils, brokerId: Int): Option[SimpleConsumer] = {
try {
- ZkUtils.readDataMaybeNull(zkClient, ZkUtils.BrokerIdsPath + "/" + brokerId)._1 match {
+ zkUtils.readDataMaybeNull(ZkUtils.BrokerIdsPath + "/" + brokerId)._1 match {
case Some(brokerInfoString) =>
Json.parseFull(brokerInfoString) match {
case Some(m) =>
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala b/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala
index 2aa6e62..e74fcb6 100755
--- a/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala
+++ b/core/src/main/scala/kafka/admin/PreferredReplicaLeaderElectionCommand.scala
@@ -23,6 +23,7 @@ import org.I0Itec.zkclient.exception.ZkNodeExistsException
import kafka.common.{TopicAndPartition, AdminCommandFailedException}
import collection._
import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.common.security.JaasUtils
object PreferredReplicaLeaderElectionCommand extends Logging {
@@ -51,15 +52,19 @@ object PreferredReplicaLeaderElectionCommand extends Logging {
val zkConnect = options.valueOf(zkConnectOpt)
var zkClient: ZkClient = null
-
+ var zkUtils: ZkUtils = null
try {
zkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000)
+ zkUtils = ZkUtils(zkConnect,
+ 30000,
+ 30000,
+ JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)))
val partitionsForPreferredReplicaElection =
if (!options.has(jsonFileOpt))
- ZkUtils.getAllPartitions(zkClient)
+ zkUtils.getAllPartitions()
else
parsePreferredReplicaElectionData(Utils.readFileAsString(options.valueOf(jsonFileOpt)))
- val preferredReplicaElectionCommand = new PreferredReplicaLeaderElectionCommand(zkClient, partitionsForPreferredReplicaElection)
+ val preferredReplicaElectionCommand = new PreferredReplicaLeaderElectionCommand(zkUtils, partitionsForPreferredReplicaElection)
preferredReplicaElectionCommand.moveLeaderToPreferredReplica()
println("Successfully started preferred replica election for partitions %s".format(partitionsForPreferredReplicaElection))
@@ -95,18 +100,18 @@ object PreferredReplicaLeaderElectionCommand extends Logging {
}
}
- def writePreferredReplicaElectionData(zkClient: ZkClient,
+ def writePreferredReplicaElectionData(zkUtils: ZkUtils,
partitionsUndergoingPreferredReplicaElection: scala.collection.Set[TopicAndPartition]) {
val zkPath = ZkUtils.PreferredReplicaLeaderElectionPath
val partitionsList = partitionsUndergoingPreferredReplicaElection.map(e => Map("topic" -> e.topic, "partition" -> e.partition))
val jsonData = Json.encode(Map("version" -> 1, "partitions" -> partitionsList))
try {
- ZkUtils.createPersistentPath(zkClient, zkPath, jsonData)
+ zkUtils.createPersistentPath(zkPath, jsonData)
info("Created preferred replica election path with %s".format(jsonData))
} catch {
case nee: ZkNodeExistsException =>
val partitionsUndergoingPreferredReplicaElection =
- PreferredReplicaLeaderElectionCommand.parsePreferredReplicaElectionData(ZkUtils.readData(zkClient, zkPath)._1)
+ PreferredReplicaLeaderElectionCommand.parsePreferredReplicaElectionData(zkUtils.readData(zkPath)._1)
throw new AdminOperationException("Preferred replica leader election currently in progress for " +
"%s. Aborting operation".format(partitionsUndergoingPreferredReplicaElection))
case e2: Throwable => throw new AdminOperationException(e2.toString)
@@ -114,20 +119,20 @@ object PreferredReplicaLeaderElectionCommand extends Logging {
}
}
-class PreferredReplicaLeaderElectionCommand(zkClient: ZkClient, partitions: scala.collection.Set[TopicAndPartition])
+class PreferredReplicaLeaderElectionCommand(zkUtils: ZkUtils, partitions: scala.collection.Set[TopicAndPartition])
extends Logging {
def moveLeaderToPreferredReplica() = {
try {
- val validPartitions = partitions.filter(p => validatePartition(zkClient, p.topic, p.partition))
- PreferredReplicaLeaderElectionCommand.writePreferredReplicaElectionData(zkClient, validPartitions)
+ val validPartitions = partitions.filter(p => validatePartition(zkUtils, p.topic, p.partition))
+ PreferredReplicaLeaderElectionCommand.writePreferredReplicaElectionData(zkUtils, validPartitions)
} catch {
case e: Throwable => throw new AdminCommandFailedException("Admin command failed", e)
}
}
- def validatePartition(zkClient: ZkClient, topic: String, partition: Int): Boolean = {
+ def validatePartition(zkUtils: ZkUtils, topic: String, partition: Int): Boolean = {
// check if partition exists
- val partitionsOpt = ZkUtils.getPartitionsForTopics(zkClient, List(topic)).get(topic)
+ val partitionsOpt = zkUtils.getPartitionsForTopics(List(topic)).get(topic)
partitionsOpt match {
case Some(partitions) =>
if(partitions.contains(partition)) {
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala
index ea34589..10182f6 100755
--- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala
+++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala
@@ -23,6 +23,7 @@ import org.I0Itec.zkclient.ZkClient
import org.I0Itec.zkclient.exception.ZkNodeExistsException
import kafka.common.{TopicAndPartition, AdminCommandFailedException}
import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.common.security.JaasUtils
object ReassignPartitionsCommand extends Logging {
@@ -38,33 +39,37 @@ object ReassignPartitionsCommand extends Logging {
CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.zkConnectOpt)
val zkConnect = opts.options.valueOf(opts.zkConnectOpt)
- var zkClient: ZkClient = ZkUtils.createZkClient(zkConnect, 30000, 30000)
+ val zkUtils = ZkUtils(zkConnect,
+ 30000,
+ 30000,
+ JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)))
try {
if(opts.options.has(opts.verifyOpt))
- verifyAssignment(zkClient, opts)
+ verifyAssignment(zkUtils, opts)
else if(opts.options.has(opts.generateOpt))
- generateAssignment(zkClient, opts)
+ generateAssignment(zkUtils, opts)
else if (opts.options.has(opts.executeOpt))
- executeAssignment(zkClient, opts)
+ executeAssignment(zkUtils, opts)
} catch {
case e: Throwable =>
println("Partitions reassignment failed due to " + e.getMessage)
println(Utils.stackTrace(e))
} finally {
+ val zkClient = zkUtils.zkClient
if (zkClient != null)
zkClient.close()
}
}
- def verifyAssignment(zkClient: ZkClient, opts: ReassignPartitionsCommandOptions) {
+ def verifyAssignment(zkUtils: ZkUtils, opts: ReassignPartitionsCommandOptions) {
if(!opts.options.has(opts.reassignmentJsonFileOpt))
CommandLineUtils.printUsageAndDie(opts.parser, "If --verify option is used, command must include --reassignment-json-file that was used during the --execute option")
val jsonFile = opts.options.valueOf(opts.reassignmentJsonFileOpt)
val jsonString = Utils.readFileAsString(jsonFile)
- val partitionsToBeReassigned = ZkUtils.parsePartitionReassignmentData(jsonString)
+ val partitionsToBeReassigned = zkUtils.parsePartitionReassignmentData(jsonString)
println("Status of partition reassignment:")
- val reassignedPartitionsStatus = checkIfReassignmentSucceeded(zkClient, partitionsToBeReassigned)
+ val reassignedPartitionsStatus = checkIfReassignmentSucceeded(zkUtils, partitionsToBeReassigned)
reassignedPartitionsStatus.foreach { partition =>
partition._2 match {
case ReassignmentCompleted =>
@@ -77,7 +82,7 @@ object ReassignPartitionsCommand extends Logging {
}
}
- def generateAssignment(zkClient: ZkClient, opts: ReassignPartitionsCommandOptions) {
+ def generateAssignment(zkUtils: ZkUtils, opts: ReassignPartitionsCommandOptions) {
if(!(opts.options.has(opts.topicsToMoveJsonFileOpt) && opts.options.has(opts.brokerListOpt)))
CommandLineUtils.printUsageAndDie(opts.parser, "If --generate option is used, command must include both --topics-to-move-json-file and --broker-list options")
val topicsToMoveJsonFile = opts.options.valueOf(opts.topicsToMoveJsonFileOpt)
@@ -86,11 +91,11 @@ object ReassignPartitionsCommand extends Logging {
if (duplicateReassignments.nonEmpty)
throw new AdminCommandFailedException("Broker list contains duplicate entries: %s".format(duplicateReassignments.mkString(",")))
val topicsToMoveJsonString = Utils.readFileAsString(topicsToMoveJsonFile)
- val topicsToReassign = ZkUtils.parseTopicsData(topicsToMoveJsonString)
+ val topicsToReassign = zkUtils.parseTopicsData(topicsToMoveJsonString)
val duplicateTopicsToReassign = CoreUtils.duplicates(topicsToReassign)
if (duplicateTopicsToReassign.nonEmpty)
throw new AdminCommandFailedException("List of topics to reassign contains duplicate entries: %s".format(duplicateTopicsToReassign.mkString(",")))
- val topicPartitionsToReassign = ZkUtils.getReplicaAssignmentForTopics(zkClient, topicsToReassign)
+ val topicPartitionsToReassign = zkUtils.getReplicaAssignmentForTopics(topicsToReassign)
var partitionsToBeReassigned : Map[TopicAndPartition, Seq[Int]] = new mutable.HashMap[TopicAndPartition, List[Int]]()
val groupedByTopic = topicPartitionsToReassign.groupBy(tp => tp._1.topic)
@@ -99,18 +104,18 @@ object ReassignPartitionsCommand extends Logging {
topicInfo._2.head._2.size)
partitionsToBeReassigned ++= assignedReplicas.map(replicaInfo => (TopicAndPartition(topicInfo._1, replicaInfo._1) -> replicaInfo._2))
}
- val currentPartitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, partitionsToBeReassigned.map(_._1.topic).toSeq)
+ val currentPartitionReplicaAssignment = zkUtils.getReplicaAssignmentForTopics(partitionsToBeReassigned.map(_._1.topic).toSeq)
println("Current partition replica assignment\n\n%s"
- .format(ZkUtils.getPartitionReassignmentZkData(currentPartitionReplicaAssignment)))
- println("Proposed partition reassignment configuration\n\n%s".format(ZkUtils.getPartitionReassignmentZkData(partitionsToBeReassigned)))
+ .format(zkUtils.getPartitionReassignmentZkData(currentPartitionReplicaAssignment)))
+ println("Proposed partition reassignment configuration\n\n%s".format(zkUtils.getPartitionReassignmentZkData(partitionsToBeReassigned)))
}
- def executeAssignment(zkClient: ZkClient, opts: ReassignPartitionsCommandOptions) {
+ def executeAssignment(zkUtils: ZkUtils, opts: ReassignPartitionsCommandOptions) {
if(!opts.options.has(opts.reassignmentJsonFileOpt))
CommandLineUtils.printUsageAndDie(opts.parser, "If --execute option is used, command must include --reassignment-json-file that was output " + "during the --generate option")
val reassignmentJsonFile = opts.options.valueOf(opts.reassignmentJsonFileOpt)
val reassignmentJsonString = Utils.readFileAsString(reassignmentJsonFile)
- val partitionsToBeReassigned = ZkUtils.parsePartitionReassignmentDataWithoutDedup(reassignmentJsonString)
+ val partitionsToBeReassigned = zkUtils.parsePartitionReassignmentDataWithoutDedup(reassignmentJsonString)
if (partitionsToBeReassigned.isEmpty)
throw new AdminCommandFailedException("Partition reassignment data file %s is empty".format(reassignmentJsonFile))
val duplicateReassignedPartitions = CoreUtils.duplicates(partitionsToBeReassigned.map{ case(tp,replicas) => tp})
@@ -125,28 +130,28 @@ object ReassignPartitionsCommand extends Logging {
.mkString(". ")
throw new AdminCommandFailedException("Partition replica lists may not contain duplicate entries: %s".format(duplicatesMsg))
}
- val reassignPartitionsCommand = new ReassignPartitionsCommand(zkClient, partitionsToBeReassigned.toMap)
+ val reassignPartitionsCommand = new ReassignPartitionsCommand(zkUtils, partitionsToBeReassigned.toMap)
// before starting assignment, output the current replica assignment to facilitate rollback
- val currentPartitionReplicaAssignment = ZkUtils.getReplicaAssignmentForTopics(zkClient, partitionsToBeReassigned.map(_._1.topic))
+ val currentPartitionReplicaAssignment = zkUtils.getReplicaAssignmentForTopics(partitionsToBeReassigned.map(_._1.topic))
println("Current partition replica assignment\n\n%s\n\nSave this to use as the --reassignment-json-file option during rollback"
- .format(ZkUtils.getPartitionReassignmentZkData(currentPartitionReplicaAssignment)))
+ .format(zkUtils.getPartitionReassignmentZkData(currentPartitionReplicaAssignment)))
// start the reassignment
if(reassignPartitionsCommand.reassignPartitions())
- println("Successfully started reassignment of partitions %s".format(ZkUtils.getPartitionReassignmentZkData(partitionsToBeReassigned.toMap)))
+ println("Successfully started reassignment of partitions %s".format(zkUtils.getPartitionReassignmentZkData(partitionsToBeReassigned.toMap)))
else
println("Failed to reassign partitions %s".format(partitionsToBeReassigned))
}
- private def checkIfReassignmentSucceeded(zkClient: ZkClient, partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]])
+ private def checkIfReassignmentSucceeded(zkUtils: ZkUtils, partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]])
:Map[TopicAndPartition, ReassignmentStatus] = {
- val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient).mapValues(_.newReplicas)
+ val partitionsBeingReassigned = zkUtils.getPartitionsBeingReassigned().mapValues(_.newReplicas)
partitionsToBeReassigned.map { topicAndPartition =>
- (topicAndPartition._1, checkIfPartitionReassignmentSucceeded(zkClient, topicAndPartition._1,
+ (topicAndPartition._1, checkIfPartitionReassignmentSucceeded(zkUtils,topicAndPartition._1,
topicAndPartition._2, partitionsToBeReassigned, partitionsBeingReassigned))
}
}
- def checkIfPartitionReassignmentSucceeded(zkClient: ZkClient, topicAndPartition: TopicAndPartition,
+ def checkIfPartitionReassignmentSucceeded(zkUtils: ZkUtils, topicAndPartition: TopicAndPartition,
reassignedReplicas: Seq[Int],
partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]],
partitionsBeingReassigned: Map[TopicAndPartition, Seq[Int]]): ReassignmentStatus = {
@@ -155,7 +160,7 @@ object ReassignPartitionsCommand extends Logging {
case Some(partition) => ReassignmentInProgress
case None =>
// check if the current replica assignment matches the expected one after reassignment
- val assignedReplicas = ZkUtils.getReplicasForPartition(zkClient, topicAndPartition.topic, topicAndPartition.partition)
+ val assignedReplicas = zkUtils.getReplicasForPartition(topicAndPartition.topic, topicAndPartition.partition)
if(assignedReplicas == newReplicas)
ReassignmentCompleted
else {
@@ -203,31 +208,31 @@ object ReassignPartitionsCommand extends Logging {
}
}
-class ReassignPartitionsCommand(zkClient: ZkClient, partitions: collection.Map[TopicAndPartition, collection.Seq[Int]])
+class ReassignPartitionsCommand(zkUtils: ZkUtils, partitions: collection.Map[TopicAndPartition, collection.Seq[Int]])
extends Logging {
def reassignPartitions(): Boolean = {
try {
- val validPartitions = partitions.filter(p => validatePartition(zkClient, p._1.topic, p._1.partition))
+ val validPartitions = partitions.filter(p => validatePartition(zkUtils, p._1.topic, p._1.partition))
if(validPartitions.isEmpty) {
false
}
else {
- val jsonReassignmentData = ZkUtils.getPartitionReassignmentZkData(validPartitions)
- ZkUtils.createPersistentPath(zkClient, ZkUtils.ReassignPartitionsPath, jsonReassignmentData)
+ val jsonReassignmentData = zkUtils.getPartitionReassignmentZkData(validPartitions)
+ zkUtils.createPersistentPath(ZkUtils.ReassignPartitionsPath, jsonReassignmentData)
true
}
} catch {
case ze: ZkNodeExistsException =>
- val partitionsBeingReassigned = ZkUtils.getPartitionsBeingReassigned(zkClient)
+ val partitionsBeingReassigned = zkUtils.getPartitionsBeingReassigned()
throw new AdminCommandFailedException("Partition reassignment currently in " +
"progress for %s. Aborting operation".format(partitionsBeingReassigned))
case e: Throwable => error("Admin command failed", e); false
}
}
- def validatePartition(zkClient: ZkClient, topic: String, partition: Int): Boolean = {
+ def validatePartition(zkUtils: ZkUtils, topic: String, partition: Int): Boolean = {
// check if partition exists
- val partitionsOpt = ZkUtils.getPartitionsForTopics(zkClient, List(topic)).get(topic)
+ val partitionsOpt = zkUtils.getPartitionsForTopics(List(topic)).get(topic)
partitionsOpt match {
case Some(partitions) =>
if(partitions.contains(partition)) {
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/admin/TopicCommand.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala
index 3abac62..9fe2606 100755
--- a/core/src/main/scala/kafka/admin/TopicCommand.scala
+++ b/core/src/main/scala/kafka/admin/TopicCommand.scala
@@ -22,6 +22,7 @@ import java.util.Properties
import kafka.common.{Topic, AdminCommandFailedException}
import kafka.utils.CommandLineUtils
import kafka.utils._
+import kafka.utils.ZkUtils._
import org.I0Itec.zkclient.ZkClient
import org.I0Itec.zkclient.exception.ZkNodeExistsException
import scala.collection._
@@ -30,6 +31,7 @@ import kafka.log.LogConfig
import kafka.consumer.Whitelist
import kafka.server.{ConfigType, OffsetManager}
import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.common.security.JaasUtils
import kafka.coordinator.ConsumerCoordinator
@@ -49,33 +51,36 @@ object TopicCommand extends Logging {
opts.checkArgs()
- val zkClient = ZkUtils.createZkClient(opts.options.valueOf(opts.zkConnectOpt), 30000, 30000)
+ val zkUtils = ZkUtils(opts.options.valueOf(opts.zkConnectOpt),
+ 30000,
+ 30000,
+ JaasUtils.isZkSecurityEnabled(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)))
var exitCode = 0
try {
if(opts.options.has(opts.createOpt))
- createTopic(zkClient, opts)
+ createTopic(zkUtils, opts)
else if(opts.options.has(opts.alterOpt))
- alterTopic(zkClient, opts)
+ alterTopic(zkUtils, opts)
else if(opts.options.has(opts.listOpt))
- listTopics(zkClient, opts)
+ listTopics(zkUtils, opts)
else if(opts.options.has(opts.describeOpt))
- describeTopic(zkClient, opts)
+ describeTopic(zkUtils, opts)
else if(opts.options.has(opts.deleteOpt))
- deleteTopic(zkClient, opts)
+ deleteTopic(zkUtils, opts)
} catch {
case e: Throwable =>
println("Error while executing topic command : " + e.getMessage)
error(Utils.stackTrace(e))
exitCode = 1
} finally {
- zkClient.close()
+ zkUtils.close()
System.exit(exitCode)
}
}
- private def getTopics(zkClient: ZkClient, opts: TopicCommandOptions): Seq[String] = {
- val allTopics = ZkUtils.getAllTopics(zkClient).sorted
+ private def getTopics(zkUtils: ZkUtils, opts: TopicCommandOptions): Seq[String] = {
+ val allTopics = zkUtils.getAllTopics().sorted
if (opts.options.has(opts.topicOpt)) {
val topicsSpec = opts.options.valueOf(opts.topicOpt)
val topicsFilter = new Whitelist(topicsSpec)
@@ -84,31 +89,31 @@ object TopicCommand extends Logging {
allTopics
}
- def createTopic(zkClient: ZkClient, opts: TopicCommandOptions) {
+ def createTopic(zkUtils: ZkUtils, opts: TopicCommandOptions) {
val topic = opts.options.valueOf(opts.topicOpt)
val configs = parseTopicConfigsToBeAdded(opts)
if (Topic.hasCollisionChars(topic))
println("WARNING: Due to limitations in metric names, topics with a period ('.') or underscore ('_') could collide. To avoid issues it is best to use either, but not both.")
if (opts.options.has(opts.replicaAssignmentOpt)) {
val assignment = parseReplicaAssignment(opts.options.valueOf(opts.replicaAssignmentOpt))
- AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, assignment, configs, update = false)
+ AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, assignment, configs, update = false)
} else {
CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.partitionsOpt, opts.replicationFactorOpt)
val partitions = opts.options.valueOf(opts.partitionsOpt).intValue
val replicas = opts.options.valueOf(opts.replicationFactorOpt).intValue
- AdminUtils.createTopic(zkClient, topic, partitions, replicas, configs)
+ AdminUtils.createTopic(zkUtils, topic, partitions, replicas, configs)
}
println("Created topic \"%s\".".format(topic))
}
- def alterTopic(zkClient: ZkClient, opts: TopicCommandOptions) {
- val topics = getTopics(zkClient, opts)
+ def alterTopic(zkUtils: ZkUtils, opts: TopicCommandOptions) {
+ val topics = getTopics(zkUtils, opts)
if (topics.length == 0) {
throw new IllegalArgumentException("Topic %s does not exist on ZK path %s".format(opts.options.valueOf(opts.topicOpt),
opts.options.valueOf(opts.zkConnectOpt)))
}
topics.foreach { topic =>
- val configs = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topic, topic)
+ val configs = AdminUtils.fetchEntityConfig(zkUtils, ConfigType.Topic, topic)
if(opts.options.has(opts.configOpt) || opts.options.has(opts.deleteConfigOpt)) {
println("WARNING: Altering topic configuration from this script has been deprecated and may be removed in future releases.")
println(" Going forward, please use kafka-configs.sh for this functionality")
@@ -118,7 +123,7 @@ object TopicCommand extends Logging {
// compile the final set of configs
configs.putAll(configsToBeAdded)
configsToBeDeleted.foreach(config => configs.remove(config))
- AdminUtils.changeTopicConfig(zkClient, topic, configs)
+ AdminUtils.changeTopicConfig(zkUtils, topic, configs)
println("Updated config for topic \"%s\".".format(topic))
}
@@ -130,16 +135,16 @@ object TopicCommand extends Logging {
"logic or ordering of the messages will be affected")
val nPartitions = opts.options.valueOf(opts.partitionsOpt).intValue
val replicaAssignmentStr = opts.options.valueOf(opts.replicaAssignmentOpt)
- AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr)
+ AdminUtils.addPartitions(zkUtils, topic, nPartitions, replicaAssignmentStr)
println("Adding partitions succeeded!")
}
}
}
- def listTopics(zkClient: ZkClient, opts: TopicCommandOptions) {
- val topics = getTopics(zkClient, opts)
+ def listTopics(zkUtils: ZkUtils, opts: TopicCommandOptions) {
+ val topics = getTopics(zkUtils, opts)
for(topic <- topics) {
- if (ZkUtils.pathExists(zkClient,ZkUtils.getDeleteTopicPath(topic))) {
+ if (zkUtils.pathExists(getDeleteTopicPath(topic))) {
println("%s - marked for deletion".format(topic))
} else {
println(topic)
@@ -147,8 +152,8 @@ object TopicCommand extends Logging {
}
}
- def deleteTopic(zkClient: ZkClient, opts: TopicCommandOptions) {
- val topics = getTopics(zkClient, opts)
+ def deleteTopic(zkUtils: ZkUtils, opts: TopicCommandOptions) {
+ val topics = getTopics(zkUtils, opts)
if (topics.length == 0) {
throw new IllegalArgumentException("Topic %s does not exist on ZK path %s".format(opts.options.valueOf(opts.topicOpt),
opts.options.valueOf(opts.zkConnectOpt)))
@@ -158,7 +163,7 @@ object TopicCommand extends Logging {
if (Topic.InternalTopics.contains(topic)) {
throw new AdminOperationException("Topic %s is a kafka internal topic and is not allowed to be marked for deletion.".format(topic))
} else {
- ZkUtils.createPersistentPath(zkClient, ZkUtils.getDeleteTopicPath(topic))
+ zkUtils.createPersistentPath(getDeleteTopicPath(topic))
println("Topic %s is marked for deletion.".format(topic))
println("Note: This will have no impact if delete.topic.enable is not set to true.")
}
@@ -173,20 +178,20 @@ object TopicCommand extends Logging {
}
}
- def describeTopic(zkClient: ZkClient, opts: TopicCommandOptions) {
- val topics = getTopics(zkClient, opts)
+ def describeTopic(zkUtils: ZkUtils, opts: TopicCommandOptions) {
+ val topics = getTopics(zkUtils, opts)
val reportUnderReplicatedPartitions = if (opts.options.has(opts.reportUnderReplicatedPartitionsOpt)) true else false
val reportUnavailablePartitions = if (opts.options.has(opts.reportUnavailablePartitionsOpt)) true else false
val reportOverriddenConfigs = if (opts.options.has(opts.topicsWithOverridesOpt)) true else false
- val liveBrokers = ZkUtils.getAllBrokersInCluster(zkClient).map(_.id).toSet
+ val liveBrokers = zkUtils.getAllBrokersInCluster().map(_.id).toSet
for (topic <- topics) {
- ZkUtils.getPartitionAssignmentForTopics(zkClient, List(topic)).get(topic) match {
+ zkUtils.getPartitionAssignmentForTopics(List(topic)).get(topic) match {
case Some(topicPartitionAssignment) =>
val describeConfigs: Boolean = !reportUnavailablePartitions && !reportUnderReplicatedPartitions
val describePartitions: Boolean = !reportOverriddenConfigs
val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1)
if (describeConfigs) {
- val configs = AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topic, topic)
+ val configs = AdminUtils.fetchEntityConfig(zkUtils, ConfigType.Topic, topic)
if (!reportOverriddenConfigs || configs.size() != 0) {
val numPartitions = topicPartitionAssignment.size
val replicationFactor = topicPartitionAssignment.head._2.size
@@ -196,8 +201,8 @@ object TopicCommand extends Logging {
}
if (describePartitions) {
for ((partitionId, assignedReplicas) <- sortedPartitions) {
- val inSyncReplicas = ZkUtils.getInSyncReplicasForPartition(zkClient, topic, partitionId)
- val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partitionId)
+ val inSyncReplicas = zkUtils.getInSyncReplicasForPartition(topic, partitionId)
+ val leader = zkUtils.getLeaderForPartition(topic, partitionId)
if ((!reportUnderReplicatedPartitions && !reportUnavailablePartitions) ||
(reportUnderReplicatedPartitions && inSyncReplicas.size < assignedReplicas.size) ||
(reportUnavailablePartitions && (!leader.isDefined || !liveBrokers.contains(leader.get)))) {
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/client/ClientUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala
index 68c7e7f..6ae0347 100755
--- a/core/src/main/scala/kafka/client/ClientUtils.scala
+++ b/core/src/main/scala/kafka/client/ClientUtils.scala
@@ -27,7 +27,7 @@ import kafka.utils.{CoreUtils, Logging}
import java.util.Properties
import util.Random
import kafka.network.BlockingChannel
-import kafka.utils.ZkUtils._
+import kafka.utils.ZkUtils
import org.I0Itec.zkclient.ZkClient
import java.io.IOException
@@ -108,11 +108,11 @@ object ClientUtils extends Logging{
/**
* Creates a blocking channel to a random broker
*/
- def channelToAnyBroker(zkClient: ZkClient, socketTimeoutMs: Int = 3000) : BlockingChannel = {
+ def channelToAnyBroker(zkUtils: ZkUtils, socketTimeoutMs: Int = 3000) : BlockingChannel = {
var channel: BlockingChannel = null
var connected = false
while (!connected) {
- val allBrokers = getAllBrokerEndPointsForChannel(zkClient, SecurityProtocol.PLAINTEXT)
+ val allBrokers = zkUtils.getAllBrokerEndPointsForChannel(SecurityProtocol.PLAINTEXT)
Random.shuffle(allBrokers).find { broker =>
trace("Connecting to broker %s:%d.".format(broker.host, broker.port))
try {
@@ -137,8 +137,8 @@ object ClientUtils extends Logging{
/**
* Creates a blocking channel to the offset manager of the given group
*/
- def channelToOffsetManager(group: String, zkClient: ZkClient, socketTimeoutMs: Int = 3000, retryBackOffMs: Int = 1000) = {
- var queryChannel = channelToAnyBroker(zkClient)
+ def channelToOffsetManager(group: String, zkUtils: ZkUtils, socketTimeoutMs: Int = 3000, retryBackOffMs: Int = 1000) = {
+ var queryChannel = channelToAnyBroker(zkUtils)
var offsetManagerChannelOpt: Option[BlockingChannel] = None
@@ -149,7 +149,7 @@ object ClientUtils extends Logging{
while (!coordinatorOpt.isDefined) {
try {
if (!queryChannel.isConnected)
- queryChannel = channelToAnyBroker(zkClient)
+ queryChannel = channelToAnyBroker(zkUtils)
debug("Querying %s:%d to locate offset manager for %s.".format(queryChannel.host, queryChannel.port, group))
queryChannel.send(ConsumerMetadataRequest(group))
val response = queryChannel.receive()
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/cluster/Partition.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala
index ee332ed..59d025b 100755
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -42,7 +42,7 @@ class Partition(val topic: String,
replicaManager: ReplicaManager) extends Logging with KafkaMetricsGroup {
private val localBrokerId = replicaManager.config.brokerId
private val logManager = replicaManager.logManager
- private val zkClient = replicaManager.zkClient
+ private val zkUtils = replicaManager.zkUtils
private val assignedReplicaMap = new Pool[Int, Replica]
// The read lock is only required when multiple reads are executed and needs to be in a consistent manner
private val leaderIsrUpdateLock = new ReentrantReadWriteLock()
@@ -87,7 +87,7 @@ class Partition(val topic: String,
case None =>
if (isReplicaLocal(replicaId)) {
val config = LogConfig.fromProps(logManager.defaultConfig.originals,
- AdminUtils.fetchEntityConfig(zkClient, ConfigType.Topic, topic))
+ AdminUtils.fetchEntityConfig(zkUtils, ConfigType.Topic, topic))
val log = logManager.createLog(TopicAndPartition(topic, partitionId), config)
val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParentFile.getAbsolutePath)
val offsetMap = checkpoint.read
@@ -426,7 +426,7 @@ class Partition(val topic: String,
private def updateIsr(newIsr: Set[Replica]) {
val newLeaderAndIsr = new LeaderAndIsr(localBrokerId, leaderEpoch, newIsr.map(r => r.brokerId).toList, zkVersion)
- val (updateSucceeded,newVersion) = ReplicationUtils.updateLeaderAndIsr(zkClient, topic, partitionId,
+ val (updateSucceeded,newVersion) = ReplicationUtils.updateLeaderAndIsr(zkUtils, topic, partitionId,
newLeaderAndIsr, controllerEpoch, zkVersion)
if(updateSucceeded) {
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala
index eb44c31..a600d5d 100644
--- a/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala
+++ b/core/src/main/scala/kafka/common/ZkNodeChangeNotificationListener.scala
@@ -44,7 +44,7 @@ trait NotificationHandler {
* @param changeExpirationMs
* @param time
*/
-class ZkNodeChangeNotificationListener(private val zkClient: ZkClient,
+class ZkNodeChangeNotificationListener(private val zkUtils: ZkUtils,
private val seqNodeRoot: String,
private val seqNodePrefix: String,
private val notificationHandler: NotificationHandler,
@@ -56,8 +56,8 @@ class ZkNodeChangeNotificationListener(private val zkClient: ZkClient,
* create seqNodeRoot and begin watching for any new children nodes.
*/
def init() {
- ZkUtils.makeSurePersistentPathExists(zkClient, seqNodeRoot)
- zkClient.subscribeChildChanges(seqNodeRoot, NodeChangeListener)
+ zkUtils.makeSurePersistentPathExists(seqNodeRoot)
+ zkUtils.zkClient.subscribeChildChanges(seqNodeRoot, NodeChangeListener)
processAllNotifications()
}
@@ -65,7 +65,7 @@ class ZkNodeChangeNotificationListener(private val zkClient: ZkClient,
* Process all changes
*/
def processAllNotifications() {
- val changes = zkClient.getChildren(seqNodeRoot)
+ val changes = zkUtils.zkClient.getChildren(seqNodeRoot)
processNotifications(changes.asScala.sorted)
}
@@ -80,7 +80,7 @@ class ZkNodeChangeNotificationListener(private val zkClient: ZkClient,
val changeId = changeNumber(notification)
if (changeId > lastExecutedChange) {
val changeZnode = seqNodeRoot + "/" + notification
- val (data, stat) = ZkUtils.readDataMaybeNull(zkClient, changeZnode)
+ val (data, stat) = zkUtils.readDataMaybeNull(changeZnode)
data map (notificationHandler.processNotification(_)) getOrElse(logger.warn(s"read null data from $changeZnode when processing notification $notification"))
}
lastExecutedChange = changeId
@@ -97,11 +97,11 @@ class ZkNodeChangeNotificationListener(private val zkClient: ZkClient,
private def purgeObsoleteNotifications(now: Long, notifications: Seq[String]) {
for (notification <- notifications.sorted) {
val notificationNode = seqNodeRoot + "/" + notification
- val (data, stat) = ZkUtils.readDataMaybeNull(zkClient, notificationNode)
+ val (data, stat) = zkUtils.readDataMaybeNull(notificationNode)
if (data.isDefined) {
if (now - stat.getCtime > changeExpirationMs) {
debug(s"Purging change notification $notificationNode")
- ZkUtils.deletePath(zkClient, notificationNode)
+ zkUtils.deletePath(notificationNode)
}
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala
index 49b683f..e73faf2 100755
--- a/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala
+++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherManager.scala
@@ -26,7 +26,7 @@ import collection.mutable.HashMap
import scala.collection.mutable
import java.util.concurrent.locks.ReentrantLock
import kafka.utils.CoreUtils.inLock
-import kafka.utils.ZkUtils._
+import kafka.utils.ZkUtils
import kafka.utils.{ShutdownableThread, SystemTime}
import kafka.common.TopicAndPartition
import kafka.client.ClientUtils
@@ -39,7 +39,7 @@ import java.util.concurrent.atomic.AtomicInteger
*/
class ConsumerFetcherManager(private val consumerIdString: String,
private val config: ConsumerConfig,
- private val zkClient : ZkClient)
+ private val zkUtils : ZkUtils)
extends AbstractFetcherManager("ConsumerFetcherManager-%d".format(SystemTime.milliseconds),
config.clientId, config.numConsumerFetchers) {
private var partitionMap: immutable.Map[TopicAndPartition, PartitionTopicInfo] = null
@@ -62,7 +62,7 @@ class ConsumerFetcherManager(private val consumerIdString: String,
}
trace("Partitions without leader %s".format(noLeaderPartitionSet))
- val brokers = getAllBrokerEndPointsForChannel(zkClient, SecurityProtocol.PLAINTEXT)
+ val brokers = zkUtils.getAllBrokerEndPointsForChannel(SecurityProtocol.PLAINTEXT)
val topicsMetadata = ClientUtils.fetchTopicMetadata(noLeaderPartitionSet.map(m => m.topic).toSet,
brokers,
config.clientId,
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/consumer/PartitionAssignor.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/PartitionAssignor.scala b/core/src/main/scala/kafka/consumer/PartitionAssignor.scala
index 849284a..5a1bdd0 100755
--- a/core/src/main/scala/kafka/consumer/PartitionAssignor.scala
+++ b/core/src/main/scala/kafka/consumer/PartitionAssignor.scala
@@ -41,19 +41,19 @@ object PartitionAssignor {
}
}
-class AssignmentContext(group: String, val consumerId: String, excludeInternalTopics: Boolean, zkClient: ZkClient) {
+class AssignmentContext(group: String, val consumerId: String, excludeInternalTopics: Boolean, zkUtils: ZkUtils) {
val myTopicThreadIds: collection.Map[String, collection.Set[ConsumerThreadId]] = {
- val myTopicCount = TopicCount.constructTopicCount(group, consumerId, zkClient, excludeInternalTopics)
+ val myTopicCount = TopicCount.constructTopicCount(group, consumerId, zkUtils, excludeInternalTopics)
myTopicCount.getConsumerThreadIdsPerTopic
}
val partitionsForTopic: collection.Map[String, Seq[Int]] =
- ZkUtils.getPartitionsForTopics(zkClient, myTopicThreadIds.keySet.toSeq)
+ zkUtils.getPartitionsForTopics(myTopicThreadIds.keySet.toSeq)
val consumersForTopic: collection.Map[String, List[ConsumerThreadId]] =
- ZkUtils.getConsumersPerTopic(zkClient, group, excludeInternalTopics)
+ zkUtils.getConsumersPerTopic(group, excludeInternalTopics)
- val consumers: Seq[String] = ZkUtils.getConsumersInGroup(zkClient, group).sorted
+ val consumers: Seq[String] = zkUtils.getConsumersInGroup(group).sorted
}
/**
http://git-wip-us.apache.org/repos/asf/kafka/blob/ce306ba4/core/src/main/scala/kafka/consumer/TopicCount.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/TopicCount.scala b/core/src/main/scala/kafka/consumer/TopicCount.scala
index 6994c8e..5706d3c 100755
--- a/core/src/main/scala/kafka/consumer/TopicCount.scala
+++ b/core/src/main/scala/kafka/consumer/TopicCount.scala
@@ -56,9 +56,9 @@ private[kafka] object TopicCount extends Logging {
consumerThreadIdsPerTopicMap
}
- def constructTopicCount(group: String, consumerId: String, zkClient: ZkClient, excludeInternalTopics: Boolean) : TopicCount = {
+ def constructTopicCount(group: String, consumerId: String, zkUtils: ZkUtils, excludeInternalTopics: Boolean) : TopicCount = {
val dirs = new ZKGroupDirs(group)
- val topicCountString = ZkUtils.readData(zkClient, dirs.consumerRegistryDir + "/" + consumerId)._1
+ val topicCountString = zkUtils.readData(dirs.consumerRegistryDir + "/" + consumerId)._1
var subscriptionPattern: String = null
var topMap: Map[String, Int] = null
try {
@@ -94,15 +94,15 @@ private[kafka] object TopicCount extends Logging {
new Whitelist(regex)
else
new Blacklist(regex)
- new WildcardTopicCount(zkClient, consumerId, filter, numStreams, excludeInternalTopics)
+ new WildcardTopicCount(zkUtils, consumerId, filter, numStreams, excludeInternalTopics)
}
}
def constructTopicCount(consumerIdString: String, topicCount: Map[String, Int]) =
new StaticTopicCount(consumerIdString, topicCount)
- def constructTopicCount(consumerIdString: String, filter: TopicFilter, numStreams: Int, zkClient: ZkClient, excludeInternalTopics: Boolean) =
- new WildcardTopicCount(zkClient, consumerIdString, filter, numStreams, excludeInternalTopics)
+ def constructTopicCount(consumerIdString: String, filter: TopicFilter, numStreams: Int, zkUtils: ZkUtils, excludeInternalTopics: Boolean) =
+ new WildcardTopicCount(zkUtils, consumerIdString, filter, numStreams, excludeInternalTopics)
}
@@ -125,13 +125,13 @@ private[kafka] class StaticTopicCount(val consumerIdString: String,
def pattern = TopicCount.staticPattern
}
-private[kafka] class WildcardTopicCount(zkClient: ZkClient,
+private[kafka] class WildcardTopicCount(zkUtils: ZkUtils,
consumerIdString: String,
topicFilter: TopicFilter,
numStreams: Int,
excludeInternalTopics: Boolean) extends TopicCount {
def getConsumerThreadIdsPerTopic = {
- val wildcardTopics = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerTopicsPath)
+ val wildcardTopics = zkUtils.getChildrenParentMayNotExist(ZkUtils.BrokerTopicsPath)
.filter(topic => topicFilter.isTopicAllowed(topic, excludeInternalTopics))
TopicCount.makeConsumerThreadIdsPerTopic(consumerIdString, Map(wildcardTopics.map((_, numStreams)): _*))
}