Index: system_test/broker_failure/bin/run-test.sh
===================================================================
--- system_test/broker_failure/bin/run-test.sh	(revision 1294440)
+++ system_test/broker_failure/bin/run-test.sh	(working copy)
@@ -68,7 +68,7 @@
 
 readonly num_msg_per_batch=500
 readonly batches_per_iteration=5
-readonly num_iterations=12
+readonly num_iterations=5
 
 readonly zk_source_port=2181
 readonly zk_mirror_port=2182
Index: core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/log4j/KafkaLog4jAppenderTest.scala	(working copy)
@@ -26,18 +26,17 @@
 import kafka.producer.async.MissingConfigException
 import kafka.serializer.Encoder
 import kafka.server.{KafkaConfig, KafkaServer}
-import kafka.utils.{TestUtils, TestZKUtils, Utils, Logging}
 import kafka.zk.{EmbeddedZookeeper, ZooKeeperTestHarness}
 import org.apache.log4j.spi.LoggingEvent
 import org.apache.log4j.{PropertyConfigurator, Logger}
 import org.junit.{After, Before, Test}
 import org.scalatest.junit.JUnit3Suite
+import kafka.utils._
 
 class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with Logging {
 
   var logDirZk: File = null
   var logDirBl: File = null
-  //  var topicLogDir: File = null
   var serverBl: KafkaServer = null
   var serverZk: KafkaServer = null
 
@@ -63,7 +62,7 @@
     logDirZk = new File(logDirZkPath)
     serverZk = TestUtils.createServer(new KafkaConfig(propsZk));
 
-    val propsBl: Properties = createBrokerConfig(brokerBl, portBl)
+    val propsBl: Properties = TestUtils.createBrokerConfig(brokerBl, portBl)
     val logDirBlPath = propsBl.getProperty("log.dir")
     logDirBl = new File(logDirBlPath)
     serverBl = TestUtils.createServer(new KafkaConfig(propsBl))
@@ -85,8 +84,6 @@
     Utils.rm(logDirBl)
 
     Thread.sleep(500)
-//    zkServer.shutdown
-//    Thread.sleep(500)
     super.tearDown()
   }
 
@@ -132,7 +129,7 @@
     props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout")
     props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n")
     props.put("log4j.appender.KAFKA.SerializerClass", "kafka.log4j.AppenderStringEncoder")
-    props.put("log4j.appender.KAFKA.BrokerList", "0:localhost:"+portBl.toString)
+    props.put("log4j.appender.KAFKA.ZkConnect", TestZKUtils.zookeeperConnect)
     props.put("log4j.logger.kafka.log4j", "INFO, KAFKA")
 
     // topic missing
@@ -148,7 +145,7 @@
     props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender")
     props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout")
     props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n")
-    props.put("log4j.appender.KAFKA.BrokerList", "0:localhost:"+portBl.toString)
+    props.put("log4j.appender.KAFKA.ZkConnect", TestZKUtils.zookeeperConnect)
     props.put("log4j.appender.KAFKA.Topic", "test-topic")
     props.put("log4j.logger.kafka.log4j", "INFO, KAFKA")
 
@@ -161,27 +158,6 @@
   }
 
   @Test
-  def testBrokerListLog4jAppends() {
-    PropertyConfigurator.configure(getLog4jConfigWithBrokerList)
-
-    for(i <- 1 to 5)
-      info("test")
-
-    Thread.sleep(2500)
-
-    var offset = 0L
-    val response = simpleConsumerBl.fetch(new FetchRequestBuilder().addFetch("test-topic", 0, offset, 1024*1024).build())
-    val fetchedMessage = response.messageSet("test-topic", 0)
-    var count = 0
-    for(message <- fetchedMessage) {
-      count = count + 1
-      offset += message.offset
-    }
-
-    assertEquals(5, count)
-  }
-
-  @Test
   def testZkConnectLog4jAppends() {
     PropertyConfigurator.configure(getLog4jConfigWithZkConnect)
 
@@ -208,18 +184,6 @@
     assertEquals(5, count)
   }
 
-  private def getLog4jConfigWithBrokerList: Properties = {
-    var props = new Properties()
-    props.put("log4j.rootLogger", "INFO")
-    props.put("log4j.appender.KAFKA", "kafka.producer.KafkaLog4jAppender")
-    props.put("log4j.appender.KAFKA.layout","org.apache.log4j.PatternLayout")
-    props.put("log4j.appender.KAFKA.layout.ConversionPattern","%-5p: %c - %m%n")
-    props.put("log4j.appender.KAFKA.BrokerList", "0:localhost:"+portBl.toString)
-    props.put("log4j.appender.KAFKA.Topic", "test-topic")
-    props.put("log4j.logger.kafka.log4j", "INFO,KAFKA")
-    props
-  }
-
   private def getLog4jConfigWithZkConnect: Properties = {
     var props = new Properties()
     props.put("log4j.rootLogger", "INFO")
@@ -232,21 +196,6 @@
     props
   }
 
