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)): _*))
   }