You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by jj...@apache.org on 2014/08/06 00:59:58 UTC

[01/37] git commit: KAFKA-1096 An old controller coming out of long GC could update its epoch to the latest controller's epoch; reviewed by Neha Narkhede

Repository: kafka
Updated Branches:
  refs/heads/transactional_messaging 6b0ae4bba -> 7a67a7226


KAFKA-1096 An old controller coming out of long GC could update its epoch to the latest controller's epoch; reviewed by Neha Narkhede


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/62f20870
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/62f20870
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/62f20870

Branch: refs/heads/transactional_messaging
Commit: 62f208704fe7fb08f085d569a26e780e0050dba0
Parents: 6b0ae4b
Author: Sriharsha Chintalapani <sc...@hortonworks.com>
Authored: Thu Jun 26 16:16:03 2014 -0700
Committer: Neha Narkhede <ne...@gmail.com>
Committed: Thu Jun 26 16:16:15 2014 -0700

----------------------------------------------------------------------
 .../kafka/controller/KafkaController.scala      | 53 +++++---------------
 1 file changed, 12 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/62f20870/core/src/main/scala/kafka/controller/KafkaController.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala
index 94bbd33..a7a21df 100644
--- a/core/src/main/scala/kafka/controller/KafkaController.scala
+++ b/core/src/main/scala/kafka/controller/KafkaController.scala
@@ -169,8 +169,6 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
   private val preferredReplicaPartitionLeaderSelector = new PreferredReplicaPartitionLeaderSelector(controllerContext)
   private val controlledShutdownPartitionLeaderSelector = new ControlledShutdownLeaderSelector(controllerContext)
   private val brokerRequestBatch = new ControllerBrokerRequestBatch(this)
-  registerControllerChangedListener()
-
   newGauge(
     "ActiveControllerCount",
     new Gauge[Int] {
@@ -298,6 +296,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
   def onControllerFailover() {
     if(isRunning) {
       info("Broker %d starting become controller state transition".format(config.brokerId))
+      //read controller epoch from zk
+      readControllerEpochFromZookeeper()
       // increment the controller epoch
       incrementControllerEpoch(zkClient)
       // before reading source of truth from zookeeper, register the listeners to get broker/topic callbacks
@@ -346,6 +346,8 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
         controllerContext.controllerChannelManager.shutdown()
         controllerContext.controllerChannelManager = null
       }
+      controllerContext.epoch=0
+      controllerContext.epochZkVersion=0
       brokerState.newState(RunningAsBroker)
     }
   }
@@ -875,8 +877,14 @@ class KafkaController(val config : KafkaConfig, zkClient: ZkClient, val brokerSt
     zkClient.subscribeDataChanges(ZkUtils.PreferredReplicaLeaderElectionPath, new PreferredReplicaElectionListener(this))
   }
 
-  private def registerControllerChangedListener() {
-    zkClient.subscribeDataChanges(ZkUtils.ControllerEpochPath, new ControllerEpochListener(this))
+  private def readControllerEpochFromZookeeper() {
+    // initialize the controller epoch and zk version by reading from zookeeper
+    if(ZkUtils.pathExists(controllerContext.zkClient, ZkUtils.ControllerEpochPath)) {
+      val epochData = ZkUtils.readData(controllerContext.zkClient, ZkUtils.ControllerEpochPath)
+      controllerContext.epoch = epochData._1.toInt
+      controllerContext.epochZkVersion = epochData._2.getVersion
+      info("Initialized controller epoch to %d and zk version %d".format(controllerContext.epoch, controllerContext.epochZkVersion))
+    }
   }
 
   def removePartitionFromReassignedPartitions(topicAndPartition: TopicAndPartition) {
@@ -1275,43 +1283,6 @@ class PreferredReplicaElectionListener(controller: KafkaController) extends IZkD
   }
 }
 
-class ControllerEpochListener(controller: KafkaController) extends IZkDataListener with Logging {
-  this.logIdent = "[ControllerEpochListener on " + controller.config.brokerId + "]: "
-  val controllerContext = controller.controllerContext
-  readControllerEpochFromZookeeper()
-
-  /**
-   * Invoked when a controller updates the epoch value
-   * @throws Exception On any error.
-   */
-  @throws(classOf[Exception])
-  def handleDataChange(dataPath: String, data: Object) {
-    debug("Controller epoch listener fired with new epoch " + data.toString)
-    inLock(controllerContext.controllerLock) {
-      // read the epoch path to get the zk version
-      readControllerEpochFromZookeeper()
-    }
-  }
-
-  /**
-   * @throws Exception
-   *             On any error.
-   */
-  @throws(classOf[Exception])
-  def handleDataDeleted(dataPath: String) {
-  }
-
-  private def readControllerEpochFromZookeeper() {
-    // initialize the controller epoch and zk version by reading from zookeeper
-    if(ZkUtils.pathExists(controllerContext.zkClient, ZkUtils.ControllerEpochPath)) {
-      val epochData = ZkUtils.readData(controllerContext.zkClient, ZkUtils.ControllerEpochPath)
-      controllerContext.epoch = epochData._1.toInt
-      controllerContext.epochZkVersion = epochData._2.getVersion
-      info("Initialized controller epoch to %d and zk version %d".format(controllerContext.epoch, controllerContext.epochZkVersion))
-    }
-  }
-}
-
 case class ReassignedPartitionsContext(var newReplicas: Seq[Int] = Seq.empty,
                                        var isrChangeListener: ReassignedPartitionsIsrChangeListener = null)
 


[02/37] git commit: KAFKA-1491; Always read coordinator information in consumer metadata response; reviewed by Neha Narkhede.

Posted by jj...@apache.org.
KAFKA-1491; Always read coordinator information in consumer metadata response; reviewed by Neha Narkhede.


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/c4b95641
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/c4b95641
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/c4b95641

Branch: refs/heads/transactional_messaging
Commit: c4b95641ea295478f9480b264652b7b52d8f71c5
Parents: 62f2087
Author: Joel Koshy <jj...@gmail.com>
Authored: Fri Jun 27 11:09:18 2014 -0700
Committer: Joel Koshy <jj...@gmail.com>
Committed: Fri Jun 27 11:09:18 2014 -0700

----------------------------------------------------------------------
 core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala | 3 ++-
 core/src/main/scala/kafka/client/ClientUtils.scala           | 5 +++++
 2 files changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/c4b95641/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
index f8cf6c3..c72ca14 100644
--- a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
+++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
@@ -29,8 +29,9 @@ object ConsumerMetadataResponse {
   def readFrom(buffer: ByteBuffer) = {
     val correlationId = buffer.getInt
     val errorCode = buffer.getShort
+    val broker = Broker.readFrom(buffer)
     val coordinatorOpt = if (errorCode == ErrorMapping.NoError)
-      Some(Broker.readFrom(buffer))
+      Some(broker)
     else
       None
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/c4b95641/core/src/main/scala/kafka/client/ClientUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala
index ba5fbdc..ce7ede3 100644
--- a/core/src/main/scala/kafka/client/ClientUtils.scala
+++ b/core/src/main/scala/kafka/client/ClientUtils.scala
@@ -160,6 +160,11 @@ object ClientUtils extends Logging{
            debug("Consumer metadata response: " + consumerMetadataResponse.toString)
            if (consumerMetadataResponse.errorCode == ErrorMapping.NoError)
              coordinatorOpt = consumerMetadataResponse.coordinatorOpt
+           else {
+             debug("Query to %s:%d to locate offset manager for %s failed - will retry in %d milliseconds."
+                  .format(queryChannel.host, queryChannel.port, group, retryBackOffMs))
+             Thread.sleep(retryBackOffMs)
+           }
          }
          catch {
            case ioe: IOException =>


[33/37] git commit: kafka-1562; kafka-topics.sh alter add partitions resets cleanup.policy; patched by Jonathan Natkins; reviewed by Jun Rao

Posted by jj...@apache.org.
kafka-1562; kafka-topics.sh alter add partitions resets cleanup.policy; patched by Jonathan Natkins; reviewed by Jun Rao


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/f8d521a9
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/f8d521a9
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/f8d521a9

Branch: refs/heads/transactional_messaging
Commit: f8d521a9616134d78966419e5cf2aa73e8d6a5c5
Parents: a01a101
Author: Jonathan Natkins <na...@wibidata.com>
Authored: Mon Aug 4 07:21:25 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Mon Aug 4 07:21:25 2014 -0700

----------------------------------------------------------------------
 .../src/main/scala/kafka/admin/AdminUtils.scala | 10 +++-
 .../main/scala/kafka/admin/TopicCommand.scala   |  4 +-
 .../unit/kafka/admin/TopicCommandTest.scala     | 63 ++++++++++++++++++++
 3 files changed, 73 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/f8d521a9/core/src/main/scala/kafka/admin/AdminUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala
index b5d8714..94c5332 100644
--- a/core/src/main/scala/kafka/admin/AdminUtils.scala
+++ b/core/src/main/scala/kafka/admin/AdminUtils.scala
@@ -96,8 +96,14 @@ object AdminUtils extends Logging {
   * @param numPartitions Number of partitions to be set
   * @param replicaAssignmentStr Manual replica assignment
   * @param checkBrokerAvailable Ignore checking if assigned replica broker is available. Only used for testing
+  * @param config Pre-existing properties that should be preserved
   */
-  def addPartitions(zkClient: ZkClient, topic: String, numPartitions: Int = 1, replicaAssignmentStr: String = "", checkBrokerAvailable: Boolean = true) {
+  def addPartitions(zkClient: ZkClient,
+                    topic: String,
+                    numPartitions: Int = 1,
+                    replicaAssignmentStr: String = "",
+                    checkBrokerAvailable: Boolean = true,
+                    config: Properties = new Properties) {
     val existingPartitionsReplicaList = ZkUtils.getReplicaAssignmentForTopics(zkClient, List(topic))
     if (existingPartitionsReplicaList.size == 0)
       throw new AdminOperationException("The topic %s does not exist".format(topic))
@@ -124,7 +130,7 @@ object AdminUtils extends Logging {
     val partitionReplicaList = existingPartitionsReplicaList.map(p => p._1.partition -> p._2)
     // add the new list
     partitionReplicaList ++= newPartitionReplicaList
-    AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, update = true)
+    AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkClient, topic, partitionReplicaList, config, true)
   }
 
   def getManualReplicaAssignment(replicaAssignmentList: String, availableBrokerList: Set[Int], startPartitionId: Int, checkBrokerAvailable: Boolean = true): Map[Int, List[Int]] = {

http://git-wip-us.apache.org/repos/asf/kafka/blob/f8d521a9/core/src/main/scala/kafka/admin/TopicCommand.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala
index 8d5c2e7..003a09c 100644
--- a/core/src/main/scala/kafka/admin/TopicCommand.scala
+++ b/core/src/main/scala/kafka/admin/TopicCommand.scala
@@ -95,11 +95,11 @@ object TopicCommand {
   def alterTopic(zkClient: ZkClient, opts: TopicCommandOptions) {
     val topics = getTopics(zkClient, opts)
     topics.foreach { topic =>
+      val configs = AdminUtils.fetchTopicConfig(zkClient, topic)
       if(opts.options.has(opts.configOpt) || opts.options.has(opts.deleteConfigOpt)) {
         val configsToBeAdded = parseTopicConfigsToBeAdded(opts)
         val configsToBeDeleted = parseTopicConfigsToBeDeleted(opts)
         // compile the final set of configs
-        val configs = AdminUtils.fetchTopicConfig(zkClient, topic)
         configs.putAll(configsToBeAdded)
         configsToBeDeleted.foreach(config => configs.remove(config))
         AdminUtils.changeTopicConfig(zkClient, topic, configs)
@@ -113,7 +113,7 @@ object TopicCommand {
           "logic or ordering of the messages will be affected")
         val nPartitions = opts.options.valueOf(opts.partitionsOpt).intValue
         val replicaAssignmentStr = opts.options.valueOf(opts.replicaAssignmentOpt)
-        AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr)
+        AdminUtils.addPartitions(zkClient, topic, nPartitions, replicaAssignmentStr, config = configs)
         println("Adding partitions succeeded!")
       }
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/f8d521a9/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
new file mode 100644
index 0000000..ac6dd20
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
@@ -0,0 +1,63 @@
+/**
+ * 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.admin
+
+import junit.framework.Assert._
+import org.junit.Test
+import org.scalatest.junit.JUnit3Suite
+import kafka.utils.Logging
+import kafka.utils.TestUtils
+import kafka.zk.ZooKeeperTestHarness
+import kafka.server.KafkaConfig
+import kafka.admin.TopicCommand.TopicCommandOptions
+import kafka.utils.ZkUtils
+
+class TopicCommandTest extends JUnit3Suite with ZooKeeperTestHarness with Logging {
+
+  @Test
+  def testConfigPreservationAcrossPartitionAlteration() {
+    val topic = "test"
+    val numPartitionsOriginal = 1
+    val cleanupKey = "cleanup.policy"
+    val cleanupVal = "compact"
+    // create brokers
+    val brokers = List(0, 1, 2)
+    TestUtils.createBrokersInZk(zkClient, brokers)
+    // create the topic
+    val createOpts = new TopicCommandOptions(Array("--partitions", numPartitionsOriginal.toString,
+      "--replication-factor", "1",
+      "--config", cleanupKey + "=" + cleanupVal,
+      "--topic", topic))
+    TopicCommand.createTopic(zkClient, createOpts)
+    val props = AdminUtils.fetchTopicConfig(zkClient, topic)
+    assertTrue("Properties after creation don't contain " + cleanupKey, props.containsKey(cleanupKey))
+    assertTrue("Properties after creation have incorrect value", props.getProperty(cleanupKey).equals(cleanupVal))
+
+    // pre-create the topic config changes path to avoid a NoNodeException
+    ZkUtils.createPersistentPath(zkClient, ZkUtils.TopicConfigChangesPath)
+
+    // modify the topic to add new partitions
+    val numPartitionsModified = 3
+    val alterOpts = new TopicCommandOptions(Array("--partitions", numPartitionsModified.toString,
+      "--config", cleanupKey + "=" + cleanupVal,
+      "--topic", topic))
+    TopicCommand.alterTopic(zkClient, alterOpts)
+    val newProps = AdminUtils.fetchTopicConfig(zkClient, topic)
+    assertTrue("Updated properties do not contain " + cleanupKey, newProps.containsKey(cleanupKey))
+    assertTrue("Updated properties have incorrect value", newProps.getProperty(cleanupKey).equals(cleanupVal))
+  }
+}
\ No newline at end of file


[24/37] git commit: kafka-1533; transient unit test failure in ProducerFailureHandlingTest; reviewed by Guozhang Wang; reviewed by Jun Rao

Posted by jj...@apache.org.
kafka-1533; transient unit test failure in ProducerFailureHandlingTest; reviewed by Guozhang Wang; reviewed by Jun Rao


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/ff05e9b3
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/ff05e9b3
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/ff05e9b3

Branch: refs/heads/transactional_messaging
Commit: ff05e9b3616a222e29a42f6e8fdf41945a417f41
Parents: 014b700
Author: Guozhang Wang <gu...@linkedin.com>
Authored: Tue Jul 22 14:14:19 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Tue Jul 22 14:14:19 2014 -0700

----------------------------------------------------------------------
 .../org/apache/kafka/clients/NetworkClient.java | 13 ++-
 .../clients/producer/internals/Metadata.java    |  7 ++
 .../kafka/api/ProducerFailureHandlingTest.scala | 88 ++++++++------------
 .../kafka/api/ProducerSendTest.scala            | 56 +++++--------
 .../integration/KafkaServerTestHarness.scala    |  2 +
 .../test/scala/unit/kafka/utils/TestUtils.scala |  2 +-
 6 files changed, 78 insertions(+), 90 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/ff05e9b3/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
index d8f9ce6..eea270a 100644
--- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
@@ -77,6 +77,9 @@ public class NetworkClient implements KafkaClient {
     /* true iff there is a metadata request that has been sent and for which we have not yet received a response */
     private boolean metadataFetchInProgress;
 
+    /* the last timestamp when no broker node is available to connect */
+    private long lastNoNodeAvailableMs;
+
     public NetworkClient(Selectable selector,
                          Metadata metadata,
                          String clientId,
@@ -94,6 +97,7 @@ public class NetworkClient implements KafkaClient {
         this.correlation = 0;
         this.nodeIndexOffset = new Random().nextInt(Integer.MAX_VALUE);
         this.metadataFetchInProgress = false;
+        this.lastNoNodeAvailableMs = 0;
     }
 
     /**
@@ -162,7 +166,10 @@ public class NetworkClient implements KafkaClient {
         }
 
         // should we update our metadata?
-        long metadataTimeout = metadata.timeToNextUpdate(now);
+        long timeToNextMetadataUpdate = metadata.timeToNextUpdate(now);
+        long timeToNextReconnectAttempt = this.lastNoNodeAvailableMs + metadata.refreshBackoff() - now;
+        // if there is no node available to connect, back off refreshing metadata
+        long metadataTimeout = Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt);
         if (!this.metadataFetchInProgress && metadataTimeout == 0)
             maybeUpdateMetadata(sends, now);
 
@@ -354,6 +361,8 @@ public class NetworkClient implements KafkaClient {
         Node node = this.leastLoadedNode(now);
         if (node == null) {
             log.debug("Give up sending metadata request since no node is available");
+            // mark the timestamp for no node available to connect
+            this.lastNoNodeAvailableMs = now;
             return;
         }
 
@@ -367,7 +376,7 @@ public class NetworkClient implements KafkaClient {
             this.inFlightRequests.add(metadataRequest);
         } else if (connectionStates.canConnect(node.id(), now)) {
             // we don't have a connection to this node right now, make one
-            log.debug("Give up sending metadata request to node {} since it is either not connected or cannot have more in flight requests", node.id());
+            log.debug("Init connection to node {} for sending metadata request in the next iteration", node.id());
             initiateConnect(node, now);
         }
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/ff05e9b3/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
index 4aa5b01..1d30f9e 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
@@ -139,4 +139,11 @@ public final class Metadata {
     public synchronized long lastUpdate() {
         return this.lastRefreshMs;
     }
+
+    /**
+     * The metadata refresh backoff in ms
+     */
+    public long refreshBackoff() {
+        return refreshBackoffMs;
+    }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/ff05e9b3/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
index 15fd5bc..789e74c 100644
--- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
@@ -21,26 +21,31 @@ import org.scalatest.junit.JUnit3Suite
 import org.junit.Test
 import org.junit.Assert._
 
-import java.util.{Random, Properties}
+import java.util.Random
 import java.lang.Integer
 import java.util.concurrent.{TimeoutException, TimeUnit, ExecutionException}
 
-import kafka.server.{KafkaConfig, KafkaServer}
-import kafka.utils.{ShutdownableThread, Utils, TestUtils}
-import kafka.zk.ZooKeeperTestHarness
+import kafka.server.KafkaConfig
+import kafka.utils.{TestZKUtils, ShutdownableThread, TestUtils}
+import kafka.integration.KafkaServerTestHarness
 import kafka.consumer.SimpleConsumer
 
 import org.apache.kafka.common.KafkaException
 import org.apache.kafka.clients.producer._
 
-class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness {
-  private val brokerId1 = 0
-  private val brokerId2 = 1
-  private val ports = TestUtils.choosePorts(2)
-  private val (port1, port2) = (ports(0), ports(1))
-  private var server1: KafkaServer = null
-  private var server2: KafkaServer = null
-  private var servers = List.empty[KafkaServer]
+class ProducerFailureHandlingTest extends JUnit3Suite with KafkaServerTestHarness {
+  private val producerBufferSize = 30000
+  private val serverMessageMaxBytes =  producerBufferSize/2
+
+  val numServers = 2
+  val configs =
+    for(props <- TestUtils.createBrokerConfigs(numServers, false))
+    yield new KafkaConfig(props) {
+      override val zkConnect = TestZKUtils.zookeeperConnect
+      override val autoCreateTopicsEnable = false
+      override val messageMaxBytes = serverMessageMaxBytes
+    }
+
 
   private var consumer1: SimpleConsumer = null
   private var consumer2: SimpleConsumer = null
@@ -50,32 +55,19 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
   private var producer3: KafkaProducer = null
   private var producer4: KafkaProducer = null
 
-  private val props1 = TestUtils.createBrokerConfig(brokerId1, port1, false)
-  private val props2 = TestUtils.createBrokerConfig(brokerId2, port2, false)
-  props1.put("auto.create.topics.enable", "false")
-  props2.put("auto.create.topics.enable", "false")
-  private val config1 = new KafkaConfig(props1)
-  private val config2 = new KafkaConfig(props2)
-  private val brokerList = TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2))
-
-  private val bufferSize = 2 * config1.messageMaxBytes
-
   private val topic1 = "topic-1"
   private val topic2 = "topic-2"
 
   override def setUp() {
     super.setUp()
-    server1 = TestUtils.createServer(config1)
-    server2 = TestUtils.createServer(config2)
-    servers = List(server1,server2)
 
     // TODO: we need to migrate to new consumers when 0.9 is final
-    consumer1 = new SimpleConsumer("localhost", port1, 100, 1024*1024, "")
-    consumer2 = new SimpleConsumer("localhost", port2, 100, 1024*1024, "")
+    consumer1 = new SimpleConsumer("localhost", configs(0).port, 100, 1024*1024, "")
+    consumer2 = new SimpleConsumer("localhost", configs(1).port, 100, 1024*1024, "")
 
-    producer1 = TestUtils.createNewProducer(brokerList, acks = 0, blockOnBufferFull = false, bufferSize = bufferSize);
-    producer2 = TestUtils.createNewProducer(brokerList, acks = 1, blockOnBufferFull = false, bufferSize = bufferSize)
-    producer3 = TestUtils.createNewProducer(brokerList, acks = -1, blockOnBufferFull = false, bufferSize = bufferSize)
+    producer1 = TestUtils.createNewProducer(brokerList, acks = 0, blockOnBufferFull = false, bufferSize = producerBufferSize);
+    producer2 = TestUtils.createNewProducer(brokerList, acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize)
+    producer3 = TestUtils.createNewProducer(brokerList, acks = -1, blockOnBufferFull = false, bufferSize = producerBufferSize)
   }
 
   override def tearDown() {
@@ -87,9 +79,6 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
     if (producer3 != null) producer3.close
     if (producer4 != null) producer4.close
 
-    server1.shutdown; Utils.rm(server1.config.logDirs)
-    server2.shutdown; Utils.rm(server2.config.logDirs)
-
     super.tearDown()
   }
 
@@ -102,7 +91,7 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
     TestUtils.createTopic(zkClient, topic1, 1, 2, servers)
 
     // send a too-large record
-    val record = new ProducerRecord(topic1, null, "key".getBytes, new Array[Byte](config1.messageMaxBytes + 1))
+    val record = new ProducerRecord(topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1))
     assertEquals("Returned metadata should have offset -1", producer1.send(record).get.offset, -1L)
   }
 
@@ -115,7 +104,7 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
     TestUtils.createTopic(zkClient, topic1, 1, 2, servers)
 
     // send a too-large record
-    val record = new ProducerRecord(topic1, null, "key".getBytes, new Array[Byte](config1.messageMaxBytes + 1))
+    val record = new ProducerRecord(topic1, null, "key".getBytes, new Array[Byte](serverMessageMaxBytes + 1))
     intercept[ExecutionException] {
       producer2.send(record).get
     }
@@ -149,7 +138,7 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
     TestUtils.createTopic(zkClient, topic1, 1, 2, servers)
 
     // producer with incorrect broker list
-    producer4 = TestUtils.createNewProducer("localhost:8686,localhost:4242", acks = 1, blockOnBufferFull = false, bufferSize = bufferSize)
+    producer4 = TestUtils.createNewProducer("localhost:8686,localhost:4242", acks = 1, blockOnBufferFull = false, bufferSize = producerBufferSize)
 
     // send a record with incorrect broker list
     val record = new ProducerRecord(topic1, null, "key".getBytes, "value".getBytes)
@@ -175,8 +164,7 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
 
     // stop IO threads and request handling, but leave networking operational
     // any requests should be accepted and queue up, but not handled
-    server1.requestHandlerPool.shutdown()
-    server2.requestHandlerPool.shutdown()
+    servers.foreach(server => server.requestHandlerPool.shutdown())
 
     producer1.send(record1).get(5000, TimeUnit.MILLISECONDS)
 
@@ -186,11 +174,11 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
 
     // TODO: expose producer configs after creating them
     // send enough messages to get buffer full
-    val msgSize = 10000
+    val tooManyRecords = 10
+    val msgSize = producerBufferSize / tooManyRecords
     val value = new Array[Byte](msgSize)
     new Random().nextBytes(value)
     val record2 = new ProducerRecord(topic1, null, "key".getBytes, value)
-    val tooManyRecords = bufferSize / ("key".getBytes.length + value.length)
 
     intercept[KafkaException] {
       for (i <- 1 to tooManyRecords)
@@ -269,17 +257,13 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
 
     // rolling bounce brokers
     for (i <- 0 until 2) {
-      server1.shutdown()
-      server1.awaitShutdown()
-      server1.startup
-
-      Thread.sleep(2000)
+      for (server <- servers) {
+        server.shutdown()
+        server.awaitShutdown()
+        server.startup
 
-      server2.shutdown()
-      server2.awaitShutdown()
-      server2.startup
-
-      Thread.sleep(2000)
+        Thread.sleep(2000)
+      }
 
       // Make sure the producer do not see any exception
       // in returned metadata due to broker failures
@@ -298,7 +282,7 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
     // double check that the leader info has been propagated after consecutive bounces
     val leader = TestUtils.waitUntilMetadataIsPropagated(servers, topic1, partition)
 
-    val fetchResponse = if(leader == server1.config.brokerId) {
+    val fetchResponse = if(leader == configs(0).brokerId) {
       consumer1.fetch(new FetchRequestBuilder().addFetch(topic1, partition, 0, Int.MaxValue).build()).messageSet(topic1, partition)
     } else {
       consumer2.fetch(new FetchRequestBuilder().addFetch(topic1, partition, 0, Int.MaxValue).build()).messageSet(topic1, partition)
@@ -317,7 +301,7 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
     var sent = 0
     var failed = false
 
-    val producer = TestUtils.createNewProducer(brokerList, bufferSize = bufferSize, retries = 10)
+    val producer = TestUtils.createNewProducer(brokerList, bufferSize = producerBufferSize, retries = 10)
 
     override def doWork(): Unit = {
       val responses =

http://git-wip-us.apache.org/repos/asf/kafka/blob/ff05e9b3/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala
index 34a7db4..d407af9 100644
--- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala
@@ -17,7 +17,6 @@
 
 package kafka.api.test
 
-import java.util.Properties
 import java.lang.{Integer, IllegalArgumentException}
 
 import org.apache.kafka.clients.producer._
@@ -25,53 +24,41 @@ import org.scalatest.junit.JUnit3Suite
 import org.junit.Test
 import org.junit.Assert._
 
-import kafka.server.{KafkaConfig, KafkaServer}
-import kafka.utils.{Utils, TestUtils}
-import kafka.zk.ZooKeeperTestHarness
+import kafka.server.KafkaConfig
+import kafka.utils.{TestZKUtils, TestUtils}
 import kafka.consumer.SimpleConsumer
 import kafka.api.FetchRequestBuilder
 import kafka.message.Message
+import kafka.integration.KafkaServerTestHarness
 
 
-class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness {
-  private val brokerId1 = 0
-  private val brokerId2 = 1
-  private val ports = TestUtils.choosePorts(2)
-  private val (port1, port2) = (ports(0), ports(1))
-  private var server1: KafkaServer = null
-  private var server2: KafkaServer = null
-  private var servers = List.empty[KafkaServer]
+class ProducerSendTest extends JUnit3Suite with KafkaServerTestHarness {
+  val numServers = 2
+  val configs =
+    for(props <- TestUtils.createBrokerConfigs(numServers, false))
+    yield new KafkaConfig(props) {
+      override val zkConnect = TestZKUtils.zookeeperConnect
+      override val numPartitions = 4
+    }
 
   private var consumer1: SimpleConsumer = null
   private var consumer2: SimpleConsumer = null
 
-  private val props1 = TestUtils.createBrokerConfig(brokerId1, port1, false)
-  private val props2 = TestUtils.createBrokerConfig(brokerId2, port2, false)
-  props1.put("num.partitions", "4")
-  props2.put("num.partitions", "4")
-  private val config1 = new KafkaConfig(props1)
-  private val config2 = new KafkaConfig(props2)
-
   private val topic = "topic"
   private val numRecords = 100
 
   override def setUp() {
     super.setUp()
-    // set up 2 brokers with 4 partitions each
-    server1 = TestUtils.createServer(config1)
-    server2 = TestUtils.createServer(config2)
-    servers = List(server1,server2)
 
     // TODO: we need to migrate to new consumers when 0.9 is final
-    consumer1 = new SimpleConsumer("localhost", port1, 100, 1024*1024, "")
-    consumer2 = new SimpleConsumer("localhost", port2, 100, 1024*1024, "")
+    consumer1 = new SimpleConsumer("localhost", configs(0).port, 100, 1024*1024, "")
+    consumer2 = new SimpleConsumer("localhost", configs(1).port, 100, 1024*1024, "")
   }
 
   override def tearDown() {
-    server1.shutdown
-    server2.shutdown
-    Utils.rm(server1.config.logDirs)
-    Utils.rm(server2.config.logDirs)
+    consumer1.close()
+    consumer2.close()
+
     super.tearDown()
   }
 
@@ -90,7 +77,7 @@ class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness {
    */
   @Test
   def testSendOffset() {
-    var producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)))
+    var producer = TestUtils.createNewProducer(brokerList)
 
     val callback = new CheckErrorCallback
 
@@ -146,7 +133,7 @@ class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness {
    */
   @Test
   def testClose() {
-    var producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)))
+    var producer = TestUtils.createNewProducer(brokerList)
 
     try {
       // create topic
@@ -182,7 +169,7 @@ class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness {
    */
   @Test
   def testSendToPartition() {
-    var producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)))
+    var producer = TestUtils.createNewProducer(brokerList)
 
     try {
       // create topic
@@ -209,7 +196,7 @@ class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness {
       }
 
       // make sure the fetched messages also respect the partitioning and ordering
-      val fetchResponse1 = if(leader1.get == server1.config.brokerId) {
+      val fetchResponse1 = if(leader1.get == configs(0).brokerId) {
         consumer1.fetch(new FetchRequestBuilder().addFetch(topic, partition, 0, Int.MaxValue).build())
       } else {
         consumer2.fetch(new FetchRequestBuilder().addFetch(topic, partition, 0, Int.MaxValue).build())
@@ -237,8 +224,7 @@ class ProducerSendTest extends JUnit3Suite with ZooKeeperTestHarness {
    */
   @Test
   def testAutoCreateTopic() {
-    var producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromConfigs(Seq(config1, config2)),
-                                            retries = 5)
+    var producer = TestUtils.createNewProducer(brokerList, retries = 5)
 
     try {
       // Send a message to auto-create the topic

http://git-wip-us.apache.org/repos/asf/kafka/blob/ff05e9b3/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
index 194dd70..3cf7c9b 100644
--- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
+++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
@@ -30,11 +30,13 @@ trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness {
 
   val configs: List[KafkaConfig]
   var servers: List[KafkaServer] = null
+  var brokerList: String = null
 
   override def setUp() {
     super.setUp
     if(configs.size <= 0)
       throw new KafkaException("Must suply at least one server config.")
+    brokerList = TestUtils.getBrokerListStrFromConfigs(configs)
     servers = configs.map(TestUtils.createServer(_))
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/ff05e9b3/core/src/test/scala/unit/kafka/utils/TestUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index 3faa884..4d01d25 100644
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -385,7 +385,7 @@ object TestUtils extends Logging {
     producerProps.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, blockOnBufferFull.toString)
     producerProps.put(ProducerConfig.BUFFER_MEMORY_CONFIG, bufferSize.toString)
     producerProps.put(ProducerConfig.RETRIES_CONFIG, retries.toString)
-    producerProps.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, "1000")
+    producerProps.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, "100")
     producerProps.put(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG, "200")
     return new KafkaProducer(producerProps)
   }


[19/37] kafka-1462; Add new request and response formats for the new consumer and coordinator communication; patched by Jun Rao; reviewed by Guozhang Wang and Jay Kreps

Posted by jj...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java
index 2652c32..7d90fce 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java
@@ -12,6 +12,7 @@
  */
 package org.apache.kafka.common.requests;
 
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -20,50 +21,112 @@ import java.util.Map;
 import org.apache.kafka.common.Cluster;
 import org.apache.kafka.common.Node;
 import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
-public class MetadataResponse {
+public class MetadataResponse extends AbstractRequestResponse {
+    private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id);
+    private static String BROKERS_KEY_NAME = "brokers";
+    private static String TOPIC_METATDATA_KEY_NAME = "topic_metadata";
+
+    // broker level field names
+    private static String NODE_ID_KEY_NAME = "node_id";
+    private static String HOST_KEY_NAME = "host";
+    private static String PORT_KEY_NAME = "port";
+
+    // topic level field names
+    private static String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code";
+    private static String TOPIC_KEY_NAME = "topic";
+    private static String PARTITION_METADATA_KEY_NAME = "partition_metadata";
+
+    // partition level field names
+    private static String PARTITION_ERROR_CODE_KEY_NAME = "partition_error_code";
+    private static String PARTITION_KEY_NAME = "partition_id";
+    private static String LEADER_KEY_NAME = "leader";
+    private static String REPLICAS_KEY_NAME = "replicas";
+    private static String ISR_KEY_NAME = "isr";
 
     private final Cluster cluster;
     private final Map<String, Errors> errors;
 
-    public MetadataResponse(Cluster cluster, Map<String, Errors> errors) {
+    public MetadataResponse(Cluster cluster) {
+        super(new Struct(curSchema));
+
+        List<Struct> brokerArray = new ArrayList<Struct>();
+        for (Node node: cluster.nodes()) {
+            Struct broker = struct.instance(BROKERS_KEY_NAME);
+            broker.set(NODE_ID_KEY_NAME, node.id());
+            broker.set(HOST_KEY_NAME, node.host());
+            broker.set(PORT_KEY_NAME, node.port());
+            brokerArray.add(broker);
+        }
+        struct.set(BROKERS_KEY_NAME, brokerArray.toArray());
+
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (String topic: cluster.topics()) {
+            Struct topicData = struct.instance(TOPIC_METATDATA_KEY_NAME);
+            topicData.set(TOPIC_ERROR_CODE_KEY_NAME, (short)0);  // no error
+            topicData.set(TOPIC_KEY_NAME, topic);
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (PartitionInfo fetchPartitionData : cluster.partitionsForTopic(topic)) {
+                Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME);
+                partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, (short)0);  // no error
+                partitionData.set(PARTITION_KEY_NAME, fetchPartitionData.partition());
+                partitionData.set(LEADER_KEY_NAME, fetchPartitionData.leader().id());
+                ArrayList<Integer> replicas = new ArrayList<Integer>();
+                for (Node node: fetchPartitionData.replicas())
+                    replicas.add(node.id());
+                partitionData.set(REPLICAS_KEY_NAME, replicas.toArray());
+                ArrayList<Integer> isr = new ArrayList<Integer>();
+                for (Node node: fetchPartitionData.inSyncReplicas())
+                    isr.add(node.id());
+                partitionData.set(ISR_KEY_NAME, isr.toArray());
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITION_METADATA_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(TOPIC_METATDATA_KEY_NAME, topicArray.toArray());
+
         this.cluster = cluster;
-        this.errors = errors;
+        this.errors = new HashMap<String, Errors>();
     }
 
     public MetadataResponse(Struct struct) {
+        super(struct);
         Map<String, Errors> errors = new HashMap<String, Errors>();
         Map<Integer, Node> brokers = new HashMap<Integer, Node>();
-        Object[] brokerStructs = (Object[]) struct.get("brokers");
+        Object[] brokerStructs = (Object[]) struct.get(BROKERS_KEY_NAME);
         for (int i = 0; i < brokerStructs.length; i++) {
             Struct broker = (Struct) brokerStructs[i];
-            int nodeId = (Integer) broker.get("node_id");
-            String host = (String) broker.get("host");
-            int port = (Integer) broker.get("port");
+            int nodeId = broker.getInt(NODE_ID_KEY_NAME);
+            String host = broker.getString(HOST_KEY_NAME);
+            int port = broker.getInt(PORT_KEY_NAME);
             brokers.put(nodeId, new Node(nodeId, host, port));
         }
         List<PartitionInfo> partitions = new ArrayList<PartitionInfo>();
-        Object[] topicInfos = (Object[]) struct.get("topic_metadata");
+        Object[] topicInfos = (Object[]) struct.get(TOPIC_METATDATA_KEY_NAME);
         for (int i = 0; i < topicInfos.length; i++) {
             Struct topicInfo = (Struct) topicInfos[i];
-            short topicError = topicInfo.getShort("topic_error_code");
-            String topic = topicInfo.getString("topic");
+            short topicError = topicInfo.getShort(TOPIC_ERROR_CODE_KEY_NAME);
+            String topic = topicInfo.getString(TOPIC_KEY_NAME);
             if (topicError == Errors.NONE.code()) {
-                Object[] partitionInfos = (Object[]) topicInfo.get("partition_metadata");
+                Object[] partitionInfos = (Object[]) topicInfo.get(PARTITION_METADATA_KEY_NAME);
                 for (int j = 0; j < partitionInfos.length; j++) {
                     Struct partitionInfo = (Struct) partitionInfos[j];
-                    short partError = partitionInfo.getShort("partition_error_code");
+                    short partError = partitionInfo.getShort(PARTITION_ERROR_CODE_KEY_NAME);
                     if (partError == Errors.NONE.code()) {
-                        int partition = partitionInfo.getInt("partition_id");
-                        int leader = partitionInfo.getInt("leader");
+                        int partition = partitionInfo.getInt(PARTITION_KEY_NAME);
+                        int leader = partitionInfo.getInt(LEADER_KEY_NAME);
                         Node leaderNode = leader == -1 ? null : brokers.get(leader);
-                        Object[] replicas = (Object[]) partitionInfo.get("replicas");
+                        Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME);
                         Node[] replicaNodes = new Node[replicas.length];
                         for (int k = 0; k < replicas.length; k++)
                             replicaNodes[k] = brokers.get(replicas[k]);
-                        Object[] isr = (Object[]) partitionInfo.get("isr");
+                        Object[] isr = (Object[]) partitionInfo.get(ISR_KEY_NAME);
                         Node[] isrNodes = new Node[isr.length];
                         for (int k = 0; k < isr.length; k++)
                             isrNodes[k] = brokers.get(isr[k]);
@@ -86,4 +149,7 @@ public class MetadataResponse {
         return this.cluster;
     }
 
+    public static MetadataResponse parse(ByteBuffer buffer) {
+        return new MetadataResponse(((Struct) curSchema.read(buffer)));
+    }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java
new file mode 100644
index 0000000..3ee5cba
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java
@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This wrapper supports both v0 and v1 of OffsetCommitRequest.
+ */
+public class OffsetCommitRequest extends AbstractRequestResponse {
+    public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_COMMIT.id);
+    private static String GROUP_ID_KEY_NAME = "group_id";
+    private static String GENERATION_ID_KEY_NAME = "group_generation_id";
+    private static String CONSUMER_ID_KEY_NAME = "consumer_id";
+    private static String TOPICS_KEY_NAME = "topics";
+
+    // topic level field names
+    private static String TOPIC_KEY_NAME = "topic";
+    private static String PARTITIONS_KEY_NAME = "partitions";
+
+    // partition level field names
+    private static String PARTITION_KEY_NAME = "partition";
+    private static String COMMIT_OFFSET_KEY_NAME = "offset";
+    private static String TIMESTAMP_KEY_NAME = "timestamp";
+    private static String METADATA_KEY_NAME = "metadata";
+
+    public static final int DEFAULT_GENERATION_ID = -1;
+    public static final String DEFAULT_CONSUMER_ID = "";
+
+    private final String groupId;
+    private final int generationId;
+    private final String consumerId;
+    private final Map<TopicPartition, PartitionData> offsetData;
+
+    public static final class PartitionData {
+        public final long offset;
+        public final long timestamp;
+        public final String metadata;
+
+        public PartitionData(long offset, long timestamp, String metadata) {
+            this.offset = offset;
+            this.timestamp = timestamp;
+            this.metadata = metadata;
+        }
+    }
+
+    /**
+     * Constructor for version 0.
+     * @param groupId
+     * @param offsetData
+     */
+    @Deprecated
+    public OffsetCommitRequest(String groupId, Map<TopicPartition, PartitionData> offsetData) {
+        super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 0)));
+        initCommonFields(groupId, offsetData);
+        this.groupId = groupId;
+        this.generationId = DEFAULT_GENERATION_ID;
+        this.consumerId = DEFAULT_CONSUMER_ID;
+        this.offsetData = offsetData;
+    }
+
+    /**
+     * Constructor for version 1.
+     * @param groupId
+     * @param generationId
+     * @param consumerId
+     * @param offsetData
+     */
+    public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map<TopicPartition, PartitionData> offsetData) {
+        super(new Struct(curSchema));
+
+        initCommonFields(groupId, offsetData);
+        struct.set(GENERATION_ID_KEY_NAME, generationId);
+        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
+        this.groupId = groupId;
+        this.generationId = generationId;
+        this.consumerId = consumerId;
+        this.offsetData = offsetData;
+    }
+
+    private void initCommonFields(String groupId, Map<TopicPartition, PartitionData> offsetData) {
+        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(offsetData);
+
+        struct.set(GROUP_ID_KEY_NAME, groupId);
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
+            Struct topicData = struct.instance(TOPICS_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
+                PartitionData fetchPartitionData = partitionEntry.getValue();
+                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset);
+                partitionData.set(TIMESTAMP_KEY_NAME, fetchPartitionData.timestamp);
+                partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata);
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
+    }
+
+    public OffsetCommitRequest(Struct struct) {
+        super(struct);
+        offsetData = new HashMap<TopicPartition, PartitionData>();
+        for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
+            Struct topicResponse = (Struct) topicResponseObj;
+            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                long offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME);
+                long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME);
+                String metadata = partitionResponse.getString(METADATA_KEY_NAME);
+                PartitionData partitionData = new PartitionData(offset, timestamp, metadata);
+                offsetData.put(new TopicPartition(topic, partition), partitionData);
+            }
+        }
+        groupId = struct.getString(GROUP_ID_KEY_NAME);
+        // This field only exists in v1.
+        if (struct.hasField(GENERATION_ID_KEY_NAME))
+            generationId = struct.getInt(GENERATION_ID_KEY_NAME);
+        else
+            generationId = DEFAULT_GENERATION_ID;
+
+        // This field only exists in v1.
+        if (struct.hasField(CONSUMER_ID_KEY_NAME))
+            consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
+        else
+            consumerId = DEFAULT_CONSUMER_ID;
+    }
+
+    public String groupId() {
+        return groupId;
+    }
+
+    public int generationId() {
+        return generationId;
+    }
+
+    public String consumerId() {
+        return consumerId;
+    }
+
+    public Map<TopicPartition, PartitionData> offsetData() {
+        return offsetData;
+    }
+
+    public static OffsetCommitRequest parse(ByteBuffer buffer, int versionId) {
+        Schema schema = ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, versionId);
+        return new OffsetCommitRequest(((Struct) schema.read(buffer)));
+    }
+
+    public static OffsetCommitRequest parse(ByteBuffer buffer) {
+        return new OffsetCommitRequest(((Struct) curSchema.read(buffer)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
new file mode 100644
index 0000000..711232a
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class OffsetCommitResponse extends AbstractRequestResponse {
+    public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_COMMIT.id);
+    private static String RESPONSES_KEY_NAME = "responses";
+
+    // topic level fields
+    private static String TOPIC_KEY_NAME = "topic";
+    private static String PARTITIONS_KEY_NAME = "partition_responses";
+
+    // partition level fields
+    private static String PARTITION_KEY_NAME = "partition";
+    private static String ERROR_CODE_KEY_NAME = "error_code";
+
+    private final Map<TopicPartition, Short> responseData;
+
+    public OffsetCommitResponse(Map<TopicPartition, Short> responseData) {
+        super(new Struct(curSchema));
+
+        Map<String, Map<Integer, Short>> topicsData = CollectionUtils.groupDataByTopic(responseData);
+
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, Map<Integer, Short>> entries: topicsData.entrySet()) {
+            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, entries.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, Short> partitionEntry : entries.getValue().entrySet()) {
+                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(ERROR_CODE_KEY_NAME, partitionEntry.getValue());
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
+        this.responseData = responseData;
+    }
+
+    public OffsetCommitResponse(Struct struct) {
+        super(struct);
+        responseData = new HashMap<TopicPartition, Short>();
+        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
+            Struct topicResponse = (Struct) topicResponseObj;
+            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
+                responseData.put(new TopicPartition(topic, partition), errorCode);
+            }
+        }
+    }
+
+    public Map<TopicPartition, Short> responseData() {
+        return responseData;
+    }
+
+    public static OffsetCommitResponse parse(ByteBuffer buffer) {
+        return new OffsetCommitResponse(((Struct) curSchema.read(buffer)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
new file mode 100644
index 0000000..90d5135
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
@@ -0,0 +1,98 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This wrapper supports both v0 and v1 of OffsetFetchRequest.
+ */
+public class OffsetFetchRequest extends AbstractRequestResponse {
+    public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_FETCH.id);
+    private static String GROUP_ID_KEY_NAME = "group_id";
+    private static String TOPICS_KEY_NAME = "topics";
+
+    // topic level field names
+    private static String TOPIC_KEY_NAME = "topic";
+    private static String PARTITIONS_KEY_NAME = "partitions";
+
+    // partition level field names
+    private static String PARTITION_KEY_NAME = "partition";
+
+    public static final int DEFAULT_GENERATION_ID = -1;
+    public static final String DEFAULT_CONSUMER_ID = "";
+
+    private final String groupId;
+    private final List<TopicPartition> partitions;
+
+    public OffsetFetchRequest(String groupId, List<TopicPartition> partitions) {
+        super(new Struct(curSchema));
+
+        Map<String, List<Integer>> topicsData = CollectionUtils.groupDataByTopic(partitions);
+
+        struct.set(GROUP_ID_KEY_NAME, groupId);
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, List<Integer>> entries: topicsData.entrySet()) {
+            Struct topicData = struct.instance(TOPICS_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, entries.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Integer partiitonId : entries.getValue()) {
+                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                partitionData.set(PARTITION_KEY_NAME, partiitonId);
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
+        this.groupId = groupId;
+        this.partitions = partitions;
+    }
+
+    public OffsetFetchRequest(Struct struct) {
+        super(struct);
+        partitions = new ArrayList<TopicPartition>();
+        for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
+            Struct topicResponse = (Struct) topicResponseObj;
+            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                partitions.add(new TopicPartition(topic, partition));
+            }
+        }
+        groupId = struct.getString(GROUP_ID_KEY_NAME);
+   }
+
+    public String groupId() {
+        return groupId;
+    }
+
+    public List<TopicPartition> partitions() {
+        return partitions;
+    }
+
+    public static OffsetFetchRequest parse(ByteBuffer buffer) {
+        return new OffsetFetchRequest(((Struct) curSchema.read(buffer)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
new file mode 100644
index 0000000..6b7c269
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
@@ -0,0 +1,107 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class OffsetFetchResponse extends AbstractRequestResponse {
+    public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_FETCH.id);
+    private static String RESPONSES_KEY_NAME = "responses";
+
+    // topic level fields
+    private static String TOPIC_KEY_NAME = "topic";
+    private static String PARTITIONS_KEY_NAME = "partition_responses";
+
+    // partition level fields
+    private static String PARTITION_KEY_NAME = "partition";
+    private static String COMMIT_OFFSET_KEY_NAME = "offset";
+    private static String METADATA_KEY_NAME = "metadata";
+    private static String ERROR_CODE_KEY_NAME = "error_code";
+
+    private final Map<TopicPartition,PartitionData> responseData;
+
+    public static final class PartitionData {
+        public final long offset;
+        public final String metadata;
+        public final short errorCode;
+
+        public PartitionData(long offset, String metadata, short errorCode) {
+            this.offset = offset;
+            this.metadata = metadata;
+            this.errorCode = errorCode;
+        }
+    }
+
+    public OffsetFetchResponse(Map<TopicPartition, PartitionData> responseData) {
+        super(new Struct(curSchema));
+
+        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
+
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, Map<Integer, PartitionData>> entries: topicsData.entrySet()) {
+            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, entries.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, PartitionData> partitionEntry : entries.getValue().entrySet()) {
+                PartitionData fetchPartitionData = partitionEntry.getValue();
+                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(COMMIT_OFFSET_KEY_NAME, fetchPartitionData.offset);
+                partitionData.set(METADATA_KEY_NAME, fetchPartitionData.metadata);
+                partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.errorCode);
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
+        this.responseData = responseData;
+    }
+
+    public OffsetFetchResponse(Struct struct) {
+        super(struct);
+        responseData = new HashMap<TopicPartition, PartitionData>();
+        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
+            Struct topicResponse = (Struct) topicResponseObj;
+            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                long offset = partitionResponse.getLong(COMMIT_OFFSET_KEY_NAME);
+                String metadata = partitionResponse.getString(METADATA_KEY_NAME);
+                short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
+                PartitionData partitionData = new PartitionData(offset, metadata, errorCode);
+                responseData.put(new TopicPartition(topic, partition), partitionData);
+            }
+        }
+    }
+
+    public Map<TopicPartition, PartitionData> responseData() {
+        return responseData;
+    }
+
+    public static OffsetFetchResponse parse(ByteBuffer buffer) {
+        return new OffsetFetchResponse(((Struct) curSchema.read(buffer)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
index 6036f6a..3dbba8a 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
@@ -1,71 +1,105 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+
 package org.apache.kafka.common.requests;
 
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.utils.CollectionUtils;
+
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.protocol.ApiKeys;
-import org.apache.kafka.common.protocol.ProtoUtils;
-import org.apache.kafka.common.protocol.types.Struct;
-import org.apache.kafka.common.record.MemoryRecords;
+public class ProduceRequest  extends AbstractRequestResponse {
+    public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id);
+    private static String ACKS_KEY_NAME = "acks";
+    private static String TIMEOUT_KEY_NAME = "timeout";
+    private static String TOPIC_DATA_KEY_NAME = "topic_data";
+
+    // topic level field names
+    private static String TOPIC_KEY_NAME = "topic";
+    private static String PARTITION_DATA_KEY_NAME = "data";
 
-public class ProduceRequest {
+    // partition level field names
+    private static String PARTITION_KEY_NAME = "partition";
+    private static String RECORD_SET_KEY_NAME = "record_set";
 
     private final short acks;
     private final int timeout;
-    private final Map<String, List<PartitionRecords>> records;
+    private final Map<TopicPartition, ByteBuffer> partitionRecords;
 
-    public ProduceRequest(short acks, int timeout) {
+    public ProduceRequest(short acks, int timeout, Map<TopicPartition, ByteBuffer> partitionRecords) {
+        super(new Struct(curSchema));
+        Map<String, Map<Integer, ByteBuffer>> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords);
+        struct.set(ACKS_KEY_NAME, acks);
+        struct.set(TIMEOUT_KEY_NAME, timeout);
+        List<Struct> topicDatas = new ArrayList<Struct>(recordsByTopic.size());
+        for (Map.Entry<String, Map<Integer, ByteBuffer>> entry : recordsByTopic.entrySet()) {
+            Struct topicData = struct.instance(TOPIC_DATA_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, entry.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, ByteBuffer> partitionEntry : entry.getValue().entrySet()) {
+                ByteBuffer buffer = partitionEntry.getValue().duplicate();
+                Struct part = topicData.instance(PARTITION_DATA_KEY_NAME)
+                                       .set(PARTITION_KEY_NAME, partitionEntry.getKey())
+                                       .set(RECORD_SET_KEY_NAME, buffer);
+                partitionArray.add(part);
+            }
+            topicData.set(PARTITION_DATA_KEY_NAME, partitionArray.toArray());
+            topicDatas.add(topicData);
+        }
+        struct.set(TOPIC_DATA_KEY_NAME, topicDatas.toArray());
         this.acks = acks;
         this.timeout = timeout;
-        this.records = new HashMap<String, List<PartitionRecords>>();
+        this.partitionRecords = partitionRecords;
     }
 
-    public void add(TopicPartition tp, MemoryRecords recs) {
-        List<PartitionRecords> found = this.records.get(tp.topic());
-        if (found == null) {
-            found = new ArrayList<PartitionRecords>();
-            records.put(tp.topic(), found);
+    public ProduceRequest(Struct struct) {
+        super(struct);
+        partitionRecords = new HashMap<TopicPartition, ByteBuffer>();
+        for (Object topicDataObj : struct.getArray(TOPIC_DATA_KEY_NAME)) {
+            Struct topicData = (Struct) topicDataObj;
+            String topic = topicData.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicData.getArray(PARTITION_DATA_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                ByteBuffer records = partitionResponse.getBytes(RECORD_SET_KEY_NAME);
+                partitionRecords.put(new TopicPartition(topic, partition), records);
+            }
         }
-        found.add(new PartitionRecords(tp, recs));
+        acks = struct.getShort(ACKS_KEY_NAME);
+        timeout = struct.getInt(TIMEOUT_KEY_NAME);
     }
 
-    public Struct toStruct() {
-        Struct produce = new Struct(ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id));
-        produce.set("acks", acks);
-        produce.set("timeout", timeout);
-        List<Struct> topicDatas = new ArrayList<Struct>(records.size());
-        for (Map.Entry<String, List<PartitionRecords>> entry : records.entrySet()) {
-            Struct topicData = produce.instance("topic_data");
-            topicData.set("topic", entry.getKey());
-            List<PartitionRecords> parts = entry.getValue();
-            Object[] partitionData = new Object[parts.size()];
-            for (int i = 0; i < parts.size(); i++) {
-                ByteBuffer buffer = parts.get(i).records.buffer();
-                buffer.flip();
-                Struct part = topicData.instance("data")
-                                       .set("partition", parts.get(i).topicPartition.partition())
-                                       .set("record_set", buffer);
-                partitionData[i] = part;
-            }
-            topicData.set("data", partitionData);
-            topicDatas.add(topicData);
-        }
-        produce.set("topic_data", topicDatas.toArray());
-        return produce;
+    public short acks() {
+        return acks;
     }
 
-    private static final class PartitionRecords {
-        public final TopicPartition topicPartition;
-        public final MemoryRecords records;
+    public int timeout() {
+        return timeout;
+    }
 
-        public PartitionRecords(TopicPartition topicPartition, MemoryRecords records) {
-            this.topicPartition = topicPartition;
-            this.records = records;
-        }
+    public Map<TopicPartition, ByteBuffer> partitionRecords() {
+        return partitionRecords;
     }
 
+    public static ProduceRequest parse(ByteBuffer buffer) {
+        return new ProduceRequest(((Struct) curSchema.read(buffer)));
+    }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
index 6cf4fb7..5220464 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
@@ -12,67 +12,83 @@
  */
 package org.apache.kafka.common.requests;
 
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.protocol.types.Struct;
+public class ProduceResponse extends AbstractRequestResponse {
+    private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id);
+    private static String RESPONSES_KEY_NAME = "responses";
+
+    // topic level field names
+    private static String TOPIC_KEY_NAME = "topic";
+    private static String PARTITION_RESPONSES_KEY_NAME = "partition_responses";
 
-public class ProduceResponse {
+    // partition level field names
+    private static String PARTITION_KEY_NAME = "partition";
+    private static String ERROR_CODE_KEY_NAME = "error_code";
+    private static String BASE_OFFSET_KEY_NAME = "base_offset";
 
     private final Map<TopicPartition, PartitionResponse> responses;
 
-    public ProduceResponse() {
-        this.responses = new HashMap<TopicPartition, PartitionResponse>();
+    public ProduceResponse(Map<TopicPartition, PartitionResponse> responses) {
+        super(new Struct(curSchema));
+        Map<String, Map<Integer, PartitionResponse>> responseByTopic = CollectionUtils.groupDataByTopic(responses);
+        List<Struct> topicDatas = new ArrayList<Struct>(responseByTopic.size());
+        for (Map.Entry<String, Map<Integer, PartitionResponse>> entry : responseByTopic.entrySet()) {
+            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, entry.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, PartitionResponse> partitionEntry : entry.getValue().entrySet()) {
+                PartitionResponse part = partitionEntry.getValue();
+                Struct partStruct = topicData.instance(PARTITION_RESPONSES_KEY_NAME)
+                                       .set(PARTITION_KEY_NAME, partitionEntry.getKey())
+                                       .set(ERROR_CODE_KEY_NAME, part.errorCode)
+                                       .set(BASE_OFFSET_KEY_NAME, part.baseOffset);
+                partitionArray.add(partStruct);
+            }
+            topicData.set(PARTITION_RESPONSES_KEY_NAME, partitionArray.toArray());
+            topicDatas.add(topicData);
+        }
+        struct.set(RESPONSES_KEY_NAME, topicDatas.toArray());
+        this.responses = responses;
     }
 
     public ProduceResponse(Struct struct) {
+        super(struct);
         responses = new HashMap<TopicPartition, PartitionResponse>();
-        for (Object topicResponse : (Object[]) struct.get("responses")) {
+        for (Object topicResponse : struct.getArray("responses")) {
             Struct topicRespStruct = (Struct) topicResponse;
-            String topic = (String) topicRespStruct.get("topic");
-            for (Object partResponse : (Object[]) topicRespStruct.get("partition_responses")) {
+            String topic = topicRespStruct.getString("topic");
+            for (Object partResponse : topicRespStruct.getArray("partition_responses")) {
                 Struct partRespStruct = (Struct) partResponse;
-                int partition = (Integer) partRespStruct.get("partition");
-                short errorCode = (Short) partRespStruct.get("error_code");
-                long offset = (Long) partRespStruct.get("base_offset");
+                int partition = partRespStruct.getInt("partition");
+                short errorCode = partRespStruct.getShort("error_code");
+                long offset = partRespStruct.getLong("base_offset");
                 TopicPartition tp = new TopicPartition(topic, partition);
-                responses.put(tp, new PartitionResponse(partition, errorCode, offset));
+                responses.put(tp, new PartitionResponse(errorCode, offset));
             }
         }
     }
 
-    public void addResponse(TopicPartition tp, int partition, short error, long baseOffset) {
-        this.responses.put(tp, new PartitionResponse(partition, error, baseOffset));
-    }
-
     public Map<TopicPartition, PartitionResponse> responses() {
         return this.responses;
     }
 
-    @Override
-    public String toString() {
-        StringBuilder b = new StringBuilder();
-        b.append('{');
-        boolean isFirst = true;
-        for (Map.Entry<TopicPartition, PartitionResponse> entry : responses.entrySet()) {
-            if (isFirst)
-                isFirst = false;
-            else
-                b.append(',');
-            b.append(entry.getKey() + " : " + entry.getValue());
-        }
-        b.append('}');
-        return b.toString();
-    }
-
-    public static class PartitionResponse {
-        public int partitionId;
+    public static final class PartitionResponse {
         public short errorCode;
         public long baseOffset;
 
-        public PartitionResponse(int partitionId, short errorCode, long baseOffset) {
-            this.partitionId = partitionId;
+        public PartitionResponse(short errorCode, long baseOffset) {
             this.errorCode = errorCode;
             this.baseOffset = baseOffset;
         }
@@ -81,9 +97,7 @@ public class ProduceResponse {
         public String toString() {
             StringBuilder b = new StringBuilder();
             b.append('{');
-            b.append("pid: ");
-            b.append(partitionId);
-            b.append(",error: ");
+            b.append("error: ");
             b.append(errorCode);
             b.append(",offset: ");
             b.append(baseOffset);
@@ -91,4 +105,8 @@ public class ProduceResponse {
             return b.toString();
         }
     }
+
+    public static ProduceResponse parse(ByteBuffer buffer) {
+        return new ProduceResponse(((Struct) curSchema.read(buffer)));
+    }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java
index 66cc2fe..f459a2a 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java
@@ -24,18 +24,24 @@ import org.apache.kafka.common.protocol.types.Struct;
 /**
  * The header for a request in the Kafka protocol
  */
-public class RequestHeader {
+public class RequestHeader extends AbstractRequestResponse {
 
     private static Field API_KEY_FIELD = REQUEST_HEADER.get("api_key");
     private static Field API_VERSION_FIELD = REQUEST_HEADER.get("api_version");
     private static Field CLIENT_ID_FIELD = REQUEST_HEADER.get("client_id");
     private static Field CORRELATION_ID_FIELD = REQUEST_HEADER.get("correlation_id");
 
-    private final Struct header;
+    private final short apiKey;
+    private final short apiVersion;
+    private final String clientId;
+    private final int correlationId;
 
     public RequestHeader(Struct header) {
-        super();
-        this.header = header;
+        super(header);
+        apiKey = struct.getShort(API_KEY_FIELD);
+        apiVersion = struct.getShort(API_VERSION_FIELD);
+        clientId = struct.getString(CLIENT_ID_FIELD);
+        correlationId = struct.getInt(CORRELATION_ID_FIELD);
     }
 
     public RequestHeader(short apiKey, String client, int correlation) {
@@ -43,43 +49,34 @@ public class RequestHeader {
     }
 
     public RequestHeader(short apiKey, short version, String client, int correlation) {
-        this(new Struct(Protocol.REQUEST_HEADER));
-        this.header.set(API_KEY_FIELD, apiKey);
-        this.header.set(API_VERSION_FIELD, version);
-        this.header.set(CLIENT_ID_FIELD, client);
-        this.header.set(CORRELATION_ID_FIELD, correlation);
+        super(new Struct(Protocol.REQUEST_HEADER));
+        struct.set(API_KEY_FIELD, apiKey);
+        struct.set(API_VERSION_FIELD, version);
+        struct.set(CLIENT_ID_FIELD, client);
+        struct.set(CORRELATION_ID_FIELD, correlation);
+        this.apiKey = apiKey;
+        this.apiVersion = version;
+        this.clientId = client;
+        this.correlationId = correlation;
     }
 
     public short apiKey() {
-        return (Short) this.header.get(API_KEY_FIELD);
+        return apiKey;
     }
 
     public short apiVersion() {
-        return (Short) this.header.get(API_VERSION_FIELD);
+        return apiVersion;
     }
 
     public String clientId() {
-        return (String) this.header.get(CLIENT_ID_FIELD);
+        return clientId;
     }
 
     public int correlationId() {
-        return (Integer) this.header.get(CORRELATION_ID_FIELD);
+        return correlationId;
     }
 
     public static RequestHeader parse(ByteBuffer buffer) {
         return new RequestHeader((Struct) Protocol.REQUEST_HEADER.read(buffer));
     }
-
-    public void writeTo(ByteBuffer buffer) {
-        header.writeTo(buffer);
-    }
-
-    public int sizeOf() {
-        return header.sizeOf();
-    }
-
-    @Override
-    public String toString() {
-        return header.toString();
-    }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java
index 257b828..dd63853 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java
@@ -28,31 +28,25 @@ import org.apache.kafka.common.protocol.types.Struct;
 /**
  * A response header in the kafka protocol.
  */
-public class ResponseHeader {
+public class ResponseHeader extends AbstractRequestResponse {
 
     private static Field CORRELATION_KEY_FIELD = RESPONSE_HEADER.get("correlation_id");
 
-    private final Struct header;
+    private final int correlationId;
 
     public ResponseHeader(Struct header) {
-        this.header = header;
+        super(header);
+        correlationId = struct.getInt(CORRELATION_KEY_FIELD);
     }
 
     public ResponseHeader(int correlationId) {
-        this(new Struct(Protocol.RESPONSE_HEADER));
-        this.header.set(CORRELATION_KEY_FIELD, correlationId);
+        super(new Struct(Protocol.RESPONSE_HEADER));
+        struct.set(CORRELATION_KEY_FIELD, correlationId);
+        this.correlationId = correlationId;
     }
 
     public int correlationId() {
-        return (Integer) header.get(CORRELATION_KEY_FIELD);
-    }
-
-    public void writeTo(ByteBuffer buffer) {
-        header.writeTo(buffer);
-    }
-
-    public int sizeOf() {
-        return header.sizeOf();
+        return correlationId;
     }
 
     public static ResponseHeader parse(ByteBuffer buffer) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java
new file mode 100644
index 0000000..ba38637
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/utils/CollectionUtils.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.utils;
+
+import org.apache.kafka.common.TopicPartition;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class CollectionUtils {
+    /**
+     * group data by topic
+     * @param data Data to be partitioned
+     * @param <T> Partition data type
+     * @return partitioned data
+     */
+    public static <T> Map<String, Map<Integer, T>> groupDataByTopic(Map<TopicPartition, T> data) {
+        Map<String, Map<Integer, T>> dataByTopic = new HashMap<String, Map<Integer, T>>();
+        for (Map.Entry<TopicPartition, T> entry: data.entrySet()) {
+            String topic = entry.getKey().topic();
+            int partition = entry.getKey().partition();
+            Map<Integer, T> topicData = dataByTopic.get(topic);
+            if (topicData == null) {
+                topicData = new HashMap<Integer, T>();
+                dataByTopic.put(topic, topicData);
+            }
+            topicData.put(partition, entry.getValue());
+        }
+        return dataByTopic;
+    }
+
+    /**
+     * group partitions by topic
+     * @param partitions
+     * @return partitions per topic
+     */
+    public static Map<String, List<Integer>> groupDataByTopic(List<TopicPartition> partitions) {
+        Map<String, List<Integer>> partitionsByTopic = new HashMap<String, List<Integer>>();
+        for (TopicPartition tp: partitions) {
+            String topic = tp.topic();
+            List<Integer> topicData = partitionsByTopic.get(topic);
+            if (topicData == null) {
+                topicData = new ArrayList<Integer>();
+                partitionsByTopic.put(topic, topicData);
+            }
+            topicData.add(tp.partition());
+        }
+        return  partitionsByTopic;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
index 2f98192..1a55242 100644
--- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
@@ -7,11 +7,13 @@ import static org.junit.Assert.assertTrue;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 
 import org.apache.kafka.clients.producer.internals.Metadata;
 import org.apache.kafka.common.Cluster;
 import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.network.NetworkReceive;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.ProtoUtils;
@@ -68,7 +70,7 @@ public class NetworkClientTest {
 
     @Test
     public void testSimpleRequestResponse() {
-        ProduceRequest produceRequest = new ProduceRequest((short) 1, 1000);
+        ProduceRequest produceRequest = new ProduceRequest((short) 1, 1000, Collections.<TopicPartition, ByteBuffer>emptyMap());
         RequestHeader reqHeader = client.nextRequestHeader(ApiKeys.PRODUCE);
         RequestSend send = new RequestSend(node.id(), reqHeader, produceRequest.toStruct());
         ClientRequest request = new ClientRequest(time.milliseconds(), true, send, null);

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
new file mode 100644
index 0000000..df37fc6
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
@@ -0,0 +1,173 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.junit.Test;
+
+import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+public class RequestResponseTest {
+
+    @Test
+    public void testSerialization() throws Exception{
+        List<AbstractRequestResponse> requestList = Arrays.asList(
+                createRequestHeader(),
+                createResponseHeader(),
+                createConsumerMetadataRequest(),
+                createConsumerMetadataResponse(),
+                createFetchRequest(),
+                createFetchResponse(),
+                createHeartBeatRequest(),
+                createHeartBeatResponse(),
+                createJoinGroupRequest(),
+                createJoinGroupResponse(),
+                createListOffsetRequest(),
+                createListOffsetResponse(),
+                createMetadataRequest(),
+                createMetadataResponse(),
+                createOffsetCommitRequest(),
+                createOffsetCommitResponse(),
+                createOffsetFetchRequest(),
+                createOffsetFetchResponse(),
+                createProduceRequest(),
+                createProduceResponse());
+
+        for (AbstractRequestResponse req: requestList) {
+            ByteBuffer buffer = ByteBuffer.allocate(req.sizeOf());
+            req.writeTo(buffer);
+            buffer.rewind();
+            Method deserializer = req.getClass().getDeclaredMethod("parse", ByteBuffer.class);
+            AbstractRequestResponse deserialized = (AbstractRequestResponse) deserializer.invoke(null, buffer);
+            assertEquals("The original and deserialized of " + req.getClass().getSimpleName() + " should be the same.", req, deserialized);
+            assertEquals("The original and deserialized of " + req.getClass().getSimpleName() + " should have the same hashcode.",
+                    req.hashCode(), deserialized.hashCode());
+        }
+    }
+
+    private AbstractRequestResponse createRequestHeader() {
+        return new RequestHeader((short)10, (short)1, "", 10);
+    }
+
+    private AbstractRequestResponse createResponseHeader() {
+        return new ResponseHeader(10);
+    }
+
+    private AbstractRequestResponse createConsumerMetadataRequest() {
+        return new ConsumerMetadataRequest("test-group");
+    }
+
+    private AbstractRequestResponse createConsumerMetadataResponse() {
+        return new ConsumerMetadataResponse((short)1, new Node(10, "host1", 2014));
+    }
+
+    private AbstractRequestResponse createFetchRequest() {
+        Map<TopicPartition, FetchRequest.PartitionData> fetchData = new HashMap<TopicPartition, FetchRequest.PartitionData>();
+        fetchData.put(new TopicPartition("test1", 0), new FetchRequest.PartitionData(100, 1000000));
+        fetchData.put(new TopicPartition("test2", 0), new FetchRequest.PartitionData(200, 1000000));
+        return new FetchRequest(-1, 100, 100000, fetchData);
+    }
+
+    private AbstractRequestResponse createFetchResponse() {
+        Map<TopicPartition, FetchResponse.PartitionData> responseData = new HashMap<TopicPartition, FetchResponse.PartitionData>();
+        responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData((short)0, 1000000, ByteBuffer.allocate(10)));
+        return new FetchResponse(responseData);
+    }
+
+    private AbstractRequestResponse createHeartBeatRequest() {
+        return new HeartbeatRequest("group1", 1, "consumer1");
+    }
+
+    private AbstractRequestResponse createHeartBeatResponse() {
+        return new HeartbeatResponse((short)0);
+    }
+
+    private AbstractRequestResponse createJoinGroupRequest() {
+        return new JoinGroupRequest("group1", 30000, Arrays.asList("topic1"), "consumer1", "strategy1");
+    }
+
+    private AbstractRequestResponse createJoinGroupResponse() {
+        return new JoinGroupResponse((short)0, 1, "consumer1", Arrays.asList(new TopicPartition("test11", 1), new TopicPartition("test2", 1)));
+    }
+
+    private AbstractRequestResponse createListOffsetRequest() {
+        Map<TopicPartition, ListOffsetRequest.PartitionData> offsetData = new HashMap<TopicPartition, ListOffsetRequest.PartitionData>();
+        offsetData.put(new TopicPartition("test", 0), new ListOffsetRequest.PartitionData(1000000L, 10));
+        return new ListOffsetRequest(-1, offsetData);
+    }
+
+    private AbstractRequestResponse createListOffsetResponse() {
+        Map<TopicPartition, ListOffsetResponse.PartitionData> responseData = new HashMap<TopicPartition, ListOffsetResponse.PartitionData>();
+        responseData.put(new TopicPartition("test", 0), new ListOffsetResponse.PartitionData((short)0, Arrays.asList(100L)));
+        return new ListOffsetResponse(responseData);
+    }
+
+    private AbstractRequestResponse createMetadataRequest() {
+        return new MetadataRequest(Arrays.asList("topic1"));
+    }
+
+    private AbstractRequestResponse createMetadataResponse() {
+        Node node = new Node(1, "host1", 1001);
+        Node[] replicas = new Node[1];
+        replicas[0] = node;
+        Node[] isr = new Node[1];
+        isr[0] = node;
+        Cluster cluster = new Cluster(Arrays.asList(node), Arrays.asList(new PartitionInfo("topic1", 1, node, replicas, isr)));
+        return new MetadataResponse(cluster);
+    }
+
+    private AbstractRequestResponse createOffsetCommitRequest() {
+        Map<TopicPartition, OffsetCommitRequest.PartitionData> commitData = new HashMap<TopicPartition, OffsetCommitRequest.PartitionData>();
+        commitData.put(new TopicPartition("test", 0), new OffsetCommitRequest.PartitionData(100, 1000000, ""));
+        return new OffsetCommitRequest("group1", 100, "consumer1", commitData);
+    }
+
+    private AbstractRequestResponse createOffsetCommitResponse() {
+        Map<TopicPartition, Short> responseData = new HashMap<TopicPartition, Short>();
+        responseData.put(new TopicPartition("test", 0), (short)0);
+        return new OffsetCommitResponse(responseData);
+    }
+
+    private AbstractRequestResponse createOffsetFetchRequest() {
+        return new OffsetFetchRequest("group1", Arrays.asList(new TopicPartition("test11", 1)));
+    }
+
+    private AbstractRequestResponse createOffsetFetchResponse() {
+        Map<TopicPartition, OffsetFetchResponse.PartitionData> responseData = new HashMap<TopicPartition, OffsetFetchResponse.PartitionData>();
+        responseData.put(new TopicPartition("test", 0), new OffsetFetchResponse.PartitionData(100L, "", (short)0));
+        return new OffsetFetchResponse(responseData);
+    }
+
+    private AbstractRequestResponse createProduceRequest() {
+        Map<TopicPartition, ByteBuffer> produceData = new HashMap<TopicPartition, ByteBuffer>();
+        produceData.put(new TopicPartition("test", 0), ByteBuffer.allocate(10));
+        return new ProduceRequest((short)0, 5000, produceData);
+    }
+
+    private AbstractRequestResponse createProduceResponse() {
+        Map<TopicPartition, ProduceResponse.PartitionResponse> responseData = new HashMap<TopicPartition, ProduceResponse.PartitionResponse>();
+        responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse((short) 0, 10000));
+        return new ProduceResponse(responseData);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala
index dfad6e6..6d00ed0 100644
--- a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala
+++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala
@@ -41,9 +41,9 @@ object ConsumerMetadataRequest {
 
 case class ConsumerMetadataRequest(group: String,
                                    versionId: Short = ConsumerMetadataRequest.CurrentVersion,
-                                   override val correlationId: Int = 0,
+                                   correlationId: Int = 0,
                                    clientId: String = ConsumerMetadataRequest.DefaultClientId)
-  extends RequestOrResponse(Some(RequestKeys.ConsumerMetadataKey), correlationId) {
+  extends RequestOrResponse(Some(RequestKeys.ConsumerMetadataKey)) {
 
   def sizeInBytes =
     2 + /* versionId */

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
index c72ca14..84f6017 100644
--- a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
+++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
@@ -40,8 +40,8 @@ object ConsumerMetadataResponse {
   
 }
 
-case class ConsumerMetadataResponse (coordinatorOpt: Option[Broker], errorCode: Short, override val correlationId: Int = 0)
-  extends RequestOrResponse(correlationId = correlationId) {
+case class ConsumerMetadataResponse (coordinatorOpt: Option[Broker], errorCode: Short, correlationId: Int = 0)
+  extends RequestOrResponse() {
 
   def sizeInBytes =
     4 + /* correlationId */

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala
index 7dacb20..5be393a 100644
--- a/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala
+++ b/core/src/main/scala/kafka/api/ControlledShutdownRequest.scala
@@ -38,9 +38,9 @@ object ControlledShutdownRequest extends Logging {
 }
 
 case class ControlledShutdownRequest(val versionId: Short,
-                                     override val correlationId: Int,
+                                     val correlationId: Int,
                                      val brokerId: Int)
-  extends RequestOrResponse(Some(RequestKeys.ControlledShutdownKey), correlationId){
+  extends RequestOrResponse(Some(RequestKeys.ControlledShutdownKey)){
 
   def this(correlationId: Int, brokerId: Int) =
     this(ControlledShutdownRequest.CurrentVersion, correlationId, brokerId)

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala b/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala
index 46ec3db..5e0a1cf 100644
--- a/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala
+++ b/core/src/main/scala/kafka/api/ControlledShutdownResponse.scala
@@ -39,10 +39,10 @@ object ControlledShutdownResponse {
 }
 
 
-case class ControlledShutdownResponse(override val correlationId: Int,
+case class ControlledShutdownResponse(val correlationId: Int,
                                       val errorCode: Short = ErrorMapping.NoError,
                                       val partitionsRemaining: Set[TopicAndPartition])
-  extends RequestOrResponse(correlationId = correlationId) {
+  extends RequestOrResponse() {
   def sizeInBytes(): Int ={
     var size =
       4 /* correlation id */ +

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/FetchRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala
index a8b73ac..55a5982 100644
--- a/core/src/main/scala/kafka/api/FetchRequest.scala
+++ b/core/src/main/scala/kafka/api/FetchRequest.scala
@@ -60,13 +60,13 @@ object FetchRequest {
 }
 
 case class FetchRequest private[kafka] (versionId: Short = FetchRequest.CurrentVersion,
-                                        override val correlationId: Int = FetchRequest.DefaultCorrelationId,
+                                        correlationId: Int = FetchRequest.DefaultCorrelationId,
                                         clientId: String = ConsumerConfig.DefaultClientId,
                                         replicaId: Int = Request.OrdinaryConsumerId,
                                         maxWait: Int = FetchRequest.DefaultMaxWait,
                                         minBytes: Int = FetchRequest.DefaultMinBytes,
                                         requestInfo: Map[TopicAndPartition, PartitionFetchInfo])
-        extends RequestOrResponse(Some(RequestKeys.FetchKey), correlationId) {
+        extends RequestOrResponse(Some(RequestKeys.FetchKey)) {
 
   /**
    * Partitions the request info into a map of maps (one for each topic).

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala b/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala
new file mode 100644
index 0000000..fb022e8
--- /dev/null
+++ b/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala
@@ -0,0 +1,45 @@
+/**
+ * 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.api
+
+import java.nio.ByteBuffer
+import org.apache.kafka.common.requests.AbstractRequestResponse
+
+private[kafka] abstract class GenericRequestOrResponseAndHeader(val header: AbstractRequestResponse,
+                                                                val body: AbstractRequestResponse,
+                                                                val name: String,
+                                                                override val requestId: Option[Short] = None)
+  extends RequestOrResponse(requestId) {
+
+  def writeTo(buffer: ByteBuffer) {
+    header.writeTo(buffer)
+    body.writeTo(buffer)
+  }
+
+  def sizeInBytes(): Int = {
+    header.sizeOf() + body.sizeOf();
+  }
+
+  override def toString(): String = {
+    describe(true)
+  }
+
+  override def describe(details: Boolean): String = {
+    val strBuffer = new StringBuilder
+    strBuffer.append("Name: " + name)
+    strBuffer.append("; header: " + header.toString)
+    strBuffer.append("; body: " + body.toString)
+    strBuffer.toString()
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala b/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala
new file mode 100644
index 0000000..932418b
--- /dev/null
+++ b/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala
@@ -0,0 +1,39 @@
+/**
+ * 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.api
+
+import java.nio.ByteBuffer
+import kafka.network.{BoundedByteBufferSend, RequestChannel}
+import kafka.common.ErrorMapping
+import kafka.network.RequestChannel.Response
+import org.apache.kafka.common.requests.{HeartbeatResponse, ResponseHeader, HeartbeatRequest, RequestHeader}
+
+object HeartbeatRequestAndHeader {
+  def readFrom(buffer: ByteBuffer): HeartbeatRequestAndHeader = {
+    val header = RequestHeader.parse(buffer)
+    val body = HeartbeatRequest.parse(buffer)
+    new HeartbeatRequestAndHeader(header, body)
+  }
+}
+
+case class HeartbeatRequestAndHeader(override val header: RequestHeader, override  val body: HeartbeatRequest)
+  extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), Some(RequestKeys.HeartbeatKey)) {
+
+  override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
+    val errorResponseHeader = new ResponseHeader(header.correlationId)
+    val errorResponseBody = new HeartbeatResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
+    val errorHeartBeatResponseAndHeader = new HeartbeatResponseAndHeader(errorResponseHeader, errorResponseBody)
+    requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorHeartBeatResponseAndHeader)))
+  }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala b/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala
new file mode 100644
index 0000000..556f38d
--- /dev/null
+++ b/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala
@@ -0,0 +1,28 @@
+/**
+ * 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.api
+
+import org.apache.kafka.common.requests.{ResponseHeader, HeartbeatResponse}
+import java.nio.ByteBuffer
+
+object HeartbeatResponseAndHeader {
+  def readFrom(buffer: ByteBuffer): HeartbeatResponseAndHeader = {
+    val header = ResponseHeader.parse(buffer)
+    val body = HeartbeatResponse.parse(buffer)
+    new HeartbeatResponseAndHeader(header, body)
+  }
+}
+
+case class HeartbeatResponseAndHeader(override val header: ResponseHeader, override val body: HeartbeatResponse)
+  extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), None) {
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala b/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala
new file mode 100644
index 0000000..9aea28c
--- /dev/null
+++ b/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala
@@ -0,0 +1,40 @@
+/**
+ * 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.api
+
+import java.nio.ByteBuffer
+import kafka.network.{BoundedByteBufferSend, RequestChannel}
+import kafka.common.ErrorMapping
+import org.apache.kafka.common.requests._
+import kafka.network.RequestChannel.Response
+import scala.Some
+
+object JoinGroupRequestAndHeader {
+  def readFrom(buffer: ByteBuffer): JoinGroupRequestAndHeader = {
+    val header = RequestHeader.parse(buffer)
+    val body = JoinGroupRequest.parse(buffer)
+    new JoinGroupRequestAndHeader(header, body)
+  }
+}
+
+case class JoinGroupRequestAndHeader(override val header: RequestHeader, override val body: JoinGroupRequest)
+  extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), Some(RequestKeys.JoinGroupKey)) {
+
+  override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
+    val errorResponseHeader = new ResponseHeader(header.correlationId)
+    val errorResponseBody = new JoinGroupResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
+    val errorHeartBeatResponseAndHeader = new JoinGroupResponseAndHeader(errorResponseHeader, errorResponseBody)
+    requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorHeartBeatResponseAndHeader)))
+  }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala b/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala
new file mode 100644
index 0000000..7389ae6
--- /dev/null
+++ b/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala
@@ -0,0 +1,28 @@
+/**
+ * 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.api
+
+import org.apache.kafka.common.requests.{JoinGroupResponse, ResponseHeader}
+import java.nio.ByteBuffer
+
+object JoinGroupResponseAndHeader {
+  def readFrom(buffer: ByteBuffer): JoinGroupResponseAndHeader = {
+    val header = ResponseHeader.parse(buffer)
+    val body = JoinGroupResponse.parse(buffer)
+    new JoinGroupResponseAndHeader(header, body)
+  }
+}
+
+case class JoinGroupResponseAndHeader(override val header: ResponseHeader, override val body: JoinGroupResponse)
+  extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), None) {
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala
index 3e40817..4ff7e8f 100644
--- a/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala
+++ b/core/src/main/scala/kafka/api/LeaderAndIsrRequest.scala
@@ -129,13 +129,13 @@ object LeaderAndIsrRequest {
 }
 
 case class LeaderAndIsrRequest (versionId: Short,
-                                override val correlationId: Int,
+                                correlationId: Int,
                                 clientId: String,
                                 controllerId: Int,
                                 controllerEpoch: Int,
                                 partitionStateInfos: Map[(String, Int), PartitionStateInfo],
                                 leaders: Set[Broker])
-    extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey), correlationId) {
+    extends RequestOrResponse(Some(RequestKeys.LeaderAndIsrKey)) {
 
   def this(partitionStateInfos: Map[(String, Int), PartitionStateInfo], leaders: Set[Broker], controllerId: Int,
            controllerEpoch: Int, correlationId: Int, clientId: String) = {

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala b/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala
index f636444..22ce48a 100644
--- a/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala
+++ b/core/src/main/scala/kafka/api/LeaderAndIsrResponse.scala
@@ -41,10 +41,10 @@ object LeaderAndIsrResponse {
 }
 
 
-case class LeaderAndIsrResponse(override val correlationId: Int,
+case class LeaderAndIsrResponse(correlationId: Int,
                                 responseMap: Map[(String, Int), Short],
                                 errorCode: Short = ErrorMapping.NoError)
-    extends RequestOrResponse(correlationId = correlationId) {
+    extends RequestOrResponse() {
   def sizeInBytes(): Int ={
     var size =
       4 /* correlation id */ + 


[21/37] git commit: KAFKA-1180 WhiteList topic filter gets a NullPointerException on complex Regex patch by Joe Stein, reviewed by Joel Koshy

Posted by jj...@apache.org.
KAFKA-1180 WhiteList topic filter gets a NullPointerException on complex Regex patch by Joe Stein, reviewed by Joel Koshy


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/592678e4
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/592678e4
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/592678e4

Branch: refs/heads/transactional_messaging
Commit: 592678e4d72151940002ffb1a367ec433b27f2ef
Parents: fc0e03f
Author: Joe Stein <jo...@stealth.ly>
Authored: Sat Jul 19 21:00:22 2014 -0400
Committer: Joe Stein <jo...@stealth.ly>
Committed: Sat Jul 19 21:00:22 2014 -0400

----------------------------------------------------------------------
 .../main/scala/kafka/consumer/TopicCount.scala  |  4 +--
 core/src/main/scala/kafka/utils/Utils.scala     | 24 ++++++++++++++++++
 .../unit/kafka/consumer/TopicFilterTest.scala   | 26 ++++++++++++++++++++
 3 files changed, 52 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/592678e4/core/src/main/scala/kafka/consumer/TopicCount.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/TopicCount.scala b/core/src/main/scala/kafka/consumer/TopicCount.scala
index c793110..8b0ae57 100644
--- a/core/src/main/scala/kafka/consumer/TopicCount.scala
+++ b/core/src/main/scala/kafka/consumer/TopicCount.scala
@@ -19,7 +19,7 @@ package kafka.consumer
 
 import scala.collection._
 import org.I0Itec.zkclient.ZkClient
-import kafka.utils.{Json, ZKGroupDirs, ZkUtils, Logging}
+import kafka.utils.{Json, ZKGroupDirs, ZkUtils, Logging, Utils}
 import kafka.common.KafkaException
 
 private[kafka] trait TopicCount {
@@ -127,7 +127,7 @@ private[kafka] class WildcardTopicCount(zkClient: ZkClient,
     makeConsumerThreadIdsPerTopic(consumerIdString, Map(wildcardTopics.map((_, numStreams)): _*))
   }
 
-  def getTopicCountMap = Map(topicFilter.regex -> numStreams)
+  def getTopicCountMap = Map(Utils.JSONEscapeString(topicFilter.regex) -> numStreams)
 
   def pattern: String = {
     topicFilter match {

http://git-wip-us.apache.org/repos/asf/kafka/blob/592678e4/core/src/main/scala/kafka/utils/Utils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala
index 63d3dda..6576adf 100644
--- a/core/src/main/scala/kafka/utils/Utils.scala
+++ b/core/src/main/scala/kafka/utils/Utils.scala
@@ -545,4 +545,28 @@ object Utils extends Logging {
       lock.unlock()
     }
   }
+
+    //JSON strings need to be escaped based on ECMA-404 standard http://json.org
+  def JSONEscapeString (s : String) : String = {
+    s.map {
+      case '"'  => "\\\""
+      case '\\' => "\\\\"
+      case '/'  => "\\/"
+      case '\b' => "\\b"
+      case '\f' => "\\f"
+      case '\n' => "\\n"
+      case '\r' => "\\r"
+      case '\t' => "\\t"
+      /* We'll unicode escape any control characters. These include:
+       * 0x0 -> 0x1f  : ASCII Control (C0 Control Codes)
+       * 0x7f         : ASCII DELETE
+       * 0x80 -> 0x9f : C1 Control Codes
+       *
+       * Per RFC4627, section 2.5, we're not technically required to
+       * encode the C1 codes, but we do to be safe.
+       */
+      case c if ((c >= '\u0000' && c <= '\u001f') || (c >= '\u007f' && c <= '\u009f')) => "\\u%04x".format(c: Int)
+      case c => c
+    }.mkString 
+  }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/592678e4/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
index d903a6f..4f124af 100644
--- a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
+++ b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
@@ -43,6 +43,10 @@ class TopicFilterTest extends JUnitSuite {
     val topicFilter3 = new Whitelist("white_listed-topic.+")
     assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1", excludeInternalTopics = true))
     assertFalse(topicFilter3.isTopicAllowed("black1", excludeInternalTopics = true))
+
+    val topicFilter4 = new Whitelist("test-(?!bad\\b)[\\w]+")
+    assertTrue(topicFilter4.isTopicAllowed("test-good", excludeInternalTopics = true))
+    assertFalse(topicFilter4.isTopicAllowed("test-bad", excludeInternalTopics = true))    
   }
 
   @Test
@@ -56,4 +60,26 @@ class TopicFilterTest extends JUnitSuite {
     assertFalse(topicFilter1.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = true))
     assertTrue(topicFilter1.isTopicAllowed(OffsetManager.OffsetsTopicName, excludeInternalTopics = false))
   }
+
+  @Test
+  def testWildcardTopicCountGetTopicCountMapEscapeJson() {
+    def getTopicCountMapKey(regex: String): String = {
+      val topicCount = new WildcardTopicCount(null, "consumerId", new Whitelist(regex), 1, true)
+      topicCount.getTopicCountMap.head._1
+    }
+    //lets make sure that the JSON strings are escaping as we expect
+    //if they are not then when they get saved to zookeeper and read back out they will be broken on parse
+    assertEquals("-\\\"-", getTopicCountMapKey("-\"-"))
+    assertEquals("-\\\\-", getTopicCountMapKey("-\\-"))
+    assertEquals("-\\/-", getTopicCountMapKey("-/-"))
+    assertEquals("-\\\\b-", getTopicCountMapKey("-\\b-"))
+    assertEquals("-\\\\f-", getTopicCountMapKey("-\\f-"))
+    assertEquals("-\\\\n-", getTopicCountMapKey("-\\n-"))
+    assertEquals("-\\\\r-", getTopicCountMapKey("-\\r-"))
+    assertEquals("-\\\\t-", getTopicCountMapKey("-\\t-"))
+    assertEquals("-\\\\u0000-", getTopicCountMapKey("-\\u0000-"))
+    assertEquals("-\\\\u001f-", getTopicCountMapKey("-\\u001f-"))
+    assertEquals("-\\\\u007f-", getTopicCountMapKey("-\\u007f-"))
+    assertEquals("-\\\\u009f-", getTopicCountMapKey("-\\u009f-"))
+  }    
 }
\ No newline at end of file


[13/37] git commit: KAFKA-1258 Automatically delete temporary directories with a shutdown hook during tests to avoid leaking temp files. Patch by Manikumar Reddy.

Posted by jj...@apache.org.
KAFKA-1258 Automatically delete temporary directories with a shutdown hook during tests to avoid leaking temp files. Patch by Manikumar Reddy.


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/4b3d03e8
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/4b3d03e8
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/4b3d03e8

Branch: refs/heads/transactional_messaging
Commit: 4b3d03e8e0d660b27c7823c03dde075d6ce8348b
Parents: 83a9aa5
Author: Jay Kreps <ja...@gmail.com>
Authored: Sat Jul 12 12:49:04 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Sat Jul 12 12:49:04 2014 -0700

----------------------------------------------------------------------
 core/src/test/scala/unit/kafka/utils/TestUtils.scala | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/4b3d03e8/core/src/test/scala/unit/kafka/utils/TestUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index 57b2bd5..3faa884 100644
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -84,6 +84,13 @@ object TestUtils extends Logging {
     val f = new File(IoTmpDir, "kafka-" + random.nextInt(1000000))
     f.mkdirs()
     f.deleteOnExit()
+
+    Runtime.getRuntime().addShutdownHook(new Thread() {
+      override def run() = {
+        Utils.rm(f)
+      }
+    })
+    
     f
   }
 


[37/37] git commit: KAFKA-1485 Upgrade to Zookeeper 3.4.6 patch by Gwen Shapira reviewed by Joe Stein

Posted by jj...@apache.org.
KAFKA-1485 Upgrade to Zookeeper 3.4.6 patch by Gwen Shapira reviewed by Joe Stein


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/7a67a722
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/7a67a722
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/7a67a722

Branch: refs/heads/transactional_messaging
Commit: 7a67a72261bcb5b09d9defb2fd95a65c89e75172
Parents: 09690e3
Author: vagrant <vagrant@precise64.(none)>
Authored: Tue Aug 5 22:24:03 2014 +0000
Committer: vagrant <vagrant@precise64.(none)>
Committed: Tue Aug 5 22:24:03 2014 +0000

----------------------------------------------------------------------
 build.gradle                                              | 2 +-
 core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala | 4 +++-
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/7a67a722/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index a72905d..9cbedb8 100644
--- a/build.gradle
+++ b/build.gradle
@@ -215,7 +215,7 @@ project(':core') {
   dependencies {
     compile project(':clients')
     compile "org.scala-lang:scala-library:$scalaVersion"
-    compile 'org.apache.zookeeper:zookeeper:3.3.4'
+    compile 'org.apache.zookeeper:zookeeper:3.4.6'
     compile 'com.101tec:zkclient:0.3'
     compile 'com.yammer.metrics:metrics-core:2.2.0'
     compile 'net.sf.jopt-simple:jopt-simple:3.2'

http://git-wip-us.apache.org/repos/asf/kafka/blob/7a67a722/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala b/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala
index d883bde..3021a8c 100644
--- a/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala
+++ b/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala
@@ -19,6 +19,7 @@ package kafka.zk
 
 import org.apache.zookeeper.server.ZooKeeperServer
 import org.apache.zookeeper.server.NIOServerCnxn
+import org.apache.zookeeper.server.NIOServerCnxnFactory
 import kafka.utils.TestUtils
 import java.net.InetSocketAddress
 import kafka.utils.Utils
@@ -29,7 +30,8 @@ class EmbeddedZookeeper(val connectString: String) {
   val tickTime = 500
   val zookeeper = new ZooKeeperServer(snapshotDir, logDir, tickTime)
   val port = connectString.split(":")(1).toInt
-  val factory = new NIOServerCnxn.Factory(new InetSocketAddress("127.0.0.1", port))
+  val factory = new NIOServerCnxnFactory()
+  factory.configure(new InetSocketAddress("127.0.0.1", port),0)
   factory.startup(zookeeper)
 
   def shutdown() {


[12/37] git commit: KAFKA-1515 Producer can hang during metadata updates. Patch by Guozhang.

Posted by jj...@apache.org.
KAFKA-1515 Producer can hang during metadata updates. Patch by Guozhang.


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/83a9aa55
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/83a9aa55
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/83a9aa55

Branch: refs/heads/transactional_messaging
Commit: 83a9aa55d340f6b2720394a49f01a88509e17e52
Parents: 8034390
Author: Jay Kreps <ja...@gmail.com>
Authored: Fri Jul 11 13:08:24 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Fri Jul 11 13:08:24 2014 -0700

----------------------------------------------------------------------
 .../org/apache/kafka/clients/NetworkClient.java | 33 +++++++++++++-------
 .../clients/producer/internals/Metadata.java    |  2 +-
 .../main/scala/kafka/tools/MirrorMaker.scala    | 31 +++++++++---------
 3 files changed, 37 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/83a9aa55/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
index f739279..d8f9ce6 100644
--- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
@@ -122,16 +122,21 @@ public class NetworkClient implements KafkaClient {
      */
     @Override
     public boolean isReady(Node node, long now) {
-        return isReady(node.id(), now);
-    }
-
-    private boolean isReady(int node, long now) {
+        int nodeId = node.id();
         if (!this.metadataFetchInProgress && this.metadata.timeToNextUpdate(now) == 0)
             // if we need to update our metadata now declare all requests unready to make metadata requests first priority
             return false;
         else
             // otherwise we are ready if we are connected and can send more requests
-            return connectionStates.isConnected(node) && inFlightRequests.canSendMore(node);
+            return isSendable(nodeId);
+    }
+
+    /**
+     * Are we connected and ready and able to send more requests to the given node?
+     * @param node The node
+     */
+    private boolean isSendable(int node) {
+        return connectionStates.isConnected(node) && inFlightRequests.canSendMore(node);
     }
 
     /**
@@ -146,21 +151,21 @@ public class NetworkClient implements KafkaClient {
     public List<ClientResponse> poll(List<ClientRequest> requests, long timeout, long now) {
         List<NetworkSend> sends = new ArrayList<NetworkSend>();
 
-        // should we update our metadata?
-        long metadataTimeout = metadata.timeToNextUpdate(now);
-        if (!this.metadataFetchInProgress && metadataTimeout == 0)
-            maybeUpdateMetadata(sends, now);
-
         for (int i = 0; i < requests.size(); i++) {
             ClientRequest request = requests.get(i);
             int nodeId = request.request().destination();
-            if (!isReady(nodeId, now))
+            if (!isSendable(nodeId))
                 throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready.");
 
             this.inFlightRequests.add(request);
             sends.add(request.request());
         }
 
+        // should we update our metadata?
+        long metadataTimeout = metadata.timeToNextUpdate(now);
+        if (!this.metadataFetchInProgress && metadataTimeout == 0)
+            maybeUpdateMetadata(sends, now);
+
         // do the I/O
         try {
             this.selector.poll(Math.min(timeout, metadataTimeout), sends);
@@ -347,9 +352,12 @@ public class NetworkClient implements KafkaClient {
      */
     private void maybeUpdateMetadata(List<NetworkSend> sends, long now) {
         Node node = this.leastLoadedNode(now);
-        if (node == null)
+        if (node == null) {
+            log.debug("Give up sending metadata request since no node is available");
             return;
+        }
 
+        log.debug("Trying to send metadata request to node {}", node.id());
         if (connectionStates.isConnected(node.id()) && inFlightRequests.canSendMore(node.id())) {
             Set<String> topics = metadata.topics();
             this.metadataFetchInProgress = true;
@@ -359,6 +367,7 @@ public class NetworkClient implements KafkaClient {
             this.inFlightRequests.add(metadataRequest);
         } else if (connectionStates.canConnect(node.id(), now)) {
             // we don't have a connection to this node right now, make one
+            log.debug("Give up sending metadata request to node {} since it is either not connected or cannot have more in flight requests", node.id());
             initiateConnect(node, now);
         }
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/83a9aa55/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
index 140237f..4aa5b01 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
@@ -130,7 +130,7 @@ public final class Metadata {
         this.version += 1;
         this.cluster = cluster;
         notifyAll();
-        log.debug("Updated cluster metadata to {}", cluster);
+        log.debug("Updated cluster metadata version {} to {}", this.version, this.cluster);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/kafka/blob/83a9aa55/core/src/main/scala/kafka/tools/MirrorMaker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala
index 4f06e34..555d751 100644
--- a/core/src/main/scala/kafka/tools/MirrorMaker.scala
+++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala
@@ -116,19 +116,6 @@ object MirrorMaker extends Logging {
     val numStreams = options.valueOf(numStreamsOpt).intValue()
     val bufferSize = options.valueOf(bufferSizeOpt).intValue()
 
-    val useNewProducer = options.has(useNewProducerOpt)
-    val producerProps = Utils.loadProps(options.valueOf(producerConfigOpt))
-
-    // create producer threads
-    val clientId = producerProps.getProperty("client.id", "")
-    val producers = (1 to numProducers).map(i => {
-      producerProps.setProperty("client.id", clientId + "-" + i)
-      if (useNewProducer)
-        new NewShinyProducer(producerProps)
-      else
-        new OldProducer(producerProps)
-    })
-
     // create consumer streams
     connectors = options.valuesOf(consumerConfigOpt).toList
       .map(cfg => new ConsumerConfig(Utils.loadProps(cfg)))
@@ -138,8 +125,21 @@ object MirrorMaker extends Logging {
     // create a data channel btw the consumers and the producers
     val mirrorDataChannel = new DataChannel(bufferSize, numConsumers, numProducers)
 
-    producerThreads = producers.zipWithIndex.map(producerAndIndex => new ProducerThread(mirrorDataChannel, producerAndIndex._1, producerAndIndex._2))
+    // create producer threads
+    val useNewProducer = options.has(useNewProducerOpt)
+    val producerProps = Utils.loadProps(options.valueOf(producerConfigOpt))
+    val clientId = producerProps.getProperty("client.id", "")
+    producerThreads = (0 until numProducers).map(i => {
+      producerProps.setProperty("client.id", clientId + "-" + i)
+      val producer =
+      if (useNewProducer)
+        new NewShinyProducer(producerProps)
+      else
+        new OldProducer(producerProps)
+      new ProducerThread(mirrorDataChannel, producer, i)
+    })
 
+    // create consumer threads
     val filterSpec = if (options.has(whitelistOpt))
       new Whitelist(options.valueOf(whitelistOpt))
     else
@@ -153,7 +153,7 @@ object MirrorMaker extends Logging {
         fatal("Unable to create stream - shutting down mirror maker.")
         connectors.foreach(_.shutdown)
     }
-    consumerThreads = streams.zipWithIndex.map(streamAndIndex => new ConsumerThread(streamAndIndex._1, mirrorDataChannel, producers, streamAndIndex._2))
+    consumerThreads = streams.zipWithIndex.map(streamAndIndex => new ConsumerThread(streamAndIndex._1, mirrorDataChannel, streamAndIndex._2))
     assert(consumerThreads.size == numConsumers)
 
     Runtime.getRuntime.addShutdownHook(new Thread() {
@@ -233,7 +233,6 @@ object MirrorMaker extends Logging {
 
   class ConsumerThread(stream: KafkaStream[Array[Byte], Array[Byte]],
                        mirrorDataChannel: DataChannel,
-                       producers: Seq[BaseProducer],
                        threadId: Int)
           extends Thread with Logging with KafkaMetricsGroup {
 


[07/37] git commit: kafka-1513; Cleanup KafkaServerStartable code; patched by Evgeny Vereshchagin; reviewed by Jun Rao

Posted by jj...@apache.org.
kafka-1513; Cleanup KafkaServerStartable code; patched by Evgeny Vereshchagin; reviewed by Jun Rao


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/b1a1cae8
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/b1a1cae8
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/b1a1cae8

Branch: refs/heads/transactional_messaging
Commit: b1a1cae880817332f172dccaa5e3e4a69228074f
Parents: 2a4718c
Author: Evgeny Vereshchagin <ev...@ya.ru>
Authored: Wed Jul 2 22:09:26 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Wed Jul 2 22:09:26 2014 -0700

----------------------------------------------------------------------
 core/src/main/scala/kafka/server/KafkaServerStartable.scala | 8 +-------
 1 file changed, 1 insertion(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/b1a1cae8/core/src/main/scala/kafka/server/KafkaServerStartable.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaServerStartable.scala b/core/src/main/scala/kafka/server/KafkaServerStartable.scala
index cef3b84..28658bb 100644
--- a/core/src/main/scala/kafka/server/KafkaServerStartable.scala
+++ b/core/src/main/scala/kafka/server/KafkaServerStartable.scala
@@ -21,13 +21,7 @@ import kafka.utils.Logging
 
 
 class KafkaServerStartable(val serverConfig: KafkaConfig) extends Logging {
-  private var server : KafkaServer = null
-
-  init
-
-  private def init() {
-    server = new KafkaServer(serverConfig)
-  }
+  private val server = new KafkaServer(serverConfig)
 
   def startup() {
     try {


[34/37] git commit: kafka-1571; MetadataTest hangs; patched by Jun Rao; reviewed by Guozhang Wang

Posted by jj...@apache.org.
kafka-1571; MetadataTest hangs; patched by Jun Rao; reviewed by Guozhang Wang


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/1d2e776d
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/1d2e776d
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/1d2e776d

Branch: refs/heads/transactional_messaging
Commit: 1d2e776d8c4ffbe59e40985ecfc3b010d99519d1
Parents: f8d521a
Author: Jun Rao <ju...@gmail.com>
Authored: Mon Aug 4 21:19:49 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Mon Aug 4 21:19:49 2014 -0700

----------------------------------------------------------------------
 .../apache/kafka/clients/producer/MetadataTest.java   | 14 +++++++++-----
 1 file changed, 9 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/1d2e776d/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
index 543304c..4547bfc 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
@@ -17,6 +17,7 @@ import static org.junit.Assert.assertTrue;
 
 import org.apache.kafka.clients.producer.internals.Metadata;
 import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.errors.TimeoutException;
 import org.apache.kafka.test.TestUtils;
 import org.junit.Test;
 
@@ -40,9 +41,7 @@ public class MetadataTest {
         Thread t2 = asyncFetch(topic);
         assertTrue("Awaiting update", t1.isAlive());
         assertTrue("Awaiting update", t2.isAlive());
-        // keep updating the metadata until no need to
-        while (metadata.timeToNextUpdate(time) == 0)
-            metadata.update(TestUtils.singletonCluster(topic, 1), time);
+        metadata.update(TestUtils.singletonCluster(topic, 1), time);
         t1.join();
         t2.join();
         assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0);
@@ -53,8 +52,13 @@ public class MetadataTest {
     private Thread asyncFetch(final String topic) {
         Thread thread = new Thread() {
             public void run() {
-                while (metadata.fetch().partitionsForTopic(topic) == null)
-                    metadata.awaitUpdate(metadata.requestUpdate(), Long.MAX_VALUE);
+                while (metadata.fetch().partitionsForTopic(topic) == null) {
+                    try {
+                        metadata.awaitUpdate(metadata.requestUpdate(), refreshBackoffMs);
+                    } catch(TimeoutException e) {
+                        // let it go
+                    }
+                }
             }
         };
         thread.start();


[06/37] git commit: kafka-1503; all partitions are using same broker as their leader after broker is down; patched by Jianwen Wang; reviewed by Guozhang Wang and Jun Rao

Posted by jj...@apache.org.
kafka-1503; all partitions are using same broker as their leader after broker is down;  patched by Jianwen Wang; reviewed by Guozhang Wang and Jun Rao


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/2a4718c1
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/2a4718c1
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/2a4718c1

Branch: refs/heads/transactional_messaging
Commit: 2a4718c1a7e8e1566c5c87468779fdd1f95fe3bc
Parents: b8d87d0
Author: Jianwen Wang <Ja...@gmail.com>
Authored: Wed Jul 2 21:54:42 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Wed Jul 2 21:54:42 2014 -0700

----------------------------------------------------------------------
 .../main/scala/kafka/controller/PartitionLeaderSelector.scala   | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/2a4718c1/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala
index d3b25fa..4a31c72 100644
--- a/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala
+++ b/core/src/main/scala/kafka/controller/PartitionLeaderSelector.scala
@@ -83,7 +83,8 @@ class OfflinePartitionLeaderSelector(controllerContext: ControllerContext, confi
                 new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, List(newLeader), currentLeaderIsrZkPathVersion + 1)
             }
           case false =>
-            val newLeader = liveBrokersInIsr.head
+            val liveReplicasInIsr = liveAssignedReplicas.filter(r => liveBrokersInIsr.contains(r))
+            val newLeader = liveReplicasInIsr.head
             debug("Some broker in ISR is alive for %s. Select %d from ISR %s to be the leader."
                   .format(topicAndPartition, newLeader, liveBrokersInIsr.mkString(",")))
             new LeaderAndIsr(newLeader, currentLeaderEpoch + 1, liveBrokersInIsr.toList, currentLeaderIsrZkPathVersion + 1)
@@ -210,4 +211,4 @@ class NoOpLeaderSelector(controllerContext: ControllerContext) extends Partition
     warn("I should never have been asked to perform leader election, returning the current LeaderAndIsr and replica assignment.")
     (currentLeaderAndIsr, controllerContext.partitionReplicaAssignment(topicAndPartition))
   }
-}
\ No newline at end of file
+}


[28/37] git commit: kafka-1549; dead brokers coming in the TopicMetadataResponse; patched by Nicu Marasoiu; reviewed by Jun Rao

Posted by jj...@apache.org.
kafka-1549; dead brokers coming in the TopicMetadataResponse; patched by Nicu Marasoiu; reviewed by Jun Rao


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/7f2278fb
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/7f2278fb
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/7f2278fb

Branch: refs/heads/transactional_messaging
Commit: 7f2278fb9de08141f21b017ba66752857dcdeba4
Parents: d9e5080
Author: Nicu Marasoiu <nm...@adobe.com>
Authored: Sun Jul 27 09:59:08 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Sun Jul 27 09:59:08 2014 -0700

----------------------------------------------------------------------
 .../main/scala/kafka/cluster/Partition.scala    | 27 ++++++++++----------
 .../main/scala/kafka/server/MetadataCache.scala | 17 ++++++------
 core/src/main/scala/kafka/utils/Utils.scala     | 11 +++++---
 3 files changed, 29 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/7f2278fb/core/src/main/scala/kafka/cluster/Partition.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala
index f2ca856..134aef9 100644
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -18,8 +18,7 @@ package kafka.cluster
 
 import kafka.common._
 import kafka.admin.AdminUtils
-import kafka.utils.{ReplicationUtils, Pool, Time, Logging}
-import kafka.utils.Utils.inLock
+import kafka.utils._
 import kafka.api.{PartitionStateInfo, LeaderAndIsr}
 import kafka.log.LogConfig
 import kafka.server.{OffsetManager, ReplicaManager}
@@ -29,7 +28,7 @@ import kafka.message.ByteBufferMessageSet
 
 import java.io.IOException
 import java.util.concurrent.locks.ReentrantReadWriteLock
-import scala.Some
+import kafka.utils.Utils.{inReadLock,inWriteLock}
 import scala.collection._
 
 import com.yammer.metrics.core.Gauge
@@ -73,7 +72,7 @@ class Partition(val topic: String,
   )
 
   def isUnderReplicated(): Boolean = {
-    inLock(leaderIsrUpdateLock.readLock()) {
+    inReadLock(leaderIsrUpdateLock) {
       leaderReplicaIfLocal() match {
         case Some(_) =>
           inSyncReplicas.size < assignedReplicas.size
@@ -115,7 +114,7 @@ class Partition(val topic: String,
   }
 
   def leaderReplicaIfLocal(): Option[Replica] = {
-    inLock(leaderIsrUpdateLock.readLock()) {
+    inReadLock(leaderIsrUpdateLock) {
       leaderReplicaIdOpt match {
         case Some(leaderReplicaId) =>
           if (leaderReplicaId == localBrokerId)
@@ -141,7 +140,7 @@ class Partition(val topic: String,
 
   def delete() {
     // need to hold the lock to prevent appendMessagesToLeader() from hitting I/O exceptions due to log being deleted
-    inLock(leaderIsrUpdateLock.writeLock()) {
+    inWriteLock(leaderIsrUpdateLock) {
       assignedReplicaMap.clear()
       inSyncReplicas = Set.empty[Replica]
       leaderReplicaIdOpt = None
@@ -156,7 +155,7 @@ class Partition(val topic: String,
   }
 
   def getLeaderEpoch(): Int = {
-    inLock(leaderIsrUpdateLock.readLock()) {
+    inReadLock(leaderIsrUpdateLock) {
       return this.leaderEpoch
     }
   }
@@ -168,7 +167,7 @@ class Partition(val topic: String,
   def makeLeader(controllerId: Int,
                  partitionStateInfo: PartitionStateInfo, correlationId: Int,
                  offsetManager: OffsetManager): Boolean = {
-    inLock(leaderIsrUpdateLock.writeLock()) {
+    inWriteLock(leaderIsrUpdateLock) {
       val allReplicas = partitionStateInfo.allReplicas
       val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch
       val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr
@@ -201,7 +200,7 @@ class Partition(val topic: String,
   def makeFollower(controllerId: Int,
                    partitionStateInfo: PartitionStateInfo,
                    correlationId: Int, offsetManager: OffsetManager): Boolean = {
-    inLock(leaderIsrUpdateLock.writeLock()) {
+    inWriteLock(leaderIsrUpdateLock) {
       val allReplicas = partitionStateInfo.allReplicas
       val leaderIsrAndControllerEpoch = partitionStateInfo.leaderIsrAndControllerEpoch
       val leaderAndIsr = leaderIsrAndControllerEpoch.leaderAndIsr
@@ -235,7 +234,7 @@ class Partition(val topic: String,
   }
 
   def updateLeaderHWAndMaybeExpandIsr(replicaId: Int, offset: Long) {
-    inLock(leaderIsrUpdateLock.writeLock()) {
+    inWriteLock(leaderIsrUpdateLock) {
       debug("Recording follower %d position %d for partition [%s,%d].".format(replicaId, offset, topic, partitionId))
       val replicaOpt = getReplica(replicaId)
       if(!replicaOpt.isDefined) {
@@ -271,7 +270,7 @@ class Partition(val topic: String,
   }
 
   def checkEnoughReplicasReachOffset(requiredOffset: Long, requiredAcks: Int): (Boolean, Short) = {
-    inLock(leaderIsrUpdateLock.readLock()) {
+    inReadLock(leaderIsrUpdateLock) {
       leaderReplicaIfLocal() match {
         case Some(_) =>
           val numAcks = inSyncReplicas.count(r => {
@@ -315,7 +314,7 @@ class Partition(val topic: String,
   }
 
   def maybeShrinkIsr(replicaMaxLagTimeMs: Long,  replicaMaxLagMessages: Long) {
-    inLock(leaderIsrUpdateLock.writeLock()) {
+    inWriteLock(leaderIsrUpdateLock) {
       leaderReplicaIfLocal() match {
         case Some(leaderReplica) =>
           val outOfSyncReplicas = getOutOfSyncReplicas(leaderReplica, replicaMaxLagTimeMs, replicaMaxLagMessages)
@@ -357,7 +356,7 @@ class Partition(val topic: String,
   }
 
   def appendMessagesToLeader(messages: ByteBufferMessageSet) = {
-    inLock(leaderIsrUpdateLock.readLock()) {
+    inReadLock(leaderIsrUpdateLock) {
       val leaderReplicaOpt = leaderReplicaIfLocal()
       leaderReplicaOpt match {
         case Some(leaderReplica) =>
@@ -400,7 +399,7 @@ class Partition(val topic: String,
   }
 
   override def toString(): String = {
-    inLock(leaderIsrUpdateLock.readLock()) {
+    inReadLock(leaderIsrUpdateLock) {
       val partitionString = new StringBuilder
       partitionString.append("Topic: " + topic)
       partitionString.append("; Partition: " + partitionId)

http://git-wip-us.apache.org/repos/asf/kafka/blob/7f2278fb/core/src/main/scala/kafka/server/MetadataCache.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala
index 7cd40e1..bf81a1a 100644
--- a/core/src/main/scala/kafka/server/MetadataCache.scala
+++ b/core/src/main/scala/kafka/server/MetadataCache.scala
@@ -25,7 +25,6 @@ import kafka.utils.Utils._
 import kafka.common.{ErrorMapping, ReplicaNotAvailableException, LeaderNotAvailableException}
 import kafka.common.TopicAndPartition
 import kafka.controller.KafkaController.StateChangeLogger
-import scala.Some
 
 /**
  *  A cache for the state (e.g., current leader) of each partition. This cache is updated through
@@ -34,14 +33,14 @@ import scala.Some
 private[server] class MetadataCache {
   private val cache: mutable.Map[String, mutable.Map[Int, PartitionStateInfo]] =
     new mutable.HashMap[String, mutable.Map[Int, PartitionStateInfo]]()
-  private val aliveBrokers: mutable.Map[Int, Broker] = new mutable.HashMap[Int, Broker]()
+  private var aliveBrokers: Map[Int, Broker] = Map()
   private val partitionMetadataLock = new ReentrantReadWriteLock()
 
   def getTopicMetadata(topics: Set[String]) = {
     val isAllTopics = topics.isEmpty
     val topicsRequested = if(isAllTopics) cache.keySet else topics
     val topicResponses: mutable.ListBuffer[TopicMetadata] = new mutable.ListBuffer[TopicMetadata]
-    inLock(partitionMetadataLock.readLock()) {
+    inReadLock(partitionMetadataLock) {
       for (topic <- topicsRequested) {
         if (isAllTopics || cache.contains(topic)) {
           val partitionStateInfos = cache(topic)
@@ -82,15 +81,15 @@ private[server] class MetadataCache {
   }
 
   def getAliveBrokers = {
-    inLock(partitionMetadataLock.readLock()) {
-      aliveBrokers.values.toList
+    inReadLock(partitionMetadataLock) {
+      aliveBrokers.values.toSeq
     }
   }
 
   def addOrUpdatePartitionInfo(topic: String,
                                partitionId: Int,
                                stateInfo: PartitionStateInfo) {
-    inLock(partitionMetadataLock.writeLock()) {
+    inWriteLock(partitionMetadataLock) {
       cache.get(topic) match {
         case Some(infos) => infos.put(partitionId, stateInfo)
         case None => {
@@ -103,7 +102,7 @@ private[server] class MetadataCache {
   }
 
   def getPartitionInfo(topic: String, partitionId: Int): Option[PartitionStateInfo] = {
-    inLock(partitionMetadataLock.readLock()) {
+    inReadLock(partitionMetadataLock) {
       cache.get(topic) match {
         case Some(partitionInfos) => partitionInfos.get(partitionId)
         case None => None
@@ -114,8 +113,8 @@ private[server] class MetadataCache {
   def updateCache(updateMetadataRequest: UpdateMetadataRequest,
                   brokerId: Int,
                   stateChangeLogger: StateChangeLogger) {
-    inLock(partitionMetadataLock.writeLock()) {
-      updateMetadataRequest.aliveBrokers.foreach(b => aliveBrokers.put(b.id, b))
+    inWriteLock(partitionMetadataLock) {
+      aliveBrokers = updateMetadataRequest.aliveBrokers.map(b => (b.id, b)).toMap
       updateMetadataRequest.partitionStateInfos.foreach { case(tp, info) =>
         if (info.leaderIsrAndControllerEpoch.leaderAndIsr.leader == LeaderAndIsr.LeaderDuringDelete) {
           removePartitionInfo(tp.topic, tp.partition)

http://git-wip-us.apache.org/repos/asf/kafka/blob/7f2278fb/core/src/main/scala/kafka/utils/Utils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala
index 6576adf..09bfbce 100644
--- a/core/src/main/scala/kafka/utils/Utils.scala
+++ b/core/src/main/scala/kafka/utils/Utils.scala
@@ -21,7 +21,7 @@ import java.io._
 import java.nio._
 import charset.Charset
 import java.nio.channels._
-import java.util.concurrent.locks.Lock
+import java.util.concurrent.locks.{ReadWriteLock, Lock}
 import java.lang.management._
 import javax.management._
 import scala.collection._
@@ -540,13 +540,18 @@ object Utils extends Logging {
   def inLock[T](lock: Lock)(fun: => T): T = {
     lock.lock()
     try {
-       return fun
+      fun
     } finally {
       lock.unlock()
     }
   }
 
-    //JSON strings need to be escaped based on ECMA-404 standard http://json.org
+  def inReadLock[T](lock: ReadWriteLock)(fun: => T): T = inLock[T](lock.readLock)(fun)
+
+  def inWriteLock[T](lock: ReadWriteLock)(fun: => T): T = inLock[T](lock.writeLock)(fun)
+
+
+  //JSON strings need to be escaped based on ECMA-404 standard http://json.org
   def JSONEscapeString (s : String) : String = {
     s.map {
       case '"'  => "\\\""


[25/37] git commit: kafka-1192; Enable DumpLogSegments tool to deserialize messages; patched by Manikumar Reddy; reviewed by Guozhang Wang and Jun Rao

Posted by jj...@apache.org.
kafka-1192; Enable DumpLogSegments tool to deserialize messages; patched by Manikumar Reddy; reviewed by Guozhang Wang and Jun Rao


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/3f1a9c4c
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/3f1a9c4c
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/3f1a9c4c

Branch: refs/heads/transactional_messaging
Commit: 3f1a9c4cee778d089d3ec3167555c2b89cdc48bb
Parents: ff05e9b
Author: Manikumar Reddy <ma...@gmail.com>
Authored: Wed Jul 23 17:08:51 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Wed Jul 23 17:08:51 2014 -0700

----------------------------------------------------------------------
 .../scala/kafka/tools/DumpLogSegments.scala     | 27 +++++++++++++++-----
 1 file changed, 21 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/3f1a9c4c/core/src/main/scala/kafka/tools/DumpLogSegments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
index 6daf87b..8e9d47b 100644
--- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala
+++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
@@ -23,7 +23,8 @@ import kafka.log._
 import kafka.utils._
 import collection.mutable
 import joptsimple.OptionParser
-
+import kafka.serializer.Decoder
+import kafka.utils.VerifiableProperties
 
 object DumpLogSegments {
 
@@ -41,7 +42,15 @@ object DumpLogSegments {
                                   .ofType(classOf[java.lang.Integer])
                                   .defaultsTo(5 * 1024 * 1024)
     val deepIterationOpt = parser.accepts("deep-iteration", "if set, uses deep instead of shallow iteration")
-    
+    val valueDecoderOpt = parser.accepts("value-decoder-class", "if set, used to deserialize the messages. This class should implement kafka.serializer.Decoder trait. Custom jar should be available in kafka/libs directory.")
+                               .withOptionalArg()
+                               .ofType(classOf[java.lang.String])
+                               .defaultsTo("kafka.serializer.StringDecoder")
+    val keyDecoderOpt = parser.accepts("key-decoder-class", "if set, used to deserialize the keys. This class should implement kafka.serializer.Decoder trait. Custom jar should be available in kafka/libs directory.")
+                               .withOptionalArg()
+                               .ofType(classOf[java.lang.String])
+                               .defaultsTo("kafka.serializer.StringDecoder")
+
     if(args.length == 0)
       CommandLineUtils.printUsageAndDie(parser, "Parse a log file and dump its contents to the console, useful for debugging a seemingly corrupt log segment.")
 
@@ -54,6 +63,9 @@ object DumpLogSegments {
     val files = options.valueOf(filesOpt).split(",")
     val maxMessageSize = options.valueOf(maxMessageSizeOpt).intValue()
     val isDeepIteration = if(options.has(deepIterationOpt)) true else false
+  
+    val valueDecoder: Decoder[_] = Utils.createObject[Decoder[_]](options.valueOf(valueDecoderOpt), new VerifiableProperties)
+    val keyDecoder: Decoder[_] = Utils.createObject[Decoder[_]](options.valueOf(keyDecoderOpt), new VerifiableProperties)
 
     val misMatchesForIndexFilesMap = new mutable.HashMap[String, List[(Long, Long)]]
     val nonConsecutivePairsForLogFilesMap = new mutable.HashMap[String, List[(Long, Long)]]
@@ -62,7 +74,7 @@ object DumpLogSegments {
       val file = new File(arg)
       if(file.getName.endsWith(Log.LogFileSuffix)) {
         println("Dumping " + file)
-        dumpLog(file, print, nonConsecutivePairsForLogFilesMap, isDeepIteration, maxMessageSize)
+        dumpLog(file, print, nonConsecutivePairsForLogFilesMap, isDeepIteration, maxMessageSize , valueDecoder, keyDecoder)
       } else if(file.getName.endsWith(Log.IndexFileSuffix)) {
         println("Dumping " + file)
         dumpIndex(file, verifyOnly, misMatchesForIndexFilesMap, maxMessageSize)
@@ -118,7 +130,9 @@ object DumpLogSegments {
                       printContents: Boolean,
                       nonConsecutivePairsForLogFilesMap: mutable.HashMap[String, List[(Long, Long)]],
                       isDeepIteration: Boolean,
-                      maxMessageSize: Int) {
+                      maxMessageSize: Int,
+                      valueDecoder: Decoder[_],
+                      keyDecoder: Decoder[_]) {
     val startOffset = file.getName().split("\\.")(0).toLong
     println("Starting offset: " + startOffset)
     val messageSet = new FileMessageSet(file, false)
@@ -147,8 +161,8 @@ object DumpLogSegments {
           print(" keysize: " + msg.keySize)
         if(printContents) {
           if(msg.hasKey)
-            print(" key: " + Utils.readString(messageAndOffset.message.key, "UTF-8"))
-          val payload = if(messageAndOffset.message.isNull) null else Utils.readString(messageAndOffset.message.payload, "UTF-8")
+            print(" key: " + keyDecoder.fromBytes(Utils.readBytes(messageAndOffset.message.key)))
+          val payload = if(messageAndOffset.message.isNull) null else valueDecoder.fromBytes(Utils.readBytes(messageAndOffset.message.payload))
           print(" payload: " + payload)
         }
         println()
@@ -186,4 +200,5 @@ object DumpLogSegments {
       }
     }
   }
+  
 }


[23/37] git commit: kafka-1462 (followup patch); Add new request and response formats for the new consumer and coordinator communication; patched by Jun Rao; reviewed by Jay Kreps

Posted by jj...@apache.org.
kafka-1462 (followup patch); Add new request and response formats for the new consumer and coordinator communication; patched by Jun Rao; reviewed by Jay Kreps


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/014b700f
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/014b700f
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/014b700f

Branch: refs/heads/transactional_messaging
Commit: 014b700f0323b4dc00d3aa0e1b598f7e2ed07957
Parents: 1e4b084
Author: Jun Rao <ju...@gmail.com>
Authored: Tue Jul 22 10:53:34 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Tue Jul 22 10:53:34 2014 -0700

----------------------------------------------------------------------
 .../kafka/api/GenericRequestAndHeader.scala     | 55 ++++++++++++++++++++
 .../api/GenericRequestOrResponseAndHeader.scala | 45 ----------------
 .../kafka/api/GenericResponseAndHeader.scala    | 46 ++++++++++++++++
 .../kafka/api/HeartbeatRequestAndHeader.scala   | 20 ++++---
 .../kafka/api/HeartbeatResponseAndHeader.scala  | 10 ++--
 .../kafka/api/JoinGroupRequestAndHeader.scala   | 17 +++---
 .../kafka/api/JoinGroupResponseAndHeader.scala  | 10 ++--
 .../api/RequestResponseSerializationTest.scala  | 12 ++---
 8 files changed, 139 insertions(+), 76 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/014b700f/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala b/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala
new file mode 100644
index 0000000..f40e19f
--- /dev/null
+++ b/core/src/main/scala/kafka/api/GenericRequestAndHeader.scala
@@ -0,0 +1,55 @@
+/**
+ * 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.api
+
+import java.nio.ByteBuffer
+import org.apache.kafka.common.requests.AbstractRequestResponse
+import kafka.api.ApiUtils._
+
+private[kafka] abstract class GenericRequestAndHeader(val versionId: Short,
+                                                      val correlationId: Int,
+                                                      val clientId: String,
+                                                      val body: AbstractRequestResponse,
+                                                      val name: String,
+                                                      override val requestId: Option[Short] = None)
+  extends RequestOrResponse(requestId) {
+
+  def writeTo(buffer: ByteBuffer) {
+    buffer.putShort(versionId)
+    buffer.putInt(correlationId)
+    writeShortString(buffer, clientId)
+    body.writeTo(buffer)
+  }
+
+  def sizeInBytes(): Int = {
+    2 /* version id */ +
+    4 /* correlation id */ +
+    (2 + clientId.length) /* client id */ +
+    body.sizeOf();
+  }
+
+  override def toString(): String = {
+    describe(true)
+  }
+
+  override def describe(details: Boolean): String = {
+    val strBuffer = new StringBuilder
+    strBuffer.append("Name: " + name)
+    strBuffer.append("; Version: " + versionId)
+    strBuffer.append("; CorrelationId: " + correlationId)
+    strBuffer.append("; ClientId: " + clientId)
+    strBuffer.append("; Body: " + body.toString)
+    strBuffer.toString()
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/014b700f/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala b/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala
deleted file mode 100644
index fb022e8..0000000
--- a/core/src/main/scala/kafka/api/GenericRequestOrResponseAndHeader.scala
+++ /dev/null
@@ -1,45 +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.api
-
-import java.nio.ByteBuffer
-import org.apache.kafka.common.requests.AbstractRequestResponse
-
-private[kafka] abstract class GenericRequestOrResponseAndHeader(val header: AbstractRequestResponse,
-                                                                val body: AbstractRequestResponse,
-                                                                val name: String,
-                                                                override val requestId: Option[Short] = None)
-  extends RequestOrResponse(requestId) {
-
-  def writeTo(buffer: ByteBuffer) {
-    header.writeTo(buffer)
-    body.writeTo(buffer)
-  }
-
-  def sizeInBytes(): Int = {
-    header.sizeOf() + body.sizeOf();
-  }
-
-  override def toString(): String = {
-    describe(true)
-  }
-
-  override def describe(details: Boolean): String = {
-    val strBuffer = new StringBuilder
-    strBuffer.append("Name: " + name)
-    strBuffer.append("; header: " + header.toString)
-    strBuffer.append("; body: " + body.toString)
-    strBuffer.toString()
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/014b700f/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala b/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala
new file mode 100644
index 0000000..a4879e2
--- /dev/null
+++ b/core/src/main/scala/kafka/api/GenericResponseAndHeader.scala
@@ -0,0 +1,46 @@
+/**
+ * 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.api
+
+import java.nio.ByteBuffer
+import org.apache.kafka.common.requests.AbstractRequestResponse
+
+private[kafka] abstract class GenericResponseAndHeader(val correlationId: Int,
+                                                       val body: AbstractRequestResponse,
+                                                       val name: String,
+                                                       override val requestId: Option[Short] = None)
+  extends RequestOrResponse(requestId) {
+
+  def writeTo(buffer: ByteBuffer) {
+    buffer.putInt(correlationId)
+    body.writeTo(buffer)
+  }
+
+  def sizeInBytes(): Int = {
+    4 /* correlation id */ +
+    body.sizeOf();
+  }
+
+  override def toString(): String = {
+    describe(true)
+  }
+
+  override def describe(details: Boolean): String = {
+    val strBuffer = new StringBuilder
+    strBuffer.append("Name: " + name)
+    strBuffer.append("; CorrelationId: " + correlationId)
+    strBuffer.append("; Body: " + body.toString)
+    strBuffer.toString()
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/014b700f/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala b/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala
index 932418b..f168d9f 100644
--- a/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala
+++ b/core/src/main/scala/kafka/api/HeartbeatRequestAndHeader.scala
@@ -16,24 +16,30 @@ package kafka.api
 import java.nio.ByteBuffer
 import kafka.network.{BoundedByteBufferSend, RequestChannel}
 import kafka.common.ErrorMapping
+import org.apache.kafka.common.requests.{HeartbeatResponse, HeartbeatRequest}
+import kafka.api.ApiUtils._
 import kafka.network.RequestChannel.Response
-import org.apache.kafka.common.requests.{HeartbeatResponse, ResponseHeader, HeartbeatRequest, RequestHeader}
+import scala.Some
 
 object HeartbeatRequestAndHeader {
   def readFrom(buffer: ByteBuffer): HeartbeatRequestAndHeader = {
-    val header = RequestHeader.parse(buffer)
+    val versionId = buffer.getShort
+    val correlationId = buffer.getInt
+    val clientId = readShortString(buffer)
     val body = HeartbeatRequest.parse(buffer)
-    new HeartbeatRequestAndHeader(header, body)
+    new HeartbeatRequestAndHeader(versionId, correlationId, clientId, body)
   }
 }
 
-case class HeartbeatRequestAndHeader(override val header: RequestHeader, override  val body: HeartbeatRequest)
-  extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), Some(RequestKeys.HeartbeatKey)) {
+case class HeartbeatRequestAndHeader(override val versionId: Short,
+                                     override val correlationId: Int,
+                                     override val clientId: String,
+                                     override val body: HeartbeatRequest)
+  extends GenericRequestAndHeader(versionId, correlationId, clientId, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), Some(RequestKeys.HeartbeatKey)) {
 
   override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
-    val errorResponseHeader = new ResponseHeader(header.correlationId)
     val errorResponseBody = new HeartbeatResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
-    val errorHeartBeatResponseAndHeader = new HeartbeatResponseAndHeader(errorResponseHeader, errorResponseBody)
+    val errorHeartBeatResponseAndHeader = new HeartbeatResponseAndHeader(correlationId, errorResponseBody)
     requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorHeartBeatResponseAndHeader)))
   }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/014b700f/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala b/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala
index 556f38d..9a71faa 100644
--- a/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala
+++ b/core/src/main/scala/kafka/api/HeartbeatResponseAndHeader.scala
@@ -12,17 +12,17 @@
  */
 package kafka.api
 
-import org.apache.kafka.common.requests.{ResponseHeader, HeartbeatResponse}
+import org.apache.kafka.common.requests.HeartbeatResponse
 import java.nio.ByteBuffer
 
 object HeartbeatResponseAndHeader {
   def readFrom(buffer: ByteBuffer): HeartbeatResponseAndHeader = {
-    val header = ResponseHeader.parse(buffer)
+    val correlationId = buffer.getInt
     val body = HeartbeatResponse.parse(buffer)
-    new HeartbeatResponseAndHeader(header, body)
+    new HeartbeatResponseAndHeader(correlationId, body)
   }
 }
 
-case class HeartbeatResponseAndHeader(override val header: ResponseHeader, override val body: HeartbeatResponse)
-  extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), None) {
+case class HeartbeatResponseAndHeader(override val correlationId: Int, override val body: HeartbeatResponse)
+  extends GenericResponseAndHeader(correlationId, body, RequestKeys.nameForKey(RequestKeys.HeartbeatKey), None) {
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/014b700f/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala b/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala
index 9aea28c..3651e86 100644
--- a/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala
+++ b/core/src/main/scala/kafka/api/JoinGroupRequestAndHeader.scala
@@ -17,24 +17,29 @@ import java.nio.ByteBuffer
 import kafka.network.{BoundedByteBufferSend, RequestChannel}
 import kafka.common.ErrorMapping
 import org.apache.kafka.common.requests._
+import kafka.api.ApiUtils._
 import kafka.network.RequestChannel.Response
 import scala.Some
 
 object JoinGroupRequestAndHeader {
   def readFrom(buffer: ByteBuffer): JoinGroupRequestAndHeader = {
-    val header = RequestHeader.parse(buffer)
+    val versionId = buffer.getShort
+    val correlationId = buffer.getInt
+    val clientId = readShortString(buffer)
     val body = JoinGroupRequest.parse(buffer)
-    new JoinGroupRequestAndHeader(header, body)
+    new JoinGroupRequestAndHeader(versionId, correlationId, clientId, body)
   }
 }
 
-case class JoinGroupRequestAndHeader(override val header: RequestHeader, override val body: JoinGroupRequest)
-  extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), Some(RequestKeys.JoinGroupKey)) {
+case class JoinGroupRequestAndHeader(override val versionId: Short,
+                                     override val correlationId: Int,
+                                     override val clientId: String,
+                                     override val body: JoinGroupRequest)
+  extends GenericRequestAndHeader(versionId, correlationId, clientId, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), Some(RequestKeys.JoinGroupKey)) {
 
   override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
-    val errorResponseHeader = new ResponseHeader(header.correlationId)
     val errorResponseBody = new JoinGroupResponse(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
-    val errorHeartBeatResponseAndHeader = new JoinGroupResponseAndHeader(errorResponseHeader, errorResponseBody)
+    val errorHeartBeatResponseAndHeader = new JoinGroupResponseAndHeader(correlationId, errorResponseBody)
     requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorHeartBeatResponseAndHeader)))
   }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/014b700f/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala b/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala
index 7389ae6..d0f07e0 100644
--- a/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala
+++ b/core/src/main/scala/kafka/api/JoinGroupResponseAndHeader.scala
@@ -12,17 +12,17 @@
  */
 package kafka.api
 
-import org.apache.kafka.common.requests.{JoinGroupResponse, ResponseHeader}
+import org.apache.kafka.common.requests.JoinGroupResponse
 import java.nio.ByteBuffer
 
 object JoinGroupResponseAndHeader {
   def readFrom(buffer: ByteBuffer): JoinGroupResponseAndHeader = {
-    val header = ResponseHeader.parse(buffer)
+    val correlationId = buffer.getInt
     val body = JoinGroupResponse.parse(buffer)
-    new JoinGroupResponseAndHeader(header, body)
+    new JoinGroupResponseAndHeader(correlationId, body)
   }
 }
 
-case class JoinGroupResponseAndHeader(override val header: ResponseHeader, override val body: JoinGroupResponse)
-  extends GenericRequestOrResponseAndHeader(header, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), None) {
+case class JoinGroupResponseAndHeader(override val correlationId: Int, override val body: JoinGroupResponse)
+  extends GenericResponseAndHeader(correlationId, body, RequestKeys.nameForKey(RequestKeys.JoinGroupKey), None) {
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/014b700f/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
index 847a36b..cd16ced 100644
--- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
+++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
@@ -196,29 +196,25 @@ object SerializationTestUtils {
   }
 
   def createHeartbeatRequestAndHeader: HeartbeatRequestAndHeader = {
-    val header = new RequestHeader(ApiKeys.HEARTBEAT.id, 0.asInstanceOf[Short], "", 1)
     val body = new HeartbeatRequest("group1", 1, "consumer1")
-    HeartbeatRequestAndHeader(header, body)
+    HeartbeatRequestAndHeader(0.asInstanceOf[Short], 1, "", body)
   }
 
   def createHeartbeatResponseAndHeader: HeartbeatResponseAndHeader = {
-    val header = new ResponseHeader(1)
     val body = new HeartbeatResponse(0.asInstanceOf[Short])
-    HeartbeatResponseAndHeader(header, body)
+    HeartbeatResponseAndHeader(1, body)
   }
 
   def createJoinGroupRequestAndHeader: JoinGroupRequestAndHeader = {
     import scala.collection.JavaConversions._
-    val header = new RequestHeader(ApiKeys.JOIN_GROUP.id, 0.asInstanceOf[Short], "", 1)
     val body = new JoinGroupRequest("group1", 30000, List("topic1"), "consumer1", "strategy1");
-    JoinGroupRequestAndHeader(header, body)
+    JoinGroupRequestAndHeader(0.asInstanceOf[Short], 1, "", body)
   }
 
   def createJoinGroupResponseAndHeader: JoinGroupResponseAndHeader = {
     import scala.collection.JavaConversions._
-    val header = new ResponseHeader(1)
     val body = new JoinGroupResponse(0.asInstanceOf[Short], 1, "consumer1", List(new TopicPartition("test11", 1)))
-    JoinGroupResponseAndHeader(header, body)
+    JoinGroupResponseAndHeader(1, body)
   }
 }
 


[04/37] git commit: KAFKA-1498 Follow-up: add metric on avg record size.

Posted by jj...@apache.org.
KAFKA-1498 Follow-up: add metric on avg record size.


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/e3dfad30
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/e3dfad30
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/e3dfad30

Branch: refs/heads/transactional_messaging
Commit: e3dfad304e6c2ffc146cdb9bc25a1d683f289ace
Parents: f1c6e97
Author: Jay Kreps <ja...@gmail.com>
Authored: Tue Jul 1 14:47:57 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Tue Jul 1 14:47:57 2014 -0700

----------------------------------------------------------------------
 .../java/org/apache/kafka/clients/producer/internals/Sender.java    | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/e3dfad30/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
index 52d209b..37b9d1a 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
@@ -343,6 +343,7 @@ public class Sender implements Runnable {
 
             this.maxRecordSizeSensor = metrics.sensor("record-size-max");
             this.maxRecordSizeSensor.add("record-size-max", "The maximum record size", new Max());
+            this.maxRecordSizeSensor.add("record-size-avg", "The average record size", new Avg());
 
             this.metrics.addMetric("requests-in-flight", "The current number of in-flight requests awaiting a response.", new Measurable() {
                 public double measure(MetricConfig config, long now) {


[10/37] git commit: kafka-1325; Fix inconsistent per topic log configs; patched by Manikumar Reddy; reviewed by Jun Rao

Posted by jj...@apache.org.
kafka-1325; Fix inconsistent per topic log configs; patched by Manikumar Reddy; reviewed by Jun Rao


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/420628d6
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/420628d6
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/420628d6

Branch: refs/heads/transactional_messaging
Commit: 420628d695cc675711b94af5cfd14653147bf7f7
Parents: cd3ce27
Author: Manikumar Reddy <ma...@gmail.com>
Authored: Thu Jul 10 08:27:33 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Thu Jul 10 08:27:33 2014 -0700

----------------------------------------------------------------------
 .../main/scala/kafka/server/KafkaConfig.scala   | 23 +++++++--
 .../main/scala/kafka/server/KafkaServer.scala   |  2 +-
 .../unit/kafka/server/KafkaConfigTest.scala     | 53 ++++++++++++++++++++
 3 files changed, 73 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/420628d6/core/src/main/scala/kafka/server/KafkaConfig.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index ef75b67..bb2e654 100644
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -35,14 +35,29 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
   private def getLogRetentionTimeMillis(): Long = {
     val millisInMinute = 60L * 1000L
     val millisInHour = 60L * millisInMinute
-    if(props.containsKey("log.retention.minutes")){
+    
+    if(props.containsKey("log.retention.ms")){
+       props.getIntInRange("log.retention.ms", (1, Int.MaxValue))
+    }
+    else if(props.containsKey("log.retention.minutes")){
        millisInMinute * props.getIntInRange("log.retention.minutes", (1, Int.MaxValue))
-    } else {
+    } 
+    else {
        millisInHour * props.getIntInRange("log.retention.hours", 24*7, (1, Int.MaxValue))
     }
-
   }
 
+  private def getLogRollTimeMillis(): Long = {
+    val millisInHour = 60L * 60L * 1000L
+    
+    if(props.containsKey("log.roll.ms")){
+       props.getIntInRange("log.roll.ms", (1, Int.MaxValue))
+    }
+    else {
+       millisInHour * props.getIntInRange("log.roll.hours", 24*7, (1, Int.MaxValue))
+    }
+  }
+  
   /*********** General Configuration ***********/
 
   /* the broker id for this server */
@@ -105,7 +120,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
   val logSegmentBytes = props.getIntInRange("log.segment.bytes", 1*1024*1024*1024, (Message.MinHeaderSize, Int.MaxValue))
 
   /* the maximum time before a new log segment is rolled out */
-  val logRollHours = props.getIntInRange("log.roll.hours", 24*7, (1, Int.MaxValue))
+  val logRollTimeMillis = getLogRollTimeMillis
 
   /* the number of hours to keep a log file before deleting it */
   val logRetentionTimeMillis = getLogRetentionTimeMillis

http://git-wip-us.apache.org/repos/asf/kafka/blob/420628d6/core/src/main/scala/kafka/server/KafkaServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
index c22e51e..5a56f57 100644
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -275,7 +275,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
   
   private def createLogManager(zkClient: ZkClient, brokerState: BrokerState): LogManager = {
     val defaultLogConfig = LogConfig(segmentSize = config.logSegmentBytes, 
-                                     segmentMs = 60L * 60L * 1000L * config.logRollHours,
+                                     segmentMs = config.logRollTimeMillis,
                                      flushInterval = config.logFlushIntervalMessages,
                                      flushMs = config.logFlushIntervalMs.toLong,
                                      retentionSize = config.logRetentionBytes,

http://git-wip-us.apache.org/repos/asf/kafka/blob/420628d6/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
index 6f4809d..2377abe 100644
--- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
@@ -45,6 +45,16 @@ class KafkaConfigTest extends JUnit3Suite {
   }
   
   @Test
+  def testLogRetentionTimeMsProvided() {
+    val props = TestUtils.createBrokerConfig(0, 8181)
+    props.put("log.retention.ms", "1800000")
+
+    val cfg = new KafkaConfig(props)
+    assertEquals(30 * 60L * 1000L, cfg.logRetentionTimeMillis)
+
+  }
+  
+  @Test
   def testLogRetentionTimeNoConfigProvided() {
     val props = TestUtils.createBrokerConfig(0, 8181)
 
@@ -63,6 +73,17 @@ class KafkaConfigTest extends JUnit3Suite {
     assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis)
 
   }
+  
+  @Test
+  def testLogRetentionTimeBothMinutesAndMsProvided() {
+    val props = TestUtils.createBrokerConfig(0, 8181)
+    props.put("log.retention.ms", "1800000")
+    props.put("log.retention.minutes", "10")
+
+    val cfg = new KafkaConfig(props)
+    assertEquals( 30 * 60L * 1000L, cfg.logRetentionTimeMillis)
+
+  }
 
   @Test
   def testAdvertiseDefaults() {
@@ -129,4 +150,36 @@ class KafkaConfigTest extends JUnit3Suite {
       new KafkaConfig(props)
     }
   }
+  
+  @Test
+  def testLogRollTimeMsProvided() {
+    val props = TestUtils.createBrokerConfig(0, 8181)
+    props.put("log.roll.ms", "1800000")
+
+    val cfg = new KafkaConfig(props)
+    assertEquals(30 * 60L * 1000L, cfg.logRollTimeMillis)
+
+  }
+  
+  @Test
+  def testLogRollTimeBothMsAndHoursProvided() {
+    val props = TestUtils.createBrokerConfig(0, 8181)
+    props.put("log.roll.ms", "1800000")
+    props.put("log.roll.hours", "1")
+
+    val cfg = new KafkaConfig(props)
+    assertEquals( 30 * 60L * 1000L, cfg.logRollTimeMillis)
+
+  }
+    
+  @Test
+  def testLogRollTimeNoConfigProvided() {
+    val props = TestUtils.createBrokerConfig(0, 8181)
+
+    val cfg = new KafkaConfig(props)
+    assertEquals(24 * 7 * 60L * 60L * 1000L, cfg.logRollTimeMillis																									)
+
+  }
+  
+
 }


[18/37] kafka-1462; Add new request and response formats for the new consumer and coordinator communication; patched by Jun Rao; reviewed by Guozhang Wang and Jay Kreps

Posted by jj...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/OffsetCommitRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala
index 630768a..861a6cf 100644
--- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala
+++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala
@@ -26,7 +26,7 @@ import kafka.network.RequestChannel.Response
 import scala.collection._
 
 object OffsetCommitRequest extends Logging {
-  val CurrentVersion: Short = 0
+  val CurrentVersion: Short = 1
   val DefaultClientId = ""
 
   def readFrom(buffer: ByteBuffer): OffsetCommitRequest = {
@@ -34,11 +34,23 @@ object OffsetCommitRequest extends Logging {
 
     // Read values from the envelope
     val versionId = buffer.getShort
+    assert(versionId == 0 || versionId == 1,
+           "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0 or 1.")
+
     val correlationId = buffer.getInt
     val clientId = readShortString(buffer)
 
     // Read the OffsetRequest 
     val consumerGroupId = readShortString(buffer)
+
+    // version 1 specific fields
+    var groupGenerationId: Int = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID
+    var consumerId: String = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID
+    if (versionId == 1) {
+      groupGenerationId = buffer.getInt
+      consumerId = readShortString(buffer)
+    }
+
     val topicCount = buffer.getInt
     val pairs = (1 to topicCount).flatMap(_ => {
       val topic = readShortString(buffer)
@@ -54,16 +66,20 @@ object OffsetCommitRequest extends Logging {
         (TopicAndPartition(topic, partitionId), OffsetAndMetadata(offset, metadata, timestamp))
       })
     })
-    OffsetCommitRequest(consumerGroupId, immutable.Map(pairs:_*), versionId, correlationId, clientId)
+    OffsetCommitRequest(consumerGroupId, immutable.Map(pairs:_*), versionId, correlationId, clientId, groupGenerationId, consumerId)
   }
 }
 
 case class OffsetCommitRequest(groupId: String,
                                requestInfo: immutable.Map[TopicAndPartition, OffsetAndMetadata],
                                versionId: Short = OffsetCommitRequest.CurrentVersion,
-                               override val correlationId: Int = 0,
-                               clientId: String = OffsetCommitRequest.DefaultClientId)
-    extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey), correlationId) {
+                               correlationId: Int = 0,
+                               clientId: String = OffsetCommitRequest.DefaultClientId,
+                               groupGenerationId: Int = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID,
+                               consumerId: String =  org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID)
+    extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey)) {
+  assert(versionId == 0 || versionId == 1,
+         "Version " + versionId + " is invalid for OffsetCommitRequest. Valid versions are 0 or 1.")
 
   lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic)
 
@@ -84,7 +100,6 @@ case class OffsetCommitRequest(groupId: String,
     OffsetCommitResponse(commitStatus, correlationId)
   }
 
-
   def writeTo(buffer: ByteBuffer) {
     // Write envelope
     buffer.putShort(versionId)
@@ -93,6 +108,12 @@ case class OffsetCommitRequest(groupId: String,
 
     // Write OffsetCommitRequest
     writeShortString(buffer, groupId)             // consumer group
+
+    // version 1 specific data
+    if (versionId == 1) {
+      buffer.putInt(groupGenerationId)
+      writeShortString(buffer, consumerId)
+    }
     buffer.putInt(requestInfoGroupedByTopic.size) // number of topics
     requestInfoGroupedByTopic.foreach( t1 => { // topic -> Map[TopicAndPartition, OffsetMetadataAndError]
       writeShortString(buffer, t1._1) // topic
@@ -110,7 +131,8 @@ case class OffsetCommitRequest(groupId: String,
     2 + /* versionId */
     4 + /* correlationId */
     shortStringLength(clientId) +
-    shortStringLength(groupId) + 
+    shortStringLength(groupId) +
+    (if (versionId == 1) 4 /* group generation id */ + shortStringLength(consumerId) else 0) +
     4 + /* topic count */
     requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => {
       val (topic, offsets) = topicAndOffsets
@@ -139,6 +161,8 @@ case class OffsetCommitRequest(groupId: String,
     offsetCommitRequest.append("; CorrelationId: " + correlationId)
     offsetCommitRequest.append("; ClientId: " + clientId)
     offsetCommitRequest.append("; GroupId: " + groupId)
+    offsetCommitRequest.append("; GroupGenerationId: " + groupGenerationId)
+    offsetCommitRequest.append("; ConsumerId: " + consumerId)
     if(details)
       offsetCommitRequest.append("; RequestInfo: " + requestInfo.mkString(","))
     offsetCommitRequest.toString()

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/OffsetCommitResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala
index 4946e97..624a1c1 100644
--- a/core/src/main/scala/kafka/api/OffsetCommitResponse.scala
+++ b/core/src/main/scala/kafka/api/OffsetCommitResponse.scala
@@ -42,8 +42,8 @@ object OffsetCommitResponse extends Logging {
 }
 
 case class OffsetCommitResponse(commitStatus: Map[TopicAndPartition, Short],
-                               override val correlationId: Int = 0)
-    extends RequestOrResponse(correlationId=correlationId) {
+                                correlationId: Int = 0)
+    extends RequestOrResponse() {
 
   lazy val commitStatusGroupedByTopic = commitStatus.groupBy(_._1.topic)
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/OffsetFetchRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala
index a32f858..c7604b9 100644
--- a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala
+++ b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala
@@ -52,9 +52,9 @@ object OffsetFetchRequest extends Logging {
 case class OffsetFetchRequest(groupId: String,
                               requestInfo: Seq[TopicAndPartition],
                               versionId: Short = OffsetFetchRequest.CurrentVersion,
-                              override val correlationId: Int = 0,
+                              correlationId: Int = 0,
                               clientId: String = OffsetFetchRequest.DefaultClientId)
-    extends RequestOrResponse(Some(RequestKeys.OffsetFetchKey), correlationId) {
+    extends RequestOrResponse(Some(RequestKeys.OffsetFetchKey)) {
 
   lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_.topic)
   

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/OffsetFetchResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/OffsetFetchResponse.scala b/core/src/main/scala/kafka/api/OffsetFetchResponse.scala
index c1222f4..e3523f8 100644
--- a/core/src/main/scala/kafka/api/OffsetFetchResponse.scala
+++ b/core/src/main/scala/kafka/api/OffsetFetchResponse.scala
@@ -45,8 +45,8 @@ object OffsetFetchResponse extends Logging {
 }
 
 case class OffsetFetchResponse(requestInfo: Map[TopicAndPartition, OffsetMetadataAndError],
-                               override val correlationId: Int = 0)
-    extends RequestOrResponse(correlationId = correlationId) {
+                               correlationId: Int = 0)
+    extends RequestOrResponse() {
 
   lazy val requestInfoGroupedByTopic = requestInfo.groupBy(_._1.topic)
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/OffsetRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/OffsetRequest.scala b/core/src/main/scala/kafka/api/OffsetRequest.scala
index 7cbc26c..3d483bc 100644
--- a/core/src/main/scala/kafka/api/OffsetRequest.scala
+++ b/core/src/main/scala/kafka/api/OffsetRequest.scala
@@ -57,10 +57,10 @@ case class PartitionOffsetRequestInfo(time: Long, maxNumOffsets: Int)
 
 case class OffsetRequest(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo],
                          versionId: Short = OffsetRequest.CurrentVersion,
-                         override val correlationId: Int = 0,
+                         correlationId: Int = 0,
                          clientId: String = OffsetRequest.DefaultClientId,
                          replicaId: Int = Request.OrdinaryConsumerId)
-    extends RequestOrResponse(Some(RequestKeys.OffsetsKey), correlationId) {
+    extends RequestOrResponse(Some(RequestKeys.OffsetsKey)) {
 
   def this(requestInfo: Map[TopicAndPartition, PartitionOffsetRequestInfo], correlationId: Int, replicaId: Int) = this(requestInfo, OffsetRequest.CurrentVersion, correlationId, OffsetRequest.DefaultClientId, replicaId)
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/OffsetResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/OffsetResponse.scala b/core/src/main/scala/kafka/api/OffsetResponse.scala
index 0e1d6e3..63c0899 100644
--- a/core/src/main/scala/kafka/api/OffsetResponse.scala
+++ b/core/src/main/scala/kafka/api/OffsetResponse.scala
@@ -51,9 +51,9 @@ case class PartitionOffsetsResponse(error: Short, offsets: Seq[Long]) {
 }
 
 
-case class OffsetResponse(override val correlationId: Int,
+case class OffsetResponse(correlationId: Int,
                           partitionErrorAndOffsets: Map[TopicAndPartition, PartitionOffsetsResponse])
-    extends RequestOrResponse(correlationId = correlationId) {
+    extends RequestOrResponse() {
 
   lazy val offsetsGroupedByTopic = partitionErrorAndOffsets.groupBy(_._1.topic)
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/ProducerRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ProducerRequest.scala b/core/src/main/scala/kafka/api/ProducerRequest.scala
index 0c295a2..b2366e7 100644
--- a/core/src/main/scala/kafka/api/ProducerRequest.scala
+++ b/core/src/main/scala/kafka/api/ProducerRequest.scala
@@ -53,12 +53,12 @@ object ProducerRequest {
 }
 
 case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion,
-                           override val correlationId: Int,
+                           correlationId: Int,
                            clientId: String,
                            requiredAcks: Short,
                            ackTimeoutMs: Int,
                            data: collection.mutable.Map[TopicAndPartition, ByteBufferMessageSet])
-    extends RequestOrResponse(Some(RequestKeys.ProduceKey), correlationId) {
+    extends RequestOrResponse(Some(RequestKeys.ProduceKey)) {
 
   /**
    * Partitions the data into a map of maps (one for each topic).

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/ProducerResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ProducerResponse.scala b/core/src/main/scala/kafka/api/ProducerResponse.scala
index 5a1d801..a286272 100644
--- a/core/src/main/scala/kafka/api/ProducerResponse.scala
+++ b/core/src/main/scala/kafka/api/ProducerResponse.scala
@@ -43,9 +43,9 @@ object ProducerResponse {
 
 case class ProducerResponseStatus(var error: Short, offset: Long)
 
-case class ProducerResponse(override val correlationId: Int,
+case class ProducerResponse(correlationId: Int,
                             status: Map[TopicAndPartition, ProducerResponseStatus])
-    extends RequestOrResponse(correlationId = correlationId) {
+    extends RequestOrResponse() {
 
   /**
    * Partitions the status map into a map of maps (one for each topic).

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/RequestKeys.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/RequestKeys.scala b/core/src/main/scala/kafka/api/RequestKeys.scala
index fbfc9d3..c24c034 100644
--- a/core/src/main/scala/kafka/api/RequestKeys.scala
+++ b/core/src/main/scala/kafka/api/RequestKeys.scala
@@ -32,6 +32,8 @@ object RequestKeys {
   val OffsetCommitKey: Short = 8
   val OffsetFetchKey: Short = 9
   val ConsumerMetadataKey: Short = 10
+  val JoinGroupKey: Short = 11
+  val HeartbeatKey: Short = 12
 
   val keyToNameAndDeserializerMap: Map[Short, (String, (ByteBuffer) => RequestOrResponse)]=
     Map(ProduceKey -> ("Produce", ProducerRequest.readFrom),
@@ -44,7 +46,10 @@ object RequestKeys {
         ControlledShutdownKey -> ("ControlledShutdown", ControlledShutdownRequest.readFrom),
         OffsetCommitKey -> ("OffsetCommit", OffsetCommitRequest.readFrom),
         OffsetFetchKey -> ("OffsetFetch", OffsetFetchRequest.readFrom),
-        ConsumerMetadataKey -> ("ConsumerMetadata", ConsumerMetadataRequest.readFrom))
+        ConsumerMetadataKey -> ("ConsumerMetadata", ConsumerMetadataRequest.readFrom),
+        JoinGroupKey -> ("JoinGroup", JoinGroupRequestAndHeader.readFrom),
+        HeartbeatKey -> ("Heartbeat", HeartbeatRequestAndHeader.readFrom)
+    )
 
   def nameForKey(key: Short): String = {
     keyToNameAndDeserializerMap.get(key) match {

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/RequestOrResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/RequestOrResponse.scala b/core/src/main/scala/kafka/api/RequestOrResponse.scala
index 57f87a4..0334343 100644
--- a/core/src/main/scala/kafka/api/RequestOrResponse.scala
+++ b/core/src/main/scala/kafka/api/RequestOrResponse.scala
@@ -30,7 +30,7 @@ object Request {
 }
 
 
-private[kafka] abstract class RequestOrResponse(val requestId: Option[Short] = None, val correlationId: Int) extends Logging {
+private[kafka] abstract class RequestOrResponse(val requestId: Option[Short] = None) extends Logging {
 
   def sizeInBytes: Int
   

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/StopReplicaRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/StopReplicaRequest.scala b/core/src/main/scala/kafka/api/StopReplicaRequest.scala
index 68fc138..5e14987 100644
--- a/core/src/main/scala/kafka/api/StopReplicaRequest.scala
+++ b/core/src/main/scala/kafka/api/StopReplicaRequest.scala
@@ -54,13 +54,13 @@ object StopReplicaRequest extends Logging {
 }
 
 case class StopReplicaRequest(versionId: Short,
-                              override val correlationId: Int,
+                              correlationId: Int,
                               clientId: String,
                               controllerId: Int,
                               controllerEpoch: Int,
                               deletePartitions: Boolean,
                               partitions: Set[TopicAndPartition])
-        extends RequestOrResponse(Some(RequestKeys.StopReplicaKey), correlationId) {
+        extends RequestOrResponse(Some(RequestKeys.StopReplicaKey)) {
 
   def this(deletePartitions: Boolean, partitions: Set[TopicAndPartition], controllerId: Int, controllerEpoch: Int, correlationId: Int) = {
     this(StopReplicaRequest.CurrentVersion, correlationId, StopReplicaRequest.DefaultClientId,

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/StopReplicaResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/StopReplicaResponse.scala b/core/src/main/scala/kafka/api/StopReplicaResponse.scala
index c90ddee..3431f3f 100644
--- a/core/src/main/scala/kafka/api/StopReplicaResponse.scala
+++ b/core/src/main/scala/kafka/api/StopReplicaResponse.scala
@@ -42,10 +42,10 @@ object StopReplicaResponse {
 }
 
 
-case class StopReplicaResponse(override val correlationId: Int,
+case class StopReplicaResponse(val correlationId: Int,
                                val responseMap: Map[TopicAndPartition, Short],
                                val errorCode: Short = ErrorMapping.NoError)
-    extends RequestOrResponse(correlationId = correlationId) {
+    extends RequestOrResponse() {
   def sizeInBytes(): Int ={
     var size =
       4 /* correlation id */ + 

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/TopicMetadataRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala
index bce004f..7dca09c 100644
--- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala
+++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala
@@ -47,10 +47,10 @@ object TopicMetadataRequest extends Logging {
 }
 
 case class TopicMetadataRequest(val versionId: Short,
-                                override val correlationId: Int,
+                                val correlationId: Int,
                                 val clientId: String,
                                 val topics: Seq[String])
- extends RequestOrResponse(Some(RequestKeys.MetadataKey), correlationId){
+ extends RequestOrResponse(Some(RequestKeys.MetadataKey)){
 
   def this(topics: Seq[String], correlationId: Int) =
     this(TopicMetadataRequest.CurrentVersion, correlationId, TopicMetadataRequest.DefaultClientId, topics)

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/TopicMetadataResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala
index b233d35..92ac4e6 100644
--- a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala
+++ b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala
@@ -35,8 +35,8 @@ object TopicMetadataResponse {
 
 case class TopicMetadataResponse(brokers: Seq[Broker],
                                  topicsMetadata: Seq[TopicMetadata],
-                                 override val correlationId: Int)
-    extends RequestOrResponse(correlationId = correlationId) {
+                                 correlationId: Int)
+    extends RequestOrResponse() {
   val sizeInBytes: Int = {
     4 + 4 + brokers.map(_.sizeInBytes).sum + 4 + topicsMetadata.map(_.sizeInBytes).sum
   }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala
index 543e262..530982e 100644
--- a/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala
+++ b/core/src/main/scala/kafka/api/UpdateMetadataRequest.scala
@@ -55,13 +55,13 @@ object UpdateMetadataRequest {
 }
 
 case class UpdateMetadataRequest (versionId: Short,
-                                  override val correlationId: Int,
+                                  correlationId: Int,
                                   clientId: String,
                                   controllerId: Int,
                                   controllerEpoch: Int,
                                   partitionStateInfos: Map[TopicAndPartition, PartitionStateInfo],
                                   aliveBrokers: Set[Broker])
-  extends RequestOrResponse(Some(RequestKeys.UpdateMetadataKey), correlationId) {
+  extends RequestOrResponse(Some(RequestKeys.UpdateMetadataKey)) {
 
   def this(controllerId: Int, controllerEpoch: Int, correlationId: Int, clientId: String,
            partitionStateInfos: Map[TopicAndPartition, PartitionStateInfo], aliveBrokers: Set[Broker]) = {

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala b/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala
index c583c1f..53f6067 100644
--- a/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala
+++ b/core/src/main/scala/kafka/api/UpdateMetadataResponse.scala
@@ -32,9 +32,9 @@ object UpdateMetadataResponse {
   }
 }
 
-case class UpdateMetadataResponse(override val correlationId: Int,
+case class UpdateMetadataResponse(correlationId: Int,
                                   errorCode: Short = ErrorMapping.NoError)
-  extends RequestOrResponse(correlationId = correlationId) {
+  extends RequestOrResponse() {
   def sizeInBytes(): Int = 4 /* correlation id */ + 2 /* error code */
 
   def writeTo(buffer: ByteBuffer) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
index 8763968..ecbfa0f 100644
--- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
+++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
@@ -133,9 +133,9 @@ class RequestSendThread(val controllerId: Int,
             isSendSuccessful = true
           } catch {
             case e: Throwable => // if the send was not successful, reconnect to broker and resend the message
-              error(("Controller %d epoch %d failed to send %s request with correlation id %s to broker %s. " +
+              error(("Controller %d epoch %d failed to send request %s to broker %s. " +
                 "Reconnecting to broker.").format(controllerId, controllerContext.epoch,
-                RequestKeys.nameForKey(request.requestId.get), request.correlationId, toBroker.toString()), e)
+                request.toString, toBroker.toString()), e)
               channel.disconnect()
               connectToBroker(toBroker, channel)
               isSendSuccessful = false
@@ -153,8 +153,8 @@ class RequestSendThread(val controllerId: Int,
           case RequestKeys.UpdateMetadataKey =>
             response = UpdateMetadataResponse.readFrom(receive.buffer)
         }
-        stateChangeLogger.trace("Controller %d epoch %d received response correlationId %d for a request sent to broker %s"
-                                  .format(controllerId, controllerContext.epoch, response.correlationId, toBroker.toString()))
+        stateChangeLogger.trace("Controller %d epoch %d received response %s for a request sent to broker %s"
+                                  .format(controllerId, controllerContext.epoch, response.toString, toBroker.toString))
 
         if(callback != null) {
           callback(response)

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala
index 08dcc55..27fc1eb 100644
--- a/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala
+++ b/core/src/main/scala/kafka/javaapi/OffsetCommitRequest.scala
@@ -21,7 +21,6 @@ import kafka.common.{OffsetAndMetadata, TopicAndPartition}
 
 class OffsetCommitRequest(groupId: String,
                           requestInfo: java.util.Map[TopicAndPartition, OffsetAndMetadata],
-                          versionId: Short,
                           correlationId: Int,
                           clientId: String) {
   val underlying = {
@@ -33,7 +32,6 @@ class OffsetCommitRequest(groupId: String,
     kafka.api.OffsetCommitRequest(
       groupId = groupId,
       requestInfo = scalaMap,
-      versionId = versionId,
       correlationId = correlationId,
       clientId = clientId
     )

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala
index 7e6da16..b0b7be1 100644
--- a/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala
+++ b/core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala
@@ -24,10 +24,10 @@ import kafka.common.ErrorMapping
 import kafka.network.RequestChannel.Response
 
 class TopicMetadataRequest(val versionId: Short,
-                           override val correlationId: Int,
+                           val correlationId: Int,
                            val clientId: String,
                            val topics: java.util.List[String])
-    extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey), correlationId) {
+    extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey)) {
 
   val underlying: kafka.api.TopicMetadataRequest = {
     import scala.collection.JavaConversions._

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
index d34ddf5..847a36b 100644
--- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
+++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
@@ -23,9 +23,14 @@ import junit.framework.Assert._
 import java.nio.ByteBuffer
 import kafka.message.{Message, ByteBufferMessageSet}
 import kafka.cluster.Broker
-import kafka.common.{OffsetAndMetadata, TopicAndPartition, ErrorMapping, OffsetMetadataAndError}
-import kafka.controller.LeaderIsrAndControllerEpoch
+import kafka.common.{OffsetAndMetadata, ErrorMapping, OffsetMetadataAndError}
 import kafka.utils.SystemTime
+import org.apache.kafka.common.requests._
+import org.apache.kafka.common.protocol.ApiKeys
+import scala.Some
+import kafka.controller.LeaderIsrAndControllerEpoch
+import kafka.common.TopicAndPartition
+import org.apache.kafka.common.TopicPartition
 
 
 object SerializationTestUtils {
@@ -146,13 +151,23 @@ object SerializationTestUtils {
     new TopicMetadataResponse(brokers, Seq(topicmetaData1, topicmetaData2), 1)
   }
 
-  def createTestOffsetCommitRequest: OffsetCommitRequest = {
+  def createTestOffsetCommitRequestV1: OffsetCommitRequest = {
     new OffsetCommitRequest("group 1", collection.immutable.Map(
       TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="some metadata", timestamp=SystemTime.milliseconds),
       TopicAndPartition(topic1, 1) -> OffsetAndMetadata(offset=100L, metadata=OffsetAndMetadata.NoMetadata, timestamp=SystemTime.milliseconds)
     ))
   }
 
+  def createTestOffsetCommitRequestV0: OffsetCommitRequest = {
+    new OffsetCommitRequest(
+      versionId = 0,
+      groupId = "group 1",
+      requestInfo = collection.immutable.Map(
+        TopicAndPartition(topic1, 0) -> OffsetAndMetadata(offset=42L, metadata="some metadata", timestamp=SystemTime.milliseconds),
+        TopicAndPartition(topic1, 1) -> OffsetAndMetadata(offset=100L, metadata=OffsetAndMetadata.NoMetadata, timestamp=SystemTime.milliseconds)
+      ))
+  }
+
   def createTestOffsetCommitResponse: OffsetCommitResponse = {
     new OffsetCommitResponse(collection.immutable.Map(TopicAndPartition(topic1, 0) -> ErrorMapping.NoError,
                                  TopicAndPartition(topic1, 1) -> ErrorMapping.NoError))
@@ -180,6 +195,31 @@ object SerializationTestUtils {
     ConsumerMetadataResponse(Some(brokers.head), ErrorMapping.NoError)
   }
 
+  def createHeartbeatRequestAndHeader: HeartbeatRequestAndHeader = {
+    val header = new RequestHeader(ApiKeys.HEARTBEAT.id, 0.asInstanceOf[Short], "", 1)
+    val body = new HeartbeatRequest("group1", 1, "consumer1")
+    HeartbeatRequestAndHeader(header, body)
+  }
+
+  def createHeartbeatResponseAndHeader: HeartbeatResponseAndHeader = {
+    val header = new ResponseHeader(1)
+    val body = new HeartbeatResponse(0.asInstanceOf[Short])
+    HeartbeatResponseAndHeader(header, body)
+  }
+
+  def createJoinGroupRequestAndHeader: JoinGroupRequestAndHeader = {
+    import scala.collection.JavaConversions._
+    val header = new RequestHeader(ApiKeys.JOIN_GROUP.id, 0.asInstanceOf[Short], "", 1)
+    val body = new JoinGroupRequest("group1", 30000, List("topic1"), "consumer1", "strategy1");
+    JoinGroupRequestAndHeader(header, body)
+  }
+
+  def createJoinGroupResponseAndHeader: JoinGroupResponseAndHeader = {
+    import scala.collection.JavaConversions._
+    val header = new ResponseHeader(1)
+    val body = new JoinGroupResponse(0.asInstanceOf[Short], 1, "consumer1", List(new TopicPartition("test11", 1)))
+    JoinGroupResponseAndHeader(header, body)
+  }
 }
 
 class RequestResponseSerializationTest extends JUnitSuite {
@@ -194,27 +234,31 @@ class RequestResponseSerializationTest extends JUnitSuite {
   private val offsetResponse = SerializationTestUtils.createTestOffsetResponse
   private val topicMetadataRequest = SerializationTestUtils.createTestTopicMetadataRequest
   private val topicMetadataResponse = SerializationTestUtils.createTestTopicMetadataResponse
-  private val offsetCommitRequest = SerializationTestUtils.createTestOffsetCommitRequest
+  private val offsetCommitRequestV0 = SerializationTestUtils.createTestOffsetCommitRequestV0
+  private val offsetCommitRequestV1 = SerializationTestUtils.createTestOffsetCommitRequestV1
   private val offsetCommitResponse = SerializationTestUtils.createTestOffsetCommitResponse
   private val offsetFetchRequest = SerializationTestUtils.createTestOffsetFetchRequest
   private val offsetFetchResponse = SerializationTestUtils.createTestOffsetFetchResponse
   private val consumerMetadataRequest = SerializationTestUtils.createConsumerMetadataRequest
   private val consumerMetadataResponse = SerializationTestUtils.createConsumerMetadataResponse
   private val consumerMetadataResponseNoCoordinator = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode)
+  private val heartbeatRequest = SerializationTestUtils.createHeartbeatRequestAndHeader
+  private val heartbeatResponse = SerializationTestUtils.createHeartbeatResponseAndHeader
+  private val joinGroupRequest = SerializationTestUtils.createJoinGroupRequestAndHeader
+  private val joinGroupResponse = SerializationTestUtils.createJoinGroupResponseAndHeader
 
   @Test
   def testSerializationAndDeserialization() {
 
     val requestsAndResponses =
-      collection.immutable.Seq(leaderAndIsrRequest, leaderAndIsrResponse,
-                               stopReplicaRequest, stopReplicaResponse,
-                               producerRequest, producerResponse,
-                               fetchRequest,
-                               offsetRequest, offsetResponse,
-                               topicMetadataRequest, topicMetadataResponse,
-                               offsetCommitRequest, offsetCommitResponse,
-                               offsetFetchRequest, offsetFetchResponse,
-                               consumerMetadataRequest, consumerMetadataResponse, consumerMetadataResponseNoCoordinator)
+      collection.immutable.Seq(leaderAndIsrRequest, leaderAndIsrResponse, stopReplicaRequest,
+                               stopReplicaResponse, producerRequest, producerResponse,
+                               fetchRequest, offsetRequest, offsetResponse, topicMetadataRequest,
+                               topicMetadataResponse, offsetCommitRequestV0, offsetCommitRequestV1,
+                               offsetCommitResponse, offsetFetchRequest, offsetFetchResponse,
+                               consumerMetadataRequest, consumerMetadataResponse,
+                               consumerMetadataResponseNoCoordinator, heartbeatRequest,
+                               heartbeatResponse, joinGroupRequest, joinGroupResponse)
 
     requestsAndResponses.foreach { original =>
       val buffer = ByteBuffer.allocate(original.sizeInBytes)
@@ -222,7 +266,9 @@ class RequestResponseSerializationTest extends JUnitSuite {
       buffer.rewind()
       val deserializer = original.getClass.getDeclaredMethod("readFrom", classOf[ByteBuffer])
       val deserialized = deserializer.invoke(null, buffer)
-      assertEquals("The original and deserialized request/response should be the same.", original, deserialized)
+      assertFalse("All serialized bytes in " + original.getClass.getSimpleName + " should have been consumed",
+                  buffer.hasRemaining)
+      assertEquals("The original and deserialized for " + original.getClass.getSimpleName + " should be the same.", original, deserialized)
     }
   }
 }


[05/37] git commit: KAFKA-1308 Publish jar of test utilities to Maven

Posted by jj...@apache.org.
KAFKA-1308 Publish jar of test utilities to Maven


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/b8d87d0b
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/b8d87d0b
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/b8d87d0b

Branch: refs/heads/transactional_messaging
Commit: b8d87d0b22cc4e87bac2e9fdfe761b23734851ff
Parents: e3dfad3
Author: Jakob Homan <jg...@gmail.com>
Authored: Wed Jul 2 15:01:06 2014 -0700
Committer: Jakob Homan <jg...@gmail.com>
Committed: Wed Jul 2 15:01:06 2014 -0700

----------------------------------------------------------------------
 build.gradle | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/b8d87d0b/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index 2f4167f..a72905d 100644
--- a/build.gradle
+++ b/build.gradle
@@ -272,7 +272,7 @@ project(':core') {
   }
 
   task testJar(type: Jar) {
-    appendix = 'test'
+    classifier = 'test'
     from sourceSets.test.output
   }
 
@@ -283,6 +283,9 @@ project(':core') {
     }
   }
 
+  artifacts {
+    archives testJar
+  }
 }
 
 project(':perf') {
@@ -372,7 +375,7 @@ project(':clients') {
   }
 
   task testJar(type: Jar) {
-    appendix = 'test'
+    classifier = 'test'
     from sourceSets.test.output
   }
 


[36/37] git commit: KAFKA-1550; Patch review tool should use git format-patch for patch generation; reviewed by Guozhang Wang and Joel Koshy

Posted by jj...@apache.org.
KAFKA-1550; Patch review tool should use git format-patch for patch generation; reviewed by Guozhang Wang and Joel Koshy


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/09690e33
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/09690e33
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/09690e33

Branch: refs/heads/transactional_messaging
Commit: 09690e339aabea6e9141f7e2ddd50449cc1c9180
Parents: 0386790
Author: Dong Lin <li...@gmail.com>
Authored: Tue Aug 5 14:28:11 2014 -0700
Committer: Joel Koshy <jj...@gmail.com>
Committed: Tue Aug 5 14:47:51 2014 -0700

----------------------------------------------------------------------
 kafka-patch-review.py | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/09690e33/kafka-patch-review.py
----------------------------------------------------------------------
diff --git a/kafka-patch-review.py b/kafka-patch-review.py
index dc45549..89afc84 100644
--- a/kafka-patch-review.py
+++ b/kafka-patch-review.py
@@ -105,11 +105,13 @@ def main():
       print 'ERROR: Your reviewboard was not created/updated. Please run the script with the --debug option to troubleshoot the problem'
       p.close()
       sys.exit(1)
-  p.close()
+  if p.close() != None:
+    print 'ERROR: reviewboard update failed. Exiting.'
+    sys.exit(1)
   if opt.debug:
     print 'rb url=',rb_url
 
-  git_command="git diff " + opt.branch + " > " + patch_file
+  git_command="git format-patch " + opt.branch + " --stdout > " + patch_file
   if opt.debug:
     print git_command
   p=os.popen(git_command)


[08/37] git commit: KAFKA-1519 Make it possible to disable the line seperator in the console consumer. Patch from Gwen Shapira.

Posted by jj...@apache.org.
KAFKA-1519 Make it possible to disable the line seperator in the console consumer. Patch from Gwen Shapira.


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/6de56b30
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/6de56b30
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/6de56b30

Branch: refs/heads/transactional_messaging
Commit: 6de56b30b0f9aee19bf110431bcb4511d9b748f2
Parents: b1a1cae
Author: Jay Kreps <ja...@gmail.com>
Authored: Thu Jul 3 21:15:01 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Thu Jul 3 21:15:01 2014 -0700

----------------------------------------------------------------------
 .../scala/kafka/utils/CommandLineUtils.scala    | 17 ++++---
 .../unit/kafka/utils/CommandLineUtilsTest.scala | 49 ++++++++++++++++++++
 2 files changed, 59 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/6de56b30/core/src/main/scala/kafka/utils/CommandLineUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/CommandLineUtils.scala b/core/src/main/scala/kafka/utils/CommandLineUtils.scala
index 1ba605c..086a624 100644
--- a/core/src/main/scala/kafka/utils/CommandLineUtils.scala
+++ b/core/src/main/scala/kafka/utils/CommandLineUtils.scala
@@ -60,14 +60,17 @@ object CommandLineUtils extends Logging {
    * Parse key-value pairs in the form key=value
    */
   def parseKeyValueArgs(args: Iterable[String]): Properties = {
-    val splits = args.map(_ split "=").filterNot(_ == null).filterNot(_.length == 0)
-    if(!splits.forall(_.length == 2)) {
-      System.err.println("Invalid command line properties: " + args.mkString(" "))
-      System.exit(1)
-    }
+    val splits = args.map(_ split "=").filterNot(_.length == 0)
+
     val props = new Properties
-    for(a <- splits)
-      props.put(a(0), a(1))
+    for(a <- splits) {
+      if (a.length == 1) props.put(a(0), "")
+      else if (a.length == 2) props.put(a(0), a(1))
+      else {
+        System.err.println("Invalid command line properties: " + args.mkString(" "))
+        System.exit(1)
+      }
+    }
     props
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/6de56b30/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala b/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala
new file mode 100644
index 0000000..e832a01
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/utils/CommandLineUtilsTest.scala
@@ -0,0 +1,49 @@
+/**
+ * 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 unit.kafka.utils
+
+import junit.framework.Assert._
+import org.junit.{Test, After, Before}
+import kafka.utils.CommandLineUtils;
+
+class CommandLineUtilsTest {
+
+
+  @Test
+  def testParseEmptyArg() {
+    val argArray = Array("my.empty.property=")
+    val props = CommandLineUtils.parseKeyValueArgs(argArray)
+    assertEquals("Empty value should be equal to empty string",props.getProperty("my.empty.property"),"")
+  }
+
+  @Test
+  def testParseSingleArg() {
+    val argArray = Array("my.property=value")
+    val props = CommandLineUtils.parseKeyValueArgs(argArray)
+    assertEquals("Value of a single property should be 'value' ",props.getProperty("my.property"),"value")
+  }
+
+  @Test
+  def testParseArgs() {
+    val argArray = Array("first.property=first","second.property=second")
+    val props = CommandLineUtils.parseKeyValueArgs(argArray)
+    assertEquals("Value of first property should be 'first'",props.getProperty("first.property"),"first")
+    assertEquals("Value of second property should be 'second'",props.getProperty("second.property"),"second")
+  }
+
+}


[26/37] git commit: KAFKA-1544 Log cleaner takes a long time to shut down. Patch from Manikumar Reddy.

Posted by jj...@apache.org.
KAFKA-1544 Log cleaner takes a long time to shut down. Patch from Manikumar Reddy.


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/db41f98e
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/db41f98e
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/db41f98e

Branch: refs/heads/transactional_messaging
Commit: db41f98ea943d806362487b8f8192360a488eb39
Parents: 3f1a9c4
Author: Jay Kreps <ja...@gmail.com>
Authored: Thu Jul 24 15:53:24 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Thu Jul 24 16:34:40 2014 -0700

----------------------------------------------------------------------
 core/src/main/scala/kafka/log/LogCleaner.scala | 14 ++++++++++++--
 1 file changed, 12 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/db41f98e/core/src/main/scala/kafka/log/LogCleaner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala
index 2faa196..afbeffc 100644
--- a/core/src/main/scala/kafka/log/LogCleaner.scala
+++ b/core/src/main/scala/kafka/log/LogCleaner.scala
@@ -28,6 +28,8 @@ import kafka.utils._
 import kafka.metrics.KafkaMetricsGroup
 import com.yammer.metrics.core.Gauge
 import java.lang.IllegalStateException
+import java.util.concurrent.CountDownLatch
+import java.util.concurrent.TimeUnit
 
 /**
  * The cleaner is responsible for removing obsolete records from logs which have the dedupe retention strategy.
@@ -173,7 +175,8 @@ class LogCleaner(val config: CleanerConfig,
                               checkDone = checkDone)
     
     @volatile var lastStats: CleanerStats = new CleanerStats()
-    
+    private val backOffWaitLatch = new CountDownLatch(1)
+
     private def checkDone(topicAndPartition: TopicAndPartition) {
       if (!isRunning.get())
         throw new ThreadShutdownException
@@ -187,6 +190,13 @@ class LogCleaner(val config: CleanerConfig,
       cleanOrSleep()
     }
     
+    
+    override def shutdown() = {
+    	 initiateShutdown()
+    	 backOffWaitLatch.countDown()
+    	 awaitShutdown()
+     }
+     
     /**
      * Clean a log if there is a dirty log available, otherwise sleep for a bit
      */
@@ -194,7 +204,7 @@ class LogCleaner(val config: CleanerConfig,
       cleanerManager.grabFilthiestLog() match {
         case None =>
           // there are no cleanable logs, sleep a while
-          time.sleep(config.backOffMs)
+          backOffWaitLatch.await(config.backOffMs, TimeUnit.MILLISECONDS)
         case Some(cleanable) =>
           // there's a log, clean it
           var endOffset = cleanable.firstDirtyOffset


[22/37] git commit: KAFKA-1539 Fsync offset checkpoint file after writing.

Posted by jj...@apache.org.
KAFKA-1539 Fsync offset checkpoint file after writing.


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/1e4b0841
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/1e4b0841
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/1e4b0841

Branch: refs/heads/transactional_messaging
Commit: 1e4b0841b37e9e6526d7a7a7c643b1369d9f03c5
Parents: 592678e
Author: Jay Kreps <ja...@gmail.com>
Authored: Mon Jul 21 10:22:50 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Mon Jul 21 10:22:50 2014 -0700

----------------------------------------------------------------------
 core/src/main/scala/kafka/server/OffsetCheckpoint.scala | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/1e4b0841/core/src/main/scala/kafka/server/OffsetCheckpoint.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/OffsetCheckpoint.scala b/core/src/main/scala/kafka/server/OffsetCheckpoint.scala
index 7af2f43..8c5b054 100644
--- a/core/src/main/scala/kafka/server/OffsetCheckpoint.scala
+++ b/core/src/main/scala/kafka/server/OffsetCheckpoint.scala
@@ -34,7 +34,8 @@ class OffsetCheckpoint(val file: File) extends Logging {
       // write to temp file and then swap with the existing file
       val temp = new File(file.getAbsolutePath + ".tmp")
 
-      val writer = new BufferedWriter(new FileWriter(temp))
+      val fileOutputStream = new FileOutputStream(temp)
+      val writer = new BufferedWriter(new OutputStreamWriter(fileOutputStream))
       try {
         // write the current version
         writer.write(0.toString)
@@ -50,8 +51,9 @@ class OffsetCheckpoint(val file: File) extends Logging {
           writer.newLine()
         }
       
-        // flush and overwrite old file
+        // flush the buffer and then fsync the underlying file
         writer.flush()
+        fileOutputStream.getFD().sync()
       } finally {
         writer.close()
       }


[27/37] git commit: KAFKA-1483 Split Brain about Leader Partitions; reviewed by Guozhang, Jun and Neha

Posted by jj...@apache.org.
KAFKA-1483 Split Brain about Leader Partitions; reviewed by Guozhang, Jun and Neha


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/d9e5080d
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/d9e5080d
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/d9e5080d

Branch: refs/heads/transactional_messaging
Commit: d9e5080dfceff12ea599f8d518ad402ba7d54c9d
Parents: db41f98
Author: Sriharsha Chintalapani <sc...@hortonworks.com>
Authored: Thu Jul 24 16:48:21 2014 -0700
Committer: Neha Narkhede <ne...@gmail.com>
Committed: Thu Jul 24 16:48:41 2014 -0700

----------------------------------------------------------------------
 .../scala/kafka/server/ReplicaManager.scala     | 35 +++++---------------
 1 file changed, 9 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/d9e5080d/core/src/main/scala/kafka/server/ReplicaManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala
index 6a56a77..897783c 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -36,9 +36,9 @@ object ReplicaManager {
   val HighWatermarkFilename = "replication-offset-checkpoint"
 }
 
-class ReplicaManager(val config: KafkaConfig, 
-                     time: Time, 
-                     val zkClient: ZkClient, 
+class ReplicaManager(val config: KafkaConfig,
+                     time: Time,
+                     val zkClient: ZkClient,
                      scheduler: Scheduler,
                      val logManager: LogManager,
                      val isShuttingDown: AtomicBoolean ) extends Logging with KafkaMetricsGroup {
@@ -46,8 +46,6 @@ class ReplicaManager(val config: KafkaConfig,
   @volatile var controllerEpoch: Int = KafkaController.InitialControllerEpoch - 1
   private val localBrokerId = config.brokerId
   private val allPartitions = new Pool[(String, Int), Partition]
-  private var leaderPartitions = new mutable.HashSet[Partition]()
-  private val leaderPartitionsLock = new Object
   private val replicaStateChangeLock = new Object
   val replicaFetcherManager = new ReplicaFetcherManager(config, this)
   private val highWatermarkCheckPointThreadStarted = new AtomicBoolean(false)
@@ -60,9 +58,7 @@ class ReplicaManager(val config: KafkaConfig,
     "LeaderCount",
     new Gauge[Int] {
       def value = {
-        leaderPartitionsLock synchronized {
-          leaderPartitions.size
-        }
+          getLeaderPartitions().size
       }
     }
   )
@@ -82,9 +78,7 @@ class ReplicaManager(val config: KafkaConfig,
   val isrShrinkRate = newMeter("IsrShrinksPerSec",  "shrinks", TimeUnit.SECONDS)
 
   def underReplicatedPartitionCount(): Int = {
-    leaderPartitionsLock synchronized {
-      leaderPartitions.count(_.isUnderReplicated)
-    }
+      getLeaderPartitions().count(_.isUnderReplicated)
   }
 
   def startHighWaterMarksCheckPointThread() = {
@@ -117,9 +111,6 @@ class ReplicaManager(val config: KafkaConfig,
     val errorCode = ErrorMapping.NoError
     getPartition(topic, partitionId) match {
       case Some(partition) =>
-        leaderPartitionsLock synchronized {
-          leaderPartitions -= partition
-        }
         if(deletePartition) {
           val removedPartition = allPartitions.remove((topic, partitionId))
           if (removedPartition != null)
@@ -331,10 +322,6 @@ class ReplicaManager(val config: KafkaConfig,
       partitionState.foreach{ case (partition, partitionStateInfo) =>
         partition.makeLeader(controllerId, partitionStateInfo, correlationId, offsetManager)}
 
-      // Finally add these partitions to the list of partitions for which the leader is the current broker
-      leaderPartitionsLock synchronized {
-        leaderPartitions ++= partitionState.keySet
-      }
     } catch {
       case e: Throwable =>
         partitionState.foreach { state =>
@@ -383,9 +370,6 @@ class ReplicaManager(val config: KafkaConfig,
       responseMap.put((partition.topic, partition.partitionId), ErrorMapping.NoError)
 
     try {
-      leaderPartitionsLock synchronized {
-        leaderPartitions --= partitionState.keySet
-      }
 
       var partitionsToMakeFollower: Set[Partition] = Set()
 
@@ -464,11 +448,7 @@ class ReplicaManager(val config: KafkaConfig,
 
   private def maybeShrinkIsr(): Unit = {
     trace("Evaluating ISR list of partitions to see which replicas can be removed from the ISR")
-    var curLeaderPartitions: List[Partition] = null
-    leaderPartitionsLock synchronized {
-      curLeaderPartitions = leaderPartitions.toList
-    }
-    curLeaderPartitions.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs, config.replicaLagMaxMessages))
+    allPartitions.values.foreach(partition => partition.maybeShrinkIsr(config.replicaLagTimeMaxMs, config.replicaLagMaxMessages))
   }
 
   def recordFollowerPosition(topic: String, partitionId: Int, replicaId: Int, offset: Long) = {
@@ -480,6 +460,9 @@ class ReplicaManager(val config: KafkaConfig,
     }
   }
 
+  private def getLeaderPartitions() : List[Partition] = {
+    allPartitions.values.filter(_.leaderReplicaIfLocal().isDefined).toList
+  }
   /**
    * Flushes the highwatermark value for all partitions to the highwatermark file
    */


[14/37] git commit: kafka-1529; transient unit test failure in testAutoCreateAfterDeleteTopic; patched by Jun Rao; reviewed by Guozhang Wang and Neha Narkhede

Posted by jj...@apache.org.
kafka-1529; transient unit test failure in testAutoCreateAfterDeleteTopic; patched by Jun Rao; reviewed by Guozhang Wang and Neha Narkhede


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/3e6b386b
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/3e6b386b
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/3e6b386b

Branch: refs/heads/transactional_messaging
Commit: 3e6b386b72b0d1a39621d7e166f2228c97646e1d
Parents: 4b3d03e
Author: Jun Rao <ju...@gmail.com>
Authored: Tue Jul 15 14:53:15 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Tue Jul 15 14:53:15 2014 -0700

----------------------------------------------------------------------
 .../unit/kafka/admin/DeleteTopicTest.scala      | 31 --------------------
 1 file changed, 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/3e6b386b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
index 5d3c57a..29cc01b 100644
--- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
@@ -203,37 +203,6 @@ class DeleteTopicTest extends JUnit3Suite with ZooKeeperTestHarness {
   }
 
   @Test
-  def testAutoCreateAfterDeleteTopic() {
-    val topicAndPartition = TopicAndPartition("test", 0)
-    val topic = topicAndPartition.topic
-    val servers = createTestTopicAndCluster(topic)
-    // start topic deletion
-    AdminUtils.deleteTopic(zkClient, topic)
-    verifyTopicDeletion(topic, servers)
-    // test if first produce request after topic deletion auto creates the topic
-    val props = new Properties()
-    props.put("metadata.broker.list", servers.map(s => s.config.hostName + ":" + s.config.port).mkString(","))
-    props.put("serializer.class", "kafka.serializer.StringEncoder")
-    props.put("producer.type", "sync")
-    props.put("request.required.acks", "1")
-    props.put("message.send.max.retries", "1")
-    val producerConfig = new ProducerConfig(props)
-    val producer = new Producer[String, String](producerConfig)
-    try {
-      producer.send(new KeyedMessage[String, String](topic, "test", "test1"))
-    } catch {
-      case e: FailedToSendMessageException => fail("Topic should have been auto created")
-      case oe: Throwable => fail("fails with exception", oe)
-    }
-    // test the topic path exists
-    assertTrue("Topic not auto created", ZkUtils.pathExists(zkClient, ZkUtils.getTopicPath(topic)))
-    // wait until leader is elected
-    val leaderIdOpt = TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0, 1000)
-    assertTrue("New leader should be elected after re-creating topic test", leaderIdOpt.isDefined)
-    servers.foreach(_.shutdown())
-  }
-
-  @Test
   def testDeleteNonExistingTopic() {
     val topicAndPartition = TopicAndPartition("test", 0)
     val topic = topicAndPartition.topic


[20/37] git commit: kafka-1462; Add new request and response formats for the new consumer and coordinator communication; patched by Jun Rao; reviewed by Guozhang Wang and Jay Kreps

Posted by jj...@apache.org.
kafka-1462; Add new request and response formats for the new consumer and coordinator communication; patched by Jun Rao; reviewed by Guozhang Wang and Jay Kreps


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/fc0e03f7
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/fc0e03f7
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/fc0e03f7

Branch: refs/heads/transactional_messaging
Commit: fc0e03f79131746da81c05c12e056862c08d79d4
Parents: 4ebcdfd
Author: Jun Rao <ju...@gmail.com>
Authored: Thu Jul 17 18:20:01 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Thu Jul 17 18:20:01 2014 -0700

----------------------------------------------------------------------
 .../clients/producer/internals/Sender.java      |  12 +-
 .../java/org/apache/kafka/common/Cluster.java   |  15 +-
 .../apache/kafka/common/protocol/ApiKeys.java   |   7 +-
 .../apache/kafka/common/protocol/Protocol.java  | 280 ++++++++++++++++++-
 .../kafka/common/protocol/types/Struct.java     |  67 +++++
 .../requests/AbstractRequestResponse.java       |  66 +++++
 .../requests/ConsumerMetadataRequest.java       |  47 ++++
 .../requests/ConsumerMetadataResponse.java      |  69 +++++
 .../kafka/common/requests/FetchRequest.java     | 132 +++++++++
 .../kafka/common/requests/FetchResponse.java    | 110 ++++++++
 .../kafka/common/requests/HeartbeatRequest.java |  64 +++++
 .../common/requests/HeartbeatResponse.java      |  45 +++
 .../kafka/common/requests/JoinGroupRequest.java |  87 ++++++
 .../common/requests/JoinGroupResponse.java      | 102 +++++++
 .../common/requests/ListOffsetRequest.java      | 114 ++++++++
 .../common/requests/ListOffsetResponse.java     | 108 +++++++
 .../kafka/common/requests/MetadataRequest.java  |  29 +-
 .../kafka/common/requests/MetadataResponse.java |  98 +++++--
 .../common/requests/OffsetCommitRequest.java    | 180 ++++++++++++
 .../common/requests/OffsetCommitResponse.java   |  87 ++++++
 .../common/requests/OffsetFetchRequest.java     |  98 +++++++
 .../common/requests/OffsetFetchResponse.java    | 107 +++++++
 .../kafka/common/requests/ProduceRequest.java   | 124 +++++---
 .../kafka/common/requests/ProduceResponse.java  |  96 ++++---
 .../kafka/common/requests/RequestHeader.java    |  49 ++--
 .../kafka/common/requests/ResponseHeader.java   |  22 +-
 .../kafka/common/utils/CollectionUtils.java     |  62 ++++
 .../apache/kafka/clients/NetworkClientTest.java |   4 +-
 .../common/requests/RequestResponseTest.java    | 173 ++++++++++++
 .../kafka/api/ConsumerMetadataRequest.scala     |   4 +-
 .../kafka/api/ConsumerMetadataResponse.scala    |   4 +-
 .../kafka/api/ControlledShutdownRequest.scala   |   4 +-
 .../kafka/api/ControlledShutdownResponse.scala  |   4 +-
 .../src/main/scala/kafka/api/FetchRequest.scala |   4 +-
 .../api/GenericRequestOrResponseAndHeader.scala |  45 +++
 .../kafka/api/HeartbeatRequestAndHeader.scala   |  39 +++
 .../kafka/api/HeartbeatResponseAndHeader.scala  |  28 ++
 .../kafka/api/JoinGroupRequestAndHeader.scala   |  40 +++
 .../kafka/api/JoinGroupResponseAndHeader.scala  |  28 ++
 .../scala/kafka/api/LeaderAndIsrRequest.scala   |   4 +-
 .../scala/kafka/api/LeaderAndIsrResponse.scala  |   4 +-
 .../scala/kafka/api/OffsetCommitRequest.scala   |  38 ++-
 .../scala/kafka/api/OffsetCommitResponse.scala  |   4 +-
 .../scala/kafka/api/OffsetFetchRequest.scala    |   4 +-
 .../scala/kafka/api/OffsetFetchResponse.scala   |   4 +-
 .../main/scala/kafka/api/OffsetRequest.scala    |   4 +-
 .../main/scala/kafka/api/OffsetResponse.scala   |   4 +-
 .../main/scala/kafka/api/ProducerRequest.scala  |   4 +-
 .../main/scala/kafka/api/ProducerResponse.scala |   4 +-
 core/src/main/scala/kafka/api/RequestKeys.scala |   7 +-
 .../scala/kafka/api/RequestOrResponse.scala     |   2 +-
 .../scala/kafka/api/StopReplicaRequest.scala    |   4 +-
 .../scala/kafka/api/StopReplicaResponse.scala   |   4 +-
 .../scala/kafka/api/TopicMetadataRequest.scala  |   4 +-
 .../scala/kafka/api/TopicMetadataResponse.scala |   4 +-
 .../scala/kafka/api/UpdateMetadataRequest.scala |   4 +-
 .../kafka/api/UpdateMetadataResponse.scala      |   4 +-
 .../controller/ControllerChannelManager.scala   |   8 +-
 .../kafka/javaapi/OffsetCommitRequest.scala     |   2 -
 .../kafka/javaapi/TopicMetadataRequest.scala    |   4 +-
 .../api/RequestResponseSerializationTest.scala  |  74 ++++-
 61 files changed, 2610 insertions(+), 239 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
index a016269..8ebe7ed 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
@@ -12,6 +12,7 @@
  */
 package org.apache.kafka.clients.producer.internals;
 
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -276,13 +277,16 @@ public class Sender implements Runnable {
      * Create a produce request from the given record batches
      */
     private ClientRequest produceRequest(long now, int destination, short acks, int timeout, List<RecordBatch> batches) {
-        ProduceRequest request = new ProduceRequest(acks, timeout);
+        Map<TopicPartition, ByteBuffer> produceRecordsByPartition = new HashMap<TopicPartition, ByteBuffer>(batches.size());
         Map<TopicPartition, RecordBatch> recordsByPartition = new HashMap<TopicPartition, RecordBatch>(batches.size());
         for (RecordBatch batch : batches) {
-            batch.records.buffer().flip();
-            request.add(batch.topicPartition, batch.records);
-            recordsByPartition.put(batch.topicPartition, batch);
+            TopicPartition tp = batch.topicPartition;
+            ByteBuffer recordsBuffer = batch.records.buffer();
+            recordsBuffer.flip();
+            produceRecordsByPartition.put(tp, recordsBuffer);
+            recordsByPartition.put(tp, batch);
         }
+        ProduceRequest request = new ProduceRequest(acks, timeout, produceRecordsByPartition);
         RequestSend send = new RequestSend(destination, this.client.nextRequestHeader(ApiKeys.PRODUCE), request.toStruct());
         return new ClientRequest(now, acks != 0, send, recordsByPartition);
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/Cluster.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java
index c62707a..d3299b9 100644
--- a/clients/src/main/java/org/apache/kafka/common/Cluster.java
+++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java
@@ -15,12 +15,7 @@ package org.apache.kafka.common;
 import org.apache.kafka.common.utils.Utils;
 
 import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 
 /**
  * A representation of a subset of the nodes, topics, and partitions in the Kafka cluster.
@@ -143,6 +138,14 @@ public final class Cluster {
         return this.partitionsByNode.get(nodeId);
     }
 
+    /**
+     * Get all topics.
+     * @return a set of all topics
+     */
+    public Set<String> topics() {
+        return this.partitionsByTopic.keySet();
+    }
+
     @Override
     public String toString() {
         return "Cluster(nodes = " + this.nodes + ", partitions = " + this.partitionsByTopicPartition.values() + ")";

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
index 6fe7573..109fc96 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
@@ -30,8 +30,11 @@ public enum ApiKeys {
     METADATA(3, "metadata"),
     LEADER_AND_ISR(4, "leader_and_isr"),
     STOP_REPLICA(5, "stop_replica"),
-    OFFSET_COMMIT(6, "offset_commit"),
-    OFFSET_FETCH(7, "offset_fetch");
+    OFFSET_COMMIT(8, "offset_commit"),
+    OFFSET_FETCH(9, "offset_fetch"),
+    CONSUMER_METADATA(10, "consumer_metadata"),
+    JOIN_GROUP(11, "join_group"),
+    HEARTBEAT(12, "heartbeat");
 
     private static ApiKeys[] codeToType;
     public static int MAX_API_KEY = -1;

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
index 044b030..7517b87 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
@@ -104,6 +104,264 @@ public class Protocol {
     public static Schema[] PRODUCE_REQUEST = new Schema[] { PRODUCE_REQUEST_V0 };
     public static Schema[] PRODUCE_RESPONSE = new Schema[] { PRODUCE_RESPONSE_V0 };
 
+    /* Offset commit api */
+    public static Schema OFFSET_COMMIT_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
+                                                                                   INT32,
+                                                                                   "Topic partition id."),
+                                                                         new Field("offset",
+                                                                                   INT64,
+                                                                                   "Message offset to be committed."),
+                                                                         new Field("timestamp",
+                                                                                   INT64,
+                                                                                   "Timestamp of the commit"),
+                                                                         new Field("metadata",
+                                                                                   STRING,
+                                                                                   "Any associated metadata the client wants to keep."));
+
+    public static Schema OFFSET_COMMIT_REQUEST_TOPIC_V0 = new Schema(new Field("topic",
+                                                                                STRING,
+                                                                                "Topic to commit."),
+                                                                       new Field("partitions",
+                                                                                 new ArrayOf(OFFSET_COMMIT_REQUEST_PARTITION_V0),
+                                                                                 "Partitions to commit offsets."));
+
+    public static Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id",
+                                                                         STRING,
+                                                                         "The consumer group id."),
+                                                               new Field("topics",
+                                                                         new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0),
+                                                                         "Topics to commit offsets."));
+
+    public static Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(new Field("group_id",
+                                                                         STRING,
+                                                                         "The consumer group id."),
+                                                               new Field("group_generation_id",
+                                                                         INT32,
+                                                                         "The generation of the consumer group."),
+                                                               new Field("consumer_id",
+                                                                         STRING,
+                                                                         "The consumer id assigned by the group coordinator."),
+                                                               new Field("topics",
+                                                                         new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0),
+                                                                         "Topics to commit offsets."));
+
+    public static Schema OFFSET_COMMIT_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
+                                                                                    INT32,
+                                                                                    "Topic partition id."),
+                                                                          new Field("error_code",
+                                                                                    INT16));
+
+    public static Schema OFFSET_COMMIT_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
+                                                                      new Field("partition_responses",
+                                                                                new ArrayOf(OFFSET_COMMIT_RESPONSE_PARTITION_V0)));
+
+    public static Schema OFFSET_COMMIT_RESPONSE_V0 = new Schema(new Field("responses",
+                                                                          new ArrayOf(OFFSET_COMMIT_RESPONSE_TOPIC_V0)));
+
+    public static Schema[] OFFSET_COMMIT_REQUEST = new Schema[] { OFFSET_COMMIT_REQUEST_V0, OFFSET_COMMIT_REQUEST_V1 };
+    /* The response types for both V0 and V1 of OFFSET_COMMIT_REQUEST are the same. */
+    public static Schema[] OFFSET_COMMIT_RESPONSE = new Schema[] { OFFSET_COMMIT_RESPONSE_V0, OFFSET_COMMIT_RESPONSE_V0};
+
+    /* Offset fetch api */
+    public static Schema OFFSET_FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
+                                                                                  INT32,
+                                                                                  "Topic partition id."));
+
+    public static Schema OFFSET_FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic",
+                                                                               STRING,
+                                                                               "Topic to fetch offset."),
+                                                                     new Field("partitions",
+                                                                                new ArrayOf(OFFSET_FETCH_REQUEST_PARTITION_V0),
+                                                                                "Partitions to fetch offsets."));
+
+    public static Schema OFFSET_FETCH_REQUEST_V0 = new Schema(new Field("group_id",
+                                                                        STRING,
+                                                                        "The consumer group id."),
+                                                              new Field("topics",
+                                                                        new ArrayOf(OFFSET_FETCH_REQUEST_TOPIC_V0),
+                                                                        "Topics to fetch offsets."));
+
+    public static Schema OFFSET_FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
+                                                                                   INT32,
+                                                                                   "Topic partition id."),
+                                                                         new Field("offset",
+                                                                                   INT64,
+                                                                                   "Last committed message offset."),
+                                                                         new Field("metadata",
+                                                                                   STRING,
+                                                                                   "Any associated metadata the client wants to keep."),
+                                                                         new Field("error_code",
+                                                                                   INT16));
+
+    public static Schema OFFSET_FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
+                                                                     new Field("partition_responses",
+                                                                               new ArrayOf(OFFSET_FETCH_RESPONSE_PARTITION_V0)));
+
+    public static Schema OFFSET_FETCH_RESPONSE_V0 = new Schema(new Field("responses",
+                                                                         new ArrayOf(OFFSET_FETCH_RESPONSE_TOPIC_V0)));
+
+    public static Schema[] OFFSET_FETCH_REQUEST = new Schema[] { OFFSET_FETCH_REQUEST_V0 };
+    public static Schema[] OFFSET_FETCH_RESPONSE = new Schema[] { OFFSET_FETCH_RESPONSE_V0 };
+
+    /* List offset api */
+    public static Schema LIST_OFFSET_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
+                                                                                 INT32,
+                                                                                 "Topic partition id."),
+                                                                       new Field("timestamp",
+                                                                                 INT64,
+                                                                                 "Timestamp."),
+                                                                       new Field("max_num_offsets",
+                                                                                 INT32,
+                                                                                 "Maximum offsets to return."));
+
+    public static Schema LIST_OFFSET_REQUEST_TOPIC_V0 = new Schema(new Field("topic",
+                                                                             STRING,
+                                                                             "Topic to list offset."),
+                                                                   new Field("partitions",
+                                                                             new ArrayOf(LIST_OFFSET_REQUEST_PARTITION_V0),
+                                                                             "Partitions to list offset."));
+
+    public static Schema LIST_OFFSET_REQUEST_V0 = new Schema(new Field("replica_id",
+                                                                       INT32,
+                                                                       "Broker id of the follower. For normal consumers, use -1."),
+                                                             new Field("topics",
+                                                                        new ArrayOf(LIST_OFFSET_REQUEST_TOPIC_V0),
+                                                                        "Topics to list offsets."));
+
+    public static Schema LIST_OFFSET_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
+                                                                                  INT32,
+                                                                                  "Topic partition id."),
+                                                                        new Field("error_code",
+                                                                                  INT16),
+                                                                        new Field("offsets",
+                                                                                  new ArrayOf(INT64),
+                                                                                  "A list of offsets."));
+
+    public static Schema LIST_OFFSET_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
+                                                                    new Field("partition_responses",
+                                                                              new ArrayOf(LIST_OFFSET_RESPONSE_PARTITION_V0)));
+
+    public static Schema LIST_OFFSET_RESPONSE_V0 = new Schema(new Field("responses",
+                                                                  new ArrayOf(LIST_OFFSET_RESPONSE_TOPIC_V0)));
+
+    public static Schema[] LIST_OFFSET_REQUEST = new Schema[] { LIST_OFFSET_REQUEST_V0 };
+    public static Schema[] LIST_OFFSET_RESPONSE = new Schema[] { LIST_OFFSET_RESPONSE_V0 };
+
+    /* Fetch api */
+    public static Schema FETCH_REQUEST_PARTITION_V0 = new Schema(new Field("partition",
+                                                                           INT32,
+                                                                           "Topic partition id."),
+                                                                 new Field("fetch_offset",
+                                                                           INT64,
+                                                                           "Message offset."),
+                                                                 new Field("max_bytes",
+                                                                           INT32,
+                                                                           "Maximum bytes to fetch."));
+
+    public static Schema FETCH_REQUEST_TOPIC_V0 = new Schema(new Field("topic",
+                                                                       STRING,
+                                                                       "Topic to fetch."),
+                                                             new Field("partitions",
+                                                                       new ArrayOf(FETCH_REQUEST_PARTITION_V0),
+                                                                       "Partitions to fetch."));
+
+    public static Schema FETCH_REQUEST_V0 = new Schema(new Field("replica_id",
+                                                                 INT32,
+                                                                 "Broker id of the follower. For normal consumers, use -1."),
+                                                       new Field("max_wait_time",
+                                                                 INT32,
+                                                                 "Maximum time in ms to wait for the response."),
+                                                       new Field("min_bytes",
+                                                                 INT32,
+                                                                 "Minimum bytes to accumulate in the response."),
+                                                       new Field("topics",
+                                                                 new ArrayOf(FETCH_REQUEST_TOPIC_V0),
+                                                                 "Topics to fetch."));
+
+    public static Schema FETCH_RESPONSE_PARTITION_V0 = new Schema(new Field("partition",
+                                                                            INT32,
+                                                                            "Topic partition id."),
+                                                                  new Field("error_code",
+                                                                            INT16),
+                                                                  new Field("high_watermark",
+                                                                            INT64,
+                                                                            "Last committed offset."),
+                                                                  new Field("record_set", BYTES));
+
+    public static Schema FETCH_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
+                                                              new Field("partition_responses",
+                                                                        new ArrayOf(FETCH_RESPONSE_PARTITION_V0)));
+
+    public static Schema FETCH_RESPONSE_V0 = new Schema(new Field("responses",
+                                                                  new ArrayOf(FETCH_RESPONSE_TOPIC_V0)));
+
+    public static Schema[] FETCH_REQUEST = new Schema[] { FETCH_REQUEST_V0 };
+    public static Schema[] FETCH_RESPONSE = new Schema[] { FETCH_RESPONSE_V0 };
+
+    /* Consumer metadata api */
+    public static Schema CONSUMER_METADATA_REQUEST_V0 = new Schema(new Field("group_id",
+                                                                             STRING,
+                                                                             "The consumer group id."));
+
+    public static Schema CONSUMER_METADATA_RESPONSE_V0 = new Schema(new Field("error_code",
+                                                                              INT16),
+                                                                    new Field("coordinator",
+                                                                              BROKER,
+                                                                              "Host and port information for the coordinator for a consumer group."));
+
+    public static Schema[] CONSUMER_METADATA_REQUEST = new Schema[] { CONSUMER_METADATA_REQUEST_V0 };
+    public static Schema[] CONSUMER_METADATA_RESPONSE = new Schema[] { CONSUMER_METADATA_RESPONSE_V0 };
+
+    /* Join group api */
+    public static Schema JOIN_GROUP_REQUEST_V0 = new Schema(new Field("group_id",
+                                                                      STRING,
+                                                                      "The consumer group id."),
+                                                            new Field("session_timeout",
+                                                                      INT32,
+                                                                      "The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms."),
+                                                            new Field("topics",
+                                                                      new ArrayOf(STRING),
+                                                                      "An array of topics to subscribe to."),
+                                                            new Field("consumer_id",
+                                                                      STRING,
+                                                                      "The assigned consumer id or an empty string for a new consumer."),
+                                                            new Field("partition_assignment_strategy",
+                                                                      STRING,
+                                                                      "The strategy for the coordinator to assign partitions."));
+
+    public static Schema JOIN_GROUP_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
+                                                                   new Field("partitions", new ArrayOf(INT32)));
+    public static Schema JOIN_GROUP_RESPONSE_V0 = new Schema(new Field("error_code",
+                                                                       INT16),
+                                                             new Field("group_generation_id",
+                                                                       INT32,
+                                                                       "The generation of the consumer group."),
+                                                             new Field("consumer_id",
+                                                                       STRING,
+                                                                       "The consumer id assigned by the group coordinator."),
+                                                             new Field("assigned_partitions",
+                                                                       new ArrayOf(JOIN_GROUP_RESPONSE_TOPIC_V0)));
+
+    public static Schema[] JOIN_GROUP_REQUEST = new Schema[] { JOIN_GROUP_REQUEST_V0 };
+    public static Schema[] JOIN_GROUP_RESPONSE = new Schema[] { JOIN_GROUP_RESPONSE_V0 };
+
+    /* Heartbeat api */
+    public static Schema HEARTBEAT_REQUEST_V0 = new Schema(new Field("group_id",
+                                                                      STRING,
+                                                                      "The consumer group id."),
+                                                            new Field("group_generation_id",
+                                                                      INT32,
+                                                                      "The generation of the consumer group."),
+                                                            new Field("consumer_id",
+                                                                      STRING,
+                                                                      "The consumer id assigned by the group coordinator."));
+
+    public static Schema HEARTBEAT_RESPONSE_V0 = new Schema(new Field("error_code",
+                                                                       INT16));
+
+    public static Schema[] HEARTBEAT_REQUEST = new Schema[] {HEARTBEAT_REQUEST_V0};
+    public static Schema[] HEARTBEAT_RESPONSE = new Schema[] {HEARTBEAT_RESPONSE_V0};
+
     /* an array of all requests and responses with all schema versions */
     public static Schema[][] REQUESTS = new Schema[ApiKeys.MAX_API_KEY + 1][];
     public static Schema[][] RESPONSES = new Schema[ApiKeys.MAX_API_KEY + 1][];
@@ -113,22 +371,28 @@ public class Protocol {
 
     static {
         REQUESTS[ApiKeys.PRODUCE.id] = PRODUCE_REQUEST;
-        REQUESTS[ApiKeys.FETCH.id] = new Schema[] {};
-        REQUESTS[ApiKeys.LIST_OFFSETS.id] = new Schema[] {};
+        REQUESTS[ApiKeys.FETCH.id] = FETCH_REQUEST;
+        REQUESTS[ApiKeys.LIST_OFFSETS.id] = LIST_OFFSET_REQUEST;
         REQUESTS[ApiKeys.METADATA.id] = METADATA_REQUEST;
         REQUESTS[ApiKeys.LEADER_AND_ISR.id] = new Schema[] {};
         REQUESTS[ApiKeys.STOP_REPLICA.id] = new Schema[] {};
-        REQUESTS[ApiKeys.OFFSET_COMMIT.id] = new Schema[] {};
-        REQUESTS[ApiKeys.OFFSET_FETCH.id] = new Schema[] {};
+        REQUESTS[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_REQUEST;
+        REQUESTS[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_REQUEST;
+        REQUESTS[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_REQUEST;
+        REQUESTS[ApiKeys.JOIN_GROUP.id] = JOIN_GROUP_REQUEST;
+        REQUESTS[ApiKeys.HEARTBEAT.id] = HEARTBEAT_REQUEST;
 
         RESPONSES[ApiKeys.PRODUCE.id] = PRODUCE_RESPONSE;
-        RESPONSES[ApiKeys.FETCH.id] = new Schema[] {};
-        RESPONSES[ApiKeys.LIST_OFFSETS.id] = new Schema[] {};
+        RESPONSES[ApiKeys.FETCH.id] = FETCH_RESPONSE;
+        RESPONSES[ApiKeys.LIST_OFFSETS.id] = LIST_OFFSET_RESPONSE;
         RESPONSES[ApiKeys.METADATA.id] = METADATA_RESPONSE;
         RESPONSES[ApiKeys.LEADER_AND_ISR.id] = new Schema[] {};
         RESPONSES[ApiKeys.STOP_REPLICA.id] = new Schema[] {};
-        RESPONSES[ApiKeys.OFFSET_COMMIT.id] = new Schema[] {};
-        RESPONSES[ApiKeys.OFFSET_FETCH.id] = new Schema[] {};
+        RESPONSES[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_RESPONSE;
+        RESPONSES[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_RESPONSE;
+        RESPONSES[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_RESPONSE;
+        RESPONSES[ApiKeys.JOIN_GROUP.id] = JOIN_GROUP_RESPONSE;
+        RESPONSES[ApiKeys.HEARTBEAT.id] = HEARTBEAT_RESPONSE;
 
         /* set the maximum version of each api */
         for (ApiKeys api : ApiKeys.values())

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
index 8cecba5..444e69e 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
@@ -83,6 +83,15 @@ public class Struct {
         return getFieldOrDefault(field);
     }
 
+    /**
+     * Check if the struct contains a field.
+     * @param name
+     * @return
+     */
+    public boolean hasField(String name) {
+        return schema.get(name) != null;
+    }
+
     public Struct getStruct(Field field) {
         return (Struct) get(field);
     }
@@ -107,6 +116,22 @@ public class Struct {
         return (Integer) get(name);
     }
 
+    public Long getLong(Field field) {
+        return (Long) get(field);
+    }
+
+    public Long getLong(String name) {
+        return (Long) get(name);
+    }
+
+    public ByteBuffer getBytes(Field field) {
+        return (ByteBuffer) get(field);
+    }
+
+    public ByteBuffer getBytes(String name) {
+        return (ByteBuffer) get(name);
+    }
+
     public Object[] getArray(Field field) {
         return (Object[]) get(field);
     }
@@ -253,4 +278,46 @@ public class Struct {
         return b.toString();
     }
 
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        for (int i = 0; i < this.values.length; i++) {
+            Field f = this.schema.get(i);
+            if (f.type() instanceof ArrayOf) {
+                Object[] arrayObject = (Object []) this.get(f);
+                for (Object arrayItem: arrayObject)
+                    result = prime * result + arrayItem.hashCode();
+            } else {
+                result = prime * result + this.get(f).hashCode();
+            }
+        }
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        Struct other = (Struct) obj;
+        if (schema != other.schema)
+            return false;
+        for (int i = 0; i < this.values.length; i++) {
+            Field f = this.schema.get(i);
+            Boolean result;
+            if (f.type() instanceof ArrayOf) {
+                result = Arrays.equals((Object []) this.get(f), (Object []) other.get(f));
+            } else {
+                result = this.get(f).equals(other.get(f));
+            }
+            if (!result)
+                return false;
+        }
+        return true;
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java
new file mode 100644
index 0000000..37aff6c
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequestResponse.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+public abstract class AbstractRequestResponse {
+    protected final Struct struct;
+
+
+    public AbstractRequestResponse(Struct struct) {
+        this.struct = struct;
+    }
+
+    public Struct toStruct() {
+        return struct;
+    }
+
+    /**
+     * Get the serialized size of this object
+     */
+    public int sizeOf() {
+        return struct.sizeOf();
+    }
+
+    /**
+     * Write this object to a buffer
+     */
+    public void writeTo(ByteBuffer buffer) {
+        struct.writeTo(buffer);
+    }
+
+    @Override
+    public String toString() {
+        return struct.toString();
+    }
+
+    @Override
+    public int hashCode() {
+        return struct.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        AbstractRequestResponse other = (AbstractRequestResponse) obj;
+        return struct.equals(other.struct);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java
new file mode 100644
index 0000000..99b52c2
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+public class ConsumerMetadataRequest extends AbstractRequestResponse {
+    public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.CONSUMER_METADATA.id);
+    private static String GROUP_ID_KEY_NAME = "group_id";
+
+    private final String groupId;
+
+    public ConsumerMetadataRequest(String groupId) {
+        super(new Struct(curSchema));
+
+        struct.set(GROUP_ID_KEY_NAME, groupId);
+        this.groupId = groupId;
+    }
+
+    public ConsumerMetadataRequest(Struct struct) {
+        super(struct);
+        groupId = struct.getString(GROUP_ID_KEY_NAME);
+    }
+
+    public String groupId() {
+        return groupId;
+    }
+
+    public static ConsumerMetadataRequest parse(ByteBuffer buffer) {
+        return new ConsumerMetadataRequest(((Struct) curSchema.read(buffer)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java
new file mode 100644
index 0000000..8b8f591
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java
@@ -0,0 +1,69 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+public class ConsumerMetadataResponse extends AbstractRequestResponse {
+    private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.CONSUMER_METADATA.id);
+    private static String ERROR_CODE_KEY_NAME = "error_code";
+    private static String COORDINATOR_KEY_NAME = "coordinator";
+
+    // coordinator level field names
+    private static String NODE_ID_KEY_NAME = "node_id";
+    private static String HOST_KEY_NAME = "host";
+    private static String PORT_KEY_NAME = "port";
+
+    private final short errorCode;
+    private final Node node;
+
+    public ConsumerMetadataResponse(short errorCode, Node node) {
+        super(new Struct(curSchema));
+        struct.set(ERROR_CODE_KEY_NAME, errorCode);
+        Struct coordinator = struct.instance(COORDINATOR_KEY_NAME);
+        coordinator.set(NODE_ID_KEY_NAME, node.id());
+        coordinator.set(HOST_KEY_NAME, node.host());
+        coordinator.set(PORT_KEY_NAME, node.port());
+        struct.set(COORDINATOR_KEY_NAME, coordinator);
+        this.errorCode = errorCode;
+        this.node = node;
+    }
+
+    public ConsumerMetadataResponse(Struct struct) {
+        super(struct);
+        errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
+        Struct broker = (Struct) struct.get(COORDINATOR_KEY_NAME);
+        int nodeId = broker.getInt(NODE_ID_KEY_NAME);
+        String host = broker.getString(HOST_KEY_NAME);
+        int port = broker.getInt(PORT_KEY_NAME);
+        node = new Node(nodeId, host, port);
+    }
+
+    public short errorCode() {
+        return errorCode;
+    }
+
+    public Node node() {
+        return node;
+    }
+
+    public static ConsumerMetadataResponse parse(ByteBuffer buffer) {
+        return new ConsumerMetadataResponse(((Struct) curSchema.read(buffer)));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
new file mode 100644
index 0000000..2fc471f
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
@@ -0,0 +1,132 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class FetchRequest extends AbstractRequestResponse {
+    public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.FETCH.id);
+    private static String REPLICA_ID_KEY_NAME = "replica_id";
+    private static String MAX_WAIT_KEY_NAME = "max_wait_time";
+    private static String MIN_BYTES_KEY_NAME = "min_bytes";
+    private static String TOPICS_KEY_NAME = "topics";
+
+    // topic level field names
+    private static String TOPIC_KEY_NAME = "topic";
+    private static String PARTITIONS_KEY_NAME = "partitions";
+
+    // partition level field names
+    private static String PARTITION_KEY_NAME = "partition";
+    private static String FETCH_OFFSET_KEY_NAME = "fetch_offset";
+    private static String MAX_BYTES_KEY_NAME = "max_bytes";
+
+    private final int replicaId;
+    private final int maxWait;
+    private final int minBytes;
+    private final Map<TopicPartition, PartitionData> fetchData;
+
+    public static final class PartitionData {
+        public final long offset;
+        public final int maxBytes;
+
+        public PartitionData(long offset, int maxBytes) {
+            this.offset = offset;
+            this.maxBytes = maxBytes;
+        }
+    }
+
+    public FetchRequest(int replicaId, int maxWait, int minBytes, Map<TopicPartition, PartitionData> fetchData) {
+        super(new Struct(curSchema));
+        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(fetchData);
+
+        struct.set(REPLICA_ID_KEY_NAME, replicaId);
+        struct.set(MAX_WAIT_KEY_NAME, maxWait);
+        struct.set(MIN_BYTES_KEY_NAME, minBytes);
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
+            Struct topicData = struct.instance(TOPICS_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
+                PartitionData fetchPartitionData = partitionEntry.getValue();
+                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(FETCH_OFFSET_KEY_NAME, fetchPartitionData.offset);
+                partitionData.set(MAX_BYTES_KEY_NAME, fetchPartitionData.maxBytes);
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
+        this.replicaId = replicaId;
+        this.maxWait = maxWait;
+        this.minBytes = minBytes;
+        this.fetchData = fetchData;
+    }
+
+    public FetchRequest(Struct struct) {
+        super(struct);
+        replicaId = struct.getInt(REPLICA_ID_KEY_NAME);
+        maxWait = struct.getInt(MAX_WAIT_KEY_NAME);
+        minBytes = struct.getInt(MIN_BYTES_KEY_NAME);
+        fetchData = new HashMap<TopicPartition, PartitionData>();
+        for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
+            Struct topicResponse = (Struct) topicResponseObj;
+            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                long offset = partitionResponse.getLong(FETCH_OFFSET_KEY_NAME);
+                int maxBytes = partitionResponse.getInt(MAX_BYTES_KEY_NAME);
+                PartitionData partitionData = new PartitionData(offset, maxBytes);
+                fetchData.put(new TopicPartition(topic, partition), partitionData);
+            }
+        }
+    }
+
+    public int replicaId() {
+        return replicaId;
+    }
+
+    public int maxWait() {
+        return maxWait;
+    }
+
+    public int minBytes() {
+        return minBytes;
+    }
+
+    public Map<TopicPartition, PartitionData> fetchData() {
+        return fetchData;
+    }
+
+    public static FetchRequest parse(ByteBuffer buffer) {
+        return new FetchRequest(((Struct) curSchema.read(buffer)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
new file mode 100644
index 0000000..f719010
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
@@ -0,0 +1,110 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class FetchResponse extends AbstractRequestResponse {
+    public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.FETCH.id);
+    private static String RESPONSES_KEY_NAME = "responses";
+
+    // topic level field names
+    private static String TOPIC_KEY_NAME = "topic";
+    private static String PARTITIONS_KEY_NAME = "partition_responses";
+
+    // partition level field names
+    private static String PARTITION_KEY_NAME = "partition";
+    private static String ERROR_CODE_KEY_NAME = "error_code";
+    private static String HIGH_WATERMARK_KEY_NAME = "high_watermark";
+    private static String RECORD_SET_KEY_NAME = "record_set";
+
+    private final Map<TopicPartition, PartitionData> responseData;
+
+    public static final class PartitionData {
+        public final short errorCode;
+        public final long highWatermark;
+        public final ByteBuffer recordSet;
+
+        public PartitionData(short errorCode, long highWatermark, ByteBuffer recordSet) {
+            this.errorCode = errorCode;
+            this.highWatermark = highWatermark;
+            this.recordSet = recordSet;
+        }
+    }
+
+    public FetchResponse(Map<TopicPartition, PartitionData> responseData) {
+        super(new Struct(curSchema));
+        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
+
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
+            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
+                PartitionData fetchPartitionData = partitionEntry.getValue();
+                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(ERROR_CODE_KEY_NAME, fetchPartitionData.errorCode);
+                partitionData.set(HIGH_WATERMARK_KEY_NAME, fetchPartitionData.highWatermark);
+                partitionData.set(RECORD_SET_KEY_NAME, fetchPartitionData.recordSet);
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
+        this.responseData = responseData;
+    }
+
+    public FetchResponse(Struct struct) {
+        super(struct);
+        responseData = new HashMap<TopicPartition, PartitionData>();
+        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
+            Struct topicResponse = (Struct) topicResponseObj;
+            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
+                long highWatermark = partitionResponse.getLong(HIGH_WATERMARK_KEY_NAME);
+                ByteBuffer recordSet = partitionResponse.getBytes(RECORD_SET_KEY_NAME);
+                PartitionData partitionData = new PartitionData(errorCode, highWatermark, recordSet);
+                responseData.put(new TopicPartition(topic, partition), partitionData);
+            }
+        }
+    }
+
+    public Map<TopicPartition, PartitionData> responseData() {
+        return responseData;
+    }
+
+    public static FetchResponse parse(ByteBuffer buffer) {
+        return new FetchResponse(((Struct) curSchema.read(buffer)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java
new file mode 100644
index 0000000..9512db2
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+public class HeartbeatRequest extends AbstractRequestResponse {
+    public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.HEARTBEAT.id);
+    private static String GROUP_ID_KEY_NAME = "group_id";
+    private static String GROUP_GENERATION_ID_KEY_NAME = "group_generation_id";
+    private static String CONSUMER_ID_KEY_NAME = "consumer_id";
+
+    private final String groupId;
+    private final int groupGenerationId;
+    private final String consumerId;
+
+    public HeartbeatRequest(String groupId, int groupGenerationId, String consumerId) {
+        super(new Struct(curSchema));
+        struct.set(GROUP_ID_KEY_NAME, groupId);
+        struct.set(GROUP_GENERATION_ID_KEY_NAME, groupGenerationId);
+        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
+        this.groupId = groupId;
+        this.groupGenerationId = groupGenerationId;
+        this.consumerId = consumerId;
+    }
+
+    public HeartbeatRequest(Struct struct) {
+        super(struct);
+        groupId = struct.getString(GROUP_ID_KEY_NAME);
+        groupGenerationId = struct.getInt(GROUP_GENERATION_ID_KEY_NAME);
+        consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
+    }
+
+    public String groupId() {
+        return groupId;
+    }
+
+    public int groupGenerationId() {
+        return groupGenerationId;
+    }
+
+    public String consumerId() {
+        return consumerId;
+    }
+
+    public static HeartbeatRequest parse(ByteBuffer buffer) {
+        return new HeartbeatRequest(((Struct) curSchema.read(buffer)));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java
new file mode 100644
index 0000000..8997ffc
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java
@@ -0,0 +1,45 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+public class HeartbeatResponse extends AbstractRequestResponse {
+    private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.HEARTBEAT.id);
+    private static String ERROR_CODE_KEY_NAME = "error_code";
+
+    private final short errorCode;
+    public HeartbeatResponse(short errorCode) {
+        super(new Struct(curSchema));
+        struct.set(ERROR_CODE_KEY_NAME, errorCode);
+        this.errorCode = errorCode;
+    }
+
+    public HeartbeatResponse(Struct struct) {
+        super(struct);
+        errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
+    }
+
+    public short errorCode() {
+        return errorCode;
+    }
+
+    public static HeartbeatResponse parse(ByteBuffer buffer) {
+        return new HeartbeatResponse(((Struct) curSchema.read(buffer)));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java
new file mode 100644
index 0000000..d6e91f3
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+public class JoinGroupRequest extends AbstractRequestResponse {
+    public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.JOIN_GROUP.id);
+    private static String GROUP_ID_KEY_NAME = "group_id";
+    private static String SESSION_TIMEOUT_KEY_NAME = "session_timeout";
+    private static String TOPICS_KEY_NAME = "topics";
+    private static String CONSUMER_ID_KEY_NAME = "consumer_id";
+    private static String STRATEGY_KEY_NAME = "partition_assignment_strategy";
+
+    private final String groupId;
+    private final int sessionTimeout;
+    private final List<String> topics;
+    private final String consumerId;
+    private final String strategy;
+
+    public JoinGroupRequest(String groupId, int sessionTimeout, List<String> topics, String consumerId, String strategy) {
+        super(new Struct(curSchema));
+        struct.set(GROUP_ID_KEY_NAME, groupId);
+        struct.set(SESSION_TIMEOUT_KEY_NAME, sessionTimeout);
+        struct.set(TOPICS_KEY_NAME, topics.toArray());
+        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
+        struct.set(STRATEGY_KEY_NAME, strategy);
+        this.groupId = groupId;
+        this.sessionTimeout = sessionTimeout;
+        this.topics = topics;
+        this.consumerId = consumerId;
+        this.strategy = strategy;
+    }
+
+    public JoinGroupRequest(Struct struct) {
+        super(struct);
+        groupId = struct.getString(GROUP_ID_KEY_NAME);
+        sessionTimeout = struct.getInt(SESSION_TIMEOUT_KEY_NAME);
+        Object[] topicsArray = struct.getArray(TOPICS_KEY_NAME);
+        topics = new ArrayList<String>();
+        for (Object topic: topicsArray)
+            topics.add((String) topic);
+        consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
+        strategy = struct.getString(STRATEGY_KEY_NAME);
+    }
+
+    public String groupId() {
+        return groupId;
+    }
+
+    public int sessionTimeout() {
+        return sessionTimeout;
+    }
+
+    public List<String> topics() {
+        return topics;
+    }
+
+    public String consumerId() {
+        return consumerId;
+    }
+
+    public String strategy() {
+        return strategy;
+    }
+
+    public static JoinGroupRequest parse(ByteBuffer buffer) {
+        return new JoinGroupRequest(((Struct) curSchema.read(buffer)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java
new file mode 100644
index 0000000..efe8979
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+public class JoinGroupResponse extends AbstractRequestResponse {
+    public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id);
+    private static String ERROR_CODE_KEY_NAME = "error_code";
+    private static String GENERATION_ID_KEY_NAME = "group_generation_id";
+    private static String CONSUMER_ID_KEY_NAME = "consumer_id";
+    private static String ASSIGNED_PARTITIONS_KEY_NAME = "assigned_partitions";
+    private static String TOPIC_KEY_NAME = "topic";
+    private static String PARTITIONS_KEY_NAME = "partitions";
+
+    public static int UNKNOWN_GENERATION_ID = -1;
+    public static String UNKNOWN_CONSUMER_ID = "";
+
+    private final short errorCode;
+    private final int generationId;
+    private final String consumerId;
+    private final List<TopicPartition> assignedPartitions;
+
+    public JoinGroupResponse(short errorCode, int generationId, String consumerId, List<TopicPartition> assignedPartitions) {
+        super(new Struct(curSchema));
+
+        Map<String, List<Integer>> partitionsByTopic = CollectionUtils.groupDataByTopic(assignedPartitions);
+
+        struct.set(ERROR_CODE_KEY_NAME, errorCode);
+        struct.set(GENERATION_ID_KEY_NAME, generationId);
+        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, List<Integer>> entries: partitionsByTopic.entrySet()) {
+            Struct topicData = struct.instance(ASSIGNED_PARTITIONS_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, entries.getKey());
+            topicData.set(PARTITIONS_KEY_NAME, entries.getValue().toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(ASSIGNED_PARTITIONS_KEY_NAME, topicArray.toArray());
+
+        this.errorCode = errorCode;
+        this.generationId = generationId;
+        this.consumerId = consumerId;
+        this.assignedPartitions = assignedPartitions;
+    }
+
+    public JoinGroupResponse(short errorCode) {
+        this(errorCode, UNKNOWN_GENERATION_ID, UNKNOWN_CONSUMER_ID, Collections.<TopicPartition>emptyList());
+    }
+
+    public JoinGroupResponse(Struct struct) {
+        super(struct);
+        assignedPartitions = new ArrayList<TopicPartition>();
+        for (Object topicDataObj : struct.getArray(ASSIGNED_PARTITIONS_KEY_NAME)) {
+            Struct topicData = (Struct) topicDataObj;
+            String topic = topicData.getString(TOPIC_KEY_NAME);
+            for (Object partitionObj : topicData.getArray(PARTITIONS_KEY_NAME))
+                assignedPartitions.add(new TopicPartition(topic, (Integer) partitionObj));
+        }
+        errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
+        generationId = struct.getInt(GENERATION_ID_KEY_NAME);
+        consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
+    }
+
+    public short errorCode() {
+        return errorCode;
+    }
+
+    public int generationId() {
+        return generationId;
+    }
+
+    public String consumerId() {
+        return consumerId;
+    }
+
+    public List<TopicPartition> assignedPartitions() {
+        return assignedPartitions;
+    }
+
+    public static JoinGroupResponse parse(ByteBuffer buffer) {
+        return new JoinGroupResponse(((Struct) curSchema.read(buffer)));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
new file mode 100644
index 0000000..99364c1
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ListOffsetRequest extends AbstractRequestResponse {
+    public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.LIST_OFFSETS.id);
+    private static String REPLICA_ID_KEY_NAME = "replica_id";
+    private static String TOPICS_KEY_NAME = "topics";
+
+    // topic level field names
+    private static String TOPIC_KEY_NAME = "topic";
+    private static String PARTITIONS_KEY_NAME = "partitions";
+
+    // partition level field names
+    private static String PARTITION_KEY_NAME = "partition";
+    private static String TIMESTAMP_KEY_NAME = "timestamp";
+    private static String MAX_NUM_OFFSETS_KEY_NAME = "max_num_offsets";
+
+    private final int replicaId;
+    private final Map<TopicPartition, PartitionData> offsetData;
+
+    public static final class PartitionData {
+        public final long timestamp;
+        public final int maxNumOffsets;
+
+        public PartitionData(long timestamp, int maxNumOffsets) {
+            this.timestamp = timestamp;
+            this.maxNumOffsets = maxNumOffsets;
+        }
+    }
+
+    public ListOffsetRequest(int replicaId, Map<TopicPartition, PartitionData> offsetData) {
+        super(new Struct(curSchema));
+        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(offsetData);
+
+        struct.set(REPLICA_ID_KEY_NAME, replicaId);
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
+            Struct topicData = struct.instance(TOPICS_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
+                PartitionData offsetPartitionData = partitionEntry.getValue();
+                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(TIMESTAMP_KEY_NAME, offsetPartitionData.timestamp);
+                partitionData.set(MAX_NUM_OFFSETS_KEY_NAME, offsetPartitionData.maxNumOffsets);
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(TOPICS_KEY_NAME, topicArray.toArray());
+        this.replicaId = replicaId;
+        this.offsetData = offsetData;
+    }
+
+    public ListOffsetRequest(Struct struct) {
+        super(struct);
+        replicaId = struct.getInt(REPLICA_ID_KEY_NAME);
+        offsetData = new HashMap<TopicPartition, PartitionData>();
+        for (Object topicResponseObj : struct.getArray(TOPICS_KEY_NAME)) {
+            Struct topicResponse = (Struct) topicResponseObj;
+            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                long timestamp = partitionResponse.getLong(TIMESTAMP_KEY_NAME);
+                int maxNumOffsets = partitionResponse.getInt(MAX_NUM_OFFSETS_KEY_NAME);
+                PartitionData partitionData = new PartitionData(timestamp, maxNumOffsets);
+                offsetData.put(new TopicPartition(topic, partition), partitionData);
+            }
+        }
+    }
+
+    public int replicaId() {
+        return replicaId;
+    }
+
+    public Map<TopicPartition, PartitionData> offsetData() {
+        return offsetData;
+    }
+
+    public static ListOffsetRequest parse(ByteBuffer buffer) {
+        return new ListOffsetRequest(((Struct) curSchema.read(buffer)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
new file mode 100644
index 0000000..ac23971
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.utils.CollectionUtils;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ListOffsetResponse extends AbstractRequestResponse {
+    public static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.LIST_OFFSETS.id);
+    private static String RESPONSES_KEY_NAME = "responses";
+
+    // topic level field names
+    private static String TOPIC_KEY_NAME = "topic";
+    private static String PARTITIONS_KEY_NAME = "partition_responses";
+
+    // partition level field names
+    private static String PARTITION_KEY_NAME = "partition";
+    private static String ERROR_CODE_KEY_NAME = "error_code";
+    private static String OFFSETS_KEY_NAME = "offsets";
+
+    private final Map<TopicPartition, PartitionData> responseData;
+
+    public static final class PartitionData {
+        public final short errorCode;
+        public final List<Long> offsets;
+
+        public PartitionData(short errorCode, List<Long> offsets) {
+            this.errorCode = errorCode;
+            this.offsets = offsets;
+        }
+    }
+
+    public ListOffsetResponse(Map<TopicPartition, PartitionData> responseData) {
+        super(new Struct(curSchema));
+        Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
+
+        List<Struct> topicArray = new ArrayList<Struct>();
+        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
+            Struct topicData = struct.instance(RESPONSES_KEY_NAME);
+            topicData.set(TOPIC_KEY_NAME, topicEntry.getKey());
+            List<Struct> partitionArray = new ArrayList<Struct>();
+            for (Map.Entry<Integer, PartitionData> partitionEntry : topicEntry.getValue().entrySet()) {
+                PartitionData offsetPartitionData = partitionEntry.getValue();
+                Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME);
+                partitionData.set(PARTITION_KEY_NAME, partitionEntry.getKey());
+                partitionData.set(ERROR_CODE_KEY_NAME, offsetPartitionData.errorCode);
+                partitionData.set(OFFSETS_KEY_NAME, offsetPartitionData.offsets.toArray());
+                partitionArray.add(partitionData);
+            }
+            topicData.set(PARTITIONS_KEY_NAME, partitionArray.toArray());
+            topicArray.add(topicData);
+        }
+        struct.set(RESPONSES_KEY_NAME, topicArray.toArray());
+        this.responseData = responseData;
+    }
+
+    public ListOffsetResponse(Struct struct) {
+        super(struct);
+        responseData = new HashMap<TopicPartition, PartitionData>();
+        for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) {
+            Struct topicResponse = (Struct) topicResponseObj;
+            String topic = topicResponse.getString(TOPIC_KEY_NAME);
+            for (Object partitionResponseObj : topicResponse.getArray(PARTITIONS_KEY_NAME)) {
+                Struct partitionResponse = (Struct) partitionResponseObj;
+                int partition = partitionResponse.getInt(PARTITION_KEY_NAME);
+                short errorCode = partitionResponse.getShort(ERROR_CODE_KEY_NAME);
+                Object[] offsets = partitionResponse.getArray(OFFSETS_KEY_NAME);
+                List<Long> offsetsList = new ArrayList<Long>();
+                for (Object offset: offsets)
+                    offsetsList.add((Long) offset);
+                PartitionData partitionData = new PartitionData(errorCode, offsetsList);
+                responseData.put(new TopicPartition(topic, partition), partitionData);
+            }
+        }
+    }
+
+    public Map<TopicPartition, PartitionData> responseData() {
+        return responseData;
+    }
+
+    public static ListOffsetResponse parse(ByteBuffer buffer) {
+        return new ListOffsetResponse(((Struct) curSchema.read(buffer)));
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/fc0e03f7/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
index f35bd87..b22ca1d 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java
@@ -12,26 +12,41 @@
  */
 package org.apache.kafka.common.requests;
 
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
-public class MetadataRequest {
+public class MetadataRequest extends AbstractRequestResponse {
+    public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id);
+    private static String TOPICS_KEY_NAME = "topics";
 
     private final List<String> topics;
 
     public MetadataRequest(List<String> topics) {
+        super(new Struct(curSchema));
+        struct.set(TOPICS_KEY_NAME, topics.toArray());
         this.topics = topics;
     }
 
-    public Struct toStruct() {
-        String[] ts = new String[topics.size()];
-        topics.toArray(ts);
-        Struct body = new Struct(ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id));
-        body.set("topics", topics.toArray());
-        return body;
+    public MetadataRequest(Struct struct) {
+        super(struct);
+        Object[] topicArray = struct.getArray(TOPICS_KEY_NAME);
+        topics = new ArrayList<String>();
+        for (Object topicObj: topicArray) {
+            topics.add((String) topicObj);
+        }
     }
 
+    public List<String> topics() {
+        return topics;
+    }
+
+    public static MetadataRequest parse(ByteBuffer buffer) {
+        return new MetadataRequest(((Struct) curSchema.read(buffer)));
+    }
 }


[31/37] git commit: kafka-1542 (trivail followup patch to fix NullPointerException); normal IOException in the new producer is logged as ERROR; patched by Jun Rao

Posted by jj...@apache.org.
kafka-1542 (trivail followup patch to fix NullPointerException); normal IOException in the new producer is logged as ERROR; patched by Jun Rao


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/50f2b245
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/50f2b245
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/50f2b245

Branch: refs/heads/transactional_messaging
Commit: 50f2b245c1a666400fab7ddfdebf7c1c2bdf3084
Parents: f489493
Author: Jun Rao <ju...@gmail.com>
Authored: Mon Jul 28 10:58:58 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Mon Jul 28 10:58:58 2014 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/kafka/common/network/Selector.java   | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/50f2b245/clients/src/main/java/org/apache/kafka/common/network/Selector.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java
index 45e57b4..4dd2cdf 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java
@@ -13,6 +13,7 @@
 package org.apache.kafka.common.network;
 
 import java.io.IOException;
+import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.nio.channels.CancelledKeyException;
@@ -262,7 +263,11 @@ public class Selector implements Selectable {
                     if (!key.isValid())
                         close(key);
                 } catch (IOException e) {
-                    log.warn("Error in I/O with {}",channel.socket().getInetAddress().getHostAddress() , e);
+                    InetAddress remoteAddress = null;
+                    Socket socket = channel.socket();
+                    if (socket != null)
+                        remoteAddress = socket.getInetAddress();
+                    log.warn("Error in I/O with {}", remoteAddress , e);
                     close(key);
                 }
             }


[35/37] git commit: trivial change to remove a compilation warning; patched by Jun Rao

Posted by jj...@apache.org.
trivial change to remove a compilation warning; patched by Jun Rao


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/0386790e
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/0386790e
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/0386790e

Branch: refs/heads/transactional_messaging
Commit: 0386790e757fe4eeae4a179e573f71ef8c7de43c
Parents: 1d2e776
Author: Jun Rao <ju...@gmail.com>
Authored: Mon Aug 4 21:24:05 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Mon Aug 4 21:24:05 2014 -0700

----------------------------------------------------------------------
 .../main/java/org/apache/kafka/common/protocol/types/Struct.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/0386790e/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
index 444e69e..121e880 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
@@ -86,7 +86,7 @@ public class Struct {
     /**
      * Check if the struct contains a field.
      * @param name
-     * @return
+     * @return Whether a field exists.
      */
     public boolean hasField(String name) {
         return schema.get(name) != null;


[17/37] git commit: KAFKA-1535 Have the metadata response contain all alive brokers rather than just the ones needed for the given topics.

Posted by jj...@apache.org.
KAFKA-1535 Have the metadata response contain all alive brokers rather than just the ones needed for the given topics.


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/4ebcdfd5
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/4ebcdfd5
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/4ebcdfd5

Branch: refs/heads/transactional_messaging
Commit: 4ebcdfd51f1e9e4c3d684204e6a785fae6c0e549
Parents: 8e444a3
Author: Jay Kreps <ja...@gmail.com>
Authored: Thu Jul 17 15:53:52 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Thu Jul 17 15:53:52 2014 -0700

----------------------------------------------------------------------
 .../main/scala/kafka/api/TopicMetadataRequest.scala    |  2 +-
 .../main/scala/kafka/api/TopicMetadataResponse.scala   | 13 +++----------
 core/src/main/scala/kafka/server/KafkaApis.scala       |  5 +++--
 core/src/main/scala/kafka/server/MetadataCache.scala   |  6 ++++++
 .../scala/kafka/tools/ReplicaVerificationTool.scala    |  2 +-
 .../kafka/api/RequestResponseSerializationTest.scala   |  2 +-
 6 files changed, 15 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/4ebcdfd5/core/src/main/scala/kafka/api/TopicMetadataRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala
index a319f2f..bce004f 100644
--- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala
+++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala
@@ -79,7 +79,7 @@ case class TopicMetadataRequest(val versionId: Short,
     val topicMetadata = topics.map {
       topic => TopicMetadata(topic, Nil, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
     }
-    val errorResponse = TopicMetadataResponse(topicMetadata, correlationId)
+    val errorResponse = TopicMetadataResponse(Seq(), topicMetadata, correlationId)
     requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/4ebcdfd5/core/src/main/scala/kafka/api/TopicMetadataResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala
index f6b7429..b233d35 100644
--- a/core/src/main/scala/kafka/api/TopicMetadataResponse.scala
+++ b/core/src/main/scala/kafka/api/TopicMetadataResponse.scala
@@ -29,34 +29,27 @@ object TopicMetadataResponse {
     val brokerMap = brokers.map(b => (b.id, b)).toMap
     val topicCount = buffer.getInt
     val topicsMetadata = (0 until topicCount).map(_ => TopicMetadata.readFrom(buffer, brokerMap))
-    new TopicMetadataResponse(topicsMetadata, correlationId)
+    new TopicMetadataResponse(brokers, topicsMetadata, correlationId)
   }
 }
 
-case class TopicMetadataResponse(topicsMetadata: Seq[TopicMetadata],
+case class TopicMetadataResponse(brokers: Seq[Broker],
+                                 topicsMetadata: Seq[TopicMetadata],
                                  override val correlationId: Int)
     extends RequestOrResponse(correlationId = correlationId) {
   val sizeInBytes: Int = {
-    val brokers = extractBrokers(topicsMetadata).values
     4 + 4 + brokers.map(_.sizeInBytes).sum + 4 + topicsMetadata.map(_.sizeInBytes).sum
   }
 
   def writeTo(buffer: ByteBuffer) {
     buffer.putInt(correlationId)
     /* brokers */
-    val brokers = extractBrokers(topicsMetadata).values
     buffer.putInt(brokers.size)
     brokers.foreach(_.writeTo(buffer))
     /* topic metadata */
     buffer.putInt(topicsMetadata.length)
     topicsMetadata.foreach(_.writeTo(buffer))
   }
-    
-  def extractBrokers(topicMetadatas: Seq[TopicMetadata]): Map[Int, Broker] = {
-    val parts = topicsMetadata.flatMap(_.partitionsMetadata)
-    val brokers = (parts.flatMap(_.replicas)) ++ (parts.map(_.leader).collect{case Some(l) => l})
-    brokers.map(b => (b.id, b)).toMap
-  }
 
   override def describe(details: Boolean):String = { toString }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/4ebcdfd5/core/src/main/scala/kafka/server/KafkaApis.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala
index 0b668f2..fd5f12e 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -591,8 +591,9 @@ class KafkaApis(val requestChannel: RequestChannel,
   def handleTopicMetadataRequest(request: RequestChannel.Request) {
     val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest]
     val topicMetadata = getTopicMetadata(metadataRequest.topics.toSet)
-    trace("Sending topic metadata %s for correlation id %d to client %s".format(topicMetadata.mkString(","), metadataRequest.correlationId, metadataRequest.clientId))
-    val response = new TopicMetadataResponse(topicMetadata, metadataRequest.correlationId)
+    val brokers = metadataCache.getAliveBrokers
+    trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(topicMetadata.mkString(","), brokers.mkString(","), metadataRequest.correlationId, metadataRequest.clientId))
+    val response = new TopicMetadataResponse(brokers, topicMetadata, metadataRequest.correlationId)
     requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/4ebcdfd5/core/src/main/scala/kafka/server/MetadataCache.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala
index 3198cdf..7cd40e1 100644
--- a/core/src/main/scala/kafka/server/MetadataCache.scala
+++ b/core/src/main/scala/kafka/server/MetadataCache.scala
@@ -81,6 +81,12 @@ private[server] class MetadataCache {
     topicResponses
   }
 
+  def getAliveBrokers = {
+    inLock(partitionMetadataLock.readLock()) {
+      aliveBrokers.values.toList
+    }
+  }
+
   def addOrUpdatePartitionInfo(topic: String,
                                partitionId: Int,
                                stateInfo: PartitionStateInfo) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/4ebcdfd5/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
index c040f49..af47836 100644
--- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
+++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
@@ -118,7 +118,7 @@ object ReplicaVerificationTool extends Logging {
     info("Getting topic metatdata...")
     val metadataTargetBrokers = ClientUtils.parseBrokerList(options.valueOf(brokerListOpt))
     val topicsMetadataResponse = ClientUtils.fetchTopicMetadata(Set[String](), metadataTargetBrokers, clientId, maxWaitMs)
-    val brokerMap = topicsMetadataResponse.extractBrokers(topicsMetadataResponse.topicsMetadata)
+    val brokerMap = topicsMetadataResponse.brokers.map(b => (b.id, b)).toMap
     val filteredTopicMetadata = topicsMetadataResponse.topicsMetadata.filter(
         topicMetadata => if (topicWhiteListFiler.isTopicAllowed(topicMetadata.topic, excludeInternalTopics = false))
           true

http://git-wip-us.apache.org/repos/asf/kafka/blob/4ebcdfd5/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
index a2117b3..d34ddf5 100644
--- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
+++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
@@ -143,7 +143,7 @@ object SerializationTestUtils {
   }
 
   def createTestTopicMetadataResponse: TopicMetadataResponse = {
-    new TopicMetadataResponse(Seq(topicmetaData1, topicmetaData2), 1)
+    new TopicMetadataResponse(brokers, Seq(topicmetaData1, topicmetaData2), 1)
   }
 
   def createTestOffsetCommitRequest: OffsetCommitRequest = {


[09/37] git commit: KAFKA-1515 Fix a bug that could result in blocking for a long period of time in the producer. Patch from Guozhang Wang.

Posted by jj...@apache.org.
KAFKA-1515 Fix a bug that could result in blocking for a long period of time in the producer. Patch from Guozhang Wang.


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/cd3ce27d
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/cd3ce27d
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/cd3ce27d

Branch: refs/heads/transactional_messaging
Commit: cd3ce27d4baf5434676ec040d64663ad3ce09817
Parents: 6de56b3
Author: Jay Kreps <ja...@gmail.com>
Authored: Tue Jul 8 13:16:56 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Tue Jul 8 13:16:56 2014 -0700

----------------------------------------------------------------------
 .../org/apache/kafka/clients/NetworkClient.java |  4 +-
 .../kafka/clients/producer/KafkaProducer.java   | 40 ++++++++++--
 .../clients/producer/internals/Metadata.java    | 68 +++++++++-----------
 .../clients/producer/internals/Sender.java      |  4 +-
 .../kafka/clients/producer/MetadataTest.java    |  9 ++-
 5 files changed, 77 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/cd3ce27d/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
index d21f922..f739279 100644
--- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
@@ -309,7 +309,7 @@ public class NetworkClient implements KafkaClient {
         }
         // we got a disconnect so we should probably refresh our metadata and see if that broker is dead
         if (this.selector.disconnected().size() > 0)
-            this.metadata.forceUpdate();
+            this.metadata.requestUpdate();
     }
 
     /**
@@ -375,7 +375,7 @@ public class NetworkClient implements KafkaClient {
             /* attempt failed, we'll try again after the backoff */
             connectionStates.disconnected(node.id());
             /* maybe the problem is our metadata, update it */
-            metadata.forceUpdate();
+            metadata.requestUpdate();
             log.debug("Error connecting to node {} at {}:{}:", node.id(), node.host(), node.port(), e);
         }
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/cd3ce27d/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
index d85ca30..f58b850 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
@@ -34,6 +34,7 @@ import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.config.ConfigException;
 import org.apache.kafka.common.errors.ApiException;
 import org.apache.kafka.common.errors.RecordTooLargeException;
+import org.apache.kafka.common.errors.TimeoutException;
 import org.apache.kafka.common.metrics.JmxReporter;
 import org.apache.kafka.common.metrics.MetricConfig;
 import org.apache.kafka.common.metrics.Metrics;
@@ -73,6 +74,7 @@ public class KafkaProducer implements Producer {
     private final Thread ioThread;
     private final CompressionType compressionType;
     private final Sensor errors;
+    private final Time time;
 
     /**
      * A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings
@@ -94,7 +96,7 @@ public class KafkaProducer implements Producer {
 
     private KafkaProducer(ProducerConfig config) {
         log.trace("Starting the Kafka producer");
-        Time time = new SystemTime();
+        this.time = new SystemTime();
         MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ProducerConfig.METRICS_NUM_SAMPLES_CONFIG))
                                                       .timeWindow(config.getLong(ProducerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG),
                                                                   TimeUnit.MILLISECONDS);
@@ -119,7 +121,7 @@ public class KafkaProducer implements Producer {
                                                  metrics,
                                                  time);
         List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG));
-        this.metadata.update(Cluster.bootstrap(addresses), 0);
+        this.metadata.update(Cluster.bootstrap(addresses), time.milliseconds());
 
         NetworkClient client = new NetworkClient(new Selector(this.metrics, time),
                                                  this.metadata,
@@ -225,8 +227,9 @@ public class KafkaProducer implements Producer {
     @Override
     public Future<RecordMetadata> send(ProducerRecord record, Callback callback) {
         try {
-            Cluster cluster = metadata.fetch(record.topic(), this.metadataFetchTimeoutMs);
-            int partition = partitioner.partition(record, cluster);
+            // first make sure the metadata for the topic is available
+            waitOnMetadata(record.topic(), this.metadataFetchTimeoutMs);
+            int partition = partitioner.partition(record, metadata.fetch());
             int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(record.key(), record.value());
             ensureValidRecordSize(serializedSize);
             TopicPartition tp = new TopicPartition(record.topic(), partition);
@@ -256,6 +259,31 @@ public class KafkaProducer implements Producer {
     }
 
     /**
+     * Wait for cluster metadata including partitions for the given topic to be available.
+     * @param topic The topic we want metadata for
+     * @param maxWaitMs The maximum time in ms for waiting on the metadata
+     */
+    private void waitOnMetadata(String topic, long maxWaitMs) {
+        if (metadata.fetch().partitionsForTopic(topic) != null) {
+            return;
+        } else {
+            long begin = time.milliseconds();
+            long remainingWaitMs = maxWaitMs;
+            while (metadata.fetch().partitionsForTopic(topic) == null) {
+                log.trace("Requesting metadata update for topic {}.", topic);
+                int version = metadata.requestUpdate();
+                metadata.add(topic);
+                sender.wakeup();
+                metadata.awaitUpdate(version, remainingWaitMs);
+                long elapsed = time.milliseconds() - begin;
+                if (elapsed >= maxWaitMs)
+                    throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms.");
+                remainingWaitMs = maxWaitMs - elapsed;
+            }
+        }
+    }
+
+    /**
      * Validate that the record size isn't too large
      */
     private void ensureValidRecordSize(int size) {
@@ -271,8 +299,10 @@ public class KafkaProducer implements Producer {
                                               " configuration.");
     }
 
+    @Override
     public List<PartitionInfo> partitionsFor(String topic) {
-        return this.metadata.fetch(topic, this.metadataFetchTimeoutMs).partitionsForTopic(topic);
+        waitOnMetadata(topic, this.metadataFetchTimeoutMs);
+        return this.metadata.fetch().partitionsForTopic(topic);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/kafka/blob/cd3ce27d/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
index 8890aa2..140237f 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
@@ -13,11 +13,9 @@
 package org.apache.kafka.clients.producer.internals;
 
 import java.util.HashSet;
-import java.util.List;
 import java.util.Set;
 
 import org.apache.kafka.common.Cluster;
-import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.errors.TimeoutException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -36,9 +34,10 @@ public final class Metadata {
 
     private final long refreshBackoffMs;
     private final long metadataExpireMs;
+    private int version;
     private long lastRefreshMs;
     private Cluster cluster;
-    private boolean forceUpdate;
+    private boolean needUpdate;
     private final Set<String> topics;
 
     /**
@@ -58,8 +57,9 @@ public final class Metadata {
         this.refreshBackoffMs = refreshBackoffMs;
         this.metadataExpireMs = metadataExpireMs;
         this.lastRefreshMs = 0L;
+        this.version = 0;
         this.cluster = Cluster.empty();
-        this.forceUpdate = false;
+        this.needUpdate = false;
         this.topics = new HashSet<String>();
     }
 
@@ -71,33 +71,10 @@ public final class Metadata {
     }
 
     /**
-     * Fetch cluster metadata including partitions for the given topic. If there is no metadata for the given topic,
-     * block waiting for an update.
-     * @param topic The topic we want metadata for
-     * @param maxWaitMs The maximum amount of time to block waiting for metadata
+     * Add the topic to maintain in the metadata
      */
-    public synchronized Cluster fetch(String topic, long maxWaitMs) {
-        List<PartitionInfo> partitions = null;
-        long begin = System.currentTimeMillis();
-        long remainingWaitMs = maxWaitMs;
-        do {
-            partitions = cluster.partitionsForTopic(topic);
-            if (partitions == null) {
-                topics.add(topic);
-                forceUpdate = true;
-                try {
-                    log.trace("Requesting metadata update for topic {}.", topic);
-                    wait(remainingWaitMs);
-                } catch (InterruptedException e) { /* this is fine, just try again */
-                }
-                long elapsed = System.currentTimeMillis() - begin;
-                if (elapsed >= maxWaitMs)
-                    throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms.");
-                remainingWaitMs = maxWaitMs - elapsed;
-            } else {
-                return cluster;
-            }
-        } while (true);
+    public synchronized void add(String topic) {
+        topics.add(topic);
     }
 
     /**
@@ -106,16 +83,35 @@ public final class Metadata {
      * been request then the expiry time is now
      */
     public synchronized long timeToNextUpdate(long nowMs) {
-        long timeToExpire = forceUpdate ? 0 : Math.max(this.lastRefreshMs + this.metadataExpireMs - nowMs, 0);
+        long timeToExpire = needUpdate ? 0 : Math.max(this.lastRefreshMs + this.metadataExpireMs - nowMs, 0);
         long timeToAllowUpdate = this.lastRefreshMs + this.refreshBackoffMs - nowMs;
         return Math.max(timeToExpire, timeToAllowUpdate);
     }
 
     /**
-     * Force an update of the current cluster info
+     * Request an update of the current cluster metadata info, return the current version before the update
      */
-    public synchronized void forceUpdate() {
-        this.forceUpdate = true;
+    public synchronized int requestUpdate() {
+        this.needUpdate = true;
+        return this.version;
+    }
+
+    /**
+     * Wait for metadata update until the current version is larger than the last version we know of
+     */
+    public synchronized void awaitUpdate(int lastVerison, long maxWaitMs) {
+        long begin = System.currentTimeMillis();
+        long remainingWaitMs = maxWaitMs;
+        while (this.version <= lastVerison) {
+            try {
+                wait(remainingWaitMs);
+            } catch (InterruptedException e) { /* this is fine */
+            }
+            long elapsed = System.currentTimeMillis() - begin;
+            if (elapsed >= maxWaitMs)
+                throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms.");
+            remainingWaitMs = maxWaitMs - elapsed;
+        }
     }
 
     /**
@@ -129,8 +125,9 @@ public final class Metadata {
      * Update the cluster metadata
      */
     public synchronized void update(Cluster cluster, long now) {
-        this.forceUpdate = false;
+        this.needUpdate = false;
         this.lastRefreshMs = now;
+        this.version += 1;
         this.cluster = cluster;
         notifyAll();
         log.debug("Updated cluster metadata to {}", cluster);
@@ -142,5 +139,4 @@ public final class Metadata {
     public synchronized long lastUpdate() {
         return this.lastRefreshMs;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/cd3ce27d/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
index 37b9d1a..a016269 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
@@ -147,7 +147,7 @@ public class Sender implements Runnable {
 
         // if there are any partitions whose leaders are not known yet, force metadata update
         if (result.unknownLeadersExist)
-            this.metadata.forceUpdate();
+            this.metadata.requestUpdate();
 
         // remove any nodes we aren't ready to send to
         Iterator<Node> iter = result.readyNodes.iterator();
@@ -252,7 +252,7 @@ public class Sender implements Runnable {
                 this.sensors.recordErrors(batch.topicPartition.topic(), batch.recordCount);
         }
         if (error.exception() instanceof InvalidMetadataException)
-            metadata.forceUpdate();
+            metadata.requestUpdate();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/kafka/blob/cd3ce27d/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
index 0d7d04c..543304c 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
@@ -31,7 +31,7 @@ public class MetadataTest {
         long time = 0;
         metadata.update(Cluster.empty(), time);
         assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0);
-        metadata.forceUpdate();
+        metadata.requestUpdate();
         assertFalse("Still no updated needed due to backoff", metadata.timeToNextUpdate(time) == 0);
         time += refreshBackoffMs;
         assertTrue("Update needed now that backoff time expired", metadata.timeToNextUpdate(time) == 0);
@@ -40,7 +40,9 @@ public class MetadataTest {
         Thread t2 = asyncFetch(topic);
         assertTrue("Awaiting update", t1.isAlive());
         assertTrue("Awaiting update", t2.isAlive());
-        metadata.update(TestUtils.singletonCluster(topic, 1), time);
+        // keep updating the metadata until no need to
+        while (metadata.timeToNextUpdate(time) == 0)
+            metadata.update(TestUtils.singletonCluster(topic, 1), time);
         t1.join();
         t2.join();
         assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0);
@@ -51,7 +53,8 @@ public class MetadataTest {
     private Thread asyncFetch(final String topic) {
         Thread thread = new Thread() {
             public void run() {
-                metadata.fetch(topic, Integer.MAX_VALUE);
+                while (metadata.fetch().partitionsForTopic(topic) == null)
+                    metadata.awaitUpdate(metadata.requestUpdate(), Long.MAX_VALUE);
             }
         };
         thread.start();


[29/37] git commit: kafka-1542; normal IOException in the new producer is logged as ERROR; patched by David Corley; reviewed by Jun Rao

Posted by jj...@apache.org.
kafka-1542; normal IOException in the new producer is logged as ERROR; patched by David Corley; reviewed by Jun Rao


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/fa34841d
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/fa34841d
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/fa34841d

Branch: refs/heads/transactional_messaging
Commit: fa34841d9889fc45b4824b729b71bfb2215a082d
Parents: 7f2278f
Author: David Corley <da...@gmail.com>
Authored: Sun Jul 27 10:55:30 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Sun Jul 27 10:55:30 2014 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/kafka/common/network/Selector.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/fa34841d/clients/src/main/java/org/apache/kafka/common/network/Selector.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java
index 93f2f1c..45e57b4 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java
@@ -262,7 +262,7 @@ public class Selector implements Selectable {
                     if (!key.isValid())
                         close(key);
                 } catch (IOException e) {
-                    log.error("Error in I/O: ", e);
+                    log.warn("Error in I/O with {}",channel.socket().getInetAddress().getHostAddress() , e);
                     close(key);
                 }
             }


[32/37] git commit: kafka-1451; Broker stuck due to leader election race; patched by Manikumar Reddy; reviewed by Jun Rao

Posted by jj...@apache.org.
kafka-1451; Broker stuck due to leader election race; patched by Manikumar Reddy; reviewed by Jun Rao


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/a01a101e
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/a01a101e
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/a01a101e

Branch: refs/heads/transactional_messaging
Commit: a01a101e82d5b06e89857e79c4b8268589d81fca
Parents: 50f2b24
Author: Manikumar Reddy <ma...@gmail.com>
Authored: Wed Jul 30 08:14:41 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Wed Jul 30 08:14:41 2014 -0700

----------------------------------------------------------------------
 .../kafka/server/ZookeeperLeaderElector.scala   | 30 +++++++++++++++-----
 1 file changed, 23 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/a01a101e/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala b/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala
index e5b6ff1..a75818a 100644
--- a/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala
+++ b/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala
@@ -50,9 +50,27 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext,
     }
   }
 
+  private def getControllerID(): Int = {
+    readDataMaybeNull(controllerContext.zkClient, electionPath)._1 match {
+       case Some(controller) => KafkaController.parseControllerId(controller)
+       case None => -1
+    }
+  }
+    
   def elect: Boolean = {
     val timestamp = SystemTime.milliseconds.toString
     val electString = Json.encode(Map("version" -> 1, "brokerid" -> brokerId, "timestamp" -> timestamp))
+   
+   leaderId = getControllerID 
+    /* 
+     * We can get here during the initial startup and the handleDeleted ZK callback. Because of the potential race condition, 
+     * it's possible that the controller has already been elected when we get here. This check will prevent the following 
+     * createEphemeralPath method from getting into an infinite loop if this broker is already the controller.
+     */
+    if(leaderId != -1) {
+       debug("Broker %d has been elected as leader, so stopping the election process.".format(leaderId))
+       return amILeader
+    }
 
     try {
       createEphemeralPathExpectConflictHandleZKBug(controllerContext.zkClient, electionPath, electString, brokerId,
@@ -64,15 +82,13 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext,
     } catch {
       case e: ZkNodeExistsException =>
         // If someone else has written the path, then
-        leaderId = readDataMaybeNull(controllerContext.zkClient, electionPath)._1 match {
-          case Some(controller) => KafkaController.parseControllerId(controller)
-          case None => {
-            warn("A leader has been elected but just resigned, this will result in another round of election")
-            -1
-          }
-        }
+        leaderId = getControllerID 
+
         if (leaderId != -1)
           debug("Broker %d was elected as leader instead of broker %d".format(leaderId, brokerId))
+        else
+          warn("A leader has been elected but just resigned, this will result in another round of election")
+
       case e2: Throwable =>
         error("Error while electing or becoming leader on broker %d".format(brokerId), e2)
         resign()


[11/37] git commit: kafka-1406; Fix scaladoc/javadoc warnings; patched by Alan Lee; reviewed by Jun Rao

Posted by jj...@apache.org.
kafka-1406; Fix scaladoc/javadoc warnings; patched by Alan Lee; reviewed by Jun Rao


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/8034390e
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/8034390e
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/8034390e

Branch: refs/heads/transactional_messaging
Commit: 8034390ef0f49a1ed180bc1e883dbbd1f6d864a0
Parents: 420628d
Author: Alan Lee <co...@gmail.com>
Authored: Thu Jul 10 09:45:05 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Thu Jul 10 09:45:05 2014 -0700

----------------------------------------------------------------------
 .../consumer/ConsumerRebalanceCallback.java     |  4 +-
 .../apache/kafka/common/network/Selectable.java |  8 +--
 .../src/main/java/kafka/etl/Props.java          | 73 +++++++++-----------
 .../javaapi/consumer/ConsumerConnector.java     |  3 +-
 core/src/main/scala/kafka/log/OffsetIndex.scala |  2 +-
 core/src/main/scala/kafka/utils/Utils.scala     |  2 +-
 6 files changed, 44 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/8034390e/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java
index 05eb6ce..f026ae4 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRebalanceCallback.java
@@ -21,8 +21,8 @@ import org.apache.kafka.common.TopicPartition;
  * every rebalance operation. This callback will execute in the user thread as part of the 
  * {@link Consumer#poll(long) poll(long)} API on every rebalance attempt.
  * Default implementation of the callback will {@link Consumer#seek(java.util.Map) seek(offsets)} to the last committed offsets in the
- * {@link #onPartitionsAssigned(Consumer, TopicPartition...) onPartitionsAssigned()} callback. And will commit offsets synchronously 
- * for the specified list of partitions to Kafka in the {@link #onPartitionsRevoked(Consumer, TopicPartition...) onPartitionsRevoked()} 
+ * {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned()} callback. And will commit offsets synchronously
+ * for the specified list of partitions to Kafka in the {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked()}
  * callback.
  */
 public interface ConsumerRebalanceCallback {

http://git-wip-us.apache.org/repos/asf/kafka/blob/8034390e/clients/src/main/java/org/apache/kafka/common/network/Selectable.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selectable.java b/clients/src/main/java/org/apache/kafka/common/network/Selectable.java
index cef75d8..b68bbf0 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/Selectable.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/Selectable.java
@@ -60,23 +60,23 @@ public interface Selectable {
     public void poll(long timeout, List<NetworkSend> sends) throws IOException;
 
     /**
-     * The list of sends that completed on the last {@link #poll(long, List<NetworkSend>) poll()} call.
+     * The list of sends that completed on the last {@link #poll(long, List) poll()} call.
      */
     public List<NetworkSend> completedSends();
 
     /**
-     * The list of receives that completed on the last {@link #poll(long, List<NetworkSend>) poll()} call.
+     * The list of receives that completed on the last {@link #poll(long, List) poll()} call.
      */
     public List<NetworkReceive> completedReceives();
 
     /**
-     * The list of connections that finished disconnecting on the last {@link #poll(long, List<NetworkSend>) poll()}
+     * The list of connections that finished disconnecting on the last {@link #poll(long, List) poll()}
      * call.
      */
     public List<Integer> disconnected();
 
     /**
-     * The list of connections that completed their connection on the last {@link #poll(long, List<NetworkSend>) poll()}
+     * The list of connections that completed their connection on the last {@link #poll(long, List) poll()}
      * call.
      */
     public List<Integer> connected();

http://git-wip-us.apache.org/repos/asf/kafka/blob/8034390e/contrib/hadoop-consumer/src/main/java/kafka/etl/Props.java
----------------------------------------------------------------------
diff --git a/contrib/hadoop-consumer/src/main/java/kafka/etl/Props.java b/contrib/hadoop-consumer/src/main/java/kafka/etl/Props.java
index 3514ec7..71eb80f 100644
--- a/contrib/hadoop-consumer/src/main/java/kafka/etl/Props.java
+++ b/contrib/hadoop-consumer/src/main/java/kafka/etl/Props.java
@@ -115,11 +115,11 @@ public class Props extends Properties {
 	}
 
 	/**
-	 * build props from a list of strings and interprate them as
+	 * build props from a list of strings and interpret them as
 	 * key, value, key, value,....
 	 * 
 	 * @param args
-	 * @return
+	 * @return props
 	 */
 	@SuppressWarnings("unchecked")
 	public static Props of(String... args) {
@@ -148,7 +148,7 @@ public class Props extends Properties {
 	/**
 	 * get property of "key" and split the value by " ," 
 	 * @param key		
-	 * @return
+	 * @return list of values
 	 */
 	public List<String> getStringList(String key) {
 		return getStringList(key, "\\s*,\\s*");
@@ -158,7 +158,7 @@ public class Props extends Properties {
 	 * get property of "key" and split the value by "sep"
 	 * @param key
 	 * @param sep
-	 * @return
+	 * @return string list of values
 	 */
 	public List<String> getStringList(String key, String sep) {
 		String val =  super.getProperty(key);
@@ -176,7 +176,7 @@ public class Props extends Properties {
 	 * get string list with default value. default delimiter is ","
 	 * @param key
 	 * @param defaultValue
-	 * @return
+	 * @return string list of values
 	 */
 	public List<String> getStringList(String key, List<String> defaultValue) {
 		if (containsKey(key))
@@ -189,7 +189,7 @@ public class Props extends Properties {
 	 * get string list with default value
 	 * @param key
 	 * @param defaultValue
-	 * @return
+	 * @return string list of values
 	 */
 	public List<String> getStringList(String key, List<String> defaultValue,
 			String sep) {
@@ -251,10 +251,10 @@ public class Props extends Properties {
 	}
 
 	/**
-	 * get boolean value
+	 * get boolean value with default value
 	 * @param key
 	 * @param defaultValue
-	 * @return
+	 * @return boolean value
 	 * @throws Exception 	if value is not of type boolean or string
 	 */
 	public Boolean getBoolean(String key, Boolean defaultValue) 
@@ -265,8 +265,7 @@ public class Props extends Properties {
 	/**
 	 * get boolean value
 	 * @param key
-	 * @param defaultValue
-	 * @return
+	 * @return boolean value
 	 * @throws Exception 	if value is not of type boolean or string or 
 	 * 										if value doesn't exist
 	 */
@@ -275,10 +274,10 @@ public class Props extends Properties {
 	}
 
 	/**
-	 * get long value
-	 * @param key
+	 * get long value with default value
+	 * @param name
 	 * @param defaultValue
-	 * @return
+	 * @return long value
 	 * @throws Exception 	if value is not of type long or string
 	 */
 	public Long getLong(String name, Long defaultValue) 
@@ -288,9 +287,8 @@ public class Props extends Properties {
 
 	/**
 	 * get long value
-	 * @param key
-	 * @param defaultValue
-	 * @return
+	 * @param name
+	 * @return long value
 	 * @throws Exception 	if value is not of type long or string or 
 	 * 										if value doesn't exist
 	 */
@@ -299,10 +297,10 @@ public class Props extends Properties {
 	}
 
 	/**
-	 * get integer value
-	 * @param key
+	 * get integer value with default value
+	 * @param name
 	 * @param defaultValue
-	 * @return
+	 * @return integer value
 	 * @throws Exception 	if value is not of type integer or string
 	 */
 	public Integer getInt(String name, Integer defaultValue) 
@@ -312,9 +310,8 @@ public class Props extends Properties {
 
 	/**
 	 * get integer value
-	 * @param key
-	 * @param defaultValue
-	 * @return
+	 * @param name
+	 * @return integer value
 	 * @throws Exception 	if value is not of type integer or string or 
 	 * 										if value doesn't exist
 	 */
@@ -323,10 +320,10 @@ public class Props extends Properties {
 	}
 
 	/**
-	 * get double value
-	 * @param key
+	 * get double value with default value
+	 * @param name
 	 * @param defaultValue
-	 * @return
+	 * @return double value
 	 * @throws Exception 	if value is not of type double or string
 	 */
 	public Double getDouble(String name, double defaultValue) 
@@ -336,9 +333,8 @@ public class Props extends Properties {
 
 	/**
 	 * get double value
-	 * @param key
-	 * @param defaultValue
-	 * @return
+	 * @param name
+	 * @return double value
 	 * @throws Exception 	if value is not of type double or string or 
 	 * 										if value doesn't exist
 	 */
@@ -347,10 +343,10 @@ public class Props extends Properties {
 	}
 
 	/**
-	 * get URI value
-	 * @param key
+	 * get URI value with default value
+	 * @param name
 	 * @param defaultValue
-	 * @return
+	 * @return URI value
 	 * @throws Exception 	if value is not of type URI or string 
 	 */
 	public URI getUri(String name, URI defaultValue) throws Exception {
@@ -359,9 +355,9 @@ public class Props extends Properties {
 
 	/**
 	 * get URI value
-	 * @param key
+	 * @param name
 	 * @param defaultValue
-	 * @return
+	 * @return URI value
 	 * @throws Exception 	if value is not of type URI or string 
 	 */
 	public URI getUri(String name, String defaultValue) 
@@ -372,9 +368,8 @@ public class Props extends Properties {
 
 	/**
 	 * get URI value
-	 * @param key
-	 * @param defaultValue
-	 * @return
+	 * @param name
+	 * @return URI value
 	 * @throws Exception 	if value is not of type URI or string or 
 	 * 										if value doesn't exist
 	 */
@@ -385,7 +380,7 @@ public class Props extends Properties {
 	/**
 	 * compare two props 
 	 * @param p
-	 * @return
+	 * @return true or false
 	 */
 	public boolean equalsProps(Props p) {
 		if (p == null) {
@@ -432,7 +427,7 @@ public class Props extends Properties {
     
     /**
      * get all property names
-     * @return
+     * @return set of property names
      */
 	public Set<String> getKeySet() {
 		return super.stringPropertyNames();
@@ -453,7 +448,7 @@ public class Props extends Properties {
 	/**
 	 * clone a Props
 	 * @param p
-	 * @return
+	 * @return props
 	 */
 	public static Props clone(Props p) {
 		return new Props(p);

http://git-wip-us.apache.org/repos/asf/kafka/blob/8034390e/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java
index 44d3d35..cc3400f 100644
--- a/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java
+++ b/core/src/main/scala/kafka/javaapi/consumer/ConsumerConnector.java
@@ -29,7 +29,8 @@ public interface ConsumerConnector {
    *  Create a list of MessageStreams of type T for each topic.
    *
    *  @param topicCountMap  a map of (topic, #streams) pair
-   *  @param decoder a decoder that converts from Message to T
+   *  @param keyDecoder a decoder that decodes the message key
+   *  @param valueDecoder a decoder that decodes the message itself
    *  @return a map of (topic, list of  KafkaStream) pairs.
    *          The number of items in the list is #streams. Each stream supports
    *          an iterator over message/metadata pairs.

http://git-wip-us.apache.org/repos/asf/kafka/blob/8034390e/core/src/main/scala/kafka/log/OffsetIndex.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/OffsetIndex.scala b/core/src/main/scala/kafka/log/OffsetIndex.scala
index 8a62dfa..1c4c7bd 100644
--- a/core/src/main/scala/kafka/log/OffsetIndex.scala
+++ b/core/src/main/scala/kafka/log/OffsetIndex.scala
@@ -346,7 +346,7 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi
   
   /**
    * Do a basic sanity check on this index to detect obvious problems
-   * @throw IllegalArgumentException if any problems are found
+   * @throws IllegalArgumentException if any problems are found
    */
   def sanityCheck() {
     require(entries == 0 || lastOffset > baseOffset,

http://git-wip-us.apache.org/repos/asf/kafka/blob/8034390e/core/src/main/scala/kafka/utils/Utils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala
index b29981b..63d3dda 100644
--- a/core/src/main/scala/kafka/utils/Utils.scala
+++ b/core/src/main/scala/kafka/utils/Utils.scala
@@ -494,7 +494,7 @@ object Utils extends Logging {
   /**
    * Create a file with the given path
    * @param path The path to create
-   * @throw KafkaStorageException If the file create fails
+   * @throws KafkaStorageException If the file create fails
    * @return The created file
    */
   def createFile(path: String): File = {


[15/37] git commit: kafka-1531; zookeeper.connection.timeout.ms is set to 10000000 in configuration file in Kafka tarball; patched by Manikumar Reddy; reviewed by Jun Rao

Posted by jj...@apache.org.
kafka-1531; zookeeper.connection.timeout.ms is set to 10000000 in configuration file in Kafka tarball; patched by Manikumar Reddy; reviewed by Jun Rao


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/b428d8cc
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/b428d8cc
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/b428d8cc

Branch: refs/heads/transactional_messaging
Commit: b428d8cc48237099af648de12d18be78d54446eb
Parents: 3e6b386
Author: Manikumar Reddy <ma...@gmail.com>
Authored: Wed Jul 16 08:13:06 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Wed Jul 16 08:13:06 2014 -0700

----------------------------------------------------------------------
 config/consumer.properties |  2 +-
 config/server.properties   | 14 +++++++-------
 2 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/b428d8cc/config/consumer.properties
----------------------------------------------------------------------
diff --git a/config/consumer.properties b/config/consumer.properties
index 7343cbc..83847de 100644
--- a/config/consumer.properties
+++ b/config/consumer.properties
@@ -20,7 +20,7 @@
 zookeeper.connect=127.0.0.1:2181
 
 # timeout in ms for connecting to zookeeper
-zookeeper.connection.timeout.ms=1000000
+zookeeper.connection.timeout.ms=6000
 
 #consumer group id
 group.id=test-consumer-group

http://git-wip-us.apache.org/repos/asf/kafka/blob/b428d8cc/config/server.properties
----------------------------------------------------------------------
diff --git a/config/server.properties b/config/server.properties
index c9e923a..f16c84c 100644
--- a/config/server.properties
+++ b/config/server.properties
@@ -37,16 +37,16 @@ port=9092
 #advertised.port=<port accessible by clients>
 
 # The number of threads handling network requests
-num.network.threads=2
+num.network.threads=3
  
 # The number of threads doing disk I/O
 num.io.threads=8
 
 # The send buffer (SO_SNDBUF) used by the socket server
-socket.send.buffer.bytes=1048576
+socket.send.buffer.bytes=102400
 
 # The receive buffer (SO_RCVBUF) used by the socket server
-socket.receive.buffer.bytes=1048576
+socket.receive.buffer.bytes=65536
 
 # The maximum size of a request that the socket server will accept (protection against OOM)
 socket.request.max.bytes=104857600
@@ -60,7 +60,7 @@ log.dirs=/tmp/kafka-logs
 # The default number of log partitions per topic. More partitions allow greater
 # parallelism for consumption, but this will also result in more files across
 # the brokers.
-num.partitions=2
+num.partitions=1
 
 ############################# Log Flush Policy #############################
 
@@ -94,11 +94,11 @@ log.retention.hours=168
 #log.retention.bytes=1073741824
 
 # The maximum size of a log segment file. When this size is reached a new log segment will be created.
-log.segment.bytes=536870912
+log.segment.bytes=1073741824
 
 # The interval at which log segments are checked to see if they can be deleted according 
 # to the retention policies
-log.retention.check.interval.ms=60000
+log.retention.check.interval.ms=300000
 
 # By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires.
 # If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction.
@@ -114,4 +114,4 @@ log.cleaner.enable=false
 zookeeper.connect=localhost:2181
 
 # Timeout in ms for connecting to zookeeper
-zookeeper.connection.timeout.ms=1000000
+zookeeper.connection.timeout.ms=6000


[30/37] git commit: kafka-1414; Speedup broker startup after hard reset; patched by Anton Karamanov; reviewed by Jay Kreps and Jun Rao

Posted by jj...@apache.org.
kafka-1414; Speedup broker startup after hard reset; patched by Anton Karamanov; reviewed by Jay Kreps and Jun Rao


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/f489493c
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/f489493c
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/f489493c

Branch: refs/heads/transactional_messaging
Commit: f489493c385e3266c3bc17db2e8ebc215a6e54e2
Parents: fa34841
Author: Anton Karamanov <at...@gmail.com>
Authored: Sun Jul 27 21:13:20 2014 -0700
Committer: Jun Rao <ju...@gmail.com>
Committed: Sun Jul 27 21:13:20 2014 -0700

----------------------------------------------------------------------
 config/server.properties                        |   4 +
 core/src/main/scala/kafka/log/LogManager.scala  | 179 ++++++++++++++-----
 .../main/scala/kafka/server/KafkaConfig.scala   |   3 +
 .../main/scala/kafka/server/KafkaServer.scala   |   1 +
 .../main/scala/kafka/utils/KafkaScheduler.scala |  18 +-
 core/src/main/scala/kafka/utils/Utils.scala     |   6 +-
 .../scala/unit/kafka/log/LogManagerTest.scala   |  97 ++--------
 .../server/HighwatermarkPersistenceTest.scala   |  17 +-
 .../unit/kafka/server/ReplicaManagerTest.scala  |  20 +--
 .../test/scala/unit/kafka/utils/TestUtils.scala |  25 +++
 10 files changed, 203 insertions(+), 167 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/config/server.properties
----------------------------------------------------------------------
diff --git a/config/server.properties b/config/server.properties
index f16c84c..5c0905a 100644
--- a/config/server.properties
+++ b/config/server.properties
@@ -62,6 +62,10 @@ log.dirs=/tmp/kafka-logs
 # the brokers.
 num.partitions=1
 
+# The number of threads per data directory to be used for log recovery at startup and flushing at shutdown.
+# This value is recommended to be increased for installations with data dirs located in RAID array.
+num.recovery.threads.per.data.dir=1
+
 ############################# Log Flush Policy #############################
 
 # Messages are immediately written to the filesystem but by default we only fsync() to sync

http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/main/scala/kafka/log/LogManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala
index 1946c94..4d2924d 100644
--- a/core/src/main/scala/kafka/log/LogManager.scala
+++ b/core/src/main/scala/kafka/log/LogManager.scala
@@ -23,6 +23,7 @@ import kafka.utils._
 import scala.collection._
 import kafka.common.{TopicAndPartition, KafkaException}
 import kafka.server.{RecoveringFromUncleanShutdown, BrokerState, OffsetCheckpoint}
+import java.util.concurrent.{Executors, ExecutorService, ExecutionException, Future}
 
 /**
  * The entry point to the kafka log management subsystem. The log manager is responsible for log creation, retrieval, and cleaning.
@@ -39,6 +40,7 @@ class LogManager(val logDirs: Array[File],
                  val topicConfigs: Map[String, LogConfig],
                  val defaultConfig: LogConfig,
                  val cleanerConfig: CleanerConfig,
+                 ioThreads: Int,
                  val flushCheckMs: Long,
                  val flushCheckpointMs: Long,
                  val retentionCheckMs: Long,
@@ -54,7 +56,7 @@ class LogManager(val logDirs: Array[File],
   createAndValidateLogDirs(logDirs)
   private val dirLocks = lockLogDirs(logDirs)
   private val recoveryPointCheckpoints = logDirs.map(dir => (dir, new OffsetCheckpoint(new File(dir, RecoveryPointCheckpointFile)))).toMap
-  loadLogs(logDirs)
+  loadLogs()
   
   private val cleaner: LogCleaner = 
     if(cleanerConfig.enableCleaner)
@@ -101,36 +103,71 @@ class LogManager(val logDirs: Array[File],
   /**
    * Recover and load all logs in the given data directories
    */
-  private def loadLogs(dirs: Seq[File]) {
-    for(dir <- dirs) {
+  private def loadLogs(): Unit = {
+    info("Loading logs.")
+
+    val threadPools = mutable.ArrayBuffer.empty[ExecutorService]
+    val jobs = mutable.Map.empty[File, Seq[Future[_]]]
+
+    for (dir <- this.logDirs) {
+      val pool = Executors.newFixedThreadPool(ioThreads)
+      threadPools.append(pool)
+
+      val cleanShutdownFile = new File(dir, Log.CleanShutdownFile)
+
+      if (cleanShutdownFile.exists) {
+        debug(
+          "Found clean shutdown file. " +
+          "Skipping recovery for all logs in data directory: " +
+          dir.getAbsolutePath)
+      } else {
+        // log recovery itself is being performed by `Log` class during initialization
+        brokerState.newState(RecoveringFromUncleanShutdown)
+      }
+
       val recoveryPoints = this.recoveryPointCheckpoints(dir).read
-      /* load the logs */
-      val subDirs = dir.listFiles()
-      if(subDirs != null) {
-        val cleanShutDownFile = new File(dir, Log.CleanShutdownFile)
-        if(cleanShutDownFile.exists())
-          info("Found clean shutdown file. Skipping recovery for all logs in data directory '%s'".format(dir.getAbsolutePath))
-        else
-          brokerState.newState(RecoveringFromUncleanShutdown)
-
-        for(dir <- subDirs) {
-          if(dir.isDirectory) {
-            info("Loading log '" + dir.getName + "'")
-            val topicPartition = Log.parseTopicPartitionName(dir.getName)
-            val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig)
-            val log = new Log(dir, 
-                              config,
-                              recoveryPoints.getOrElse(topicPartition, 0L),
-                              scheduler,
-                              time)
-            val previous = this.logs.put(topicPartition, log)
-            if(previous != null)
-              throw new IllegalArgumentException("Duplicate log directories found: %s, %s!".format(log.dir.getAbsolutePath, previous.dir.getAbsolutePath))
+
+      val jobsForDir = for {
+        dirContent <- Option(dir.listFiles).toList
+        logDir <- dirContent if logDir.isDirectory
+      } yield {
+        Utils.runnable {
+          debug("Loading log '" + logDir.getName + "'")
+
+          val topicPartition = Log.parseTopicPartitionName(logDir.getName)
+          val config = topicConfigs.getOrElse(topicPartition.topic, defaultConfig)
+          val logRecoveryPoint = recoveryPoints.getOrElse(topicPartition, 0L)
+
+          val current = new Log(logDir, config, logRecoveryPoint, scheduler, time)
+          val previous = this.logs.put(topicPartition, current)
+
+          if (previous != null) {
+            throw new IllegalArgumentException(
+              "Duplicate log directories found: %s, %s!".format(
+              current.dir.getAbsolutePath, previous.dir.getAbsolutePath))
           }
         }
-        cleanShutDownFile.delete()
       }
+
+      jobs(cleanShutdownFile) = jobsForDir.map(pool.submit).toSeq
+    }
+
+
+    try {
+      for ((cleanShutdownFile, dirJobs) <- jobs) {
+        dirJobs.foreach(_.get)
+        cleanShutdownFile.delete()
+      }
+    } catch {
+      case e: ExecutionException => {
+        error("There was an error in one of the threads during logs loading: " + e.getCause)
+        throw e.getCause
+      }
+    } finally {
+      threadPools.foreach(_.shutdown())
     }
+
+    info("Logs loading complete.")
   }
 
   /**
@@ -160,31 +197,69 @@ class LogManager(val logDirs: Array[File],
     if(cleanerConfig.enableCleaner)
       cleaner.startup()
   }
-  
+
   /**
    * Close all the logs
    */
   def shutdown() {
     info("Shutting down.")
+
+    val threadPools = mutable.ArrayBuffer.empty[ExecutorService]
+    val jobs = mutable.Map.empty[File, Seq[Future[_]]]
+
+    // stop the cleaner first
+    if (cleaner != null) {
+      Utils.swallow(cleaner.shutdown())
+    }
+
+    // close logs in each dir
+    for (dir <- this.logDirs) {
+      debug("Flushing and closing logs at " + dir)
+
+      val pool = Executors.newFixedThreadPool(ioThreads)
+      threadPools.append(pool)
+
+      val logsInDir = logsByDir.getOrElse(dir.toString, Map()).values
+
+      val jobsForDir = logsInDir map { log =>
+        Utils.runnable {
+          // flush the log to ensure latest possible recovery point
+          log.flush()
+          log.close()
+        }
+      }
+
+      jobs(dir) = jobsForDir.map(pool.submit).toSeq
+    }
+
+
     try {
-      // stop the cleaner first
-      if(cleaner != null)
-        Utils.swallow(cleaner.shutdown())
-      // flush the logs to ensure latest possible recovery point
-      allLogs.foreach(_.flush())
-      // close the logs
-      allLogs.foreach(_.close())
-      // update the last flush point
-      checkpointRecoveryPointOffsets()
-      // mark that the shutdown was clean by creating the clean shutdown marker file
-      logDirs.foreach(dir => Utils.swallow(new File(dir, Log.CleanShutdownFile).createNewFile()))
+      for ((dir, dirJobs) <- jobs) {
+        dirJobs.foreach(_.get)
+
+        // update the last flush point
+        debug("Updating recovery points at " + dir)
+        checkpointLogsInDir(dir)
+
+        // mark that the shutdown was clean by creating marker file
+        debug("Writing clean shutdown marker at " + dir)
+        Utils.swallow(new File(dir, Log.CleanShutdownFile).createNewFile())
+      }
+    } catch {
+      case e: ExecutionException => {
+        error("There was an error in one of the threads during LogManager shutdown: " + e.getCause)
+        throw e.getCause
+      }
     } finally {
+      threadPools.foreach(_.shutdown())
       // regardless of whether the close succeeded, we need to unlock the data directories
       dirLocks.foreach(_.destroy())
     }
+
     info("Shutdown complete.")
   }
 
+
   /**
    * Truncate the partition logs to the specified offsets and checkpoint the recovery point to this offset
    *
@@ -230,14 +305,19 @@ class LogManager(val logDirs: Array[File],
    * to avoid recovering the whole log on startup.
    */
   def checkpointRecoveryPointOffsets() {
-    val recoveryPointsByDir = this.logsByTopicPartition.groupBy(_._2.dir.getParent.toString)
-    for(dir <- logDirs) {
-        val recoveryPoints = recoveryPointsByDir.get(dir.toString)
-        if(recoveryPoints.isDefined)
-          this.recoveryPointCheckpoints(dir).write(recoveryPoints.get.mapValues(_.recoveryPoint))
+    this.logDirs.foreach(checkpointLogsInDir)
+  }
+
+  /**
+   * Make a checkpoint for all logs in provided directory.
+   */
+  private def checkpointLogsInDir(dir: File): Unit = {
+    val recoveryPoints = this.logsByDir.get(dir.toString)
+    if (recoveryPoints.isDefined) {
+      this.recoveryPointCheckpoints(dir).write(recoveryPoints.get.mapValues(_.recoveryPoint))
     }
   }
-  
+
   /**
    * Get the log if it exists, otherwise return None
    */
@@ -366,13 +446,22 @@ class LogManager(val logDirs: Array[File],
    * Get all the partition logs
    */
   def allLogs(): Iterable[Log] = logs.values
-  
+
   /**
    * Get a map of TopicAndPartition => Log
    */
   def logsByTopicPartition = logs.toMap
 
   /**
+   * Map of log dir to logs by topic and partitions in that dir
+   */
+  private def logsByDir = {
+    this.logsByTopicPartition.groupBy {
+      case (_, log) => log.dir.getParent
+    }
+  }
+
+  /**
    * Flush any log which has exceeded its flush interval and has unwritten messages.
    */
   private def flushDirtyLogs() = {

http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/main/scala/kafka/server/KafkaConfig.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index 50b09ed..1a45f87 100644
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -190,6 +190,9 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
   /* the frequency with which we update the persistent record of the last flush which acts as the log recovery point */
   val logFlushOffsetCheckpointIntervalMs = props.getIntInRange("log.flush.offset.checkpoint.interval.ms", 60000, (0, Int.MaxValue))
 
+  /* the number of threads per data directory to be used for log recovery at startup and flushing at shutdown */
+  val numRecoveryThreadsPerDataDir = props.getIntInRange("num.recovery.threads.per.data.dir", 1, (1, Int.MaxValue))
+
   /* enable auto creation of topic on the server */
   val autoCreateTopicsEnable = props.getBoolean("auto.create.topics.enable", true)
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/main/scala/kafka/server/KafkaServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
index def1dc2..2871118 100644
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -303,6 +303,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
                    topicConfigs = configs,
                    defaultConfig = defaultLogConfig,
                    cleanerConfig = cleanerConfig,
+                   ioThreads = config.numRecoveryThreadsPerDataDir,
                    flushCheckMs = config.logFlushSchedulerIntervalMs,
                    flushCheckpointMs = config.logFlushOffsetCheckpointIntervalMs,
                    retentionCheckMs = config.logCleanupIntervalMs,

http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/main/scala/kafka/utils/KafkaScheduler.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/KafkaScheduler.scala b/core/src/main/scala/kafka/utils/KafkaScheduler.scala
index 8e37505..9a16343 100644
--- a/core/src/main/scala/kafka/utils/KafkaScheduler.scala
+++ b/core/src/main/scala/kafka/utils/KafkaScheduler.scala
@@ -93,16 +93,14 @@ class KafkaScheduler(val threads: Int,
     debug("Scheduling task %s with initial delay %d ms and period %d ms."
         .format(name, TimeUnit.MILLISECONDS.convert(delay, unit), TimeUnit.MILLISECONDS.convert(period, unit)))
     ensureStarted
-    val runnable = new Runnable {
-      def run() = {
-        try {
-          trace("Begining execution of scheduled task '%s'.".format(name))
-          fun()
-        } catch {
-          case t: Throwable => error("Uncaught exception in scheduled task '" + name +"'", t)
-        } finally {
-          trace("Completed execution of scheduled task '%s'.".format(name))
-        }
+    val runnable = Utils.runnable {
+      try {
+        trace("Begining execution of scheduled task '%s'.".format(name))
+        fun()
+      } catch {
+        case t: Throwable => error("Uncaught exception in scheduled task '" + name +"'", t)
+      } finally {
+        trace("Completed execution of scheduled task '%s'.".format(name))
       }
     }
     if(period >= 0)

http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/main/scala/kafka/utils/Utils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/utils/Utils.scala b/core/src/main/scala/kafka/utils/Utils.scala
index 09bfbce..da52b42 100644
--- a/core/src/main/scala/kafka/utils/Utils.scala
+++ b/core/src/main/scala/kafka/utils/Utils.scala
@@ -49,9 +49,9 @@ object Utils extends Logging {
    * @param fun A function
    * @return A Runnable that just executes the function
    */
-  def runnable(fun: () => Unit): Runnable = 
-    new Runnable() {
-      def run() = fun()
+  def runnable(fun: => Unit): Runnable =
+    new Runnable {
+      def run() = fun
     }
 
   /**

http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
index d03d4c4..7d4c70c 100644
--- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
@@ -35,21 +35,11 @@ class LogManagerTest extends JUnit3Suite {
   var logManager: LogManager = null
   val name = "kafka"
   val veryLargeLogFlushInterval = 10000000L
-  val cleanerConfig = CleanerConfig(enableCleaner = false)
 
   override def setUp() {
     super.setUp()
     logDir = TestUtils.tempDir()
-    logManager = new LogManager(logDirs = Array(logDir), 
-                                topicConfigs = Map(), 
-                                defaultConfig = logConfig, 
-                                cleanerConfig = cleanerConfig, 
-                                flushCheckMs = 1000L, 
-                                flushCheckpointMs = 100000L, 
-                                retentionCheckMs = 1000L, 
-                                scheduler = time.scheduler, 
-                                time = time,
-                                brokerState = new BrokerState())
+    logManager = createLogManager()
     logManager.startup
     logDir = logManager.logDirs(0)
   }
@@ -125,18 +115,7 @@ class LogManagerTest extends JUnit3Suite {
     logManager.shutdown()
 
     val config = logConfig.copy(segmentSize = 10 * (setSize - 1), retentionSize = 5L * 10L * setSize + 10L)
-    logManager = new LogManager(
-      logDirs = Array(logDir),
-      topicConfigs = Map(),
-      defaultConfig = config,
-      cleanerConfig = cleanerConfig,
-      flushCheckMs = 1000L,
-      flushCheckpointMs = 100000L,
-      retentionCheckMs = 1000L,
-      scheduler = time.scheduler,
-      brokerState = new BrokerState(),
-      time = time
-    )
+    logManager = createLogManager()
     logManager.startup
 
     // create a log
@@ -176,18 +155,7 @@ class LogManagerTest extends JUnit3Suite {
   def testTimeBasedFlush() {
     logManager.shutdown()
     val config = logConfig.copy(flushMs = 1000)
-    logManager = new LogManager(
-      logDirs = Array(logDir),
-      topicConfigs = Map(),
-      defaultConfig = config,
-      cleanerConfig = cleanerConfig,
-      flushCheckMs = 1000L,
-      flushCheckpointMs = 10000L,
-      retentionCheckMs = 1000L,
-      scheduler = time.scheduler,
-      brokerState = new BrokerState(),
-      time = time
-    )
+    logManager = createLogManager()
     logManager.startup
     val log = logManager.createLog(TopicAndPartition(name, 0), config)
     val lastFlush = log.lastFlushTime
@@ -209,19 +177,8 @@ class LogManagerTest extends JUnit3Suite {
                      TestUtils.tempDir(), 
                      TestUtils.tempDir())
     logManager.shutdown()
-    logManager = new LogManager(
-      logDirs = dirs,
-      topicConfigs = Map(),
-      defaultConfig = logConfig,
-      cleanerConfig = cleanerConfig,
-      flushCheckMs = 1000L,
-      flushCheckpointMs = 10000L,
-      retentionCheckMs = 1000L,
-      scheduler = time.scheduler,
-      brokerState = new BrokerState(),
-      time = time
-    )
-    
+    logManager = createLogManager()
+
     // verify that logs are always assigned to the least loaded partition
     for(partition <- 0 until 20) {
       logManager.createLog(TopicAndPartition("test", partition), logConfig)
@@ -237,18 +194,7 @@ class LogManagerTest extends JUnit3Suite {
   @Test
   def testTwoLogManagersUsingSameDirFails() {
     try {
-      new LogManager(
-        logDirs = Array(logDir),
-        topicConfigs = Map(),
-        defaultConfig = logConfig,
-        cleanerConfig = cleanerConfig,
-        flushCheckMs = 1000L,
-        flushCheckpointMs = 10000L,
-        retentionCheckMs = 1000L,
-        scheduler = time.scheduler,
-        brokerState = new BrokerState(),
-        time = time
-      )
+      createLogManager()
       fail("Should not be able to create a second log manager instance with the same data directory")
     } catch {
       case e: KafkaException => // this is good 
@@ -270,16 +216,8 @@ class LogManagerTest extends JUnit3Suite {
   def testRecoveryDirectoryMappingWithTrailingSlash() {
     logManager.shutdown()
     logDir = TestUtils.tempDir()
-    logManager = new LogManager(logDirs = Array(new File(logDir.getAbsolutePath + File.separator)),
-      topicConfigs = Map(),
-      defaultConfig = logConfig,
-      cleanerConfig = cleanerConfig,
-      flushCheckMs = 1000L,
-      flushCheckpointMs = 100000L,
-      retentionCheckMs = 1000L,
-      scheduler = time.scheduler,
-      time = time,
-      brokerState = new BrokerState())
+    logManager = TestUtils.createLogManager(
+      logDirs = Array(new File(logDir.getAbsolutePath + File.separator)))
     logManager.startup
     verifyCheckpointRecovery(Seq(TopicAndPartition("test-a", 1)), logManager)
   }
@@ -293,16 +231,7 @@ class LogManagerTest extends JUnit3Suite {
     logDir = new File("data" + File.separator + logDir.getName)
     logDir.mkdirs()
     logDir.deleteOnExit()
-    logManager = new LogManager(logDirs = Array(logDir),
-      topicConfigs = Map(),
-      defaultConfig = logConfig,
-      cleanerConfig = cleanerConfig,
-      flushCheckMs = 1000L,
-      flushCheckpointMs = 100000L,
-      retentionCheckMs = 1000L,
-      scheduler = time.scheduler,
-      time = time,
-      brokerState = new BrokerState())
+    logManager = createLogManager()
     logManager.startup
     verifyCheckpointRecovery(Seq(TopicAndPartition("test-a", 1)), logManager)
   }
@@ -327,4 +256,12 @@ class LogManagerTest extends JUnit3Suite {
       }
     }
   }
+
+
+  private def createLogManager(logDirs: Array[File] = Array(this.logDir)): LogManager = {
+    TestUtils.createLogManager(
+      defaultConfig = logConfig,
+      logDirs = logDirs,
+      time = this.time)
+  }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
index 558a5d6..e532c28 100644
--- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
+++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
@@ -32,16 +32,11 @@ class HighwatermarkPersistenceTest extends JUnit3Suite {
 
   val configs = TestUtils.createBrokerConfigs(2).map(new KafkaConfig(_))
   val topic = "foo"
-  val logManagers = configs.map(config => new LogManager(logDirs = config.logDirs.map(new File(_)).toArray,
-                                                         topicConfigs = Map(),
-                                                         defaultConfig = LogConfig(),
-                                                         cleanerConfig = CleanerConfig(),
-                                                         flushCheckMs = 30000,
-                                                         flushCheckpointMs = 10000L,
-                                                         retentionCheckMs = 30000,
-                                                         scheduler = new KafkaScheduler(1),
-                                                         brokerState = new BrokerState(),
-                                                         time = new MockTime))
+  val logManagers = configs map { config =>
+    TestUtils.createLogManager(
+      logDirs = config.logDirs.map(new File(_)).toArray,
+      cleanerConfig = CleanerConfig())
+  }
     
   @After
   def teardown() {
@@ -147,4 +142,4 @@ class HighwatermarkPersistenceTest extends JUnit3Suite {
     replicaManager.highWatermarkCheckpoints(new File(replicaManager.config.logDirs(0)).getAbsolutePath).read.getOrElse(TopicAndPartition(topic, partition), 0L)
   }
   
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 518d416..9abf219 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -18,7 +18,6 @@
 package kafka.server
 
 import kafka.utils.{MockScheduler, MockTime, TestUtils}
-import kafka.log.{CleanerConfig, LogManager, LogConfig}
 
 import java.util.concurrent.atomic.AtomicBoolean
 import java.io.File
@@ -37,7 +36,7 @@ class ReplicaManagerTest extends JUnit3Suite {
     val props = TestUtils.createBrokerConfig(1)
     val config = new KafkaConfig(props)
     val zkClient = EasyMock.createMock(classOf[ZkClient])
-    val mockLogMgr = createLogManager(config.logDirs.map(new File(_)).toArray)
+    val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray)
     val time: MockTime = new MockTime()
     val rm = new ReplicaManager(config, time, zkClient, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false))
     val partition = rm.getOrCreatePartition(topic, 1, 1)
@@ -51,26 +50,11 @@ class ReplicaManagerTest extends JUnit3Suite {
     props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath)
     val config = new KafkaConfig(props)
     val zkClient = EasyMock.createMock(classOf[ZkClient])
-    val mockLogMgr = createLogManager(config.logDirs.map(new File(_)).toArray)
+    val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray)
     val time: MockTime = new MockTime()
     val rm = new ReplicaManager(config, time, zkClient, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false))
     val partition = rm.getOrCreatePartition(topic, 1, 1)
     partition.getOrCreateReplica(1)
     rm.checkpointHighWatermarks()
   }
-
-  private def createLogManager(logDirs: Array[File]): LogManager = {
-    val time = new MockTime()
-    return new LogManager(logDirs,
-      topicConfigs = Map(),
-      defaultConfig = new LogConfig(),
-      cleanerConfig = CleanerConfig(enableCleaner = false),
-      flushCheckMs = 1000L,
-      flushCheckpointMs = 100000L,
-      retentionCheckMs = 1000L,
-      scheduler = time.scheduler,
-      brokerState = new BrokerState(),
-      time = time)
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/f489493c/core/src/test/scala/unit/kafka/utils/TestUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index 4d01d25..c4e13c5 100644
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -39,6 +39,7 @@ import kafka.serializer.{StringEncoder, DefaultEncoder, Encoder}
 import kafka.common.TopicAndPartition
 import kafka.admin.AdminUtils
 import kafka.producer.ProducerConfig
+import kafka.log._
 
 import junit.framework.AssertionFailedError
 import junit.framework.Assert._
@@ -689,6 +690,30 @@ object TestUtils extends Logging {
   def checkIfReassignPartitionPathExists(zkClient: ZkClient): Boolean = {
     ZkUtils.pathExists(zkClient, ZkUtils.ReassignPartitionsPath)
   }
+
+
+  /**
+   * Create new LogManager instance with default configuration for testing
+   */
+  def createLogManager(
+    logDirs: Array[File] = Array.empty[File],
+    defaultConfig: LogConfig = LogConfig(),
+    cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false),
+    time: MockTime = new MockTime()) =
+  {
+    new LogManager(
+      logDirs = logDirs,
+      topicConfigs = Map(),
+      defaultConfig = defaultConfig,
+      cleanerConfig = cleanerConfig,
+      ioThreads = 4,
+      flushCheckMs = 1000L,
+      flushCheckpointMs = 10000L,
+      retentionCheckMs = 1000L,
+      scheduler = time.scheduler,
+      time = time,
+      brokerState = new BrokerState())
+  }
 }
 
 object TestZKUtils {


[16/37] git commit: KAFKA-1512 Add per-ip connection limits.

Posted by jj...@apache.org.
KAFKA-1512 Add per-ip connection limits.


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/8e444a35
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/8e444a35
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/8e444a35

Branch: refs/heads/transactional_messaging
Commit: 8e444a3562d6723b9f33cbdaa6a461409c84c98b
Parents: b428d8c
Author: Jay Kreps <ja...@gmail.com>
Authored: Mon Jun 30 21:28:04 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Wed Jul 16 09:53:15 2014 -0700

----------------------------------------------------------------------
 .../main/scala/kafka/network/SocketServer.scala | 146 +++++++++++++------
 .../main/scala/kafka/server/KafkaConfig.scala   |   6 +
 .../main/scala/kafka/server/KafkaServer.scala   |   3 +-
 .../unit/kafka/network/SocketServerTest.scala   |  20 ++-
 4 files changed, 129 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/8e444a35/core/src/main/scala/kafka/network/SocketServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala
index 4976d9c..8e99de0 100644
--- a/core/src/main/scala/kafka/network/SocketServer.scala
+++ b/core/src/main/scala/kafka/network/SocketServer.scala
@@ -23,6 +23,8 @@ import java.net._
 import java.io._
 import java.nio.channels._
 
+import scala.collection._
+
 import kafka.common.KafkaException
 import kafka.metrics.KafkaMetricsGroup
 import kafka.utils._
@@ -41,7 +43,9 @@ class SocketServer(val brokerId: Int,
                    val maxQueuedRequests: Int,
                    val sendBufferSize: Int,
                    val recvBufferSize: Int,
-                   val maxRequestSize: Int = Int.MaxValue) extends Logging with KafkaMetricsGroup {
+                   val maxRequestSize: Int = Int.MaxValue,
+                   val maxConnectionsPerIp: Int = Int.MaxValue,
+                   val maxConnectionsPerIpOverrides: Map[String, Int] = Map[String, Int]()) extends Logging with KafkaMetricsGroup {
   this.logIdent = "[Socket Server on Broker " + brokerId + "], "
   private val time = SystemTime
   private val processors = new Array[Processor](numProcessorThreads)
@@ -55,17 +59,23 @@ class SocketServer(val brokerId: Int,
    * Start the socket server
    */
   def startup() {
+    val quotas = new ConnectionQuotas(maxConnectionsPerIp, maxConnectionsPerIpOverrides)
     for(i <- 0 until numProcessorThreads) {
-      processors(i) = new Processor(i, time, maxRequestSize, aggregateIdleMeter,
-        newMeter("NetworkProcessor-" + i + "-IdlePercent", "percent", TimeUnit.NANOSECONDS),
-        numProcessorThreads, requestChannel)
+      processors(i) = new Processor(i, 
+                                    time, 
+                                    maxRequestSize, 
+                                    aggregateIdleMeter,
+                                    newMeter("NetworkProcessor-" + i + "-IdlePercent", "percent", TimeUnit.NANOSECONDS),
+                                    numProcessorThreads, 
+                                    requestChannel,
+                                    quotas)
       Utils.newThread("kafka-network-thread-%d-%d".format(port, i), processors(i), false).start()
     }
     // register the processor threads for notification of responses
     requestChannel.addResponseListener((id:Int) => processors(id).wakeup())
    
     // start accepting connections
-    this.acceptor = new Acceptor(host, port, processors, sendBufferSize, recvBufferSize)
+    this.acceptor = new Acceptor(host, port, processors, sendBufferSize, recvBufferSize, quotas)
     Utils.newThread("kafka-socket-acceptor", acceptor, false).start()
     acceptor.awaitStartup
     info("Started")
@@ -87,7 +97,7 @@ class SocketServer(val brokerId: Int,
 /**
  * A base class with some helper variables and methods
  */
-private[kafka] abstract class AbstractServerThread extends Runnable with Logging {
+private[kafka] abstract class AbstractServerThread(connectionQuotas: ConnectionQuotas) extends Runnable with Logging {
 
   protected val selector = Selector.open();
   private val startupLatch = new CountDownLatch(1)
@@ -131,13 +141,48 @@ private[kafka] abstract class AbstractServerThread extends Runnable with Logging
    */
   def wakeup() = selector.wakeup()
   
+  /**
+   * Close the given key and associated socket
+   */
+  def close(key: SelectionKey) {
+    if(key != null) {
+      key.attach(null)
+      close(key.channel.asInstanceOf[SocketChannel])
+      swallowError(key.cancel())
+    }
+  }
+  
+  def close(channel: SocketChannel) {
+    if(channel != null) {
+      debug("Closing connection from " + channel.socket.getRemoteSocketAddress())
+      connectionQuotas.dec(channel.socket.getInetAddress)
+      swallowError(channel.socket().close())
+      swallowError(channel.close())
+    }
+  }
+  
+  /**
+   * Close all open connections
+   */
+  def closeAll() {
+    val iter = this.selector.keys().iterator()
+    while (iter.hasNext) {
+      val key = iter.next()
+      close(key)
+    }
+  }
+  
 }
 
 /**
  * Thread that accepts and configures new connections. There is only need for one of these
  */
-private[kafka] class Acceptor(val host: String, val port: Int, private val processors: Array[Processor],
-                              val sendBufferSize: Int, val recvBufferSize: Int) extends AbstractServerThread {
+private[kafka] class Acceptor(val host: String, 
+                              val port: Int, 
+                              private val processors: Array[Processor],
+                              val sendBufferSize: Int, 
+                              val recvBufferSize: Int,
+                              connectionQuotas: ConnectionQuotas) extends AbstractServerThread(connectionQuotas) {
   val serverChannel = openServerSocket(host, port)
 
   /**
@@ -158,14 +203,14 @@ private[kafka] class Acceptor(val host: String, val port: Int, private val proce
             key = iter.next
             iter.remove()
             if(key.isAcceptable)
-                accept(key, processors(currentProcessor))
-              else
-                throw new IllegalStateException("Unrecognized key state for acceptor thread.")
+               accept(key, processors(currentProcessor))
+            else
+               throw new IllegalStateException("Unrecognized key state for acceptor thread.")
 
-              // round robin to the next processor thread
-              currentProcessor = (currentProcessor + 1) % processors.length
+            // round robin to the next processor thread
+            currentProcessor = (currentProcessor + 1) % processors.length
           } catch {
-            case e: Throwable => error("Error in acceptor", e)
+            case e: Throwable => error("Error while accepting connection", e)
           }
         }
       }
@@ -187,6 +232,7 @@ private[kafka] class Acceptor(val host: String, val port: Int, private val proce
         new InetSocketAddress(host, port)
     val serverChannel = ServerSocketChannel.open()
     serverChannel.configureBlocking(false)
+    serverChannel.socket().setReceiveBufferSize(recvBufferSize)
     try {
       serverChannel.socket.bind(socketAddress)
       info("Awaiting socket connections on %s:%d.".format(socketAddress.getHostName, port))
@@ -202,19 +248,24 @@ private[kafka] class Acceptor(val host: String, val port: Int, private val proce
    */
   def accept(key: SelectionKey, processor: Processor) {
     val serverSocketChannel = key.channel().asInstanceOf[ServerSocketChannel]
-    serverSocketChannel.socket().setReceiveBufferSize(recvBufferSize)
-
     val socketChannel = serverSocketChannel.accept()
-    socketChannel.configureBlocking(false)
-    socketChannel.socket().setTcpNoDelay(true)
-    socketChannel.socket().setSendBufferSize(sendBufferSize)
+    try {
+      connectionQuotas.inc(socketChannel.socket().getInetAddress)
+      socketChannel.configureBlocking(false)
+      socketChannel.socket().setTcpNoDelay(true)
+      socketChannel.socket().setSendBufferSize(sendBufferSize)
 
-    debug("Accepted connection from %s on %s. sendBufferSize [actual|requested]: [%d|%d] recvBufferSize [actual|requested]: [%d|%d]"
-          .format(socketChannel.socket.getInetAddress, socketChannel.socket.getLocalSocketAddress,
+      debug("Accepted connection from %s on %s. sendBufferSize [actual|requested]: [%d|%d] recvBufferSize [actual|requested]: [%d|%d]"
+            .format(socketChannel.socket.getInetAddress, socketChannel.socket.getLocalSocketAddress,
                   socketChannel.socket.getSendBufferSize, sendBufferSize,
                   socketChannel.socket.getReceiveBufferSize, recvBufferSize))
 
-    processor.accept(socketChannel)
+      processor.accept(socketChannel)
+    } catch {
+      case e: TooManyConnectionsException =>
+        info("Rejected connection from %s, address already has the configured maximum of %d connections.".format(e.ip, e.count))
+        close(socketChannel)
+    }
   }
 
 }
@@ -229,7 +280,8 @@ private[kafka] class Processor(val id: Int,
                                val aggregateIdleMeter: Meter,
                                val idleMeter: Meter,
                                val totalProcessorThreads: Int,
-                               val requestChannel: RequestChannel) extends AbstractServerThread {
+                               val requestChannel: RequestChannel,
+                               connectionQuotas: ConnectionQuotas) extends AbstractServerThread(connectionQuotas) {
   
   private val newConnections = new ConcurrentLinkedQueue[SocketChannel]()
 
@@ -324,26 +376,6 @@ private[kafka] class Processor(val id: Int,
       }
     }
   }
-  
-  private def close(key: SelectionKey) {
-    val channel = key.channel.asInstanceOf[SocketChannel]
-    debug("Closing connection from " + channel.socket.getRemoteSocketAddress())
-    swallowError(channel.socket().close())
-    swallowError(channel.close())
-    key.attach(null)
-    swallowError(key.cancel())
-  }
-  
-  /*
-   * Close all open connections
-   */
-  private def closeAll() {
-    val iter = this.selector.keys().iterator()
-    while (iter.hasNext) {
-      val key = iter.next()
-      close(key)
-    }
-  }
 
   /**
    * Queue up a new connection for reading
@@ -419,3 +451,31 @@ private[kafka] class Processor(val id: Int,
   private def channelFor(key: SelectionKey) = key.channel().asInstanceOf[SocketChannel]
 
 }
+
+class ConnectionQuotas(val defaultMax: Int, overrideQuotas: Map[String, Int]) {
+  private val overrides = overrideQuotas.map(entry => (InetAddress.getByName(entry._1), entry._2))
+  private val counts = mutable.Map[InetAddress, Int]()
+  
+  def inc(addr: InetAddress) {
+    counts synchronized {
+      val count = counts.getOrElse(addr, 0)
+      counts.put(addr, count + 1)
+      val max = overrides.getOrElse(addr, defaultMax)
+      if(count >= max)
+        throw new TooManyConnectionsException(addr, max)
+    }
+  }
+  
+  def dec(addr: InetAddress) {
+    counts synchronized {
+      val count = counts.get(addr).get
+      if(count == 1)
+        counts.remove(addr)
+      else
+        counts.put(addr, count - 1)
+    }
+  }
+  
+}
+
+class TooManyConnectionsException(val ip: InetAddress, val count: Int) extends KafkaException("Too many connections from %s (maximum = %d)".format(ip, count))

http://git-wip-us.apache.org/repos/asf/kafka/blob/8e444a35/core/src/main/scala/kafka/server/KafkaConfig.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index bb2e654..50b09ed 100644
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -106,6 +106,12 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro
 
   /* the maximum number of bytes in a socket request */
   val socketRequestMaxBytes: Int = props.getIntInRange("socket.request.max.bytes", 100*1024*1024, (1, Int.MaxValue))
+  
+  /* the maximum number of connections we allow from each ip address */
+  val maxConnectionsPerIp: Int = props.getIntInRange("max.connections.per.ip", Int.MaxValue, (1, Int.MaxValue))
+  
+  /* per-ip or hostname overrides to the default maximum number of connections */
+  val maxConnectionsPerIpOverrides = props.getMap("max.connections.per.ip.overrides").map(entry => (entry._1, entry._2.toInt))
 
   /*********** Log Configuration ***********/
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/8e444a35/core/src/main/scala/kafka/server/KafkaServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
index 5a56f57..def1dc2 100644
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -91,7 +91,8 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg
                                     config.queuedMaxRequests,
                                     config.socketSendBufferBytes,
                                     config.socketReceiveBufferBytes,
-                                    config.socketRequestMaxBytes)
+                                    config.socketRequestMaxBytes,
+                                    config.maxConnectionsPerIp)
     socketServer.startup()
 
     replicaManager = new ReplicaManager(config, time, zkClient, kafkaScheduler, logManager, isShuttingDown)

http://git-wip-us.apache.org/repos/asf/kafka/blob/8e444a35/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
index 1c492de..3b83a86 100644
--- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
+++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
@@ -40,7 +40,8 @@ class SocketServerTest extends JUnitSuite {
                                               maxQueuedRequests = 50,
                                               sendBufferSize = 300000,
                                               recvBufferSize = 300000,
-                                              maxRequestSize = 50)
+                                              maxRequestSize = 50,
+                                              maxConnectionsPerIp = 5)
   server.startup()
 
   def sendRequest(socket: Socket, id: Short, request: Array[Byte]) {
@@ -75,7 +76,7 @@ class SocketServerTest extends JUnitSuite {
   def cleanup() {
     server.shutdown()
   }
-
+  
   @Test
   def simpleRequest() {
     val socket = connect()
@@ -139,4 +140,19 @@ class SocketServerTest extends JUnitSuite {
     // doing a subsequent send should throw an exception as the connection should be closed.
     sendRequest(socket, 0, bytes)
   }
+  
+  @Test
+  def testMaxConnectionsPerIp() {
+    // make the maximum allowable number of connections and then leak them
+    val conns = (0 until server.maxConnectionsPerIp).map(i => connect())
+    
+    // now try one more (should fail)
+    try {
+      val conn = connect()
+      sendRequest(conn, 100, "hello".getBytes)
+      assertEquals(-1, conn.getInputStream().read())
+    } catch {
+      case e: IOException => // this is good
+    }
+  }
 }


[03/37] git commit: KAFKA-1498 Misc. producer performance enhancements. Patch from Guozhang.

Posted by jj...@apache.org.
KAFKA-1498 Misc. producer performance enhancements. Patch from Guozhang.


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/f1c6e97d
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/f1c6e97d
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/f1c6e97d

Branch: refs/heads/transactional_messaging
Commit: f1c6e97d718581566d037a48640ac3d869d1f15a
Parents: c4b9564
Author: Jay Kreps <ja...@gmail.com>
Authored: Tue Jul 1 13:21:36 2014 -0700
Committer: Jay Kreps <ja...@gmail.com>
Committed: Tue Jul 1 13:21:36 2014 -0700

----------------------------------------------------------------------
 .../org/apache/kafka/clients/NetworkClient.java | 20 +++---
 .../kafka/clients/producer/KafkaProducer.java   | 16 +++--
 .../clients/producer/internals/Metadata.java    | 15 ++--
 .../producer/internals/RecordAccumulator.java   | 76 ++++++++++++++------
 .../clients/producer/internals/Sender.java      | 26 ++++---
 .../kafka/common/record/MemoryRecords.java      | 26 +++++--
 .../apache/kafka/clients/NetworkClientTest.java |  2 +-
 .../kafka/clients/producer/MetadataTest.java    | 10 +--
 .../clients/producer/RecordAccumulatorTest.java | 14 ++--
 .../kafka/clients/producer/SenderTest.java      |  6 +-
 .../main/scala/kafka/tools/MirrorMaker.scala    | 63 ++++++++--------
 .../kafka/api/ProducerFailureHandlingTest.scala | 25 ++++---
 .../unit/kafka/network/SocketServerTest.scala   |  8 ++-
 13 files changed, 187 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
index 522881c..d21f922 100644
--- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
@@ -116,7 +116,7 @@ public class NetworkClient implements KafkaClient {
 
     /**
      * Check if the node with the given id is ready to send more requests.
-     * @param nodeId The node id
+     * @param node The given node id
      * @param now The current time in ms
      * @return true if the node is ready
      */
@@ -126,8 +126,8 @@ public class NetworkClient implements KafkaClient {
     }
 
     private boolean isReady(int node, long now) {
-        if (this.metadata.needsUpdate(now))
-            // if we need to update our metadata declare all requests unready to metadata requests first priority
+        if (!this.metadataFetchInProgress && this.metadata.timeToNextUpdate(now) == 0)
+            // if we need to update our metadata now declare all requests unready to make metadata requests first priority
             return false;
         else
             // otherwise we are ready if we are connected and can send more requests
@@ -144,9 +144,12 @@ public class NetworkClient implements KafkaClient {
      */
     @Override
     public List<ClientResponse> poll(List<ClientRequest> requests, long timeout, long now) {
-        // should we update our metadata?
         List<NetworkSend> sends = new ArrayList<NetworkSend>();
-        maybeUpdateMetadata(sends, now);
+
+        // should we update our metadata?
+        long metadataTimeout = metadata.timeToNextUpdate(now);
+        if (!this.metadataFetchInProgress && metadataTimeout == 0)
+            maybeUpdateMetadata(sends, now);
 
         for (int i = 0; i < requests.size(); i++) {
             ClientRequest request = requests.get(i);
@@ -160,7 +163,7 @@ public class NetworkClient implements KafkaClient {
 
         // do the I/O
         try {
-            this.selector.poll(timeout, sends);
+            this.selector.poll(Math.min(timeout, metadataTimeout), sends);
         } catch (IOException e) {
             log.error("Unexpected error during I/O in producer network thread", e);
         }
@@ -340,12 +343,9 @@ public class NetworkClient implements KafkaClient {
     }
 
     /**
-     * Add a metadata request to the list of sends if we need to make one
+     * Add a metadata request to the list of sends if we can make one
      */
     private void maybeUpdateMetadata(List<NetworkSend> sends, long now) {
-        if (this.metadataFetchInProgress || !metadata.needsUpdate(now))
-            return;
-
         Node node = this.leastLoadedNode(now);
         if (node == null)
             return;

http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
index 00775ab..d85ca30 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
@@ -22,7 +22,6 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.kafka.clients.NetworkClient;
-import org.apache.kafka.clients.producer.internals.FutureRecordMetadata;
 import org.apache.kafka.clients.producer.internals.Metadata;
 import org.apache.kafka.clients.producer.internals.Partitioner;
 import org.apache.kafka.clients.producer.internals.RecordAccumulator;
@@ -232,10 +231,14 @@ public class KafkaProducer implements Producer {
             ensureValidRecordSize(serializedSize);
             TopicPartition tp = new TopicPartition(record.topic(), partition);
             log.trace("Sending record {} with callback {} to topic {} partition {}", record, callback, record.topic(), partition);
-            FutureRecordMetadata future = accumulator.append(tp, record.key(), record.value(), compressionType, callback);
-            this.sender.wakeup();
-            return future;
-            // For API exceptions return them in the future;
+            RecordAccumulator.RecordAppendResult result = accumulator.append(tp, record.key(), record.value(), compressionType, callback);
+            if (result.batchIsFull || result.newBatchCreated) {
+                log.trace("Waking up the sender since topic {} partition {} is either full or getting a new batch", record.topic(), partition);
+                this.sender.wakeup();
+            }
+            return result.future;
+            // Handling exceptions and record the errors;
+            // For API exceptions return them in the future,
             // for other exceptions throw directly
         } catch (ApiException e) {
             log.debug("Exception occurred during message send:", e);
@@ -246,6 +249,9 @@ public class KafkaProducer implements Producer {
         } catch (InterruptedException e) {
             this.errors.record();
             throw new KafkaException(e);
+        } catch (KafkaException e) {
+            this.errors.record();
+            throw e;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
index 57bc285..8890aa2 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
@@ -101,15 +101,14 @@ public final class Metadata {
     }
 
     /**
-     * Does the current cluster info need to be updated? An update is needed if it has been at least refreshBackoffMs
-     * since our last update and either (1) an update has been requested or (2) the current metadata has expired (more
-     * than metadataExpireMs has passed since the last refresh)
+     * The next time to update the cluster info is the maximum of the time the current info will expire
+     * and the time the current info can be updated (i.e. backoff time has elapsed); If an update has
+     * been request then the expiry time is now
      */
-    public synchronized boolean needsUpdate(long now) {
-        long msSinceLastUpdate = now - this.lastRefreshMs;
-        boolean updateAllowed = msSinceLastUpdate >= this.refreshBackoffMs;
-        boolean updateNeeded = this.forceUpdate || msSinceLastUpdate >= this.metadataExpireMs;
-        return updateAllowed && updateNeeded;
+    public synchronized long timeToNextUpdate(long nowMs) {
+        long timeToExpire = forceUpdate ? 0 : Math.max(this.lastRefreshMs + this.metadataExpireMs - nowMs, 0);
+        long timeToAllowUpdate = this.lastRefreshMs + this.refreshBackoffMs - nowMs;
+        return Math.max(timeToExpire, timeToAllowUpdate);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
index 1ed3c28..c5d4700 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
@@ -120,27 +120,29 @@ public final class RecordAccumulator {
     }
 
     /**
-     * Add a record to the accumulator.
+     * Add a record to the accumulator, return the append result
      * <p>
-     * This method will block if sufficient memory isn't available for the record unless blocking has been disabled.
-     * 
+     * The append result will contain the future metadata, and flag for whether the appended batch is full or a new batch is created
+     * <p>
+     *
      * @param tp The topic/partition to which this record is being sent
      * @param key The key for the record
      * @param value The value for the record
      * @param compression The compression codec for the record
      * @param callback The user-supplied callback to execute when the request is complete
      */
-    public FutureRecordMetadata append(TopicPartition tp, byte[] key, byte[] value, CompressionType compression, Callback callback) throws InterruptedException {
+    public RecordAppendResult append(TopicPartition tp, byte[] key, byte[] value, CompressionType compression, Callback callback) throws InterruptedException {
         if (closed)
             throw new IllegalStateException("Cannot send after the producer is closed.");
         // check if we have an in-progress batch
         Deque<RecordBatch> dq = dequeFor(tp);
         synchronized (dq) {
-            RecordBatch batch = dq.peekLast();
-            if (batch != null) {
-                FutureRecordMetadata future = batch.tryAppend(key, value, callback);
-                if (future != null)
-                    return future;
+            RecordBatch last = dq.peekLast();
+            if (last != null) {
+                FutureRecordMetadata future = last.tryAppend(key, value, callback);
+                if (future != null) {
+                    return new RecordAppendResult(future, dq.size() > 1 || last.records.isFull(), false);
+                }
             }
         }
 
@@ -156,15 +158,15 @@ public final class RecordAccumulator {
                     // Somebody else found us a batch, return the one we waited for! Hopefully this doesn't happen
                     // often...
                     free.deallocate(buffer);
-                    return future;
+                    return new RecordAppendResult(future, dq.size() > 1 || last.records.isFull(), false);
                 }
             }
-            MemoryRecords records = MemoryRecords.emptyRecords(buffer, compression);
+            MemoryRecords records = MemoryRecords.emptyRecords(buffer, compression, this.batchSize);
             RecordBatch batch = new RecordBatch(tp, records, time.milliseconds());
             FutureRecordMetadata future = Utils.notNull(batch.tryAppend(key, value, callback));
 
             dq.addLast(batch);
-            return future;
+            return new RecordAppendResult(future, dq.size() > 1 || batch.records.isFull(), true);
         }
     }
 
@@ -181,7 +183,9 @@ public final class RecordAccumulator {
     }
 
     /**
-     * Get a list of nodes whose partitions are ready to be sent.
+     * Get a list of nodes whose partitions are ready to be sent, and the time to when any partition will be ready if no
+     * partitions are ready yet; If the ready nodes list is non-empty, the timeout value will be 0. Also return the flag
+     * for whether there are any unknown leaders for the accumulated partition batches.
      * <p>
      * A destination node is ready to send data if ANY one of its partition is not backing off the send and ANY of the
      * following are true :
@@ -193,31 +197,39 @@ public final class RecordAccumulator {
      * <li>The accumulator has been closed
      * </ol>
      */
-    public Set<Node> ready(Cluster cluster, long now) {
+    public ReadyCheckResult ready(Cluster cluster, long nowMs) {
         Set<Node> readyNodes = new HashSet<Node>();
-        boolean exhausted = this.free.queued() > 0;
+        long nextReadyCheckDelayMs = Long.MAX_VALUE;
+        boolean unknownLeadersExist = false;
 
+        boolean exhausted = this.free.queued() > 0;
         for (Map.Entry<TopicPartition, Deque<RecordBatch>> entry : this.batches.entrySet()) {
             TopicPartition part = entry.getKey();
             Deque<RecordBatch> deque = entry.getValue();
-            // if the leader is unknown use an Unknown node placeholder
+
             Node leader = cluster.leaderFor(part);
-            if (!readyNodes.contains(leader)) {
+            if (leader == null) {
+                unknownLeadersExist = true;
+            } else if (!readyNodes.contains(leader)) {
                 synchronized (deque) {
                     RecordBatch batch = deque.peekFirst();
                     if (batch != null) {
-                        boolean backingOff = batch.attempts > 0 && batch.lastAttemptMs + retryBackoffMs > now;
+                        boolean backingOff = batch.attempts > 0 && batch.lastAttemptMs + retryBackoffMs > nowMs;
+                        long waitedTimeMs = nowMs - batch.lastAttemptMs;
+                        long timeToWaitMs = backingOff ? retryBackoffMs : lingerMs;
+                        long timeLeftMs = Math.max(timeToWaitMs - waitedTimeMs, 0);
                         boolean full = deque.size() > 1 || batch.records.isFull();
-                        boolean expired = now - batch.createdMs >= lingerMs;
+                        boolean expired = waitedTimeMs >= lingerMs;
                         boolean sendable = full || expired || exhausted || closed;
                         if (sendable && !backingOff)
                             readyNodes.add(leader);
+                        nextReadyCheckDelayMs = Math.min(timeLeftMs, nextReadyCheckDelayMs);
                     }
                 }
             }
         }
 
-        return readyNodes;
+        return new ReadyCheckResult(readyNodes, nextReadyCheckDelayMs, unknownLeadersExist);
     }
 
     /**
@@ -311,4 +323,28 @@ public final class RecordAccumulator {
         this.closed = true;
     }
 
+
+    public final static class RecordAppendResult {
+        public final FutureRecordMetadata future;
+        public final boolean batchIsFull;
+        public final boolean newBatchCreated;
+
+        public RecordAppendResult(FutureRecordMetadata future, boolean batchIsFull, boolean newBatchCreated) {
+            this.future = future;
+            this.batchIsFull = batchIsFull;
+            this.newBatchCreated = newBatchCreated;
+        }
+    }
+
+    public final static class ReadyCheckResult {
+        public final Set<Node> readyNodes;
+        public final long nextReadyCheckDelayMs;
+        public final boolean unknownLeadersExist;
+
+        public ReadyCheckResult(Set<Node> readyNodes, long nextReadyCheckDelayMs, boolean unknownLeadersExist) {
+            this.readyNodes = readyNodes;
+            this.nextReadyCheckDelayMs = nextReadyCheckDelayMs;
+            this.unknownLeadersExist = unknownLeadersExist;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
index 6fb5b82..52d209b 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
@@ -17,7 +17,6 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.kafka.clients.ClientRequest;
 import org.apache.kafka.clients.ClientResponse;
@@ -123,13 +122,13 @@ public class Sender implements Runnable {
         // okay we stopped accepting requests but there may still be
         // requests in the accumulator or waiting for acknowledgment,
         // wait until these are completed.
-        do {
+        while (this.accumulator.hasUnsent() || this.client.inFlightRequestCount() > 0) {
             try {
                 run(time.milliseconds());
             } catch (Exception e) {
                 log.error("Uncaught error in kafka producer I/O thread: ", e);
             }
-        } while (this.accumulator.hasUnsent() || this.client.inFlightRequestCount() > 0);
+        }
 
         this.client.close();
 
@@ -144,10 +143,14 @@ public class Sender implements Runnable {
     public void run(long now) {
         Cluster cluster = metadata.fetch();
         // get the list of partitions with data ready to send
-        Set<Node> ready = this.accumulator.ready(cluster, now);
+        RecordAccumulator.ReadyCheckResult result = this.accumulator.ready(cluster, now);
+
+        // if there are any partitions whose leaders are not known yet, force metadata update
+        if (result.unknownLeadersExist)
+            this.metadata.forceUpdate();
 
         // remove any nodes we aren't ready to send to
-        Iterator<Node> iter = ready.iterator();
+        Iterator<Node> iter = result.readyNodes.iterator();
         while (iter.hasNext()) {
             Node node = iter.next();
             if (!this.client.ready(node, now))
@@ -155,16 +158,20 @@ public class Sender implements Runnable {
         }
 
         // create produce requests
-        Map<Integer, List<RecordBatch>> batches = this.accumulator.drain(cluster, ready, this.maxRequestSize, now);
+        Map<Integer, List<RecordBatch>> batches = this.accumulator.drain(cluster, result.readyNodes, this.maxRequestSize, now);
         List<ClientRequest> requests = createProduceRequests(batches, now);
         sensors.updateProduceRequestMetrics(requests);
 
-        if (ready.size() > 0) {
-            log.trace("Nodes with data ready to send: {}", ready);
+        if (result.readyNodes.size() > 0) {
+            log.trace("Nodes with data ready to send: {}", result.readyNodes);
             log.trace("Created {} produce requests: {}", requests.size(), requests);
         }
 
-        List<ClientResponse> responses = this.client.poll(requests, 100L, now);
+        // if some partitions are already ready to be sent, the select time would be 0;
+        // otherwise if some partition already has some data accumulated but not ready yet,
+        // the select time will be the time difference between now and its linger expiry time;
+        // otherwise the select time will be the time difference between now and the metadata expiry time;
+        List<ClientResponse> responses = this.client.poll(requests, result.nextReadyCheckDelayMs, now);
         for (ClientResponse response : responses) {
             if (response.wasDisconnected())
                 handleDisconnect(response, now);
@@ -307,6 +314,7 @@ public class Sender implements Runnable {
 
             this.batchSizeSensor = metrics.sensor("batch-size");
             this.batchSizeSensor.add("batch-size-avg", "The average number of bytes sent per partition per-request.", new Avg());
+            this.batchSizeSensor.add("batch-size-max", "The max number of bytes sent per partition per-request.", new Max());
 
             this.compressionRateSensor = metrics.sensor("compression-rate");
             this.compressionRateSensor.add("compression-rate-avg", "The average compression rate of record batches.", new Avg());

http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
index 759f577..040e5b9 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
@@ -29,13 +29,15 @@ public class MemoryRecords implements Records {
 
     private final Compressor compressor;
     private final int capacity;
+    private final int sizeLimit;
     private ByteBuffer buffer;
     private boolean writable;
 
     // Construct a writable memory records
-    private MemoryRecords(ByteBuffer buffer, CompressionType type, boolean writable) {
+    private MemoryRecords(ByteBuffer buffer, CompressionType type, boolean writable, int sizeLimit) {
         this.writable = writable;
         this.capacity = buffer.capacity();
+        this.sizeLimit = sizeLimit;
         if (this.writable) {
             this.buffer = null;
             this.compressor = new Compressor(buffer, type);
@@ -45,12 +47,16 @@ public class MemoryRecords implements Records {
         }
     }
 
+    public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type, int capacity) {
+        return new MemoryRecords(buffer, type, true, capacity);
+    }
+
     public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type) {
-        return new MemoryRecords(buffer, type, true);
+        return emptyRecords(buffer, type, buffer.capacity());
     }
 
     public static MemoryRecords iterableRecords(ByteBuffer buffer) {
-        return new MemoryRecords(buffer, CompressionType.NONE, false);
+        return new MemoryRecords(buffer, CompressionType.NONE, false, buffer.capacity());
     }
 
     /**
@@ -88,14 +94,22 @@ public class MemoryRecords implements Records {
      * Note that the return value is based on the estimate of the bytes written to the compressor, which may not be
      * accurate if compression is really used. When this happens, the following append may cause dynamic buffer
      * re-allocation in the underlying byte buffer stream.
+     *
+     * Also note that besides the records' capacity, there is also a size limit for the batch. This size limit may be
+     * smaller than the capacity (e.g. when appending a single message whose size is larger than the batch size, the
+     * capacity will be the message size, but the size limit will still be the batch size), and when the records' size has
+     * exceed this limit we also mark this record as full.
      */
     public boolean hasRoomFor(byte[] key, byte[] value) {
-        return this.writable && this.capacity >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD +
-                                                 Record.recordSize(key, value);
+        return this.writable &&
+            this.capacity >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD + Record.recordSize(key, value) &&
+            this.sizeLimit >= this.compressor.estimatedBytesWritten();
     }
 
     public boolean isFull() {
-        return !this.writable || this.capacity <= this.compressor.estimatedBytesWritten();
+        return !this.writable ||
+            this.capacity <= this.compressor.estimatedBytesWritten() ||
+            this.sizeLimit <= this.compressor.estimatedBytesWritten();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
index 6a3cdcc..2f98192 100644
--- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
@@ -54,7 +54,7 @@ public class NetworkClientTest {
         client.poll(reqs, 1, time.milliseconds());
         selector.clear();
         assertFalse("After we forced the disconnection the client is no longer ready.", client.ready(node, time.milliseconds()));
-        assertTrue("Metadata should get updated.", metadata.needsUpdate(time.milliseconds()));
+        assertTrue("Metadata should get updated.", metadata.timeToNextUpdate(time.milliseconds()) == 0);
     }
 
     @Test(expected = IllegalStateException.class)

http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
index 8b4ac0f..0d7d04c 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
@@ -30,11 +30,11 @@ public class MetadataTest {
     public void testMetadata() throws Exception {
         long time = 0;
         metadata.update(Cluster.empty(), time);
-        assertFalse("No update needed.", metadata.needsUpdate(time));
+        assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0);
         metadata.forceUpdate();
-        assertFalse("Still no updated needed due to backoff", metadata.needsUpdate(time));
+        assertFalse("Still no updated needed due to backoff", metadata.timeToNextUpdate(time) == 0);
         time += refreshBackoffMs;
-        assertTrue("Update needed now that backoff time expired", metadata.needsUpdate(time));
+        assertTrue("Update needed now that backoff time expired", metadata.timeToNextUpdate(time) == 0);
         String topic = "my-topic";
         Thread t1 = asyncFetch(topic);
         Thread t2 = asyncFetch(topic);
@@ -43,9 +43,9 @@ public class MetadataTest {
         metadata.update(TestUtils.singletonCluster(topic, 1), time);
         t1.join();
         t2.join();
-        assertFalse("No update needed.", metadata.needsUpdate(time));
+        assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0);
         time += metadataExpireMs;
-        assertTrue("Update needed due to stale metadata.", metadata.needsUpdate(time));
+        assertTrue("Update needed due to stale metadata.", metadata.timeToNextUpdate(time) == 0);
     }
 
     private Thread asyncFetch(final String topic) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java
index 93b58d0..0762b35 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java
@@ -62,10 +62,10 @@ public class RecordAccumulatorTest {
         int appends = 1024 / msgSize;
         for (int i = 0; i < appends; i++) {
             accum.append(tp1, key, value, CompressionType.NONE, null);
-            assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).size());
+            assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size());
         }
         accum.append(tp1, key, value, CompressionType.NONE, null);
-        assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()));
+        assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()).readyNodes);
         List<RecordBatch> batches = accum.drain(cluster, Collections.singleton(node), Integer.MAX_VALUE, 0).get(node.id());
         assertEquals(1, batches.size());
         RecordBatch batch = batches.get(0);
@@ -83,7 +83,7 @@ public class RecordAccumulatorTest {
         int batchSize = 512;
         RecordAccumulator accum = new RecordAccumulator(batchSize, 10 * 1024, 0L, 100L, false, metrics, time);
         accum.append(tp1, key, new byte[2 * batchSize], CompressionType.NONE, null);
-        assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()));
+        assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()).readyNodes);
     }
 
     @Test
@@ -91,9 +91,9 @@ public class RecordAccumulatorTest {
         long lingerMs = 10L;
         RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, lingerMs, 100L, false, metrics, time);
         accum.append(tp1, key, value, CompressionType.NONE, null);
-        assertEquals("No partitions should be ready", 0, accum.ready(cluster, time.milliseconds()).size());
+        assertEquals("No partitions should be ready", 0, accum.ready(cluster, time.milliseconds()).readyNodes.size());
         time.sleep(10);
-        assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()));
+        assertEquals("Our partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()).readyNodes);
         List<RecordBatch> batches = accum.drain(cluster, Collections.singleton(node), Integer.MAX_VALUE, 0).get(node.id());
         assertEquals(1, batches.size());
         RecordBatch batch = batches.get(0);
@@ -113,7 +113,7 @@ public class RecordAccumulatorTest {
             for (int i = 0; i < appends; i++)
                 accum.append(tp, key, value, CompressionType.NONE, null);
         }
-        assertEquals("Partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()));
+        assertEquals("Partition's leader should be ready", Collections.singleton(node), accum.ready(cluster, time.milliseconds()).readyNodes);
 
         List<RecordBatch> batches = accum.drain(cluster, Collections.singleton(node), 1024, 0).get(node.id());
         assertEquals("But due to size bound only one partition should have been retrieved", 1, batches.size());
@@ -144,7 +144,7 @@ public class RecordAccumulatorTest {
         int read = 0;
         long now = time.milliseconds();
         while (read < numThreads * msgs) {
-            Set<Node> nodes = accum.ready(cluster, now);
+            Set<Node> nodes = accum.ready(cluster, now).readyNodes;
             List<RecordBatch> batches = accum.drain(cluster, nodes, 5 * 1024, 0).get(node.id());
             if (batches != null) {
                 for (RecordBatch batch : batches) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java
index 5489aca..ef2ca65 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java
@@ -69,7 +69,7 @@ public class SenderTest {
     @Test
     public void testSimple() throws Exception {
         int offset = 0;
-        Future<RecordMetadata> future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null);
+        Future<RecordMetadata> future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null).future;
         sender.run(time.milliseconds()); // connect
         sender.run(time.milliseconds()); // send produce request
         assertEquals("We should have a single produce request in flight.", 1, client.inFlightRequestCount());
@@ -95,7 +95,7 @@ public class SenderTest {
                                    new Metrics(),
                                    time);
         // do a successful retry
-        Future<RecordMetadata> future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null);
+        Future<RecordMetadata> future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null).future;
         sender.run(time.milliseconds()); // connect
         sender.run(time.milliseconds()); // send produce request
         assertEquals(1, client.inFlightRequestCount());
@@ -112,7 +112,7 @@ public class SenderTest {
         assertEquals(offset, future.get().offset());
 
         // do an unsuccessful retry
-        future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null);
+        future = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null).future;
         sender.run(time.milliseconds()); // send produce request
         for (int i = 0; i < maxRetries + 1; i++) {
             client.disconnect(client.requests().peek().request().destination());

http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/core/src/main/scala/kafka/tools/MirrorMaker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala
index 7638391..4f06e34 100644
--- a/core/src/main/scala/kafka/tools/MirrorMaker.scala
+++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala
@@ -21,22 +21,22 @@ import kafka.utils.{SystemTime, Utils, CommandLineUtils, Logging}
 import kafka.consumer._
 import kafka.serializer._
 import kafka.producer.{OldProducer, NewShinyProducer, BaseProducer}
+import kafka.metrics.KafkaMetricsGroup
+
 import org.apache.kafka.clients.producer.ProducerRecord
 
-import scala.collection.mutable.ListBuffer
 import scala.collection.JavaConversions._
 
-import java.util.concurrent.{TimeUnit, BlockingQueue, ArrayBlockingQueue, CountDownLatch}
-
 import joptsimple.OptionParser
-import kafka.metrics.KafkaMetricsGroup
-import com.yammer.metrics.core.Gauge
+import java.util.Random
+import java.util.concurrent.atomic.AtomicInteger
+import java.util.concurrent.{TimeUnit, BlockingQueue, ArrayBlockingQueue, CountDownLatch}
 
 object MirrorMaker extends Logging {
 
   private var connectors: Seq[ZookeeperConsumerConnector] = null
   private var consumerThreads: Seq[ConsumerThread] = null
-  private var producerThreads: ListBuffer[ProducerThread] = null
+  private var producerThreads: Seq[ProducerThread] = null
 
   private val shutdownMessage : ProducerRecord = new ProducerRecord("shutdown", "shutdown".getBytes)
 
@@ -138,13 +138,7 @@ object MirrorMaker extends Logging {
     // create a data channel btw the consumers and the producers
     val mirrorDataChannel = new DataChannel(bufferSize, numConsumers, numProducers)
 
-    producerThreads = new ListBuffer[ProducerThread]()
-    var producerIndex: Int = 1
-    for(producer <- producers) {
-      val producerThread = new ProducerThread(mirrorDataChannel, producer, producerIndex)
-      producerThreads += producerThread
-      producerIndex += 1
-    }
+    producerThreads = producers.zipWithIndex.map(producerAndIndex => new ProducerThread(mirrorDataChannel, producerAndIndex._1, producerAndIndex._2))
 
     val filterSpec = if (options.has(whitelistOpt))
       new Whitelist(options.valueOf(whitelistOpt))
@@ -190,14 +184,11 @@ object MirrorMaker extends Logging {
 
   class DataChannel(capacity: Int, numProducers: Int, numConsumers: Int) extends KafkaMetricsGroup {
 
-    val queue = new ArrayBlockingQueue[ProducerRecord](capacity)
+    val queues = new Array[BlockingQueue[ProducerRecord]](numConsumers)
+    for (i <- 0 until numConsumers)
+      queues(i) = new ArrayBlockingQueue[ProducerRecord](capacity)
 
-    newGauge(
-      "MirrorMaker-DataChannel-Size",
-      new Gauge[Int] {
-        def value = queue.size
-      }
-    )
+    private val counter = new AtomicInteger(new Random().nextInt())
 
     // We use a single meter for aggregated wait percentage for the data channel.
     // Since meter is calculated as total_recorded_value / time_window and
@@ -205,23 +196,37 @@ object MirrorMaker extends Logging {
     // time should be discounted by # threads.
     private val waitPut = newMeter("MirrorMaker-DataChannel-WaitOnPut", "percent", TimeUnit.NANOSECONDS)
     private val waitTake = newMeter("MirrorMaker-DataChannel-WaitOnTake", "percent", TimeUnit.NANOSECONDS)
+    private val channelSizeHist = newHistogram("MirrorMaker-DataChannel-Size")
 
     def put(record: ProducerRecord) {
+      // If the key of the message is empty, use round-robin to select the queue
+      // Otherwise use the queue based on the key value so that same key-ed messages go to the same queue
+      val queueId =
+        if(record.key() != null) {
+          Utils.abs(java.util.Arrays.hashCode(record.key())) % numConsumers
+        } else {
+          Utils.abs(counter.getAndIncrement()) % numConsumers
+        }
+      val queue = queues(queueId)
+
       var putSucceed = false
       while (!putSucceed) {
         val startPutTime = SystemTime.nanoseconds
         putSucceed = queue.offer(record, 500, TimeUnit.MILLISECONDS)
         waitPut.mark((SystemTime.nanoseconds - startPutTime) / numProducers)
       }
+      channelSizeHist.update(queue.size)
     }
 
-    def take(): ProducerRecord = {
+    def take(queueId: Int): ProducerRecord = {
+      val queue = queues(queueId)
       var data: ProducerRecord = null
       while (data == null) {
         val startTakeTime = SystemTime.nanoseconds
         data = queue.poll(500, TimeUnit.MILLISECONDS)
         waitTake.mark((SystemTime.nanoseconds - startTakeTime) / numConsumers)
       }
+      channelSizeHist.update(queue.size)
       data
     }
   }
@@ -242,18 +247,8 @@ object MirrorMaker extends Logging {
       info("Starting mirror maker consumer thread " + threadName)
       try {
         for (msgAndMetadata <- stream) {
-          // If the key of the message is empty, put it into the universal channel
-          // Otherwise use a pre-assigned producer to send the message
-          if (msgAndMetadata.key == null) {
-            trace("Send the non-keyed message the producer channel.")
-            val data = new ProducerRecord(msgAndMetadata.topic, msgAndMetadata.message)
-            mirrorDataChannel.put(data)
-          } else {
-            val producerId = Utils.abs(java.util.Arrays.hashCode(msgAndMetadata.key)) % producers.size()
-            trace("Send message with key %s to producer %d.".format(java.util.Arrays.toString(msgAndMetadata.key), producerId))
-            val producer = producers(producerId)
-            producer.send(msgAndMetadata.topic, msgAndMetadata.key, msgAndMetadata.message)
-          }
+          val data = new ProducerRecord(msgAndMetadata.topic, msgAndMetadata.message)
+          mirrorDataChannel.put(data)
         }
       } catch {
         case e: Throwable =>
@@ -287,7 +282,7 @@ object MirrorMaker extends Logging {
       info("Starting mirror maker producer thread " + threadName)
       try {
         while (true) {
-          val data: ProducerRecord = dataChannel.take
+          val data: ProducerRecord = dataChannel.take(threadId)
           trace("Sending message with value size %d".format(data.value().size))
           if(data eq shutdownMessage) {
             info("Received shutdown message")

http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
index d146444..15fd5bc 100644
--- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
@@ -21,7 +21,7 @@ import org.scalatest.junit.JUnit3Suite
 import org.junit.Test
 import org.junit.Assert._
 
-import java.util.Properties
+import java.util.{Random, Properties}
 import java.lang.Integer
 import java.util.concurrent.{TimeoutException, TimeUnit, ExecutionException}
 
@@ -76,8 +76,6 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
     producer1 = TestUtils.createNewProducer(brokerList, acks = 0, blockOnBufferFull = false, bufferSize = bufferSize);
     producer2 = TestUtils.createNewProducer(brokerList, acks = 1, blockOnBufferFull = false, bufferSize = bufferSize)
     producer3 = TestUtils.createNewProducer(brokerList, acks = -1, blockOnBufferFull = false, bufferSize = bufferSize)
-    // producer with incorrect broker list
-    producer4 = TestUtils.createNewProducer("localhost:8686,localhost:4242", acks = 1, blockOnBufferFull = false, bufferSize = bufferSize)
   }
 
   override def tearDown() {
@@ -150,6 +148,9 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
     // create topic
     TestUtils.createTopic(zkClient, topic1, 1, 2, servers)
 
+    // producer with incorrect broker list
+    producer4 = TestUtils.createNewProducer("localhost:8686,localhost:4242", acks = 1, blockOnBufferFull = false, bufferSize = bufferSize)
+
     // send a record with incorrect broker list
     val record = new ProducerRecord(topic1, null, "key".getBytes, "value".getBytes)
     intercept[ExecutionException] {
@@ -168,28 +169,32 @@ class ProducerFailureHandlingTest extends JUnit3Suite with ZooKeeperTestHarness
     TestUtils.createTopic(zkClient, topic1, 1, 2, servers)
 
     // first send a message to make sure the metadata is refreshed
-    val record = new ProducerRecord(topic1, null, "key".getBytes, "value".getBytes)
-    producer1.send(record).get
-    producer2.send(record).get
+    val record1 = new ProducerRecord(topic1, null, "key".getBytes, "value".getBytes)
+    producer1.send(record1).get
+    producer2.send(record1).get
 
     // stop IO threads and request handling, but leave networking operational
     // any requests should be accepted and queue up, but not handled
     server1.requestHandlerPool.shutdown()
     server2.requestHandlerPool.shutdown()
 
-    producer1.send(record).get(5000, TimeUnit.MILLISECONDS)
+    producer1.send(record1).get(5000, TimeUnit.MILLISECONDS)
 
     intercept[TimeoutException] {
-      producer2.send(record).get(5000, TimeUnit.MILLISECONDS)
+      producer2.send(record1).get(5000, TimeUnit.MILLISECONDS)
     }
 
     // TODO: expose producer configs after creating them
     // send enough messages to get buffer full
-    val tooManyRecords = bufferSize / ("key".getBytes.length + "value".getBytes.length)
+    val msgSize = 10000
+    val value = new Array[Byte](msgSize)
+    new Random().nextBytes(value)
+    val record2 = new ProducerRecord(topic1, null, "key".getBytes, value)
+    val tooManyRecords = bufferSize / ("key".getBytes.length + value.length)
 
     intercept[KafkaException] {
       for (i <- 1 to tooManyRecords)
-        producer2.send(record)
+        producer2.send(record2)
     }
 
     // do not close produce2 since it will block

http://git-wip-us.apache.org/repos/asf/kafka/blob/f1c6e97d/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
index 62fb02c..1c492de 100644
--- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
+++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
@@ -128,11 +128,15 @@ class SocketServerTest extends JUnitSuite {
 
   @Test(expected = classOf[IOException])
   def testSocketsCloseOnShutdown() {
-    // open a connection and then shutdown the server
+    // open a connection
     val socket = connect()
+    val bytes = new Array[Byte](40)
+    // send a request first to make sure the connection has been picked up by the socket server
+    sendRequest(socket, 0, bytes)
+    processRequest(server.requestChannel)
+    // then shutdown the server
     server.shutdown()
     // doing a subsequent send should throw an exception as the connection should be closed.
-    val bytes = new Array[Byte](10)
     sendRequest(socket, 0, bytes)
   }
 }