-  private def createBrokerConfig(nodeId: Int, port: Int): Properties = {
-    val props = new Properties
-    props.put("brokerid", nodeId.toString)
-    props.put("port", port.toString)
-    props.put("log.dir", getLogDir.getAbsolutePath)
-    props.put("log.flush.interval", "1")
-    props.put("enable.zookeeper", "false")
-    props.put("num.partitions", "1")
-    props.put("log.retention.hours", "10")
-    props.put("log.cleanup.interval.mins", "5")
-    props.put("log.file.size", "1000")
-    props.put("zk.connect", zkConnect.toString)
-    props
-  }
-
   private def getLogDir(): File = {
     val dir = TestUtils.tempDir()
     dir
Index: core/src/test/scala/unit/kafka/utils/TestUtils.scala
===================================================================
--- core/src/test/scala/unit/kafka/utils/TestUtils.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/utils/TestUtils.scala	(working copy)
@@ -32,6 +32,7 @@
 import collection.mutable.ListBuffer
 import kafka.consumer.{KafkaMessageStream, ConsumerConfig}
 import scala.collection.Map
+import kafka.serializer.Encoder
 
 /**
  * Utility functions to help with testing
@@ -134,6 +135,7 @@
     props.put("zk.sessiontimeout.ms", "400")
     props.put("zk.synctime.ms", "200")
     props.put("autocommit.interval.ms", "1000")
+    props.put("rebalance.retries.max", "4")
 
     props
   }
@@ -275,14 +277,13 @@
   /**
    * Create a producer for the given host and port
    */
-  def createProducer(host: String, port: Int): SyncProducer = {
+  def createProducer[K, V](zkConnect: String): Producer[K, V] = {
     val props = new Properties()
-    props.put("host", host)
-    props.put("port", port.toString)
+    props.put("zk.connect", zkConnect)
     props.put("buffer.size", "65536")
     props.put("connect.timeout.ms", "100000")
     props.put("reconnect.interval", "10000")
-    return new SyncProducer(new SyncProducerConfig(props))
+    new Producer[K, V](new ProducerConfig(props))
   }
 
   def updateConsumerOffset(config : ConsumerConfig, path : String, offset : Long) = {
@@ -308,6 +309,12 @@
     brokers
   }
 
+  def deleteBrokersInZk(zkClient: ZkClient, ids: Seq[Int]): Seq[Broker] = {
+    val brokers = ids.map(id => new Broker(id, "localhost" + System.currentTimeMillis(), "localhost", 6667))
+    brokers.foreach(b => ZkUtils.deletePath(zkClient, ZkUtils.BrokerIdsPath + "/" + b))
+    brokers
+  }
+
   def getConsumedMessages[T](nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaMessageStream[T]]]): List[T]= {
     var messages: List[T] = Nil
     for ((topic, messageStreams) <- topicMessageStreams) {
@@ -335,3 +342,31 @@
 object TestZKUtils {
   val zookeeperConnect = "127.0.0.1:2182"  
 }
+
+class StringSerializer extends Encoder[String] {
+  def toEvent(message: Message):String = message.toString
+  def toMessage(event: String):Message = new Message(event.getBytes)
+  def getTopic(event: String): String = event.concat("-topic")
+}
+
+class NegativePartitioner extends Partitioner[String] {
+  def partition(data: String, numPartitions: Int): Int = {
+    -1
+  }
+}
+
+class StaticPartitioner extends Partitioner[String] {
+  def partition(data: String, numPartitions: Int): Int = {
+    (data.length % numPartitions)
+  }
+}
+
+class HashPartitioner extends Partitioner[String] {
+  def partition(data: String, numPartitions: Int): Int = {
+    (data.hashCode % numPartitions)
+  }
+}
+
+class FixedValuePartitioner extends Partitioner[Int] {
+  def partition(data: Int, numPartitions: Int): Int = data
+}
Index: core/src/test/scala/unit/kafka/log/LogOffsetTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/log/LogOffsetTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/log/LogOffsetTest.scala	(working copy)
@@ -29,6 +29,7 @@
 import org.apache.log4j._
 import kafka.zk.ZooKeeperTestHarness
 import org.scalatest.junit.JUnit3Suite
+import kafka.admin.CreateTopicCommand
 import kafka.api.{FetchRequestBuilder, OffsetRequest}
 
 object LogOffsetTest {
@@ -51,7 +52,7 @@
     val config: Properties = createBrokerConfig(1, brokerPort)
     val logDirPath = config.getProperty("log.dir")
     logDir = new File(logDirPath)
-    
+
     server = TestUtils.createServer(new KafkaConfig(config))
     simpleConsumer = new SimpleConsumer("localhost", brokerPort, 1000000, 64*1024)
   }
@@ -94,10 +95,12 @@
   @Test
   def testGetOffsetsBeforeLatestTime() {
     val topicPartition = "kafka-" + 0
-    val topicPartitionPath = getLogDir.getAbsolutePath + "/" + topicPartition
     val topic = topicPartition.split("-").head
     val part = Integer.valueOf(topicPartition.split("-").last).intValue
 
+    // setup brokers in zookeeper as owners of partitions for this test
+    CreateTopicCommand.createTopic(zookeeper.client, topic, 1, 1, "1")
+
     val logManager = server.getLogManager
     val log = logManager.getOrCreateLog(topic, part)
 
@@ -133,6 +136,9 @@
     val topic = topicPartition.split("-").head
     val part = Integer.valueOf(topicPartition.split("-").last).intValue
 
+    // setup brokers in zookeeper as owners of partitions for this test
+    CreateTopicCommand.createTopic(zookeeper.client, topic, 1, 1, "1")
+
     var offsetChanged = false
     for(i <- 1 to 14) {
       val consumerOffsets = simpleConsumer.getOffsetsBefore(topic, part,
@@ -147,11 +153,13 @@
 
   @Test
   def testGetOffsetsBeforeNow() {
-    val topicPartition = "kafka-" + LogOffsetTest.random.nextInt(10)
-    val topicPartitionPath = getLogDir.getAbsolutePath + "/" + topicPartition
+    val topicPartition = "kafka-" + LogOffsetTest.random.nextInt(3)
     val topic = topicPartition.split("-").head
     val part = Integer.valueOf(topicPartition.split("-").last).intValue
 
+    // setup brokers in zookeeper as owners of partitions for this test
+    CreateTopicCommand.createTopic(zookeeper.client, topic, 3, 1, "1,1,1")
+
     val logManager = server.getLogManager
     val log = logManager.getOrCreateLog(topic, part)
     val message = new Message(Integer.toString(42).getBytes())
@@ -172,11 +180,13 @@
 
   @Test
   def testGetOffsetsBeforeEarliestTime() {
-    val topicPartition = "kafka-" + LogOffsetTest.random.nextInt(10)
-    val topicPartitionPath = getLogDir.getAbsolutePath + "/" + topicPartition
+    val topicPartition = "kafka-" + LogOffsetTest.random.nextInt(3)
     val topic = topicPartition.split("-").head
     val part = Integer.valueOf(topicPartition.split("-").last).intValue
 
+    // setup brokers in zookeeper as owners of partitions for this test
+    CreateTopicCommand.createTopic(zookeeper.client, topic, 3, 1, "1,1,1")
+
     val logManager = server.getLogManager
     val log = logManager.getOrCreateLog(topic, part)
     val message = new Message(Integer.toString(42).getBytes())
Index: core/src/test/scala/unit/kafka/log/LogManagerTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/log/LogManagerTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/log/LogManagerTest.scala	(working copy)
@@ -25,6 +25,7 @@
 import kafka.zk.ZooKeeperTestHarness
 import kafka.utils.{TestZKUtils, Utils, MockTime, TestUtils}
 import org.scalatest.junit.JUnit3Suite
+import kafka.admin.CreateTopicCommand
 
 class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness {
 
@@ -34,6 +35,8 @@
   var logManager: LogManager = null
   var config:KafkaConfig = null
   val zookeeperConnect = TestZKUtils.zookeeperConnect
+  val name = "kafka"
+  val veryLargeLogFlushInterval = 10000000L
 
   override def setUp() {
     super.setUp()
@@ -41,9 +44,13 @@
     config = new KafkaConfig(props) {
                    override val logFileSize = 1024
                  }
-    logManager = new LogManager(config, null, time, -1, maxLogAge, false)
+    logManager = new LogManager(config, time, veryLargeLogFlushInterval, maxLogAge, false)
     logManager.startup
     logDir = logManager.logDir
+
+    // setup brokers in zookeeper as owners of partitions for this test
+    CreateTopicCommand.createTopic(zookeeper.client, name, 3, 1, "0,0,0")
+
   }
 
   override def tearDown() {
@@ -55,7 +62,6 @@
   
   @Test
   def testCreateLog() {
-    val name = "kafka"
     val log = logManager.getOrCreateLog(name, 0)
     val logFile = new File(config.logDir, name + "-0")
     assertTrue(logFile.exists)
@@ -64,7 +70,6 @@
 
   @Test
   def testGetLog() {
-    val name = "kafka"
     val log = logManager.getLog(name, 0)
     val logFile = new File(config.logDir, name + "-0")
     assertTrue(!logFile.exists)
@@ -72,7 +77,7 @@
 
   @Test
   def testCleanupExpiredSegments() {
-    val log = logManager.getOrCreateLog("cleanup", 0)
+    val log = logManager.getOrCreateLog(name, 0)
     var offset = 0L
     for(i <- 0 until 1000) {
       var set = TestUtils.singleMessageSet("test".getBytes())
@@ -111,11 +116,11 @@
       override val logRetentionSize = (5 * 10 * setSize + 10).asInstanceOf[Int] // keep exactly 6 segments + 1 roll over
       override val logRetentionHours = retentionHours
     }
-    logManager = new LogManager(config, null, time, -1, retentionMs, false)
+    logManager = new LogManager(config, time, veryLargeLogFlushInterval, retentionMs, false)
     logManager.startup
 
     // create a log
-    val log = logManager.getOrCreateLog("cleanup", 0)
+    val log = logManager.getOrCreateLog(name, 0)
     var offset = 0L
 
     // add a bunch of messages that should be larger than the retentionSize
@@ -151,14 +156,14 @@
     logManager.close
     Thread.sleep(100)
     config = new KafkaConfig(props) {
-                   override val logFileSize = 1024 *1024 *1024 
+                   override val logFileSize = 1024 *1024 *1024
                    override val flushSchedulerThreadRate = 50
                    override val flushInterval = Int.MaxValue
                    override val flushIntervalMap = Utils.getTopicFlushIntervals("timebasedflush:100")
                  }
-    logManager = new LogManager(config, null, time, -1, maxLogAge, false)
+    logManager = new LogManager(config, time, veryLargeLogFlushInterval, maxLogAge, false)
     logManager.startup
-    val log = logManager.getOrCreateLog("timebasedflush", 0)
+    val log = logManager.getOrCreateLog(name, 0)
     for(i <- 0 until 200) {
       var set = TestUtils.singleMessageSet("test".getBytes())
       log.append(set)
@@ -177,12 +182,12 @@
                    override val logFileSize = 256
                    override val topicPartitionsMap = Utils.getTopicPartitions("testPartition:2")
                  }
-    
-    logManager = new LogManager(config, null, time, -1, maxLogAge, false)
+
+    logManager = new LogManager(config, time, veryLargeLogFlushInterval, maxLogAge, false)
     logManager.startup
-    
-    for(i <- 0 until 2) {
-      val log = logManager.getOrCreateLog("testPartition", i)
+
+    for(i <- 0 until 1) {
+      val log = logManager.getOrCreateLog(name, i)
       for(i <- 0 until 250) {
         var set = TestUtils.singleMessageSet("test".getBytes())
         log.append(set)
@@ -191,7 +196,7 @@
 
     try
     {
-      val log = logManager.getOrCreateLog("testPartition", 2)
+      val log = logManager.getOrCreateLog(name, 2)
       assertTrue("Should not come here", log != null)
     } catch {
        case _ =>
Index: core/src/test/scala/unit/kafka/integration/FetcherTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/integration/FetcherTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/integration/FetcherTest.scala	(working copy)
@@ -28,6 +28,7 @@
 import org.scalatest.junit.JUnit3Suite
 import kafka.integration.KafkaServerTestHarness
 import kafka.utils.TestUtils
+import kafka.producer.{ProducerData, Producer}
 
 class FetcherTest extends JUnit3Suite with KafkaServerTestHarness {
 
@@ -35,7 +36,7 @@
   val configs = 
     for(props <- TestUtils.createBrokerConfigs(numNodes))
       yield new KafkaConfig(props)
-  val messages = new mutable.HashMap[Int, ByteBufferMessageSet]
+  val messages = new mutable.HashMap[Int, Seq[Message]]
   val topic = "topic"
   val cluster = new Cluster(configs.map(c => new Broker(c.brokerId, c.brokerId.toString, "localhost", c.port)))
   val shutdown = ZookeeperConsumerConnector.shutdownCommand
@@ -79,11 +80,10 @@
   def sendMessages(messagesPerNode: Int): Int = {
     var count = 0
     for(conf <- configs) {
-      val producer = TestUtils.createProducer("localhost", conf.port)
+      val producer: Producer[String, Message] = TestUtils.createProducer(zkConnect)
       val ms = 0.until(messagesPerNode).map(x => new Message((conf.brokerId * 5 + x).toString.getBytes)).toArray
-      val mSet = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = ms: _*)
-      messages += conf.brokerId -> mSet
-      producer.send(topic, mSet)
+      messages += conf.brokerId -> ms
+      producer.send(new ProducerData[String, Message](topic, topic, ms))
       producer.close()
       count += ms.size
     }
Index: core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala	(working copy)
@@ -18,7 +18,6 @@
 package kafka.integration
 
 import junit.framework.Assert._
-import kafka.zk.ZooKeeperTestHarness
 import java.nio.channels.ClosedByInterruptException
 import java.util.concurrent.atomic.AtomicInteger
 import kafka.utils.{ZKGroupTopicDirs, Logging}
@@ -27,15 +26,16 @@
 import org.apache.log4j.{Level, Logger}
 import org.scalatest.junit.JUnit3Suite
 import kafka.utils.TestUtils
+import kafka.message.Message
+import kafka.producer.{Producer, ProducerData}
 
-class AutoOffsetResetTest extends JUnit3Suite with ZooKeeperTestHarness with Logging {
+class AutoOffsetResetTest extends JUnit3Suite with KafkaServerTestHarness with Logging {
 
   val topic = "test_topic"
   val group = "default_group"
   val testConsumer = "consumer"
   val brokerPort = 9892
-  val kafkaConfig = new KafkaConfig(TestUtils.createBrokerConfig(0, brokerPort))
-  var kafkaServer : KafkaServer = null
+  val configs = List(new KafkaConfig(TestUtils.createBrokerConfig(0, brokerPort)))
   val numMessages = 10
   val largeOffset = 10000
   val smallOffset = -1
@@ -44,7 +44,6 @@
 
   override def setUp() {
     super.setUp()
-    kafkaServer = TestUtils.createServer(kafkaConfig)
 
     // temporarily set request handler logger to a higher level
     requestHandlerLogger.setLevel(Level.FATAL)
@@ -53,15 +52,14 @@
   override def tearDown() {
     // restore set request handler logger to a higher level
     requestHandlerLogger.setLevel(Level.ERROR)
-    kafkaServer.shutdown
     super.tearDown
   }
   
   def testEarliestOffsetResetForward() = {
-    val producer = TestUtils.createProducer("localhost", brokerPort)
+    val producer: Producer[String, Message] = TestUtils.createProducer(zkConnect)
 
     for(i <- 0 until numMessages) {
-      producer.send(topic, TestUtils.singleMessageSet("test".getBytes()))
+      producer.send(new ProducerData[String, Message](topic, topic, new Message("test".getBytes())))
     }
 
     // update offset in zookeeper for consumer to jump "forward" in time
@@ -71,7 +69,7 @@
     consumerProps.put("consumer.timeout.ms", "2000")
     val consumerConfig = new ConsumerConfig(consumerProps)
     
-    TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0-0", largeOffset)
+    TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0", largeOffset)
     info("Updated consumer offset to " + largeOffset)
 
     Thread.sleep(500)
@@ -112,10 +110,10 @@
   }
 
   def testEarliestOffsetResetBackward() = {
-    val producer = TestUtils.createProducer("localhost", brokerPort)
+    val producer: Producer[String, Message] = TestUtils.createProducer(zkConnect)
 
     for(i <- 0 until numMessages) {
-      producer.send(topic, TestUtils.singleMessageSet("test".getBytes()))
+      producer.send(new ProducerData[String, Message](topic, topic, new Message("test".getBytes())))
     }
 
     // update offset in zookeeper for consumer to jump "forward" in time
@@ -125,7 +123,7 @@
     consumerProps.put("consumer.timeout.ms", "2000")
     val consumerConfig = new ConsumerConfig(consumerProps)
 
-    TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0-0", smallOffset)
+    TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0", smallOffset)
     info("Updated consumer offset to " + smallOffset)
 
 
@@ -145,7 +143,7 @@
               }
             }
             catch {
-              case _: InterruptedException => 
+              case _: InterruptedException =>
               case _: ClosedByInterruptException =>
               case e => throw e
             }
@@ -159,16 +157,15 @@
 
     threadList(0).join(2000)
 
-    info("Asserting...")
     assertEquals(numMessages, nMessages.get)
     consumerConnector.shutdown
   }
 
   def testLatestOffsetResetForward() = {
-    val producer = TestUtils.createProducer("localhost", brokerPort)
+    val producer: Producer[String, Message] = TestUtils.createProducer(zkConnect)
 
     for(i <- 0 until numMessages) {
-      producer.send(topic, TestUtils.singleMessageSet("test".getBytes()))
+      producer.send(new ProducerData[String, Message](topic, topic, new Message("test".getBytes())))
     }
 
     // update offset in zookeeper for consumer to jump "forward" in time
@@ -178,7 +175,7 @@
     consumerProps.put("consumer.timeout.ms", "2000")
     val consumerConfig = new ConsumerConfig(consumerProps)
 
-    TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0-0", largeOffset)
+    TestUtils.updateConsumerOffset(consumerConfig, dirs.consumerOffsetDir + "/" + "0", largeOffset)
     info("Updated consumer offset to " + largeOffset)
 
 
@@ -198,7 +195,7 @@
               }
             }
             catch {
-              case _: InterruptedException => 
+              case _: InterruptedException =>
               case _: ClosedByInterruptException =>
               case e => throw e
             }
Index: core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala	(working copy)
@@ -17,25 +17,25 @@
 
 package kafka.integration
 
-import java.io.File
 import java.nio.ByteBuffer
-import java.util.Properties
 import junit.framework.Assert._
-import kafka.api.{OffsetDetail, FetchRequest, FetchRequestBuilder, ProducerRequest}
+import kafka.api.{OffsetDetail, FetchRequest, FetchRequestBuilder}
 import kafka.common.{FetchRequestFormatException, OffsetOutOfRangeException, InvalidPartitionException}
-import kafka.message.{DefaultCompressionCodec, NoCompressionCodec, Message, ByteBufferMessageSet}
-import kafka.producer.{ProducerData, Producer, ProducerConfig}
-import kafka.serializer.StringDecoder
 import kafka.server.{KafkaRequestHandler, KafkaConfig}
-import kafka.utils.TestUtils
 import org.apache.log4j.{Level, Logger}
 import org.scalatest.junit.JUnit3Suite
+import java.util.Properties
+import kafka.producer.{ProducerData, Producer, ProducerConfig}
+import kafka.serializer.StringDecoder
+import kafka.message.Message
+import java.io.File
+import kafka.utils.{TestZKUtils, TestUtils}
 import scala.collection._
 
 /**
  * End to end tests of the primitive apis against a local server
  */
-class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with KafkaServerTestHarness {
+class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness {
   
   val port = TestUtils.choosePort
   val props = TestUtils.createBrokerConfig(0, port)
@@ -45,6 +45,20 @@
   val configs = List(config)
   val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler])
 
+//<<<<<<< .mine
+  override def setUp() {
+    super.setUp
+    // temporarily set request handler logger to a higher level
+    requestHandlerLogger.setLevel(Level.FATAL)
+  }
+
+  override def tearDown() {
+    // restore set request handler logger to a higher level
+    requestHandlerLogger.setLevel(Level.ERROR)
+
+    super.tearDown
+  }
+
   def testFetchRequestCanProperlySerialize() {
     val request = new FetchRequestBuilder()
       .correlationId(100)
@@ -83,7 +97,7 @@
     val topic = "test-topic"
     val props = new Properties()
     props.put("serializer.class", "kafka.serializer.StringEncoder")
-    props.put("broker.list", "0:localhost:" + port)
+    props.put("zk.connect", TestZKUtils.zookeeperConnect)
     val config = new ProducerConfig(props)
 
     val stringProducer1 = new Producer[String, String](config)
@@ -111,7 +125,7 @@
     val topic = "test-topic"
     val props = new Properties()
     props.put("serializer.class", "kafka.serializer.StringEncoder")
-    props.put("broker.list", "0:localhost:" + port)
+    props.put("zk.connect", TestZKUtils.zookeeperConnect)
     props.put("compression", "true")
     val config = new ProducerConfig(props)
 
@@ -133,14 +147,13 @@
     // send some messages
     val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
     {
-      val messages = new mutable.HashMap[String, ByteBufferMessageSet]
+      val messages = new mutable.HashMap[String, Seq[Message]]
       val builder = new FetchRequestBuilder()
       for( (topic, partition) <- topics) {
-        val set = new ByteBufferMessageSet(NoCompressionCodec,
-                                           new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
-        messages += topic -> set
-        producer.send(topic, set)
-        set.getBuffer.rewind
+        val messageList = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
+        val producerData = new ProducerData[String, Message](topic, topic, messageList)
+        messages += topic -> messageList
+        producer.send(producerData)
         builder.addFetch(topic, partition, 0, 10000)
       }
 
@@ -150,7 +163,7 @@
       val response = consumer.fetch(request)
       for( (topic, partition) <- topics) {
         val fetched = response.messageSet(topic, partition)
-        TestUtils.checkEquals(messages(topic).iterator, fetched.iterator)
+        TestUtils.checkEquals(messages(topic).iterator, fetched.map(messageAndOffset => messageAndOffset.message).iterator)
       }
     }
 
@@ -172,7 +185,7 @@
       } catch {
         case e: OffsetOutOfRangeException => "this is good"
       }
-    }    
+    }
 
     {
       // send some invalid partitions
@@ -199,14 +212,13 @@
     // send some messages
     val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
     {
-      val messages = new mutable.HashMap[String, ByteBufferMessageSet]
+      val messages = new mutable.HashMap[String, Seq[Message]]
       val builder = new FetchRequestBuilder()
       for( (topic, partition) <- topics) {
-        val set = new ByteBufferMessageSet(DefaultCompressionCodec,
-                                           new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
-        messages += topic -> set
-        producer.send(topic, set)
-        set.getBuffer.rewind
+        val messageList = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
+        val producerData = new ProducerData[String, Message](topic, topic, messageList)
+        messages += topic -> messageList
+        producer.send(producerData)
         builder.addFetch(topic, partition, 0, 10000)
       }
 
@@ -216,7 +228,7 @@
       val response = consumer.fetch(request)
       for( (topic, partition) <- topics) {
         val fetched = response.messageSet(topic, partition)
-        TestUtils.checkEquals(messages(topic).iterator, fetched.iterator)
+        TestUtils.checkEquals(messages(topic).iterator, fetched.map(messageAndOffset => messageAndOffset.message).iterator)
       }
     }
 
@@ -264,56 +276,50 @@
   def testMultiProduce() {
     // send some messages
     val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
-    val messages = new mutable.HashMap[String, ByteBufferMessageSet]
+    val messages = new mutable.HashMap[String, Seq[Message]]
     val builder = new FetchRequestBuilder()
-    var produceList: List[ProducerRequest] = Nil
+    var produceList: List[ProducerData[String, Message]] = Nil
     for( (topic, partition) <- topics) {
-      val set = new ByteBufferMessageSet(NoCompressionCodec,
-                                         new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
-      messages += topic -> set
-      produceList ::= new ProducerRequest(topic, 0, set)
+      val messageList = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
+      val producerData = new ProducerData[String, Message](topic, topic, messageList)
+      messages += topic -> messageList
+      producer.send(producerData)
       builder.addFetch(topic, partition, 0, 10000)
     }
-    producer.multiSend(produceList.toArray)
+    producer.send(produceList: _*)
 
-    for (messageSet <- messages.values)
-      messageSet.getBuffer.rewind
-      
     // wait a bit for produced message to be available
     Thread.sleep(200)
     val request = builder.build()
     val response = consumer.fetch(request)
     for( (topic, partition) <- topics) {
       val fetched = response.messageSet(topic, partition)
-      TestUtils.checkEquals(messages(topic).iterator, fetched.iterator)
+      TestUtils.checkEquals(messages(topic).iterator, fetched.map(messageAndOffset => messageAndOffset.message).iterator)
     }
   }
 
   def testMultiProduceWithCompression() {
     // send some messages
     val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
-    val messages = new mutable.HashMap[String, ByteBufferMessageSet]
+    val messages = new mutable.HashMap[String, Seq[Message]]
     val builder = new FetchRequestBuilder()
-    var produceList: List[ProducerRequest] = Nil
+    var produceList: List[ProducerData[String, Message]] = Nil
     for( (topic, partition) <- topics) {
-      val set = new ByteBufferMessageSet(DefaultCompressionCodec,
-                                         new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
-      messages += topic -> set
-      produceList ::= new ProducerRequest(topic, 0, set)
+      val messageList = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
+      val producerData = new ProducerData[String, Message](topic, topic, messageList)
+      messages += topic -> messageList
+      producer.send(producerData)
       builder.addFetch(topic, partition, 0, 10000)
     }
-    producer.multiSend(produceList.toArray)
+    producer.send(produceList: _*)
 
-    for (messageSet <- messages.values)
-      messageSet.getBuffer.rewind
-
     // wait a bit for produced message to be available
     Thread.sleep(200)
     val request = builder.build()
     val response = consumer.fetch(request)
     for( (topic, partition) <- topics) {
       val fetched = response.messageSet(topic, 0)
-      TestUtils.checkEquals(messages(topic).iterator, fetched.iterator)
+      TestUtils.checkEquals(messages(topic).iterator, fetched.map(messageAndOffset => messageAndOffset.message).iterator)
     }
   }
 
Index: core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala
===================================================================
--- core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala	(working copy)
@@ -20,33 +20,35 @@
 import kafka.consumer.SimpleConsumer
 import org.scalatest.junit.JUnit3Suite
 import java.util.Properties
-import kafka.producer.{SyncProducerConfig, SyncProducer}
+import kafka.utils.TestZKUtils
+import kafka.producer.{ProducerConfig, Producer}
+import kafka.message.Message
 
-trait ProducerConsumerTestHarness extends JUnit3Suite {
+trait ProducerConsumerTestHarness extends JUnit3Suite with KafkaServerTestHarness {
   
     val port: Int
     val host = "localhost"
-    var producer: SyncProducer = null
+    var producer: Producer[String, Message] = null
     var consumer: SimpleConsumer = null
 
     override def setUp() {
+      super.setUp
       val props = new Properties()
-      props.put("host", host)
-      props.put("port", port.toString)
+      props.put("partitioner.class", "kafka.utils.StaticPartitioner")
+      props.put("zk.connect", TestZKUtils.zookeeperConnect)
       props.put("buffer.size", "65536")
       props.put("connect.timeout.ms", "100000")
       props.put("reconnect.interval", "10000")
-      producer = new SyncProducer(new SyncProducerConfig(props))
+      producer = new Producer(new ProducerConfig(props))
       consumer = new SimpleConsumer(host,
                                    port,
                                    1000000,
                                    64*1024)
-      super.setUp
     }
 
    override def tearDown() {
-     super.tearDown
      producer.close()
      consumer.close()
+     super.tearDown
    }
 }
Index: core/src/test/scala/unit/kafka/integration/LogCorruptionTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/integration/LogCorruptionTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/integration/LogCorruptionTest.scala	(working copy)
@@ -24,10 +24,11 @@
 import kafka.common.InvalidMessageSizeException
 import kafka.consumer.{ZookeeperConsumerConnector, ConsumerConfig}
 import kafka.integration.{KafkaServerTestHarness, ProducerConsumerTestHarness}
-import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet}
+import kafka.message.Message
 import kafka.utils.{Utils, TestUtils}
 import org.scalatest.junit.JUnit3Suite
 import org.apache.log4j.{Logger, Level}
+import kafka.producer.ProducerData
 
 class LogCorruptionTest extends JUnit3Suite with ProducerConsumerTestHarness with KafkaServerTestHarness {
   val port = TestUtils.choosePort
@@ -47,8 +48,9 @@
     fetcherLogger.setLevel(Level.FATAL)
 
     // send some messages
-    val sent1 = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message("hello".getBytes()))
-    producer.send(topic, sent1)
+    val producerData = new ProducerData[String, Message](topic, topic, List(new Message("hello".getBytes())))
+
+    producer.send(producerData)
     Thread.sleep(200)
 
     // corrupt the file on disk
Index: core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala	(working copy)
@@ -17,33 +17,31 @@
 
 package kafka.integration
 
-import kafka.api.{FetchRequestBuilder, ProducerRequest}
+import kafka.api.FetchRequestBuilder
 import kafka.common.OffsetOutOfRangeException
-import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet}
-import kafka.server.{KafkaRequestHandler, KafkaServer, KafkaConfig}
-import kafka.utils.{TestUtils, Utils}
-import kafka.zk.ZooKeeperTestHarness
+import kafka.message.{Message, ByteBufferMessageSet}
+import kafka.server.{KafkaRequestHandler, KafkaConfig}
+import kafka.utils.TestUtils
 import org.apache.log4j.{Level, Logger}
 import org.scalatest.junit.JUnit3Suite
 import scala.collection._
+import kafka.producer.ProducerData
 
 /**
  * End to end tests of the primitive apis against a local server
  */
-class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness with ZooKeeperTestHarness  {
+class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness {
 
   val port = TestUtils.choosePort
   val props = TestUtils.createBrokerConfig(0, port)
   val config = new KafkaConfig(props)
   val configs = List(config)
-  var servers: List[KafkaServer] = null
   val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler])
 
   override def setUp() {
     super.setUp
     if(configs.size <= 0)
       throw new IllegalArgumentException("Must suply at least one server config.")
-    servers = configs.map(TestUtils.createServer(_))
 
     // temporarily set request handler logger to a higher level
     requestHandlerLogger.setLevel(Level.FATAL)    
@@ -54,24 +52,21 @@
     requestHandlerLogger.setLevel(Level.ERROR)
 
     super.tearDown    
-    servers.map(server => server.shutdown())
-    servers.map(server => Utils.rm(server.config.logDir))
   }
   
   def testProduceAndFetch() {
     // send some messages
     val topic = "test"
-    val sent = new ByteBufferMessageSet(NoCompressionCodec,
-                                        new Message("hello".getBytes()), new Message("there".getBytes()))
-    producer.send(topic, sent)
-    sent.getBuffer.rewind
+    val sentMessages = List(new Message("hello".getBytes()), new Message("there".getBytes()))
+    val producerData = new ProducerData[String, Message](topic, topic, sentMessages)
 
+    producer.send(producerData)
     var fetchedMessage: ByteBufferMessageSet = null
     while(fetchedMessage == null || fetchedMessage.validBytes == 0) {
       val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
       fetchedMessage = fetched.messageSet(topic, 0)
     }
-    TestUtils.checkEquals(sent.iterator, fetchedMessage.iterator)
+    TestUtils.checkEquals(sentMessages.iterator, fetchedMessage.map(m => m.message).iterator)
 
     // send an invalid offset
     try {
@@ -87,14 +82,12 @@
     // send some messages, with non-ordered topics
     val topicOffsets = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
     {
-      val messages = new mutable.HashMap[String, ByteBufferMessageSet]
+      val messages = new mutable.HashMap[String, Seq[Message]]
       val builder = new FetchRequestBuilder()
       for( (topic, offset) <- topicOffsets) {
-        val set = new ByteBufferMessageSet(NoCompressionCodec,
-                                           new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
-        producer.send(topic, set)
-        set.getBuffer.rewind
-        messages += topic -> set
+        val producedData = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
+        messages += topic -> producedData
+        producer.send(new ProducerData[String, Message](topic, topic, producedData))
         builder.addFetch(topic, offset, 0, 10000)
       }
 
@@ -104,7 +97,7 @@
       val response = consumer.fetch(request)
       for( (topic, offset) <- topicOffsets) {
         val fetched = response.messageSet(topic, offset)
-        TestUtils.checkEquals(messages(topic).iterator, fetched.iterator)
+        TestUtils.checkEquals(messages(topic).iterator, fetched.map(m => m.message).iterator)
       }
     }
 
@@ -121,7 +114,7 @@
           responses.messageSet(topic, offset).iterator
           fail("Expected an OffsetOutOfRangeException exception to be thrown")
         } catch {
-          case e: OffsetOutOfRangeException => 
+          case e: OffsetOutOfRangeException =>
         }
       }
     }
@@ -130,60 +123,50 @@
   def testMultiProduce() {
     // send some messages
     val topics = List("test1", "test2", "test3");
-    val messages = new mutable.HashMap[String, ByteBufferMessageSet]
+    val messages = new mutable.HashMap[String, Seq[Message]]
     val builder = new FetchRequestBuilder()
-    var produceList: List[ProducerRequest] = Nil
+    var produceList: List[ProducerData[String, Message]] = Nil
     for(topic <- topics) {
-      val set = new ByteBufferMessageSet(NoCompressionCodec,
-                                         new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
+      val set = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
       messages += topic -> set
-      produceList ::= new ProducerRequest(topic, 0, set)
+      produceList ::= new ProducerData[String, Message](topic, topic, set)
       builder.addFetch(topic, 0, 0, 10000)
     }
-    producer.multiSend(produceList.toArray)
+    producer.send(produceList: _*)
 
-    for (messageSet <- messages.values)
-      messageSet.getBuffer.rewind
-
     // wait a bit for produced message to be available
     Thread.sleep(200)
     val request = builder.build()
     val response = consumer.fetch(request)
     for(topic <- topics) {
       val fetched = response.messageSet(topic, 0)
-      TestUtils.checkEquals(messages(topic).iterator, fetched.iterator)
+      TestUtils.checkEquals(messages(topic).iterator, fetched.map(m => m.message).iterator)
     }
   }
 
   def testMultiProduceResend() {
     // send some messages
     val topics = List("test1", "test2", "test3");
-    val messages = new mutable.HashMap[String, ByteBufferMessageSet]
+    val messages = new mutable.HashMap[String, Seq[Message]]
     val builder = new FetchRequestBuilder()
-    var produceList: List[ProducerRequest] = Nil
+    var produceList: List[ProducerData[String, Message]] = Nil
     for(topic <- topics) {
-      val set = new ByteBufferMessageSet(NoCompressionCodec,
-                                         new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
+      val set = List(new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
       messages += topic -> set
-      produceList ::= new ProducerRequest(topic, 0, set)
+      produceList ::= new ProducerData[String, Message](topic, topic, set)
       builder.addFetch(topic, 0, 0, 10000)
     }
-    producer.multiSend(produceList.toArray)
+    producer.send(produceList: _*)
 
-    // resend the same multisend
-    producer.multiSend(produceList.toArray)
-
-    for (messageSet <- messages.values)
-      messageSet.getBuffer.rewind
-
+    producer.send(produceList: _*)
     // wait a bit for produced message to be available
     Thread.sleep(750)
     val request = builder.build()
     val response = consumer.fetch(request)
     for(topic <- topics) {
       val topicMessages = response.messageSet(topic, 0)
-      TestUtils.checkEquals(TestUtils.stackedIterator(messages(topic).map(m => m.message).iterator,
-                                                      messages(topic).map(m => m.message).iterator),
+      TestUtils.checkEquals(TestUtils.stackedIterator(messages(topic).iterator,
+                                                      messages(topic).iterator),
                             topicMessages.iterator.map(_.message))
     }
   }
Index: core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala	(working copy)
@@ -21,11 +21,11 @@
 import kafka.server.KafkaConfig
 import kafka.common.MessageSizeTooLargeException
 import java.util.Properties
-import kafka.api.ProducerRequest
 import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet}
 import kafka.integration.KafkaServerTestHarness
 import kafka.utils.{TestZKUtils, SystemTime, TestUtils}
 import org.scalatest.junit.JUnit3Suite
+import kafka.api.ProducerRequest
 
 class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness {
   private var messageBytes =  new Array[Byte](2);
Index: core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala	(working copy)
@@ -18,24 +18,42 @@
 package kafka.producer
 
 import org.easymock.EasyMock
-import kafka.api.ProducerRequest
 import org.junit.Test
-import org.scalatest.junit.JUnitSuite
 import kafka.producer.async._
 import java.util.concurrent.LinkedBlockingQueue
 import junit.framework.Assert._
-import collection.SortedSet
-import kafka.cluster.{Broker, Partition}
-import collection.mutable.{HashMap, ListBuffer}
+import kafka.cluster.Broker
+import collection.mutable.ListBuffer
 import collection.Map
 import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message}
 import kafka.serializer.{StringEncoder, StringDecoder, Encoder}
 import java.util.{LinkedList, Properties}
-import kafka.utils.{TestZKUtils, TestUtils}
 import kafka.common.{InvalidConfigException, NoBrokersForPartitionException, InvalidPartitionException}
+import kafka.api.{PartitionMetadata, TopicMetadata, TopicMetadataRequest, ProducerRequest}
+import kafka.utils.{NegativePartitioner, TestZKUtils, TestUtils}
+import kafka.zk.ZooKeeperTestHarness
+import org.scalatest.junit.JUnit3Suite
+import kafka.utils.TestUtils._
+import kafka.server.KafkaConfig
+import org.I0Itec.zkclient.ZkClient
 
-class AsyncProducerTest extends JUnitSuite {
+class AsyncProducerTest extends JUnit3Suite with ZooKeeperTestHarness {
+  val props = createBrokerConfigs(1)
+  val configs = props.map(p => new KafkaConfig(p) { override val flushInterval = 1})
+  var zkClient: ZkClient = null
+  var brokers: Seq[Broker] = null
 
+  override def setUp() {
+    super.setUp()
+    zkClient = zookeeper.client
+    // create brokers in zookeeper
+    brokers = TestUtils.createBrokersInZk(zkClient, configs.map(config => config.brokerId))
+  }
+
+  override def tearDown() {
+    super.tearDown()
+  }
+
   @Test
   def testProducerQueueSize() {
     // a mock event handler that blocks
@@ -50,7 +68,7 @@
 
     val props = new Properties()
     props.put("serializer.class", "kafka.serializer.StringEncoder")
-    props.put("broker.list", "0:localhost:9092")
+    props.put("zk.connect", TestZKUtils.zookeeperConnect)
     props.put("producer.type", "async")
     props.put("queue.size", "10")
     props.put("batch.size", "1")
@@ -72,13 +90,13 @@
   def testProduceAfterClosed() {
     val props = new Properties()
     props.put("serializer.class", "kafka.serializer.StringEncoder")
-    props.put("broker.list", "0:localhost:9092")
+    props.put("zk.connect", TestZKUtils.zookeeperConnect)
     props.put("producer.type", "async")
     props.put("batch.size", "1")
 
     val config = new ProducerConfig(props)
     val produceData = getProduceData(10)
-    val producer = new Producer[String, String](config)
+    val producer = new Producer[String, String](config, zkClient)
     producer.close
 
     try {
@@ -157,19 +175,36 @@
     producerDataList.append(new ProducerData[Int,Message]("topic2", 4, new Message("msg5".getBytes)))
 
     val props = new Properties()
-    props.put("broker.list", "0:localhost:9092,1:localhost:9092")
+    props.put("zk.connect", zkConnect)
+    val broker1 = new Broker(0, "localhost", "localhost", 9092)
+    val broker2 = new Broker(1, "localhost", "localhost", 9093)
+    // form expected partitions metadata
+    val partition1Metadata = new PartitionMetadata(0, Some(broker1), List(broker1, broker2))
+    val partition2Metadata = new PartitionMetadata(1, Some(broker2), List(broker1, broker2))
+    val topic1Metadata = new TopicMetadata("topic1", List(partition1Metadata, partition2Metadata))
+    val topic2Metadata = new TopicMetadata("topic2", List(partition1Metadata, partition2Metadata))
 
     val intPartitioner = new Partitioner[Int] {
       def partition(key: Int, numPartitions: Int): Int = key % numPartitions
     }
     val config = new ProducerConfig(props)
+
+    val syncProducer = getSyncProducer(List("topic1", "topic2"), List(topic1Metadata, topic2Metadata))
+
+    val producerPool = EasyMock.createMock(classOf[ProducerPool])
+    producerPool.getZkClient
+    EasyMock.expectLastCall().andReturn(zkClient)
+    producerPool.addProducers(config)
+    EasyMock.expectLastCall()
+    producerPool.getAnyProducer
+    EasyMock.expectLastCall().andReturn(syncProducer).times(2)
+    EasyMock.replay(producerPool)
     val handler = new DefaultEventHandler[Int,String](config,
                                                       partitioner = intPartitioner,
                                                       encoder = null.asInstanceOf[Encoder[String]],
-                                                      producerPool = null,
-                                                      populateProducerPool = false,
-                                                      brokerPartitionInfo = null)
+                                                      producerPool)
 
+
     val topic1Broker1Data = new ListBuffer[ProducerData[Int,Message]]
     topic1Broker1Data.appendAll(List(new ProducerData[Int,Message]("topic1", 0, new Message("msg1".getBytes)),
                                      new ProducerData[Int,Message]("topic1", 2, new Message("msg3".getBytes))))
@@ -181,29 +216,34 @@
     topic2Broker2Data.appendAll(List(new ProducerData[Int,Message]("topic2", 1, new Message("msg2".getBytes))))
     val expectedResult = Map(
         0 -> Map(
-              ("topic1", -1) -> topic1Broker1Data,
-              ("topic2", -1) -> topic2Broker1Data),
+              ("topic1", 0) -> topic1Broker1Data,
+              ("topic2", 0) -> topic2Broker1Data),
         1 -> Map(
-              ("topic1", -1) -> topic1Broker2Data,
-              ("topic2", -1) -> topic2Broker2Data)
+              ("topic1", 1) -> topic1Broker2Data,
+              ("topic2", 1) -> topic2Broker2Data)
       )
 
     val actualResult = handler.partitionAndCollate(producerDataList)
     assertEquals(expectedResult, actualResult)
+    EasyMock.verify(syncProducer)
+    EasyMock.verify(producerPool)
   }
 
   @Test
   def testSerializeEvents() {
     val produceData = TestUtils.getMsgStrings(5).map(m => new ProducerData[String,String]("topic1",m))
     val props = new Properties()
-    props.put("broker.list", "0:localhost:9092,1:localhost:9092")
+    props.put("zk.connect", zkConnect)
     val config = new ProducerConfig(props)
+    // form expected partitions metadata
+    val topic1Metadata = getTopicMetadata("topic1", 0, "localhost", 9092)
+
+    val syncProducer = getSyncProducer(List("topic1"), List(topic1Metadata))
+    val producerPool = getMockProducerPool(config, syncProducer)
     val handler = new DefaultEventHandler[String,String](config,
                                                          partitioner = null.asInstanceOf[Partitioner[String]],
                                                          encoder = new StringEncoder,
-                                                         producerPool = null,
-                                                         populateProducerPool = false,
-                                                         brokerPartitionInfo = null)
+                                                         producerPool)
 
     val serializedData = handler.serialize(produceData)
     val decoder = new StringDecoder
@@ -216,14 +256,20 @@
     val producerDataList = new ListBuffer[ProducerData[String,Message]]
     producerDataList.append(new ProducerData[String,Message]("topic1", "key1", new Message("msg1".getBytes)))
     val props = new Properties()
-    props.put("broker.list", "0:localhost:9092,1:localhost:9092")
+    props.put("zk.connect", TestZKUtils.zookeeperConnect)
     val config = new ProducerConfig(props)
+
+    // form expected partitions metadata
+    val topic1Metadata = getTopicMetadata("topic1", 0, "localhost", 9092)
+
+    val syncProducer = getSyncProducer(List("topic1"), List(topic1Metadata))
+
+    val producerPool = getMockProducerPool(config, syncProducer)
+
     val handler = new DefaultEventHandler[String,String](config,
                                                          partitioner = new NegativePartitioner,
                                                          encoder = null.asInstanceOf[Encoder[String]],
-                                                         producerPool = null,
-                                                         populateProducerPool = false,
-                                                         brokerPartitionInfo = null)
+                                                         producerPool)
     try {
       handler.partitionAndCollate(producerDataList)
       fail("Should fail with InvalidPartitionException")
@@ -231,34 +277,29 @@
     catch {
       case e: InvalidPartitionException => // expected, do nothing
     }
+    EasyMock.verify(syncProducer)
+    EasyMock.verify(producerPool)
   }
 
-  private def getMockBrokerPartitionInfo(): BrokerPartitionInfo ={
-    new BrokerPartitionInfo {
-      def getBrokerPartitionInfo(topic: String = null): SortedSet[Partition] = SortedSet.empty[Partition]
+  @Test
+  def testNoBroker() {
+    val props = new Properties()
+    props.put("zk.connect", zkConnect)
 
-      def getBrokerInfo(brokerId: Int): Option[Broker] = None
+    val config = new ProducerConfig(props)
+    // create topic metadata with 0 partitions
+    val topic1Metadata = new TopicMetadata("topic1", Seq.empty)
 
-      def getAllBrokerInfo: Map[Int, Broker] = new HashMap[Int, Broker]
+    val syncProducer = getSyncProducer(List("topic1"), List(topic1Metadata))
 
-      def updateInfo = {}
+    val producerPool = getMockProducerPool(config, syncProducer)
 
-      def close = {}
-    }
-  }
-
-  @Test
-  def testNoBroker() {
     val producerDataList = new ListBuffer[ProducerData[String,String]]
     producerDataList.append(new ProducerData[String,String]("topic1", "msg1"))
-    val props = new Properties()
-    val config = new ProducerConfig(props)
     val handler = new DefaultEventHandler[String,String](config,
                                                          partitioner = null.asInstanceOf[Partitioner[String]],
                                                          encoder = new StringEncoder,
-                                                         producerPool = null,
-                                                         populateProducerPool = false,
-                                                         brokerPartitionInfo = getMockBrokerPartitionInfo)
+                                                         producerPool)
     try {
       handler.handle(producerDataList)
       fail("Should fail with NoBrokersForPartitionException")
@@ -266,12 +307,14 @@
     catch {
       case e: NoBrokersForPartitionException => // expected, do nothing
     }
+    EasyMock.verify(syncProducer)
+    EasyMock.verify(producerPool)
   }
 
   @Test
   def testIncompatibleEncoder() {
     val props = new Properties()
-    props.put("broker.list", "0:localhost:9092,1:localhost:9092")
+    props.put("zk.connect", TestZKUtils.zookeeperConnect)
     val config = new ProducerConfig(props)
 
     val producer=new Producer[String, String](config)
@@ -286,14 +329,28 @@
   @Test
   def testRandomPartitioner() {
     val props = new Properties()
-    props.put("broker.list", "0:localhost:9092,1:localhost:9092")
+    props.put("zk.connect", TestZKUtils.zookeeperConnect)
     val config = new ProducerConfig(props)
+
+    // create topic metadata with 0 partitions
+    val topic1Metadata = getTopicMetadata("topic1", 0, "localhost", 9092)
+    val topic2Metadata = getTopicMetadata("topic2", 0, "localhost", 9092)
+
+    val syncProducer = getSyncProducer(List("topic1", "topic2"), List(topic1Metadata, topic2Metadata))
+
+    val producerPool = EasyMock.createMock(classOf[ProducerPool])
+    producerPool.getZkClient
+    EasyMock.expectLastCall().andReturn(zkClient)
+    producerPool.addProducers(config)
+    EasyMock.expectLastCall()
+    producerPool.getAnyProducer
+    EasyMock.expectLastCall().andReturn(syncProducer).times(2)
+    EasyMock.replay(producerPool)
+
     val handler = new DefaultEventHandler[String,String](config,
                                                          partitioner = null.asInstanceOf[Partitioner[String]],
                                                          encoder = null.asInstanceOf[Encoder[String]],
-                                                         producerPool = null,
-                                                         populateProducerPool = false,
-                                                         brokerPartitionInfo = null)
+                                                         producerPool)
     val producerDataList = new ListBuffer[ProducerData[String,Message]]
     producerDataList.append(new ProducerData[String,Message]("topic1", new Message("msg1".getBytes)))
     producerDataList.append(new ProducerData[String,Message]("topic2", new Message("msg2".getBytes)))
@@ -302,41 +359,51 @@
     val partitionedData = handler.partitionAndCollate(producerDataList)
     for ((brokerId, dataPerBroker) <- partitionedData) {
       for ( ((topic, partitionId), dataPerTopic) <- dataPerBroker)
-        assertTrue(partitionId == ProducerRequest.RandomPartition)
+        assertTrue(partitionId == 0)
     }
+    EasyMock.verify(producerPool)
   }
 
   @Test
   def testBrokerListAndAsync() {
+    val props = new Properties()
+    props.put("serializer.class", "kafka.serializer.StringEncoder")
+    props.put("producer.type", "async")
+    props.put("batch.size", "5")
+    props.put("zk.connect", TestZKUtils.zookeeperConnect)
+
+    val config = new ProducerConfig(props)
+
     val topic = "topic1"
+    val topic1Metadata = getTopicMetadata(topic, 0, "localhost", 9092)
+
     val msgs = TestUtils.getMsgStrings(10)
     val mockSyncProducer = EasyMock.createMock(classOf[SyncProducer])
-    mockSyncProducer.multiSend(EasyMock.aryEq(Array(new ProducerRequest(topic, ProducerRequest.RandomPartition,
-      messagesToSet(msgs.take(5))))))
+    mockSyncProducer.send(new TopicMetadataRequest(List(topic)))
+    EasyMock.expectLastCall().andReturn(List(topic1Metadata))
+    mockSyncProducer.multiSend(EasyMock.aryEq(Array(new ProducerRequest(topic, 0, messagesToSet(msgs.take(5))))))
     EasyMock.expectLastCall
-    mockSyncProducer.multiSend(EasyMock.aryEq(Array(new ProducerRequest(topic, ProducerRequest.RandomPartition,
-      messagesToSet(msgs.takeRight(5))))))
+    mockSyncProducer.multiSend(EasyMock.aryEq(Array(new ProducerRequest(topic, 0, messagesToSet(msgs.takeRight(5))))))
     EasyMock.expectLastCall
-    mockSyncProducer.close
-    EasyMock.expectLastCall
     EasyMock.replay(mockSyncProducer)
 
-    val props = new Properties()
-    props.put("serializer.class", "kafka.serializer.StringEncoder")
-    props.put("producer.type", "async")
-    props.put("batch.size", "5")
-    props.put("broker.list", "0:localhost:9092")
+    val producerPool = EasyMock.createMock(classOf[ProducerPool])
+    producerPool.getZkClient
+    EasyMock.expectLastCall().andReturn(zkClient)
+    producerPool.addProducers(config)
+    EasyMock.expectLastCall()
+    producerPool.getAnyProducer
+    EasyMock.expectLastCall().andReturn(mockSyncProducer)
+    producerPool.getProducer(0)
+    EasyMock.expectLastCall().andReturn(mockSyncProducer).times(2)
+    producerPool.close()
+    EasyMock.expectLastCall()
+    EasyMock.replay(producerPool)
 
-    val config = new ProducerConfig(props)
-    val producerPool = new ProducerPool(config)
-    producerPool.addProducer(0, mockSyncProducer)
-
     val handler = new DefaultEventHandler[String,String](config,
                                                       partitioner = null.asInstanceOf[Partitioner[String]],
                                                       encoder = new StringEncoder,
-                                                      producerPool = producerPool,
-                                                      populateProducerPool = false,
-                                                      brokerPartitionInfo = null)
+                                                      producerPool = producerPool)
 
     val producer = new Producer[String, String](config, handler)
     try {
@@ -349,6 +416,7 @@
     }
 
     EasyMock.verify(mockSyncProducer)
+    EasyMock.verify(producerPool)
   }
 
   @Test
@@ -380,7 +448,7 @@
   def testInvalidConfiguration() {
     val props = new Properties()
     props.put("serializer.class", "kafka.serializer.StringEncoder")
-    props.put("broker.list", "0:localhost:9092")
+    props.put("broker.list", TestZKUtils.zookeeperConnect)
     props.put("zk.connect", TestZKUtils.zookeeperConnect)
     props.put("producer.type", "async")
 
@@ -398,6 +466,34 @@
     new ByteBufferMessageSet(NoCompressionCodec, messages.map(m => encoder.toMessage(m)): _*)
   }
 
+  private def getSyncProducer(topic: Seq[String], topicMetadata: Seq[TopicMetadata]): SyncProducer = {
+    val syncProducer = EasyMock.createMock(classOf[SyncProducer])
+    topic.zip(topicMetadata).foreach { topicAndMetadata =>
+      syncProducer.send(new TopicMetadataRequest(List(topicAndMetadata._1)))
+      EasyMock.expectLastCall().andReturn(List(topicAndMetadata._2))
+    }
+    EasyMock.replay(syncProducer)
+    syncProducer
+  }
+
+  private def getMockProducerPool(config: ProducerConfig, syncProducer: SyncProducer): ProducerPool = {
+    val producerPool = EasyMock.createMock(classOf[ProducerPool])
+    producerPool.getZkClient
+    EasyMock.expectLastCall().andReturn(zkClient)
+    producerPool.addProducers(config)
+    EasyMock.expectLastCall()
+    producerPool.getAnyProducer
+    EasyMock.expectLastCall().andReturn(syncProducer)
+    EasyMock.replay(producerPool)
+    producerPool
+  }
+
+  private def getTopicMetadata(topic: String, brokerId: Int, brokerHost: String, brokerPort: Int): TopicMetadata = {
+    val broker1 = new Broker(brokerId, brokerHost, brokerHost, brokerPort)
+    val partition1Metadata = new PartitionMetadata(brokerId, Some(broker1), List(broker1))
+    new TopicMetadata(topic, List(partition1Metadata))
+  }
+
   class MockProducer(override val config: SyncProducerConfig) extends SyncProducer(config) {
     override def send(topic: String, messages: ByteBufferMessageSet): Unit = {
       Thread.sleep(1000)
Index: core/src/test/scala/unit/kafka/producer/ProducerTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/producer/ProducerTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/producer/ProducerTest.scala	(working copy)
@@ -17,20 +17,21 @@
 
 package kafka.producer
 
-import junit.framework.Assert._
-import java.util.Properties
-import kafka.api.FetchRequestBuilder
+import org.scalatest.junit.JUnit3Suite
+import kafka.zk.ZooKeeperTestHarness
 import kafka.consumer.SimpleConsumer
+import org.I0Itec.zkclient.ZkClient
+import kafka.server.{KafkaConfig, KafkaRequestHandler, KafkaServer}
+import java.util.Properties
+import org.apache.log4j.{Level, Logger}
+import org.junit.Test
+import kafka.utils.{TestZKUtils, Utils, TestUtils}
 import kafka.message.Message
-import kafka.serializer.Encoder
-import kafka.server.{KafkaRequestHandler, KafkaServer, KafkaConfig}
-import kafka.utils.{TestUtils, TestZKUtils, Utils}
-import kafka.zk.EmbeddedZookeeper
-import org.apache.log4j.{Logger, Level}
-import org.junit.{After, Before, Test}
-import org.scalatest.junit.JUnitSuite
+import kafka.admin.CreateTopicCommand
+import kafka.api.FetchRequestBuilder
+import org.junit.Assert._
 
-class ProducerTest extends JUnitSuite {
+class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness {
   private val topic = "test-topic"
   private val brokerId1 = 0
   private val brokerId2 = 1  
@@ -40,13 +41,13 @@
   private var server2: KafkaServer = null
   private var consumer1: SimpleConsumer = null
   private var consumer2: SimpleConsumer = null
-  private var zkServer:EmbeddedZookeeper = null
   private val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler])
+  private var zkClient: ZkClient = null
 
-  @Before
-  def setUp() {
+  override def setUp() {
+    super.setUp()
     // set up 2 brokers with 4 partitions each
-    zkServer = new EmbeddedZookeeper(TestZKUtils.zookeeperConnect)
+    zkClient = zookeeper.client
 
     val props1 = TestUtils.createBrokerConfig(brokerId1, port1)
     val config1 = new KafkaConfig(props1) {
@@ -73,8 +74,7 @@
     Thread.sleep(500)
   }
 
-  @After
-  def tearDown() {
+  override def tearDown() {
     // restore set request handler logger to a higher level
     requestHandlerLogger.setLevel(Level.ERROR)
     server1.shutdown
@@ -82,38 +82,43 @@
     Utils.rm(server1.config.logDir)
     Utils.rm(server2.config.logDir)    
     Thread.sleep(500)
-    zkServer.shutdown
-    Thread.sleep(500)
+    super.tearDown()
   }
 
   @Test
   def testZKSendToNewTopic() {
     val props = new Properties()
     props.put("serializer.class", "kafka.serializer.StringEncoder")
-    props.put("partitioner.class", "kafka.producer.StaticPartitioner")
+    props.put("partitioner.class", "kafka.utils.StaticPartitioner")
     props.put("zk.connect", TestZKUtils.zookeeperConnect)
 
     val config = new ProducerConfig(props)
 
     val producer = new Producer[String, String](config)
     try {
-      // Available broker id, partition id at this stage should be (0,0), (1,0)
-      // this should send the message to broker 0 on partition 0
+      // Available partition ids should be 0, 1, 2 and 3. The data in both cases should get sent to partition 0, but
+      // since partition 0 can exist on any of the two brokers, we need to fetch from both brokers
       producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1")))
       Thread.sleep(100)
-      // Available broker id, partition id at this stage should be (0,0), (0,1), (0,2), (0,3), (1,0)
-      // Since 4 % 5 = 4, this should send the message to broker 1 on partition 0
       producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1")))
-      Thread.sleep(100)
-      // cross check if brokers got the messages
+      Thread.sleep(1000)
+      // cross check if one of the brokers got the messages
       val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
-      val messageSet1 = response1.messageSet("new-topic", 0)
-      assertTrue("Message set should have 1 message", messageSet1.iterator.hasNext)
-      assertEquals(new Message("test1".getBytes), messageSet1.head.message)
+      val messageSet1 = response1.messageSet("new-topic", 0).iterator
       val response2 = consumer2.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
-      val messageSet2 = response2.messageSet("new-topic", 0)
-      assertTrue("Message set should have 1 message", messageSet2.iterator.hasNext)
-      assertEquals(new Message("test1".getBytes), messageSet2.head.message)
+      val messageSet2 = response2.messageSet("new-topic", 0).iterator
+      assertTrue("Message set should have 1 message", (messageSet1.hasNext || messageSet2.hasNext))
+
+      if(messageSet1.hasNext) {
+        assertEquals(new Message("test1".getBytes), messageSet1.next.message)
+        assertTrue("Message set should have 1 message", messageSet1.hasNext)
+        assertEquals(new Message("test1".getBytes), messageSet1.next.message)
+      }
+      else {
+        assertEquals(new Message("test1".getBytes), messageSet2.next.message)
+        assertTrue("Message set should have 1 message", messageSet2.hasNext)
+        assertEquals(new Message("test1".getBytes), messageSet2.next.message)
+      }
     } catch {
       case e: Exception => fail("Not expected", e)
     }
@@ -124,34 +129,40 @@
   def testZKSendWithDeadBroker() {
     val props = new Properties()
     props.put("serializer.class", "kafka.serializer.StringEncoder")
-    props.put("partitioner.class", "kafka.producer.StaticPartitioner")
+    props.put("partitioner.class", "kafka.utils.StaticPartitioner")
     props.put("zk.connect", TestZKUtils.zookeeperConnect)
 
+    // create topic
+    CreateTopicCommand.createTopic(zkClient, "new-topic", 4, 2, "0:1,0:1,0:1,0:1")
+
     val config = new ProducerConfig(props)
 
     val producer = new Producer[String, String](config)
     try {
-      // Available broker id, partition id at this stage should be (0,0), (1,0)
-      // Hence, this should send the message to broker 0 on partition 0
+      // Available partition ids should be 0, 1, 2 and 3. The data in both cases should get sent to partition 0 and
+      // all partitions have broker 0 as the leader.
       producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1")))
       Thread.sleep(100)
       // kill 2nd broker
-      server2.shutdown
+      server1.shutdown
       Thread.sleep(100)
-      // Available broker id, partition id at this stage should be (0,0), (0,1), (0,2), (0,3), (1,0)
-      // Since 4 % 5 = 4, in a normal case, it would send to broker 1 on partition 0. But since broker 1 is down,
-      // 4 % 4 = 0, So it should send the message to broker 0 on partition 0
+
+      // Since all partitions are unavailable, this request will be dropped
       producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1")))
       Thread.sleep(100)
+
+      // restart server 1
+      server1.startup()
+      Thread.sleep(100)
+
       // cross check if brokers got the messages
       val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
-      val messageSet1Iter = response1.messageSet("new-topic", 0).iterator
-      assertTrue("Message set should have 1 message", messageSet1Iter.hasNext)
-      assertEquals(new Message("test1".getBytes), messageSet1Iter.next.message)
-      assertTrue("Message set should have another message", messageSet1Iter.hasNext)
-      assertEquals(new Message("test1".getBytes), messageSet1Iter.next.message)
+      val messageSet1 = response1.messageSet("new-topic", 0).iterator
+      assertTrue("Message set should have 1 message", messageSet1.hasNext)
+      assertEquals(new Message("test1".getBytes), messageSet1.next.message)
+      assertFalse("Message set should have another message", messageSet1.hasNext)
     } catch {
-      case e: Exception => fail("Not expected")
+      case e: Exception => fail("Not expected", e)
     }
     producer.close
   }
@@ -160,7 +171,7 @@
   def testZKSendToExistingTopicWithNoBrokers() {
     val props = new Properties()
     props.put("serializer.class", "kafka.serializer.StringEncoder")
-    props.put("partitioner.class", "kafka.producer.StaticPartitioner")
+    props.put("partitioner.class", "kafka.utils.StaticPartitioner")
     props.put("zk.connect", TestZKUtils.zookeeperConnect)
 
     val config = new ProducerConfig(props)
@@ -168,19 +179,19 @@
     val producer = new Producer[String, String](config)
     var server: KafkaServer = null
 
+    // create topic
+    CreateTopicCommand.createTopic(zkClient, "new-topic", 4, 2, "0:1,0:1,0:1,0:1")
+
     try {
-      // shutdown server1
-      server1.shutdown
-      Thread.sleep(100)
-      // Available broker id, partition id at this stage should be (1,0)
-      // this should send the message to broker 1 on partition 0
+      // Available partition ids should be 0, 1, 2 and 3. The data in both cases should get sent to partition 0 and
+      // all partitions have broker 0 as the leader.
       producer.send(new ProducerData[String, String]("new-topic", "test", Array("test")))
       Thread.sleep(100)
       // cross check if brokers got the messages
-      val response1 = consumer2.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
-      val messageSet1 = response1.messageSet("new-topic", 0)
-      assertTrue("Message set should have 1 message", messageSet1.iterator.hasNext)
-      assertEquals(new Message("test".getBytes), messageSet1.head.message)
+      val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
+      val messageSet1 = response1.messageSet("new-topic", 0).iterator
+      assertTrue("Message set should have 1 message", messageSet1.hasNext)
+      assertEquals(new Message("test".getBytes), messageSet1.next.message)
 
       // shutdown server2
       server2.shutdown
@@ -189,7 +200,7 @@
       Utils.rm(server2.config.logDir)
       Thread.sleep(100)
       // start it up again. So broker 2 exists under /broker/ids, but nothing exists under /broker/topics/new-topic
-      val props2 = TestUtils.createBrokerConfig(brokerId1, port1)
+      val props2 = TestUtils.createBrokerConfig(brokerId2, port2)
       val config2 = new KafkaConfig(props2) {
         override val numPartitions = 4
       }
@@ -202,9 +213,9 @@
 
       // cross check if brokers got the messages
       val response2 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
-      val messageSet2 = response2.messageSet("new-topic", 0)
-      assertTrue("Message set should have 1 message", messageSet2.iterator.hasNext)
-      assertEquals(new Message("test".getBytes), messageSet2.head.message)
+      val messageSet2 = response1.messageSet("new-topic", 0).iterator
+      assertTrue("Message set should have 1 message", messageSet2.hasNext)
+      assertEquals(new Message("test".getBytes), messageSet2.next.message)
 
     } catch {
       case e: Exception => fail("Not expected", e)
@@ -213,29 +224,5 @@
       producer.close
     }
   }
-
 }
 
-class StringSerializer extends Encoder[String] {
-  def toEvent(message: Message):String = message.toString
-  def toMessage(event: String):Message = new Message(event.getBytes)
-  def getTopic(event: String): String = event.concat("-topic")
-}
-
-class NegativePartitioner extends Partitioner[String] {
-  def partition(data: String, numPartitions: Int): Int = {
-    -1
-  }
-}
-
-class StaticPartitioner extends Partitioner[String] {
-  def partition(data: String, numPartitions: Int): Int = {
-    (data.length % numPartitions)
-  }
-}
-
-class HashPartitioner extends Partitioner[String] {
-  def partition(data: String, numPartitions: Int): Int = {
-    (data.hashCode % numPartitions)
-  }
-}
Index: core/src/test/scala/unit/kafka/zk/ZKLoadBalanceTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/zk/ZKLoadBalanceTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/zk/ZKLoadBalanceTest.scala	(working copy)
@@ -1,126 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * 
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package kafka.zk
-
-import junit.framework.Assert._
-import java.util.Collections
-import kafka.consumer.{ConsumerConfig, ZookeeperConsumerConnector}
-import java.lang.Thread
-import org.scalatest.junit.JUnit3Suite
-import kafka.utils.{TestUtils, ZkUtils, ZKGroupTopicDirs}
-
-class ZKLoadBalanceTest extends JUnit3Suite with ZooKeeperTestHarness {
-  var dirs : ZKGroupTopicDirs = null
-  val topic = "topic1"
-  val group = "group1"
-  val firstConsumer = "consumer1"
-  val secondConsumer = "consumer2"
-
-  override def setUp() {
-    super.setUp()
-
-    dirs = new ZKGroupTopicDirs(group, topic)
-  }
-
-  def testLoadBalance() {
-    // create the first partition
-    ZkUtils.setupPartition(zookeeper.client, 400, "broker1", 1111, "topic1", 1)
-    // add the first consumer
-    val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, firstConsumer))
-    val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, false)
-    zkConsumerConnector1.createMessageStreams(Map(topic -> 1))
-
-    {
-      // check Partition Owner Registry
-      val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir)
-      val expected_1 = List( ("400-0", "group1_consumer1-0") )
-      checkSetEqual(actual_1, expected_1)
-    }
-
-    // add a second consumer
-    val consumerConfig2 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, secondConsumer))
-    val ZKConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, false)
-    ZKConsumerConnector2.createMessageStreams(Map(topic -> 1))
-    // wait a bit to make sure rebalancing logic is triggered
-    Thread.sleep(200)
-
-    {
-      // check Partition Owner Registry
-      val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir)
-      val expected_2 = List( ("400-0", "group1_consumer1-0") )
-      checkSetEqual(actual_2, expected_2)
-    }
-
-    {
-      // add a few more partitions
-      val brokers = List(
-        (200, "broker2", 1111, "topic1", 2),
-        (300, "broker3", 1111, "topic1", 2) )
-
-      for ((brokerID, host, port, topic, nParts) <- brokers)
-        ZkUtils.setupPartition(zookeeper.client, brokerID, host, port, topic, nParts)
-
-
-      // wait a bit to make sure rebalancing logic is triggered
-      Thread.sleep(1500)
-      // check Partition Owner Registry
-      val actual_3 = getZKChildrenValues(dirs.consumerOwnerDir)
-      val expected_3 = List( ("200-0", "group1_consumer1-0"),
-                             ("200-1", "group1_consumer1-0"),
-                             ("300-0", "group1_consumer1-0"),
-                             ("300-1", "group1_consumer2-0"),
-                             ("400-0", "group1_consumer2-0") )
-      checkSetEqual(actual_3, expected_3)
-    }
-
-    {
-      // now delete a partition
-      ZkUtils.deletePartition(zookeeper.client, 400, "topic1")
-
-      // wait a bit to make sure rebalancing logic is triggered
-      Thread.sleep(500)
-      // check Partition Owner Registry
-      val actual_4 = getZKChildrenValues(dirs.consumerOwnerDir)
-      val expected_4 = List( ("200-0", "group1_consumer1-0"),
-                             ("200-1", "group1_consumer1-0"),
-                             ("300-0", "group1_consumer2-0"),
-                             ("300-1", "group1_consumer2-0") )
-      checkSetEqual(actual_4, expected_4)
-    }
-
-    zkConsumerConnector1.shutdown
-    ZKConsumerConnector2.shutdown
-  }
-
-  private def getZKChildrenValues(path : String) : Seq[Tuple2[String,String]] = {
-    import scala.collection.JavaConversions
-    val children = zookeeper.client.getChildren(path)
-    Collections.sort(children)
-    val childrenAsSeq : Seq[java.lang.String] = JavaConversions.asBuffer(children)
-    childrenAsSeq.map(partition =>
-      (partition, zookeeper.client.readData(path + "/" + partition).asInstanceOf[String]))
-  }
-
-  private def checkSetEqual(actual : Seq[Tuple2[String,String]], expected : Seq[Tuple2[String,String]]) {
-    assertEquals(expected.length, actual.length)
-    for (i <- 0 until expected.length) {
-      assertEquals(expected(i)._1, actual(i)._1)
-      assertEquals(expected(i)._2, actual(i)._2)
-    }
-  }
-}
Index: core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala	(working copy)
@@ -17,16 +17,17 @@
 package kafka.server
 
 import java.io.File
-import kafka.producer.{SyncProducer, SyncProducerConfig}
 import kafka.consumer.SimpleConsumer
 import java.util.Properties
 import org.junit.Test
 import junit.framework.Assert._
-import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet}
+import kafka.message.{Message, ByteBufferMessageSet}
 import org.scalatest.junit.JUnit3Suite
 import kafka.zk.ZooKeeperTestHarness
 import kafka.utils.{TestUtils, Utils}
-import kafka.api.{FetchResponse, FetchRequestBuilder, FetchRequest}
+import kafka.producer._
+import kafka.admin.CreateTopicCommand
+import kafka.api.FetchRequestBuilder
 
 class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness {
   val port = TestUtils.choosePort
@@ -38,26 +39,20 @@
 
     val host = "localhost"
     val topic = "test"
-    val sent1 = new ByteBufferMessageSet(NoCompressionCodec, new Message("hello".getBytes()), new Message("there".getBytes()))
-    val sent2 = new ByteBufferMessageSet(NoCompressionCodec, new Message("more".getBytes()), new Message("messages".getBytes()))
+    val sent1 = List(new Message("hello".getBytes()), new Message("there".getBytes()))
+    val sent2 = List( new Message("more".getBytes()), new Message("messages".getBytes()))
 
     {
-      val producer = new SyncProducer(getProducerConfig(host,
-                                                        port,
-                                                        64*1024,
-                                                        100000,
-                                                        10000))
-      val consumer = new SimpleConsumer(host,
-                                        port,
-                                        1000000,
-                                        64*1024)
-
       val server = new KafkaServer(config)
       server.startup()
 
+      // create topic
+      CreateTopicCommand.createTopic(zookeeper.client, topic, 1, 1, "0")
+
+      val producer = new Producer[Int, Message](getProducerConfig(64*1024, 100000, 10000))
+
       // send some messages
-      producer.send(topic, sent1)
-      sent1.getBuffer.rewind
+      producer.send(new ProducerData[Int, Message](topic, 0, sent1))
 
       Thread.sleep(200)
       // do a clean shutdown
@@ -68,11 +63,7 @@
 
 
     {
-      val producer = new SyncProducer(getProducerConfig(host,
-                                                        port,
-                                                        64*1024,
-                                                        100000,
-                                                        10000))
+      val producer = new Producer[Int, Message](getProducerConfig(64*1024, 100000, 10000))
       val consumer = new SimpleConsumer(host,
                                         port,
                                         1000000,
@@ -81,18 +72,19 @@
       val server = new KafkaServer(config)
       server.startup()
 
-      // bring the server back again and read the messages
+      Thread.sleep(100)
+
       var fetchedMessage: ByteBufferMessageSet = null
       while(fetchedMessage == null || fetchedMessage.validBytes == 0) {
         val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
         fetchedMessage = fetched.messageSet(topic, 0)
       }
-      TestUtils.checkEquals(sent1.iterator, fetchedMessage.iterator)
+      TestUtils.checkEquals(sent1.iterator, fetchedMessage.map(m => m.message).iterator)
       val newOffset = fetchedMessage.validBytes
 
       // send some more messages
-      producer.send(topic, sent2)
-      sent2.getBuffer.rewind
+      println("Sending messages to topic " + topic)
+      producer.send(new ProducerData[Int, Message](topic, 0, sent2))
 
       Thread.sleep(200)
 
@@ -101,7 +93,7 @@
         val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, newOffset, 10000).build())
         fetchedMessage = fetched.messageSet(topic, 0)
       }
-      TestUtils.checkEquals(sent2.map(m => m.message).iterator, fetchedMessage.map(m => m.message).iterator)
+      TestUtils.checkEquals(sent2.iterator, fetchedMessage.map(m => m.message).iterator)
 
       server.shutdown()
       Utils.rm(server.config.logDir)
@@ -109,14 +101,14 @@
 
   }
 
-  private def getProducerConfig(host: String, port: Int, bufferSize: Int, connectTimeout: Int,
-                                reconnectInterval: Int): SyncProducerConfig = {
+  private def getProducerConfig(bufferSize: Int, connectTimeout: Int,
+                                reconnectInterval: Int): ProducerConfig = {
     val props = new Properties()
-    props.put("host", host)
-    props.put("port", port.toString)
+    props.put("zk.connect", zkConnect)
+    props.put("partitioner.class", "kafka.utils.FixedValuePartitioner")
     props.put("buffer.size", bufferSize.toString)
     props.put("connect.timeout.ms", connectTimeout.toString)
     props.put("reconnect.interval", reconnectInterval.toString)
-    new SyncProducerConfig(props)
+    new ProducerConfig(props)
   }
 }
Index: core/src/test/scala/unit/kafka/admin/AdminTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/admin/AdminTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/admin/AdminTest.scala	(working copy)
@@ -169,7 +169,8 @@
       case Some(metadata) => assertEquals(topic, metadata.topic)
         assertNotNull("partition metadata list cannot be null", metadata.partitionsMetadata)
         assertEquals("partition metadata list length should be 2", 2, metadata.partitionsMetadata.size)
-        assertNull("leader should not be assigned for now", metadata.partitionsMetadata.head.leader.getOrElse(null))
+        assertEquals("leader of partition 0 should be 0", 0, metadata.partitionsMetadata.head.leader.get.id)
+        assertEquals("leader of partition 1 should be 1", 1, metadata.partitionsMetadata.last.leader.get.id)
         val actualReplicaAssignment = metadata.partitionsMetadata.map(p => p.replicas)
         val actualReplicaList = actualReplicaAssignment.map(r => r.map(b => b.id.toString).toList).toList
         assertEquals(expectedReplicaAssignment.toList, actualReplicaList)
Index: core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala	(working copy)
@@ -22,15 +22,19 @@
 import kafka.integration.KafkaServerTestHarness
 import kafka.server._
 import scala.collection._
-import kafka.utils.{Utils, Logging}
-import kafka.utils.{TestZKUtils, TestUtils}
 import org.scalatest.junit.JUnit3Suite
 import org.apache.log4j.{Level, Logger}
 import kafka.message._
 import kafka.serializer.StringDecoder
+import kafka.admin.CreateTopicCommand
+import org.I0Itec.zkclient.ZkClient
+import kafka.utils._
+import kafka.producer.{ProducerConfig, ProducerData, Producer}
+import java.util.{Collections, Properties}
 
 class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with Logging {
 
+  var dirs : ZKGroupTopicDirs = null
   val zookeeperConnect = TestZKUtils.zookeeperConnect
   val numNodes = 2
   val numParts = 2
@@ -48,25 +52,28 @@
   val consumer3 = "consumer3"
   val nMessages = 2
 
+  override def setUp() {
+    super.setUp()
+    dirs = new ZKGroupTopicDirs(group, topic)
+  }
+
   def testBasic() {
     val requestHandlerLogger = Logger.getLogger(classOf[KafkaApis])
     requestHandlerLogger.setLevel(Level.FATAL)
 
-    var actualMessages: List[Message] = Nil
-
     // test consumer timeout logic
     val consumerConfig0 = new ConsumerConfig(
       TestUtils.createConsumerProperties(zkConnect, group, consumer0)) {
       override val consumerTimeoutMs = 200
     }
     val zkConsumerConnector0 = new ZookeeperConsumerConnector(consumerConfig0, true)
-    val topicMessageStreams0 = zkConsumerConnector0.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2))
+    val topicMessageStreams0 = zkConsumerConnector0.createMessageStreams(Predef.Map(topic -> 1))
 
     // no messages to consume, we should hit timeout;
     // also the iterator should support re-entrant, so loop it twice
-    for (i <- 0 until  2) {
+    for (i <- 0 until 2) {
       try {
-        getMessagesSortedByChecksum(nMessages*2, topicMessageStreams0)
+        getMessages(nMessages*2, topicMessageStreams0)
         fail("should get an exception")
       }
       catch {
@@ -78,14 +85,26 @@
     zkConsumerConnector0.shutdown
 
     // send some messages to each broker
-    val sentMessages1 = sendMessages(nMessages, "batch1")
+    val sentMessages1_1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages)
+    val sentMessages1_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages)
+    val sentMessages1 = (sentMessages1_1 ++ sentMessages1_2).sortWith((s,t) => s.checksum < t.checksum)
+
     // create a consumer
     val consumerConfig1 = new ConsumerConfig(
       TestUtils.createConsumerProperties(zkConnect, group, consumer1))
     val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true)
-    val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2))
-    val receivedMessages1 = getMessagesSortedByChecksum(nMessages*2, topicMessageStreams1)
+    val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> 1))
+    val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1)
+    assertEquals(sentMessages1.size, receivedMessages1.size)
     assertEquals(sentMessages1, receivedMessages1)
+
+    // also check partition ownership
+    val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir)
+    val expected_1 = List( ("0", "group1_consumer1-0"),
+                           ("1", "group1_consumer1-0"))
+//   assertEquals(expected_1, actual_1)
+    assertEquals(expected_1, actual_1)
+
     // commit consumed offsets
     zkConsumerConnector1.commitOffsets
 
@@ -93,15 +112,25 @@
     val consumerConfig2 = new ConsumerConfig(
       TestUtils.createConsumerProperties(zkConnect, group, consumer2))
     val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true)
-    val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2))
+    val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Predef.Map(topic -> 1))
     // send some messages to each broker
-    val sentMessages2 = sendMessages(nMessages, "batch2")
+    val sentMessages2_1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages)
+    val sentMessages2_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages)
+    val sentMessages2 = (sentMessages2_1 ++ sentMessages2_2).sortWith((s,t) => s.checksum < t.checksum)
+
     Thread.sleep(200)
-    val receivedMessages2_1 = getMessagesSortedByChecksum(nMessages, topicMessageStreams1)
-    val receivedMessages2_2 = getMessagesSortedByChecksum(nMessages, topicMessageStreams2)
+
+    val receivedMessages2_1 = getMessages(nMessages, topicMessageStreams1)
+    val receivedMessages2_2 = getMessages(nMessages, topicMessageStreams2)
     val receivedMessages2 = (receivedMessages2_1 ::: receivedMessages2_2).sortWith((s,t) => s.checksum < t.checksum)
     assertEquals(sentMessages2, receivedMessages2)
 
+    // also check partition ownership
+    val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir)
+    val expected_2 = List( ("0", "group1_consumer1-0"),
+                           ("1", "group1_consumer2-0"))
+   assertEquals(expected_2, actual_2)
+
     // create a consumer with empty map
     val consumerConfig3 = new ConsumerConfig(
       TestUtils.createConsumerProperties(zkConnect, group, consumer3))
@@ -109,13 +138,20 @@
     val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]())
     // send some messages to each broker
     Thread.sleep(200)
-    val sentMessages3 = sendMessages(nMessages, "batch3")
+    val sentMessages3_1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages)
+    val sentMessages3_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages)
+    val sentMessages3 = (sentMessages3_1 ++ sentMessages3_2).sortWith((s,t) => s.checksum < t.checksum)
     Thread.sleep(200)
-    val receivedMessages3_1 = getMessagesSortedByChecksum(nMessages, topicMessageStreams1)
-    val receivedMessages3_2 = getMessagesSortedByChecksum(nMessages, topicMessageStreams2)
+    val receivedMessages3_1 = getMessages(nMessages, topicMessageStreams1)
+    val receivedMessages3_2 = getMessages(nMessages, topicMessageStreams2)
     val receivedMessages3 = (receivedMessages3_1 ::: receivedMessages3_2).sortWith((s,t) => s.checksum < t.checksum)
+    assertEquals(sentMessages3.size, receivedMessages3.size)
     assertEquals(sentMessages3, receivedMessages3)
 
+    // also check partition ownership
+    val actual_3 = getZKChildrenValues(dirs.consumerOwnerDir)
+   assertEquals(expected_2, actual_3)
+
     zkConsumerConnector1.shutdown
     zkConsumerConnector2.shutdown
     zkConsumerConnector3.shutdown
@@ -127,48 +163,73 @@
     val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandler])
     requestHandlerLogger.setLevel(Level.FATAL)
 
-    println("Sending messages for 1st consumer")
     // send some messages to each broker
-    val sentMessages1 = sendMessages(nMessages, "batch1", DefaultCompressionCodec)
+    val sentMessages1_1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, GZIPCompressionCodec)
+    val sentMessages1_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec)
+    val sentMessages1 = (sentMessages1_1 ++ sentMessages1_2).sortWith((s,t) => s.checksum < t.checksum)
+
     // create a consumer
     val consumerConfig1 = new ConsumerConfig(
       TestUtils.createConsumerProperties(zkConnect, group, consumer1))
     val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true)
-    val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2))
-    val receivedMessages1 = getMessagesSortedByChecksum(nMessages*2, topicMessageStreams1)
+    val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> 1))
+    val receivedMessages1 = getMessages(nMessages*2, topicMessageStreams1)
+    assertEquals(sentMessages1.size, receivedMessages1.size)
     assertEquals(sentMessages1, receivedMessages1)
+
+    // also check partition ownership
+    val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir)
+    val expected_1 = List( ("0", "group1_consumer1-0"),
+                           ("1", "group1_consumer1-0"))
+   assertEquals(expected_1, actual_1)
+
     // commit consumed offsets
     zkConsumerConnector1.commitOffsets
 
-    println("Sending more messages for 2nd consumer")
     // create a consumer
     val consumerConfig2 = new ConsumerConfig(
       TestUtils.createConsumerProperties(zkConnect, group, consumer2))
     val zkConsumerConnector2 = new ZookeeperConsumerConnector(consumerConfig2, true)
-    val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Predef.Map(topic -> numNodes*numParts/2))
+    val topicMessageStreams2 = zkConsumerConnector2.createMessageStreams(Predef.Map(topic -> 1))
     // send some messages to each broker
-    val sentMessages2 = sendMessages(nMessages, "batch2", DefaultCompressionCodec)
+    val sentMessages2_1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, GZIPCompressionCodec)
+    val sentMessages2_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec)
+    val sentMessages2 = (sentMessages2_1 ++ sentMessages2_2).sortWith((s,t) => s.checksum < t.checksum)
+
     Thread.sleep(200)
-    val receivedMessages2_1 = getMessagesSortedByChecksum(nMessages, topicMessageStreams1)
-    val receivedMessages2_2 = getMessagesSortedByChecksum(nMessages, topicMessageStreams2)
+
+    val receivedMessages2_1 = getMessages(nMessages, topicMessageStreams1)
+    val receivedMessages2_2 = getMessages(nMessages, topicMessageStreams2)
     val receivedMessages2 = (receivedMessages2_1 ::: receivedMessages2_2).sortWith((s,t) => s.checksum < t.checksum)
     assertEquals(sentMessages2, receivedMessages2)
 
+    // also check partition ownership
+    val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir)
+    val expected_2 = List( ("0", "group1_consumer1-0"),
+                           ("1", "group1_consumer2-0"))
+   assertEquals(expected_2, actual_2)
+
     // create a consumer with empty map
-    println("Sending more messages for 3rd consumer")
     val consumerConfig3 = new ConsumerConfig(
       TestUtils.createConsumerProperties(zkConnect, group, consumer3))
     val zkConsumerConnector3 = new ZookeeperConsumerConnector(consumerConfig3, true)
     val topicMessageStreams3 = zkConsumerConnector3.createMessageStreams(new mutable.HashMap[String, Int]())
     // send some messages to each broker
     Thread.sleep(200)
-    val sentMessages3 = sendMessages(nMessages, "batch3", DefaultCompressionCodec)
+    val sentMessages3_1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, GZIPCompressionCodec)
+    val sentMessages3_2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, GZIPCompressionCodec)
+    val sentMessages3 = (sentMessages3_1 ++ sentMessages3_2).sortWith((s,t) => s.checksum < t.checksum)
     Thread.sleep(200)
-    val receivedMessages3_1 = getMessagesSortedByChecksum(nMessages, topicMessageStreams1)
-    val receivedMessages3_2 = getMessagesSortedByChecksum(nMessages, topicMessageStreams2)
+    val receivedMessages3_1 = getMessages(nMessages, topicMessageStreams1)
+    val receivedMessages3_2 = getMessages(nMessages, topicMessageStreams2)
     val receivedMessages3 = (receivedMessages3_1 ::: receivedMessages3_2).sortWith((s,t) => s.checksum < t.checksum)
+    assertEquals(sentMessages3.size, receivedMessages3.size)
     assertEquals(sentMessages3, receivedMessages3)
 
+    // also check partition ownership
+    val actual_3 = getZKChildrenValues(dirs.consumerOwnerDir)
+   assertEquals(expected_2, actual_3)
+
     zkConsumerConnector1.shutdown
     zkConsumerConnector2.shutdown
     zkConsumerConnector3.shutdown
@@ -187,7 +248,10 @@
     Thread.sleep(500)
 
     // send some messages to each broker
-    val sentMessages = sendMessages(configs.head, 200, "batch1", DefaultCompressionCodec)
+    val sentMessages1 = sendMessagesToBrokerPartition(configs.head, topic, 0, 200, DefaultCompressionCodec)
+    val sentMessages2 = sendMessagesToBrokerPartition(configs.last, topic, 1, 200, DefaultCompressionCodec)
+    val sentMessages = (sentMessages1 ++ sentMessages2).sortWith((s,t) => s.checksum < t.checksum)
+
     // test consumer timeout logic
     val consumerConfig0 = new ConsumerConfig(
       TestUtils.createConsumerProperties(zkConnect, group, consumer0)) {
@@ -195,16 +259,30 @@
     }
     val zkConsumerConnector0 = new ZookeeperConsumerConnector(consumerConfig0, true)
     val topicMessageStreams0 = zkConsumerConnector0.createMessageStreams(Predef.Map(topic -> 1))
-    getMessagesSortedByChecksum(100, topicMessageStreams0)
+    getMessages(100, topicMessageStreams0)
+
+    // also check partition ownership
+    val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir)
+    val expected_1 = List( ("0", "group1_consumer0-0"),
+                           ("1", "group1_consumer0-0"))
+   assertEquals(expected_1, actual_1)
+
     zkConsumerConnector0.shutdown
     // at this point, only some part of the message set was consumed. So consumed offset should still be 0
     // also fetched offset should be 0
     val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig0, true)
     val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> 1))
-    val receivedMessages = getMessagesSortedByChecksum(400, topicMessageStreams1)
+    val receivedMessages = getMessages(400, topicMessageStreams1)
     val sortedReceivedMessages = receivedMessages.sortWith((s,t) => s.checksum < t.checksum)
     val sortedSentMessages = sentMessages.sortWith((s,t) => s.checksum < t.checksum)
     assertEquals(sortedSentMessages, sortedReceivedMessages)
+
+    // also check partition ownership
+    val actual_2 = getZKChildrenValues(dirs.consumerOwnerDir)
+    val expected_2 = List( ("0", "group1_consumer0-0"),
+                           ("1", "group1_consumer0-0"))
+   assertEquals(expected_2, actual_2)
+
     zkConsumerConnector1.shutdown
 
     requestHandlerLogger.setLevel(Level.ERROR)
@@ -214,17 +292,18 @@
     val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandler])
     requestHandlerLogger.setLevel(Level.FATAL)
 
-    val sentMessages = sendMessages(nMessages, "batch1", NoCompressionCodec).
-      map(m => Utils.toString(m.payload, "UTF-8")).
+    // send some messages to each broker
+    val sentMessages1 = sendMessagesToBrokerPartition(configs.head, topic, 0, nMessages, NoCompressionCodec)
+    val sentMessages2 = sendMessagesToBrokerPartition(configs.last, topic, 1, nMessages, NoCompressionCodec)
+    val sentMessages = (sentMessages1 ++ sentMessages2).map(m => Utils.toString(m.payload, "UTF-8")).
       sortWith((s, t) => s.compare(t) == -1)
-    val consumerConfig = new ConsumerConfig(
-      TestUtils.createConsumerProperties(zkConnect, group, consumer1))
 
+    val consumerConfig = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumer1))
+
     val zkConsumerConnector =
       new ZookeeperConsumerConnector(consumerConfig, true)
     val topicMessageStreams =
-      zkConsumerConnector.createMessageStreams(
-        Predef.Map(topic -> numNodes*numParts/2), new StringDecoder)
+      zkConsumerConnector.createMessageStreams(Predef.Map(topic -> 1), new StringDecoder)
 
     var receivedMessages: List[String] = Nil
     for ((topic, messageStreams) <- topicMessageStreams) {
@@ -245,31 +324,106 @@
     requestHandlerLogger.setLevel(Level.ERROR)
   }
 
-  def sendMessages(conf: KafkaConfig, messagesPerNode: Int, header: String, compression: CompressionCodec): List[Message]= {
+  def testLeaderSelectionForPartition() {
+    val zkClient = new ZkClient(zookeeperConnect, 6000, 30000, ZKStringSerializer)
+
+    // create topic topic1 with 1 partition on broker 0
+    CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "0")
+
+    // send some messages to each broker
+    val sentMessages1 = sendMessages(configs.head, nMessages, "batch1", NoCompressionCodec, 1)
+
+    // create a consumer
+    val consumerConfig1 = new ConsumerConfig(
+      TestUtils.createConsumerProperties(zkConnect, group, consumer1))
+    val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true)
+    val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> 1))
+    val topicRegistry = zkConsumerConnector1.getTopicRegistry
+    assertEquals(1, topicRegistry.map(r => r._1).size)
+    assertEquals(topic, topicRegistry.map(r => r._1).head)
+    val topicsAndPartitionsInRegistry = topicRegistry.map(r => (r._1, r._2.map(p => p._1)))
+    val brokerPartition = topicsAndPartitionsInRegistry.head._2.head
+    assertEquals(0, brokerPartition.brokerId)
+    assertEquals(0, brokerPartition.partId)
+
+    // also check partition ownership
+    val actual_1 = getZKChildrenValues(dirs.consumerOwnerDir)
+    val expected_1 = List( ("0", "group1_consumer1-0"))
+   assertEquals(expected_1, actual_1)
+
+    val receivedMessages1 = getMessages(nMessages, topicMessageStreams1)
+    assertEquals(nMessages, receivedMessages1.size)
+    assertEquals(sentMessages1, receivedMessages1)
+  }
+
+  def sendMessagesToBrokerPartition(config: KafkaConfig, topic: String, partition: Int, numMessages: Int,
+                                    compression: CompressionCodec = NoCompressionCodec): List[Message] = {
+    val header = "test-%d-%d".format(config.brokerId, partition)
+    val props = new Properties()
+    props.put("zk.connect", zkConnect)
+    props.put("partitioner.class", "kafka.utils.FixedValuePartitioner")
+    props.put("compression.codec", compression.codec.toString)
+    val producer: Producer[Int, Message] = new Producer[Int, Message](new ProducerConfig(props))
+    val ms = 0.until(numMessages).map(x =>
+      new Message((header + config.brokerId + "-" + partition + "-" + x).getBytes)).toArray
+    producer.send(new ProducerData[Int, Message](topic, partition, ms))
+    debug("Sent %d messages to broker %d for topic %s and partition %d".format(ms.size, config.brokerId, topic, partition))
+    producer
+    ms.toList
+  }
+
+  def sendMessages(conf: KafkaConfig, messagesPerNode: Int, header: String, compression: CompressionCodec, numParts: Int): List[Message]= {
     var messages: List[Message] = Nil
-    val producer = TestUtils.createProducer("localhost", conf.port)
+    val props = new Properties()
+    props.put("zk.connect", zkConnect)
+    props.put("partitioner.class", "kafka.utils.FixedValuePartitioner")
+    val producer: Producer[Int, Message] = new Producer[Int, Message](new ProducerConfig(props))
+
     for (partition <- 0 until numParts) {
       val ms = 0.until(messagesPerNode).map(x =>
         new Message((header + conf.brokerId + "-" + partition + "-" + x).getBytes)).toArray
-      val mSet = new ByteBufferMessageSet(compressionCodec = compression, messages = ms: _*)
       for (message <- ms)
         messages ::= message
-      producer.send(topic, partition, mSet)
+      producer.send(new ProducerData[Int, Message](topic, partition, ms))
+      debug("Sent %d messages to broker %d for topic %s and partition %d".format(ms.size, conf.brokerId, topic, partition))
     }
     producer.close()
-    messages
+    messages.reverse
   }
 
   def sendMessages(messagesPerNode: Int, header: String, compression: CompressionCodec = NoCompressionCodec): List[Message]= {
     var messages: List[Message] = Nil
     for(conf <- configs) {
-      messages ++= sendMessages(conf, messagesPerNode, header, compression)
+      messages ++= sendMessages(conf, messagesPerNode, header, compression, numParts)
     }
     messages.sortWith((s,t) => s.checksum < t.checksum)
   }
 
-  def getMessagesSortedByChecksum(nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaMessageStream[Message]]]): List[Message]= {
-    val messages = TestUtils.getConsumedMessages(nMessagesPerThread, topicMessageStreams)
+  def getMessages(nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaMessageStream[Message]]]): List[Message]= {
+    var messages: List[Message] = Nil
+    for ((topic, messageStreams) <- topicMessageStreams) {
+      for (messageStream <- messageStreams) {
+        val iterator = messageStream.iterator
+        for (i <- 0 until nMessagesPerThread) {
+          assertTrue(iterator.hasNext)
+          val message = iterator.next
+          messages ::= message
+          debug("received message: " + Utils.toString(message.payload, "UTF-8"))
+        }
+      }
+    }
     messages.sortWith((s,t) => s.checksum < t.checksum)
   }
+
+  def getZKChildrenValues(path : String) : Seq[Tuple2[String,String]] = {
+    import scala.collection.JavaConversions
+    val children = zookeeper.client.getChildren(path)
+    Collections.sort(children)
+    val childrenAsSeq : Seq[java.lang.String] = JavaConversions.asBuffer(children)
+    childrenAsSeq.map(partition =>
+      (partition, zookeeper.client.readData(path + "/" + partition).asInstanceOf[String]))
+  }
+
 }
+
+
Index: core/src/test/scala/unit/kafka/javaapi/integration/PrimitiveApiTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/javaapi/integration/PrimitiveApiTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/javaapi/integration/PrimitiveApiTest.scala	(working copy)
@@ -1,415 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- * 
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package kafka.javaapi.integration
-
-import scala.collection._
-import kafka.api.FetchRequestBuilder
-import kafka.common.{InvalidPartitionException, OffsetOutOfRangeException}
-import kafka.javaapi.ProducerRequest
-import kafka.javaapi.message.ByteBufferMessageSet
-import kafka.message.{DefaultCompressionCodec, NoCompressionCodec, Message}
-import kafka.server.{KafkaRequestHandler, KafkaConfig}
-import kafka.utils.TestUtils
-import org.apache.log4j.{Level, Logger}
-import org.scalatest.junit.JUnit3Suite
-
-/**
- * End to end tests of the primitive apis against a local server
- */
-class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with kafka.integration.KafkaServerTestHarness {
-  
-  val port = 9999
-  val props = TestUtils.createBrokerConfig(0, port)
-  val config = new KafkaConfig(props)
-  val configs = List(config)
-  val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler])
-
-  def testProduceAndFetch() {
-    // send some messages
-    val topic = "test"
-
-    // send an empty messageset first
-    val sent2 = new ByteBufferMessageSet(NoCompressionCodec, getMessageList(Seq.empty[Message]: _*))
-    producer.send(topic, sent2)
-
-    Thread.sleep(200)
-    sent2.getBuffer.rewind
-    val fetched2 = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
-    val fetchedMessage2 = fetched2.messageSet(topic, 0)
-    TestUtils.checkEquals(sent2.iterator, fetchedMessage2.iterator)
-
-
-    // send some messages
-    val sent3 = new ByteBufferMessageSet(NoCompressionCodec,
-                                         getMessageList(
-                                           new Message("hello".getBytes()),new Message("there".getBytes())))
-    producer.send(topic, sent3)
-
-    Thread.sleep(200)
-    sent3.getBuffer.rewind
-    var messageSet: ByteBufferMessageSet = null
-    while(messageSet == null || messageSet.validBytes == 0) {
-      val fetched3 = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
-      messageSet = fetched3.messageSet(topic, 0).asInstanceOf[ByteBufferMessageSet]
-    }
-    TestUtils.checkEquals(sent3.iterator, messageSet.iterator)
-
-    // temporarily set request handler logger to a higher level
-    requestHandlerLogger.setLevel(Level.FATAL)
-
-    // send an invalid offset
-    try {
-      val fetchedWithError = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, -1, 10000).build())
-      val messageWithError = fetchedWithError.messageSet(topic, 0)
-      messageWithError.iterator
-      fail("Fetch with invalid offset should throw an exception when iterating over response")
-    } catch {
-      case e: OffsetOutOfRangeException => "this is good"
-    }
-
-    // restore set request handler logger to a higher level
-    requestHandlerLogger.setLevel(Level.ERROR)
-  }
-
-  def testProduceAndFetchWithCompression() {
-    // send some messages
-    val topic = "test"
-
-    // send an empty messageset first
-    val sent2 = new ByteBufferMessageSet(DefaultCompressionCodec, getMessageList(Seq.empty[Message]: _*))
-    producer.send(topic, sent2)
-
-    Thread.sleep(200)
-    sent2.getBuffer.rewind
-    val fetched2 = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
-    val message2 = fetched2.messageSet(topic, 0)
-    TestUtils.checkEquals(sent2.iterator, message2.iterator)
-
-
-    // send some messages
-    val sent3 = new ByteBufferMessageSet( DefaultCompressionCodec,
-                                          getMessageList(
-                                            new Message("hello".getBytes()),new Message("there".getBytes())))
-    producer.send(topic, sent3)
-
-    Thread.sleep(200)
-    sent3.getBuffer.rewind
-    var fetchedMessage: ByteBufferMessageSet = null
-    while(fetchedMessage == null || fetchedMessage.validBytes == 0) {
-      val fetched3 = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
-      fetchedMessage = fetched3.messageSet(topic, 0).asInstanceOf[ByteBufferMessageSet]
-    }
-    TestUtils.checkEquals(sent3.iterator, fetchedMessage.iterator)
-
-    // temporarily set request handler logger to a higher level
-    requestHandlerLogger.setLevel(Level.FATAL)
-
-    // send an invalid offset
-    try {
-      val fetchedWithError = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, -1, 10000).build())
-      val messageWithError = fetchedWithError.messageSet(topic, 0)
-      messageWithError.iterator
-      fail("Fetch with invalid offset should throw an exception when iterating over response")
-    } catch {
-      case e: OffsetOutOfRangeException => "this is good"
-    }
-
-    // restore set request handler logger to a higher level
-    requestHandlerLogger.setLevel(Level.ERROR)
-  }
-
-  def testProduceAndMultiFetch() {
-    // send some messages
-    val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
-    {
-      val messages = new mutable.HashMap[String, ByteBufferMessageSet]
-      val builder = new FetchRequestBuilder()
-      for( (topic, partition) <- topics) {
-        val set = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec,
-                                           messages = getMessageList(new Message(("a_" + topic).getBytes),
-                                                                     new Message(("b_" + topic).getBytes)))
-        messages += topic -> set
-        producer.send(topic, set)
-        set.getBuffer.rewind
-        builder.addFetch(topic, partition, 0, 10000)
-      }
-
-      // wait a bit for produced message to be available
-      Thread.sleep(200)
-      val request = builder.build()
-      val response = consumer.fetch(request)
-      for( (topic, partition) <- topics) {
-        val messageSet = response.messageSet(topic, partition)
-        TestUtils.checkEquals(messages(topic).iterator, messageSet.iterator)
-      }
-    }
-
-    // temporarily set request handler logger to a higher level
-    requestHandlerLogger.setLevel(Level.FATAL)
-
-    {
-      // send some invalid offsets
-      val builder = new FetchRequestBuilder()
-      for( (topic, partition) <- topics)
-         builder.addFetch(topic, partition, -1, 10000)
-
-      val request = builder.build()
-      val response = consumer.fetch(request)
-      for( (topic, partition) <- topics) {
-        try {
-            val iter = response.messageSet(topic, partition).iterator
-            while (iter.hasNext)
-              iter.next
-            fail("MessageSet for invalid offset should throw exception")
-        } catch {
-          case e: OffsetOutOfRangeException => "this is good"
-        }
-      }
-    }    
-
-    {
-      // send some invalid partitions
-      val builder = new FetchRequestBuilder()
-      for( (topic, _) <- topics)
-        builder.addFetch(topic, -1, 0, 10000)
-
-      val request = builder.build()
-      val response = consumer.fetch(request)
-      for( (topic, _) <- topics) {
-        try {
-          val iter = response.messageSet(topic, -1).iterator
-          while (iter.hasNext)
-            iter.next
-          fail("MessageSet for invalid partition should throw exception")
-        } catch {
-          case e: InvalidPartitionException => "this is good"
-        }
-      }
-    }
-
-    // restore set request handler logger to a higher level
-    requestHandlerLogger.setLevel(Level.ERROR)
-  }
-
-  def testProduceAndMultiFetchWithCompression() {
-    // send some messages
-    val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
-    {
-      val messages = new mutable.HashMap[String, ByteBufferMessageSet]
-      val builder = new FetchRequestBuilder()
-      for( (topic, partition) <- topics) {
-        val set = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec,
-                                           messages = getMessageList(new Message(("a_" + topic).getBytes),
-                                                                     new Message(("b_" + topic).getBytes)))
-        messages += topic -> set
-        producer.send(topic, set)
-        set.getBuffer.rewind
-        builder.addFetch(topic, partition, 0, 10000)
-      }
-
-      // wait a bit for produced message to be available
-      Thread.sleep(200)
-      val request = builder.build()
-      val response = consumer.fetch(request)
-      for( (topic, partition) <- topics) {
-        val iter = response.messageSet(topic, partition).iterator
-        if (iter.hasNext) {
-          TestUtils.checkEquals(messages(topic).iterator, iter)
-        } else {
-          fail("fewer responses than expected")
-        }
-      }
-    }
-
-    // temporarily set request handler logger to a higher level
-    requestHandlerLogger.setLevel(Level.FATAL)
-
-    {
-      // send some invalid offsets
-      val builder = new FetchRequestBuilder()
-      for( (topic, partition) <- topics)
-        builder.addFetch(topic, partition, -1, 10000)
-
-      val request = builder.build()
-      val response = consumer.fetch(request)
-      for( (topic, partition) <- topics) {
-        try {
-          val iter = response.messageSet(topic, partition).iterator
-          while (iter.hasNext)
-            iter.next
-          fail("Expected exception when fetching invalid offset")
-        } catch {
-          case e: OffsetOutOfRangeException => "this is good"
-        }
-      }
-    }
-
-    {
-      // send some invalid partitions
-      val builder = new FetchRequestBuilder()
-      for( (topic, _) <- topics)
-        builder.addFetch(topic, -1, 0, 10000)
-
-      val request = builder.build()
-      val response = consumer.fetch(request)
-      for( (topic, _) <- topics) {
-        try {
-          val iter = response.messageSet(topic, -1).iterator
-          while (iter.hasNext)
-            iter.next
-          fail("Expected exception when fetching invalid partition")
-        } catch {
-          case e: InvalidPartitionException => "this is good"
-        }
-      }
-    }
-
-    // restore set request handler logger to a higher level
-    requestHandlerLogger.setLevel(Level.ERROR)
-  }
-
-  def testProduceAndMultiFetchJava() {
-    // send some messages
-    val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
-    {
-      val messages = new mutable.HashMap[String, ByteBufferMessageSet]
-      val builder = new FetchRequestBuilder()
-      for( (topic, partition) <- topics) {
-        val set = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec,
-                                           messages = getMessageList(new Message(("a_" + topic).getBytes),
-                                                                     new Message(("b_" + topic).getBytes)))
-        messages += topic -> set
-        producer.send(topic, set)
-        set.getBuffer.rewind
-        builder.addFetch(topic, partition, 0, 10000)
-      }
-
-      // wait a bit for produced message to be available
-      Thread.sleep(200)
-      val request = builder.build()
-      val response = consumer.fetch(request)
-      for( (topic, partition) <- topics) {
-        val iter = response.messageSet(topic, partition).iterator
-        if (iter.hasNext) {
-          TestUtils.checkEquals(messages(topic).iterator, iter)
-        } else {
-          fail("fewer responses than expected")
-        }
-      }
-    }
-  }
-
-  def testProduceAndMultiFetchJavaWithCompression() {
-    // send some messages
-    val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
-    {
-      val messages = new mutable.HashMap[String, ByteBufferMessageSet]
-      val builder = new FetchRequestBuilder()
-      for( (topic, partition) <- topics) {
-        val set = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec,
-                                           messages = getMessageList(new Message(("a_" + topic).getBytes),
-                                                                     new Message(("b_" + topic).getBytes)))
-        messages += topic -> set
-        producer.send(topic, set)
-        set.getBuffer.rewind
-        builder.addFetch(topic, partition, 0, 10000)
-      }
-
-      // wait a bit for produced message to be available
-      Thread.sleep(200)
-      val request = builder.build()
-      val response = consumer.fetch(request)
-      for( (topic, partition) <- topics) {
-        val iter = response.messageSet(topic, partition).iterator
-        TestUtils.checkEquals(messages(topic).iterator, iter)
-      }
-    }
-  }
-
-  def testMultiProduce() {
-    // send some messages
-    val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
-    val messages = new mutable.HashMap[String, ByteBufferMessageSet]
-    val builder = new FetchRequestBuilder()
-    var produceList: List[ProducerRequest] = Nil
-    for( (topic, partition) <- topics) {
-      val set = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec,
-                                         messages = getMessageList(new Message(("a_" + topic).getBytes),
-                                                                   new Message(("b_" + topic).getBytes)))
-      messages += topic -> set
-      produceList ::= new ProducerRequest(topic, 0, set)
-      builder.addFetch(topic, partition, 0, 10000)
-    }
-    producer.multiSend(produceList.toArray)
-
-    for (messageSet <- messages.values)
-      messageSet.getBuffer.rewind
-      
-    // wait a bit for produced message to be available
-    Thread.sleep(200)
-    val request = builder.build()
-    val response = consumer.fetch(request)
-    for( (topic, partition) <- topics) {
-      val iter = response.messageSet(topic, partition).iterator
-      if (iter.hasNext) {
-        TestUtils.checkEquals(messages(topic).iterator, iter)
-      } else {
-        fail("fewer responses than expected")
-      }
-    }
-  }
-
-  def testMultiProduceWithCompression() {
-    // send some messages
-    val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
-    val messages = new mutable.HashMap[String, ByteBufferMessageSet]
-    val builder = new FetchRequestBuilder()
-    var produceList: List[ProducerRequest] = Nil
-    for( (topic, partition) <- topics) {
-      val set = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec,
-                                         messages = getMessageList(new Message(("a_" + topic).getBytes),
-                                                                   new Message(("b_" + topic).getBytes)))
-      messages += topic -> set
-      produceList ::= new ProducerRequest(topic, 0, set)
-      builder.addFetch(topic, partition, 0, 10000)
-    }
-    producer.multiSend(produceList.toArray)
-
-    for (messageSet <- messages.values)
-      messageSet.getBuffer.rewind
-
-    // wait a bit for produced message to be available
-    Thread.sleep(200)
-    val request = builder.build()
-    val response = consumer.fetch(request)
-    for( (topic, partition) <- topics) {
-      val iter = response.messageSet(topic, partition).iterator
-      if (iter.hasNext) {
-        TestUtils.checkEquals(messages(topic).iterator, iter)
-      } else {
-        fail("fewer responses than expected")
-      }
-    }
-  }
-
-  private def getMessageList(messages: Message*): java.util.List[Message] = {
-    val messageList = new java.util.ArrayList[Message]()
-    messages.foreach(m => messageList.add(m))
-    messageList
-  }
-}
Index: core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala
===================================================================
--- core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala	(revision 1294440)
+++ core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala	(working copy)
@@ -24,10 +24,10 @@
 import kafka.utils.TestUtils
 import org.scalatest.junit.JUnit3Suite
 import scala.collection.JavaConversions._
-import kafka.javaapi.message.ByteBufferMessageSet
 import kafka.consumer.{ConsumerConfig, KafkaMessageStream}
 import org.apache.log4j.{Level, Logger}
 import kafka.message._
+import kafka.javaapi.producer.{ProducerData, Producer}
 
 class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHarness with Logging {
 
@@ -67,16 +67,17 @@
 
   def sendMessages(conf: KafkaConfig, messagesPerNode: Int, header: String, compressed: CompressionCodec): List[Message]= {
     var messages: List[Message] = Nil
-    val producer = new kafka.javaapi.producer.SyncProducer(TestUtils.createProducer("localhost", conf.port))
+    val producer: kafka.producer.Producer[Int, Message] = TestUtils.createProducer(zkConnect)
+    val javaProducer: Producer[Int, Message] = new kafka.javaapi.producer.Producer(producer)
     for (partition <- 0 until numParts) {
       val ms = 0.until(messagesPerNode).map(x =>
         new Message((header + conf.brokerId + "-" + partition + "-" + x).getBytes)).toArray
-      val mSet = new ByteBufferMessageSet(compressionCodec = compressed, messages = getMessageList(ms: _*))
       for (message <- ms)
         messages ::= message
-      producer.send(topic, partition, mSet)
+      import scala.collection.JavaConversions._
+      javaProducer.send(new ProducerData[Int, Message](topic, partition, asList(ms)))
     }
-    producer.close()
+    javaProducer.close
     messages
   }
 
@@ -106,12 +107,6 @@
     messages.sortWith((s,t) => s.checksum < t.checksum)
   }
 
-  private def getMessageList(messages: Message*): java.util.List[Message] = {
-    val messageList = new java.util.ArrayList[Message]()
-    messages.foreach(m => messageList.add(m))
-    messageList
-  }
-
   private def toJavaMap(scalaMap: Map[String, Int]): java.util.Map[String, java.lang.Integer] = {
     val javaMap = new java.util.HashMap[String, java.lang.Integer]()
     scalaMap.foreach(m => javaMap.put(m._1, m._2.asInstanceOf[java.lang.Integer]))
Index: core/src/main/scala/kafka/cluster/Partition.scala
===================================================================
--- core/src/main/scala/kafka/cluster/Partition.scala	(revision 1294440)
+++ core/src/main/scala/kafka/cluster/Partition.scala	(working copy)
@@ -17,41 +17,14 @@
 
 package kafka.cluster
 
-object Partition {
-  def parse(s: String): Partition = {
-    val pieces = s.split("-")
-    if(pieces.length != 2)
-      throw new IllegalArgumentException("Expected name in the form x-y.")
-    new Partition(pieces(0).toInt, pieces(1).toInt)
-  }
-}
+case class Partition(val brokerId: Int, val partId: Int, val topic: String = "") extends Ordered[Partition] {
 
-class Partition(val brokerId: Int, val partId: Int) extends Ordered[Partition] {
+  def name = partId
 
-  def this(name: String) = {
-    this(1, 1)
-  }
-  
-  def name = brokerId + "-" + partId
-  
-  override def toString(): String = name
-
   def compare(that: Partition) =
-    if (this.brokerId == that.brokerId)
+    if (this.topic == that.topic)
       this.partId - that.partId
     else
-      this.brokerId - that.brokerId
+      this.topic.compareTo(that.topic)
 
-  override def equals(other: Any): Boolean = {
-    other match {
-      case that: Partition =>
-        (that canEqual this) && brokerId == that.brokerId && partId == that.partId
-      case _ => false
-    }
-  }
-
-  def canEqual(other: Any): Boolean = other.isInstanceOf[Partition]
-
-  override def hashCode: Int = 31 * (17 + brokerId) + partId
-
 }
Index: core/src/main/scala/kafka/log/LogManager.scala
===================================================================
--- core/src/main/scala/kafka/log/LogManager.scala	(revision 1294440)
+++ core/src/main/scala/kafka/log/LogManager.scala	(working copy)
@@ -19,7 +19,6 @@
 
 import java.io._
 import kafka.utils._
-import scala.actors.Actor
 import scala.collection._
 import java.util.concurrent.CountDownLatch
 import kafka.server.{KafkaConfig, KafkaZooKeeper}
@@ -32,7 +31,6 @@
  */
 @threadsafe
 private[kafka] class LogManager(val config: KafkaConfig,
-                                private val scheduler: KafkaScheduler,
                                 private val time: Time,
                                 val logCleanupIntervalMs: Long,
                                 val logCleanupDefaultAgeMs: Long,
@@ -47,12 +45,12 @@
   private val topicPartitionsMap = config.topicPartitionsMap
   private val logCreationLock = new Object
   private val random = new java.util.Random
-  private var zkActor: Actor = null
   private val startupLatch: CountDownLatch = new CountDownLatch(1)
   private val logFlusherScheduler = new KafkaScheduler(1, "kafka-logflusher-", false)
   private val logFlushIntervalMap = config.flushIntervalMap
   private val logRetentionMSMap = getLogRetentionMSMap(config.logRetentionHoursMap)
   private val logRetentionSize = config.logRetentionSize
+  private val scheduler = new KafkaScheduler(1, "kafka-logcleaner-", false)
 
   /* Initialize a log for each subdirectory of the main log directory */
   private val logs = new Pool[String, Pool[Int, Log]]()
@@ -78,35 +76,6 @@
     }
   }
   
-  /* Schedule the cleanup task to delete old logs */
-  if(scheduler != null) {
-    info("starting log cleaner every " + logCleanupIntervalMs + " ms")    
-    scheduler.scheduleWithRate(cleanupLogs, 60 * 1000, logCleanupIntervalMs)
-  }
-
-  kafkaZookeeper.startup
-  zkActor = new Actor {
-    def act() {
-      loop {
-        receive {
-          case topic: String =>
-            try {
-              kafkaZookeeper.registerTopicInZk(topic)
-            }
-            catch {
-              case e => error(e) // log it and let it go
-            }
-          case StopActor =>
-            info("zkActor stopped")
-            exit
-        }
-      }
-    }
-  }
-  zkActor.start
-
-  case object StopActor
-
   private def getLogRetentionMSMap(logRetentionHourMap: Map[String, Int]) : Map[String, Long] = {
     var ret = new mutable.HashMap[String, Long]
     for ( (topic, hour) <- logRetentionHourMap )
@@ -118,22 +87,29 @@
    *  Register this broker in ZK for the first time.
    */
   def startup() {
+    kafkaZookeeper.startup
     kafkaZookeeper.registerBrokerInZk()
-    for (topic <- getAllTopics)
-      kafkaZookeeper.registerTopicInZk(topic)
-    startupLatch.countDown
+
+    /* Schedule the cleanup task to delete old logs */
+    if(scheduler != null) {
+      if(scheduler.hasShutdown) {
+        println("Restarting log cleaner scheduler")
+        scheduler.startUp
+      }
+      info("starting log cleaner every " + logCleanupIntervalMs + " ms")
+      scheduler.scheduleWithRate(cleanupLogs, 60 * 1000, logCleanupIntervalMs)
+    }
+
+    if(logFlusherScheduler.hasShutdown) logFlusherScheduler.startUp
     info("Starting log flusher every " + config.flushSchedulerThreadRate + " ms with the following overrides " + logFlushIntervalMap)
     logFlusherScheduler.scheduleWithRate(flushAllLogs, config.flushSchedulerThreadRate, config.flushSchedulerThreadRate)
+    startupLatch.countDown
   }
 
   private def awaitStartup() {
     startupLatch.await
   }
 
-  private def registerNewTopicInZK(topic: String) {
-    zkActor ! topic
-  }
-
   /**
    * Create a log for the given topic and the given partition
    */
@@ -186,6 +162,10 @@
    * Create the log if it does not exist, if it exists just return it
    */
   def getOrCreateLog(topic: String, partition: Int): Log = {
+    // TODO: Change this later
+    if(!ZkUtils.isPartitionOnBroker(kafkaZookeeper.zkClient, topic, partition, config.brokerId))
+      throw new InvalidPartitionException("Broker %d does not host partition %d for topic %s".
+        format(config.brokerId, partition, topic))
     var hasNewTopic = false
     var parts = getLogPool(topic, partition)
     if (parts == null) {
@@ -196,6 +176,7 @@
     }
     var log = parts.get(partition)
     if(log == null) {
+      // check if this broker hosts this partition
       log = createLog(topic, partition)
       val found = parts.putIfNotExists(partition, log)
       if(found != null) {
@@ -207,8 +188,6 @@
         info("Created log for '" + topic + "'-" + partition)
     }
 
-    if (hasNewTopic)
-      registerNewTopicInZK(topic)
     log
   }
   
@@ -279,11 +258,11 @@
    */
   def close() {
     info("Closing log manager")
+    scheduler.shutdown()
     logFlusherScheduler.shutdown()
     val iter = getLogIterator
     while(iter.hasNext)
       iter.next.close()
-    zkActor ! StopActor
     kafkaZookeeper.close
   }
   
Index: core/src/main/scala/kafka/producer/ZKBrokerPartitionInfo.scala
===================================================================
--- core/src/main/scala/kafka/producer/ZKBrokerPartitionInfo.scala	(revision 1294440)
+++ core/src/main/scala/kafka/producer/ZKBrokerPartitionInfo.scala	(working copy)
@@ -1,376 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package kafka.producer
-
-import kafka.utils.{ZKStringSerializer, ZkUtils, ZKConfig}
-import collection.mutable.HashMap
-import collection.immutable.Map
-import kafka.utils.Logging
-import collection.immutable.TreeSet
-import kafka.cluster.{Broker, Partition}
-import org.apache.zookeeper.Watcher.Event.KeeperState
-import org.I0Itec.zkclient.{IZkStateListener, IZkChildListener, ZkClient}
-import collection.SortedSet
-
-private[producer] object ZKBrokerPartitionInfo {
-
-  /**
-   * Generate a mapping from broker id to (brokerId, numPartitions) for the list of brokers
-   * specified
-   * @param topic the topic to which the brokers have registered
-   * @param brokerList the list of brokers for which the partitions info is to be generated
-   * @return a sequence of (brokerId, numPartitions) for brokers in brokerList
-   */
-  private def getBrokerPartitions(zkClient: ZkClient, topic: String, brokerList: List[Int]): SortedSet[Partition] = {
-    val brokerTopicPath = ZkUtils.BrokerTopicsPath + "/" + topic
-    val numPartitions = brokerList.map(bid => ZkUtils.readData(zkClient, brokerTopicPath + "/" + bid).toInt)
-    val brokerPartitions = brokerList.zip(numPartitions)
-
-    val sortedBrokerPartitions = brokerPartitions.sortWith((id1, id2) => id1._1 < id2._1)
-
-    var brokerParts = SortedSet.empty[Partition]
-    sortedBrokerPartitions.foreach { bp =>
-      for(i <- 0 until bp._2) {
-        val bidPid = new Partition(bp._1, i)
-        brokerParts = brokerParts + bidPid
-      }
-    }
-    brokerParts
-  }
-}
-
-/**
- * If zookeeper based auto partition discovery is enabled, fetch broker info like
- * host, port, number of partitions from zookeeper
- */
-private[producer] class ZKBrokerPartitionInfo(config: ZKConfig, producerCbk: (Int, String, Int) => Unit) extends BrokerPartitionInfo with Logging {
-  private val zkWatcherLock = new Object
-  private val zkClient = new ZkClient(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs,
-    ZKStringSerializer)
-  // maintain a map from topic -> list of (broker, num_partitions) from zookeeper
-  private var topicBrokerPartitions = getZKTopicPartitionInfo
-  // maintain a map from broker id to the corresponding Broker object
-  private var allBrokers = getZKBrokerInfo
-
-  // use just the brokerTopicsListener for all watchers
-  private val brokerTopicsListener = new BrokerTopicsListener(topicBrokerPartitions, allBrokers)
-  // register listener for change of topics to keep topicsBrokerPartitions updated
-  zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath, brokerTopicsListener)
-
-  // register listener for change of brokers for each topic to keep topicsBrokerPartitions updated
-  topicBrokerPartitions.keySet.foreach {topic =>
-    zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath + "/" + topic, brokerTopicsListener)
-    debug("Registering listener on path: " + ZkUtils.BrokerTopicsPath + "/" + topic)
-  }
-
-  // register listener for new broker
-  zkClient.subscribeChildChanges(ZkUtils.BrokerIdsPath, brokerTopicsListener)
-
-  // register listener for session expired event
-  zkClient.subscribeStateChanges(new ZKSessionExpirationListener(brokerTopicsListener))
-
-  /**
-   * Return a sequence of (brokerId, numPartitions)
-   * @param topic the topic for which this information is to be returned
-   * @return a sequence of (brokerId, numPartitions). Returns a zero-length
-   * sequence if no brokers are available.
-   */
-  def getBrokerPartitionInfo(topic: String): SortedSet[Partition] = {
-    zkWatcherLock synchronized {
-      val brokerPartitions = topicBrokerPartitions.get(topic)
-      var numBrokerPartitions = SortedSet.empty[Partition]
-      brokerPartitions match {
-        case Some(bp) =>
-          bp.size match {
-            case 0 => // no brokers currently registered for this topic. Find the list of all brokers in the cluster.
-              numBrokerPartitions = bootstrapWithExistingBrokers(topic)
-              topicBrokerPartitions += (topic -> numBrokerPartitions)
-            case _ => numBrokerPartitions = TreeSet[Partition]() ++ bp
-          }
-        case None =>  // no brokers currently registered for this topic. Find the list of all brokers in the cluster.
-          numBrokerPartitions = bootstrapWithExistingBrokers(topic)
-          topicBrokerPartitions += (topic -> numBrokerPartitions)
-      }
-      numBrokerPartitions
-    }
-  }
-
-  /**
-   * Generate the host and port information for the broker identified
-   * by the given broker id
-   * @param brokerId the broker for which the info is to be returned
-   * @return host and port of brokerId
-   */
-  def getBrokerInfo(brokerId: Int): Option[Broker] =  {
-    zkWatcherLock synchronized {
-      allBrokers.get(brokerId)
-    }
-  }
-
-  /**
-   * Generate a mapping from broker id to the host and port for all brokers
-   * @return mapping from id to host and port of all brokers
-   */
-  def getAllBrokerInfo: Map[Int, Broker] = allBrokers
-
-  def close = zkClient.close
-
-  def updateInfo = {
-    zkWatcherLock synchronized {
-      topicBrokerPartitions = getZKTopicPartitionInfo
-      allBrokers = getZKBrokerInfo
-    }
-  }
-
-  private def bootstrapWithExistingBrokers(topic: String): scala.collection.immutable.SortedSet[Partition] = {
-   debug("Currently, no brokers are registered under topic: " + topic)
-    debug("Bootstrapping topic: " + topic + " with available brokers in the cluster with default " +
-      "number of partitions = 1")
-    val allBrokersIds = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerIdsPath)
-    trace("List of all brokers currently registered in zookeeper = " + allBrokersIds.toString)
-    // since we do not have the in formation about number of partitions on these brokers, just assume single partition
-    // i.e. pick partition 0 from each broker as a candidate
-    val numBrokerPartitions = TreeSet[Partition]() ++ allBrokersIds.map(b => new Partition(b.toInt, 0))
-    // add the rest of the available brokers with default 1 partition for this topic, so all of the brokers
-    // participate in hosting this topic.
-    debug("Adding following broker id, partition id for NEW topic: " + topic + "=" + numBrokerPartitions.toString)
-    numBrokerPartitions
-  }
-
-  /**
-   * Generate a sequence of (brokerId, numPartitions) for all topics
-   * registered in zookeeper
-   * @return a mapping from topic to sequence of (brokerId, numPartitions)
-   */
-  private def getZKTopicPartitionInfo(): collection.mutable.Map[String, SortedSet[Partition]] = {
-    val brokerPartitionsPerTopic = new HashMap[String, SortedSet[Partition]]()
-    ZkUtils.makeSurePersistentPathExists(zkClient, ZkUtils.BrokerTopicsPath)
-    val topics = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerTopicsPath)
-    topics.foreach { topic =>
-    // find the number of broker partitions registered for this topic
-      val brokerTopicPath = ZkUtils.BrokerTopicsPath + "/" + topic
-      val brokerList = ZkUtils.getChildrenParentMayNotExist(zkClient, brokerTopicPath)
-      val numPartitions = brokerList.map(bid => ZkUtils.readData(zkClient, brokerTopicPath + "/" + bid).toInt)
-      val brokerPartitions = brokerList.map(bid => bid.toInt).zip(numPartitions)
-      val sortedBrokerPartitions = brokerPartitions.sortWith((id1, id2) => id1._1 < id2._1)
-      debug("Broker ids and # of partitions on each for topic: " + topic + " = " + sortedBrokerPartitions.toString)
-
-      var brokerParts = SortedSet.empty[Partition]
-      sortedBrokerPartitions.foreach { bp =>
-        for(i <- 0 until bp._2) {
-          val bidPid = new Partition(bp._1, i)
-          brokerParts = brokerParts + bidPid
-        }
-      }
-      brokerPartitionsPerTopic += (topic -> brokerParts)
-      debug("Sorted list of broker ids and partition ids on each for topic: " + topic + " = " + brokerParts.toString)
-    }
-    brokerPartitionsPerTopic
-  }
-
-  /**
-   * Generate a mapping from broker id to (brokerId, numPartitions) for all brokers
-   * registered in zookeeper
-   * @return a mapping from brokerId to (host, port)
-   */
-  private def getZKBrokerInfo(): Map[Int, Broker] = {
-    val allBrokerIds = ZkUtils.getChildrenParentMayNotExist(zkClient, ZkUtils.BrokerIdsPath).map(bid => bid.toInt)
-    val brokers = ZkUtils.getBrokerInfoFromIds(zkClient, allBrokerIds)
-    allBrokerIds.zip(brokers).toMap
-  }
-
-  /**
-   * Listens to new broker registrations under a particular topic, in zookeeper and
-   * keeps the related data structures updated
-   */
-  class BrokerTopicsListener(val originalBrokerTopicsPartitionsMap: collection.mutable.Map[String, SortedSet[Partition]],
-                             val originalBrokerIdMap: Map[Int, Broker]) extends IZkChildListener with Logging {
-    private var oldBrokerTopicPartitionsMap = collection.mutable.Map.empty[String, SortedSet[Partition]] ++
-                                              originalBrokerTopicsPartitionsMap
-    private var oldBrokerIdMap = collection.mutable.Map.empty[Int, Broker] ++ originalBrokerIdMap
-
-    debug("[BrokerTopicsListener] Creating broker topics listener to watch the following paths - \n" +
-      "/broker/topics, /broker/topics/topic, /broker/ids")
-    debug("[BrokerTopicsListener] Initialized this broker topics listener with initial mapping of broker id to " +
-      "partition id per topic with " + oldBrokerTopicPartitionsMap.toString)
-
-    @throws(classOf[Exception])
-    def handleChildChange(parentPath : String, currentChildren : java.util.List[String]) {
-      val curChilds: java.util.List[String] = if(currentChildren != null) currentChildren
-                                              else new java.util.ArrayList[String]()
-
-      zkWatcherLock synchronized {
-        trace("Watcher fired for path: " + parentPath + " with change " + curChilds.toString)
-        import scala.collection.JavaConversions._
-
-        parentPath match {
-          case "/brokers/topics" =>        // this is a watcher for /broker/topics path
-            val updatedTopics = asBuffer(curChilds)
-            debug("[BrokerTopicsListener] List of topics changed at " + parentPath + " Updated topics -> " +
-                curChilds.toString)
-            debug("[BrokerTopicsListener] Old list of topics: " + oldBrokerTopicPartitionsMap.keySet.toString)
-            debug("[BrokerTopicsListener] Updated list of topics: " + updatedTopics.toSet.toString)
-            val newTopics = updatedTopics.toSet &~ oldBrokerTopicPartitionsMap.keySet
-            debug("[BrokerTopicsListener] List of newly registered topics: " + newTopics.toString)
-            newTopics.foreach { topic =>
-              val brokerTopicPath = ZkUtils.BrokerTopicsPath + "/" + topic
-              val brokerList = ZkUtils.getChildrenParentMayNotExist(zkClient, brokerTopicPath)
-              processNewBrokerInExistingTopic(topic, brokerList)
-              zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath + "/" + topic,
-                brokerTopicsListener)
-            }
-          case "/brokers/ids"    =>        // this is a watcher for /broker/ids path
-            debug("[BrokerTopicsListener] List of brokers changed in the Kafka cluster " + parentPath +
-                "\t Currently registered list of brokers -> " + curChilds.toString)
-            processBrokerChange(parentPath, curChilds)
-          case _ =>
-            val pathSplits = parentPath.split("/")
-            val topic = pathSplits.last
-            if(pathSplits.length == 4 && pathSplits(2).equals("topics")) {
-              debug("[BrokerTopicsListener] List of brokers changed at " + parentPath + "\t Currently registered " +
-                  " list of brokers -> " + curChilds.toString + " for topic -> " + topic)
-              processNewBrokerInExistingTopic(topic, asBuffer(curChilds))
-            }
-        }
-
-        // update the data structures tracking older state values
-        oldBrokerTopicPartitionsMap = collection.mutable.Map.empty[String, SortedSet[Partition]] ++ topicBrokerPartitions
-        oldBrokerIdMap = collection.mutable.Map.empty[Int, Broker] ++  allBrokers
-      }
-    }
-
-    def processBrokerChange(parentPath: String, curChilds: Seq[String]) {
-      if(parentPath.equals(ZkUtils.BrokerIdsPath)) {
-        import scala.collection.JavaConversions._
-        val updatedBrokerList = asBuffer(curChilds).map(bid => bid.toInt)
-        val newBrokers = updatedBrokerList.toSet &~ oldBrokerIdMap.keySet
-        debug("[BrokerTopicsListener] List of newly registered brokers: " + newBrokers.toString)
-        newBrokers.foreach { bid =>
-          val brokerInfo = ZkUtils.readData(zkClient, ZkUtils.BrokerIdsPath + "/" + bid)
-          val brokerHostPort = brokerInfo.split(":")
-          allBrokers += (bid -> new Broker(bid, brokerHostPort(1), brokerHostPort(1), brokerHostPort(2).toInt))
-          debug("[BrokerTopicsListener] Invoking the callback for broker: " + bid)
-          producerCbk(bid, brokerHostPort(1), brokerHostPort(2).toInt)
-        }
-        // remove dead brokers from the in memory list of live brokers
-        val deadBrokers = oldBrokerIdMap.keySet &~ updatedBrokerList.toSet
-        debug("[BrokerTopicsListener] Deleting broker ids for dead brokers: " + deadBrokers.toString)
-        deadBrokers.foreach {bid =>
-          allBrokers = allBrokers - bid
-          // also remove this dead broker from particular topics
-          topicBrokerPartitions.keySet.foreach{ topic =>
-            topicBrokerPartitions.get(topic) match {
-              case Some(oldBrokerPartitionList) =>
-                val aliveBrokerPartitionList = oldBrokerPartitionList.filter(bp => bp.brokerId != bid)
-                topicBrokerPartitions += (topic -> aliveBrokerPartitionList)
-                debug("[BrokerTopicsListener] Removing dead broker ids for topic: " + topic + "\t " +
-                  "Updated list of broker id, partition id = " + aliveBrokerPartitionList.toString)
-              case None =>
-            }
-          }
-        }
-      }
-    }
-
-    /**
-     * Generate the updated mapping of (brokerId, numPartitions) for the new list of brokers
-     * registered under some topic
-     * @param parentPath the path of the topic under which the brokers have changed
-     * @param curChilds the list of changed brokers
-     */
-    def processNewBrokerInExistingTopic(topic: String, curChilds: Seq[String]) = {
-      // find the old list of brokers for this topic
-      oldBrokerTopicPartitionsMap.get(topic) match {
-        case Some(brokersParts) =>
-          debug("[BrokerTopicsListener] Old list of brokers: " + brokersParts.map(bp => bp.brokerId).toString)
-        case None =>
-      }
-
-      val updatedBrokerList = curChilds.map(b => b.toInt)
-      import ZKBrokerPartitionInfo._
-      val updatedBrokerParts:SortedSet[Partition] = getBrokerPartitions(zkClient, topic, updatedBrokerList.toList)
-      debug("[BrokerTopicsListener] Currently registered list of brokers for topic: " + topic + " are " +
-          curChilds.toString)
-      // update the number of partitions on existing brokers
-      var mergedBrokerParts: SortedSet[Partition] = TreeSet[Partition]() ++ updatedBrokerParts
-      topicBrokerPartitions.get(topic) match {
-        case Some(oldBrokerParts) =>
-          debug("[BrokerTopicsListener] Unregistered list of brokers for topic: " + topic + " are " +
-            oldBrokerParts.toString)
-          mergedBrokerParts = oldBrokerParts ++ updatedBrokerParts
-        case None =>
-      }
-      // keep only brokers that are alive
-      mergedBrokerParts = mergedBrokerParts.filter(bp => allBrokers.contains(bp.brokerId))
-      topicBrokerPartitions += (topic -> mergedBrokerParts)
-      debug("[BrokerTopicsListener] List of broker partitions for topic: " + topic + " are " +
-          mergedBrokerParts.toString)
-    }
-
-    def resetState = {
-      trace("[BrokerTopicsListener] Before reseting broker topic partitions state " +
-          oldBrokerTopicPartitionsMap.toString)
-      oldBrokerTopicPartitionsMap = collection.mutable.Map.empty[String, SortedSet[Partition]] ++ topicBrokerPartitions
-      debug("[BrokerTopicsListener] After reseting broker topic partitions state " +
-          oldBrokerTopicPartitionsMap.toString)
-      trace("[BrokerTopicsListener] Before reseting broker id map state " + oldBrokerIdMap.toString)
-      oldBrokerIdMap = collection.mutable.Map.empty[Int, Broker] ++  allBrokers
-      debug("[BrokerTopicsListener] After reseting broker id map state " + oldBrokerIdMap.toString)
-    }
-  }
-
-  /**
-   * Handles the session expiration event in zookeeper
-   */
-  class ZKSessionExpirationListener(val brokerTopicsListener: BrokerTopicsListener)
-    extends IZkStateListener {
-
-    @throws(classOf[Exception])
-    def handleStateChanged(state: KeeperState) {
-      // do nothing, since zkclient will do reconnect for us.
-    }
-
-    /**
-     * Called after the zookeeper session has expired and a new session has been created. You would have to re-create
-     * any ephemeral nodes here.
-     *
-     * @throws Exception
-     *             On any error.
-     */
-    @throws(classOf[Exception])
-    def handleNewSession() {
-      /**
-       *  When we get a SessionExpired event, we lost all ephemeral nodes and zkclient has reestablished a
-       *  connection for us.
-       */
-      info("ZK expired; release old list of broker partitions for topics ")
-      topicBrokerPartitions = getZKTopicPartitionInfo
-      allBrokers = getZKBrokerInfo
-      brokerTopicsListener.resetState
-
-      // register listener for change of brokers for each topic to keep topicsBrokerPartitions updated
-      // NOTE: this is probably not required here. Since when we read from getZKTopicPartitionInfo() above,
-      // it automatically recreates the watchers there itself
-      topicBrokerPartitions.keySet.foreach(topic => zkClient.subscribeChildChanges(ZkUtils.BrokerTopicsPath + "/" + topic,
-        brokerTopicsListener))
-      // there is no need to re-register other listeners as they are listening on the child changes of
-      // permanent nodes
-    }
-
-  }
-
-}
Index: core/src/main/scala/kafka/producer/ProducerConfig.scala
===================================================================
--- core/src/main/scala/kafka/producer/ProducerConfig.scala	(revision 1294440)
+++ core/src/main/scala/kafka/producer/ProducerConfig.scala	(working copy)
@@ -29,15 +29,15 @@
    *  to pass in static broker and per-broker partition information. Format-    *
    *  brokerid1:host1:port1, brokerid2:host2:port2*/
   val brokerList = Utils.getString(props, "broker.list", null)
-  if(brokerList != null && Utils.getString(props, "partitioner.class", null) != null)
-    throw new InvalidConfigException("partitioner.class cannot be used when broker.list is set")
+  if(brokerList != null)
+    throw new InvalidConfigException("broker.list is deprecated. Use zk.connect instead")
 
   /** If both broker.list and zk.connect options are specified, throw an exception */
-  if(brokerList != null && zkConnect != null)
-    throw new InvalidConfigException("only one of broker.list and zk.connect can be specified")
+  if(zkConnect == null)
+    throw new InvalidConfigException("zk.connect property is required")
 
   /** the partitioner class for partitioning events amongst sub-topics */
-  val partitionerClass = Utils.getString(props, "partitioner.class", null)
+  val partitionerClass = Utils.getString(props, "partitioner.class", "kafka.producer.DefaultPartitioner")
 
   /** this parameter specifies whether the messages are sent asynchronously *
    * or not. Valid values are - async for asynchronous send                 *
Index: core/src/main/scala/kafka/producer/ProducerPool.scala
===================================================================
--- core/src/main/scala/kafka/producer/ProducerPool.scala	(revision 1294440)
+++ core/src/main/scala/kafka/producer/ProducerPool.scala	(working copy)
@@ -13,19 +13,23 @@
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  * See the License for the specific language governing permissions and
  * limitations under the License.
-*/
+ */
 
 package kafka.producer
 
+import kafka.cluster.Broker
 import java.util.Properties
-import kafka.cluster.Broker
-import kafka.utils.Logging
-import java.util.concurrent.ConcurrentHashMap
+import org.I0Itec.zkclient.ZkClient
+import kafka.utils.{ZkUtils, Utils, Logging}
+import collection.mutable.HashMap
+import java.lang.Object
+import kafka.common.{UnavailableProducerException, NoBrokersForPartitionException}
 
-class ProducerPool(private val config: ProducerConfig) extends Logging {
-  private val syncProducers = new ConcurrentHashMap[Int, SyncProducer]
+class ProducerPool(val config: ProducerConfig, val zkClient: ZkClient) extends Logging {
+  private val syncProducers = new HashMap[Int, SyncProducer]
+  private val lock = new Object()
 
-  def addProducer(broker: Broker) {
+  private def addProducer(broker: Broker) {
     val props = new Properties()
     props.put("host", broker.host)
     props.put("port", broker.port.toString)
@@ -42,17 +46,48 @@
     syncProducers.put(brokerId, syncProducer)
   }
 
+  def addProducers(config: ProducerConfig) {
+    lock.synchronized {
+      debug("Connecting to %s for creating sync producers for all brokers in the cluster".format(config.zkConnect))
+      val brokers = ZkUtils.getAllBrokersInCluster(zkClient)
+      brokers.foreach(broker => addProducer(broker))
+    }
+  }
+
   def getProducer(brokerId: Int) : SyncProducer = {
-    syncProducers.get(brokerId)
+    lock.synchronized {
+      val producer = syncProducers.get(brokerId)
+      producer match {
+        case Some(p) => p
+        case None => throw new UnavailableProducerException("Sync producer for broker id %d does not exist".format(brokerId))
+      }
+    }
   }
 
+  def getAnyProducer: SyncProducer = {
+    lock.synchronized {
+      if(syncProducers.size == 0) {
+        // refresh the list of brokers from zookeeper
+        info("No sync producers available. Refreshing the available broker list from ZK and creating sync producers")
+        addProducers(config)
+        if(syncProducers.size == 0)
+          throw new NoBrokersForPartitionException("No brokers available")
+      }
+      syncProducers.get(Utils.random.nextInt(syncProducers.size)).get
+    }
+  }
+
+  def getZkClient: ZkClient = zkClient
+
   /**
    * Closes all the producers in the pool
    */
   def close() = {
-    info("Closing all sync producers")
-    val iter = syncProducers.values.iterator
-    while(iter.hasNext)
-      iter.next.close
+    lock.synchronized {
+      info("Closing all sync producers")
+      val iter = syncProducers.values.iterator
+      while(iter.hasNext)
+        iter.next.close
+    }
   }
 }
Index: core/src/main/scala/kafka/producer/SyncProducer.scala
===================================================================
--- core/src/main/scala/kafka/producer/SyncProducer.scala	(revision 1294440)
+++ core/src/main/scala/kafka/producer/SyncProducer.scala	(working copy)
@@ -26,6 +26,7 @@
 import scala.math._
 import kafka.common.MessageSizeTooLargeException
 import java.nio.ByteBuffer
+import kafka.utils.Utils._
 
 object SyncProducer {
   val RequestKey: Short = 0
@@ -124,6 +125,21 @@
     send(new BoundedByteBufferSend(new MultiProducerRequest(produces)))
   }
 
+  def send(request: TopicMetadataRequest): Seq[TopicMetadata] = {
+    lock synchronized {
+      getOrMakeConnection()
+      var response: Tuple2[Receive,Int] = null
+      try {
+        sendRequest(request, channel)
+        response = getResponse(channel)
+      } catch {
+        case e : java.io.IOException => error("Failed to write topic metadata request on the socket channel", e)
+      }
+      // TODO: handle any errors in the response and throw the relevant exception
+      TopicMetadataRequest.deserializeTopicsMetadataResponse(response._1.buffer)
+    }
+  }
+
   def close() = {
     lock synchronized {
       disconnect()
Index: core/src/main/scala/kafka/producer/ConfigBrokerPartitionInfo.scala
===================================================================
--- core/src/main/scala/kafka/producer/ConfigBrokerPartitionInfo.scala	(revision 1294440)
+++ core/src/main/scala/kafka/producer/ConfigBrokerPartitionInfo.scala	(working copy)
@@ -1,95 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
-*/
-package kafka.producer
-
-import collection.mutable.HashMap
-import collection.Map
-import collection.SortedSet
-import kafka.cluster.{Broker, Partition}
-import kafka.common.InvalidConfigException
-import kafka.api.ProducerRequest
-
-private[producer] class ConfigBrokerPartitionInfo(config: ProducerConfig) extends BrokerPartitionInfo {
-  private val brokerPartitions: SortedSet[Partition] = getConfigTopicPartitionInfo
-  private val allBrokers = getConfigBrokerInfo
-
-  /**
-   * Return a sequence of (brokerId, numPartitions)
-   * @param topic this value is null 
-   * @return a sequence of (brokerId, numPartitions)
-   */
-  def getBrokerPartitionInfo(topic: String): SortedSet[Partition] = brokerPartitions
-
-  /**
-   * Generate the host and port information for the broker identified
-   * by the given broker id
-   * @param brokerId the broker for which the info is to be returned
-   * @return host and port of brokerId
-   */
-  def getBrokerInfo(brokerId: Int): Option[Broker] = {
-    allBrokers.get(brokerId)
-  }
-
-  /**
-   * Generate a mapping from broker id to the host and port for all brokers
-   * @return mapping from id to host and port of all brokers
-   */
-  def getAllBrokerInfo: Map[Int, Broker] = allBrokers
-
-  def close {}
-
-  def updateInfo = {}
-
-  /**
-   * Generate a sequence of (brokerId, numPartitions) for all brokers
-   * specified in the producer configuration
-   * @return sequence of (brokerId, numPartitions)
-   */
-  private def getConfigTopicPartitionInfo(): SortedSet[Partition] = {
-    val brokerInfoList = config.brokerList.split(",")
-    if(brokerInfoList.size == 0) throw new InvalidConfigException("broker.list is empty")
-    // check if each individual broker info is valid => (brokerId: brokerHost: brokerPort)
-    brokerInfoList.foreach { bInfo =>
-      val brokerInfo = bInfo.split(":")
-      if(brokerInfo.size < 3) throw new InvalidConfigException("broker.list has invalid value")
-    }
-    val brokerIds = brokerInfoList.map(bInfo => bInfo.split(":").head.toInt)
-    var brokerParts = SortedSet.empty[Partition]
-    brokerIds.foreach { bid =>
-        val bidPid = new Partition(bid, ProducerRequest.RandomPartition)
-        brokerParts += bidPid
-    }
-    brokerParts
-  }
-
-  /**
-   * Generate the host and port information for for all brokers
-   * specified in the producer configuration
-   * @return mapping from brokerId to (host, port) for all brokers
-   */
-  private def getConfigBrokerInfo(): Map[Int, Broker] = {
-    val brokerInfo = new HashMap[Int, Broker]()
-    val brokerInfoList = config.brokerList.split(",")
-    brokerInfoList.foreach{ bInfo =>
-      val brokerIdHostPort = bInfo.split(":")
-      brokerInfo += (brokerIdHostPort(0).toInt -> new Broker(brokerIdHostPort(0).toInt, brokerIdHostPort(1),
-        brokerIdHostPort(1), brokerIdHostPort(2).toInt))
-    }
-    brokerInfo
-  }
-
-}
\ No newline at end of file
Index: core/src/main/scala/kafka/producer/Producer.scala
===================================================================
--- core/src/main/scala/kafka/producer/Producer.scala	(revision 1294440)
+++ core/src/main/scala/kafka/producer/Producer.scala	(working copy)
@@ -22,15 +22,14 @@
 import java.util.concurrent.{TimeUnit, LinkedBlockingQueue}
 import kafka.serializer.Encoder
 import java.util.concurrent.atomic.{AtomicLong, AtomicBoolean}
+import org.I0Itec.zkclient.ZkClient
 
 class Producer[K,V](config: ProducerConfig,
                     private val eventHandler: EventHandler[K,V]) // for testing only
 extends Logging {
   private val hasShutdown = new AtomicBoolean(false)
-  if(!Utils.propertyExists(config.zkConnect) && !Utils.propertyExists(config.brokerList))
-    throw new InvalidConfigException("At least one of zk.connect or broker.list must be specified")
-  if (Utils.propertyExists(config.zkConnect) && Utils.propertyExists(config.brokerList))
-    throw new InvalidConfigException("Only one of zk.connect and broker.list should be provided")
+  if(!Utils.propertyExists(config.zkConnect))
+    throw new InvalidConfigException("zk.connect property must be specified in the producer")
   if (config.batchSize > config.queueSize)
     throw new InvalidConfigException("Batch size can't be larger than queue size.")
 
@@ -52,15 +51,16 @@
    * This constructor can be used when all config parameters will be specified through the
    * ProducerConfig object
    * @param config Producer Configuration object
+   * @param zkClient The ZkClient instance use by the producer to connect to zookeeper. used ONLY for testing
    */
-  def this(config: ProducerConfig) =
+  def this(config: ProducerConfig, zkClient: ZkClient = null) =
     this(config,
          new DefaultEventHandler[K,V](config,
                                       Utils.getObject[Partitioner[K]](config.partitionerClass),
                                       Utils.getObject[Encoder[V]](config.serializerClass),
-                                      new ProducerPool(config),
-                                      populateProducerPool= true,
-                                      brokerPartitionInfo= null))
+                                      new ProducerPool(config, if(zkClient == null)
+                                      new ZkClient(config.zkConnect, config.zkSessionTimeoutMs,
+                                        config.zkConnectionTimeoutMs, ZKStringSerializer) else zkClient)))
 
   /**
    * Sends the data, partitioned by key to the topic using either the
Index: core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala
===================================================================
--- core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala	(revision 1294440)
+++ core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala	(working copy)
@@ -16,44 +16,70 @@
 */
 package kafka.producer
 
-import collection.Map
-import collection.SortedSet
 import kafka.cluster.{Broker, Partition}
+import collection.mutable.HashMap
+import kafka.api.{TopicMetadataRequest, TopicMetadata}
+import java.lang.IllegalStateException
+import kafka.common.NoLeaderForPartitionException
+import kafka.utils.Logging
 
-trait BrokerPartitionInfo {
+class BrokerPartitionInfo(producerPool: ProducerPool) extends Logging {
+  val topicPartitionInfo = new HashMap[String, TopicMetadata]()
+  val zkClient = producerPool.getZkClient
+
   /**
    * Return a sequence of (brokerId, numPartitions).
    * @param topic the topic for which this information is to be returned
    * @return a sequence of (brokerId, numPartitions). Returns a zero-length
    * sequence if no brokers are available.
    */  
-  def getBrokerPartitionInfo(topic: String = null): SortedSet[Partition]
+  def getBrokerPartitionInfo(topic: String): Seq[(Partition, Broker)] = {
+    // check if the cache has metadata for this topic
+    val topicMetadata = topicPartitionInfo.get(topic)
+    val metadata: TopicMetadata =
+    topicMetadata match {
+      case Some(m) => m
+      case None =>
+        // refresh the topic metadata cache
+        info("Fetching metadata for topic %s".format(topic))
+        updateInfo(topic)
+        val topicMetadata = topicPartitionInfo.get(topic)
+        topicMetadata match {
+          case Some(m) => m
+          case None => throw new IllegalStateException("Failed to fetch topic metadata for topic: " + topic)
+        }
+    }
+    val partitionMetadata = metadata.partitionsMetadata
+    partitionMetadata.map { m =>
+      m.leader match {
+        case Some(leader) => (new Partition(leader.id, m.partitionId, topic) -> leader)
+        case None =>  throw new NoLeaderForPartitionException("No leader for topic %s, partition %d".format(topic, m.partitionId))
+      }
+    }.sortWith((s, t) => s._1.partId < t._1.partId)
+  }
 
   /**
-   * Generate the host and port information for the broker identified
-   * by the given broker id 
-   * @param brokerId the broker for which the info is to be returned
-   * @return host and port of brokerId
+   * It updates the cache by issuing a get topic metadata request to a random broker.
+   * @param topic the topic for which the metadata is to be fetched
    */
-  def getBrokerInfo(brokerId: Int): Option[Broker]
-
-  /**
-   * Generate a mapping from broker id to the host and port for all brokers
-   * @return mapping from id to host and port of all brokers
-   */
-  def getAllBrokerInfo: Map[Int, Broker]
-
-  /**
-   * This is relevant to the ZKBrokerPartitionInfo. It updates the ZK cache
-   * by reading from zookeeper and recreating the data structures. This API
-   * is invoked by the producer, when it detects that the ZK cache of
-   * ZKBrokerPartitionInfo is stale.
-   *
-   */
-  def updateInfo
-
-  /**
-   * Cleanup
-   */
-  def close
+  def updateInfo(topic: String = null) = {
+    val producer = producerPool.getAnyProducer
+    if(topic != null) {
+      val topicMetadataRequest = new TopicMetadataRequest(List(topic))
+      val topicMetadataList = producer.send(topicMetadataRequest)
+      val topicMetadata:Option[TopicMetadata] = if(topicMetadataList.size > 0) Some(topicMetadataList.head) else None
+      topicMetadata match {
+        case Some(metadata) =>
+          info("Fetched metadata for topics %s".format(topic))
+          topicPartitionInfo += (topic -> metadata)
+        case None =>
+      }
+    }else {
+      // refresh cache for all topics
+      val topics = topicPartitionInfo.keySet.toList
+      val topicMetadata = producer.send(new TopicMetadataRequest(topics))
+      info("Fetched metadata for topics %s".format(topicMetadata.mkString(",")))
+      topicMetadata.foreach(metadata => topicPartitionInfo += (metadata.topic -> metadata))
+    }
+  }
 }
Index: core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala
===================================================================
--- core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala	(revision 1294440)
+++ core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala	(working copy)
@@ -19,56 +19,26 @@
 
 import kafka.api.ProducerRequest
 import kafka.serializer.Encoder
-import java.util.Properties
 import kafka.producer._
-import kafka.utils.{ZKConfig, Utils, Logging}
 import kafka.cluster.{Partition, Broker}
 import collection.mutable.{ListBuffer, HashMap}
 import scala.collection.Map
 import kafka.common.{FailedToSendMessageException, InvalidPartitionException, NoBrokersForPartitionException}
 import kafka.message.{Message, NoCompressionCodec, ByteBufferMessageSet}
+import kafka.utils.{Utils, Logging}
 
 class DefaultEventHandler[K,V](config: ProducerConfig,                               // this api is for testing
                                private val partitioner: Partitioner[K],              // use the other constructor
                                private val encoder: Encoder[V],
-                               private val producerPool: ProducerPool,
-                               private val populateProducerPool: Boolean,
-                               private var brokerPartitionInfo: BrokerPartitionInfo)
+                               private val producerPool: ProducerPool)
   extends EventHandler[K,V] with Logging {
 
-  private val lock = new Object()
-  private val zkEnabled = Utils.propertyExists(config.zkConnect)
-  if(brokerPartitionInfo == null) {
-    zkEnabled match {
-      case true =>
-        val zkProps = new Properties()
-        zkProps.put("zk.connect", config.zkConnect)
-        zkProps.put("zk.sessiontimeout.ms", config.zkSessionTimeoutMs.toString)
-        zkProps.put("zk.connectiontimeout.ms", config.zkConnectionTimeoutMs.toString)
-        zkProps.put("zk.synctime.ms", config.zkSyncTimeMs.toString)
-        brokerPartitionInfo = new ZKBrokerPartitionInfo(new ZKConfig(zkProps), producerCbk)
-      case false =>
-        brokerPartitionInfo = new ConfigBrokerPartitionInfo(config)
-    }
-  }
+  val brokerPartitionInfo = new BrokerPartitionInfo(producerPool)
 
-  // pool of producers, one per broker
-  if(populateProducerPool) {
-    val allBrokers = brokerPartitionInfo.getAllBrokerInfo
-    allBrokers.foreach(b => producerPool.addProducer(new Broker(b._1, b._2.host, b._2.host, b._2.port)))
-  }
+  // add producers to the producer pool
+  producerPool.addProducers(config)
 
-  /**
-   * Callback to add a new producer to the producer pool. Used by ZKBrokerPartitionInfo
-   * on registration of new broker in zookeeper
-   * @param bid the id of the broker
-   * @param host the hostname of the broker
-   * @param port the port of the broker
-   */
-  private def producerCbk(bid: Int, host: String, port: Int) =  {
-    if(populateProducerPool) producerPool.addProducer(new Broker(bid, host, host, port))
-    else debug("Skipping the callback since populateProducerPool = false")
-  }
+  private val lock = new Object()
 
   def handle(events: Seq[ProducerData[K,V]]) {
     lock synchronized {
@@ -81,7 +51,7 @@
       val partitionedData = partitionAndCollate(messages)
       for ( (brokerid, eventsPerBrokerMap) <- partitionedData) {
         if (logger.isTraceEnabled)
-          eventsPerBrokerMap.foreach(partitionAndEvent => trace("Handling event for Topic: %s, Broker: %d, Partition: %d"
+          eventsPerBrokerMap.foreach(partitionAndEvent => trace("Handling event for Topic: %s, Broker: %d, Partitions: %s"
             .format(partitionAndEvent._1, brokerid, partitionAndEvent._2)))
         val messageSetPerBroker = groupMessagesToSet(eventsPerBrokerMap)
 
@@ -98,7 +68,7 @@
               numRetries +=1
               Thread.sleep(config.producerRetryBackoffMs)
               try {
-                brokerPartitionInfo.updateInfo
+                brokerPartitionInfo.updateInfo()
                 handleSerializedData(eventsPerBroker, 0)
                 return
               }
@@ -125,15 +95,15 @@
       val brokerPartition = topicPartitionsList(partitionIndex)
 
       var dataPerBroker: HashMap[(String, Int), Seq[ProducerData[K,Message]]] = null
-      ret.get(brokerPartition.brokerId) match {
+      ret.get(brokerPartition._2.id) match {
         case Some(element) =>
           dataPerBroker = element.asInstanceOf[HashMap[(String, Int), Seq[ProducerData[K,Message]]]]
         case None =>
           dataPerBroker = new HashMap[(String, Int), Seq[ProducerData[K,Message]]]
-          ret.put(brokerPartition.brokerId, dataPerBroker)
+          ret.put(brokerPartition._2.id, dataPerBroker)
       }
 
-      val topicAndPartition = (event.getTopic, brokerPartition.partId)
+      val topicAndPartition = (event.getTopic, brokerPartition._1.partId)
       var dataPerTopicPartition: ListBuffer[ProducerData[K,Message]] = null
       dataPerBroker.get(topicAndPartition) match {
         case Some(element) =>
@@ -147,9 +117,9 @@
     ret
   }
 
-  private def getPartitionListForTopic(pd: ProducerData[K,Message]): Seq[Partition] = {
+  private def getPartitionListForTopic(pd: ProducerData[K,Message]): Seq[(Partition, Broker)] = {
     debug("Getting the number of broker partitions registered for topic: " + pd.getTopic)
-    val topicPartitionsList = brokerPartitionInfo.getBrokerPartitionInfo(pd.getTopic).toSeq
+    val topicPartitionsList = brokerPartitionInfo.getBrokerPartitionInfo(pd.getTopic)
     debug("Broker partitions registered for topic: " + pd.getTopic + " = " + topicPartitionsList)
     val totalNumPartitions = topicPartitionsList.length
     if(totalNumPartitions == 0) throw new NoBrokersForPartitionException("Partition = " + pd.getKey)
@@ -168,7 +138,7 @@
       throw new InvalidPartitionException("Invalid number of partitions: " + numPartitions +
               "\n Valid values are > 0")
     val partition = if(key == null) Utils.getNextRandomInt(numPartitions)
-                    else partitioner.partition(key , numPartitions)
+                    else partitioner.partition(key, numPartitions)
     if(partition < 0 || partition >= numPartitions)
       throw new InvalidPartitionException("Invalid partition id : " + partition +
               "\n Valid values are in the range inclusive [0, " + (numPartitions-1) + "]")
@@ -235,7 +205,5 @@
   def close() {
     if (producerPool != null)
       producerPool.close    
-    if (brokerPartitionInfo != null)
-      brokerPartitionInfo.close
   }
 }
Index: core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala
===================================================================
--- core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala	(revision 1294440)
+++ core/src/main/scala/kafka/producer/KafkaLog4jAppender.scala	(working copy)
@@ -18,7 +18,7 @@
 package kafka.producer
 
 import async.MissingConfigException
-import org.apache.log4j.spi.{LoggingEvent, ErrorCode}
+import org.apache.log4j.spi.LoggingEvent
 import org.apache.log4j.AppenderSkeleton
 import org.apache.log4j.helpers.LogLog
 import kafka.utils.Logging
Index: core/src/main/scala/kafka/admin/CreateTopicCommand.scala
===================================================================
--- core/src/main/scala/kafka/admin/CreateTopicCommand.scala	(revision 1294440)
+++ core/src/main/scala/kafka/admin/CreateTopicCommand.scala	(working copy)
@@ -18,10 +18,10 @@
 package kafka.admin
 
 import joptsimple.OptionParser
-import kafka.utils.{Utils, ZKStringSerializer, ZkUtils}
 import org.I0Itec.zkclient.ZkClient
+import kafka.utils.{Logging, Utils, ZKStringSerializer, ZkUtils}
 
-object CreateTopicCommand {
+object CreateTopicCommand extends Logging {
 
   def main(args: Array[String]): Unit = {
     val parser = new OptionParser
@@ -91,6 +91,7 @@
       replicaAssignment = AdminUtils.assignReplicasToBrokers(brokerList, numPartitions, replicationFactor)
     else
       replicaAssignment = getManualReplicaAssignment(replicaAssignmentStr, brokerList.toSet)
+    debug("Replica assignment list for %s is %s".format(topic, replicaAssignment))
     AdminUtils.createReplicaAssignmentPathInZK(topic, replicaAssignment, zkClient)
   }
 
@@ -104,8 +105,8 @@
       if (brokerList.size != brokerList.toSet.size)
         throw new AdministrationException("duplicate brokers in replica assignment: " + brokerList)
       if (!brokerList.toSet.subsetOf(availableBrokerList))
-        throw new AdministrationException("some specified brokers not available. specified brokers: " + brokerList +
-                "available broker:" + availableBrokerList)
+        throw new AdministrationException("some specified brokers not available. specified brokers: " + brokerList.toString +
+                "available broker:" + availableBrokerList.toString)
       ret(i) = brokerList.toList
       if (ret(i).size != ret(0).size)
         throw new AdministrationException("partition " + i + " has different replication factor: " + brokerList)
Index: core/src/main/scala/kafka/admin/AdminUtils.scala
===================================================================
--- core/src/main/scala/kafka/admin/AdminUtils.scala	(revision 1294440)
+++ core/src/main/scala/kafka/admin/AdminUtils.scala	(working copy)
@@ -80,6 +80,10 @@
       for (i <- 0 until replicaAssignmentList.size) {
         val zkPath = ZkUtils.getTopicPartitionReplicasPath(topic, i.toString)
         ZkUtils.updatePersistentPath(zkClient, zkPath, Utils.seqToCSV(replicaAssignmentList(i)))
+        // TODO: Remove this with leader election patch
+        // assign leader for the partition i
+//        ZkUtils.updateEphemeralPath(zkClient, ZkUtils.getTopicPartitionLeaderPath(topic, i.toString),
+//          replicaAssignmentList(i).head)
         debug("Updated path %s with %s for replica assignment".format(zkPath, Utils.seqToCSV(replicaAssignmentList(i))))
       }
     }
@@ -103,18 +107,19 @@
         for (i <-0 until partitionMetadata.size) {
           val replicas = ZkUtils.readData(zkClient, ZkUtils.getTopicPartitionReplicasPath(topic, partitions(i).toString))
           val inSyncReplicas = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.getTopicPartitionInSyncPath(topic, partitions(i).toString))
-          val leader = ZkUtils.readDataMaybeNull(zkClient, ZkUtils.getTopicPartitionLeaderPath(topic, partitions(i).toString))
+          val leader = ZkUtils.getLeaderForPartition(zkClient, topic, partitions(i))
           debug("replicas = " + replicas + ", in sync replicas = " + inSyncReplicas + ", leader = " + leader)
 
           partitionMetadata(i) = new PartitionMetadata(partitions(i),
-            if (leader == null) None else Some(getBrokerInfoFromCache(zkClient, cachedBrokerInfo, List(leader.toInt)).head),
+            leader match { case None => None case Some(l) => Some(getBrokerInfoFromCache(zkClient, cachedBrokerInfo, List(l.toInt)).head) },
             getBrokerInfoFromCache(zkClient, cachedBrokerInfo, Utils.getCSVList(replicas).map(id => id.toInt)),
             getBrokerInfoFromCache(zkClient, cachedBrokerInfo, Utils.getCSVList(inSyncReplicas).map(id => id.toInt)),
             None /* Return log segment metadata when getOffsetsBefore will be replaced with this API */)
         }
         Some(new TopicMetadata(topic, partitionMetadata))
-      } else
+      } else {
         None
+      }
     }
 
     metadataList.toList
Index: core/src/main/scala/kafka/common/NoLeaderForPartitionException.scala
===================================================================
--- core/src/main/scala/kafka/common/NoLeaderForPartitionException.scala	(revision 0)
+++ core/src/main/scala/kafka/common/NoLeaderForPartitionException.scala	(revision 0)
@@ -0,0 +1,25 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.common
+
+/**
+ * Thrown when a request is made for partition, but no leader exists for that partition
+ */
+class NoLeaderForPartitionException(message: String) extends RuntimeException(message) {
+  def this() = this(null)
+}
\ No newline at end of file
Index: core/src/main/scala/kafka/consumer/SimpleConsumer.scala
===================================================================
--- core/src/main/scala/kafka/consumer/SimpleConsumer.scala	(revision 1294440)
+++ core/src/main/scala/kafka/consumer/SimpleConsumer.scala	(working copy)
@@ -22,6 +22,7 @@
 import kafka.api._
 import kafka.network._
 import kafka.utils._
+import kafka.utils.Utils._
 
 /**
  * A consumer of kafka messages
@@ -77,16 +78,16 @@
       getOrMakeConnection()
       var response: Tuple2[Receive,Int] = null
       try {
-        sendRequest(request)
-        response = getResponse
+        sendRequest(request, channel)
+        response = getResponse(channel)
       } catch {
         case e : java.io.IOException =>
           info("Reconnect in fetch request due to socket error: ", e)
           // retry once
           try {
             channel = connect
-            sendRequest(request)
-            response = getResponse
+            sendRequest(request, channel)
+            response = getResponse(channel)
           } catch {
             case ioe: java.io.IOException => channel = null; throw ioe;
           }
@@ -115,16 +116,16 @@
       getOrMakeConnection()
       var response: Tuple2[Receive,Int] = null
       try {
-        sendRequest(new OffsetRequest(topic, partition, time, maxNumOffsets))
-        response = getResponse
+        sendRequest(new OffsetRequest(topic, partition, time, maxNumOffsets), channel)
+        response = getResponse(channel)
       } catch {
         case e : java.io.IOException =>
           info("Reconnect in get offetset request due to socket error: ", e)
           // retry once
           try {
             channel = connect
-            sendRequest(new OffsetRequest(topic, partition, time, maxNumOffsets))
-            response = getResponse
+            sendRequest(new OffsetRequest(topic, partition, time, maxNumOffsets), channel)
+            response = getResponse(channel)
           } catch {
             case ioe: java.io.IOException => channel = null; throw ioe;
           }
@@ -133,20 +134,6 @@
     }
   }
 
-  private def sendRequest(request: Request) = {
-    val send = new BoundedByteBufferSend(request)
-    send.writeCompletely(channel)
-  }
-
-  private def getResponse(): Tuple2[Receive,Int] = {
-    val response = new BoundedByteBufferReceive()
-    response.readCompletely(channel)
-
-    // this has the side effect of setting the initial position of buffer correctly
-    val errorCode: Int = response.buffer.getShort
-    (response, errorCode)
-  }
-
   private def getOrMakeConnection() {
     if(channel == null) {
       channel = connect()
Index: core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala
===================================================================
--- core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala	(revision 1294440)
+++ core/src/main/scala/kafka/consumer/ZookeeperConsumerConnector.scala	(working copy)
@@ -29,9 +29,9 @@
 import kafka.api.OffsetRequest
 import java.util.UUID
 import kafka.serializer.Decoder
-import kafka.common.{ConsumerRebalanceFailedException, InvalidConfigException}
 import java.lang.IllegalStateException
 import kafka.utils.ZkUtils._
+import kafka.common.{NoBrokersForPartitionException, ConsumerRebalanceFailedException, InvalidConfigException}
 
 /**
  * This class handles the consumers interaction with zookeeper
@@ -201,6 +201,9 @@
     ret
   }
 
+  // this API is used by unit tests only
+  def getTopicRegistry: Pool[String, Pool[Partition, PartitionTopicInfo]] = topicRegistry
+
   private def registerConsumerInZK(dirs: ZKGroupDirs, consumerIdString: String, topicCount: TopicCount) = {
     info("begin registering consumer " + consumerIdString + " in ZK")
     createEphemeralPathExpectConflict(zkClient, dirs.consumerRegistryDir + "/" + consumerIdString, topicCount.toJsonString)
@@ -368,7 +371,7 @@
                                 kafkaMessageStreams: Map[String,List[KafkaMessageStream[T]]])
     extends IZkChildListener {
     private val dirs = new ZKGroupDirs(group)
-    private var oldPartitionsPerTopicMap: mutable.Map[String,List[String]] = new mutable.HashMap[String,List[String]]()
+    private var oldPartitionsPerTopicMap: mutable.Map[String, Seq[String]] = new mutable.HashMap[String, Seq[String]]()
     private var oldConsumersPerTopicMap: mutable.Map[String,List[String]] = new mutable.HashMap[String,List[String]]()
 
     @throws(classOf[Exception])
@@ -379,18 +382,17 @@
     private def releasePartitionOwnership()= {
       info("Releasing partition ownership")
       for ((topic, infos) <- topicRegistry) {
-        val topicDirs = new ZKGroupTopicDirs(group, topic)
         for(partition <- infos.keys) {
-          val znode = topicDirs.consumerOwnerDir + "/" + partition
-          deletePath(zkClient, znode)
-          debug("Consumer " + consumerIdString + " releasing " + znode)
+          val partitionOwnerPath = getConsumerPartitionOwnerPath(group, topic, partition.partId.toString)
+          deletePath(zkClient, partitionOwnerPath)
+          debug("Consumer " + consumerIdString + " releasing " + partitionOwnerPath)
         }
       }
     }
 
     private def getRelevantTopicMap(myTopicThreadIdsMap: Map[String, Set[String]],
-                                    newPartMap: Map[String,List[String]],
-                                    oldPartMap: Map[String,List[String]],
+                                    newPartMap: Map[String, Seq[String]],
+                                    oldPartMap: Map[String, Seq[String]],
                                     newConsumerMap: Map[String,List[String]],
                                     oldConsumerMap: Map[String,List[String]]): Map[String, Set[String]] = {
       var relevantTopicThreadIdsMap = new mutable.HashMap[String, Set[String]]()
@@ -477,7 +479,7 @@
 
         val topicDirs = new ZKGroupTopicDirs(group, topic)
         val curConsumers = consumersPerTopicMap.get(topic).get
-        var curPartitions: List[String] = partitionsPerTopicMap.get(topic).get
+        var curPartitions: Seq[String] = partitionsPerTopicMap.get(topic).get
 
         val nPartsPerConsumer = curPartitions.size / curConsumers.size
         val nConsumersWithExtraPart = curPartitions.size % curConsumers.size
@@ -599,8 +601,7 @@
         val topic = partitionOwner._1._1
         val partition = partitionOwner._1._2
         val consumerThreadId = partitionOwner._2
-        val topicDirs = new ZKGroupTopicDirs(group, topic)
-        val partitionOwnerPath = topicDirs.consumerOwnerDir + "/" + partition
+        val partitionOwnerPath = getConsumerPartitionOwnerPath(group, topic,partition)
         try {
           createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId)
           info(consumerThreadId + " successfully owned partition " + partition + " for topic " + topic)
@@ -618,37 +619,47 @@
       else true
     }
 
-    private def addPartitionTopicInfo(topicDirs: ZKGroupTopicDirs, partitionString: String,
+    private def addPartitionTopicInfo(topicDirs: ZKGroupTopicDirs, partition: String,
                                       topic: String, consumerThreadId: String) {
-      val partition = Partition.parse(partitionString)
       val partTopicInfoMap = topicRegistry.get(topic)
 
-      val znode = topicDirs.consumerOffsetDir + "/" + partition.name
+      // find the leader for this partition
+      val leaderOpt = getLeaderForPartition(zkClient, topic, partition.toInt)
+      leaderOpt match {
+        case None => throw new NoBrokersForPartitionException("No leader available for partition %s on topic %s".
+          format(partition, topic))
+        case Some(l) => debug("Leader for partition %s for topic %s is %d".format(partition, topic, l))
+      }
+      val leader = leaderOpt.get
+
+      val znode = topicDirs.consumerOffsetDir + "/" + partition
       val offsetString = readDataMaybeNull(zkClient, znode)
       // If first time starting a consumer, set the initial offset based on the config
       var offset : Long = 0L
       if (offsetString == null)
         offset = config.autoOffsetReset match {
               case OffsetRequest.SmallestTimeString =>
-                  earliestOrLatestOffset(topic, partition.brokerId, partition.partId, OffsetRequest.EarliestTime)
+                  earliestOrLatestOffset(topic, leader, partition.toInt, OffsetRequest.EarliestTime)
               case OffsetRequest.LargestTimeString =>
-                  earliestOrLatestOffset(topic, partition.brokerId, partition.partId, OffsetRequest.LatestTime)
+                  earliestOrLatestOffset(topic, leader, partition.toInt, OffsetRequest.LatestTime)
               case _ =>
                   throw new InvalidConfigException("Wrong value in autoOffsetReset in ConsumerConfig")
         }
       else
         offset = offsetString.toLong
+
+      val partitionObject = new Partition(leader, partition.toInt, topic)
       val queue = queues.get((topic, consumerThreadId))
       val consumedOffset = new AtomicLong(offset)
       val fetchedOffset = new AtomicLong(offset)
       val partTopicInfo = new PartitionTopicInfo(topic,
-                                                 partition.brokerId,
-                                                 partition,
+                                                 leader,
+                                                 partitionObject,
                                                  queue,
                                                  consumedOffset,
                                                  fetchedOffset,
                                                  new AtomicInteger(config.fetchSize))
-      partTopicInfoMap.put(partition, partTopicInfo)
+      partTopicInfoMap.put(partitionObject, partTopicInfo)
       debug(partTopicInfo + " selected new offset " + offset)
     }
   }
Index: core/src/main/scala/kafka/utils/Utils.scala
===================================================================
--- core/src/main/scala/kafka/utils/Utils.scala	(revision 1294440)
+++ core/src/main/scala/kafka/utils/Utils.scala	(working copy)
@@ -29,6 +29,7 @@
 import kafka.message.{NoCompressionCodec, CompressionCodec}
 import org.I0Itec.zkclient.ZkClient
 import java.util.{Random, Properties}
+import kafka.network.{BoundedByteBufferReceive, Receive, BoundedByteBufferSend, Request}
 
 /**
  * Helper functions!
@@ -669,6 +670,20 @@
       case _ => // swallow
     }
   }
+
+  def sendRequest(request: Request, channel: SocketChannel) = {
+    val send = new BoundedByteBufferSend(request)
+    send.writeCompletely(channel)
+  }
+
+  def getResponse(channel: SocketChannel): Tuple2[Receive,Int] = {
+    val response = new BoundedByteBufferReceive()
+    response.readCompletely(channel)
+
+    // this has the side effect of setting the initial position of buffer correctly
+    val errorCode: Int = response.buffer.getShort
+    (response, errorCode)
+  }
 }
 
 class SnapshotStats(private val monitorDurationNs: Long = 600L * 1000L * 1000L * 1000L) {
Index: core/src/main/scala/kafka/utils/ZkUtils.scala
===================================================================
--- core/src/main/scala/kafka/utils/ZkUtils.scala	(revision 1294440)
+++ core/src/main/scala/kafka/utils/ZkUtils.scala	(working copy)
@@ -35,7 +35,7 @@
   }
 
   def getTopicPartitionsPath(topic: String): String ={
-    getTopicPath(topic) + "/" + "partitions"
+    getTopicPath(topic) + "/partitions"
   }
 
   def getTopicPartitionPath(topic: String, partitionId: String): String ={
@@ -62,6 +62,38 @@
       ZkUtils.getChildren(zkClient, ZkUtils.BrokerIdsPath).sorted
   }
 
+  def getAllBrokersInCluster(zkClient: ZkClient): Seq[Broker] = {
+    val brokerIds = ZkUtils.getChildren(zkClient, ZkUtils.BrokerIdsPath).sorted
+    getBrokerInfoFromIds(zkClient, brokerIds.map(b => b.toInt))
+  }
+
+  def getLeaderForPartition(zkClient: ZkClient, topic: String, partition: Int): Option[Int] = {
+    // TODO: When leader election is implemented, change this method to return the leader as follows
+    // until then, assume the first replica as the leader
+//    val leader = readDataMaybeNull(zkClient, getTopicPartitionLeaderPath(topic, partition.toString))
+    val replicaListString = readDataMaybeNull(zkClient, getTopicPartitionReplicasPath(topic, partition.toString))
+    val replicas = Utils.getCSVList(replicaListString)
+    replicas.size match {
+      case 0 => None
+      case _ => Some(replicas.head.toInt)
+    }
+  }
+
+  def getReplicasForPartition(zkClient: ZkClient, topic: String, partition: Int): Seq[String] = {
+    val replicaListString = readDataMaybeNull(zkClient, getTopicPartitionReplicasPath(topic, partition.toString))
+    if(replicaListString == null)
+      Seq.empty[String]
+    else {
+      Utils.getCSVList(replicaListString)
+    }
+  }
+
+  def isPartitionOnBroker(zkClient: ZkClient, topic: String, partition: Int, brokerId: Int): Boolean = {
+    val replicas = getReplicasForPartition(zkClient, topic, partition)
+    debug("The list of replicas for topic %s, partition %d is %s".format(topic, partition, replicas))
+    replicas.contains(brokerId.toString)
+  }
+
   def registerBrokerInZk(zkClient: ZkClient, id: Int, host: String, creator: String, port: Int) {
     val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + id
     val broker = new Broker(id, creator, host, port)
@@ -77,6 +109,11 @@
     info("Registering broker " + brokerIdPath + " succeeded with " + broker)
   }
 
+  def getConsumerPartitionOwnerPath(group: String, topic: String, partition: String): String = {
+    val topicDirs = new ZKGroupTopicDirs(group, topic)
+    topicDirs.consumerOwnerDir + "/" + partition
+  }
+
   /**
    *  make sure a persistent path exists in ZK. Create the path if not exist.
    */
@@ -269,30 +306,17 @@
     cluster
   }
 
-  def getPartitionsForTopics(zkClient: ZkClient, topics: Iterator[String]): mutable.Map[String, List[String]] = {
-    val ret = new mutable.HashMap[String, List[String]]()
-    for (topic <- topics) {
-      var partList: List[String] = Nil
-      val brokers = getChildrenParentMayNotExist(zkClient, BrokerTopicsPath + "/" + topic)
-      for (broker <- brokers) {
-        val nParts = readData(zkClient, BrokerTopicsPath + "/" + topic + "/" + broker).toInt
-        for (part <- 0 until nParts)
-          partList ::= broker + "-" + part
-      }
-      partList = partList.sortWith((s,t) => s < t)
-      ret += (topic -> partList)
+  def getPartitionsForTopics(zkClient: ZkClient, topics: Iterator[String]): mutable.Map[String, Seq[String]] = {
+    val ret = new mutable.HashMap[String, Seq[String]]()
+    topics.foreach { topic =>
+      // get the partitions that exist for topic
+      val partitions = getChildrenParentMayNotExist(zkClient, getTopicPartitionsPath(topic))
+      debug("children of /brokers/topics/%s are %s".format(topic, partitions))
+      ret += (topic -> partitions.sortWith((s,t) => s < t))
     }
     ret
   }
 
-  def setupPartition(zkClient : ZkClient, brokerId: Int, host: String, port: Int, topic: String, nParts: Int) {
-    val brokerIdPath = BrokerIdsPath + "/" + brokerId
-    val broker = new Broker(brokerId, brokerId.toString, host, port)
-    createEphemeralPathExpectConflict(zkClient, brokerIdPath, broker.getZKString)
-    val brokerPartTopicPath = BrokerTopicsPath + "/" + topic + "/" + brokerId
-    createEphemeralPathExpectConflict(zkClient, brokerPartTopicPath, nParts.toString)    
-  }
-
   def deletePartition(zkClient : ZkClient, brokerId: Int, topic: String) {
     val brokerIdPath = BrokerIdsPath + "/" + brokerId
     zkClient.delete(brokerIdPath)
Index: core/src/main/scala/kafka/utils/UpdateOffsetsInZK.scala
===================================================================
--- core/src/main/scala/kafka/utils/UpdateOffsetsInZK.scala	(revision 1294440)
+++ core/src/main/scala/kafka/utils/UpdateOffsetsInZK.scala	(working copy)
@@ -19,12 +19,11 @@
 
 import org.I0Itec.zkclient.ZkClient
 import kafka.consumer.{SimpleConsumer, ConsumerConfig}
-import kafka.cluster.Partition
 import kafka.api.OffsetRequest
 import java.lang.IllegalStateException
 
 /**
- *  A utility that updates the offset of every broker partition to the offset of latest log segment file, in ZK.
+ *  A utility that updates the offset of every broker partition to the offset of earliest or latest log segment file, in ZK.
  */
 object UpdateOffsetsInZK {
   val Earliest = "earliest"
@@ -46,7 +45,7 @@
   private def getAndSetOffsets(zkClient: ZkClient, offsetOption: Long, config: ConsumerConfig, topic: String): Unit = {
     val cluster = ZkUtils.getCluster(zkClient)
     val partitionsPerTopicMap = ZkUtils.getPartitionsForTopics(zkClient, List(topic).iterator)
-    var partitions: List[String] = Nil
+    var partitions: Seq[String] = Nil
 
     partitionsPerTopicMap.get(topic) match {
       case Some(l) =>  partitions = l.sortWith((s,t) => s < t)
@@ -54,22 +53,29 @@
     }
 
     var numParts = 0
-    for (partString <- partitions) {
-      val part = Partition.parse(partString)
-      val broker = cluster.getBroker(part.brokerId) match {
+    for (partition <- partitions) {
+      val brokerHostingPartition = ZkUtils.getLeaderForPartition(zkClient, topic, partition.toInt)
+
+      val broker = brokerHostingPartition match {
         case Some(b) => b
-        case None => throw new IllegalStateException("Broker " + part.brokerId + " is unavailable. Cannot issue " +
+        case None => throw new IllegalStateException("Broker " + brokerHostingPartition + " is unavailable. Cannot issue " +
           "getOffsetsBefore request")
       }
-      val consumer = new SimpleConsumer(broker.host, broker.port, 10000, 100 * 1024)
-      val offsets = consumer.getOffsetsBefore(topic, part.partId, offsetOption, 1)
+
+      val brokerInfos = ZkUtils.getBrokerInfoFromIds(zkClient, List(broker))
+      if(brokerInfos.size == 0)
+        throw new IllegalStateException("Broker information for broker id %d does not exist in ZK".format(broker))
+
+      val brokerInfo = brokerInfos.head
+      val consumer = new SimpleConsumer(brokerInfo.host, brokerInfo.port, 10000, 100 * 1024)
+      val offsets = consumer.getOffsetsBefore(topic, partition.toInt, offsetOption, 1)
       val topicDirs = new ZKGroupTopicDirs(config.groupId, topic)
-      
-      println("updating partition " + part.name + " with new offset: " + offsets(0))
-      ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + part.name, offsets(0).toString)
+
+      println("updating partition " + partition + " with new offset: " + offsets(0))
+      ZkUtils.updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + partition, offsets(0).toString)
       numParts += 1
     }
-    println("updated the offset for " + numParts + " partitions")    
+    println("updated the offset for " + numParts + " partitions")
   }
 
   private def usage() = {
Index: core/src/main/scala/kafka/utils/KafkaScheduler.scala
===================================================================
--- core/src/main/scala/kafka/utils/KafkaScheduler.scala	(revision 1294440)
+++ core/src/main/scala/kafka/utils/KafkaScheduler.scala	(working copy)
@@ -19,6 +19,7 @@
 
 import java.util.concurrent._
 import java.util.concurrent.atomic._
+import java.lang.IllegalStateException
 
 /**
  * A scheduler for running jobs in the background
@@ -26,25 +27,41 @@
  */
 class KafkaScheduler(val numThreads: Int, val baseThreadName: String, isDaemon: Boolean) extends Logging {
   private val threadId = new AtomicLong(0)
-  private val executor = new ScheduledThreadPoolExecutor(numThreads, new ThreadFactory() {
-    def newThread(runnable: Runnable): Thread = {
-      val t = new Thread(runnable, baseThreadName + threadId.getAndIncrement)
-      t.setDaemon(isDaemon)
-      t
-    }
-  })
-  executor.setContinueExistingPeriodicTasksAfterShutdownPolicy(false)
-  executor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false)
+  private var executor:ScheduledThreadPoolExecutor = null
+  startUp
 
-  def scheduleWithRate(fun: () => Unit, delayMs: Long, periodMs: Long) =
+  def startUp = {
+    executor = new ScheduledThreadPoolExecutor(numThreads, new ThreadFactory() {
+      def newThread(runnable: Runnable): Thread = {
+        val t = new Thread(runnable, baseThreadName + threadId.getAndIncrement)
+        t.setDaemon(isDaemon)
+        t
+      }
+    })
+    executor.setContinueExistingPeriodicTasksAfterShutdownPolicy(false)
+    executor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false)
+  }
+
+  def hasShutdown: Boolean = executor.isShutdown
+
+  private def checkIfExecutorHasStarted = {
+    if(executor == null)
+      throw new IllegalStateException("Kafka scheduler has not been started")
+  }
+
+  def scheduleWithRate(fun: () => Unit, delayMs: Long, periodMs: Long) = {
+    checkIfExecutorHasStarted
     executor.scheduleAtFixedRate(Utils.loggedRunnable(fun), delayMs, periodMs, TimeUnit.MILLISECONDS)
+  }
 
   def shutdownNow() {
+    checkIfExecutorHasStarted
     executor.shutdownNow()
     info("force shutdown scheduler " + baseThreadName)
   }
 
   def shutdown() {
+    checkIfExecutorHasStarted
     executor.shutdown()
     info("shutdown scheduler " + baseThreadName)
   }
Index: core/src/main/scala/kafka/server/KafkaZooKeeper.scala
===================================================================
--- core/src/main/scala/kafka/server/KafkaZooKeeper.scala	(revision 1294440)
+++ core/src/main/scala/kafka/server/KafkaZooKeeper.scala	(working copy)
@@ -51,21 +51,6 @@
     ZkUtils.registerBrokerInZk(zkClient, config.brokerId, hostName, creatorId, config.port)
   }
 
-  def registerTopicInZk(topic: String) {
-    registerTopicInZkInternal(topic)
-    lock synchronized {
-      topics ::= topic
-    }
-  }
-
-  def registerTopicInZkInternal(topic: String) {
-    val brokerTopicPath = ZkUtils.BrokerTopicsPath + "/" + topic + "/" + config.brokerId
-    val numParts = logManager.getTopicPartitionsMap.getOrElse(topic, config.numPartitions)
-    info("Begin registering broker topic " + brokerTopicPath + " with " + numParts.toString + " partitions")
-    ZkUtils.createEphemeralPathExpectConflict(zkClient, brokerTopicPath, numParts.toString)
-    info("End registering broker topic " + brokerTopicPath)
-  }
-
   /**
    *  When we get a SessionExpired event, we lost all ephemeral nodes and zkclient has reestablished a
    *  connection for us. We need to re-register this broker in the broker registry.
@@ -87,11 +72,6 @@
     def handleNewSession() {
       info("re-registering broker info in ZK for broker " + config.brokerId)
       registerBrokerInZk()
-      lock synchronized {
-        info("re-registering broker topics in ZK for broker " + config.brokerId)
-        for (topic <- topics)
-          registerTopicInZkInternal(topic)
-      }
       info("done re-registering broker")
     }
   }
Index: core/src/main/scala/kafka/server/KafkaServer.scala
===================================================================
--- core/src/main/scala/kafka/server/KafkaServer.scala	(revision 1294440)
+++ core/src/main/scala/kafka/server/KafkaServer.scala	(working copy)
@@ -20,7 +20,7 @@
 import java.util.concurrent._
 import java.util.concurrent.atomic._
 import java.io.File
-import kafka.utils.{Mx4jLoader, Utils, SystemTime, KafkaScheduler, Logging}
+import kafka.utils.{Mx4jLoader, Utils, SystemTime, Logging}
 import kafka.network.{SocketServerStats, SocketServer}
 import kafka.log.LogManager
 
@@ -36,7 +36,6 @@
   private val statsMBeanName = "kafka:type=kafka.SocketServerStats"
   var socketServer: SocketServer = null
   var requestHandlerPool: KafkaRequestHandlerPool = null
-  val scheduler = new KafkaScheduler(1, "kafka-logcleaner-", false)
   private var logManager: LogManager = null
 
   /**
@@ -52,7 +51,6 @@
       cleanShutDownFile.delete
     }
     logManager = new LogManager(config,
-                                scheduler,
                                 SystemTime,
                                 1000L * 60 * config.logCleanupIntervalMinutes,
                                 1000L * 60 * 60 * config.logRetentionHours,
@@ -85,7 +83,6 @@
     val canShutdown = isShuttingDown.compareAndSet(false, true);
     if (canShutdown) {
       info("Shutting down Kafka server")
-      scheduler.shutdown()
       if (socketServer != null)
         socketServer.shutdown()
       if(requestHandlerPool != null)
Index: core/src/main/scala/kafka/server/KafkaApis.scala
===================================================================
--- core/src/main/scala/kafka/server/KafkaApis.scala	(revision 1294440)
+++ core/src/main/scala/kafka/server/KafkaApis.scala	(working copy)
@@ -173,6 +173,7 @@
           }
       }
     }
+    info("Sending response for topic metadata request")
     Some(new TopicMetadataSend(topicsMetadata))
   }
 }
Index: core/src/main/scala/kafka/api/TopicMetadata.scala
===================================================================
--- core/src/main/scala/kafka/api/TopicMetadata.scala	(revision 1294440)
+++ core/src/main/scala/kafka/api/TopicMetadata.scala	(working copy)
@@ -148,8 +148,8 @@
   }
 }
 
-case class PartitionMetadata(partitionId: Int, leader: Option[Broker], replicas: Seq[Broker], isr: Seq[Broker],
-                             logMetadata: Option[LogMetadata]) {
+case class PartitionMetadata(partitionId: Int, leader: Option[Broker], replicas: Seq[Broker], isr: Seq[Broker] = Seq.empty,
+                             logMetadata: Option[LogMetadata] = None) {
   def sizeInBytes: Int = {
     var size: Int = 4 /* partition id */ + 1 /* if leader exists*/
 
Index: core/src/main/scala/kafka/api/TopicMetadataRequest.scala
===================================================================
--- core/src/main/scala/kafka/api/TopicMetadataRequest.scala	(revision 1294440)
+++ core/src/main/scala/kafka/api/TopicMetadataRequest.scala	(working copy)
@@ -72,6 +72,7 @@
   def serializeTopicMetadata(topicMetadata: Seq[TopicMetadata]): ByteBuffer = {
     val size = topicMetadata.foldLeft(4 /* num topics */)(_ + _.sizeInBytes)
     val buffer = ByteBuffer.allocate(size)
+    debug("Allocating buffer of size %d for topic metadata response".format(size))
     /* number of topics */
     buffer.putInt(topicMetadata.size)
     /* topic partition_metadata */
@@ -122,13 +123,16 @@
 }
 
 class TopicMetadataSend(topicsMetadata: Seq[TopicMetadata]) extends Send {
-  private var size: Int = topicsMetadata.foldLeft(0)(_ + _.sizeInBytes)
+  private var size: Int = topicsMetadata.foldLeft(4)(_ + _.sizeInBytes)
   private val header = ByteBuffer.allocate(6)
-  val metadata = TopicMetadataRequest.serializeTopicMetadata(topicsMetadata)
   header.putInt(size + 2)
   header.putShort(ErrorMapping.NoError.asInstanceOf[Short])
   header.rewind()
 
+  val metadata = TopicMetadataRequest.serializeTopicMetadata(topicsMetadata)
+  metadata.rewind()
+
+  trace("Wrote size %d in header".format(size + 2))
   var complete: Boolean = false
 
   def writeTo(channel: GatheringByteChannel): Int = {
@@ -136,9 +140,13 @@
     var written = 0
     if(header.hasRemaining)
       written += channel.write(header)
+    trace("Wrote %d bytes for header".format(written))
+
     if(!header.hasRemaining && metadata.hasRemaining)
       written += channel.write(metadata)
 
+    trace("Wrote %d bytes for header and metadata".format(written))
+
     if(!metadata.hasRemaining)
       complete = true
     written
