You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by jk...@apache.org on 2015/01/30 03:39:31 UTC

[1/7] kafka git commit: KAFKA-1760: New consumer.

Repository: kafka
Updated Branches:
  refs/heads/trunk 11ec9bf5a -> 0699ff2ce


http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 cd16ced..a1f72f8 100644
--- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
+++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
@@ -192,7 +192,7 @@ object SerializationTestUtils {
   }
 
   def createConsumerMetadataResponse: ConsumerMetadataResponse = {
-    ConsumerMetadataResponse(Some(brokers.head), ErrorMapping.NoError)
+    ConsumerMetadataResponse(Some(brokers.head), ErrorMapping.NoError, 0)
   }
 
   def createHeartbeatRequestAndHeader: HeartbeatRequestAndHeader = {
@@ -237,7 +237,7 @@ class RequestResponseSerializationTest extends JUnitSuite {
   private val offsetFetchResponse = SerializationTestUtils.createTestOffsetFetchResponse
   private val consumerMetadataRequest = SerializationTestUtils.createConsumerMetadataRequest
   private val consumerMetadataResponse = SerializationTestUtils.createConsumerMetadataResponse
-  private val consumerMetadataResponseNoCoordinator = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode)
+  private val consumerMetadataResponseNoCoordinator = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, 0)
   private val heartbeatRequest = SerializationTestUtils.createHeartbeatRequestAndHeader
   private val heartbeatResponse = SerializationTestUtils.createHeartbeatResponseAndHeader
   private val joinGroupRequest = SerializationTestUtils.createJoinGroupRequestAndHeader

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 3cf7c9b..ef4c9ae 100644
--- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
+++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
@@ -17,11 +17,14 @@
 
 package kafka.integration
 
+import java.util.Arrays
+import scala.collection.mutable.Buffer
 import kafka.server._
 import kafka.utils.{Utils, TestUtils}
 import org.scalatest.junit.JUnit3Suite
 import kafka.zk.ZooKeeperTestHarness
 import kafka.common.KafkaException
+import kafka.utils.TestUtils
 
 /**
  * A test harness that brings up some number of broker nodes
@@ -29,15 +32,22 @@ import kafka.common.KafkaException
 trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness {
 
   val configs: List[KafkaConfig]
-  var servers: List[KafkaServer] = null
+  var servers: Buffer[KafkaServer] = null
   var brokerList: String = null
-
+  var alive: Array[Boolean] = null
+  
+  def serverForId(id: Int) = servers.find(s => s.config.brokerId == id)
+  
+  def bootstrapUrl = configs.map(c => c.hostName + ":" + c.port).mkString(",")
+  
   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(_))
+    servers = configs.map(TestUtils.createServer(_)).toBuffer
+    alive = new Array[Boolean](servers.length)
+    Arrays.fill(alive, true)
   }
 
   override def tearDown() {
@@ -45,4 +55,27 @@ trait KafkaServerTestHarness extends JUnit3Suite with ZooKeeperTestHarness {
     servers.map(server => server.config.logDirs.map(Utils.rm(_)))
     super.tearDown
   }
+  
+  /**
+   * Pick a broker at random and kill it if it isn't already dead
+   * Return the id of the broker killed
+   */
+  def killRandomBroker(): Int = {
+    val index = TestUtils.random.nextInt(servers.length)
+    if(alive(index)) {
+      servers(index).shutdown()
+      alive(index) = false
+    }
+    index
+  }
+  
+  /**
+   * Restart any dead brokers
+   */
+  def restartDeadBrokers() {
+    for(i <- 0 until servers.length if !alive(i)) {
+      servers(i) = TestUtils.createServer(configs(i))
+      alive(i) = true
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala
index a5386a0..aeb7a19 100644
--- a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala
+++ b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala
@@ -32,6 +32,7 @@ import kafka.common.{TopicAndPartition, ErrorMapping, UnknownTopicOrPartitionExc
 import kafka.utils.{StaticPartitioner, TestUtils, Utils}
 import kafka.serializer.StringEncoder
 import java.util.Properties
+import TestUtils._
 
 /**
  * End to end tests of the primitive apis against a local server
@@ -113,7 +114,8 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
   }
 
   private def produceAndMultiFetch(producer: Producer[String, String]) {
-    createSimpleTopicsAndAwaitLeader(zkClient, List("test1", "test2", "test3", "test4"))
+    for(topic <- List("test1", "test2", "test3", "test4"))
+      TestUtils.createTopic(zkClient, topic, servers = servers)
 
     // send some messages
     val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
@@ -181,7 +183,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
 
   private def multiProduce(producer: Producer[String, String]) {
     val topics = Map("test4" -> 0, "test1" -> 0, "test2" -> 0, "test3" -> 0)
-    createSimpleTopicsAndAwaitLeader(zkClient, topics.keys)
+    topics.keys.map(topic => TestUtils.createTopic(zkClient, topic, servers = servers))
 
     val messages = new mutable.HashMap[String, Seq[String]]
     val builder = new FetchRequestBuilder()
@@ -215,7 +217,7 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
 
   def testPipelinedProduceRequests() {
     val topics = Map("test4" -> 0, "test1" -> 0, "test2" -> 0, "test3" -> 0)
-    createSimpleTopicsAndAwaitLeader(zkClient, topics.keys)
+    topics.keys.map(topic => TestUtils.createTopic(zkClient, topic, servers = servers))
     val props = new Properties()
     props.put("request.required.acks", "0")
     val pipelinedProducer: Producer[String, String] =
@@ -265,15 +267,4 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
       assertEquals(messages(topic), fetched.map(messageAndOffset => Utils.readString(messageAndOffset.message.payload)))
     }
   }
-
-  /**
-   * For testing purposes, just create these topics each with one partition and one replica for
-   * which the provided broker should the leader for.  Create and wait for broker to lead.  Simple.
-   */
-  private def createSimpleTopicsAndAwaitLeader(zkClient: ZkClient, topics: Iterable[String]) {
-    for( topic <- topics ) {
-      AdminUtils.createTopic(zkClient, topic, partitions = 1, replicationFactor = 1)
-      TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, partition = 0)
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/core/src/test/scala/unit/kafka/utils/MockScheduler.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/MockScheduler.scala b/core/src/test/scala/unit/kafka/utils/MockScheduler.scala
index d5896ed..c674078 100644
--- a/core/src/test/scala/unit/kafka/utils/MockScheduler.scala
+++ b/core/src/test/scala/unit/kafka/utils/MockScheduler.scala
@@ -36,6 +36,8 @@ class MockScheduler(val time: Time) extends Scheduler {
   
   /* a priority queue of tasks ordered by next execution time */
   var tasks = new PriorityQueue[MockTask]()
+  
+  def isStarted = true
 
   def startup() {}
   

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala
index b364ac2..cfea63b 100644
--- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala
+++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala
@@ -90,4 +90,21 @@ class SchedulerTest {
       assertTrue("Should count to 20", counter1.get >= 20)
     }
   }
+  
+  @Test
+  def testRestart() {
+    // schedule a task to increment a counter
+    mockTime.scheduler.schedule("test1", counter1.getAndIncrement, delay=1)
+    mockTime.sleep(1)
+    assertEquals(1, counter1.get())
+    
+    // restart the scheduler
+    mockTime.scheduler.shutdown()
+    mockTime.scheduler.startup()
+    
+    // schedule another task to increment the counter
+    mockTime.scheduler.schedule("test1", counter1.getAndIncrement, delay=1)
+    mockTime.sleep(1)
+    assertEquals(2, counter1.get())
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 ac15d34..54755e8 100644
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -45,6 +45,7 @@ import kafka.log._
 import junit.framework.AssertionFailedError
 import junit.framework.Assert._
 import org.apache.kafka.clients.producer.KafkaProducer
+import collection.Iterable
 
 import scala.collection.Map
 
@@ -709,24 +710,21 @@ object TestUtils extends Logging {
   /**
    * 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())
+  def createLogManager(logDirs: Array[File] = Array.empty[File],
+                       defaultConfig: LogConfig = LogConfig(),
+                       cleanerConfig: CleanerConfig = CleanerConfig(enableCleaner = false),
+                       time: MockTime = new MockTime()): LogManager = {
+    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())
   }
 
   def sendMessagesToPartition(configs: Seq[KafkaConfig],


[6/7] kafka git commit: KAFKA-1760: New consumer.

Posted by jk...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java
index bdf4b26..416d703 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java
@@ -9,53 +9,98 @@
  * 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.clients.consumer;
 
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
+
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.AbstractIterator;
 
 /**
- * A container that holds the list {@link ConsumerRecord} per partition for a particular topic. There is one for every topic returned by a 
- * {@link Consumer#poll(long)} operation. 
+ * A container that holds the list {@link ConsumerRecord} per partition for a
+ * particular topic. There is one for every topic returned by a
+ * {@link Consumer#poll(long)} operation.
  */
-public class ConsumerRecords<K,V> {
+public class ConsumerRecords<K,V> implements Iterable<ConsumerRecord<K,V>> {
 
-    private final String topic;
-    private final Map<Integer, List<ConsumerRecord<K,V>>> recordsPerPartition;
-    
-    public ConsumerRecords(String topic, Map<Integer, List<ConsumerRecord<K,V>>> records) {
-        this.topic = topic;
-        this.recordsPerPartition = records;
+    private final Map<TopicPartition, List<ConsumerRecord<K,V>>> records;
+
+    public ConsumerRecords(Map<TopicPartition, List<ConsumerRecord<K,V>>> records) {
+        this.records = records;
     }
-    
+
     /**
-     * @param partitions The input list of partitions for a particular topic. If no partitions are 
-     * specified, returns records for all partitions
-     * @return The list of {@link ConsumerRecord}s associated with the given partitions.
+     * Get just the records for the given partition
+     * 
+     * @param partition The partition to get records for
      */
-    public List<ConsumerRecord<K,V>> records(int... partitions) {
-        List<ConsumerRecord<K,V>> recordsToReturn = new ArrayList<ConsumerRecord<K,V>>();
-        if(partitions.length == 0) {
-            // return records for all partitions
-            for(Entry<Integer, List<ConsumerRecord<K,V>>> record : recordsPerPartition.entrySet()) {
-                recordsToReturn.addAll(record.getValue());
-            }
-        } else {
-           for(int partition : partitions) {
-               List<ConsumerRecord<K,V>> recordsForThisPartition = recordsPerPartition.get(partition);
-               recordsToReturn.addAll(recordsForThisPartition);
-           }
+    public Iterable<ConsumerRecord<K,V>> records(TopicPartition partition) {
+        List<ConsumerRecord<K,V>> recs = this.records.get(partition);
+        if (recs == null)
+            return Collections.emptyList();
+        else
+            return recs;
+    }
+
+    /**
+     * Get just the records for the given topic
+     */
+    public Iterable<ConsumerRecord<K,V>> records(String topic) {
+        if (topic == null)
+            throw new IllegalArgumentException("Topic must be non-null.");
+        List<List<ConsumerRecord<K,V>>> recs = new ArrayList<List<ConsumerRecord<K,V>>>();
+        for (Map.Entry<TopicPartition, List<ConsumerRecord<K,V>>> entry : records.entrySet()) {
+            if (entry.getKey().equals(topic))
+                recs.add(entry.getValue());
         }
-        return recordsToReturn;
+        return new ConcatenatedIterable<K,V>(recs);
     }
 
+    @Override
+    public Iterator<ConsumerRecord<K,V>> iterator() {
+        return new ConcatenatedIterable<K,V>(records.values()).iterator();
+    }
+    
     /**
-     * @return The topic of all records associated with this instance
+     * The number of records for all topics
      */
-    public String topic() {
-        return this.topic;
+    public int count() {
+        int count = 0;
+        for(List<ConsumerRecord<K,V>> recs: this.records.values())
+            count += recs.size();
+        return count;
+    }
+
+    private static class ConcatenatedIterable<K,V> implements Iterable<ConsumerRecord<K,V>> {
+
+        private final Iterable<? extends Iterable<ConsumerRecord<K,V>>> iterables;
+
+        public ConcatenatedIterable(Iterable<? extends Iterable<ConsumerRecord<K,V>>> iterables) {
+            this.iterables = iterables;
+        }
+
+        @Override
+        public Iterator<ConsumerRecord<K,V>> iterator() {
+            return new AbstractIterator<ConsumerRecord<K,V>>() {
+                Iterator<? extends Iterable<ConsumerRecord<K,V>>> iters = iterables.iterator();
+                Iterator<ConsumerRecord<K,V>> current;
+
+                public ConsumerRecord<K,V> makeNext() {
+                    if (current == null || !current.hasNext()) {
+                        if (iters.hasNext())
+                            current = iters.next().iterator();
+                        else
+                            return allDone();
+                    }
+                    return current.next();
+                }
+            };
+        }
     }
+
 }


[3/7] kafka git commit: KAFKA-1760: New consumer.

Posted by jk...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 121e880..ee1f78f 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
@@ -1,18 +1,14 @@
 /**
- * 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.
+ * 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.protocol.types;
 
@@ -124,14 +120,6 @@ public class Struct {
         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);
     }
@@ -148,6 +136,14 @@ public class Struct {
         return (String) get(name);
     }
 
+    public ByteBuffer getBytes(Field field) {
+        return (ByteBuffer) get(field);
+    }
+
+    public ByteBuffer getBytes(String name) {
+        return (ByteBuffer) get(name);
+    }
+
     /**
      * Set the given field to the specified value
      * 
@@ -175,9 +171,9 @@ public class Struct {
     }
 
     /**
-     * Create a struct for the schema of a container type (struct or array).
-     * Note that for array type, this method assumes that the type is an array of schema and creates a struct
-     * of that schema. Arrays of other types can't be instantiated with this method.
+     * Create a struct for the schema of a container type (struct or array). Note that for array type, this method
+     * assumes that the type is an array of schema and creates a struct of that schema. Arrays of other types can't be
+     * instantiated with this method.
      * 
      * @param field The field to create an instance of
      * @return The struct

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java b/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java
index e4d688c..2e54b56 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java
@@ -41,4 +41,8 @@ public final class LogEntry {
     public String toString() {
         return "LogEntry(" + offset + ", " + record + ")";
     }
+    
+    public int size() {
+        return record.size() + Records.LOG_OVERHEAD;
+    }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 040e5b9..cc4084f 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
@@ -55,7 +55,7 @@ public class MemoryRecords implements Records {
         return emptyRecords(buffer, type, buffer.capacity());
     }
 
-    public static MemoryRecords iterableRecords(ByteBuffer buffer) {
+    public static MemoryRecords readableRecords(ByteBuffer buffer) {
         return new MemoryRecords(buffer, CompressionType.NONE, false, buffer.capacity());
     }
 
@@ -94,22 +94,21 @@ 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.
+     * 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) &&
-            this.sizeLimit >= this.compressor.estimatedBytesWritten();
+        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() ||
-            this.sizeLimit <= this.compressor.estimatedBytesWritten();
+        return !this.writable || this.capacity <= this.compressor.estimatedBytesWritten() ||
+               this.sizeLimit <= this.compressor.estimatedBytesWritten();
     }
 
     /**
@@ -132,7 +131,7 @@ public class MemoryRecords implements Records {
     public int sizeInBytes() {
         return compressor.buffer().position();
     }
-    
+
     /**
      * The compression rate of this record set
      */
@@ -162,6 +161,25 @@ public class MemoryRecords implements Records {
         ByteBuffer copy = (ByteBuffer) this.buffer.duplicate().flip();
         return new RecordsIterator(copy, CompressionType.NONE, false);
     }
+    
+    @Override
+    public String toString() {
+      Iterator<LogEntry> iter = iterator();
+      StringBuilder builder = new StringBuilder();
+      builder.append('[');
+      while(iter.hasNext()) {
+        LogEntry entry = iter.next();
+        builder.append('(');
+        builder.append("offset=");
+        builder.append(entry.offset());
+        builder.append(",");
+        builder.append("record=");
+        builder.append(entry.record());
+        builder.append(")");
+      }
+      builder.append(']');
+      return builder.toString();
+    }
 
     public static class RecordsIterator extends AbstractIterator<LogEntry> {
         private final ByteBuffer buffer;
@@ -174,7 +192,7 @@ public class MemoryRecords implements Records {
             this.type = type;
             this.buffer = buffer;
             this.shallow = shallow;
-            stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type);
+            this.stream = Compressor.wrapForInput(new ByteBufferInputStream(this.buffer), type);
         }
 
         /*
@@ -199,7 +217,10 @@ public class MemoryRecords implements Records {
                     ByteBuffer rec;
                     if (type == CompressionType.NONE) {
                         rec = buffer.slice();
-                        buffer.position(buffer.position() + size);
+                        int newPos = buffer.position() + size;
+                        if(newPos > buffer.limit())
+                          return allDone();
+                        buffer.position(newPos);
                         rec.limit(size);
                     } else {
                         byte[] recordBuffer = new byte[size];
@@ -207,7 +228,6 @@ public class MemoryRecords implements Records {
                         rec = ByteBuffer.wrap(recordBuffer);
                     }
                     LogEntry entry = new LogEntry(offset, new Record(rec));
-                    entry.record().ensureValid();
 
                     // decide whether to go shallow or deep iteration if it is compressed
                     CompressionType compression = entry.record().compressionType();

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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
index 99b52c2..4c99d4a 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java
@@ -20,13 +20,14 @@ 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 static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.CONSUMER_METADATA.id);
+    private static final String GROUP_ID_KEY_NAME = "group_id";
 
     private final String groupId;
 
     public ConsumerMetadataRequest(String groupId) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
 
         struct.set(GROUP_ID_KEY_NAME, groupId);
         this.groupId = groupId;
@@ -42,6 +43,6 @@ public class ConsumerMetadataRequest extends AbstractRequestResponse {
     }
 
     public static ConsumerMetadataRequest parse(ByteBuffer buffer) {
-        return new ConsumerMetadataRequest(((Struct) curSchema.read(buffer)));
+        return new ConsumerMetadataRequest(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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
index 8b8f591..173333b 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java
@@ -21,20 +21,21 @@ 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";
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.CONSUMER_METADATA.id);
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
+    private static final 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 static final String NODE_ID_KEY_NAME = "node_id";
+    private static final String HOST_KEY_NAME = "host";
+    private static final String PORT_KEY_NAME = "port";
 
     private final short errorCode;
     private final Node node;
 
     public ConsumerMetadataResponse(short errorCode, Node node) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
         struct.set(ERROR_CODE_KEY_NAME, errorCode);
         Struct coordinator = struct.instance(COORDINATOR_KEY_NAME);
         coordinator.set(NODE_ID_KEY_NAME, node.id());
@@ -64,6 +65,6 @@ public class ConsumerMetadataResponse extends AbstractRequestResponse {
     }
 
     public static ConsumerMetadataResponse parse(ByteBuffer buffer) {
-        return new ConsumerMetadataResponse(((Struct) curSchema.read(buffer)));
+        return new ConsumerMetadataResponse(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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
index 2fc471f..2529a09 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java
@@ -1,21 +1,23 @@
 /**
- * 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.
+ * 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 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;
@@ -23,27 +25,23 @@ 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";
+    
+    public static final int CONSUMER_REPLICA_ID = -1;
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.FETCH.id);
+    private static final String REPLICA_ID_KEY_NAME = "replica_id";
+    private static final String MAX_WAIT_KEY_NAME = "max_wait_time";
+    private static final String MIN_BYTES_KEY_NAME = "min_bytes";
+    private static final String TOPICS_KEY_NAME = "topics";
 
     // topic level field names
-    private static String TOPIC_KEY_NAME = "topic";
-    private static String PARTITIONS_KEY_NAME = "partitions";
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final 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 static final String PARTITION_KEY_NAME = "partition";
+    private static final String FETCH_OFFSET_KEY_NAME = "fetch_offset";
+    private static final String MAX_BYTES_KEY_NAME = "max_bytes";
 
     private final int replicaId;
     private final int maxWait;
@@ -60,15 +58,25 @@ public class FetchRequest extends AbstractRequestResponse {
         }
     }
 
+    /**
+     * Create a non-replica fetch request
+     */
+    public FetchRequest(int maxWait, int minBytes, Map<TopicPartition, PartitionData> fetchData) {
+        this(CONSUMER_REPLICA_ID, maxWait, minBytes, fetchData);
+    }
+
+    /**
+     * Create a replica fetch request
+     */
     public FetchRequest(int replicaId, int maxWait, int minBytes, Map<TopicPartition, PartitionData> fetchData) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
         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()) {
+        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>();
@@ -127,6 +135,6 @@ public class FetchRequest extends AbstractRequestResponse {
     }
 
     public static FetchRequest parse(ByteBuffer buffer) {
-        return new FetchRequest(((Struct) curSchema.read(buffer)));
+        return new FetchRequest(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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
index f719010..c1e5f44 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java
@@ -30,18 +30,19 @@ 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";
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.FETCH.id);
+    private static final String RESPONSES_KEY_NAME = "responses";
 
     // topic level field names
-    private static String TOPIC_KEY_NAME = "topic";
-    private static String PARTITIONS_KEY_NAME = "partition_responses";
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final 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 static final String PARTITION_KEY_NAME = "partition";
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
+    private static final String HIGH_WATERMARK_KEY_NAME = "high_watermark";
+    private static final String RECORD_SET_KEY_NAME = "record_set";
 
     private final Map<TopicPartition, PartitionData> responseData;
 
@@ -58,7 +59,7 @@ public class FetchResponse extends AbstractRequestResponse {
     }
 
     public FetchResponse(Map<TopicPartition, PartitionData> responseData) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
         Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
 
         List<Struct> topicArray = new ArrayList<Struct>();
@@ -105,6 +106,6 @@ public class FetchResponse extends AbstractRequestResponse {
     }
 
     public static FetchResponse parse(ByteBuffer buffer) {
-        return new FetchResponse(((Struct) curSchema.read(buffer)));
+        return new FetchResponse(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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
index 9512db2..cfdb5de 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatRequest.java
@@ -20,17 +20,18 @@ 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 static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.HEARTBEAT.id);
+    private static final String GROUP_ID_KEY_NAME = "group_id";
+    private static final String GROUP_GENERATION_ID_KEY_NAME = "group_generation_id";
+    private static final 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));
+        super(new Struct(CURRENT_SCHEMA));
         struct.set(GROUP_ID_KEY_NAME, groupId);
         struct.set(GROUP_GENERATION_ID_KEY_NAME, groupGenerationId);
         struct.set(CONSUMER_ID_KEY_NAME, consumerId);
@@ -59,6 +60,6 @@ public class HeartbeatRequest extends AbstractRequestResponse {
     }
 
     public static HeartbeatRequest parse(ByteBuffer buffer) {
-        return new HeartbeatRequest(((Struct) curSchema.read(buffer)));
+        return new HeartbeatRequest(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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
index 8997ffc..ea964f7 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/HeartbeatResponse.java
@@ -20,12 +20,13 @@ 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 static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.HEARTBEAT.id);
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
 
     private final short errorCode;
     public HeartbeatResponse(short errorCode) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
         struct.set(ERROR_CODE_KEY_NAME, errorCode);
         this.errorCode = errorCode;
     }
@@ -40,6 +41,6 @@ public class HeartbeatResponse extends AbstractRequestResponse {
     }
 
     public static HeartbeatResponse parse(ByteBuffer buffer) {
-        return new HeartbeatResponse(((Struct) curSchema.read(buffer)));
+        return new HeartbeatResponse(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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
index d6e91f3..a1d48c9 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupRequest.java
@@ -22,12 +22,13 @@ 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 static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.JOIN_GROUP.id);
+    private static final String GROUP_ID_KEY_NAME = "group_id";
+    private static final String SESSION_TIMEOUT_KEY_NAME = "session_timeout";
+    private static final String TOPICS_KEY_NAME = "topics";
+    private static final String CONSUMER_ID_KEY_NAME = "consumer_id";
+    private static final String STRATEGY_KEY_NAME = "partition_assignment_strategy";
 
     private final String groupId;
     private final int sessionTimeout;
@@ -36,7 +37,7 @@ public class JoinGroupRequest extends AbstractRequestResponse {
     private final String strategy;
 
     public JoinGroupRequest(String groupId, int sessionTimeout, List<String> topics, String consumerId, String strategy) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
         struct.set(GROUP_ID_KEY_NAME, groupId);
         struct.set(SESSION_TIMEOUT_KEY_NAME, sessionTimeout);
         struct.set(TOPICS_KEY_NAME, topics.toArray());
@@ -82,6 +83,6 @@ public class JoinGroupRequest extends AbstractRequestResponse {
     }
 
     public static JoinGroupRequest parse(ByteBuffer buffer) {
-        return new JoinGroupRequest(((Struct) curSchema.read(buffer)));
+        return new JoinGroupRequest(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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
index efe8979..1e9f349 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/JoinGroupResponse.java
@@ -23,16 +23,17 @@ 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 static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.JOIN_GROUP.id);
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
+    private static final String GENERATION_ID_KEY_NAME = "group_generation_id";
+    private static final String CONSUMER_ID_KEY_NAME = "consumer_id";
+    private static final String ASSIGNED_PARTITIONS_KEY_NAME = "assigned_partitions";
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String PARTITIONS_KEY_NAME = "partitions";
+
+    public static final int UNKNOWN_GENERATION_ID = -1;
+    public static final String UNKNOWN_CONSUMER_ID = "";
 
     private final short errorCode;
     private final int generationId;
@@ -40,7 +41,7 @@ public class JoinGroupResponse extends AbstractRequestResponse {
     private final List<TopicPartition> assignedPartitions;
 
     public JoinGroupResponse(short errorCode, int generationId, String consumerId, List<TopicPartition> assignedPartitions) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
 
         Map<String, List<Integer>> partitionsByTopic = CollectionUtils.groupDataByTopic(assignedPartitions);
 
@@ -97,6 +98,6 @@ public class JoinGroupResponse extends AbstractRequestResponse {
     }
 
     public static JoinGroupResponse parse(ByteBuffer buffer) {
-        return new JoinGroupResponse(((Struct) curSchema.read(buffer)));
+        return new JoinGroupResponse(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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
index 99364c1..05c5fed 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java
@@ -30,18 +30,19 @@ 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";
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.LIST_OFFSETS.id);
+    private static final String REPLICA_ID_KEY_NAME = "replica_id";
+    private static final String TOPICS_KEY_NAME = "topics";
 
     // topic level field names
-    private static String TOPIC_KEY_NAME = "topic";
-    private static String PARTITIONS_KEY_NAME = "partitions";
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final 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 static final String PARTITION_KEY_NAME = "partition";
+    private static final String TIMESTAMP_KEY_NAME = "timestamp";
+    private static final String MAX_NUM_OFFSETS_KEY_NAME = "max_num_offsets";
 
     private final int replicaId;
     private final Map<TopicPartition, PartitionData> offsetData;
@@ -55,9 +56,13 @@ public class ListOffsetRequest extends AbstractRequestResponse {
             this.maxNumOffsets = maxNumOffsets;
         }
     }
+    
+    public ListOffsetRequest(Map<TopicPartition, PartitionData> offsetData) {
+    	this(-1, offsetData);
+    }
 
     public ListOffsetRequest(int replicaId, Map<TopicPartition, PartitionData> offsetData) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
         Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(offsetData);
 
         struct.set(REPLICA_ID_KEY_NAME, replicaId);
@@ -109,6 +114,6 @@ public class ListOffsetRequest extends AbstractRequestResponse {
     }
 
     public static ListOffsetRequest parse(ByteBuffer buffer) {
-        return new ListOffsetRequest(((Struct) curSchema.read(buffer)));
+        return new ListOffsetRequest(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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
index ac23971..b2e473e 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetResponse.java
@@ -30,17 +30,18 @@ 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";
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.LIST_OFFSETS.id);
+    private static final String RESPONSES_KEY_NAME = "responses";
 
     // topic level field names
-    private static String TOPIC_KEY_NAME = "topic";
-    private static String PARTITIONS_KEY_NAME = "partition_responses";
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final 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 static final String PARTITION_KEY_NAME = "partition";
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
+    private static final String OFFSETS_KEY_NAME = "offsets";
 
     private final Map<TopicPartition, PartitionData> responseData;
 
@@ -55,7 +56,7 @@ public class ListOffsetResponse extends AbstractRequestResponse {
     }
 
     public ListOffsetResponse(Map<TopicPartition, PartitionData> responseData) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
         Map<String, Map<Integer, PartitionData>> topicsData = CollectionUtils.groupDataByTopic(responseData);
 
         List<Struct> topicArray = new ArrayList<Struct>();
@@ -103,6 +104,6 @@ public class ListOffsetResponse extends AbstractRequestResponse {
     }
 
     public static ListOffsetResponse parse(ByteBuffer buffer) {
-        return new ListOffsetResponse(((Struct) curSchema.read(buffer)));
+        return new ListOffsetResponse(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 b22ca1d..0186783 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
@@ -22,13 +22,14 @@ import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 public class MetadataRequest extends AbstractRequestResponse {
-    public static Schema curSchema = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id);
-    private static String TOPICS_KEY_NAME = "topics";
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id);
+    private static final String TOPICS_KEY_NAME = "topics";
 
     private final List<String> topics;
 
     public MetadataRequest(List<String> topics) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
         struct.set(TOPICS_KEY_NAME, topics.toArray());
         this.topics = topics;
     }
@@ -47,6 +48,6 @@ public class MetadataRequest extends AbstractRequestResponse {
     }
 
     public static MetadataRequest parse(ByteBuffer buffer) {
-        return new MetadataRequest(((Struct) curSchema.read(buffer)));
+        return new MetadataRequest(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 d97962d..13daf59 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
@@ -28,32 +28,33 @@ import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
 
 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";
+    
+    private static Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id);
+    private static final String BROKERS_KEY_NAME = "brokers";
+    private static final 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";
+    private static final String NODE_ID_KEY_NAME = "node_id";
+    private static final String HOST_KEY_NAME = "host";
+    private static final 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";
+    private static final String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code";
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final 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 static final String PARTITION_ERROR_CODE_KEY_NAME = "partition_error_code";
+    private static final String PARTITION_KEY_NAME = "partition_id";
+    private static final String LEADER_KEY_NAME = "leader";
+    private static final String REPLICAS_KEY_NAME = "replicas";
+    private static final String ISR_KEY_NAME = "isr";
 
     private final Cluster cluster;
     private final Map<String, Errors> errors;
 
     public MetadataResponse(Cluster cluster) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
 
         List<Struct> brokerArray = new ArrayList<Struct>();
         for (Node node: cluster.nodes()) {
@@ -147,6 +148,6 @@ public class MetadataResponse extends AbstractRequestResponse {
     }
 
     public static MetadataResponse parse(ByteBuffer buffer) {
-        return new MetadataResponse(((Struct) curSchema.read(buffer)));
+        return new MetadataResponse(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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
index 3ee5cba..4fb48c8 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitRequest.java
@@ -3,15 +3,21 @@
  * 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 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;
@@ -19,31 +25,26 @@ 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";
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_COMMIT.id);
+    private static final String GROUP_ID_KEY_NAME = "group_id";
+    private static final String GENERATION_ID_KEY_NAME = "group_generation_id";
+    private static final String CONSUMER_ID_KEY_NAME = "consumer_id";
+    private static final String TOPICS_KEY_NAME = "topics";
 
     // topic level field names
-    private static String TOPIC_KEY_NAME = "topic";
-    private static String PARTITIONS_KEY_NAME = "partitions";
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final 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";
+    private static final String PARTITION_KEY_NAME = "partition";
+    private static final String COMMIT_OFFSET_KEY_NAME = "offset";
+    private static final String TIMESTAMP_KEY_NAME = "timestamp";
+    private static final String METADATA_KEY_NAME = "metadata";
 
     public static final int DEFAULT_GENERATION_ID = -1;
     public static final String DEFAULT_CONSUMER_ID = "";
@@ -88,7 +89,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse {
      * @param offsetData
      */
     public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map<TopicPartition, PartitionData> offsetData) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
 
         initCommonFields(groupId, offsetData);
         struct.set(GENERATION_ID_KEY_NAME, generationId);
@@ -104,7 +105,7 @@ public class OffsetCommitRequest extends AbstractRequestResponse {
 
         struct.set(GROUP_ID_KEY_NAME, groupId);
         List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, Map<Integer, PartitionData>> topicEntry: topicsData.entrySet()) {
+        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>();
@@ -175,6 +176,6 @@ public class OffsetCommitRequest extends AbstractRequestResponse {
     }
 
     public static OffsetCommitRequest parse(ByteBuffer buffer) {
-        return new OffsetCommitRequest(((Struct) curSchema.read(buffer)));
+        return new OffsetCommitRequest(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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
index 711232a..2ab1dc6 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java
@@ -26,21 +26,22 @@ 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";
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_COMMIT.id);
+    private static final String RESPONSES_KEY_NAME = "responses";
 
     // topic level fields
-    private static String TOPIC_KEY_NAME = "topic";
-    private static String PARTITIONS_KEY_NAME = "partition_responses";
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final 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 static final String PARTITION_KEY_NAME = "partition";
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
 
     private final Map<TopicPartition, Short> responseData;
 
     public OffsetCommitResponse(Map<TopicPartition, Short> responseData) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
 
         Map<String, Map<Integer, Short>> topicsData = CollectionUtils.groupDataByTopic(responseData);
 
@@ -82,6 +83,6 @@ public class OffsetCommitResponse extends AbstractRequestResponse {
     }
 
     public static OffsetCommitResponse parse(ByteBuffer buffer) {
-        return new OffsetCommitResponse(((Struct) curSchema.read(buffer)));
+        return new OffsetCommitResponse(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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
index 90d5135..333483f 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchRequest.java
@@ -28,16 +28,17 @@ 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";
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.OFFSET_FETCH.id);
+    private static final String GROUP_ID_KEY_NAME = "group_id";
+    private static final String TOPICS_KEY_NAME = "topics";
 
     // topic level field names
-    private static String TOPIC_KEY_NAME = "topic";
-    private static String PARTITIONS_KEY_NAME = "partitions";
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String PARTITIONS_KEY_NAME = "partitions";
 
     // partition level field names
-    private static String PARTITION_KEY_NAME = "partition";
+    private static final String PARTITION_KEY_NAME = "partition";
 
     public static final int DEFAULT_GENERATION_ID = -1;
     public static final String DEFAULT_CONSUMER_ID = "";
@@ -46,7 +47,7 @@ public class OffsetFetchRequest extends AbstractRequestResponse {
     private final List<TopicPartition> partitions;
 
     public OffsetFetchRequest(String groupId, List<TopicPartition> partitions) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
 
         Map<String, List<Integer>> topicsData = CollectionUtils.groupDataByTopic(partitions);
 
@@ -93,6 +94,6 @@ public class OffsetFetchRequest extends AbstractRequestResponse {
     }
 
     public static OffsetFetchRequest parse(ByteBuffer buffer) {
-        return new OffsetFetchRequest(((Struct) curSchema.read(buffer)));
+        return new OffsetFetchRequest(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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
index 6b7c269..04c88c0 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java
@@ -3,43 +3,45 @@
  * 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 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.Errors;
 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";
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.OFFSET_FETCH.id);
+    private static final String RESPONSES_KEY_NAME = "responses";
 
     // topic level fields
-    private static String TOPIC_KEY_NAME = "topic";
-    private static String PARTITIONS_KEY_NAME = "partition_responses";
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final 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 static final String PARTITION_KEY_NAME = "partition";
+    private static final String COMMIT_OFFSET_KEY_NAME = "offset";
+    private static final String METADATA_KEY_NAME = "metadata";
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
 
-    private final Map<TopicPartition,PartitionData> responseData;
+    private final Map<TopicPartition, PartitionData> responseData;
 
     public static final class PartitionData {
         public final long offset;
@@ -51,15 +53,19 @@ public class OffsetFetchResponse extends AbstractRequestResponse {
             this.metadata = metadata;
             this.errorCode = errorCode;
         }
+
+        public boolean hasError() {
+            return this.errorCode != Errors.NONE.code();
+        }
     }
 
     public OffsetFetchResponse(Map<TopicPartition, PartitionData> responseData) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
 
         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()) {
+        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>();
@@ -102,6 +108,6 @@ public class OffsetFetchResponse extends AbstractRequestResponse {
     }
 
     public static OffsetFetchResponse parse(ByteBuffer buffer) {
-        return new OffsetFetchResponse(((Struct) curSchema.read(buffer)));
+        return new OffsetFetchResponse(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 3dbba8a..03a0ab1 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
@@ -27,25 +27,26 @@ import java.util.List;
 import java.util.Map;
 
 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";
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id);
+    private static final String ACKS_KEY_NAME = "acks";
+    private static final String TIMEOUT_KEY_NAME = "timeout";
+    private static final 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";
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String PARTITION_DATA_KEY_NAME = "data";
 
     // partition level field names
-    private static String PARTITION_KEY_NAME = "partition";
-    private static String RECORD_SET_KEY_NAME = "record_set";
+    private static final String PARTITION_KEY_NAME = "partition";
+    private static final String RECORD_SET_KEY_NAME = "record_set";
 
     private final short acks;
     private final int timeout;
     private final Map<TopicPartition, ByteBuffer> partitionRecords;
 
     public ProduceRequest(short acks, int timeout, Map<TopicPartition, ByteBuffer> partitionRecords) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
         Map<String, Map<Integer, ByteBuffer>> recordsByTopic = CollectionUtils.groupDataByTopic(partitionRecords);
         struct.set(ACKS_KEY_NAME, acks);
         struct.set(TIMEOUT_KEY_NAME, timeout);
@@ -100,6 +101,6 @@ public class ProduceRequest  extends AbstractRequestResponse {
     }
 
     public static ProduceRequest parse(ByteBuffer buffer) {
-        return new ProduceRequest(((Struct) curSchema.read(buffer)));
+        return new ProduceRequest(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 5220464..e42d7db 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
@@ -26,22 +26,23 @@ import java.util.List;
 import java.util.Map;
 
 public class ProduceResponse extends AbstractRequestResponse {
-    private static Schema curSchema = ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id);
-    private static String RESPONSES_KEY_NAME = "responses";
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id);
+    private static final 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";
+    private static final String TOPIC_KEY_NAME = "topic";
+    private static final String PARTITION_RESPONSES_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 BASE_OFFSET_KEY_NAME = "base_offset";
+    private static final String PARTITION_KEY_NAME = "partition";
+    private static final String ERROR_CODE_KEY_NAME = "error_code";
+    private static final String BASE_OFFSET_KEY_NAME = "base_offset";
 
     private final Map<TopicPartition, PartitionResponse> responses;
 
     public ProduceResponse(Map<TopicPartition, PartitionResponse> responses) {
-        super(new Struct(curSchema));
+        super(new Struct(CURRENT_SCHEMA));
         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()) {
@@ -107,6 +108,6 @@ public class ProduceResponse extends AbstractRequestResponse {
     }
 
     public static ProduceResponse parse(ByteBuffer buffer) {
-        return new ProduceResponse(((Struct) curSchema.read(buffer)));
+        return new ProduceResponse(((Struct) CURRENT_SCHEMA.read(buffer)));
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java
index 3c001d3..13be6a3 100644
--- a/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java
+++ b/clients/src/main/java/org/apache/kafka/common/serialization/Deserializer.java
@@ -29,7 +29,7 @@ public interface Deserializer<T> {
      * @param isKey whether is for key or value
      */
     public void configure(Map<String, ?> configs, boolean isKey);
-
+    
     /**
      *
      * @param topic topic associated with the data

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
index 527dd0f..8a305b0 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
@@ -17,6 +17,9 @@ import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.UnsupportedEncodingException;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -224,6 +227,18 @@ public class Utils {
     }
 
     /**
+     * Sleep for a bit
+     * @param ms The duration of the sleep
+     */
+    public static void sleep(long ms) {
+        try {
+            Thread.sleep(ms);
+        } catch (InterruptedException e) {
+            // this is okay, we just wake up early
+        }
+    }
+
+    /**
      * Instantiate the class
      */
     public static Object newInstance(Class<?> c) {
@@ -313,4 +328,31 @@ public class Utils {
                 ? "[" + host + "]:" + port // IPv6
                 : host + ":" + port;
     }
+    
+    /**
+     * Create a string representation of an array joined by the given separator
+     * @param strs The array of items
+     * @param seperator The separator
+     * @return The string representation.
+     */
+    public static <T> String join(T[] strs, String seperator) {
+        return join(Arrays.asList(strs), seperator);
+    }
+    
+    /**
+     * Create a string representation of a list joined by the given separator
+     * @param list The list of items
+     * @param seperator The separator
+     * @return The string representation.
+     */
+    public static <T> String join(Collection<T> list, String seperator) {
+        StringBuilder sb = new StringBuilder();
+        Iterator<T> iter = list.iterator();
+        while(iter.hasNext()) {
+            sb.append(iter.next());
+            if(iter.hasNext())
+            sb.append(seperator);  
+        }
+      return sb.toString();
+    }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/test/java/org/apache/kafka/clients/MockClient.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java
index 47b5d4a..67bee40 100644
--- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java
+++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java
@@ -59,13 +59,33 @@ public class MockClient implements KafkaClient {
     }
 
     @Override
-    public List<ClientResponse> poll(List<ClientRequest> requests, long timeoutMs, long now) {
-        this.requests.addAll(requests);
-        List<ClientResponse> copy = new ArrayList<ClientResponse>(this.responses);
+    public void send(ClientRequest request) {
+        this.requests.add(request);
+    }
+
+    @Override
+    public List<ClientResponse> poll(long timeoutMs, long now) {
+        for(ClientResponse response: this.responses)
+            if (response.request().hasCallback()) 
+                response.request().callback().onComplete(response);
+        List<ClientResponse> copy = new ArrayList<ClientResponse>();
         this.responses.clear();
         return copy;
     }
 
+    @Override
+    public List<ClientResponse> completeAll(int node, long now) {
+        return completeAll(now);
+    }
+
+    @Override
+    public List<ClientResponse> completeAll(long now) {
+        List<ClientResponse> responses = poll(0, now);
+        if (requests.size() > 0)
+            throw new IllegalStateException("Requests without responses remain.");
+        return responses;
+    }
+
     public Queue<ClientRequest> requests() {
         return this.requests;
     }
@@ -81,6 +101,11 @@ public class MockClient implements KafkaClient {
     }
 
     @Override
+    public int inFlightRequestCount(int nodeId) {
+        return requests.size();
+    }
+
+    @Override
     public RequestHeader nextRequestHeader(ApiKeys key) {
         return new RequestHeader(key.id, "mock", correlation++);
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 1a55242..5debcd6 100644
--- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java
@@ -5,7 +5,6 @@ import static org.junit.Assert.assertFalse;
 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;
@@ -46,14 +45,13 @@ public class NetworkClientTest {
 
     @Test
     public void testReadyAndDisconnect() {
-        List<ClientRequest> reqs = new ArrayList<ClientRequest>();
         assertFalse("Client begins unready as it has no connection.", client.ready(node, time.milliseconds()));
         assertEquals("The connection is established as a side-effect of the readiness check", 1, selector.connected().size());
-        client.poll(reqs, 1, time.milliseconds());
+        client.poll(1, time.milliseconds());
         selector.clear();
         assertTrue("Now the client is ready", client.ready(node, time.milliseconds()));
         selector.disconnect(node.id());
-        client.poll(reqs, 1, time.milliseconds());
+        client.poll(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.timeToNextUpdate(time.milliseconds()) == 0);
@@ -65,7 +63,8 @@ public class NetworkClientTest {
                                            client.nextRequestHeader(ApiKeys.METADATA),
                                            new MetadataRequest(Arrays.asList("test")).toStruct());
         ClientRequest request = new ClientRequest(time.milliseconds(), false, send, null);
-        client.poll(Arrays.asList(request), 1, time.milliseconds());
+        client.send(request);
+        client.poll(1, time.milliseconds());
     }
 
     @Test
@@ -73,9 +72,11 @@ public class NetworkClientTest {
         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);
+        TestCallbackHandler handler = new TestCallbackHandler();
+        ClientRequest request = new ClientRequest(time.milliseconds(), true, send, handler);
         awaitReady(client, node);
-        client.poll(Arrays.asList(request), 1, time.milliseconds());
+        client.send(request);
+        client.poll(1, time.milliseconds());
         assertEquals(1, client.inFlightRequestCount());
         ResponseHeader respHeader = new ResponseHeader(reqHeader.correlationId());
         Struct resp = new Struct(ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id));
@@ -86,16 +87,26 @@ public class NetworkClientTest {
         resp.writeTo(buffer);
         buffer.flip();
         selector.completeReceive(new NetworkReceive(node.id(), buffer));
-        List<ClientResponse> responses = client.poll(new ArrayList<ClientRequest>(), 1, time.milliseconds());
+        List<ClientResponse> responses = client.poll(1, time.milliseconds());
         assertEquals(1, responses.size());
-        ClientResponse response = responses.get(0);
-        assertTrue("Should have a response body.", response.hasResponse());
-        assertEquals("Should be correlated to the original request", request, response.request());
+        assertTrue("The handler should have executed.", handler.executed);
+        assertTrue("Should have a response body.", handler.response.hasResponse());
+        assertEquals("Should be correlated to the original request", request, handler.response.request());
     }
 
     private void awaitReady(NetworkClient client, Node node) {
         while (!client.ready(node, time.milliseconds()))
-            client.poll(new ArrayList<ClientRequest>(), 1, time.milliseconds());
+            client.poll(1, time.milliseconds());
+    }
+    
+    private static class TestCallbackHandler implements RequestCompletionHandler {
+        public boolean executed = false;
+        public ClientResponse response;
+        
+        public void onComplete(ClientResponse response) {
+            this.executed = true;
+            this.response = response;
+        }
     }
 
 }


[2/7] kafka git commit: KAFKA-1760: New consumer.

Posted by jk...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerExampleTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerExampleTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerExampleTest.java
deleted file mode 100644
index 29ad25e..0000000
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerExampleTest.java
+++ /dev/null
@@ -1,297 +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 org.apache.kafka.clients.consumer;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Properties;
-
-import org.apache.kafka.common.TopicPartition;
-import org.junit.Test;
-
-/**
- * TODO: Clean this after the consumer implementation is complete. Until then, it is useful to write some sample test code using the new APIs
- *
- */
-public class ConsumerExampleTest {
-    /**
-     * This example demonstrates how to use the consumer to leverage Kafka's group management functionality for automatic consumer load 
-     * balancing and failure detection. This example assumes that the offsets are stored in Kafka and are automatically committed periodically, 
-     * as controlled by the auto.commit.interval.ms config 
-     */
-//    @Test
-//    public void testConsumerGroupManagementWithAutoOffsetCommits() {
-//        Properties props = new Properties();
-//        props.put("metadata.broker.list", "localhost:9092");
-//        props.put("group.id", "test");
-//        props.put("session.timeout.ms", "1000");
-//        props.put("auto.commit.enable", "true");
-//        props.put("auto.commit.interval.ms", "10000");
-//        KafkaConsumer consumer = new KafkaConsumer(props);
-//        // subscribe to some topics
-//        consumer.subscribe("foo", "bar");
-//        boolean isRunning = true;
-//        while(isRunning) {
-//            Map<String, ConsumerRecords> records = consumer.poll(100);
-//            process(records);
-//        }
-//        consumer.close();
-//    }
-
-    /**
-     * This example demonstrates how to use the consumer to leverage Kafka's group management functionality for automatic consumer load 
-     * balancing and failure detection. This example assumes that the offsets are stored in Kafka and are manually committed using the 
-     * commit() API. This example also demonstrates rewinding the consumer's offsets if processing of consumed messages fails.
-     */
-//     @Test
-//     public void testConsumerGroupManagementWithManualOffsetCommit() {
-//         Properties props = new Properties();
-//         props.put("metadata.broker.list", "localhost:9092");
-//         props.put("group.id", "test");
-//         props.put("session.timeout.ms", "1000");
-//         props.put("auto.commit.enable", "false");
-//         KafkaConsumer consumer = new KafkaConsumer(props);
-//         // subscribe to some topics
-//         consumer.subscribe("foo", "bar");
-//         int commitInterval = 100;
-//         int numRecords = 0;
-//         boolean isRunning = true;
-//         Map<TopicPartition, Long> consumedOffsets = new HashMap<TopicPartition, Long>();
-//         while(isRunning) {
-//             Map<String, ConsumerRecords> records = consumer.poll(100);
-//             try {
-//                 Map<TopicPartition, Long> lastConsumedOffsets = process(records);
-//                 consumedOffsets.putAll(lastConsumedOffsets);
-//                 numRecords += records.size();
-//                 // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance
-//                 if(numRecords % commitInterval == 0) 
-//                     consumer.commit(true);
-//             } catch(Exception e) {
-//                 // rewind consumer's offsets for failed partitions
-//                 List<TopicPartition> failedPartitions = getFailedPartitions();
-//                 Map<TopicPartition, Long> offsetsToRewindTo = new HashMap<TopicPartition, Long>();
-//                 for(TopicPartition failedPartition : failedPartitions) {
-//                     // rewind to the last consumed offset for the failed partition. Since process() failed for this partition, the consumed offset
-//                     // should still be pointing to the last successfully processed offset and hence is the right offset to rewind consumption to.
-//                     offsetsToRewindTo.put(failedPartition, consumedOffsets.get(failedPartition));
-//                 }
-//                 // seek to new offsets only for partitions that failed the last process()
-//                 consumer.seek(offsetsToRewindTo);
-//             }
-//         }         
-//         consumer.close();
-//     }
-
-     private List<TopicPartition> getFailedPartitions() { return null; }
-
-    /**
-     * This example demonstrates the consumer can be used to leverage Kafka's group management functionality along with custom offset storage. 
-     * In this example, the assumption made is that the user chooses to store the consumer offsets outside Kafka. This requires the user to 
-     * plugin logic for retrieving the offsets from a custom store and provide the offsets to the consumer in the ConsumerRebalanceCallback
-     * callback. The onPartitionsAssigned callback is invoked after the consumer is assigned a new set of partitions on rebalance <i>and</i>
-     * before the consumption restarts post rebalance. This is the right place to supply offsets from a custom store to the consumer.
-     */
-//    @Test
-//    public void testConsumerRebalanceWithCustomOffsetStore() {
-//        Properties props = new Properties();
-//        props.put("metadata.broker.list", "localhost:9092");
-//        props.put("group.id", "test");
-//        props.put("session.timeout.ms", "1000");
-//        props.put("auto.commit.enable", "true");
-//        props.put("auto.commit.interval.ms", "10000");
-//        KafkaConsumer consumer = new KafkaConsumer(props,
-//                                                   new ConsumerRebalanceCallback() {
-//                                                        public void onPartitionsAssigned(Consumer consumer, Collection<TopicPartition> partitions) {
-//                                                           Map<TopicPartition, Long> lastCommittedOffsets = getLastCommittedOffsetsFromCustomStore(partitions);
-//                                                           consumer.seek(lastCommittedOffsets);
-//                                                        }
-//                                                        public void onPartitionsRevoked(Consumer consumer, Collection<TopicPartition> partitions) {
-//                                                            Map<TopicPartition, Long> offsets = getLastConsumedOffsets(partitions); // implemented by the user
-//                                                           commitOffsetsToCustomStore(offsets);                                            // implemented by the user
-//                                                        }
-//                                                        private Map<TopicPartition, Long> getLastCommittedOffsetsFromCustomStore(Collection<TopicPartition> partitions) {
-//                                                            return null;
-//                                                        }
-//                                                        private Map<TopicPartition, Long> getLastConsumedOffsets(Collection<TopicPartition> partitions) { return null; }
-//                                                        private void commitOffsetsToCustomStore(Map<TopicPartition, Long> offsets) {}
-//                                                    });
-//        // subscribe to topics
-//        consumer.subscribe("foo", "bar");
-//        int commitInterval = 100;
-//        int numRecords = 0;
-//        boolean isRunning = true;
-//        while(isRunning) {
-//            Map<String, ConsumerRecords> records = consumer.poll(100);
-//            Map<TopicPartition, Long> consumedOffsets = process(records);
-//            numRecords += records.size();
-//            // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance
-//            if(numRecords % commitInterval == 0) 
-//                commitOffsetsToCustomStore(consumedOffsets);
-//        }
-//        consumer.close();
-//    }
-
-    /**
-     * This example demonstrates how the consumer can be used to leverage Kafka's group management functionality along with Kafka based offset storage. 
-     * In this example, the assumption made is that the user chooses to use Kafka based offset management. 
-     */
-//    @Test
-//    public void testConsumerRewindWithGroupManagementAndKafkaOffsetStorage() {
-//        Properties props = new Properties();
-//        props.put("metadata.broker.list", "localhost:9092");
-//        props.put("group.id", "test");
-//        props.put("session.timeout.ms", "1000");
-//        props.put("auto.commit.enable", "false");        
-//        KafkaConsumer consumer = new KafkaConsumer(props,
-//                                                   new ConsumerRebalanceCallback() {
-//                                                        boolean rewindOffsets = true;
-//                                                        public void onPartitionsAssigned(Consumer consumer, Collection<TopicPartition> partitions) {
-//                                                            if(rewindOffsets) {
-//                                                            Map<TopicPartition, Long> latestCommittedOffsets = consumer.committed(null);
-//                                                            Map<TopicPartition, Long> newOffsets = rewindOffsets(latestCommittedOffsets, 100);
-//                                                            consumer.seek(newOffsets);
-//                                                            }
-//                                                        }
-//                                                        public void onPartitionsRevoked(Consumer consumer, Collection<TopicPartition> partitions) {
-//                                                            consumer.commit(true);
-//                                                        }
-//                                                        // this API rewinds every partition back by numberOfMessagesToRewindBackTo messages 
-//                                                        private Map<TopicPartition, Long> rewindOffsets(Map<TopicPartition, Long> currentOffsets,
-//                                                                                                        long numberOfMessagesToRewindBackTo) {
-//                                                            Map<TopicPartition, Long> newOffsets = new HashMap<TopicPartition, Long>();
-//                                                            for(Map.Entry<TopicPartition, Long> offset : currentOffsets.entrySet()) {
-//                                                                newOffsets.put(offset.getKey(), offset.getValue() - numberOfMessagesToRewindBackTo);
-//                                                            }
-//                                                            return newOffsets;
-//                                                        }
-//                                                    });
-//        // subscribe to topics
-//        consumer.subscribe("foo", "bar");
-//        int commitInterval = 100;
-//        int numRecords = 0;
-//        boolean isRunning = true;
-//        while(isRunning) {
-//            Map<String, ConsumerRecords> records = consumer.poll(100);
-//            Map<TopicPartition, Long> consumedOffsets = process(records);
-//            numRecords += records.size();
-//            // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance
-//            if(numRecords % commitInterval == 0) 
-//                commitOffsetsToCustomStore(consumedOffsets);
-//        }
-//        consumer.close();
-//    }
-
-    /**
-     * This example demonstrates how the consumer can be used to subscribe to specific partitions of certain topics and consume upto the latest
-     * available message for each of those partitions before shutting down. When used to subscribe to specific partitions, the user foregoes 
-     * the group management functionality and instead relies on manually configuring the consumer instances to subscribe to a set of partitions.
-     * This example assumes that the user chooses to use Kafka based offset storage. The user still has to specify a group.id to use Kafka 
-     * based offset management. However, session.timeout.ms is not required since the Kafka consumer only does failure detection with group 
-     * management.
-     */
-//    @Test
-//    public void testConsumerWithKafkaBasedOffsetManagement() {
-//        Properties props = new Properties();
-//        props.put("metadata.broker.list", "localhost:9092");
-//        props.put("group.id", "test");
-//        props.put("auto.commit.enable", "true");
-//        props.put("auto.commit.interval.ms", "10000");
-//        KafkaConsumer consumer = new KafkaConsumer(props);
-//        // subscribe to some partitions of topic foo
-//        TopicPartition partition0 = new TopicPartition("foo", 0);
-//        TopicPartition partition1 = new TopicPartition("foo", 1);
-//        TopicPartition[] partitions = new TopicPartition[2];
-//        partitions[0] = partition0;
-//        partitions[1] = partition1;
-//        consumer.subscribe(partitions);
-//        // find the last committed offsets for partitions 0,1 of topic foo
-//        Map<TopicPartition, Long> lastCommittedOffsets = consumer.committed(null);
-//        // seek to the last committed offsets to avoid duplicates
-//        consumer.seek(lastCommittedOffsets);        
-//        // find the offsets of the latest available messages to know where to stop consumption
-//        Map<TopicPartition, Long> latestAvailableOffsets = consumer.offsetsBeforeTime(-2, null);
-//        boolean isRunning = true;
-//        while(isRunning) {
-//            Map<String, ConsumerRecords> records = consumer.poll(100);
-//            Map<TopicPartition, Long> consumedOffsets = process(records);
-//            for(TopicPartition partition : partitions) {
-//                if(consumedOffsets.get(partition) >= latestAvailableOffsets.get(partition))
-//                    isRunning = false;
-//                else
-//                    isRunning = true;
-//            }
-//        }
-//        consumer.close();
-//    }
-
-    /**
-     * This example demonstrates how the consumer can be used to subscribe to specific partitions of certain topics and consume upto the latest
-     * available message for each of those partitions before shutting down. When used to subscribe to specific partitions, the user foregoes 
-     * the group management functionality and instead relies on manually configuring the consumer instances to subscribe to a set of partitions.
-     * This example assumes that the user chooses to use custom offset storage.
-     */
-    @Test
-    public void testConsumerWithCustomOffsetManagement() {
-//        Properties props = new Properties();
-//        props.put("metadata.broker.list", "localhost:9092");
-//        KafkaConsumer consumer = new KafkaConsumer(props);
-//        // subscribe to some partitions of topic foo
-//        TopicPartition partition0 = new TopicPartition("foo", 0);
-//        TopicPartition partition1 = new TopicPartition("foo", 1);
-//        TopicPartition[] partitions = new TopicPartition[2];
-//        partitions[0] = partition0;
-//        partitions[1] = partition1;
-//        consumer.subscribe(partitions);
-//        Map<TopicPartition, Long> lastCommittedOffsets = getLastCommittedOffsetsFromCustomStore();
-//        // seek to the last committed offsets to avoid duplicates
-//        consumer.seek(lastCommittedOffsets);        
-//        // find the offsets of the latest available messages to know where to stop consumption
-//        Map<TopicPartition, Long> latestAvailableOffsets = consumer.offsetsBeforeTime(-2, null);
-//        boolean isRunning = true;
-//        while(isRunning) {
-//            Map<String, ConsumerRecords> records = consumer.poll(100);
-//            Map<TopicPartition, Long> consumedOffsets = process(records);
-//            // commit offsets for partitions 0,1 for topic foo to custom store
-//            commitOffsetsToCustomStore(consumedOffsets);
-//            for(TopicPartition partition : partitions) {
-//                if(consumedOffsets.get(partition) >= latestAvailableOffsets.get(partition))
-//                    isRunning = false;
-//                else
-//                    isRunning = true;
-//            }            
-//        }         
-//        consumer.close();
-    }
-
-    private Map<TopicPartition, Long> getLastCommittedOffsetsFromCustomStore() { return null; }
-    private void commitOffsetsToCustomStore(Map<TopicPartition, Long> consumedOffsets) {}
-    private Map<TopicPartition, Long> process(Map<String, ConsumerRecords> records) {
-        Map<TopicPartition, Long> processedOffsets = new HashMap<TopicPartition, Long>();
-        for(Entry<String, ConsumerRecords> recordMetadata : records.entrySet()) {
-            List<ConsumerRecord> recordsPerTopic = recordMetadata.getValue().records();
-            for(int i = 0;i < recordsPerTopic.size();i++) {
-                ConsumerRecord record = recordsPerTopic.get(i);
-                // process record
-                try {
-                    processedOffsets.put(record.topicAndPartition(), record.offset());
-                } catch (Exception e) {
-                    e.printStackTrace();
-                }                
-            }
-        }
-        return processedOffsets; 
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java
new file mode 100644
index 0000000..e51d2df
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/MockConsumerTest.java
@@ -0,0 +1,32 @@
+package org.apache.kafka.clients.consumer;
+
+import static org.junit.Assert.*;
+
+import java.util.Iterator;
+
+import org.apache.kafka.common.TopicPartition;
+import org.junit.Test;
+
+public class MockConsumerTest {
+    
+    private MockConsumer<String, String> consumer = new MockConsumer<String, String>();
+
+    @Test
+    public void testSimpleMock() {
+        consumer.subscribe("topic");
+        assertEquals(0, consumer.poll(1000).count());
+        ConsumerRecord<String, String> rec1 = new ConsumerRecord<String, String>("test", 0, 0, "key1", "value1");
+        ConsumerRecord<String, String> rec2 = new ConsumerRecord<String, String>("test", 0, 1, "key2", "value2");
+        consumer.addRecord(rec1);
+        consumer.addRecord(rec2);
+        ConsumerRecords<String, String> recs = consumer.poll(1);
+        Iterator<ConsumerRecord<String, String>> iter = recs.iterator();
+        assertEquals(rec1, iter.next());
+        assertEquals(rec2, iter.next());
+        assertFalse(iter.hasNext());
+        assertEquals(1L, consumer.position(new TopicPartition("test", 0)));
+        consumer.commit(CommitType.SYNC);
+        assertEquals(1L, consumer.committed(new TopicPartition("test", 0)));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java
new file mode 100644
index 0000000..864f1c7
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java
@@ -0,0 +1,61 @@
+package org.apache.kafka.clients.consumer.internals;
+
+import static org.junit.Assert.*;
+import static java.util.Arrays.asList;
+
+import java.util.Collections;
+
+import org.apache.kafka.common.TopicPartition;
+import org.junit.Test;
+
+public class SubscriptionStateTest {
+    
+    private final SubscriptionState state = new SubscriptionState();
+    private final TopicPartition tp0 = new TopicPartition("test", 0);
+    private final TopicPartition tp1 = new TopicPartition("test", 1);
+
+    @Test
+    public void partitionSubscription() {      
+        state.subscribe(tp0);
+        assertEquals(Collections.singleton(tp0), state.assignedPartitions());
+        state.committed(tp0, 1);
+        state.fetched(tp0, 1);
+        state.consumed(tp0, 1);
+        assertAllPositions(tp0, 1L);
+        state.unsubscribe(tp0);
+        assertTrue(state.assignedPartitions().isEmpty());
+        assertAllPositions(tp0, null);
+    }
+    
+    public void topicSubscription() {
+        state.subscribe("test");
+        assertEquals(1, state.subscribedTopics().size());
+        assertTrue(state.assignedPartitions().isEmpty());
+        assertTrue(state.partitionsAutoAssigned());
+        state.changePartitionAssignment(asList(tp0));
+        state.committed(tp0, 1);
+        state.fetched(tp0, 1);
+        state.consumed(tp0, 1);
+        assertAllPositions(tp0, 1L);
+        state.changePartitionAssignment(asList(tp1));
+        assertAllPositions(tp0, null);
+        assertEquals(Collections.singleton(tp1), state.assignedPartitions());
+    }
+    
+    @Test(expected = IllegalArgumentException.class)
+    public void cantChangeFetchPositionForNonAssignedPartition() {
+        state.fetched(tp0, 1);
+    }
+    
+    @Test(expected = IllegalArgumentException.class)
+    public void cantChangeConsumedPositionForNonAssignedPartition() {
+        state.consumed(tp0, 1);
+    }
+    
+    public void assertAllPositions(TopicPartition tp, Long offset) {
+        assertEquals(offset, state.committed(tp));
+        assertEquals(offset, state.fetched(tp));
+        assertEquals(offset, state.consumed(tp));
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java
index 1236803..77b23e7 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java
@@ -43,7 +43,7 @@ public class BufferPoolTest {
      */
     @Test
     public void testSimple() throws Exception {
-        int totalMemory = 64 * 1024;
+        long totalMemory = 64 * 1024;
         int size = 1024;
         BufferPool pool = new BufferPool(totalMemory, size, false, metrics, time, metricGroup, metricTags);
         ByteBuffer buffer = pool.allocate(size);
@@ -100,7 +100,7 @@ public class BufferPoolTest {
         ByteBuffer buffer = pool.allocate(1024);
         CountDownLatch doDealloc = asyncDeallocate(pool, buffer);
         CountDownLatch allocation = asyncAllocate(pool, 5 * 1024);
-        assertEquals("Allocation shouldn't have happened yet, waiting on memory.", 1, allocation.getCount());
+        assertEquals("Allocation shouldn't have happened yet, waiting on memory.", 1L, allocation.getCount());
         doDealloc.countDown(); // return the memory
         allocation.await();
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java
index 3676b05..d3377ef 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java
@@ -38,7 +38,7 @@ public class MockProducerTest {
         Future<RecordMetadata> metadata = producer.send(record);
         assertTrue("Send should be immediately complete", metadata.isDone());
         assertFalse("Send should be successful", isError(metadata));
-        assertEquals("Offset should be 0", 0, metadata.get().offset());
+        assertEquals("Offset should be 0", 0L, metadata.get().offset());
         assertEquals(topic, metadata.get().topic());
         assertEquals("We should have the record in our history", asList(record), producer.history());
         producer.clear();

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java
index 1d077fd..82d8083 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java
@@ -1,30 +1,23 @@
 /**
- * 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.
+ * 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.clients.producer;
 
 import static java.util.Arrays.asList;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotSame;
 import static org.junit.Assert.assertTrue;
 
 import java.util.List;
 
-
-import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.clients.producer.internals.Partitioner;
 import org.apache.kafka.common.Cluster;
 import org.apache.kafka.common.Node;
@@ -34,7 +27,6 @@ import org.junit.Test;
 public class PartitionerTest {
 
     private byte[] key = "key".getBytes();
-    private byte[] value = "value".getBytes();
     private Partitioner partitioner = new Partitioner();
     private Node node0 = new Node(0, "localhost", 99);
     private Node node1 = new Node(1, "localhost", 100);
@@ -48,33 +40,28 @@ public class PartitionerTest {
 
     @Test
     public void testUserSuppliedPartitioning() {
-        assertEquals("If the user supplies a partition we should use it.",
-                     0,
-                     partitioner.partition(new ProducerRecord<byte[], byte[]>("test", 0, key, value), cluster));
+        assertEquals("If the user supplies a partition we should use it.", 0, partitioner.partition("test", key, 0, cluster));
     }
 
     @Test
     public void testKeyPartitionIsStable() {
-        int partition = partitioner.partition(new ProducerRecord<byte[], byte[]>("test", key, value), cluster);
-        assertEquals("Same key should yield same partition",
-                     partition,
-                     partitioner.partition(new ProducerRecord<byte[], byte[]>("test", key, "value2".getBytes()), cluster));
+        int partition = partitioner.partition("test", key, null, cluster);
+        assertEquals("Same key should yield same partition", partition, partitioner.partition("test", key, null, cluster));
     }
 
     @Test
     public void testRoundRobinIsStable() {
-        int startPart = partitioner.partition(new ProducerRecord<byte[], byte[]>("test", value), cluster);
+        int startPart = partitioner.partition("test", null, null, cluster);
         for (int i = 1; i <= 100; i++) {
-            int partition = partitioner.partition(new ProducerRecord<byte[], byte[]>("test", value), cluster);
-            assertEquals("Should yield a different partition each call with round-robin partitioner",
-                partition, (startPart + i) % 2);
-      }
+            int partition = partitioner.partition("test", null, null, cluster);
+            assertEquals("Should yield a different partition each call with round-robin partitioner", partition, (startPart + i) % 2);
+        }
     }
 
     @Test
     public void testRoundRobinWithDownNode() {
         for (int i = 0; i < partitions.size(); i++) {
-            int part = partitioner.partition(new ProducerRecord<byte[], byte[]>("test", value), cluster);
+            int part = partitioner.partition("test", null, null, cluster);
             assertTrue("We should never choose a leader-less node in round robin", part >= 0 && part < 2);
         }
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 66cbdf5..888b929 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
@@ -72,14 +72,14 @@ public class SenderTest {
 
     @Test
     public void testSimple() throws Exception {
-        int offset = 0;
+        long offset = 0;
         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());
         client.respond(produceResponse(tp.topic(), tp.partition(), offset, Errors.NONE.code()));
         sender.run(time.milliseconds());
-        assertEquals("All requests completed.", offset, client.inFlightRequestCount());
+        assertEquals("All requests completed.", offset, (long) client.inFlightRequestCount());
         sender.run(time.milliseconds());
         assertTrue("Request should be completed", future.isDone());
         assertEquals(offset, future.get().offset());
@@ -110,7 +110,7 @@ public class SenderTest {
         sender.run(time.milliseconds()); // reconnect
         sender.run(time.milliseconds()); // resend
         assertEquals(1, client.inFlightRequestCount());
-        int offset = 0;
+        long offset = 0;
         client.respond(produceResponse(tp.topic(), tp.partition(), offset, Errors.NONE.code()));
         sender.run(time.milliseconds());
         assertTrue("Request should have retried and completed", future.isDone());

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java
index 3c442a2..16d3fed 100644
--- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java
@@ -110,18 +110,18 @@ public class ConfigDefTest {
 
     @Test(expected = ConfigException.class)
     public void testInvalidDefaultRange() {
-        ConfigDef def = new ConfigDef().define("name", Type.INT, -1, Range.between(0,10), Importance.HIGH, "docs");
+        new ConfigDef().define("name", Type.INT, -1, Range.between(0,10), Importance.HIGH, "docs");
     }
 
     @Test(expected = ConfigException.class)
     public void testInvalidDefaultString() {
-        ConfigDef def = new ConfigDef().define("name", Type.STRING, "bad", ValidString.in(Arrays.asList("valid", "values")), Importance.HIGH, "docs");
+        new ConfigDef().define("name", Type.STRING, "bad", ValidString.in("valid", "values"), Importance.HIGH, "docs");
     }
 
     @Test
     public void testValidators() {
         testValidators(Type.INT, Range.between(0,10), 5, new Object[]{1, 5, 9}, new Object[]{-1, 11});
-        testValidators(Type.STRING, ValidString.in(Arrays.asList("good", "values", "default")), "default",
+        testValidators(Type.STRING, ValidString.in("good", "values", "default"), "default",
                 new Object[]{"good", "values", "default"}, new Object[]{"bad", "inputs"});
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java
index 74c1957..a14659a 100644
--- a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java
@@ -12,7 +12,6 @@
  */
 package org.apache.kafka.common.network;
 
-import static java.util.Arrays.asList;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -74,7 +73,7 @@ public class SelectorTest {
         // disconnect
         this.server.closeConnections();
         while (!selector.disconnected().contains(node))
-            selector.poll(1000L, EMPTY);
+            selector.poll(1000L);
 
         // reconnect and do another request
         blockingConnect(node);
@@ -89,7 +88,8 @@ public class SelectorTest {
         int node = 0;
         blockingConnect(node);
         selector.disconnect(node);
-        selector.poll(10, asList(createSend(node, "hello1")));
+        selector.send(createSend(node, "hello1"));
+        selector.poll(10);
         assertEquals("Request should not have succeeded", 0, selector.completedSends().size());
         assertEquals("There should be a disconnect", 1, selector.disconnected().size());
         assertTrue("The disconnect should be from our node", selector.disconnected().contains(node));
@@ -104,7 +104,9 @@ public class SelectorTest {
     public void testCantSendWithInProgress() throws Exception {
         int node = 0;
         blockingConnect(node);
-        selector.poll(1000L, asList(createSend(node, "test1"), createSend(node, "test2")));
+        selector.send(createSend(node, "test1"));
+        selector.send(createSend(node, "test2"));
+        selector.poll(1000L);
     }
 
     /**
@@ -112,7 +114,8 @@ public class SelectorTest {
      */
     @Test(expected = IllegalStateException.class)
     public void testCantSendWithoutConnecting() throws Exception {
-        selector.poll(1000L, asList(createSend(0, "test")));
+        selector.send(createSend(0, "test"));
+        selector.poll(1000L);
     }
 
     /**
@@ -131,7 +134,7 @@ public class SelectorTest {
         int node = 0;
         selector.connect(node, new InetSocketAddress("localhost", TestUtils.choosePort()), BUFFER_SIZE, BUFFER_SIZE);
         while (selector.disconnected().contains(node))
-            selector.poll(1000L, EMPTY);
+            selector.poll(1000L);
     }
 
     /**
@@ -152,14 +155,13 @@ public class SelectorTest {
         int[] requests = new int[conns];
         int[] responses = new int[conns];
         int responseCount = 0;
-        List<NetworkSend> sends = new ArrayList<NetworkSend>();
         for (int i = 0; i < conns; i++)
-            sends.add(createSend(i, i + "-" + 0));
+            selector.send(createSend(i, i + "-" + 0));
 
         // loop until we complete all requests
         while (responseCount < conns * reqs) {
             // do the i/o
-            selector.poll(0L, sends);
+            selector.poll(0L);
 
             assertEquals("No disconnects should have occurred.", 0, selector.disconnected().size());
 
@@ -175,12 +177,11 @@ public class SelectorTest {
             }
 
             // prepare new sends for the next round
-            sends.clear();
             for (NetworkSend send : selector.completedSends()) {
                 int dest = send.destination();
                 requests[dest]++;
                 if (requests[dest] < reqs)
-                    sends.add(createSend(dest, dest + "-" + requests[dest]));
+                    selector.send(createSend(dest, dest + "-" + requests[dest]));
             }
         }
     }
@@ -212,10 +213,34 @@ public class SelectorTest {
         blockingConnect(0);
     }
 
+    @Test
+    public void testMute() throws Exception {
+        blockingConnect(0);
+        blockingConnect(1);
+
+        selector.send(createSend(0, "hello"));
+        selector.send(createSend(1, "hi"));
+
+        selector.mute(1);
+
+        while (selector.completedReceives().isEmpty())
+            selector.poll(5);
+        assertEquals("We should have only one response", 1, selector.completedReceives().size());
+        assertEquals("The response should not be from the muted node", 0, selector.completedReceives().get(0).source());
+
+        selector.unmute(1);
+        do {
+            selector.poll(5);
+        } while (selector.completedReceives().isEmpty());
+        assertEquals("We should have only one response", 1, selector.completedReceives().size());
+        assertEquals("The response should be from the previously muted node", 1, selector.completedReceives().get(0).source());
+    }
+
     private String blockingRequest(int node, String s) throws IOException {
-        selector.poll(1000L, asList(createSend(node, s)));
+        selector.send(createSend(node, s));
+        selector.poll(1000L);
         while (true) {
-            selector.poll(1000L, EMPTY);
+            selector.poll(1000L);
             for (NetworkReceive receive : selector.completedReceives())
                 if (receive.source() == node)
                     return asString(receive);
@@ -226,7 +251,7 @@ public class SelectorTest {
     private void blockingConnect(int node) throws IOException {
         selector.connect(node, new InetSocketAddress("localhost", server.port), BUFFER_SIZE, BUFFER_SIZE);
         while (!selector.connected().contains(node))
-            selector.poll(10000L, EMPTY);
+            selector.poll(10000L);
     }
 
     private NetworkSend createSend(int node, String s) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java
index a39fab5..4c2ea34 100644
--- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java
@@ -16,6 +16,9 @@
  */
 package org.apache.kafka.common.utils;
 
+import java.util.Arrays;
+import java.util.Collections;
+
 import org.junit.Test;
 
 import static org.apache.kafka.common.utils.Utils.getHost;
@@ -48,4 +51,11 @@ public class UtilsTest {
         assertEquals("[::1]:1234", formatAddress("::1", 1234));
         assertEquals("[2001:db8:85a3:8d3:1319:8a2e:370:7348]:5678", formatAddress("2001:db8:85a3:8d3:1319:8a2e:370:7348", 5678));
     }
+    
+    @Test
+    public void testJoin() {
+        assertEquals("", Utils.join(Collections.emptyList(), ","));
+        assertEquals("1", Utils.join(Arrays.asList("1"), ","));
+        assertEquals("1,2,3", Utils.join(Arrays.asList(1, 2, 3), ","));
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/test/java/org/apache/kafka/test/MockSelector.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/test/MockSelector.java b/clients/src/test/java/org/apache/kafka/test/MockSelector.java
index d61de52..ea89b06 100644
--- a/clients/src/test/java/org/apache/kafka/test/MockSelector.java
+++ b/clients/src/test/java/org/apache/kafka/test/MockSelector.java
@@ -1,18 +1,14 @@
 /**
- * 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.
+ * 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.test;
 
@@ -26,13 +22,13 @@ import org.apache.kafka.common.network.NetworkSend;
 import org.apache.kafka.common.network.Selectable;
 import org.apache.kafka.common.utils.Time;
 
-
 /**
  * A fake selector to use for testing
  */
 public class MockSelector implements Selectable {
 
     private final Time time;
+    private final List<NetworkSend> initiatedSends = new ArrayList<NetworkSend>();
     private final List<NetworkSend> completedSends = new ArrayList<NetworkSend>();
     private final List<NetworkReceive> completedReceives = new ArrayList<NetworkReceive>();
     private final List<Integer> disconnected = new ArrayList<Integer>();
@@ -68,8 +64,14 @@ public class MockSelector implements Selectable {
     }
 
     @Override
-    public void poll(long timeout, List<NetworkSend> sends) throws IOException {
-        this.completedSends.addAll(sends);
+    public void send(NetworkSend send) {
+        this.initiatedSends.add(send);
+    }
+
+    @Override
+    public void poll(long timeout) throws IOException {
+        this.completedSends.addAll(this.initiatedSends);
+        this.initiatedSends.clear();
         time.sleep(timeout);
     }
 
@@ -101,4 +103,20 @@ public class MockSelector implements Selectable {
         return connected;
     }
 
+    @Override
+    public void mute(int id) {
+    }
+
+    @Override
+    public void unmute(int id) {
+    }
+
+    @Override
+    public void muteAll() {
+    }
+
+    @Override
+    public void unmuteAll() {
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 6d00ed0..a3b1b78 100644
--- a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala
+++ b/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala
@@ -63,7 +63,7 @@ case class ConsumerMetadataRequest(group: String,
 
   override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
     // return ConsumerCoordinatorNotAvailable for all uncaught errors
-    val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode)
+    val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, correlationId)
     requestChannel.sendResponse(new Response(request, new BoundedByteBufferSend(errorResponse)))
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 84f6017..24aaf95 100644
--- a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
+++ b/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
@@ -40,7 +40,7 @@ object ConsumerMetadataResponse {
   
 }
 
-case class ConsumerMetadataResponse (coordinatorOpt: Option[Broker], errorCode: Short, correlationId: Int = 0)
+case class ConsumerMetadataResponse (coordinatorOpt: Option[Broker], errorCode: Short, correlationId: Int)
   extends RequestOrResponse() {
 
   def sizeInBytes =

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 b230e9a..e6ad8be 100644
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -90,7 +90,7 @@ class Partition(val topic: String,
           val checkpoint = replicaManager.highWatermarkCheckpoints(log.dir.getParentFile.getAbsolutePath)
           val offsetMap = checkpoint.read
           if (!offsetMap.contains(TopicAndPartition(topic, partitionId)))
-            warn("No checkpointed highwatermark is found for partition [%s,%d]".format(topic, partitionId))
+            info("No checkpointed highwatermark is found for partition [%s,%d]".format(topic, partitionId))
           val offset = offsetMap.getOrElse(TopicAndPartition(topic, partitionId), 0L).min(log.logEndOffset)
           val localReplica = new Replica(replicaId, this, time, offset, Some(log))
           addReplicaIfNotExists(localReplica)

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 fbef34c..14b22ab 100644
--- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
+++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
@@ -141,7 +141,7 @@ class RequestSendThread(val controllerId: Int,
               connectToBroker(toBroker, channel)
               isSendSuccessful = false
               // backoff before retrying the connection and send
-              Utils.swallow(Thread.sleep(300))
+              Utils.swallowTrace(Thread.sleep(300))
           }
         }
         if (receive != null) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/core/src/main/scala/kafka/log/LogConfig.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala
index 4631bc7..8b67aee 100644
--- a/core/src/main/scala/kafka/log/LogConfig.scala
+++ b/core/src/main/scala/kafka/log/LogConfig.scala
@@ -179,12 +179,12 @@ object LogConfig {
       .define(FileDeleteDelayMsProp, LONG, Defaults.FileDeleteDelayMs, atLeast(0), MEDIUM, FileDeleteDelayMsDoc)
       .define(MinCleanableDirtyRatioProp, DOUBLE, Defaults.MinCleanableDirtyRatio, between(0, 1), MEDIUM,
         MinCleanableRatioDoc)
-      .define(CleanupPolicyProp, STRING, if (Defaults.Compact) Compact else Delete, in(asList(Compact, Delete)), MEDIUM,
+      .define(CleanupPolicyProp, STRING, if (Defaults.Compact) Compact else Delete, in(Compact, Delete), MEDIUM,
         CompactDoc)
       .define(UncleanLeaderElectionEnableProp, BOOLEAN, Defaults.UncleanLeaderElectionEnable,
         MEDIUM, UncleanLeaderElectionEnableDoc)
       .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), MEDIUM, MinInSyncReplicasDoc)
-      .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(seqAsJavaList(BrokerCompressionCodec.brokerCompressionOptions)), MEDIUM, CompressionTypeDoc)
+      .define(CompressionTypeProp, STRING, Defaults.CompressionType, in(BrokerCompressionCodec.brokerCompressionOptions:_*), MEDIUM, CompressionTypeDoc)
   }
 
   def configNames() = {

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 ec8d9f7..48bc435 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -17,6 +17,12 @@
 
 package kafka.server
 
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.JoinGroupResponse
+import org.apache.kafka.common.requests.HeartbeatResponse
+import org.apache.kafka.common.requests.ResponseHeader
+import org.apache.kafka.common.protocol.types.Struct
+
 import kafka.api._
 import kafka.common._
 import kafka.log._
@@ -26,6 +32,9 @@ import kafka.network.RequestChannel.Response
 import kafka.controller.KafkaController
 import kafka.utils.{SystemTime, Logging}
 
+import java.nio.ByteBuffer
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic._
 import scala.collection._
 
 import org.I0Itec.zkclient.ZkClient
@@ -43,6 +52,7 @@ class KafkaApis(val requestChannel: RequestChannel,
 
   this.logIdent = "[KafkaApi-%d] ".format(brokerId)
   val metadataCache = new MetadataCache
+  private var consumerGroupGenerationId = 0
 
   /**
    * Top-level method that handles all requests and multiplexes to the right api
@@ -62,6 +72,8 @@ class KafkaApis(val requestChannel: RequestChannel,
         case RequestKeys.OffsetCommitKey => handleOffsetCommitRequest(request)
         case RequestKeys.OffsetFetchKey => handleOffsetFetchRequest(request)
         case RequestKeys.ConsumerMetadataKey => handleConsumerMetadataRequest(request)
+        case RequestKeys.JoinGroupKey => handleJoinGroupRequest(request)
+        case RequestKeys.HeartbeatKey => handleHeartbeatRequest(request)
         case requestId => throw new KafkaException("Unknown api code " + requestId)
       }
     } catch {
@@ -442,6 +454,23 @@ class KafkaApis(val requestChannel: RequestChannel,
     requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response)))
   }
 
+  def handleJoinGroupRequest(request: RequestChannel.Request) {
+    val joinGroupReq = request.requestObj.asInstanceOf[JoinGroupRequestAndHeader]
+    val topics = JavaConversions.asScalaIterable(joinGroupReq.body.topics()).toSet
+    val partitions = this.replicaManager.logManager.allLogs.filter(log => topics.contains(log.topicAndPartition.topic))
+    val partitionList = partitions.map(_.topicAndPartition).map(tp => new org.apache.kafka.common.TopicPartition(tp.topic, tp.partition)).toBuffer
+    this.consumerGroupGenerationId += 1
+    val response = new JoinGroupResponse(ErrorMapping.NoError, this.consumerGroupGenerationId, joinGroupReq.body.consumerId, JavaConversions.asJavaList(partitionList))
+    val send = new BoundedByteBufferSend(new JoinGroupResponseAndHeader(joinGroupReq.correlationId, response))
+    requestChannel.sendResponse(new RequestChannel.Response(request, send))
+  }
+  
+  def handleHeartbeatRequest(request: RequestChannel.Request) {
+    val hbReq = request.requestObj.asInstanceOf[HeartbeatRequestAndHeader]
+    val send = new BoundedByteBufferSend(new HeartbeatResponseAndHeader(hbReq.correlationId, new HeartbeatResponse(Errors.NONE.code)))
+    requestChannel.sendResponse(new RequestChannel.Response(request, send))
+  }
+  
   def close() {
     // TODO currently closing the API is an no-op since the API no longer maintain any modules
     // maybe removing the closing call in the end when KafkaAPI becomes a pure stateless layer

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 e58fbb9..fb948b9 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -327,7 +327,7 @@ class ReplicaManager(val config: KafkaConfig,
           BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages)
 
           trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d"
-            .format(messages.size, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset))
+            .format(messages.sizeInBytes, topicAndPartition.topic, topicAndPartition.partition, info.firstOffset, info.lastOffset))
           (topicAndPartition, LogAppendResult(info))
         } catch {
           // NOTE: Failed produce requests metric is not incremented for known exceptions

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/core/src/main/scala/kafka/tools/ConsoleConsumer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala
index e455cb9..910691e 100644
--- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala
+++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala
@@ -113,8 +113,6 @@ object ConsoleConsumer extends Logging {
       KafkaMetricsReporter.startReporters(verifiableProps)
     }
 
-
-
     val consumerProps = if (options.has(consumerConfigOpt))
       Utils.loadProps(options.valueOf(consumerConfigOpt))
     else

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/core/src/main/scala/kafka/tools/ConsumerPerformance.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala
index 093c800..c39c067 100644
--- a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala
+++ b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala
@@ -17,14 +17,21 @@
 
 package kafka.tools
 
-import java.util.concurrent.CountDownLatch
+import scala.collection.JavaConversions._
 import java.util.concurrent.atomic.AtomicLong
 import java.nio.channels.ClosedByInterruptException
 import org.apache.log4j.Logger
+import org.apache.kafka.clients.consumer.KafkaConsumer
+import org.apache.kafka.common.record.Record
+import org.apache.kafka.common.record.Records
+import org.apache.kafka.common.serialization.ByteArrayDeserializer
 import kafka.message.Message
 import kafka.utils.{ZkUtils, CommandLineUtils}
 import java.util.{ Random, Properties }
-import kafka.consumer._
+import kafka.consumer.Consumer
+import kafka.consumer.ConsumerConnector
+import kafka.consumer.KafkaStream
+import kafka.consumer.ConsumerTimeoutException
 import java.text.SimpleDateFormat
 
 /**
@@ -42,50 +49,98 @@ object ConsumerPerformance {
 
     if (!config.hideHeader) {
       if (!config.showDetailedStats)
-        println("start.time, end.time, fetch.size, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec")
+        println("start.time, end.time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec")
       else
-        println("time, fetch.size, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec")
+        println("time, data.consumed.in.MB, MB.sec, data.consumed.in.nMsg, nMsg.sec")
     }
 
-    // clean up zookeeper state for this group id for every perf run
-    ZkUtils.maybeDeletePath(config.consumerConfig.zkConnect, "/consumers/" + config.consumerConfig.groupId)
-
-    val consumerConnector: ConsumerConnector = Consumer.create(config.consumerConfig)
-
-    val topicMessageStreams = consumerConnector.createMessageStreams(Map(config.topic -> config.numThreads))
-    var threadList = List[ConsumerPerfThread]()
-    for ((topic, streamList) <- topicMessageStreams)
-      for (i <- 0 until streamList.length)
-        threadList ::= new ConsumerPerfThread(i, "kafka-zk-consumer-" + i, streamList(i), config,
-          totalMessagesRead, totalBytesRead)
-
-    logger.info("Sleeping for 1 second.")
-    Thread.sleep(1000)
-    logger.info("starting threads")
-    val startMs = System.currentTimeMillis
-    for (thread <- threadList)
-      thread.start
-
-    for (thread <- threadList)
-      thread.join
-
-    val endMs = System.currentTimeMillis
-    val elapsedSecs = (endMs - startMs - config.consumerConfig.consumerTimeoutMs) / 1000.0
+    var startMs, endMs = 0L
+    if(config.useNewConsumer) {
+      val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](config.props)
+      consumer.subscribe(config.topic)
+      startMs = System.currentTimeMillis
+      consume(consumer, config.numMessages, 1000, config, totalMessagesRead, totalBytesRead)
+      endMs = System.currentTimeMillis
+    } else {
+      import kafka.consumer.ConsumerConfig
+      val consumerConfig = new ConsumerConfig(config.props)
+      val consumerConnector: ConsumerConnector = Consumer.create(consumerConfig)
+      val topicMessageStreams = consumerConnector.createMessageStreams(Map(config.topic -> config.numThreads))
+      var threadList = List[ConsumerPerfThread]()
+      for ((topic, streamList) <- topicMessageStreams)
+        for (i <- 0 until streamList.length)
+          threadList ::= new ConsumerPerfThread(i, "kafka-zk-consumer-" + i, streamList(i), config, totalMessagesRead, totalBytesRead)
+
+      logger.info("Sleeping for 1 second.")
+      Thread.sleep(1000)
+      logger.info("starting threads")
+      startMs = System.currentTimeMillis
+      for (thread <- threadList)
+        thread.start
+      for (thread <- threadList)
+        thread.join
+      endMs = System.currentTimeMillis - consumerConfig.consumerTimeoutMs 
+    }
+    val elapsedSecs = (endMs - startMs) / 1000.0
     if (!config.showDetailedStats) {
       val totalMBRead = (totalBytesRead.get * 1.0) / (1024 * 1024)
-      println(("%s, %s, %d, %.4f, %.4f, %d, %.4f").format(config.dateFormat.format(startMs), config.dateFormat.format(endMs),
-        config.consumerConfig.fetchMessageMaxBytes, totalMBRead, totalMBRead / elapsedSecs, totalMessagesRead.get,
-        totalMessagesRead.get / elapsedSecs))
+      println(("%s, %s, %.4f, %.4f, %d, %.4f").format(config.dateFormat.format(startMs), config.dateFormat.format(endMs),
+        totalMBRead, totalMBRead / elapsedSecs, totalMessagesRead.get, totalMessagesRead.get / elapsedSecs))
     }
     System.exit(0)
   }
+  
+  def consume(consumer: KafkaConsumer[Array[Byte], Array[Byte]], count: Long, timeout: Long, config: ConsumerPerfConfig, totalMessagesRead: AtomicLong, totalBytesRead: AtomicLong) {
+    var bytesRead = 0L
+    var messagesRead = 0L
+    val startMs = System.currentTimeMillis
+    var lastReportTime: Long = startMs
+    var lastBytesRead = 0L
+    var lastMessagesRead = 0L
+    var lastConsumed = System.currentTimeMillis
+    while(messagesRead < count && lastConsumed >= System.currentTimeMillis - timeout) {
+      val records = consumer.poll(100)
+      if(records.count() > 0)
+        lastConsumed = System.currentTimeMillis
+      for(record <- records) {
+        messagesRead += 1
+        if(record.key != null)
+          bytesRead += record.key.size
+        if(record.value != null)
+          bytesRead += record.value.size 
+      
+        if (messagesRead % config.reportingInterval == 0) {
+          if (config.showDetailedStats)
+            printProgressMessage(0, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, System.currentTimeMillis, config.dateFormat)
+          lastReportTime = System.currentTimeMillis
+          lastMessagesRead = messagesRead
+          lastBytesRead = bytesRead
+        }
+      }
+    }
+    totalMessagesRead.set(messagesRead)
+    totalBytesRead.set(bytesRead)
+  }
+  
+  def printProgressMessage(id: Int, bytesRead: Long, lastBytesRead: Long, messagesRead: Long, lastMessagesRead: Long,
+    startMs: Long, endMs: Long, dateFormat: SimpleDateFormat) = {
+    val elapsedMs: Double = endMs - startMs
+    val totalMBRead = (bytesRead * 1.0) / (1024 * 1024)
+    val mbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024)
+    println(("%s, %d, %.4f, %.4f, %d, %.4f").format(dateFormat.format(endMs), id, totalMBRead,
+        1000.0 * (mbRead / elapsedMs), messagesRead, ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0))
+  }
 
   class ConsumerPerfConfig(args: Array[String]) extends PerfConfig(args) {
-    val zkConnectOpt = parser.accepts("zookeeper", "REQUIRED: The connection string for the zookeeper connection in the form host:port. " +
-      "Multiple URLS can be given to allow fail-over.")
+    val zkConnectOpt = parser.accepts("zookeeper", "The connection string for the zookeeper connection in the form host:port. " +
+      "Multiple URLS can be given to allow fail-over. This option is only used with the old consumer.")
       .withRequiredArg
       .describedAs("urls")
       .ofType(classOf[String])
+    val bootstrapServersOpt = parser.accepts("broker-list", "A broker list to use for connecting if using the new consumer.")
+      .withRequiredArg()
+      .describedAs("host")
+      .ofType(classOf[String])
     val topicOpt = parser.accepts("topic", "REQUIRED: The topic to consume from.")
       .withRequiredArg
       .describedAs("topic")
@@ -117,20 +172,35 @@ object ConsumerPerformance {
       .describedAs("count")
       .ofType(classOf[java.lang.Integer])
       .defaultsTo(1)
+    val useNewConsumerOpt = parser.accepts("new-consumer", "Use the new consumer implementation.")
 
     val options = parser.parse(args: _*)
 
-    CommandLineUtils.checkRequiredArgs(parser, options, topicOpt, zkConnectOpt)
-
+    CommandLineUtils.checkRequiredArgs(parser, options, topicOpt)
+   
+    val useNewConsumer = options.has(useNewConsumerOpt)
+    
     val props = new Properties
-    props.put("group.id", options.valueOf(groupIdOpt))
-    props.put("socket.receive.buffer.bytes", options.valueOf(socketBufferSizeOpt).toString)
-    props.put("fetch.message.max.bytes", options.valueOf(fetchSizeOpt).toString)
-    props.put("auto.offset.reset", if (options.has(resetBeginningOffsetOpt)) "largest" else "smallest")
-    props.put("zookeeper.connect", options.valueOf(zkConnectOpt))
-    props.put("consumer.timeout.ms", "5000")
-    props.put("num.consumer.fetchers", options.valueOf(numFetchersOpt).toString)
-    val consumerConfig = new ConsumerConfig(props)
+    if(useNewConsumer) {
+      import org.apache.kafka.clients.consumer.ConsumerConfig
+      props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, options.valueOf(bootstrapServersOpt))
+      props.put(ConsumerConfig.GROUP_ID_CONFIG, options.valueOf(groupIdOpt))
+      props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, options.valueOf(socketBufferSizeOpt).toString)
+      props.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, options.valueOf(fetchSizeOpt).toString)
+      props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, if (options.has(resetBeginningOffsetOpt)) "latest" else "earliest")
+      props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer])
+      props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer])
+      props.put(ConsumerConfig.CHECK_CRCS_CONFIG, "false")
+    } else {
+      CommandLineUtils.checkRequiredArgs(parser, options, zkConnectOpt)
+      props.put("group.id", options.valueOf(groupIdOpt))
+      props.put("socket.receive.buffer.bytes", options.valueOf(socketBufferSizeOpt).toString)
+      props.put("fetch.message.max.bytes", options.valueOf(fetchSizeOpt).toString)
+      props.put("auto.offset.reset", if (options.has(resetBeginningOffsetOpt)) "largest" else "smallest")
+      props.put("zookeeper.connect", options.valueOf(zkConnectOpt))
+      props.put("consumer.timeout.ms", "1000")
+      props.put("num.consumer.fetchers", options.valueOf(numFetchersOpt).toString)
+    }
     val numThreads = options.valueOf(numThreadsOpt).intValue
     val topic = options.valueOf(topicOpt)
     val numMessages = options.valueOf(numMessagesOpt).longValue
@@ -161,7 +231,7 @@ object ConsumerPerformance {
 
           if (messagesRead % config.reportingInterval == 0) {
             if (config.showDetailedStats)
-              printMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, System.currentTimeMillis)
+              printProgressMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, lastReportTime, System.currentTimeMillis, config.dateFormat)
             lastReportTime = System.currentTimeMillis
             lastMessagesRead = messagesRead
             lastBytesRead = bytesRead
@@ -176,18 +246,9 @@ object ConsumerPerformance {
       totalMessagesRead.addAndGet(messagesRead)
       totalBytesRead.addAndGet(bytesRead)
       if (config.showDetailedStats)
-        printMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, System.currentTimeMillis)
+        printProgressMessage(threadId, bytesRead, lastBytesRead, messagesRead, lastMessagesRead, startMs, System.currentTimeMillis, config.dateFormat)
     }
 
-    private def printMessage(id: Int, bytesRead: Long, lastBytesRead: Long, messagesRead: Long, lastMessagesRead: Long,
-      startMs: Long, endMs: Long) = {
-      val elapsedMs = endMs - startMs
-      val totalMBRead = (bytesRead * 1.0) / (1024 * 1024)
-      val mbRead = ((bytesRead - lastBytesRead) * 1.0) / (1024 * 1024)
-      println(("%s, %d, %d, %.4f, %.4f, %d, %.4f").format(config.dateFormat.format(endMs), id,
-        config.consumerConfig.fetchMessageMaxBytes, totalMBRead,
-        1000.0 * (mbRead / elapsedMs), messagesRead, ((messagesRead - lastMessagesRead) / elapsedMs) * 1000.0))
-    }
   }
 
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala b/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala
index 7602b8d..900f7df 100644
--- a/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala
+++ b/core/src/main/scala/kafka/tools/SimpleConsumerPerformance.scala
@@ -79,7 +79,7 @@ object SimpleConsumerPerformance {
         done = true
       else
         // we only did one fetch so we find the offset for the first (head) messageset
-        offset += messageSet.validBytes
+        offset = messageSet.last.nextOffset
 
       totalBytesRead += bytesRead
       totalMessagesRead += messagesRead

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 9a16343..7ceadcc 100644
--- a/core/src/main/scala/kafka/utils/KafkaScheduler.scala
+++ b/core/src/main/scala/kafka/utils/KafkaScheduler.scala
@@ -41,6 +41,11 @@ trait Scheduler {
   def shutdown()
   
   /**
+   * Check if the scheduler has been started
+   */
+  def isStarted: Boolean
+  
+  /**
    * Schedule a task
    * @param name The name of this task
    * @param delay The amount of time to wait before the first execution
@@ -63,13 +68,13 @@ trait Scheduler {
 class KafkaScheduler(val threads: Int, 
                      val threadNamePrefix: String = "kafka-scheduler-", 
                      daemon: Boolean = true) extends Scheduler with Logging {
-  @volatile private var executor: ScheduledThreadPoolExecutor = null
+  private var executor: ScheduledThreadPoolExecutor = null
   private val schedulerThreadId = new AtomicInteger(0)
   
   override def startup() {
     debug("Initializing task scheduler.")
     this synchronized {
-      if(executor != null)
+      if(isStarted)
         throw new IllegalStateException("This scheduler has already been started!")
       executor = new ScheduledThreadPoolExecutor(threads)
       executor.setContinueExistingPeriodicTasksAfterShutdownPolicy(false)
@@ -83,34 +88,45 @@ class KafkaScheduler(val threads: Int,
   
   override def shutdown() {
     debug("Shutting down task scheduler.")
-    ensureStarted
-    executor.shutdown()
-    executor.awaitTermination(1, TimeUnit.DAYS)
-    this.executor = null
+    this synchronized {
+      if(isStarted) {
+        executor.shutdown()
+        executor.awaitTermination(1, TimeUnit.DAYS)
+        this.executor = null
+      }
+    }
   }
 
   def schedule(name: String, fun: ()=>Unit, delay: Long, period: Long, unit: TimeUnit) = {
     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 = 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))
+    this synchronized {
+      ensureStarted
+      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)
+        executor.scheduleAtFixedRate(runnable, delay, period, unit)
+      else
+        executor.schedule(runnable, delay, unit)
+    }
+  }
+  
+  def isStarted: Boolean = {
+    this synchronized {
+      executor != null
     }
-    if(period >= 0)
-      executor.scheduleAtFixedRate(runnable, delay, period, unit)
-    else
-      executor.schedule(runnable, delay, unit)
   }
   
   private def ensureStarted = {
-    if(executor == null)
+    if(!isStarted)
       throw new IllegalStateException("Kafka scheduler has not been started")
   }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/core/src/test/scala/integration/kafka/api/ConsumerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala
new file mode 100644
index 0000000..798f035
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/api/ConsumerTest.scala
@@ -0,0 +1,286 @@
+/**
+ * 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.KafkaException
+import org.apache.kafka.clients.producer.ProducerConfig
+import org.apache.kafka.clients.producer.ProducerRecord
+import org.apache.kafka.clients.consumer.Consumer
+import org.apache.kafka.clients.consumer.KafkaConsumer
+import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback
+import org.apache.kafka.clients.consumer.ConsumerRecord
+import org.apache.kafka.clients.consumer.ConsumerConfig
+import org.apache.kafka.clients.consumer.CommitType
+import org.apache.kafka.common.serialization.ByteArrayDeserializer
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.clients.consumer.NoOffsetForPartitionException
+import scala.collection.mutable.Buffer
+import scala.collection.JavaConversions._
+import java.util.ArrayList
+import java.util.Arrays
+import org.junit.Assert._
+import kafka.utils.TestUtils
+import kafka.utils.Logging
+import kafka.server.OffsetManager
+
+/**
+ * Integration tests for the new consumer that cover basic usage as well as server failures
+ */
+class ConsumerTest extends IntegrationTestHarness with Logging {
+
+  val producerCount = 1
+  val consumerCount = 2
+  val serverCount = 3
+
+  val topic = "topic"
+  val part = 0
+  val tp = new TopicPartition(topic, part)
+
+  // configure the servers and clients
+  this.serverConfig.setProperty("controlled.shutdown.enable", "false") // speed up shutdown
+  this.serverConfig.setProperty("offsets.topic.replication.factor", "3") // don't want to lose offset
+  this.serverConfig.setProperty("offsets.topic.num.partitions", "1")
+  this.producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "all")
+  this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test")
+  this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString)
+  this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
+  
+  override def setUp() {
+    super.setUp()
+    // this will trigger the creation of the consumer offsets topic
+    this.consumers(0).partitionsFor(OffsetManager.OffsetsTopicName)
+  }
+
+  def testSimpleConsumption() {
+    val numRecords = 10000
+    sendRecords(numRecords)
+
+    assertEquals(0, this.consumers(0).subscriptions.size)
+    this.consumers(0).subscribe(tp)
+    assertEquals(1, this.consumers(0).subscriptions.size)
+    
+    this.consumers(0).seek(tp, 0)
+    consumeRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0)
+  }
+
+  def testAutoOffsetReset() {
+    sendRecords(1)
+    this.consumers(0).subscribe(tp)
+    consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0)
+  }
+
+  def testSeek() {
+    val consumer = this.consumers(0)
+    val totalRecords = 50L
+    sendRecords(totalRecords.toInt)
+    consumer.subscribe(tp)
+
+    consumer.seekToEnd(tp)
+    assertEquals(totalRecords, consumer.position(tp))
+    assertFalse(consumer.poll(totalRecords).iterator().hasNext())
+
+    consumer.seekToBeginning(tp)
+    assertEquals(0, consumer.position(tp), 0)
+    consumeRecords(consumer, numRecords = 1, startingOffset = 0)
+
+    val mid = totalRecords / 2
+    consumer.seek(tp, mid)
+    assertEquals(mid, consumer.position(tp))
+    consumeRecords(consumer, numRecords = 1, startingOffset = mid.toInt)
+  }
+
+  def testGroupConsumption() {
+    // we need to do this test with only one server since we have the hack join group 
+    // that just assigns the partition hosted on the local machine (with two we might get the wrong machine
+    this.servers.last.shutdown()
+    this.servers.head.shutdown()
+    sendRecords(10)
+    this.consumers(0).subscribe(topic)
+    consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0)
+  }
+
+  def testPositionAndCommit() {
+    sendRecords(5)
+
+    // committed() on a partition with no committed offset throws an exception
+    intercept[NoOffsetForPartitionException] {
+      this.consumers(0).committed(new TopicPartition(topic, 15))
+    }
+
+    // position() on a partition that we aren't subscribed to throws an exception
+    intercept[IllegalArgumentException] {
+      this.consumers(0).position(new TopicPartition(topic, 15))
+    }
+
+    this.consumers(0).subscribe(tp)
+
+    assertEquals("position() on a partition that we are subscribed to should reset the offset", 0L, this.consumers(0).position(tp))
+    this.consumers(0).commit(CommitType.SYNC)
+    assertEquals(0L, this.consumers(0).committed(tp))
+
+    consumeRecords(this.consumers(0), 5, 0)
+    assertEquals("After consuming 5 records, position should be 5", 5L, this.consumers(0).position(tp))
+    this.consumers(0).commit(CommitType.SYNC)
+    assertEquals("Committed offset should be returned", 5L, this.consumers(0).committed(tp));
+
+    sendRecords(1)
+
+    // another consumer in the same group should get the same position
+    this.consumers(1).subscribe(tp)
+    consumeRecords(this.consumers(1), 1, 5)
+  }
+
+  def testPartitionsFor() {
+    val numParts = 2;
+    TestUtils.createTopic(this.zkClient, topic, numParts, 1, this.servers)
+    val parts = this.consumers(0).partitionsFor(topic)
+    assertNotNull(parts)
+    assertEquals(2, parts.length)
+    assertNull(this.consumers(0).partitionsFor("non-existant-topic"))
+  }
+
+  def testConsumptionWithBrokerFailures() = consumeWithBrokerFailures(numRecords = 1000)
+
+  /*
+   * 1. Produce a bunch of messages
+   * 2. Then consume the messages while killing and restarting brokers at random
+   */
+  def consumeWithBrokerFailures(numRecords: Int) {
+    TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers)
+    sendRecords(numRecords)
+    this.producers.map(_.close)
+    var consumed = 0
+    val consumer = this.consumers(0)
+    consumer.subscribe(topic)
+    while (consumed < numRecords) {
+      // check that we are getting the messages in order
+      for (record <- consumer.poll(200)) {
+        assertEquals(consumed.toLong, record.offset())
+        consumed += 1
+      }
+      consumer.commit(CommitType.SYNC);
+
+      /* restart any dead brokers, and kill a broker (with probability 1/3) */
+      restartDeadBrokers()
+      if (TestUtils.random.nextInt(3) == 0) {
+        info("Killing broker")
+        killRandomBroker()
+      }
+    }
+  }
+
+  def testSeekAndCommitWithBrokerFailures() = seekAndCommitWithBrokerFailures(20)
+
+  def seekAndCommitWithBrokerFailures(numIters: Int) {
+    // create a topic and send it some data
+    val numRecords = 1000
+    TestUtils.createTopic(this.zkClient, topic, 1, serverCount, this.servers)
+    sendRecords(numRecords)
+    this.producers.map(_.close)
+
+    val consumer = this.consumers(0)
+    consumer.subscribe(tp)
+    consumer.seek(tp, 0)
+    for (iter <- 0 until numIters) {
+      val coin = TestUtils.random.nextInt(4)
+      if (coin == 0) {
+        info("Seeking to end of log")
+        consumer.seekToEnd()
+        assertEquals(1000.toLong, consumer.position(tp))
+      } else if (coin == 1) {
+        val pos = TestUtils.random.nextInt(numRecords).toLong
+        info("Seeking to " + pos)
+        consumer.seek(tp, pos)
+        assertEquals(pos, consumer.position(tp))
+      } else if (coin == 2) {
+        info("Committing offset.")
+        consumer.commit(CommitType.SYNC)
+        assertEquals(consumer.position(tp), consumer.committed(tp))
+      } else {
+        restartDeadBrokers()
+        killRandomBroker()
+      }
+    }
+  }
+  
+  def testPartitionReassignmentCallback() {
+    val callback = new TestConsumerReassignmentCallback()
+    this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "200"); // timeout quickly to avoid slow test
+    val consumer0 = new KafkaConsumer(this.consumerConfig, callback, new ByteArrayDeserializer(), new ByteArrayDeserializer())
+    consumer0.subscribe("test")
+        
+    // the initial subscription should cause a callback execution
+    while(callback.callsToAssigned == 0)
+      consumer0.poll(50)
+    
+    // get metadata for the topic
+    var parts = consumer0.partitionsFor(OffsetManager.OffsetsTopicName)
+    while(parts == null)
+      parts = consumer0.partitionsFor(OffsetManager.OffsetsTopicName)
+    assertEquals(1, parts.size)
+    assertNotNull(parts(0).leader())
+    
+    // shutdown the co-ordinator
+    val coordinator = parts(0).leader().id()
+    this.servers(coordinator).shutdown()
+    
+    // this should cause another callback execution
+    while(callback.callsToAssigned < 2)
+      consumer0.poll(50)
+    assertEquals(2, callback.callsToAssigned)
+    assertEquals(2, callback.callsToRevoked)
+    
+    consumer0.close()
+  }
+  
+  class TestConsumerReassignmentCallback extends ConsumerRebalanceCallback {
+    var callsToAssigned = 0
+    var callsToRevoked = 0
+    def onPartitionsAssigned(consumer: Consumer[_,_], partitions: java.util.Collection[TopicPartition]) {
+      info("onPartitionsAssigned called.")
+      callsToAssigned += 1
+    }
+    def onPartitionsRevoked(consumer: Consumer[_,_], partitions: java.util.Collection[TopicPartition]) {
+      info("onPartitionsRevoked called.")
+      callsToRevoked += 1
+    } 
+  }
+
+  private def sendRecords(numRecords: Int) {
+    val futures = (0 until numRecords).map { i =>
+      this.producers(0).send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes))
+    }
+    futures.map(_.get)
+  }
+
+  private def consumeRecords(consumer: Consumer[Array[Byte], Array[Byte]], numRecords: Int, startingOffset: Int) {
+    val records = new ArrayList[ConsumerRecord[Array[Byte], Array[Byte]]]()
+    val maxIters = numRecords * 300
+    var iters = 0
+    while (records.size < numRecords) {
+      for (record <- consumer.poll(50))
+        records.add(record)
+      if(iters > maxIters)
+        throw new IllegalStateException("Failed to consume the expected records after " + iters + " iterations.");
+      iters += 1
+    }
+    for (i <- 0 until numRecords) {
+      val record = records.get(i)
+      val offset = startingOffset + i
+      assertEquals(topic, record.topic())
+      assertEquals(part, record.partition())
+      assertEquals(offset.toLong, record.offset())
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
new file mode 100644
index 0000000..5650b4a
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
@@ -0,0 +1,73 @@
+/**
+ * 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.clients.producer.ProducerConfig
+import org.apache.kafka.clients.consumer.ConsumerConfig
+import org.scalatest.junit.JUnit3Suite
+import collection._
+import kafka.utils.TestUtils
+import java.util.Properties
+import java.util.Arrays
+import org.apache.kafka.clients.consumer.KafkaConsumer
+import org.apache.kafka.clients.producer.KafkaProducer
+import kafka.server.KafkaConfig
+import kafka.integration.KafkaServerTestHarness
+import scala.collection.mutable.Buffer
+
+/**
+ * A helper class for writing integration tests that involve producers, consumers, and servers
+ */
+trait IntegrationTestHarness extends KafkaServerTestHarness {
+
+  val producerCount: Int
+  val consumerCount: Int
+  val serverCount: Int
+  lazy val producerConfig = new Properties
+  lazy val consumerConfig = new Properties
+  lazy val serverConfig = new Properties
+  override lazy val configs = {
+    val cfgs = TestUtils.createBrokerConfigs(serverCount)
+    cfgs.map(_.putAll(serverConfig))
+    cfgs.map(new KafkaConfig(_))
+  }
+  
+  var consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]()
+  var producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]()
+  
+  override def setUp() {
+    super.setUp()
+    producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapUrl)
+    producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer])
+    producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer])
+    consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, this.bootstrapUrl)
+    consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer])
+    consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer])
+    for(i <- 0 until producerCount)
+      producers += new KafkaProducer(producerConfig)
+    for(i <- 0 until consumerCount)
+      consumers += new KafkaConsumer(consumerConfig)
+  }
+  
+  override def tearDown() {
+    producers.map(_.close())
+    consumers.map(_.close())
+    super.tearDown()
+  }
+
+}


[4/7] kafka git commit: KAFKA-1760: New consumer.

Posted by jk...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java
index fa88ac1..f50da82 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java
@@ -9,185 +9,174 @@
  * 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.clients.consumer;
 
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.ObjectOutputStream;
 import java.util.ArrayList;
-import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
+
+import org.apache.kafka.clients.consumer.internals.SubscriptionState;
 import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.MetricName;
 
 /**
- * A mock of the {@link Consumer} interface you can use for testing code that uses Kafka.
- * This class is <i> not threadsafe </i>
+ * A mock of the {@link Consumer} interface you can use for testing code that uses Kafka. This class is <i> not
+ * threadsafe </i>
  * <p>
- * The consumer runs in the user thread and multiplexes I/O over TCP connections to each of the brokers it
- * needs to communicate with. Failure to close the consumer after use will leak these resources.
+ * The consumer runs in the user thread and multiplexes I/O over TCP connections to each of the brokers it needs to
+ * communicate with. Failure to close the consumer after use will leak these resources.
  */
-public class MockConsumer implements Consumer<byte[], byte[]> {
+public class MockConsumer<K, V> implements Consumer<K, V> {
+
+    private final Map<String, List<PartitionInfo>> partitions;
+    private final SubscriptionState subscriptions;
+    private Map<TopicPartition, List<ConsumerRecord<K, V>>> records;
+    private boolean closed;
 
-    private final Set<TopicPartition> subscribedPartitions;
-    private final Set<String> subscribedTopics;
-    private final Map<TopicPartition, Long> committedOffsets; 
-    private final Map<TopicPartition, Long> consumedOffsets;
-    
     public MockConsumer() {
-        subscribedPartitions = new HashSet<TopicPartition>();
-        subscribedTopics = new HashSet<String>();
-        committedOffsets = new HashMap<TopicPartition, Long>();
-        consumedOffsets = new HashMap<TopicPartition, Long>();
+        this.subscriptions = new SubscriptionState();
+        this.partitions = new HashMap<String, List<PartitionInfo>>();
+        this.records = new HashMap<TopicPartition, List<ConsumerRecord<K, V>>>();
+        this.closed = false;
     }
     
     @Override
-    public void subscribe(String... topics) {
-        if(subscribedPartitions.size() > 0)
-            throw new IllegalStateException("Subcription to topics and partitions is mutually exclusive");
-        for(String topic : topics) {
-            subscribedTopics.add(topic);
-        }
+    public synchronized Set<TopicPartition> subscriptions() {
+        return this.subscriptions.assignedPartitions();
     }
 
     @Override
-    public void subscribe(TopicPartition... partitions) {
-        if(subscribedTopics.size() > 0)
-            throw new IllegalStateException("Subcription to topics and partitions is mutually exclusive");
-        for(TopicPartition partition : partitions) {
-            subscribedPartitions.add(partition);
-            consumedOffsets.put(partition, 0L);
-        }
+    public synchronized void subscribe(String... topics) {
+        ensureNotClosed();
+        for (String topic : topics)
+            this.subscriptions.subscribe(topic);
     }
 
-    public void unsubscribe(String... topics) {
-        // throw an exception if the topic was never subscribed to
-        for(String topic:topics) {
-            if(!subscribedTopics.contains(topic))
-                throw new IllegalStateException("Topic " + topic + " was never subscribed to. subscribe(" + topic + ") should be called prior" +
-                        " to unsubscribe(" + topic + ")");
-            subscribedTopics.remove(topic);
-        }
+    @Override
+    public synchronized void subscribe(TopicPartition... partitions) {
+        ensureNotClosed();
+        for (TopicPartition partition : partitions)
+            this.subscriptions.subscribe(partition);
     }
 
-    public void unsubscribe(TopicPartition... partitions) {
-        // throw an exception if the partition was never subscribed to
-        for(TopicPartition partition:partitions) {
-            if(!subscribedPartitions.contains(partition))
-                throw new IllegalStateException("Partition " + partition + " was never subscribed to. subscribe(new TopicPartition(" + 
-                                                 partition.topic() + "," + partition.partition() + ") should be called prior" +
-                                                " to unsubscribe(new TopicPartition(" + partition.topic() + "," + partition.partition() + ")");
-            subscribedPartitions.remove(partition);                    
-            committedOffsets.remove(partition);
-            consumedOffsets.remove(partition);
-        }
+    public synchronized void unsubscribe(String... topics) {
+        ensureNotClosed();
+        for (String topic : topics)
+            this.subscriptions.unsubscribe(topic);
+    }
+
+    public synchronized void unsubscribe(TopicPartition... partitions) {
+        ensureNotClosed();
+        for (TopicPartition partition : partitions)
+            this.subscriptions.unsubscribe(partition);
     }
 
     @Override
-    public Map<String, ConsumerRecords<byte[], byte[]>> poll(long timeout) {
-        // hand out one dummy record, 1 per topic
-        Map<String, List<ConsumerRecord>> records = new HashMap<String, List<ConsumerRecord>>();
-        Map<String, ConsumerRecords<byte[], byte[]>> recordMetadata = new HashMap<String, ConsumerRecords<byte[], byte[]>>();
-        for(TopicPartition partition : subscribedPartitions) {
-            // get the last consumed offset
-            long messageSequence = consumedOffsets.get(partition);
-            ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
-            ObjectOutputStream outputStream;
-            try {
-                outputStream = new ObjectOutputStream(byteStream);
-                outputStream.writeLong(messageSequence++);
-                outputStream.close();
-            } catch (IOException e) {
-                e.printStackTrace();
-            }
-            List<ConsumerRecord> recordsForTopic = records.get(partition.topic());
-            if(recordsForTopic == null) {
-                recordsForTopic = new ArrayList<ConsumerRecord>();
-                records.put(partition.topic(), recordsForTopic);
-            }
-            recordsForTopic.add(new ConsumerRecord(partition.topic(), partition.partition(), null, byteStream.toByteArray(), messageSequence));
-            consumedOffsets.put(partition, messageSequence);
+    public synchronized ConsumerRecords<K, V> poll(long timeout) {
+        ensureNotClosed();
+        // update the consumed offset
+        for (Map.Entry<TopicPartition, List<ConsumerRecord<K, V>>> entry : this.records.entrySet()) {
+            List<ConsumerRecord<K, V>> recs = entry.getValue();
+            if (!recs.isEmpty())
+                this.subscriptions.consumed(entry.getKey(), recs.get(recs.size() - 1).offset());
         }
-        for(Entry<String, List<ConsumerRecord>> recordsPerTopic : records.entrySet()) {
-            Map<Integer, List<ConsumerRecord>> recordsPerPartition = new HashMap<Integer, List<ConsumerRecord>>();
-            for(ConsumerRecord record : recordsPerTopic.getValue()) {
-                List<ConsumerRecord> recordsForThisPartition = recordsPerPartition.get(record.partition());
-                if(recordsForThisPartition == null) {
-                    recordsForThisPartition = new ArrayList<ConsumerRecord>();
-                    recordsPerPartition.put(record.partition(), recordsForThisPartition);
-                }
-                recordsForThisPartition.add(record);
-            }
-            recordMetadata.put(recordsPerTopic.getKey(), new ConsumerRecords(recordsPerTopic.getKey(), recordsPerPartition));
+
+        ConsumerRecords<K, V> copy = new ConsumerRecords<K, V>(this.records);
+        this.records = new HashMap<TopicPartition, List<ConsumerRecord<K, V>>>();
+        return copy;
+    }
+
+    public synchronized void addRecord(ConsumerRecord<K, V> record) {
+        ensureNotClosed();
+        TopicPartition tp = new TopicPartition(record.topic(), record.partition());
+        this.subscriptions.assignedPartitions().add(tp);
+        List<ConsumerRecord<K, V>> recs = this.records.get(tp);
+        if (recs == null) {
+            recs = new ArrayList<ConsumerRecord<K, V>>();
+            this.records.put(tp, recs);
         }
-        return recordMetadata;
+        recs.add(record);
     }
 
     @Override
-    public OffsetMetadata commit(Map<TopicPartition, Long> offsets, boolean sync) {
-        if(!sync)
-            return null;
-        for(Entry<TopicPartition, Long> partitionOffset : offsets.entrySet()) {
-            committedOffsets.put(partitionOffset.getKey(), partitionOffset.getValue());            
-        }        
-        return new OffsetMetadata(committedOffsets, null);
+    public synchronized void commit(Map<TopicPartition, Long> offsets, CommitType commitType) {
+        ensureNotClosed();
+        for (Entry<TopicPartition, Long> entry : offsets.entrySet())
+            subscriptions.committed(entry.getKey(), entry.getValue());
     }
 
     @Override
-    public OffsetMetadata commit(boolean sync) {
-        if(!sync)
-            return null;
-        return commit(consumedOffsets, sync);
+    public synchronized void commit(CommitType commitType) {
+        ensureNotClosed();
+        commit(this.subscriptions.allConsumed(), commitType);
     }
 
     @Override
-    public void seek(Map<TopicPartition, Long> offsets) {
-        // change the fetch offsets
-        for(Entry<TopicPartition, Long> partitionOffset : offsets.entrySet()) {
-            consumedOffsets.put(partitionOffset.getKey(), partitionOffset.getValue());            
-        }
+    public synchronized void seek(TopicPartition partition, long offset) {
+        ensureNotClosed();
+        subscriptions.seek(partition, offset);
     }
 
     @Override
-    public Map<TopicPartition, Long> committed(Collection<TopicPartition> partitions) {
-        Map<TopicPartition, Long> offsets = new HashMap<TopicPartition, Long>();
-        for(TopicPartition partition : partitions) {
-            offsets.put(new TopicPartition(partition.topic(), partition.partition()), committedOffsets.get(partition));
-        }
-        return offsets;
+    public synchronized long committed(TopicPartition partition) {
+        ensureNotClosed();
+        return subscriptions.committed(partition);
     }
 
     @Override
-    public Map<TopicPartition, Long> position(Collection<TopicPartition> partitions) {
-        Map<TopicPartition, Long> positions = new HashMap<TopicPartition, Long>();
-        for(TopicPartition partition : partitions) {
-            positions.put(partition, consumedOffsets.get(partition));
-        }
-        return positions;
+    public synchronized long position(TopicPartition partition) {
+        ensureNotClosed();
+        return subscriptions.consumed(partition);
+    }
+
+    @Override
+    public synchronized void seekToBeginning(TopicPartition... partitions) {
+        ensureNotClosed();
+        throw new UnsupportedOperationException();
     }
 
     @Override
-    public Map<TopicPartition, Long> offsetsBeforeTime(long timestamp,
-            Collection<TopicPartition> partitions) {
+    public synchronized void seekToEnd(TopicPartition... partitions) {
+        ensureNotClosed();
         throw new UnsupportedOperationException();
     }
 
     @Override
     public Map<MetricName, ? extends Metric> metrics() {
-        return null;
+        ensureNotClosed();
+        return Collections.emptyMap();
     }
 
     @Override
-    public void close() {
-       // unsubscribe from all partitions
-        TopicPartition[] allPartitions = new TopicPartition[subscribedPartitions.size()];
-        unsubscribe(subscribedPartitions.toArray(allPartitions));
+    public synchronized List<PartitionInfo> partitionsFor(String topic) {
+        ensureNotClosed();
+        List<PartitionInfo> parts = this.partitions.get(topic);
+        if (parts == null)
+            return Collections.emptyList();
+        else
+            return parts;
+    }
+
+    public synchronized void updatePartitions(String topic, List<PartitionInfo> partitions) {
+        ensureNotClosed();
+        this.partitions.put(topic, partitions);
+    }
+
+    @Override
+    public synchronized void close() {
+        ensureNotClosed();
+        this.closed = true;
+    }
+
+    private void ensureNotClosed() {
+        if (this.closed)
+            throw new IllegalStateException("This consumer has already been closed.");
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java
new file mode 100644
index 0000000..a21f97b
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/NoOffsetForPartitionException.java
@@ -0,0 +1,29 @@
+/**
+ * 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.clients.consumer;
+
+import org.apache.kafka.common.KafkaException;
+
+/**
+ * Indicates that there is no stored offset and no defined offset reset policy
+ */
+public class NoOffsetForPartitionException extends KafkaException {
+
+    private static final long serialVersionUID = 1L;
+
+    public NoOffsetForPartitionException(String message) {
+        super(message);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java
deleted file mode 100644
index ea423ad..0000000
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetMetadata.java
+++ /dev/null
@@ -1,59 +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 org.apache.kafka.clients.consumer;
-
-import java.util.Map;
-
-import org.apache.kafka.common.TopicPartition;
-
-/**
- * The metadata for an offset commit that has been acknowledged by the server
- */
-public final class OffsetMetadata {
-
-    private final Map<TopicPartition, Long> offsets;
-    private final Map<TopicPartition, RuntimeException> errors;
-    
-    public OffsetMetadata(Map<TopicPartition, Long> offsets, Map<TopicPartition, RuntimeException> errors) {
-        super();
-        this.offsets = offsets;
-        this.errors = errors;
-    }
-
-    public OffsetMetadata(Map<TopicPartition, Long> offsets) {
-        this(offsets, null);
-    }
-
-    /**
-     * The offset of the record in the topic/partition.
-     */
-    public long offset(TopicPartition partition) {
-        if(this.errors != null)
-            throw errors.get(partition);
-        return offsets.get(partition);
-    }
-
-    /**
-     * @return The exception corresponding to the error code returned by the server
-     */
-    public Exception error(TopicPartition partition) {
-        if(errors != null)
-            return errors.get(partition);
-        else
-            return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
new file mode 100644
index 0000000..d9483ec
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Heartbeat.java
@@ -0,0 +1,47 @@
+package org.apache.kafka.clients.consumer.internals;
+
+/**
+ * A helper class for managing the heartbeat to the co-ordinator
+ */
+public final class Heartbeat {
+    
+    /* The number of heartbeats to attempt to complete per session timeout interval.
+     * so, e.g., with a session timeout of 3 seconds we would attempt a heartbeat
+     * once per second.
+     */
+    private final static int HEARTBEATS_PER_SESSION_INTERVAL = 3;
+
+    private final long timeout;
+    private long lastHeartbeatSend;
+    private long lastHeartbeatResponse;
+
+    public Heartbeat(long timeout, long now) {
+        this.timeout = timeout;
+        this.lastHeartbeatSend = now;
+        this.lastHeartbeatResponse = now;
+    }
+
+    public void sentHeartbeat(long now) {
+        this.lastHeartbeatSend = now;
+    }
+
+    public void receivedResponse(long now) {
+        this.lastHeartbeatResponse = now;
+    }
+
+    public void markDead() {
+        this.lastHeartbeatResponse = -1;
+    }
+
+    public boolean isAlive(long now) {
+        return now - lastHeartbeatResponse <= timeout;
+    }
+
+    public boolean shouldHeartbeat(long now) {
+        return now - lastHeartbeatSend > (1.0 / HEARTBEATS_PER_SESSION_INTERVAL) * this.timeout;
+    }
+    
+    public long lastHeartbeatSend() {
+        return this.lastHeartbeatSend;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java
new file mode 100644
index 0000000..7e57a39
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/NoOpConsumerRebalanceCallback.java
@@ -0,0 +1,30 @@
+/**
+ * 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.clients.consumer.internals;
+
+import java.util.Collection;
+
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceCallback;
+import org.apache.kafka.common.TopicPartition;
+
+public class NoOpConsumerRebalanceCallback implements ConsumerRebalanceCallback {
+
+    @Override
+    public void onPartitionsAssigned(Consumer<?,?> consumer, Collection<TopicPartition> partitions) {}
+
+    @Override
+    public void onPartitionsRevoked(Consumer<?,?> consumer, Collection<TopicPartition> partitions) {}
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
new file mode 100644
index 0000000..71ce20d
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java
@@ -0,0 +1,166 @@
+package org.apache.kafka.clients.consumer.internals;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.kafka.common.TopicPartition;
+
+/**
+ * A class for tracking the topics, partitions, and offsets for the consumer
+ */
+public class SubscriptionState {
+
+    /* the list of topics the user has requested */
+    private final Set<String> subscribedTopics;
+
+    /* the list of partitions the user has requested */
+    private final Set<TopicPartition> subscribedPartitions;
+
+    /* the list of partitions currently assigned */
+    private final Set<TopicPartition> assignedPartitions;
+
+    /* the offset exposed to the user */
+    private final Map<TopicPartition, Long> consumed;
+
+    /* the current point we have fetched up to */
+    private final Map<TopicPartition, Long> fetched;
+
+    /* the last committed offset for each partition */
+    private final Map<TopicPartition, Long> committed;
+
+    /* do we need to request a partition assignment from the co-ordinator? */
+    private boolean needsPartitionAssignment;
+
+    public SubscriptionState() {
+        this.subscribedTopics = new HashSet<String>();
+        this.subscribedPartitions = new HashSet<TopicPartition>();
+        this.assignedPartitions = new HashSet<TopicPartition>();
+        this.consumed = new HashMap<TopicPartition, Long>();
+        this.fetched = new HashMap<TopicPartition, Long>();
+        this.committed = new HashMap<TopicPartition, Long>();
+        this.needsPartitionAssignment = false;
+    }
+
+    public void subscribe(String topic) {
+        if (this.subscribedPartitions.size() > 0)
+            throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive");
+        if (!this.subscribedTopics.contains(topic)) {
+            this.subscribedTopics.add(topic);
+            this.needsPartitionAssignment = true;
+        }
+    }
+
+    public void unsubscribe(String topic) {
+        if (!this.subscribedTopics.contains(topic))
+            throw new IllegalStateException("Topic " + topic + " was never subscribed to.");
+        this.subscribedTopics.remove(topic);
+        this.needsPartitionAssignment = true;
+        for(TopicPartition tp: assignedPartitions())
+            if(topic.equals(tp.topic()))
+                clearPartition(tp);
+    }
+
+    public void subscribe(TopicPartition tp) {
+        if (this.subscribedTopics.size() > 0)
+            throw new IllegalStateException("Subcription to topics and partitions are mutually exclusive");
+        this.subscribedPartitions.add(tp);
+        this.assignedPartitions.add(tp);
+    }
+
+    public void unsubscribe(TopicPartition partition) {
+        if (!subscribedPartitions.contains(partition))
+            throw new IllegalStateException("Partition " + partition + " was never subscribed to.");
+        subscribedPartitions.remove(partition);
+        clearPartition(partition);
+    }
+    
+    private void clearPartition(TopicPartition tp) {
+        this.assignedPartitions.remove(tp);
+        this.committed.remove(tp);
+        this.fetched.remove(tp);
+        this.consumed.remove(tp);
+    }
+
+    public void clearAssignment() {
+        this.assignedPartitions.clear();
+        this.committed.clear();
+        this.fetched.clear();
+        this.needsPartitionAssignment = !subscribedTopics().isEmpty();
+    }
+
+    public Set<String> subscribedTopics() {
+        return this.subscribedTopics;
+    }
+
+    public Long fetched(TopicPartition tp) {
+        return this.fetched.get(tp);
+    }
+
+    public void fetched(TopicPartition tp, long offset) {
+        if (!this.assignedPartitions.contains(tp))
+            throw new IllegalArgumentException("Can't change the fetch position for a partition you are not currently subscribed to.");
+        this.fetched.put(tp, offset);
+    }
+
+    public void committed(TopicPartition tp, long offset) {
+        this.committed.put(tp, offset);
+    }
+
+    public Long committed(TopicPartition tp) {
+        return this.committed.get(tp);
+    }
+    
+    public void seek(TopicPartition tp, long offset) {
+        fetched(tp, offset);
+        consumed(tp, offset);
+    }
+
+    public Set<TopicPartition> assignedPartitions() {
+        return this.assignedPartitions;
+    }
+
+    public boolean partitionsAutoAssigned() {
+        return !this.subscribedTopics.isEmpty();
+    }
+
+    public void consumed(TopicPartition tp, long offset) {
+        if (!this.assignedPartitions.contains(tp))
+            throw new IllegalArgumentException("Can't change the consumed position for a partition you are not currently subscribed to.");
+        this.consumed.put(tp, offset);
+    }
+
+    public Long consumed(TopicPartition partition) {
+        return this.consumed.get(partition);
+    }
+
+    public Map<TopicPartition, Long> allConsumed() {
+        return this.consumed;
+    }
+
+    public boolean hasAllFetchPositions() {
+        return this.fetched.size() >= this.assignedPartitions.size();
+    }
+
+    public Set<TopicPartition> missingFetchPositions() {
+        Set<TopicPartition> copy = new HashSet<TopicPartition>(this.assignedPartitions);
+        copy.removeAll(this.fetched.keySet());
+        return copy;
+    }
+
+    public boolean needsPartitionAssignment() {
+        return this.needsPartitionAssignment;
+    }
+
+    public void changePartitionAssignment(List<TopicPartition> assignments) {
+        for (TopicPartition tp : assignments)
+            if (!this.subscribedTopics.contains(tp.topic()))
+                throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic.");
+        this.clearAssignment();
+        this.assignedPartitions.addAll(assignments);
+        this.needsPartitionAssignment = false;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 fc71710..ebc4c53 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
@@ -329,8 +329,7 @@ public class KafkaProducer<K,V> implements Producer<K,V> {
                         " to class " + producerConfig.getClass(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG).getName() +
                         " specified in value.serializer");
             }
-            ProducerRecord<byte[], byte[]> serializedRecord = new ProducerRecord<byte[], byte[]>(record.topic(), record.partition(), serializedKey, serializedValue);
-            int partition = partitioner.partition(serializedRecord, metadata.fetch());
+            int partition = partitioner.partition(record.topic(), serializedKey, record.partition(), metadata.fetch());
             int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(serializedKey, serializedValue);
             ensureValidRecordSize(serializedSize);
             TopicPartition tp = new TopicPartition(record.topic(), partition);

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
index 904976f..84530f2 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
@@ -100,7 +100,7 @@ public class MockProducer implements Producer<byte[], byte[]> {
     public synchronized Future<RecordMetadata> send(ProducerRecord<byte[], byte[]> record, Callback callback) {
         int partition = 0;
         if (this.cluster.partitionsForTopic(record.topic()) != null)
-            partition = partitioner.partition(record, this.cluster);
+            partition = partitioner.partition(record.topic(), record.key(), record.partition(), this.cluster);
         ProduceRequestResult result = new ProduceRequestResult();
         FutureRecordMetadata future = new FutureRecordMetadata(result, 0);
         TopicPartition topicPartition = new TopicPartition(record.topic(), partition);

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
index 8b3e565..9a43d66 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
@@ -16,9 +16,9 @@ import static org.apache.kafka.common.config.ConfigDef.Range.atLeast;
 import static org.apache.kafka.common.config.ConfigDef.Range.between;
 import static org.apache.kafka.common.config.ConfigDef.ValidString.in;
 
-import java.util.Arrays;
 import java.util.Map;
 
+import org.apache.kafka.clients.CommonClientConfigs;
 import org.apache.kafka.common.config.AbstractConfig;
 import org.apache.kafka.common.config.ConfigDef;
 import org.apache.kafka.common.config.ConfigDef.Importance;
@@ -38,23 +38,17 @@ public class ProducerConfig extends AbstractConfig {
     private static final ConfigDef config;
 
     /** <code>bootstrap.servers</code> */
-    public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers";
-    private static final String BOOSTRAP_SERVERS_DOC = "A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. Data will be load " + "balanced over all servers irrespective of which servers are specified here for bootstrapping&mdash;this list only "
-                                                       + "impacts the initial hosts used to discover the full set of servers. This list should be in the form "
-                                                       + "<code>host1:port1,host2:port2,...</code>. Since these servers are just used for the initial connection to "
-                                                       + "discover the full cluster membership (which may change dynamically), this list need not contain the full set of "
-                                                       + "servers (you may want more than one, though, in case a server is down). If no server in this list is available sending "
-                                                       + "data will fail until on becomes available.";
+    public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG;
 
     /** <code>metadata.fetch.timeout.ms</code> */
     public static final String METADATA_FETCH_TIMEOUT_CONFIG = "metadata.fetch.timeout.ms";
-    private static final String METADATA_FETCH_TIMEOUT_DOC = "The first time data is sent to a topic we must fetch metadata about that topic to know which servers host the " + "topic's partitions. This configuration controls the maximum amount of time we will block waiting for the metadata "
+    private static final String METADATA_FETCH_TIMEOUT_DOC = "The first time data is sent to a topic we must fetch metadata about that topic to know which servers host the topic's partitions. This "
                                                              + "fetch to succeed before throwing an exception back to the client.";
 
     /** <code>metadata.max.age.ms</code> */
-    public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms";
-    private static final String METADATA_MAX_AGE_DOC = "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any " + " partition leadership changes to proactively discover any new brokers or partitions.";
-
+    public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG;
+    private static final String METADATA_MAX_AGE_DOC = CommonClientConfigs.METADATA_MAX_AGE_DOC;
+            
     /** <code>batch.size</code> */
     public static final String BATCH_SIZE_CONFIG = "batch.size";
     private static final String BATCH_SIZE_DOC = "The producer will attempt to batch records together into fewer requests whenever multiple records are being sent" + " to the same partition. This helps performance on both the client and the server. This configuration controls the "
@@ -113,17 +107,13 @@ public class ProducerConfig extends AbstractConfig {
                                                 + "for example, would have the effect of reducing the number of requests sent but would add up to 5ms of latency to records sent in the absense of load.";
 
     /** <code>client.id</code> */
-    public static final String CLIENT_ID_CONFIG = "client.id";
-    private static final String CLIENT_ID_DOC = "The id string to pass to the server when making requests. The purpose of this is to be able to track the source " + "of requests beyond just ip/port by allowing a logical application name to be included with the request. The "
-                                                + "application can set any string it wants as this has no functional purpose other than in logging and metrics.";
+    public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG;
 
     /** <code>send.buffer.bytes</code> */
-    public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes";
-    private static final String SEND_BUFFER_DOC = "The size of the TCP send buffer to use when sending data";
+    public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG;
 
     /** <code>receive.buffer.bytes</code> */
-    public static final String RECEIVE_BUFFER_CONFIG = "receive.buffer.bytes";
-    private static final String RECEIVE_BUFFER_DOC = "The size of the TCP receive buffer to use when reading data";
+    public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG;
 
     /** <code>max.request.size</code> */
     public static final String MAX_REQUEST_SIZE_CONFIG = "max.request.size";
@@ -131,8 +121,7 @@ public class ProducerConfig extends AbstractConfig {
                                                        + "batches the producer will send in a single request to avoid sending huge requests.";
 
     /** <code>reconnect.backoff.ms</code> */
-    public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms";
-    private static final String RECONNECT_BACKOFF_MS_DOC = "The amount of time to wait before attempting to reconnect to a given host when a connection fails." + " This avoids a scenario where the client repeatedly attempts to connect to a host in a tight loop.";
+    public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG;
 
     /** <code>block.on.buffer.full</code> */
     public static final String BLOCK_ON_BUFFER_FULL_CONFIG = "block.on.buffer.full";
@@ -147,8 +136,7 @@ public class ProducerConfig extends AbstractConfig {
                                               + "may appear first.";
 
     /** <code>retry.backoff.ms</code> */
-    public static final String RETRY_BACKOFF_MS_CONFIG = "retry.backoff.ms";
-    private static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed produce request to a given topic partition." + " This avoids repeated sending-and-failing in a tight loop.";
+    public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG;
 
     /** <code>compression.type</code> */
     public static final String COMPRESSION_TYPE_CONFIG = "compression.type";
@@ -156,17 +144,13 @@ public class ProducerConfig extends AbstractConfig {
                                                        + "Compression is of full batches of data, so the efficacy of batching will also impact the compression ratio (more batching means better compression).";
 
     /** <code>metrics.sample.window.ms</code> */
-    public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms";
-    private static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The metrics system maintains a configurable number of samples over a fixed window size. This configuration " + "controls the size of the window. For example we might maintain two samples each measured over a 30 second period. "
-                                                               + "When a window expires we erase and overwrite the oldest window.";
+    public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG;
 
     /** <code>metrics.num.samples</code> */
-    public static final String METRICS_NUM_SAMPLES_CONFIG = "metrics.num.samples";
-    private static final String METRICS_NUM_SAMPLES_DOC = "The number of samples maintained to compute metrics.";
+    public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG;
 
     /** <code>metric.reporters</code> */
-    public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters";
-    private static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the <code>MetricReporter</code> interface allows " + "plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics.";
+    public static final String METRIC_REPORTER_CLASSES_CONFIG = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG;
 
     /** <code>max.in.flight.requests.per.connection</code> */
     public static final String MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION = "max.in.flight.requests.per.connection";
@@ -183,22 +167,22 @@ public class ProducerConfig extends AbstractConfig {
     private static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the <code>Serializer</code> interface.";
 
     static {
-        config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, BOOSTRAP_SERVERS_DOC)
+        config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, CommonClientConfigs.BOOSTRAP_SERVERS_DOC)
                                 .define(BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), Importance.HIGH, BUFFER_MEMORY_DOC)
                                 .define(RETRIES_CONFIG, Type.INT, 0, between(0, Integer.MAX_VALUE), Importance.HIGH, RETRIES_DOC)
                                 .define(ACKS_CONFIG,
                                         Type.STRING,
                                         "1",
-                                        in(Arrays.asList("all", "-1", "0", "1")),
+                                        in("all","-1", "0", "1"),
                                         Importance.HIGH,
                                         ACKS_DOC)
                                 .define(COMPRESSION_TYPE_CONFIG, Type.STRING, "none", Importance.HIGH, COMPRESSION_TYPE_DOC)
                                 .define(BATCH_SIZE_CONFIG, Type.INT, 16384, atLeast(0), Importance.MEDIUM, BATCH_SIZE_DOC)
                                 .define(TIMEOUT_CONFIG, Type.INT, 30 * 1000, atLeast(0), Importance.MEDIUM, TIMEOUT_DOC)
                                 .define(LINGER_MS_CONFIG, Type.LONG, 0, atLeast(0L), Importance.MEDIUM, LINGER_MS_DOC)
-                                .define(CLIENT_ID_CONFIG, Type.STRING, "", Importance.MEDIUM, CLIENT_ID_DOC)
-                                .define(SEND_BUFFER_CONFIG, Type.INT, 128 * 1024, atLeast(0), Importance.MEDIUM, SEND_BUFFER_DOC)
-                                .define(RECEIVE_BUFFER_CONFIG, Type.INT, 32 * 1024, atLeast(0), Importance.MEDIUM, RECEIVE_BUFFER_DOC)
+                                .define(CLIENT_ID_CONFIG, Type.STRING, "", Importance.MEDIUM, CommonClientConfigs.CLIENT_ID_DOC)
+                                .define(SEND_BUFFER_CONFIG, Type.INT, 128 * 1024, atLeast(0), Importance.MEDIUM, CommonClientConfigs.SEND_BUFFER_DOC)
+                                .define(RECEIVE_BUFFER_CONFIG, Type.INT, 32 * 1024, atLeast(0), Importance.MEDIUM, CommonClientConfigs.RECEIVE_BUFFER_DOC)
                                 .define(MAX_REQUEST_SIZE_CONFIG,
                                         Type.INT,
                                         1 * 1024 * 1024,
@@ -206,9 +190,9 @@ public class ProducerConfig extends AbstractConfig {
                                         Importance.MEDIUM,
                                         MAX_REQUEST_SIZE_DOC)
                                 .define(BLOCK_ON_BUFFER_FULL_CONFIG, Type.BOOLEAN, true, Importance.LOW, BLOCK_ON_BUFFER_FULL_DOC)
-                                .define(RECONNECT_BACKOFF_MS_CONFIG, Type.LONG, 10L, atLeast(0L), Importance.LOW, RECONNECT_BACKOFF_MS_DOC)
-                                .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, "", Importance.LOW, METRIC_REPORTER_CLASSES_DOC)
-                                .define(RETRY_BACKOFF_MS_CONFIG, Type.LONG, 100L, atLeast(0L), Importance.LOW, RETRY_BACKOFF_MS_DOC)
+                                .define(RECONNECT_BACKOFF_MS_CONFIG, Type.LONG, 50L, atLeast(0L), Importance.LOW, CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC)
+                                .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, "", Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC)
+                                .define(RETRY_BACKOFF_MS_CONFIG, Type.LONG, 100L, atLeast(0L), Importance.LOW, CommonClientConfigs.RETRY_BACKOFF_MS_DOC)
                                 .define(METADATA_FETCH_TIMEOUT_CONFIG,
                                         Type.LONG,
                                         60 * 1000,
@@ -221,8 +205,8 @@ public class ProducerConfig extends AbstractConfig {
                                         30000,
                                         atLeast(0),
                                         Importance.LOW,
-                                        METRICS_SAMPLE_WINDOW_MS_DOC)
-                                .define(METRICS_NUM_SAMPLES_CONFIG, Type.INT, 2, atLeast(1), Importance.LOW, METRICS_NUM_SAMPLES_DOC)
+                                        CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC)
+                                .define(METRICS_NUM_SAMPLES_CONFIG, Type.INT, 2, atLeast(1), Importance.LOW, CommonClientConfigs.METRICS_NUM_SAMPLES_DOC)
                                 .define(MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION,
                                         Type.INT,
                                         5,

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 dcf4658..3aff624 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
@@ -78,9 +78,9 @@ public final class Metadata {
     }
 
     /**
-     * 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
+     * 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 long timeToNextUpdate(long nowMs) {
         long timeToExpire = needUpdate ? 0 : Math.max(this.lastRefreshMs + this.metadataExpireMs - nowMs, 0);
@@ -120,6 +120,15 @@ public final class Metadata {
     }
 
     /**
+     * Add one or more topics to maintain metadata for
+     */
+    public synchronized void addTopics(String... topics) {
+        for (String topic : topics)
+            this.topics.add(topic);
+        requestUpdate();
+    }
+
+    /**
      * Get the list of topics we are currently maintaining metadata for
      */
     public synchronized Set<String> topics() {
@@ -137,6 +146,13 @@ public final class Metadata {
         notifyAll();
         log.debug("Updated cluster metadata version {} to {}", this.version, this.cluster);
     }
+    
+    /**
+     * @return The current metadata version
+     */
+    public synchronized int version() {
+        return this.version;
+    }
 
     /**
      * The last time metadata was updated.

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java
index 483899d..8112e6d 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java
@@ -20,7 +20,6 @@ import java.util.List;
 import java.util.Random;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import org.apache.kafka.clients.producer.ProducerRecord;
 import org.apache.kafka.common.Cluster;
 import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.utils.Utils;
@@ -40,32 +39,34 @@ public class Partitioner {
     /**
      * Compute the partition for the given record.
      * 
-     * @param record The record being sent
+     * @param topic The topic name
+     * @param key The key to partition on (or null if no key)
+     * @param partition The partition to use (or null if none)
      * @param cluster The current cluster metadata
      */
-    public int partition(ProducerRecord<byte[], byte[]> record, Cluster cluster) {
-        List<PartitionInfo> partitions = cluster.partitionsForTopic(record.topic());
+    public int partition(String topic, byte[] key, Integer partition, Cluster cluster) {
+        List<PartitionInfo> partitions = cluster.partitionsForTopic(topic);
         int numPartitions = partitions.size();
-        if (record.partition() != null) {
+        if (partition != null) {
             // they have given us a partition, use it
-            if (record.partition() < 0 || record.partition() >= numPartitions)
-                throw new IllegalArgumentException("Invalid partition given with record: " + record.partition()
+            if (partition < 0 || partition >= numPartitions)
+                throw new IllegalArgumentException("Invalid partition given with record: " + partition
                                                    + " is not in the range [0..."
                                                    + numPartitions
                                                    + "].");
-            return record.partition();
-        } else if (record.key() == null) {
+            return partition;
+        } else if (key == null) {
             // choose the next available node in a round-robin fashion
             for (int i = 0; i < numPartitions; i++) {
-                int partition = Utils.abs(counter.getAndIncrement()) % numPartitions;
-                if (partitions.get(partition).leader() != null)
-                    return partition;
+                int part = Utils.abs(counter.getAndIncrement()) % numPartitions;
+                if (partitions.get(part).leader() != null)
+                    return part;
             }
             // no partitions are available, give a non-available partition
             return Utils.abs(counter.getAndIncrement()) % numPartitions;
         } else {
             // hash the key to choose a partition
-            return Utils.abs(Utils.murmur2(record.key())) % numPartitions;
+            return Utils.abs(Utils.murmur2(key)) % numPartitions;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 ccc03d8..8726809 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
@@ -23,6 +23,7 @@ import java.util.Map;
 import org.apache.kafka.clients.ClientRequest;
 import org.apache.kafka.clients.ClientResponse;
 import org.apache.kafka.clients.KafkaClient;
+import org.apache.kafka.clients.RequestCompletionHandler;
 import org.apache.kafka.common.Cluster;
 import org.apache.kafka.common.Node;
 import org.apache.kafka.common.TopicPartition;
@@ -146,7 +147,8 @@ public class Sender implements Runnable {
     /**
      * Run a single iteration of sending
      * 
-     * @param now The current POSIX time in milliseconds
+     * @param now
+     *            The current POSIX time in milliseconds
      */
     public void run(long now) {
         Cluster cluster = metadata.fetch();
@@ -169,9 +171,12 @@ public class Sender implements Runnable {
         }
 
         // create produce requests
-        Map<Integer, List<RecordBatch>> batches = this.accumulator.drain(cluster, result.readyNodes, this.maxRequestSize, now);
+        Map<Integer, List<RecordBatch>> batches = this.accumulator.drain(cluster,
+                                                                         result.readyNodes,
+                                                                         this.maxRequestSize,
+                                                                         now);
+        sensors.updateProduceRequestMetrics(batches);
         List<ClientRequest> requests = createProduceRequests(batches, now);
-        sensors.updateProduceRequestMetrics(requests);
 
         // If we have any nodes that are ready to send + have sendable data, poll with 0 timeout so this can immediately
         // loop and try sending more data. Otherwise, the timeout is determined by nodes that have partitions with data
@@ -183,18 +188,14 @@ public class Sender implements Runnable {
             log.trace("Created {} produce requests: {}", requests.size(), requests);
             pollTimeout = 0;
         }
+        for (ClientRequest request : requests)
+            client.send(request);
 
         // 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, pollTimeout, now);
-        for (ClientResponse response : responses) {
-            if (response.wasDisconnected())
-                handleDisconnect(response, now);
-            else
-                handleResponse(response, now);
-        }
+        this.client.poll(pollTimeout, now);
     }
 
     /**
@@ -206,45 +207,44 @@ public class Sender implements Runnable {
         this.wakeup();
     }
 
-    private void handleDisconnect(ClientResponse response, long now) {
-        log.trace("Cancelled request {} due to node {} being disconnected", response, response.request().request().destination());
-        int correlation = response.request().request().header().correlationId();
-        @SuppressWarnings("unchecked")
-        Map<TopicPartition, RecordBatch> responseBatches = (Map<TopicPartition, RecordBatch>) response.request().attachment();
-        for (RecordBatch batch : responseBatches.values())
-            completeBatch(batch, Errors.NETWORK_EXCEPTION, -1L, correlation, now);
-    }
-
     /**
      * Handle a produce response
      */
-    private void handleResponse(ClientResponse response, long now) {
+    private void handleProduceResponse(ClientResponse response, Map<TopicPartition, RecordBatch> batches, long now) {
         int correlationId = response.request().request().header().correlationId();
-        log.trace("Received produce response from node {} with correlation id {}",
-                  response.request().request().destination(),
-                  correlationId);
-        @SuppressWarnings("unchecked")
-        Map<TopicPartition, RecordBatch> batches = (Map<TopicPartition, RecordBatch>) response.request().attachment();
-        // if we have a response, parse it
-        if (response.hasResponse()) {
-            ProduceResponse produceResponse = new ProduceResponse(response.responseBody());
-            for (Map.Entry<TopicPartition, ProduceResponse.PartitionResponse> entry : produceResponse.responses().entrySet()) {
-                TopicPartition tp = entry.getKey();
-                ProduceResponse.PartitionResponse partResp = entry.getValue();
-                Errors error = Errors.forCode(partResp.errorCode);
-                RecordBatch batch = batches.get(tp);
-                completeBatch(batch, error, partResp.baseOffset, correlationId, now);
-            }
-            this.sensors.recordLatency(response.request().request().destination(), response.requestLatencyMs());
-        } else {
-            // this is the acks = 0 case, just complete all requests
+        if (response.wasDisconnected()) {
+            log.trace("Cancelled request {} due to node {} being disconnected", response, response.request()
+                                                                                                  .request()
+                                                                                                  .destination());
             for (RecordBatch batch : batches.values())
-                completeBatch(batch, Errors.NONE, -1L, correlationId, now);
+                completeBatch(batch, Errors.NETWORK_EXCEPTION, -1L, correlationId, now);
+        } else {
+            log.trace("Received produce response from node {} with correlation id {}",
+                      response.request().request().destination(),
+                      correlationId);
+            // if we have a response, parse it
+            if (response.hasResponse()) {
+                ProduceResponse produceResponse = new ProduceResponse(response.responseBody());
+                for (Map.Entry<TopicPartition, ProduceResponse.PartitionResponse> entry : produceResponse.responses()
+                                                                                                         .entrySet()) {
+                    TopicPartition tp = entry.getKey();
+                    ProduceResponse.PartitionResponse partResp = entry.getValue();
+                    Errors error = Errors.forCode(partResp.errorCode);
+                    RecordBatch batch = batches.get(tp);
+                    completeBatch(batch, error, partResp.baseOffset, correlationId, now);
+                }
+                this.sensors.recordLatency(response.request().request().destination(), response.requestLatencyMs());
+            } else {
+                // this is the acks = 0 case, just complete all requests
+                for (RecordBatch batch : batches.values())
+                    completeBatch(batch, Errors.NONE, -1L, correlationId, now);
+            }
         }
     }
 
     /**
      * Complete or retry the given batch of records.
+     * 
      * @param batch The record batch
      * @param error The error (or null if none)
      * @param baseOffset The base offset assigned to the records if successful
@@ -294,7 +294,7 @@ public class Sender implements Runnable {
      */
     private ClientRequest produceRequest(long now, int destination, short acks, int timeout, List<RecordBatch> batches) {
         Map<TopicPartition, ByteBuffer> produceRecordsByPartition = new HashMap<TopicPartition, ByteBuffer>(batches.size());
-        Map<TopicPartition, RecordBatch> recordsByPartition = new HashMap<TopicPartition, RecordBatch>(batches.size());
+        final Map<TopicPartition, RecordBatch> recordsByPartition = new HashMap<TopicPartition, RecordBatch>(batches.size());
         for (RecordBatch batch : batches) {
             TopicPartition tp = batch.topicPartition;
             ByteBuffer recordsBuffer = batch.records.buffer();
@@ -303,8 +303,15 @@ public class Sender implements Runnable {
             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);
+        RequestSend send = new RequestSend(destination,
+                                           this.client.nextRequestHeader(ApiKeys.PRODUCE),
+                                           request.toStruct());
+        RequestCompletionHandler callback = new RequestCompletionHandler() {
+            public void onComplete(ClientResponse response) {
+                handleProduceResponse(response, recordsByPartition, time.milliseconds());
+            }
+        };
+        return new ClientRequest(now, acks != 0, send, callback);
     }
 
     /**
@@ -428,44 +435,38 @@ public class Sender implements Runnable {
             }
         }
 
-        public void updateProduceRequestMetrics(List<ClientRequest> requests) {
+        public void updateProduceRequestMetrics(Map<Integer, List<RecordBatch>> batches) {
             long now = time.milliseconds();
-            for (int i = 0; i < requests.size(); i++) {
-                ClientRequest request = requests.get(i);
+            for (List<RecordBatch> nodeBatch : batches.values()) {
                 int records = 0;
-
-                if (request.attachment() != null) {
-                    Map<TopicPartition, RecordBatch> responseBatches = (Map<TopicPartition, RecordBatch>) request.attachment();
-                    for (RecordBatch batch : responseBatches.values()) {
-
-                        // register all per-topic metrics at once
-                        String topic = batch.topicPartition.topic();
-                        maybeRegisterTopicMetrics(topic);
-
-                        // per-topic record send rate
-                        String topicRecordsCountName = "topic." + topic + ".records-per-batch";
-                        Sensor topicRecordCount = Utils.notNull(this.metrics.getSensor(topicRecordsCountName));
-                        topicRecordCount.record(batch.recordCount);
-
-                        // per-topic bytes send rate
-                        String topicByteRateName = "topic." + topic + ".bytes";
-                        Sensor topicByteRate = Utils.notNull(this.metrics.getSensor(topicByteRateName));
-                        topicByteRate.record(batch.records.sizeInBytes());
-
-                        // per-topic compression rate
-                        String topicCompressionRateName = "topic." + topic + ".compression-rate";
-                        Sensor topicCompressionRate = Utils.notNull(this.metrics.getSensor(topicCompressionRateName));
-                        topicCompressionRate.record(batch.records.compressionRate());
-
-                        // global metrics
-                        this.batchSizeSensor.record(batch.records.sizeInBytes(), now);
-                        this.queueTimeSensor.record(batch.drainedMs - batch.createdMs, now);
-                        this.compressionRateSensor.record(batch.records.compressionRate());
-                        this.maxRecordSizeSensor.record(batch.maxRecordSize, now);
-                        records += batch.recordCount;
-                    }
-                    this.recordsPerRequestSensor.record(records, now);
+                for (RecordBatch batch : nodeBatch) {
+                    // register all per-topic metrics at once
+                    String topic = batch.topicPartition.topic();
+                    maybeRegisterTopicMetrics(topic);
+
+                    // per-topic record send rate
+                    String topicRecordsCountName = "topic." + topic + ".records-per-batch";
+                    Sensor topicRecordCount = Utils.notNull(this.metrics.getSensor(topicRecordsCountName));
+                    topicRecordCount.record(batch.recordCount);
+
+                    // per-topic bytes send rate
+                    String topicByteRateName = "topic." + topic + ".bytes";
+                    Sensor topicByteRate = Utils.notNull(this.metrics.getSensor(topicByteRateName));
+                    topicByteRate.record(batch.records.sizeInBytes());
+
+                    // per-topic compression rate
+                    String topicCompressionRateName = "topic." + topic + ".compression-rate";
+                    Sensor topicCompressionRate = Utils.notNull(this.metrics.getSensor(topicCompressionRateName));
+                    topicCompressionRate.record(batch.records.compressionRate());
+
+                    // global metrics
+                    this.batchSizeSensor.record(batch.records.sizeInBytes(), now);
+                    this.queueTimeSensor.record(batch.drainedMs - batch.createdMs, now);
+                    this.compressionRateSensor.record(batch.records.compressionRate());
+                    this.maxRecordSizeSensor.record(batch.maxRecordSize, now);
+                    records += batch.recordCount;
                 }
+                this.recordsPerRequestSensor.record(records, now);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 d3299b9..d7ccbcd 100644
--- a/clients/src/main/java/org/apache/kafka/common/Cluster.java
+++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java
@@ -26,6 +26,7 @@ public final class Cluster {
     private final Map<TopicPartition, PartitionInfo> partitionsByTopicPartition;
     private final Map<String, List<PartitionInfo>> partitionsByTopic;
     private final Map<Integer, List<PartitionInfo>> partitionsByNode;
+    private final Map<Integer, Node> nodesById;
 
     /**
      * Create a new cluster with the given nodes and partitions
@@ -37,6 +38,10 @@ public final class Cluster {
         List<Node> copy = new ArrayList<Node>(nodes);
         Collections.shuffle(copy);
         this.nodes = Collections.unmodifiableList(copy);
+        
+        this.nodesById = new HashMap<Integer, Node>();
+        for(Node node: nodes)
+          this.nodesById.put(node.id(), node);
 
         // index the partitions by topic/partition for quick lookup
         this.partitionsByTopicPartition = new HashMap<TopicPartition, PartitionInfo>(partitions.size());
@@ -97,6 +102,15 @@ public final class Cluster {
     public List<Node> nodes() {
         return this.nodes;
     }
+    
+    /**
+     * Get the node by the node id (or null if no such node exists)
+     * @param id The id of the node
+     * @return The node, or null if no such node exists
+     */
+    public Node nodeById(int id) {
+        return this.nodesById.get(id);
+    }
 
     /**
      * Get the current leader for the given topic-partition

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java
index b15aa2c..28562f9 100644
--- a/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java
+++ b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java
@@ -69,10 +69,10 @@ public class PartitionInfo {
 
     @Override
     public String toString() {
-        return String.format("Partition(topic = %s, partition = %d, leader = %d, replicas = %s, isr = %s",
+        return String.format("Partition(topic = %s, partition = %d, leader = %s, replicas = %s, isr = %s",
                              topic,
                              partition,
-                             leader.id(),
+                             leader == null? "none" : leader.id(),
                              fmtNodeIds(replicas),
                              fmtNodeIds(inSyncReplicas));
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
index 98cb79b..38ce10b 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
@@ -21,6 +21,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.kafka.common.utils.Utils;
+
 /**
  * This class is used for specifying the set of expected configurations, their type, their defaults, their
  * documentation, and any special validation logic used for checking the correctness of the values the user provides.
@@ -292,39 +294,23 @@ public class ConfigDef {
       this.validStrings = validStrings;
     }
 
-    public static ValidString in(List<String> validStrings) {
-      return new ValidString(validStrings);
+    public static ValidString in(String... validStrings) {
+      return new ValidString(Arrays.asList(validStrings));
     }
 
     @Override
     public void ensureValid(String name, Object o) {
-
       String s = (String) o;
-
       if (!validStrings.contains(s)) {
-        throw new ConfigException(name,o,"String must be one of:" +join(validStrings));
+        throw new ConfigException(name,o,"String must be one of: " + Utils.join(validStrings, ", "));
       }
 
     }
 
     public String toString() {
-      return "[" + join(validStrings) + "]";
+      return "[" + Utils.join(validStrings, ", ") + "]";
     }
 
-    private String join(List<String> list)
-    {
-      StringBuilder sb = new StringBuilder();
-      boolean first = true;
-      for (String item : list)
-      {
-        if (first)
-          first = false;
-        else
-          sb.append(",");
-        sb.append(item);
-      }
-      return sb.toString();
-    }
   }
 
     private static class ConfigKey {

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java b/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java
index 7c948b1..a566b90 100644
--- a/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java
+++ b/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java
@@ -22,7 +22,7 @@ import org.apache.kafka.common.KafkaException;
  * Any API exception that is part of the public protocol and should be a subclass of this class and be part of this
  * package.
  */
-public abstract class ApiException extends KafkaException {
+public class ApiException extends KafkaException {
 
     private static final long serialVersionUID = 1L;
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 b68bbf0..b5f8d83 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
@@ -1,18 +1,14 @@
 /**
- * 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.
+ * 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.network;
 
@@ -51,13 +47,17 @@ public interface Selectable {
     public void close();
 
     /**
-     * Initiate any sends provided, and make progress on any other I/O operations in-flight (connections,
-     * disconnections, existing sends, and receives)
+     * Queue the given request for sending in the subsequent {@poll(long)} calls
+     * @param send The request to send
+     */
+    public void send(NetworkSend send);
+
+    /**
+     * Do I/O. Reads, writes, connection establishment, etc.
      * @param timeout The amount of time to block if there is nothing to do
-     * @param sends The new sends to initiate
      * @throws IOException
      */
-    public void poll(long timeout, List<NetworkSend> sends) throws IOException;
+    public void poll(long timeout) throws IOException;
 
     /**
      * The list of sends that completed on the last {@link #poll(long, List) poll()} call.
@@ -81,4 +81,26 @@ public interface Selectable {
      */
     public List<Integer> connected();
 
+    /**
+     * Disable reads from the given connection
+     * @param id The id for the connection
+     */
+    public void mute(int id);
+
+    /**
+     * Re-enable reads from the given connection
+     * @param id The id for the connection
+     */
+    public void unmute(int id);
+
+    /**
+     * Disable reads from all connections
+     */
+    public void muteAll();
+
+    /**
+     * Re-enable reads from all connections
+     */
+    public void unmuteAll();
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 74d695b..e18a769 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
@@ -12,6 +12,7 @@
  */
 package org.apache.kafka.common.network;
 
+import java.io.EOFException;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
@@ -81,6 +82,7 @@ public class Selector implements Selectable {
     private final List<NetworkReceive> completedReceives;
     private final List<Integer> disconnected;
     private final List<Integer> connected;
+    private final List<Integer> failedSends;
     private final Time time;
     private final SelectorMetrics sensors;
     private final String metricGrpPrefix;
@@ -103,6 +105,7 @@ public class Selector implements Selectable {
         this.completedReceives = new ArrayList<NetworkReceive>();
         this.connected = new ArrayList<Integer>();
         this.disconnected = new ArrayList<Integer>();
+        this.failedSends = new ArrayList<Integer>();
         this.sensors = new SelectorMetrics(metrics);
     }
 
@@ -179,10 +182,26 @@ public class Selector implements Selectable {
     }
 
     /**
+     * Queue the given request for sending in the subsequent {@poll(long)} calls
+     * @param send The request to send
+     */
+    public void send(NetworkSend send) {
+        SelectionKey key = keyForId(send.destination());
+        Transmissions transmissions = transmissions(key);
+        if (transmissions.hasSend())
+            throw new IllegalStateException("Attempt to begin a send operation with prior send operation still in progress.");
+        transmissions.send = send;
+        try {
+            key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
+        } catch (CancelledKeyException e) {
+            close(key);
+            this.failedSends.add(send.destination());
+        }
+    }
+
+    /**
      * Do whatever I/O can be done on each connection without blocking. This includes completing connections, completing
      * disconnections, initiating new sends, or making progress on in-progress sends or receives.
-     * <p>
-     * The provided network sends will be started.
      * 
      * When this call is completed the user can check for completed sends, receives, connections or disconnects using
      * {@link #completedSends()}, {@link #completedReceives()}, {@link #connected()}, {@link #disconnected()}. These
@@ -190,29 +209,13 @@ public class Selector implements Selectable {
      * completed I/O.
      * 
      * @param timeout The amount of time to wait, in milliseconds. If negative, wait indefinitely.
-     * @param sends The list of new sends to begin
-     * 
      * @throws IllegalStateException If a send is given for which we have no existing connection or for which there is
      *         already an in-progress send
      */
     @Override
-    public void poll(long timeout, List<NetworkSend> sends) throws IOException {
+    public void poll(long timeout) throws IOException {
         clear();
 
-        /* register for write interest on any new sends */
-        for (NetworkSend send : sends) {
-            SelectionKey key = keyForId(send.destination());
-            Transmissions transmissions = transmissions(key);
-            if (transmissions.hasSend())
-                throw new IllegalStateException("Attempt to begin a send operation with prior send operation still in progress.");
-            transmissions.send = send;
-            try {
-                key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
-            } catch (CancelledKeyException e) {
-                close(key);
-            }
-        }
-
         /* check ready keys */
         long startSelect = time.nanoseconds();
         int readyKeys = select(timeout);
@@ -266,21 +269,34 @@ public class Selector implements Selectable {
                     }
 
                     /* cancel any defunct sockets */
-                    if (!key.isValid())
+                    if (!key.isValid()) {
                         close(key);
+                        this.disconnected.add(transmissions.id);
+                    }
                 } catch (IOException e) {
-                    InetAddress remoteAddress = null;
-                    Socket socket = channel.socket();
-                    if (socket != null)
-                        remoteAddress = socket.getInetAddress();
-                    log.warn("Error in I/O with {}", remoteAddress , e);
+                    String desc = socketDescription(channel);
+                    if(e instanceof EOFException)
+                        log.info("Connection {} disconnected", desc);
+                    else
+                        log.warn("Error in I/O with connection to {}", desc, e);
                     close(key);
+                    this.disconnected.add(transmissions.id);
                 }
             }
         }
         long endIo = time.nanoseconds();
         this.sensors.ioTime.record(endIo - endSelect, time.milliseconds());
     }
+    
+    private String socketDescription(SocketChannel channel) {
+        Socket socket = channel.socket();
+        if(socket == null)
+            return "[unconnected socket]";
+        else if(socket.getInetAddress() != null)
+            return socket.getInetAddress().toString();
+        else
+            return socket.getLocalAddress().toString();
+    }
 
     @Override
     public List<NetworkSend> completedSends() {
@@ -302,6 +318,36 @@ public class Selector implements Selectable {
         return this.connected;
     }
 
+    @Override
+    public void mute(int id) {
+        mute(this.keyForId(id));
+    }
+
+    private void mute(SelectionKey key) {
+        key.interestOps(key.interestOps() & ~SelectionKey.OP_READ);
+    }
+
+    @Override
+    public void unmute(int id) {
+        unmute(this.keyForId(id));
+    }
+
+    private void unmute(SelectionKey key) {
+        key.interestOps(key.interestOps() | SelectionKey.OP_READ);
+    }
+
+    @Override
+    public void muteAll() {
+        for (SelectionKey key : this.keys.values())
+            mute(key);
+    }
+
+    @Override
+    public void unmuteAll() {
+        for (SelectionKey key : this.keys.values())
+            unmute(key);
+    }
+
     /**
      * Clear the results from the prior poll
      */
@@ -310,6 +356,8 @@ public class Selector implements Selectable {
         this.completedReceives.clear();
         this.connected.clear();
         this.disconnected.clear();
+        this.disconnected.addAll(this.failedSends);
+        this.failedSends.clear();
     }
 
     /**
@@ -335,7 +383,6 @@ public class Selector implements Selectable {
         SocketChannel channel = channel(key);
         Transmissions trans = transmissions(key);
         if (trans != null) {
-            this.disconnected.add(trans.id);
             this.keys.remove(trans.id);
             trans.clearReceive();
             trans.clearSend();

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
index 3316b6a..a8deac4 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
@@ -19,36 +19,62 @@ package org.apache.kafka.common.protocol;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.kafka.common.errors.*;
-
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.errors.InvalidTopicException;
+import org.apache.kafka.common.errors.LeaderNotAvailableException;
+import org.apache.kafka.common.errors.NetworkException;
+import org.apache.kafka.common.errors.NotEnoughReplicasAfterAppendException;
+import org.apache.kafka.common.errors.NotEnoughReplicasException;
+import org.apache.kafka.common.errors.NotLeaderForPartitionException;
+import org.apache.kafka.common.errors.OffsetMetadataTooLarge;
+import org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.errors.RecordBatchTooLargeException;
+import org.apache.kafka.common.errors.RecordTooLargeException;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
 
 /**
  * This class contains all the client-server errors--those errors that must be sent from the server to the client. These
  * are thus part of the protocol. The names can be changed but the error code cannot.
- *
+ * 
  * Do not add exceptions that occur only on the client or only on the server here.
  */
 public enum Errors {
     UNKNOWN(-1, new UnknownServerException("The server experienced an unexpected error when processing the request")),
     NONE(0, null),
-    OFFSET_OUT_OF_RANGE(1, new OffsetOutOfRangeException("The requested offset is not within the range of offsets maintained by the server.")),
-    CORRUPT_MESSAGE(2, new CorruptRecordException("The message contents does not match the message CRC or the message is otherwise corrupt.")),
-    UNKNOWN_TOPIC_OR_PARTITION(3, new UnknownTopicOrPartitionException("This server does not host this topic-partition.")),
+    OFFSET_OUT_OF_RANGE(1,
+            new OffsetOutOfRangeException("The requested offset is not within the range of offsets maintained by the server.")),
+    CORRUPT_MESSAGE(2,
+            new CorruptRecordException("The message contents does not match the message CRC or the message is otherwise corrupt.")),
+    UNKNOWN_TOPIC_OR_PARTITION(3,
+            new UnknownTopicOrPartitionException("This server does not host this topic-partition.")),
     // TODO: errorCode 4 for InvalidFetchSize
-    LEADER_NOT_AVAILABLE(5, new LeaderNotAvailableException("There is no leader for this topic-partition as we are in the middle of a leadership election.")),
-    NOT_LEADER_FOR_PARTITION(6, new NotLeaderForPartitionException("This server is not the leader for that topic-partition.")),
+    LEADER_NOT_AVAILABLE(5,
+            new LeaderNotAvailableException("There is no leader for this topic-partition as we are in the middle of a leadership election.")),
+    NOT_LEADER_FOR_PARTITION(6,
+            new NotLeaderForPartitionException("This server is not the leader for that topic-partition.")),
     REQUEST_TIMED_OUT(7, new TimeoutException("The request timed out.")),
-    // TODO: errorCode 8, 9, 11
-    MESSAGE_TOO_LARGE(10, new RecordTooLargeException("The request included a message larger than the max message size the server will accept.")),
+    MESSAGE_TOO_LARGE(10,
+            new RecordTooLargeException("The request included a message larger than the max message size the server will accept.")),
     OFFSET_METADATA_TOO_LARGE(12, new OffsetMetadataTooLarge("The metadata field of the offset request was too large.")),
     NETWORK_EXCEPTION(13, new NetworkException("The server disconnected before a response was received.")),
-    // TODO: errorCode 14, 15, 16
-    INVALID_TOPIC_EXCEPTION(17, new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")),
-    RECORD_LIST_TOO_LARGE(18, new RecordBatchTooLargeException("The request included message batch larger than the configured segment size on the server.")),
-    NOT_ENOUGH_REPLICAS(19, new NotEnoughReplicasException("Messages are rejected since there are fewer in-sync replicas than required.")),
-    NOT_ENOUGH_REPLICAS_AFTER_APPEND(20, new NotEnoughReplicasAfterAppendException("Messages are written to the log, but to fewer in-sync replicas than required."));
+    OFFSET_LOAD_IN_PROGRESS(14, new ApiException("The coordinator is loading offsets and can't process requests.")),
+    CONSUMER_COORDINATOR_NOT_AVAILABLE(15, new ApiException("The coordinator is not available.")),
+    NOT_COORDINATOR_FOR_CONSUMER(16, new ApiException("This is not the correct co-ordinator for this consumer.")),
+    INVALID_TOPIC_EXCEPTION(17,
+            new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")),
+    RECORD_LIST_TOO_LARGE(18,
+            new RecordBatchTooLargeException("The request included message batch larger than the configured segment size on the server.")),
+    NOT_ENOUGH_REPLICAS(19,
+            new NotEnoughReplicasException("Messages are rejected since there are fewer in-sync replicas than required.")),
+    NOT_ENOUGH_REPLICAS_AFTER_APPEND(20,
+            new NotEnoughReplicasAfterAppendException("Messages are written to the log, but to fewer in-sync replicas than required."));
+
     private static Map<Class<?>, Errors> classToError = new HashMap<Class<?>, Errors>();
     private static Map<Short, Errors> codeToError = new HashMap<Short, Errors>();
+
     static {
         for (Errors error : Errors.values()) {
             codeToError.put(error.code(), error);
@@ -84,8 +110,9 @@ public enum Errors {
      * Throw the exception corresponding to this error if there is one
      */
     public void maybeThrow() {
-        if (exception != null)
+        if (exception != null) {
             throw this.exception;
+        }
     }
 
     /**


[7/7] kafka git commit: KAFKA-1760: New consumer.

Posted by jk...@apache.org.
KAFKA-1760: New consumer.


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

Branch: refs/heads/trunk
Commit: 0699ff2ce60abb466cab5315977a224f1a70a4da
Parents: 11ec9bf
Author: Jay Kreps <ja...@gmail.com>
Authored: Sun Jan 11 11:29:48 2015 -0800
Committer: Jay Kreps <ja...@gmail.com>
Committed: Thu Jan 29 02:55:35 2015 -0800

----------------------------------------------------------------------
 build.gradle                                    |    1 +
 .../org/apache/kafka/clients/ClientRequest.java |   19 +-
 .../kafka/clients/ClusterConnectionStates.java  |   30 +-
 .../kafka/clients/CommonClientConfigs.java      |   58 +
 .../apache/kafka/clients/ConnectionState.java   |    2 +-
 .../org/apache/kafka/clients/KafkaClient.java   |   44 +-
 .../org/apache/kafka/clients/NetworkClient.java |  138 +-
 .../kafka/clients/NodeConnectionState.java      |   31 -
 .../kafka/clients/RequestCompletionHandler.java |   23 +
 .../kafka/clients/consumer/CommitType.java      |    5 +
 .../apache/kafka/clients/consumer/Consumer.java |  109 +-
 .../kafka/clients/consumer/ConsumerConfig.java  |  296 ++-
 .../consumer/ConsumerRebalanceCallback.java     |   89 +-
 .../kafka/clients/consumer/ConsumerRecord.java  |   89 +-
 .../kafka/clients/consumer/ConsumerRecords.java |  107 +-
 .../kafka/clients/consumer/KafkaConsumer.java   | 1865 +++++++++++++-----
 .../kafka/clients/consumer/MockConsumer.java    |  233 ++-
 .../consumer/NoOffsetForPartitionException.java |   29 +
 .../kafka/clients/consumer/OffsetMetadata.java  |   59 -
 .../clients/consumer/internals/Heartbeat.java   |   47 +
 .../NoOpConsumerRebalanceCallback.java          |   30 +
 .../consumer/internals/SubscriptionState.java   |  166 ++
 .../kafka/clients/producer/KafkaProducer.java   |    3 +-
 .../kafka/clients/producer/MockProducer.java    |    2 +-
 .../kafka/clients/producer/ProducerConfig.java  |   64 +-
 .../clients/producer/internals/Metadata.java    |   22 +-
 .../clients/producer/internals/Partitioner.java |   27 +-
 .../clients/producer/internals/Sender.java      |  155 +-
 .../java/org/apache/kafka/common/Cluster.java   |   14 +
 .../org/apache/kafka/common/PartitionInfo.java  |    4 +-
 .../apache/kafka/common/config/ConfigDef.java   |   26 +-
 .../kafka/common/errors/ApiException.java       |    2 +-
 .../apache/kafka/common/network/Selectable.java |   58 +-
 .../apache/kafka/common/network/Selector.java   |   99 +-
 .../apache/kafka/common/protocol/Errors.java    |   59 +-
 .../kafka/common/protocol/types/Struct.java     |   46 +-
 .../apache/kafka/common/record/LogEntry.java    |    4 +
 .../kafka/common/record/MemoryRecords.java      |   48 +-
 .../requests/ConsumerMetadataRequest.java       |    9 +-
 .../requests/ConsumerMetadataResponse.java      |   17 +-
 .../kafka/common/requests/FetchRequest.java     |   74 +-
 .../kafka/common/requests/FetchResponse.java    |   21 +-
 .../kafka/common/requests/HeartbeatRequest.java |   13 +-
 .../common/requests/HeartbeatResponse.java      |    9 +-
 .../kafka/common/requests/JoinGroupRequest.java |   17 +-
 .../common/requests/JoinGroupResponse.java      |   25 +-
 .../common/requests/ListOffsetRequest.java      |   25 +-
 .../common/requests/ListOffsetResponse.java     |   19 +-
 .../kafka/common/requests/MetadataRequest.java  |    9 +-
 .../kafka/common/requests/MetadataResponse.java |   33 +-
 .../common/requests/OffsetCommitRequest.java    |   45 +-
 .../common/requests/OffsetCommitResponse.java   |   17 +-
 .../common/requests/OffsetFetchRequest.java     |   17 +-
 .../common/requests/OffsetFetchResponse.java    |   46 +-
 .../kafka/common/requests/ProduceRequest.java   |   21 +-
 .../kafka/common/requests/ProduceResponse.java  |   19 +-
 .../common/serialization/Deserializer.java      |    2 +-
 .../org/apache/kafka/common/utils/Utils.java    |   42 +
 .../org/apache/kafka/clients/MockClient.java    |   31 +-
 .../apache/kafka/clients/NetworkClientTest.java |   35 +-
 .../clients/consumer/ConsumerExampleTest.java   |  297 ---
 .../clients/consumer/MockConsumerTest.java      |   32 +
 .../internals/SubscriptionStateTest.java        |   61 +
 .../kafka/clients/producer/BufferPoolTest.java  |    4 +-
 .../clients/producer/MockProducerTest.java      |    2 +-
 .../kafka/clients/producer/PartitionerTest.java |   49 +-
 .../kafka/clients/producer/SenderTest.java      |    6 +-
 .../kafka/common/config/ConfigDefTest.java      |    6 +-
 .../kafka/common/network/SelectorTest.java      |   53 +-
 .../apache/kafka/common/utils/UtilsTest.java    |   10 +
 .../org/apache/kafka/test/MockSelector.java     |   52 +-
 .../kafka/api/ConsumerMetadataRequest.scala     |    2 +-
 .../kafka/api/ConsumerMetadataResponse.scala    |    2 +-
 .../main/scala/kafka/cluster/Partition.scala    |    2 +-
 .../controller/ControllerChannelManager.scala   |    2 +-
 core/src/main/scala/kafka/log/LogConfig.scala   |    4 +-
 .../src/main/scala/kafka/server/KafkaApis.scala |   29 +
 .../scala/kafka/server/ReplicaManager.scala     |    2 +-
 .../scala/kafka/tools/ConsoleConsumer.scala     |    2 -
 .../scala/kafka/tools/ConsumerPerformance.scala |  169 +-
 .../kafka/tools/SimpleConsumerPerformance.scala |    2 +-
 .../main/scala/kafka/utils/KafkaScheduler.scala |   56 +-
 .../integration/kafka/api/ConsumerTest.scala    |  286 +++
 .../kafka/api/IntegrationTestHarness.scala      |   73 +
 .../api/RequestResponseSerializationTest.scala  |    4 +-
 .../integration/KafkaServerTestHarness.scala    |   39 +-
 .../kafka/integration/PrimitiveApiTest.scala    |   19 +-
 .../scala/unit/kafka/utils/MockScheduler.scala  |    2 +
 .../scala/unit/kafka/utils/SchedulerTest.scala  |   17 +
 .../test/scala/unit/kafka/utils/TestUtils.scala |   34 +-
 90 files changed, 4002 insertions(+), 1965 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/build.gradle
----------------------------------------------------------------------
diff --git a/build.gradle b/build.gradle
index a980f61..6844372 100644
--- a/build.gradle
+++ b/build.gradle
@@ -369,6 +369,7 @@ project(':clients') {
   }
 
   javadoc {
+    include "**/org/apache/kafka/clients/consumer/*"
     include "**/org/apache/kafka/clients/producer/*"
     include "**/org/apache/kafka/common/*"
     include "**/org/apache/kafka/common/errors/*"

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java
index d32c319..ed4c0d9 100644
--- a/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java
+++ b/clients/src/main/java/org/apache/kafka/clients/ClientRequest.java
@@ -22,24 +22,25 @@ public final class ClientRequest {
     private final long createdMs;
     private final boolean expectResponse;
     private final RequestSend request;
-    private final Object attachment;
+    private final RequestCompletionHandler callback;
 
     /**
      * @param createdMs The unix timestamp in milliseconds for the time at which this request was created.
      * @param expectResponse Should we expect a response message or is this request complete once it is sent?
      * @param request The request
-     * @param attachment Associated data with the request
+     * @param callback A callback to execute when the response has been received (or null if no callback is necessary)
      */
-    public ClientRequest(long createdMs, boolean expectResponse, RequestSend request, Object attachment) {
+    public ClientRequest(long createdMs, boolean expectResponse, RequestSend request, RequestCompletionHandler callback) {
         this.createdMs = createdMs;
-        this.attachment = attachment;
+        this.callback = callback;
         this.request = request;
         this.expectResponse = expectResponse;
     }
 
     @Override
     public String toString() {
-        return "ClientRequest(expectResponse=" + expectResponse + ", payload=" + attachment + ", request=" + request + ")";
+        return "ClientRequest(expectResponse=" + expectResponse + ", callback=" + callback + ", request=" + request
+                + ")";
     }
 
     public boolean expectResponse() {
@@ -50,8 +51,12 @@ public final class ClientRequest {
         return request;
     }
 
-    public Object attachment() {
-        return attachment;
+    public boolean hasCallback() {
+        return callback != null;
+    }
+
+    public RequestCompletionHandler callback() {
+        return callback;
     }
 
     public long createdTime() {

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
index 8aece7e..574287d 100644
--- a/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
+++ b/clients/src/main/java/org/apache/kafka/clients/ClusterConnectionStates.java
@@ -119,16 +119,42 @@ final class ClusterConnectionStates {
     public void disconnected(int node) {
         nodeState(node).state = ConnectionState.DISCONNECTED;
     }
-
+    
     /**
-     * Get the state of our connection to the given state
+     * Get the state of our connection to the given node
      * @param node The id of the node
      * @return The state of our connection
      */
+    public ConnectionState connectionState(int node) {
+        return nodeState(node).state;
+    }
+    
+    /**
+     * Get the state of a given node
+     * @param node The node to fetch the state for
+     */
     private NodeConnectionState nodeState(int node) {
         NodeConnectionState state = this.nodeState.get(node);
         if (state == null)
             throw new IllegalStateException("No entry found for node " + node);
         return state;
     }
+    
+    /**
+     * The state of our connection to a node
+     */
+    private static class NodeConnectionState {
+
+        ConnectionState state;
+        long lastConnectAttemptMs;
+
+        public NodeConnectionState(ConnectionState state, long lastConnectAttempt) {
+            this.state = state;
+            this.lastConnectAttemptMs = lastConnectAttempt;
+        }
+
+        public String toString() {
+            return "NodeState(" + state + ", " + lastConnectAttemptMs + ")";
+        }
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java
new file mode 100644
index 0000000..06fcfe6
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java
@@ -0,0 +1,58 @@
+/**
+ * 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.clients;
+
+/**
+ * Some configurations shared by both producer and consumer
+ */
+public class CommonClientConfigs {
+    
+    /*
+     * NOTE: DO NOT CHANGE EITHER CONFIG NAMES AS THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE.
+     */
+
+    public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers";
+    public static final String BOOSTRAP_SERVERS_DOC = "A list of host/port pairs to use for establishing the initial connection to the Kafka cluster. The client will make use of all servers irrespective of which servers are specified here for bootstrapping&mdash;this list only impacts the initial hosts used to discover the full set of servers. This list should be in the form "
+                                                       + "<code>host1:port1,host2:port2,...</code>. Since these servers are just used for the initial connection to "
+                                                       + "discover the full cluster membership (which may change dynamically), this list need not contain the full set of "
+                                                       + "servers (you may want more than one, though, in case a server is down).";
+    
+    public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms";
+    public static final String METADATA_MAX_AGE_DOC = "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any partition leadership changes to proactively discover any new brokers or partitions.";
+    
+    public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes";
+    public static final String SEND_BUFFER_DOC = "The size of the TCP send buffer (SO_SNDBUF) to use when sending data.";
+
+    public static final String RECEIVE_BUFFER_CONFIG = "receive.buffer.bytes";
+    public static final String RECEIVE_BUFFER_DOC = "The size of the TCP receive buffer (SO_RCVBUF) to use when reading data.";
+
+    public static final String CLIENT_ID_CONFIG = "client.id";
+    public static final String CLIENT_ID_DOC = "An id string to pass to the server when making requests. The purpose of this is to be able to track the source of requests beyond just ip/port by allowing a logical application name to be included in server-side request logging.";
+
+    public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms";
+    public static final String RECONNECT_BACKOFF_MS_DOC = "The amount of time to wait before attempting to reconnect to a given host. This avoids repeatedly connecting to a host in a tight loop. This backoff applies to all requests sent by the consumer to the broker.";
+
+    public static final String RETRY_BACKOFF_MS_CONFIG = "retry.backoff.ms";
+    public static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed fetch request to a given topic partition. This avoids repeated fetching-and-failing in a tight loop.";
+    
+    public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms";
+    public static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The number of samples maintained to compute metrics.";
+
+    public static final String METRICS_NUM_SAMPLES_CONFIG = "metrics.num.samples";
+    public static final String METRICS_NUM_SAMPLES_DOC = "The number of samples maintained to compute metrics.";
+
+    public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters";
+    public static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the <code>MetricReporter</code> interface allows plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics.";
+    
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java b/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java
index ab7e322..3867f8e 100644
--- a/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java
+++ b/clients/src/main/java/org/apache/kafka/clients/ConnectionState.java
@@ -15,6 +15,6 @@ package org.apache.kafka.clients;
 /**
  * The states of a node connection
  */
-enum ConnectionState {
+public enum ConnectionState {
     DISCONNECTED, CONNECTING, CONNECTED
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java
index 3976955..8a3e55a 100644
--- a/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/KafkaClient.java
@@ -26,6 +26,7 @@ public interface KafkaClient {
     /**
      * Check if we are currently ready to send another request to the given node but don't attempt to connect if we
      * aren't.
+     * 
      * @param node The node to check
      * @param now The current timestamp
      */
@@ -34,6 +35,7 @@ public interface KafkaClient {
     /**
      * Initiate a connection to the given node (if necessary), and return true if already connected. The readiness of a
      * node will change only when poll is invoked.
+     * 
      * @param node The node to connect to.
      * @param now The current time
      * @return true iff we are ready to immediately initiate the sending of another request to the given node.
@@ -44,6 +46,7 @@ public interface KafkaClient {
      * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When
      * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled
      * connections.
+     * 
      * @param node The node to check
      * @param now The current timestamp
      * @return The number of milliseconds to wait.
@@ -51,19 +54,44 @@ public interface KafkaClient {
     public long connectionDelay(Node node, long now);
 
     /**
-     * Initiate the sending of the given requests and return any completed responses. Requests can only be sent on ready
-     * connections.
-     * @param requests The requests to send
+     * Queue up the given request for sending. Requests can only be sent on ready connections.
+     * 
+     * @param request The request
+     * @param now The current time
+     */
+    public void send(ClientRequest request);
+
+    /**
+     * Do actual reads and writes from sockets.
+     * 
      * @param timeout The maximum amount of time to wait for responses in ms
      * @param now The current time in ms
      * @throws IllegalStateException If a request is sent to an unready node
      */
-    public List<ClientResponse> poll(List<ClientRequest> requests, long timeout, long now);
+    public List<ClientResponse> poll(long timeout, long now);
+
+    /**
+     * Complete all in-flight requests for a given node
+     * 
+     * @param node The node to complete requests for
+     * @param now The current time in ms
+     * @return All requests that complete during this time period.
+     */
+    public List<ClientResponse> completeAll(int node, long now);
+
+    /**
+     * Complete all in-flight requests
+     * 
+     * @param now The current time in ms
+     * @return All requests that complete during this time period.
+     */
+    public List<ClientResponse> completeAll(long now);
 
     /**
      * Choose the node with the fewest outstanding requests. This method will prefer a node with an existing connection,
      * but will potentially choose a node for which we don't yet have a connection if all existing connections are in
      * use.
+     * 
      * @param now The current time in ms
      * @return The node with the fewest in-flight requests.
      */
@@ -75,7 +103,15 @@ public interface KafkaClient {
     public int inFlightRequestCount();
 
     /**
+     * Get the total in-flight requests for a particular node
+     * 
+     * @param nodeId The id of the node
+     */
+    public int inFlightRequestCount(int nodeId);
+
+    /**
      * Generate a request header for the next request
+     * 
      * @param key The API key of the request
      */
     public RequestHeader nextRequestHeader(ApiKeys key);

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 6746275..5950191 100644
--- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
@@ -102,6 +102,7 @@ public class NetworkClient implements KafkaClient {
 
     /**
      * Begin connecting to the given node, return true if we are already connected and ready to send to that node.
+     * 
      * @param node The node to check
      * @param now The current timestamp
      * @return True if we are ready to send to the given node
@@ -122,6 +123,7 @@ public class NetworkClient implements KafkaClient {
      * Returns the number of milliseconds to wait, based on the connection state, before attempting to send data. When
      * disconnected, this respects the reconnect backoff time. When connecting or connected, this handles slow/stalled
      * connections.
+     * 
      * @param node The node to check
      * @param now The current timestamp
      * @return The number of milliseconds to wait.
@@ -133,7 +135,8 @@ public class NetworkClient implements KafkaClient {
 
     /**
      * Check if the node with the given id is ready to send more requests.
-     * @param node The given node id
+     * 
+     * @param node The node
      * @param now The current time in ms
      * @return true if the node is ready
      */
@@ -141,7 +144,8 @@ public class NetworkClient implements KafkaClient {
     public boolean isReady(Node 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
+            // 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
@@ -150,6 +154,7 @@ public class NetworkClient implements KafkaClient {
 
     /**
      * Are we connected and ready and able to send more requests to the given node?
+     * 
      * @param node The node
      */
     private boolean isSendable(int node) {
@@ -157,49 +162,106 @@ public class NetworkClient implements KafkaClient {
     }
 
     /**
-     * Initiate the given requests and check for any new responses, waiting up to the specified time. Requests can only
-     * be sent for ready nodes.
-     * @param requests The requests to initiate
+     * Return the state of the connection to the given node
+     * 
+     * @param node The node to check
+     * @return The connection state
+     */
+    public ConnectionState connectionState(int node) {
+        return connectionStates.connectionState(node);
+    }
+
+    /**
+     * Queue up the given request for sending. Requests can only be sent out to ready nodes.
+     * 
+     * @param request The request
+     * @param now The current time
+     */
+    @Override
+    public void send(ClientRequest request) {
+        int nodeId = request.request().destination();
+        if (!isSendable(nodeId))
+            throw new IllegalStateException("Attempt to send a request to node " + nodeId + " which is not ready.");
+
+        this.inFlightRequests.add(request);
+        selector.send(request.request());
+    }
+
+    /**
+     * Do actual reads and writes to sockets.
+     * 
      * @param timeout The maximum amount of time to wait (in ms) for responses if there are none immediately
      * @param now The current time in milliseconds
      * @return The list of responses received
      */
     @Override
-    public List<ClientResponse> poll(List<ClientRequest> requests, long timeout, long now) {
-        List<NetworkSend> sends = new ArrayList<NetworkSend>();
-
-        for (int i = 0; i < requests.size(); i++) {
-            ClientRequest request = requests.get(i);
-            int nodeId = request.request().destination();
-            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());
-        }
-
+    public List<ClientResponse> poll(long timeout, long now) {
         // should we update our metadata?
         long timeToNextMetadataUpdate = metadata.timeToNextUpdate(now);
         long timeToNextReconnectAttempt = Math.max(this.lastNoNodeAvailableMs + metadata.refreshBackoff() - now, 0);
         long waitForMetadataFetch = (this.metadataFetchInProgress ? Integer.MAX_VALUE : 0);
         // if there is no node available to connect, back off refreshing metadata
-        long metadataTimeout = Math.max(Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt), waitForMetadataFetch);
+        long metadataTimeout = Math.max(Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt),
+                                        waitForMetadataFetch);
         if (!this.metadataFetchInProgress && metadataTimeout == 0)
-            maybeUpdateMetadata(sends, now);
-
+            maybeUpdateMetadata(now);
         // do the I/O
         try {
-            this.selector.poll(Math.min(timeout, metadataTimeout), sends);
+            this.selector.poll(Math.min(timeout, metadataTimeout));
         } catch (IOException e) {
             log.error("Unexpected error during I/O in producer network thread", e);
         }
 
+        // process completed actions
         List<ClientResponse> responses = new ArrayList<ClientResponse>();
         handleCompletedSends(responses, now);
         handleCompletedReceives(responses, now);
         handleDisconnections(responses, now);
         handleConnections();
 
+        // invoke callbacks
+        for (ClientResponse response : responses) {
+            if (response.request().hasCallback()) {
+                try {
+                    response.request().callback().onComplete(response);
+                } catch (Exception e) {
+                    log.error("Uncaught error in request completion:", e);
+                }
+            }
+        }
+
+        return responses;
+    }
+
+    /**
+     * Await all the outstanding responses for requests on the given connection
+     * 
+     * @param node The node to block on
+     * @param now The current time in ms
+     * @return All the collected responses
+     */
+    @Override
+    public List<ClientResponse> completeAll(int node, long now) {
+        try {
+            this.selector.muteAll();
+            this.selector.unmute(node);
+            List<ClientResponse> responses = new ArrayList<ClientResponse>();
+            while (inFlightRequestCount(node) > 0)
+                responses.addAll(poll(Integer.MAX_VALUE, now));
+            return responses;
+        } finally {
+            this.selector.unmuteAll();
+        }
+    }
+
+    /**
+     * Wait for all outstanding requests to complete.
+     */
+    @Override
+    public List<ClientResponse> completeAll(long now) {
+        List<ClientResponse> responses = new ArrayList<ClientResponse>();
+        while (inFlightRequestCount() > 0)
+            responses.addAll(poll(Integer.MAX_VALUE, now));
         return responses;
     }
 
@@ -212,7 +274,16 @@ public class NetworkClient implements KafkaClient {
     }
 
     /**
+     * Get the number of in-flight requests for a given node
+     */
+    @Override
+    public int inFlightRequestCount(int nodeId) {
+        return this.inFlightRequests.inFlightRequestCount(nodeId);
+    }
+
+    /**
      * Generate a request header for the given API key
+     * 
      * @param key The api key
      * @return A request header with the appropriate client id and correlation id
      */
@@ -242,6 +313,7 @@ public class NetworkClient implements KafkaClient {
      * prefer a node with an existing connection, but will potentially choose a node for which we don't yet have a
      * connection if all existing connections are in use. This method will never choose a node for which there is no
      * existing connection and from which we have disconnected within the reconnect backoff period.
+     * 
      * @return The node with the fewest in-flight requests.
      */
     public Node leastLoadedNode(long now) {
@@ -261,12 +333,12 @@ public class NetworkClient implements KafkaClient {
                 found = node;
             }
         }
-
         return found;
     }
 
     /**
      * Handle any completed request send. In particular if no response is expected consider the request complete.
+     * 
      * @param responses The list of responses to update
      * @param now The current time
      */
@@ -283,6 +355,7 @@ public class NetworkClient implements KafkaClient {
 
     /**
      * Handle any completed receives and update the response list with the responses received.
+     * 
      * @param responses The list of responses to update
      * @param now The current time
      */
@@ -317,6 +390,7 @@ public class NetworkClient implements KafkaClient {
 
     /**
      * Handle any disconnected connections
+     * 
      * @param responses The list of responses that completed with the disconnection
      * @param now The current time
      */
@@ -353,10 +427,8 @@ public class NetworkClient implements KafkaClient {
      */
     private void correlate(RequestHeader requestHeader, ResponseHeader responseHeader) {
         if (requestHeader.correlationId() != responseHeader.correlationId())
-            throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId() +
-                                            ") does not match request (" +
-                                            requestHeader.correlationId() +
-                                            ")");
+            throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId()
+                    + ") does not match request (" + requestHeader.correlationId() + ")");
     }
 
     /**
@@ -371,7 +443,7 @@ public class NetworkClient implements KafkaClient {
     /**
      * Add a metadata request to the list of sends if we can make one
      */
-    private void maybeUpdateMetadata(List<NetworkSend> sends, long now) {
+    private void maybeUpdateMetadata(long now) {
         // Beware that the behavior of this method and the computation of timeouts for poll() are
         // highly dependent on the behavior of leastLoadedNode.
         Node node = this.leastLoadedNode(now);
@@ -382,17 +454,16 @@ public class NetworkClient implements KafkaClient {
             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;
             ClientRequest metadataRequest = metadataRequest(now, node.id(), topics);
             log.debug("Sending metadata request {} to node {}", metadataRequest, node.id());
-            sends.add(metadataRequest.request());
+            this.selector.send(metadataRequest.request());
             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("Init connection to node {} for sending metadata request in the next iteration", node.id());
+            log.debug("Initialize connection to node {} for sending metadata request", node.id());
             initiateConnect(node, now);
             // If initiateConnect failed immediately, this node will be put into blackout and we
             // should allow immediately retrying in case there is another candidate node. If it
@@ -412,7 +483,10 @@ public class NetworkClient implements KafkaClient {
         try {
             log.debug("Initiating connection to node {} at {}:{}.", node.id(), node.host(), node.port());
             this.connectionStates.connecting(node.id(), now);
-            selector.connect(node.id(), new InetSocketAddress(node.host(), node.port()), this.socketSendBuffer, this.socketReceiveBuffer);
+            selector.connect(node.id(),
+                             new InetSocketAddress(node.host(), node.port()),
+                             this.socketSendBuffer,
+                             this.socketReceiveBuffer);
         } catch (IOException e) {
             /* attempt failed, we'll try again after the backoff */
             connectionStates.disconnected(node.id());

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java b/clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java
deleted file mode 100644
index 752a979..0000000
--- a/clients/src/main/java/org/apache/kafka/clients/NodeConnectionState.java
+++ /dev/null
@@ -1,31 +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 org.apache.kafka.clients;
-
-/**
- * The state of our connection to a node
- */
-final class NodeConnectionState {
-
-    ConnectionState state;
-    long lastConnectAttemptMs;
-
-    public NodeConnectionState(ConnectionState state, long lastConnectAttempt) {
-        this.state = state;
-        this.lastConnectAttemptMs = lastConnectAttempt;
-    }
-
-    public String toString() {
-        return "NodeState(" + state + ", " + lastConnectAttemptMs + ")";
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java b/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java
new file mode 100644
index 0000000..6fee4e4
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/RequestCompletionHandler.java
@@ -0,0 +1,23 @@
+/**
+ * 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.clients;
+
+/**
+ * A callback interface for attaching an action to be executed when a request is complete and the corresponding response
+ * has been received. This handler will also be invoked if there is a disconnection while handling the request.
+ */
+public interface RequestCompletionHandler {
+
+    public void onComplete(ClientResponse response);
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java b/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java
new file mode 100644
index 0000000..072cc2e
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/CommitType.java
@@ -0,0 +1,5 @@
+package org.apache.kafka.clients.consumer;
+
+public enum CommitType {
+    SYNC, ASYNC
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java
index c0c636b..8f587bc 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java
@@ -9,14 +9,16 @@
  * 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.clients.consumer;
 
 import java.io.Closeable;
-import java.util.Collection;
+import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.MetricName;
 
@@ -24,102 +26,85 @@ import org.apache.kafka.common.MetricName;
  * @see KafkaConsumer
  * @see MockConsumer
  */
-public interface Consumer<K,V> extends Closeable {
+public interface Consumer<K, V> extends Closeable {
+    
+    /**
+     * @see KafkaConsumer#subscriptions()
+     */
+    public Set<TopicPartition> subscriptions();
 
     /**
-     * Incrementally subscribe to the given list of topics. This API is mutually exclusive to 
-     * {@link #subscribe(TopicPartition...) subscribe(partitions)} 
-     * @param topics A variable list of topics that the consumer subscribes to
-     */ 
-    public void subscribe(String...topics);
+     * @see KafkaConsumer#subscribe(String...)
+     */
+    public void subscribe(String... topics);
 
     /**
-     * Incrementally subscribes to a specific topic and partition. This API is mutually exclusive to 
-     * {@link #subscribe(String...) subscribe(topics)}
-     * @param partitions Partitions to subscribe to
-     */ 
+     * @see KafkaConsumer#subscribe(TopicPartition...)
+     */
     public void subscribe(TopicPartition... partitions);
 
     /**
-     * Unsubscribe from the specific topics. Messages for this topic will not be returned from the next {@link #poll(long) poll()}
-     * onwards. This should be used in conjunction with {@link #subscribe(String...) subscribe(topics)}. It is an error to
-     * unsubscribe from a topic that was never subscribed to using {@link #subscribe(String...) subscribe(topics)} 
-     * @param topics Topics to unsubscribe from
+     * @see KafkaConsumer#unsubscribe(String...)
      */
     public void unsubscribe(String... topics);
 
     /**
-     * Unsubscribe from the specific topic partitions. Messages for these partitions will not be returned from the next 
-     * {@link #poll(long) poll()} onwards. This should be used in conjunction with 
-     * {@link #subscribe(TopicPartition...) subscribe(topic, partitions)}. It is an error to
-     * unsubscribe from a partition that was never subscribed to using {@link #subscribe(TopicPartition...) subscribe(partitions)}
-     * @param partitions Partitions to unsubscribe from
+     * @see KafkaConsumer#unsubscribe(TopicPartition...)
      */
     public void unsubscribe(TopicPartition... partitions);
-    
+
     /**
-     * Fetches data for the subscribed list of topics and partitions
-     * @param timeout  The time, in milliseconds, spent waiting in poll if data is not available. If 0, waits indefinitely. Must not be negative
-     * @return Map of topic to records for the subscribed topics and partitions as soon as data is available for a topic partition. Availability
-     *         of data is controlled by {@link ConsumerConfig#FETCH_MIN_BYTES_CONFIG} and {@link ConsumerConfig#FETCH_MAX_WAIT_MS_CONFIG}.
-     *         If no data is available for timeout ms, returns an empty list
+     * @see KafkaConsumer#poll(long)
      */
-    public Map<String, ConsumerRecords<K,V>> poll(long timeout);
+    public ConsumerRecords<K, V> poll(long timeout);
 
     /**
-     * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions.
-     * @param sync If true, the commit should block until the consumer receives an acknowledgment 
-     * @return An {@link OffsetMetadata} object that contains the partition, offset and a corresponding error code. Returns null
-     * if the sync flag is set to false 
+     * @see KafkaConsumer#commit(CommitType)
      */
-    public OffsetMetadata commit(boolean sync);
+    public void commit(CommitType commitType);
 
     /**
-     * Commits the specified offsets for the specified list of topics and partitions to Kafka.
-     * @param offsets The map of offsets to commit for the given topic partitions
-     * @param sync If true, commit will block until the consumer receives an acknowledgment 
-     * @return An {@link OffsetMetadata} object that contains the partition, offset and a corresponding error code. Returns null
-     * if the sync flag is set to false. 
+     * @see KafkaConsumer#commit(Map, CommitType)
      */
-    public OffsetMetadata commit(Map<TopicPartition, Long> offsets, boolean sync);
-    
+    public void commit(Map<TopicPartition, Long> offsets, CommitType commitType);
+
     /**
-     * Overrides the fetch positions that the consumer will use on the next fetch request. If the consumer subscribes to a list of topics
-     * using {@link #subscribe(String...) subscribe(topics)}, an exception will be thrown if the specified topic partition is not owned by
-     * the consumer.  
-     * @param offsets The map of fetch positions per topic and partition
+     * @see KafkaConsumer#seek(TopicPartition, long)
      */
-    public void seek(Map<TopicPartition, Long> offsets);
+    public void seek(TopicPartition partition, long offset);
 
     /**
-     * Returns the fetch position of the <i>next message</i> for the specified topic partition to be used on the next {@link #poll(long) poll()}
-     * @param partitions Partitions for which the fetch position will be returned
-     * @return The position from which data will be fetched for the specified partition on the next {@link #poll(long) poll()}
+     * @see KafkaConsumer#seekToBeginning(TopicPartition...)
      */
-    public Map<TopicPartition, Long> position(Collection<TopicPartition> partitions);
-    
+    public void seekToBeginning(TopicPartition... partitions);
+
     /**
-     * Fetches the last committed offsets for the input list of partitions 
-     * @param partitions The list of partitions to return the last committed offset for
-     * @return  The list of offsets for the specified list of partitions
+     * @see KafkaConsumer#seekToEnd(TopicPartition...)
      */
-    public Map<TopicPartition, Long> committed(Collection<TopicPartition> partitions);
-    
+    public void seekToEnd(TopicPartition... partitions);
+
+    /**
+     * @see KafkaConsumer#position(TopicPartition)
+     */
+    public long position(TopicPartition partition);
+
     /**
-     * Fetches offsets before a certain timestamp
-     * @param timestamp The unix timestamp. Value -1 indicates earliest available timestamp. Value -2 indicates latest available timestamp. 
-     * @param partitions The list of partitions for which the offsets are returned
-     * @return The offsets for messages that were written to the server before the specified timestamp.
+     * @see KafkaConsumer#committed(TopicPartition)
      */
-    public Map<TopicPartition, Long> offsetsBeforeTime(long timestamp, Collection<TopicPartition> partitions);
+    public long committed(TopicPartition partition);
 
     /**
-     * Return a map of metrics maintained by the consumer
+     * @see KafkaConsumer#metrics()
      */
     public Map<MetricName, ? extends Metric> metrics();
 
     /**
-     * Close this consumer
+     * @see KafkaConsumer#partitionsFor(String)
+     */
+    public List<PartitionInfo> partitionsFor(String topic);
+
+    /**
+     * @see KafkaConsumer#close()
      */
     public void close();
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
index 57c1807..6d4ff7c 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
@@ -9,13 +9,16 @@
  * 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.clients.consumer;
 
 import static org.apache.kafka.common.config.ConfigDef.Range.atLeast;
+import static org.apache.kafka.common.config.ConfigDef.ValidString.in;
 
 import java.util.Map;
 
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceCallback;
 import org.apache.kafka.common.config.AbstractConfig;
 import org.apache.kafka.common.config.ConfigDef;
 import org.apache.kafka.common.config.ConfigDef.Importance;
@@ -27,130 +30,121 @@ import org.apache.kafka.common.config.ConfigDef.Type;
 public class ConsumerConfig extends AbstractConfig {
     private static final ConfigDef config;
 
-    /**
-     * The identifier of the group this consumer belongs to. This is required if the consumer uses either the
-     * group management functionality by using {@link Consumer#subscribe(String...) subscribe(topics)}. This is also required
-     * if the consumer uses the default Kafka based offset management strategy. 
+    /*
+     * NOTE: DO NOT CHANGE EITHER CONFIG STRINGS OR THEIR JAVA VARIABLE NAMES AS
+     * THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE.
      */
-    public static final String GROUP_ID_CONFIG = "group.id";
-    
+
     /**
-     * The timeout after which, if the {@link Consumer#poll(long) poll(timeout)} is not invoked, the consumer is
-     * marked dead and a rebalance operation is triggered for the group identified by {@link #GROUP_ID_CONFIG}. Relevant 
-     * if the consumer uses the group management functionality by invoking {@link Consumer#subscribe(String...) subscribe(topics)} 
+     * <code>group.id</code>
      */
-    public static final String SESSION_TIMEOUT_MS = "session.timeout.ms";
+    public static final String GROUP_ID_CONFIG = "group.id";
+    private static final String GROUP_ID_DOC = "A unique string that identifies the consumer group this consumer belongs to. This property is required if the consumer uses either the group management functionality by using <code>subscribe(topic)</code> or the Kafka-based offset management strategy.";
 
     /**
-     * The number of times a consumer sends a heartbeat to the co-ordinator broker within a {@link #SESSION_TIMEOUT_MS} time window.
-     * This frequency affects the latency of a rebalance operation since the co-ordinator broker notifies a consumer of a rebalance 
-     * in the heartbeat response. Relevant if the consumer uses the group management functionality by invoking 
-     * {@link Consumer#subscribe(String...) subscribe(topics)} 
+     * <code>session.timeout.ms</code>
      */
-    public static final String HEARTBEAT_FREQUENCY = "heartbeat.frequency";
+    public static final String SESSION_TIMEOUT_MS_CONFIG = "session.timeout.ms";
+    private static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect failures when using Kafka's group management facilities.";
 
     /**
-     * A list of URLs to use for establishing the initial connection to the cluster. This list should be in the form
-     * <code>host1:port1,host2:port2,...</code>. These urls are just used for the initial connection to discover the
-     * full cluster membership (which may change dynamically) so this list need not contain the full set of servers (you
-     * may want more than one, though, in case a server is down).
+     * <code>bootstrap.servers</code>
      */
-    public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers";
+    public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG;
 
     /**
-     * If true, periodically commit to Kafka the offsets of messages already returned by the consumer. This committed 
-     * offset will be used when the process fails as the position from which the consumption will begin.
+     * <code>enable.auto.commit</code>
      */
     public static final String ENABLE_AUTO_COMMIT_CONFIG = "enable.auto.commit";
-    
+    private static final String ENABLE_AUTO_COMMIT_DOC = "If true the consumer's offset will be periodically committed in the background.";
+
     /**
-     * The friendly name of the partition assignment strategy that the server will use to distribute partition ownership
-     * amongst consumer instances when group management is used
+     * <code>auto.commit.interval.ms</code>
      */
-    public static final String PARTITION_ASSIGNMENT_STRATEGY = "partition.assignment.strategy";
-    
+    public static final String AUTO_COMMIT_INTERVAL_MS_CONFIG = "auto.commit.interval.ms";
+    private static final String AUTO_COMMIT_INTERVAL_MS_DOC = "The frequency in milliseconds that the consumer offsets are auto-committed to Kafka if <code>enable.auto.commit</code> is set to <code>true</code>.";
+
     /**
-     * The frequency in milliseconds that the consumer offsets are committed to Kafka. Relevant if {@link #ENABLE_AUTO_COMMIT_CONFIG}
-     * is turned on.
+     * <code>partition.assignment.strategy</code>
      */
-    public static final String AUTO_COMMIT_INTERVAL_MS_CONFIG = "auto.commit.interval.ms";
-    
+    public static final String PARTITION_ASSIGNMENT_STRATEGY_CONFIG = "partition.assignment.strategy";
+    private static final String PARTITION_ASSIGNMENT_STRATEGY_DOC = "The friendly name of the partition assignment strategy that the server will use to distribute partition ownership amongst consumer instances when group management is used";
+
     /**
-     * What to do when there is no initial offset in Kafka or if an offset is out of range:
-     * <ul>
-     * <li> smallest:      automatically reset the offset to the smallest offset
-     * <li> largest:       automatically reset the offset to the largest offset
-     * <li> disable:       throw exception to the consumer if no previous offset is found for the consumer's group
-     * <li> anything else: throw exception to the consumer. 
-     * </ul> 
+     * <code>auto.offset.reset</code>
      */
     public static final String AUTO_OFFSET_RESET_CONFIG = "auto.offset.reset";
-    
+    private static final String AUTO_OFFSET_RESET_DOC = "What to do when there is no initial offset in Kafka or if the current offset does not exist any more on the server (e.g. because that data has been deleted): <ul><li>smallest: automatically reset the offset to the smallest offset<li>largest: automatically reset the offset to the largest offset</li><li>none: throw exception to the consumer if no previous offset is found for the consumer's group</li><li>anything else: throw exception to the consumer.</li></ul>";
+
     /**
-     * The minimum amount of data the server should return for a fetch request. If insufficient data is available the 
-     * request will wait for that much data to accumulate before answering the request.
+     * <code>fetch.min.bytes</code>
      */
     public static final String FETCH_MIN_BYTES_CONFIG = "fetch.min.bytes";
-    
+    private static final String FETCH_MIN_BYTES_DOC = "The minimum amount of data the server should return for a fetch request. If insufficient data is available the request will wait for that much data to accumulate before answering the request. The default setting of 1 byte means that fetch requests are answered as soon as a single byte of data is available or the fetch request times out waiting for data to arrive. Setting this to something greater than 1 will cause the server to wait for larger amounts of data to accumulate which can improve server throughput a bit at the cost of some additional latency.";
+
     /**
-     * The maximum amount of time the server will block before answering the fetch request if there isn't sufficient 
-     * data to immediately satisfy {@link #FETCH_MIN_BYTES_CONFIG}. This should be less than or equal to the timeout used in 
-     * {@link KafkaConsumer#poll(long) poll(timeout)}
+     * <code>fetch.max.wait.ms</code>
      */
     public static final String FETCH_MAX_WAIT_MS_CONFIG = "fetch.max.wait.ms";
-    
+    private static final String FETCH_MAX_WAIT_MS_DOC = "The maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy the requirement given by fetch.min.bytes.";
+
+    /** <code>metadata.max.age.ms</code> */
+    public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG;
+
     /**
-     * The maximum amount of time to block waiting to fetch metadata about a topic the first time a record is received 
-     * from that topic. The consumer will throw a TimeoutException if it could not successfully fetch metadata within
-     * this timeout.
+     * <code>max.partition.fetch.bytes</code>
      */
-    public static final String METADATA_FETCH_TIMEOUT_CONFIG = "metadata.fetch.timeout.ms";
+    public static final String MAX_PARTITION_FETCH_BYTES_CONFIG = "max.partition.fetch.bytes";
+    private static final String MAX_PARTITION_FETCH_BYTES_DOC = "The maximum amount of data per-partition the server will return. The maximum total memory used for a request will be <code>#partitions * max.partition.fetch.bytes</code>. This size must be at least as large as the maximum message size the server allows or else it is possible for the producer to send messages larger than the consumer can fetch. If that happens, the consumer can get stuck trying to fetch a large message on a certain partition.";
+
+    /** <code>send.buffer.bytes</code> */
+    public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG;
+
+    /** <code>receive.buffer.bytes</code> */
+    public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG;
 
     /**
-     * The total memory used by the consumer to buffer records received from the server. This config is meant to control
-     * the consumer's memory usage, so it is the size of the global fetch buffer that will be shared across all partitions. 
+     * <code>client.id</code>
      */
-    public static final String TOTAL_BUFFER_MEMORY_CONFIG = "total.memory.bytes";
+    public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG;
 
     /**
-     * The minimum amount of memory that should be used to fetch at least one message for a partition. This puts a lower
-     * bound on the consumer's memory utilization when there is at least one message for a partition available on the server.
-     * This size must be at least as large as the maximum message size the server allows or else it is possible for the producer 
-     * to send messages larger than the consumer can fetch. If that happens, the consumer can get stuck trying to fetch a large 
-     * message on a certain partition. 
+     * <code>reconnect.backoff.ms</code>
      */
-    public static final String FETCH_BUFFER_CONFIG = "fetch.buffer.bytes";
-    
+    public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG;
+
     /**
-     * The id string to pass to the server when making requests. The purpose of this is to be able to track the source
-     * of requests beyond just ip/port by allowing a logical application name to be included.
+     * <code>retry.backoff.ms</code>
      */
-    public static final String CLIENT_ID_CONFIG = "client.id";
+    public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG;
 
     /**
-     * The size of the TCP send buffer to use when fetching data
+     * <code>metrics.sample.window.ms</code>
      */
-    public static final String SOCKET_RECEIVE_BUFFER_CONFIG = "socket.receive.buffer.bytes";
+    public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG;
 
     /**
-     * The amount of time to wait before attempting to reconnect to a given host. This avoids repeatedly connecting to a
-     * host in a tight loop. This backoff applies to all requests sent by the consumer to the broker.
+     * <code>metrics.num.samples</code>
      */
-    public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms";
+    public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG;
 
-    /** <code>metrics.sample.window.ms</code> */
-    public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms";
-    private static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The metrics system maintains a configurable number of samples over a fixed window size. This configuration " + "controls the size of the window. For example we might maintain two samples each measured over a 30 second period. "
-                                                               + "When a window expires we erase and overwrite the oldest window.";
-
-    /** <code>metrics.num.samples</code> */
-    public static final String METRICS_NUM_SAMPLES_CONFIG = "metrics.num.samples";
-    private static final String METRICS_NUM_SAMPLES_DOC = "The number of samples maintained to compute metrics.";
+    /**
+     * <code>metric.reporters</code>
+     */
+    public static final String METRIC_REPORTER_CLASSES_CONFIG = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG;
 
-    /** <code>metric.reporters</code> */
-    public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters";
-    private static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the <code>MetricReporter</code> interface allows " + "plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics.";
+    /**
+     * <code>rebalance.callback.class</code>
+     */
+    public static final String CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG = "rebalance.callback.class";
+    private static final String CONSUMER_REBALANCE_CALLBACK_CLASS_DOC = "A user-provided callback to execute when partition assignments change.";
 
+    /**
+     * <code>check.crcs</code>
+     */
+    public static final String CHECK_CRCS_CONFIG = "check.crcs";
+    private static final String CHECK_CRCS_DOC = "Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance.";
+    
     /** <code>key.deserializer</code> */
     public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer";
     private static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the <code>Deserializer</code> interface.";
@@ -160,38 +154,134 @@ public class ConsumerConfig extends AbstractConfig {
     private static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the <code>Deserializer</code> interface.";
 
     static {
-        /* TODO: add config docs */
-        config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, "blah blah")
-                                .define(GROUP_ID_CONFIG, Type.STRING, Importance.HIGH, "blah blah")
-                                .define(SESSION_TIMEOUT_MS, Type.LONG, 1000, Importance.HIGH, "blah blah")
-                                .define(HEARTBEAT_FREQUENCY, Type.INT, 3, Importance.MEDIUM, "blah blah")
-                                .define(PARTITION_ASSIGNMENT_STRATEGY, Type.STRING, Importance.MEDIUM, "blah blah")
-                                .define(METADATA_FETCH_TIMEOUT_CONFIG, Type.LONG, 60 * 1000, atLeast(0), Importance.MEDIUM, "blah blah")
-                                .define(ENABLE_AUTO_COMMIT_CONFIG, Type.BOOLEAN, true, Importance.MEDIUM, "blah blah")
-                                .define(AUTO_COMMIT_INTERVAL_MS_CONFIG, Type.LONG, 5000, atLeast(0), Importance.LOW, "blah blah")
-                                .define(CLIENT_ID_CONFIG, Type.STRING, "", Importance.LOW, "blah blah")
-                                .define(TOTAL_BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), Importance.LOW, "blah blah")
-                                .define(FETCH_BUFFER_CONFIG, Type.INT, 1 * 1024 * 1024, atLeast(0), Importance.HIGH, "blah blah")
-                                .define(SOCKET_RECEIVE_BUFFER_CONFIG, Type.INT, 128 * 1024, atLeast(0), Importance.LOW, "blah blah")
-                                .define(FETCH_MIN_BYTES_CONFIG, Type.LONG, 1024, atLeast(0), Importance.HIGH, "blah blah")
-                                .define(FETCH_MAX_WAIT_MS_CONFIG, Type.LONG, 500, atLeast(0), Importance.LOW, "blah blah")
-                                .define(RECONNECT_BACKOFF_MS_CONFIG, Type.LONG, 10L, atLeast(0L), Importance.LOW, "blah blah")
-                                .define(AUTO_OFFSET_RESET_CONFIG, Type.STRING, "largest", Importance.MEDIUM, "blah blah")
+        config = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG,
+                                        Type.LIST,
+                                        Importance.HIGH,
+                                        CommonClientConfigs.BOOSTRAP_SERVERS_DOC)
+                                .define(GROUP_ID_CONFIG, Type.STRING, "", Importance.HIGH, GROUP_ID_DOC)
+                                .define(SESSION_TIMEOUT_MS_CONFIG,
+                                        Type.LONG,
+                                        30000,
+                                        Importance.HIGH,
+                                        SESSION_TIMEOUT_MS_DOC)
+                                .define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
+                                        Type.STRING,
+                                        "blah",
+                                        Importance.MEDIUM,
+                                        PARTITION_ASSIGNMENT_STRATEGY_DOC)
+                                .define(METADATA_MAX_AGE_CONFIG,
+                                        Type.LONG,
+                                        5 * 60 * 1000,
+                                        atLeast(0),
+                                        Importance.LOW,
+                                        CommonClientConfigs.METADATA_MAX_AGE_DOC)
+                                .define(ENABLE_AUTO_COMMIT_CONFIG,
+                                        Type.BOOLEAN,
+                                        true,
+                                        Importance.MEDIUM,
+                                        ENABLE_AUTO_COMMIT_DOC)
+                                .define(AUTO_COMMIT_INTERVAL_MS_CONFIG,
+                                        Type.LONG,
+                                        5000,
+                                        atLeast(0),
+                                        Importance.LOW,
+                                        AUTO_COMMIT_INTERVAL_MS_DOC)
+                                .define(CLIENT_ID_CONFIG,
+                                        Type.STRING,
+                                        "",
+                                        Importance.LOW,
+                                        CommonClientConfigs.CLIENT_ID_DOC)
+                                .define(MAX_PARTITION_FETCH_BYTES_CONFIG,
+                                        Type.INT,
+                                        1 * 1024 * 1024,
+                                        atLeast(0),
+                                        Importance.HIGH,
+                                        MAX_PARTITION_FETCH_BYTES_DOC)
+                                .define(SEND_BUFFER_CONFIG,
+                                        Type.INT,
+                                        128 * 1024,
+                                        atLeast(0),
+                                        Importance.MEDIUM,
+                                        CommonClientConfigs.SEND_BUFFER_DOC)
+                                .define(RECEIVE_BUFFER_CONFIG,
+                                        Type.INT,
+                                        32 * 1024,
+                                        atLeast(0),
+                                        Importance.MEDIUM,
+                                        CommonClientConfigs.RECEIVE_BUFFER_DOC)
+                                .define(FETCH_MIN_BYTES_CONFIG,
+                                        Type.INT,
+                                        1024,
+                                        atLeast(0),
+                                        Importance.HIGH,
+                                        FETCH_MIN_BYTES_DOC)
+                                .define(FETCH_MAX_WAIT_MS_CONFIG,
+                                        Type.INT,
+                                        500,
+                                        atLeast(0),
+                                        Importance.LOW,
+                                        FETCH_MAX_WAIT_MS_DOC)
+                                .define(RECONNECT_BACKOFF_MS_CONFIG,
+                                        Type.LONG,
+                                        50L,
+                                        atLeast(0L),
+                                        Importance.LOW,
+                                        CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC)
+                                .define(RETRY_BACKOFF_MS_CONFIG,
+                                        Type.LONG,
+                                        100L,
+                                        atLeast(0L),
+                                        Importance.LOW,
+                                        CommonClientConfigs.RETRY_BACKOFF_MS_DOC)
+                                .define(AUTO_OFFSET_RESET_CONFIG,
+                                        Type.STRING,
+                                        "latest",
+                                        in("latest", "earliest", "none"),
+                                        Importance.MEDIUM,
+                                        AUTO_OFFSET_RESET_DOC)
+                                .define(CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG,
+                                        Type.CLASS,
+                                        NoOpConsumerRebalanceCallback.class,
+                                        Importance.LOW,
+                                        CONSUMER_REBALANCE_CALLBACK_CLASS_DOC)
+                                .define(CHECK_CRCS_CONFIG,
+                                        Type.BOOLEAN,
+                                        true,
+                                        Importance.LOW,
+                                        CHECK_CRCS_DOC)                                
                                 .define(METRICS_SAMPLE_WINDOW_MS_CONFIG,
                                         Type.LONG,
                                         30000,
                                         atLeast(0),
                                         Importance.LOW,
-                                        METRICS_SAMPLE_WINDOW_MS_DOC)
-                                .define(METRICS_NUM_SAMPLES_CONFIG, Type.INT, 2, atLeast(1), Importance.LOW, METRICS_NUM_SAMPLES_DOC)
-                                .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, "", Importance.LOW, METRIC_REPORTER_CLASSES_DOC)
-                                .define(KEY_DESERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, KEY_DESERIALIZER_CLASS_DOC)
-                                .define(VALUE_DESERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, VALUE_DESERIALIZER_CLASS_DOC);
-
+                                        CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC)
+                                .define(METRICS_NUM_SAMPLES_CONFIG,
+                                        Type.INT,
+                                        2,
+                                        atLeast(1),
+                                        Importance.LOW,
+                                        CommonClientConfigs.METRICS_NUM_SAMPLES_DOC)
+                                .define(METRIC_REPORTER_CLASSES_CONFIG,
+                                        Type.LIST,
+                                        "",
+                                        Importance.LOW,
+                                        CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC)
+                                .define(KEY_DESERIALIZER_CLASS_CONFIG,
+                                        Type.CLASS,
+                                        Importance.HIGH,
+                                        KEY_DESERIALIZER_CLASS_DOC)
+                                .define(VALUE_DESERIALIZER_CLASS_CONFIG,
+                                        Type.CLASS,
+                                        Importance.HIGH,
+                                        VALUE_DESERIALIZER_CLASS_DOC);
     }
 
     ConsumerConfig(Map<? extends Object, ? extends Object> props) {
         super(config, props);
     }
 
+    public static void main(String[] args) {
+        System.out.println(config.toHtmlTable());
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/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 e4cf7d1..74dfdba 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
@@ -9,7 +9,7 @@
  * 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.clients.consumer;
 
 import java.util.Collection;
@@ -17,34 +17,77 @@ import java.util.Collection;
 import org.apache.kafka.common.TopicPartition;
 
 /**
- * A callback interface that the user can implement to manage customized offsets on the start and end of 
- * 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, Collection) onPartitionsAssigned()} callback. And will commit offsets synchronously
- * for the specified list of partitions to Kafka in the {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked()}
- * callback.
+ * A callback interface that the user can implement to trigger custom actions when the set of partitions assigned to the
+ * consumer changes.
+ * <p>
+ * This is applicable when the consumer is having Kafka auto-manage group membership, if the consumer's directly subscribe to partitions
+ * those partitions will never be reassigned and this callback is not applicable.
+ * <p>
+ * When Kafka is managing the group membership, a partition re-assignment will be triggered any time the members of the group changes or the subscription
+ * of the members changes. This can occur when processes die, new process instances are added or old instances come back to life after failure.
+ * <p>
+ * There are many uses for this functionality. One common use is saving offsets in a custom store. By saving offsets in
+ * the {@link #onPartitionsRevoked(Consumer, Collection)} call we can ensure that any time partition assignment changes
+ * the offset gets saved.
+ * <p>
+ * Another use is flushing out any kind of cache of intermediate results the consumer may be keeping. For example,
+ * consider a case where the consumer is subscribed to a topic containing user page views, and the goal is to count the
+ * number of page views per users for each five minute window. Let's say the topic is partitioned by the user id so that
+ * all events for a particular user will go to a single consumer instance. The consumer can keep in memory a running
+ * tally of actions per user and only flush these out to a remote data store when it's cache gets to big. However if a
+ * partition is reassigned it may want to automatically trigger a flush of this cache, before the new owner takes over
+ * consumption.
+ * <p>
+ * This callback will execute in the user thread as part of the {@link Consumer#poll(long) poll(long)} call whenever partition assignment changes.
+ * <p>
+ * It is guaranteed that all consumer processes will invoke {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked} prior to 
+ * any process invoking {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned}. So if offsets or other state is saved in the 
+ * {@link #onPartitionsRevoked(Consumer, Collection) onPartitionsRevoked} call it is guaranteed to be saved by the time the process taking over that
+ * partition has their {@link #onPartitionsAssigned(Consumer, Collection) onPartitionsAssigned} callback called to load the state.
+ * <p>
+ * Here is pseudo-code for a callback implementation for saving offsets:
+ * <pre>
+ * {@code
+ *   public class SaveOffsetsOnRebalance implements ConsumerRebalanceCallback {
+ *       public void onPartitionsAssigned(Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {
+ *           // read the offsets from an external store using some custom code not described here
+ *           for(TopicPartition partition: partitions)
+ *              consumer.position(partition, readOffsetFromExternalStore(partition));
+ *       }      
+ *       public void onPartitionsRevoked(Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {
+ *           // save the offsets in an external store using some custom code not described here
+ *           for(TopicPartition partition: partitions)
+ *              saveOffsetInExternalStore(consumer.position(partition));
+ *       }
+ *   }
+ * }
+ * </pre>
  */
 public interface ConsumerRebalanceCallback {
 
     /**
-     * A callback method the user can implement to provide handling of customized offsets on completion of a successful 
-     * rebalance operation. This method will be called after a rebalance operation completes and before the consumer 
-     * starts fetching data.
-     * <p> 
-     * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer}  
-     * @param partitions The list of partitions that are assigned to the consumer after rebalance
+     * A callback method the user can implement to provide handling of customized offsets on completion of a successful
+     * partition re-assignement. This method will be called after an offset re-assignement completes and before the
+     * consumer starts fetching data.
+     * <p>
+     * It is guaranteed that all the processes in a consumer group will execute their
+     * {@link #onPartitionsRevoked(Consumer, Collection)} callback before any instance executes its
+     * {@link #onPartitionsAssigned(Consumer, Collection)} callback.
+     * 
+     * @param partitions The list of partitions that are now assigned to the consumer (may include partitions previously
+     *            assigned to the consumer)
      */
-    public void onPartitionsAssigned(Consumer<?,?> consumer, Collection<TopicPartition> partitions);
-    
+    public void onPartitionsAssigned(Consumer<?, ?> consumer, Collection<TopicPartition> partitions);
+
     /**
-     * A callback method the user can implement to provide handling of offset commits to a customized store on the 
-     * start of a rebalance operation. This method will be called before a rebalance operation starts and after the 
-     * consumer stops fetching data. It is recommended that offsets should be committed in this callback to 
-     * either Kafka or a custom offset store to prevent duplicate data 
-     * <p> 
-     * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer}  
+     * A callback method the user can implement to provide handling of offset commits to a customized store on the start
+     * of a rebalance operation. This method will be called before a rebalance operation starts and after the consumer
+     * stops fetching data. It is recommended that offsets should be committed in this callback to either Kafka or a
+     * custom offset store to prevent duplicate data
+     * <p>
+     * For examples on usage of this API, see Usage Examples section of {@link KafkaConsumer KafkaConsumer}
+     * 
      * @param partitions The list of partitions that were assigned to the consumer on the last rebalance
      */
-    public void onPartitionsRevoked(Consumer<?,?> consumer, Collection<TopicPartition> partitions);
+    public void onPartitionsRevoked(Consumer<?, ?> consumer, Collection<TopicPartition> partitions);
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
index 16af70a..466254e 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java
@@ -9,119 +9,76 @@
  * 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.clients.consumer;
 
-import org.apache.kafka.common.TopicPartition;
-
 /**
- * A key/value pair to be received from Kafka. This consists of a topic name and a partition number, from which the 
- * record is being received and an offset that points to the record in a Kafka partition. 
+ * A key/value pair to be received from Kafka. This consists of a topic name and a partition number, from which the
+ * record is being received and an offset that points to the record in a Kafka partition.
  */
-public final class ConsumerRecord<K,V> {
-    private final TopicPartition partition; 
+public final class ConsumerRecord<K, V> {
+    private final String topic;
+    private final int partition;
+    private final long offset;
     private final K key;
     private final V value;
-    private final long offset;
-    private volatile Exception error;
-    
-    /**
-     * Creates a record to be received from a specified topic and partition
-     * 
-     * @param topic     The topic this record is received from
-     * @param partitionId The partition of the topic this record is received from
-     * @param key       The key of the record, if one exists
-     * @param value     The record contents
-     * @param offset    The offset of this record in the corresponding Kafka partition
-     */
-    public ConsumerRecord(String topic, int partitionId, K key, V value, long offset) {
-        this(topic, partitionId, key, value, offset, null);
-    }
 
     /**
      * Create a record with no key
      * 
      * @param topic The topic this record is received from
-     * @param partitionId The partition of the topic this record is received from
-     * @param value The record contents
+     * @param partition The partition of the topic this record is received from
      * @param offset The offset of this record in the corresponding Kafka partition
+     * @param value The record contents
      */
-    public ConsumerRecord(String topic, int partitionId, V value, long offset) {
-        this(topic, partitionId, null, value, offset);
-    }
-
-    /**
-     * Creates a record with an error code
-     * @param topic     The topic this record is received from
-     * @param partitionId The partition of the topic this record is received from
-     * @param error     The exception corresponding to the error code returned by the server for this topic partition
-     */
-    public ConsumerRecord(String topic, int partitionId, Exception error) {
-        this(topic, partitionId, null, null, -1L, error);
-    }
-    
-    private ConsumerRecord(String topic, int partitionId, K key, V value, long offset, Exception error) {
+    public ConsumerRecord(String topic, int partition, long offset, K key, V value) {
         if (topic == null)
             throw new IllegalArgumentException("Topic cannot be null");
-        this.partition = new TopicPartition(topic, partitionId);
+        this.topic = topic;
+        this.partition = partition;
+        this.offset = offset;
         this.key = key;
         this.value = value;
-        this.offset = offset;  
-        this.error = error;
     }
-    
+
     /**
      * The topic this record is received from
      */
     public String topic() {
-        return partition.topic();
+        return this.topic;
     }
 
     /**
-     * The partition from which this record is received 
+     * The partition from which this record is received
      */
     public int partition() {
-        return partition.partition();
+        return this.partition;
     }
-    
-    /**
-     * The TopicPartition object containing the topic and partition
-     */
-    public TopicPartition topicAndPartition() {
-        return partition;
-    }
-    
+
     /**
      * The key (or null if no key is specified)
-     * @throws Exception The exception thrown while fetching this record.
      */
     public K key() throws Exception {
-        if (this.error != null)
-            throw this.error;
         return key;
     }
 
     /**
      * The value
-     * @throws Exception The exception thrown while fetching this record.
      */
     public V value() throws Exception {
-        if (this.error != null)
-            throw this.error;
         return value;
     }
 
     /**
      * The position of this record in the corresponding Kafka partition.
-     * @throws Exception The exception thrown while fetching this record.
      */
-    public long offset() throws Exception {
-        if (this.error != null)
-            throw this.error;
+    public long offset() {
         return offset;
     }
 
-    public Exception error() {
-        return this.error;
+    @Override
+    public String toString() {
+        return "ConsumerRecord(topic = " + topic() + ", partition = " + partition() + ", offset = " + offset()
+                + ", key = " + key + ", value = " + value + ")";
     }
 }


[5/7] kafka git commit: KAFKA-1760: New consumer.

Posted by jk...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/0699ff2c/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
index 76efc21..300c551 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
@@ -9,380 +9,447 @@
  * 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.clients.consumer;
 
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.kafka.clients.ClientRequest;
+import org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.ConnectionState;
+import org.apache.kafka.clients.NetworkClient;
+import org.apache.kafka.clients.RequestCompletionHandler;
+import org.apache.kafka.clients.consumer.internals.Heartbeat;
+import org.apache.kafka.clients.consumer.internals.SubscriptionState;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.internals.Metadata;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.metrics.JmxReporter;
+import org.apache.kafka.common.metrics.Measurable;
 import org.apache.kafka.common.metrics.MetricConfig;
 import org.apache.kafka.common.MetricName;
 import org.apache.kafka.common.metrics.Metrics;
 import org.apache.kafka.common.metrics.MetricsReporter;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.metrics.stats.Avg;
+import org.apache.kafka.common.metrics.stats.Count;
+import org.apache.kafka.common.metrics.stats.Max;
+import org.apache.kafka.common.metrics.stats.Rate;
 import org.apache.kafka.common.serialization.Deserializer;
+import org.apache.kafka.common.network.Selector;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.record.LogEntry;
+import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.requests.ConsumerMetadataRequest;
+import org.apache.kafka.common.requests.ConsumerMetadataResponse;
+import org.apache.kafka.common.requests.FetchRequest;
+import org.apache.kafka.common.requests.FetchRequest.PartitionData;
+import org.apache.kafka.common.requests.FetchResponse;
+import org.apache.kafka.common.requests.HeartbeatRequest;
+import org.apache.kafka.common.requests.HeartbeatResponse;
+import org.apache.kafka.common.requests.JoinGroupRequest;
+import org.apache.kafka.common.requests.JoinGroupResponse;
+import org.apache.kafka.common.requests.ListOffsetRequest;
+import org.apache.kafka.common.requests.ListOffsetResponse;
+import org.apache.kafka.common.requests.OffsetCommitRequest;
+import org.apache.kafka.common.requests.OffsetCommitResponse;
+import org.apache.kafka.common.requests.OffsetFetchRequest;
+import org.apache.kafka.common.requests.OffsetFetchResponse;
+import org.apache.kafka.common.requests.RequestHeader;
+import org.apache.kafka.common.requests.RequestSend;
 import org.apache.kafka.common.utils.ClientUtils;
 import org.apache.kafka.common.utils.SystemTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.net.InetSocketAddress;
-import java.util.*;
-
 /**
  * A Kafka client that consumes records from a Kafka cluster.
- * <P>
- * The consumer is <i>thread safe</i> and should generally be shared among all threads for best performance.
  * <p>
- * The consumer is single threaded and multiplexes I/O over TCP connections to each of the brokers it
- * needs to communicate with. Failure to close the consumer after use will leak these resources.
+ * It will transparently handle the failure of servers in the Kafka cluster, and transparently adapt as partitions of
+ * data it subscribes to migrate within the cluster. This client also interacts with the server to allow groups of
+ * consumers to load balance consumption using consumer groups (as described below).
+ * <p>
+ * The consumer maintains TCP connections to the necessary brokers to fetch data for the topics it subscribes to.
+ * Failure to close the consumer after use will leak these connections.
+ * <p>
+ * The consumer is thread safe but generally will be used only from within a single thread. The consumer client has no
+ * threads of it's own, all work is done in the caller's thread when calls are made on the various methods exposed.
+ * 
+ * <h3>Offsets and Consumer Position</h3>
+ * Kafka maintains a numerical offset for each record in a partition. This offset acts as a kind of unique identifier of
+ * a record within that partition, and also denotes the position of the consumer in the partition. That is, a consumer
+ * which has position 5 has consumed records with offsets 0 through 4 and will next receive record with offset 5. There
+ * are actually two notions of position relevant to the user of the consumer.
+ * <p>
+ * The {@link #position(TopicPartition) position} of the consumer gives the offset of the next record that will be given
+ * out. It will be one larger than the highest offset the consumer has seen in that partition. It automatically advances
+ * every time the consumer receives data calls {@link #poll(long)} and receives messages.
+ * <p>
+ * The {@link #commit(CommitType) committed position} is the last offset that has been saved securely. Should the
+ * process fail and restart, this is the offset that it will recover to. The consumer can either automatically commit
+ * offsets periodically, or it can choose to control this committed position manually by calling
+ * {@link #commit(CommitType) commit}.
+ * <p>
+ * This distinction gives the consumer control over when a record is considered consumed. It is discussed in further
+ * detail below.
+ * 
+ * <h3>Consumer Groups</h3>
+ * 
+ * Kafka uses the concept of <i>consumer groups</i> to allow a pool of processes to divide up the work of consuming and
+ * processing records. These processes can either be running on the same machine or, as is more likely, they can be
+ * distributed over many machines to provide additional scalability and fault tolerance for processing.
+ * <p>
+ * Each Kafka consumer must specify a consumer group that it belongs to. Kafka will deliver each message in the
+ * subscribed topics to one process in each consumer group. This is achieved by balancing the partitions in the topic
+ * over the consumer processes in each group. So if there is a topic with four partitions, and a consumer group with two
+ * processes, each process would consume from two partitions. This group membership is maintained dynamically: if a
+ * process fails the partitions assigned to it will be reassigned to other processes in the same group, and if a new
+ * process joins the group, partitions will be moved from existing consumers to this new process.
+ * <p>
+ * So if two processes subscribe to a topic both specifying different groups they will each get all the records in that
+ * topic; if they both specify the same group they will each get about half the records.
+ * <p>
+ * Conceptually you can think of a consumer group as being a single logical subscriber that happens to be made up of
+ * multiple processes. As a multi-subscriber system, Kafka naturally supports having any number of consumer groups for a
+ * given topic without duplicating data (additional consumers are actually quite cheap).
+ * <p>
+ * This is a slight generalization of the functionality that is common in messaging systems. To get semantics similar to
+ * a queue in a traditional messaging system all processes would be part of a single consumer group and hence record
+ * delivery would be balanced over the group like with a queue. Unlike a traditional messaging system, though, you can
+ * have multiple such groups. To get semantics similar to pub-sub in a traditional messaging system each process would
+ * have it's own consumer group, so each process would subscribe to all the records published to the topic.
+ * <p>
+ * In addition, when offsets are committed they are always committed for a given consumer group.
+ * <p>
+ * It is also possible for the consumer to manually specify the partitions it subscribes to, which disables this dynamic
+ * partition balancing.
+ * 
  * <h3>Usage Examples</h3>
- * The consumer APIs offer flexibility to cover a variety of consumption use cases. Following are some examples to demonstrate the correct use of 
- * the available APIs. Each of the examples assumes the presence of a user implemented process() method that processes a given batch of messages
- * and returns the offset of the latest processed message per partition. Note that process() is not part of the consumer API and is only used as
- * a convenience method to demonstrate the different use cases of the consumer APIs. Here is a sample implementation of such a process() method.
+ * The consumer APIs offer flexibility to cover a variety of consumption use cases. Here are some examples to
+ * demonstrate how to use them.
+ * 
+ * <h4>Simple Processing</h4>
+ * This example demonstrates the simplest usage of Kafka's consumer api.
+ * 
  * <pre>
- * {@code
- * private Map<TopicPartition, Long> process(Map<String, ConsumerRecord<byte[], byte[]> records) {
- *     Map<TopicPartition, Long> processedOffsets = new HashMap<TopicPartition, Long>();
- *     for(Entry<String, ConsumerRecords<byte[], byte[]>> recordMetadata : records.entrySet()) {
- *          List<ConsumerRecord<byte[], byte[]>> recordsPerTopic = recordMetadata.getValue().records();
- *          for(int i = 0;i < recordsPerTopic.size();i++) {
- *               ConsumerRecord<byte[], byte[]> record = recordsPerTopic.get(i);
- *               // process record
- *               try {
- *               	processedOffsets.put(record.topicAndpartition(), record.offset());
- *               } catch (Exception e) {
- *               	e.printStackTrace();
- *               }               
- *          }
+ *     Properties props = new Properties();
+ *     props.put(&quot;metadata.broker.list&quot;, &quot;localhost:9092&quot;);
+ *     props.put(&quot;group.id&quot;, &quot;test&quot;);
+ *     props.put(&quot;enable.auto.commit&quot;, &quot;true&quot;);
+ *     props.put(&quot;auto.commit.interval.ms&quot;, &quot;1000&quot;);
+ *     props.put(&quot;session.timeout.ms&quot;, &quot;30000&quot;);
+ *     props.put(&quot;key.serializer&quot;, &quot;org.apache.kafka.common.serializers.StringSerializer&quot;);
+ *     props.put(&quot;value.serializer&quot;, &quot;org.apache.kafka.common.serializers.StringSerializer&quot;);
+ *     KafkaConsumer&lt;String, String&gt; consumer = new KafkaConsumer&lt;String, String&gt;(props);
+ *     consumer.subscribe(&quot;foo&quot;, &quot;bar&quot;);
+ *     while (true) {
+ *         ConsumerRecords&lt;String, String&gt; records = consumer.poll(100);
+ *         for (ConsumerRecord&lt;String, String&gt; record : records)
+ *             System.out.printf(&quot;offset = %d, key = %s, value = %s&quot;, record.offset(), record.key(), record.value());
  *     }
- *     return processedOffsets; 
- * }
- * }
  * </pre>
+ * 
+ * Setting <code>enable.auto.commit</code> means that offsets are committed automatically with a frequency controlled by
+ * the config <code>auto.commit.interval.ms</code>.
  * <p>
- * This example demonstrates how the consumer can be used to leverage Kafka's group management functionality for automatic consumer load 
- * balancing and failover. This example assumes that the offsets are stored in Kafka and are automatically committed periodically, 
- * as controlled by the auto.commit.interval.ms config
- * <pre>
- * {@code  
- * Properties props = new Properties();
- * props.put("metadata.broker.list", "localhost:9092");
- * props.put("group.id", "test");
- * props.put("session.timeout.ms", "1000");
- * props.put("enable.auto.commit", "true");
- * props.put("auto.commit.interval.ms", "10000");
- * KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<byte[], byte[]>(props);
- * consumer.subscribe("foo", "bar");
- * boolean isRunning = true;
- * while(isRunning) {
- *   Map<String, ConsumerRecords<byte[], byte[]>> records = consumer.poll(100);
- *   process(records);
- * }
- * consumer.close();
- * }
- * </pre>
- * This example demonstrates how the consumer can be used to leverage Kafka's group management functionality for automatic consumer load 
- * balancing and failover. This example assumes that the offsets are stored in Kafka and are manually committed using 
- * the commit(boolean) API. This example also demonstrates rewinding the consumer's offsets if processing of the consumed
- * messages fails. Note that this method of rewinding offsets using {@link #seek(Map) seek(offsets)} is only useful for rewinding the offsets
- * of the current consumer instance. As such, this will not trigger a rebalance or affect the fetch offsets for the other consumer instances.
+ * The connection to the cluster is bootstrapped by specifying a list of one or more brokers to contact using the
+ * configuration <code>metadata.broker.list</code>. This list is just used to discover the rest of the brokers in the
+ * cluster and need not be an exhaustive list of servers in the cluster (though you may want to specify more than one in
+ * case there are servers down when the client is connecting).
+ * <p>
+ * In this example the client is subscribing to the topics <i>foo</i> and <i>bar</i> as part of a group of consumers
+ * called <i>test</i> as described above.
+ * <p>
+ * The broker will automatically detect failed processes in the <i>test</i> group by using a heartbeat mechanism. The
+ * consumer will automatically ping the cluster periodically, which let's the cluster know that it is alive. As long as
+ * the consumer is able to do this it is considered alive and retains the right to consume from the partitions assigned
+ * to it. If it stops heartbeating for a period of time longer than <code>session.timeout.ms</code> then it will be
+ * considered dead and it's partitions will be assigned to another process.
+ * <p>
+ * The serializers settings specify how to turn the objects the user provides into bytes. By specifying the string
+ * serializers we are saying that our record's key and value will just be simple strings.
+ * 
+ * <h4>Controlling When Messages Are Considered Consumed</h4>
+ * 
+ * In this example we will consume a batch of records and batch them up in memory, when we have sufficient records
+ * batched we will insert them into a database. If we allowed offsets to auto commit as in the previous example messages
+ * would be considered consumed after they were given out by the consumer, and it would be possible that our process
+ * could fail after we have read messages into our in-memory buffer but before they had been inserted into the database.
+ * To avoid this we will manually commit the offsets only once the corresponding messages have been inserted into the
+ * database. This gives us exact control of when a message is considered consumed. This raises the opposite possibility:
+ * the process could fail in the interval after the insert into the database but before the commit (even though this
+ * would likely just be a few milliseconds, it is a possibility). In this case the process that took over consumption
+ * would consume from last committed offset and would repeat the insert of the last batch of data. Used in this way
+ * Kafka provides what is often called "at-least once delivery" guarantees, as each message will likely be delivered one
+ * time but in failure cases could be duplicated.
+ * 
  * <pre>
- * {@code  
- * Properties props = new Properties();
- * props.put("metadata.broker.list", "localhost:9092");
- * props.put("group.id", "test");
- * props.put("session.timeout.ms", "1000");
- * props.put("enable.auto.commit", "false");
- * KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<byte[], byte[]>(props);
- * consumer.subscribe("foo", "bar");
- * int commitInterval = 100;
- * int numRecords = 0;
- * boolean isRunning = true;
- * Map<TopicPartition, Long> consumedOffsets = new HashMap<TopicPartition, Long>();
- * while(isRunning) {
- *     Map<String, ConsumerRecords<byte[], byte[]>> records = consumer.poll(100);
- *     try {
- *         Map<TopicPartition, Long> lastConsumedOffsets = process(records);
- *         consumedOffsets.putAll(lastConsumedOffsets);
- *         numRecords += records.size();
- *         // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance
- *         if(numRecords % commitInterval == 0) 
- *           consumer.commit(false);
- *     } catch(Exception e) {
- *         try {
- *             // rewind consumer's offsets for failed partitions
- *             // assume failedPartitions() returns the list of partitions for which the processing of the last batch of messages failed
- *             List<TopicPartition> failedPartitions = failedPartitions();   
- *             Map<TopicPartition, Long> offsetsToRewindTo = new HashMap<TopicPartition, Long>();
- *             for(TopicPartition failedPartition : failedPartitions) {
- *                 // rewind to the last consumed offset for the failed partition. Since process() failed for this partition, the consumed offset
- *                 // should still be pointing to the last successfully processed offset and hence is the right offset to rewind consumption to.
- *                 offsetsToRewindTo.put(failedPartition, consumedOffsets.get(failedPartition));
+ *     Properties props = new Properties();
+ *     props.put(&quot;metadata.broker.list&quot;, &quot;localhost:9092&quot;);
+ *     props.put(&quot;group.id&quot;, &quot;test&quot;);
+ *     props.put(&quot;enable.auto.commit&quot;, &quot;false&quot;);
+ *     props.put(&quot;auto.commit.interval.ms&quot;, &quot;1000&quot;);
+ *     props.put(&quot;session.timeout.ms&quot;, &quot;30000&quot;);
+ *     props.put(&quot;key.serializer&quot;, &quot;org.apache.kafka.common.serializers.StringSerializer&quot;);
+ *     props.put(&quot;value.serializer&quot;, &quot;org.apache.kafka.common.serializers.StringSerializer&quot;);
+ *     KafkaConsumer&lt;String, String&gt; consumer = new KafkaConsumer&lt;String, String&gt;(props);
+ *     consumer.subscribe(&quot;foo&quot;, &quot;bar&quot;);
+ *     int commitInterval = 200;
+ *     List&lt;ConsumerRecord&lt;String, String&gt;&gt; buffer = new ArrayList&lt;ConsumerRecord&lt;String, String&gt;&gt;();
+ *     while (true) {
+ *         ConsumerRecords&lt;String, String&gt; records = consumer.poll(100);
+ *         for (ConsumerRecord&lt;String, String&gt; record : records) {
+ *             buffer.add(record);
+ *             if (buffer.size() &gt;= commitInterval) {
+ *                 insertIntoDb(buffer);
+ *                 consumer.commit(CommitType.SYNC);
+ *                 buffer.clear();
  *             }
- *             // seek to new offsets only for partitions that failed the last process()
- *             consumer.seek(offsetsToRewindTo);
- *         } catch(Exception e) {  break; } // rewind failed
+ *         }
  *     }
- * }         
- * consumer.close();
- * }
  * </pre>
+ * 
+ * <h4>Subscribing To Specific Partitions</h4>
+ * 
+ * In the previous examples we subscribed to the topics we were interested in and let Kafka give our particular process
+ * a fair share of the partitions for those topics. This provides a simple load balancing mechanism so multiple
+ * instances of our program can divided up the work of processing records.
  * <p>
- * This example demonstrates how to rewind the offsets of the entire consumer group. It is assumed that the user has chosen to use Kafka's 
- * group management functionality for automatic consumer load balancing and failover. This example also assumes that the offsets are stored in 
- * Kafka. If group management is used, the right place to systematically rewind offsets for <i>every</i> consumer instance is inside the 
- * ConsumerRebalanceCallback. The onPartitionsAssigned callback is invoked after the consumer is assigned a new set of partitions on rebalance 
- * <i>and</i> before the consumption restarts post rebalance. This is the right place to supply the newly rewound offsets to the consumer. It 
- * is recommended that if you foresee the requirement to ever reset the consumer's offsets in the presence of group management, that you 
- * always configure the consumer to use the ConsumerRebalanceCallback with a flag that protects whether or not the offset rewind logic is used.
- * This method of rewinding offsets is useful if you notice an issue with your message processing after successful consumption and offset commit.
- * And you would like to rewind the offsets for the entire consumer group as part of rolling out a fix to your processing logic. In this case,
- * you would configure each of your consumer instances with the offset rewind configuration flag turned on and bounce each consumer instance 
- * in a rolling restart fashion. Each restart will trigger a rebalance and eventually all consumer instances would have rewound the offsets for 
- * the partitions they own, effectively rewinding the offsets for the entire consumer group.   
- * <pre>
- * {@code  
- * Properties props = new Properties();
- * props.put("metadata.broker.list", "localhost:9092");
- * props.put("group.id", "test");
- * props.put("session.timeout.ms", "1000");
- * props.put("enable.auto.commit", "false");
- * KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<byte[], byte[]>(
- *                                            props,
- *                                            new ConsumerRebalanceCallback() {
- *                                                boolean rewindOffsets = true;  // should be retrieved from external application config
- *                                                public void onPartitionsAssigned(Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {
- *                                                    Map<TopicPartition, Long> latestCommittedOffsets = consumer.committed(partitions);
- *                                                    if(rewindOffsets)
- *                                                        Map<TopicPartition, Long> newOffsets = rewindOffsets(latestCommittedOffsets, 100);
- *                                                    consumer.seek(newOffsets);
- *                                                }
- *                                                public void onPartitionsRevoked(Consumer<?, ?> consumer, Collection<TopicPartition> partitions) {
- *                                                    consumer.commit(true);
- *                                                }
- *                                                // this API rewinds every partition back by numberOfMessagesToRewindBackTo messages 
- *                                                private Map<TopicPartition, Long> rewindOffsets(Map<TopicPartition, Long> currentOffsets,
- *                                                                                                long numberOfMessagesToRewindBackTo) {
- *                                                    Map<TopicPartition, Long> newOffsets = new HashMap<TopicPartition, Long>();
- *                                                    for(Map.Entry<TopicPartition, Long> offset : currentOffsets.entrySet()) 
- *                                                        newOffsets.put(offset.getKey(), offset.getValue() - numberOfMessagesToRewindBackTo);
- *                                                    return newOffsets;
- *                                                }
- *                                            });
- * consumer.subscribe("foo", "bar");
- * int commitInterval = 100;
- * int numRecords = 0;
- * boolean isRunning = true;
- * Map<TopicPartition, Long> consumedOffsets = new HashMap<TopicPartition, Long>();
- * while(isRunning) {
- *     Map<String, ConsumerRecords<byte[], byte[]>> records = consumer.poll(100);
- *     Map<TopicPartition, Long> lastConsumedOffsets = process(records);
- *     consumedOffsets.putAll(lastConsumedOffsets);
- *     numRecords += records.size();
- *     // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance
- *     if(numRecords % commitInterval == 0) 
- *         consumer.commit(consumedOffsets, true);
- * }
- * consumer.commit(true);
- * consumer.close();
- * }
- * </pre>
- * This example demonstrates how the consumer can be used to leverage Kafka's group management functionality along with custom offset storage. 
- * In this example, the assumption made is that the user chooses to store the consumer offsets outside Kafka. This requires the user to 
- * plugin logic for retrieving the offsets from a custom store and provide the offsets to the consumer in the ConsumerRebalanceCallback
- * callback. The onPartitionsAssigned callback is invoked after the consumer is assigned a new set of partitions on rebalance <i>and</i>
- * before the consumption restarts post rebalance. This is the right place to supply offsets from a custom store to the consumer.
+ * In this mode the consumer will just get the partitions it subscribes to and if the consumer instance fails no attempt
+ * will be made to rebalance partitions to other instances.
  * <p>
- * Similarly, the user would also be required to plugin logic for storing the consumer's offsets to a custom store. The onPartitionsRevoked 
- * callback is invoked right after the consumer has stopped fetching data and before the partition ownership changes. This is the right place 
- * to commit the offsets for the current set of partitions owned by the consumer.  
- * <pre>
- * {@code  
- * Properties props = new Properties();
- * props.put("metadata.broker.list", "localhost:9092");
- * props.put("group.id", "test");
- * props.put("session.timeout.ms", "1000");
- * props.put("enable.auto.commit", "false"); // since enable.auto.commit only applies to Kafka based offset storage
- * KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<byte[], byte[]>(
- *                                            props,
- *                                            new ConsumerRebalanceCallback() {
- *                                                public void onPartitionsAssigned(Consumer<?,?> consumer, Collection<TopicPartition> partitions) {
- *                                                    Map<TopicPartition, Long> lastCommittedOffsets = getLastCommittedOffsetsFromCustomStore(partitions);
- *                                                    consumer.seek(lastCommittedOffsets);
- *                                                }
- *                                                public void onPartitionsRevoked(Consumer<?,?> consumer, Collection<TopicPartition> partitions) {
- *                                                    Map<TopicPartition, Long> offsets = getLastConsumedOffsets(partitions);
- *                                                    commitOffsetsToCustomStore(offsets); 
- *                                                }
- *                                                // following APIs should be implemented by the user for custom offset management
- *                                                private Map<TopicPartition, Long> getLastCommittedOffsetsFromCustomStore(Collection<TopicPartition> partitions) {
- *                                                    return null;
- *                                                }
- *                                                private Map<TopicPartition, Long> getLastConsumedOffsets(Collection<TopicPartition> partitions) { return null; }
- *                                                private void commitOffsetsToCustomStore(Map<TopicPartition, Long> offsets) {}
- *                                            });
- * consumer.subscribe("foo", "bar");
- * int commitInterval = 100;
- * int numRecords = 0;
- * boolean isRunning = true;
- * Map<TopicPartition, Long> consumedOffsets = new HashMap<TopicPartition, Long>();
- * while(isRunning) {
- *     Map<String, ConsumerRecords<byte[], byte[]>> records = consumer.poll(100);
- *     Map<TopicPartition, Long> lastConsumedOffsets = process(records);
- *     consumedOffsets.putAll(lastConsumedOffsets);
- *     numRecords += records.size();
- *     // commit offsets for all partitions of topics foo, bar synchronously, owned by this consumer instance
- *     if(numRecords % commitInterval == 0) 
- *         commitOffsetsToCustomStore(consumedOffsets);
- * }
- * consumer.commit(true);
- * consumer.close();
- * }
- * </pre>
- * This example demonstrates how the consumer can be used to subscribe to specific partitions of certain topics and consume upto the latest
- * available message for each of those partitions before shutting down. When used to subscribe to specific partitions, the user foregoes 
- * the group management functionality and instead relies on manually configuring the consumer instances to subscribe to a set of partitions.
- * This example assumes that the user chooses to use Kafka based offset storage. The user still has to specify a group.id to use Kafka 
- * based offset management. However, session.timeout.ms is not required since the Kafka consumer only does automatic failover when group 
- * management is used.
- * <pre>
- * {@code  
- * Properties props = new Properties();
- * props.put("metadata.broker.list", "localhost:9092");
- * props.put("group.id", "test");
- * props.put("enable.auto.commit", "true");
- * props.put("auto.commit.interval.ms", "10000");
- * KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<byte[], byte[]>(props);
- * // subscribe to some partitions of topic foo
- * TopicPartition partition0 = new TopicPartition("foo", 0);
- * TopicPartition partition1 = new TopicPartition("foo", 1);
- * TopicPartition[] partitions = new TopicPartition[2];
- * partitions[0] = partition0;
- * partitions[1] = partition1;
- * consumer.subscribe(partitions);
- * // find the last committed offsets for partitions 0,1 of topic foo
- * Map<TopicPartition, Long> lastCommittedOffsets = consumer.committed(Arrays.asList(partitions));
- * // seek to the last committed offsets to avoid duplicates
- * consumer.seek(lastCommittedOffsets);        
- * // find the offsets of the latest available messages to know where to stop consumption
- * Map<TopicPartition, Long> latestAvailableOffsets = consumer.offsetsBeforeTime(-2, Arrays.asList(partitions));
- * boolean isRunning = true;
- * Map<TopicPartition, Long> consumedOffsets = new HashMap<TopicPartition, Long>();
- * while(isRunning) {
- *     Map<String, ConsumerRecords<byte[], byte[]>> records = consumer.poll(100);
- *     Map<TopicPartition, Long> lastConsumedOffsets = process(records);
- *     consumedOffsets.putAll(lastConsumedOffsets);
- *     for(TopicPartition partition : partitions) {
- *         if(consumedOffsets.get(partition) >= latestAvailableOffsets.get(partition))
- *             isRunning = false;
- *         else
- *             isRunning = true;
- *     }
- * }
- * consumer.commit(true);
- * consumer.close();
- * }
- * </pre>
- * This example demonstrates how the consumer can be used to subscribe to specific partitions of certain topics and consume upto the latest
- * available message for each of those partitions before shutting down. When used to subscribe to specific partitions, the user foregoes 
- * the group management functionality and instead relies on manually configuring the consumer instances to subscribe to a set of partitions.
- * This example assumes that the user chooses to use custom offset storage.
+ * There are several cases where this makes sense:
+ * <ul>
+ * <li>The first case is if the process is maintaining some kind of local state associated with that partition (like a
+ * local on-disk key-value store) and hence it should only get records for the partition it is maintaining on disk.
+ * <li>Another case is if the process itself is highly available and will be restarted if it fails (perhaps using a
+ * cluster management framework like YARN, Mesos, or AWS facilities, or as part of a stream processing framework). In
+ * this case there is no need for Kafka to detect the failure and reassign the partition, rather the consuming process
+ * will be restarted on another machine.
+ * </ul>
+ * <p>
+ * This mode is easy to specify, rather than subscribing to the topic, the consumer just subscribes to particular
+ * partitions:
+ * 
  * <pre>
- * {@code  
- * Properties props = new Properties();
- * props.put("metadata.broker.list", "localhost:9092");
- * KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<byte[], byte[]>(props);
- * // subscribe to some partitions of topic foo
- * TopicPartition partition0 = new TopicPartition("foo", 0);
- * TopicPartition partition1 = new TopicPartition("foo", 1);
- * TopicPartition[] partitions = new TopicPartition[2];
- * partitions[0] = partition0;
- * partitions[1] = partition1;
- * consumer.subscribe(partitions);
- * Map<TopicPartition, Long> lastCommittedOffsets = getLastCommittedOffsetsFromCustomStore();
- * // seek to the last committed offsets to avoid duplicates
- * consumer.seek(lastCommittedOffsets);        
- * // find the offsets of the latest available messages to know where to stop consumption
- * Map<TopicPartition, Long> latestAvailableOffsets = consumer.offsetsBeforeTime(-2, Arrays.asList(partitions));
- * boolean isRunning = true;
- * Map<TopicPartition, Long> consumedOffsets = new HashMap<TopicPartition, Long>();
- * while(isRunning) {
- *     Map<String, ConsumerRecords<byte[], byte[]>> records = consumer.poll(100);
- *     Map<TopicPartition, Long> lastConsumedOffsets = process(records);
- *     consumedOffsets.putAll(lastConsumedOffsets);
- *     // commit offsets for partitions 0,1 for topic foo to custom store
- *     commitOffsetsToCustomStore(consumedOffsets);
- *     for(TopicPartition partition : partitions) {
- *         if(consumedOffsets.get(partition) >= latestAvailableOffsets.get(partition))
- *             isRunning = false;
- *         else
- *             isRunning = true;
- *     }            
- * }      
- * commitOffsetsToCustomStore(consumedOffsets);   
- * consumer.close();
- * }
+ *     String topic = &quot;foo&quot;;
+ *     TopicPartition partition0 = new TopicPartition(topic, 0);
+ *     TopicPartition partition1 = new TopicPartition(topic, 1);
+ *     consumer.subscribe(partition0);
+ *     consumer.subscribe(partition1);
  * </pre>
+ * 
+ * The group that the consumer specifies is still used for committing offsets, but now the set of partitions will only
+ * be changed if the consumer specifies new partitions, and no attempt at failure detection will be made.
+ * <p>
+ * It isn't possible to mix both subscription to specific partitions (with no load balancing) and to topics (with load
+ * balancing) using the same consumer instance.
+ * 
+ * <h4>Managing Your Own Offsets</h4>
+ * 
+ * The consumer application need not use Kafka's built-in offset storage, it can store offsets in a store of it's own
+ * choosing. The primary use case for this is allowing the application to store both the offset and the results of the
+ * consumption in the same system in a way that both the results and offsets are stored atomically. This is not always
+ * possible, but when it is it will make the consumption fully atomic and give "exactly once" semantics that are
+ * stronger than the default "at-least once" semantics you get with Kafka's offset commit functionality.
+ * <p>
+ * Here are a couple of examples of this type of usage:
+ * <ul>
+ * <li>If the results of the consumption are being stored in a relational database, storing the offset in the database
+ * as well can allow committing both the results and offset in a single transaction. Thus either the transaction will
+ * succeed and the offset will be updated based on what was consumed or the result will not be stored and the offset
+ * won't be updated.
+ * <li>If the results are being stored in a local store it may be possible to store the offset there as well. For
+ * example a search index could be built by subscribing to a particular partition and storing both the offset and the
+ * indexed data together. If this is done in a way that is atomic, it is often possible to have it be the case that even
+ * if a crash occurs that causes unsync'd data to be lost, whatever is left has the corresponding offset stored as well.
+ * This means that in this case the indexing process that comes back having lost recent updates just resumes indexing
+ * from what it has ensuring that no updates are lost.
+ * </ul>
+ * 
+ * Each record comes with it's own offset, so to manage your own offset you just need to do the following:
+ * <ol>
+ * <li>Configure <code>enable.auto.commit=false</code>
+ * <li>Use the offset provided with each {@link ConsumerRecord} to save your position.
+ * <li>On restart restore the position of the consumer using {@link #seek(TopicPartition, long)}.
+ * </ol>
+ * 
+ * This type of usage is simplest when the partition assignment is also done manually (this would be likely in the
+ * search index use case described above). If the partition assignment is done automatically special care will also be
+ * needed to handle the case where partition assignments change. This can be handled using a special callback specified
+ * using <code>rebalance.callback.class</code>, which specifies an implementation of the interface
+ * {@link ConsumerRebalanceCallback}. When partitions are taken from a consumer the consumer will want to commit its
+ * offset for those partitions by implementing
+ * {@link ConsumerRebalanceCallback#onPartitionsRevoked(Consumer, Collection)}. When partitions are assigned to a
+ * consumer, the consumer will want to look up the offset for those new partitions an correctly initialize the consumer
+ * to that position by implementing {@link ConsumerRebalanceCallback#onPartitionsAssigned(Consumer, Collection)}.
+ * <p>
+ * Another common use for {@link ConsumerRebalanceCallback} is to flush any caches the application maintains for
+ * partitions that are moved elsewhere.
+ * 
+ * <h4>Controlling The Consumer's Position</h4>
+ * 
+ * In most use cases the consumer will simply consume records from beginning to end, periodically committing it's
+ * position (either automatically or manually). However Kafka allows the consumer to manually control it's position,
+ * moving forward or backwards in a partition at will. This means a consumer can re-consume older records, or skip to
+ * the most recent records without actually consuming the intermediate records.
+ * <p>
+ * There are several instances where manually controlling the consumer's position can be useful.
+ * <p>
+ * One case is for time-sensitive record processing it may make sense for a consumer that falls far enough behind to not
+ * attempt to catch up processing all records, but rather just skip to the most recent records.
+ * <p>
+ * Another use case is for a system that maintains local state as described in the previous section. In such a system
+ * the consumer will want to initialize it's position on start-up to whatever is contained in the local store. Likewise
+ * if the local state is destroyed (say because the disk is lost) the state may be recreated on a new machine by
+ * reconsuming all the data and recreating the state (assuming that Kafka is retaining sufficient history).
+ * 
+ * Kafka allows specifying the position using {@link #seek(TopicPartition, long)} to specify the new position. Special
+ * methods for seeking to the earliest and latest offset the server maintains are also available (
+ * {@link #seekToBeginning(TopicPartition...)} and {@link #seekToEnd(TopicPartition...)} respectively).
+ * 
+ * <h3>Multithreaded Processing</h3>
+ * 
+ * The Kafka consumer is threadsafe but coarsely synchronized. All network I/O happens in the thread of the application
+ * making the call. We have intentionally avoided implementing a particular threading model for processing.
+ * <p>
+ * This leaves several options for implementing multi-threaded processing of records.
+ * 
+ * <h4>1. One Consumer Per Thread</h4>
+ * 
+ * A simple option is to give each thread it's own consumer instance. Here are the pros and cons of this approach:
+ * <ul>
+ * <li><b>PRO</b>: It is the easiest to implement
+ * <li><b>PRO</b>: It is often the fastest as no inter-thread co-ordination is needed
+ * <li><b>PRO</b>: It makes in-order processing on a per-partition basis very easy to implement (each thread just
+ * processes messages in the order it receives them).
+ * <li><b>CON</b>: More consumers means more TCP connections to the cluster (one per thread). In general Kafka handles
+ * connections very efficiently so this is generally a small cost.
+ * <li><b>CON</b>: Multiple consumers means more requests being sent to the server and slightly less batching of data
+ * which can cause some drop in I/O throughput.
+ * <li><b>CON</b>: The number of total threads across all processes will be limited by the total number of partitions.
+ * </ul>
+ * 
+ * <h4>2. Decouple Consumption and Processing</h4>
+ * 
+ * Another alternative is to have one or more consumer threads that do all data consumption and hands off
+ * {@link ConsumerRecords} instances to a blocking queue consumed by a pool of processor threads that actually handle
+ * the record processing.
+ * 
+ * This option likewise has pros and cons:
+ * <ul>
+ * <li><b>PRO</b>: This option allows independently scaling the number of consumers and processors. This makes it
+ * possible to have a single consumer that feeds many processor threads, avoiding any limitation on partitions.
+ * <li><b>CON</b>: Guaranteeing order across the processors requires particular care as the threads will execute
+ * independently an earlier chunk of data may actually be processed after a later chunk of data just due to the luck of
+ * thread execution timing. For processing that has no ordering requirements this is not a problem.
+ * <li><b>CON</b>: Manually committing the position becomes harder as it requires that all threads co-ordinate to ensure
+ * that processing is complete for that partition.
+ * </ul>
+ * 
+ * There are many possible variations on this approach. For example each processor thread can have it's own queue, and
+ * the consumer threads can hash into these queues using the TopicPartition to ensure in-order consumption and simplify
+ * commit.
+ * 
  */
-public class KafkaConsumer<K,V> implements Consumer<K,V> {
+public class KafkaConsumer<K, V> implements Consumer<K, V> {
 
     private static final Logger log = LoggerFactory.getLogger(KafkaConsumer.class);
+    private static final long EARLIEST_OFFSET_TIMESTAMP = -2L;
+    private static final long LATEST_OFFSET_TIMESTAMP = -1L;
+    private static final AtomicInteger consumerAutoId = new AtomicInteger(1);
 
-    private final long metadataFetchTimeoutMs;
-    private final long totalMemorySize;
-    private final Metrics metrics;
-    private final Set<String> subscribedTopics;
-    private final Set<TopicPartition> subscribedPartitions;
+    private final Time time;
+    private final ConsumerMetrics metrics;
     private final Deserializer<K> keyDeserializer;
     private final Deserializer<V> valueDeserializer;
+    private final SubscriptionState subscriptions;
+    private final Metadata metadata;
+    private final Heartbeat heartbeat;
+    private final NetworkClient client;
+    private final int maxWaitMs;
+    private final int minBytes;
+    private final int fetchSize;
+    private final boolean autoCommit;
+    private final long autoCommitIntervalMs;
+    private final String group;
+    private final long sessionTimeoutMs;
+    private final long retryBackoffMs;
+    private final String partitionAssignmentStrategy;
+    private final AutoOffsetResetStrategy offsetResetStrategy;
+    private final ConsumerRebalanceCallback rebalanceCallback;
+    private final List<PartitionRecords<K, V>> records;
+    private final boolean checkCrcs;
+    private long lastCommitAttemptMs;
+    private String consumerId;
+    private Node consumerCoordinator;
+    private boolean closed = false;
+    private int generation;
 
     /**
      * A consumer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings
-     * are documented <a href="http://kafka.apache.org/documentation.html#consumerconfigs">here</a>. Values can be
-     * either strings or Objects of the appropriate type (for example a numeric configuration would accept either the
+     * are documented <a href="http://kafka.apache.org/documentation.html#consumerconfigs" >here</a>. Values can be
+     * either strings or objects of the appropriate type (for example a numeric configuration would accept either the
      * string "42" or the integer 42).
      * <p>
      * Valid configuration strings are documented at {@link ConsumerConfig}
-     * @param configs   The consumer configs
+     * 
+     * @param configs The consumer configs
      */
     public KafkaConsumer(Map<String, Object> configs) {
-        this(configs, null);
+        this(configs, null, null, null);
     }
 
     /**
-     * A consumer is instantiated by providing a set of key-value pairs as configuration and a {@link ConsumerRebalanceCallback} 
-     * implementation 
+     * A consumer is instantiated by providing a set of key-value pairs as configuration, a
+     * {@link ConsumerRebalanceCallback} implementation, a key and a value {@link Deserializer}.
      * <p>
      * Valid configuration strings are documented at {@link ConsumerConfig}
-     * @param configs   The consumer configs
-     * @param callback  A callback interface that the user can implement to manage customized offsets on the start and end of 
-     *                  every rebalance operation.  
+     * 
+     * @param configs The consumer configs
+     * @param callback A callback interface that the user can implement to manage customized offsets on the start and
+     *            end of every rebalance operation.
+     * @param keyDeserializer The deserializer for key that implements {@link Deserializer}. The configure() method
+     *            won't be called in the consumer when the deserializer is passed in directly.
+     * @param valueDeserializer The deserializer for value that implements {@link Deserializer}. The configure() method
+     *            won't be called in the consumer when the deserializer is passed in directly.
      */
-    public KafkaConsumer(Map<String, Object> configs, ConsumerRebalanceCallback callback) {
-        this(configs, callback, null, null);
-    }
-
-    /**
-     * A consumer is instantiated by providing a set of key-value pairs as configuration, a {@link ConsumerRebalanceCallback}
-     * implementation, a key and a value {@link Deserializer}.
-     * <p>
-     * Valid configuration strings are documented at {@link ConsumerConfig}
-     * @param configs   The consumer configs
-     * @param callback  A callback interface that the user can implement to manage customized offsets on the start and end of
-     *                  every rebalance operation.
-     * @param keyDeserializer  The deserializer for key that implements {@link Deserializer}. The configure() method won't
-     *                         be called in the consumer when the deserializer is passed in directly.
-     * @param valueDeserializer  The deserializer for value that implements {@link Deserializer}. The configure() method
-     *                           won't be called in the consumer when the deserializer is passed in directly.
-     */
-    public KafkaConsumer(Map<String, Object> configs, ConsumerRebalanceCallback callback, Deserializer<K> keyDeserializer, Deserializer<V> valueDeserializer) {
+    public KafkaConsumer(Map<String, Object> configs,
+                         ConsumerRebalanceCallback callback,
+                         Deserializer<K> keyDeserializer,
+                         Deserializer<V> valueDeserializer) {
         this(new ConsumerConfig(addDeserializerToConfig(configs, keyDeserializer, valueDeserializer)),
-             callback, keyDeserializer, valueDeserializer);
+             callback,
+             keyDeserializer,
+             valueDeserializer);
     }
 
     private static Map<String, Object> addDeserializerToConfig(Map<String, Object> configs,
-                                                               Deserializer<?> keyDeserializer, Deserializer<?> valueDeserializer) {
+                                                               Deserializer<?> keyDeserializer,
+                                                               Deserializer<?> valueDeserializer) {
         Map<String, Object> newConfigs = new HashMap<String, Object>();
         newConfigs.putAll(configs);
         if (keyDeserializer != null)
@@ -393,24 +460,13 @@ public class KafkaConsumer<K,V> implements Consumer<K,V> {
     }
 
     /**
-     * A consumer is instantiated by providing a {@link java.util.Properties} object as configuration.      
-     * Valid configuration strings are documented at {@link ConsumerConfig}
+     * A consumer is instantiated by providing a {@link java.util.Properties} object as configuration. Valid
+     * configuration strings are documented at {@link ConsumerConfig} A consumer is instantiated by providing a
+     * {@link java.util.Properties} object as configuration. Valid configuration strings are documented at
+     * {@link ConsumerConfig}
      */
     public KafkaConsumer(Properties properties) {
-        this(properties, null);
-    }
-
-    /**
-     * A consumer is instantiated by providing a {@link java.util.Properties} object as configuration and a 
-     * {@link ConsumerRebalanceCallback} implementation.
-     * <p>
-     * Valid configuration strings are documented at {@link ConsumerConfig}
-     * @param properties The consumer configuration properties
-     * @param callback   A callback interface that the user can implement to manage customized offsets on the start and end of 
-     *                   every rebalance operation.  
-     */
-    public KafkaConsumer(Properties properties, ConsumerRebalanceCallback callback) {
-        this(properties, callback, null, null);
+        this(properties, null, null, null);
     }
 
     /**
@@ -418,21 +474,28 @@ public class KafkaConsumer<K,V> implements Consumer<K,V> {
      * {@link ConsumerRebalanceCallback} implementation, a key and a value {@link Deserializer}.
      * <p>
      * Valid configuration strings are documented at {@link ConsumerConfig}
+     * 
      * @param properties The consumer configuration properties
-     * @param callback   A callback interface that the user can implement to manage customized offsets on the start and end of
-     *                   every rebalance operation.
-     * @param keyDeserializer  The deserializer for key that implements {@link Deserializer}. The configure() method won't
-     *                         be called in the consumer when the deserializer is passed in directly.
-     * @param valueDeserializer  The deserializer for value that implements {@link Deserializer}. The configure() method
-     *                           won't be called in the consumer when the deserializer is passed in directly.
-     */
-    public KafkaConsumer(Properties properties, ConsumerRebalanceCallback callback, Deserializer<K> keyDeserializer, Deserializer<V> valueDeserializer) {
+     * @param callback A callback interface that the user can implement to manage customized offsets on the start and
+     *            end of every rebalance operation.
+     * @param keyDeserializer The deserializer for key that implements {@link Deserializer}. The configure() method
+     *            won't be called in the consumer when the deserializer is passed in directly.
+     * @param valueDeserializer The deserializer for value that implements {@link Deserializer}. The configure() method
+     *            won't be called in the consumer when the deserializer is passed in directly.
+     */
+    public KafkaConsumer(Properties properties,
+                         ConsumerRebalanceCallback callback,
+                         Deserializer<K> keyDeserializer,
+                         Deserializer<V> valueDeserializer) {
         this(new ConsumerConfig(addDeserializerToConfig(properties, keyDeserializer, valueDeserializer)),
-             callback, keyDeserializer, valueDeserializer);
+             callback,
+             keyDeserializer,
+             valueDeserializer);
     }
 
     private static Properties addDeserializerToConfig(Properties properties,
-                                                      Deserializer<?> keyDeserializer, Deserializer<?> valueDeserializer) {
+                                                      Deserializer<?> keyDeserializer,
+                                                      Deserializer<?> valueDeserializer) {
         Properties newProperties = new Properties();
         newProperties.putAll(properties);
         if (keyDeserializer != null)
@@ -442,17 +505,12 @@ public class KafkaConsumer<K,V> implements Consumer<K,V> {
         return newProperties;
     }
 
-    private KafkaConsumer(ConsumerConfig config, ConsumerRebalanceCallback callback, Deserializer<K> keyDeserializer, Deserializer<V> valueDeserializer) {
-        log.trace("Starting the Kafka consumer");
-        subscribedTopics = new HashSet<String>();
-        subscribedPartitions = new HashSet<TopicPartition>();
-        this.metrics = new Metrics(new MetricConfig(),
-                                   Collections.singletonList((MetricsReporter) new JmxReporter("kafka.consumer.")),
-                                   new SystemTime());
-        this.metadataFetchTimeoutMs = config.getLong(ConsumerConfig.METADATA_FETCH_TIMEOUT_CONFIG);
-        this.totalMemorySize = config.getLong(ConsumerConfig.TOTAL_BUFFER_MEMORY_CONFIG);
-        List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG));
-
+    @SuppressWarnings("unchecked")
+    private KafkaConsumer(ConsumerConfig config,
+                          ConsumerRebalanceCallback callback,
+                          Deserializer<K> keyDeserializer,
+                          Deserializer<V> valueDeserializer) {
+        log.debug("Starting the Kafka consumer");
         if (keyDeserializer == null)
             this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
                                                                 Deserializer.class);
@@ -463,181 +521,1072 @@ public class KafkaConsumer<K,V> implements Consumer<K,V> {
                                                                   Deserializer.class);
         else
             this.valueDeserializer = valueDeserializer;
+        if (callback == null)
+            this.rebalanceCallback = config.getConfiguredInstance(ConsumerConfig.CONSUMER_REBALANCE_CALLBACK_CLASS_CONFIG,
+                                                                  ConsumerRebalanceCallback.class);
+        else
+            this.rebalanceCallback = callback;
+        this.time = new SystemTime();
+        this.maxWaitMs = config.getInt(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG);
+        this.minBytes = config.getInt(ConsumerConfig.FETCH_MIN_BYTES_CONFIG);
+        this.fetchSize = config.getInt(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG);
+        this.group = config.getString(ConsumerConfig.GROUP_ID_CONFIG);
+        this.records = new LinkedList<PartitionRecords<K, V>>();
+        this.sessionTimeoutMs = config.getLong(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG);
+        this.heartbeat = new Heartbeat(this.sessionTimeoutMs, time.milliseconds());
+        this.partitionAssignmentStrategy = config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG);
+        this.offsetResetStrategy = AutoOffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG)
+                                                                         .toUpperCase());
+        this.checkCrcs = config.getBoolean(ConsumerConfig.CHECK_CRCS_CONFIG);
+        this.autoCommit = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG);
+        this.autoCommitIntervalMs = config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG);
+        this.lastCommitAttemptMs = time.milliseconds();
+
+        MetricConfig metricConfig = new MetricConfig().samples(config.getInt(ConsumerConfig.METRICS_NUM_SAMPLES_CONFIG))
+                                                      .timeWindow(config.getLong(ConsumerConfig.METRICS_SAMPLE_WINDOW_MS_CONFIG),
+                                                                  TimeUnit.MILLISECONDS);
+        String clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG);
+        String jmxPrefix = "kafka.consumer";
+        if(clientId .length() <= 0)
+          clientId = "consumer-" + consumerAutoId.getAndIncrement();
+        List<MetricsReporter> reporters = config.getConfiguredInstances(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG,
+                                                                        MetricsReporter.class);
+        reporters.add(new JmxReporter(jmxPrefix));
+        Metrics metrics = new Metrics(metricConfig, reporters, time);
+        this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
+        this.metadata = new Metadata(retryBackoffMs, config.getLong(ConsumerConfig.METADATA_MAX_AGE_CONFIG));
+        List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG));
+        this.metadata.update(Cluster.bootstrap(addresses), 0);
+
+        String metricsGroup = "consumer";
+        Map<String, String> metricsTags = new LinkedHashMap<String, String>();
+        metricsTags.put("client-id", clientId);
+        long reconnectBackoffMs = config.getLong(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG);
+        int sendBuffer = config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG);
+        int receiveBuffer = config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG);
+        this.client = new NetworkClient(new Selector(metrics, time, metricsGroup, metricsTags),
+                                        this.metadata,
+                                        clientId,
+                                        100,
+                                        reconnectBackoffMs,
+                                        sendBuffer,
+                                        receiveBuffer);
+        this.subscriptions = new SubscriptionState();
+        this.metrics = new ConsumerMetrics(metrics, metricsGroup, metricsTags);
 
         config.logUnused();
-        log.debug("Kafka consumer started");
+
+        this.consumerCoordinator = null;
+        this.consumerId = "";
+        this.generation = -1;
+        log.debug("Kafka consumer created");
+    }
+
+    /**
+     * The set of partitions currently assigned to this consumer. If subscription happened by directly subscribing to
+     * partitions using {@link #subscribe(TopicPartition...)} then this will simply return the list of partitions that
+     * were subscribed to. If subscription was done by specifying only the topic using {@link #subscribe(String...)}
+     * then this will give the set of topics currently assigned to the consumer (which may be none if the assignment
+     * hasn't happened yet, or the partitions are in the process of getting reassigned).
+     */
+    public synchronized Set<TopicPartition> subscriptions() {
+        return Collections.unmodifiableSet(this.subscriptions.assignedPartitions());
     }
 
     /**
      * Incrementally subscribes to the given list of topics and uses the consumer's group management functionality
      * <p>
-     * As part of group management, the consumer will keep track of the list of consumers that belong to a particular group and
-     * will trigger a rebalance operation if one of the following events trigger -
+     * As part of group management, the consumer will keep track of the list of consumers that belong to a particular
+     * group and will trigger a rebalance operation if one of the following events trigger -
      * <ul>
-     * <li> Number of partitions change for any of the subscribed list of topics 
-     * <li> Topic is created or deleted 
-     * <li> An existing member of the consumer group dies 
-     * <li> A new member is added to an existing consumer group via the join API 
-     * </ul> 
+     * <li>Number of partitions change for any of the subscribed list of topics
+     * <li>Topic is created or deleted
+     * <li>An existing member of the consumer group dies
+     * <li>A new member is added to an existing consumer group via the join API
+     * </ul>
+     * 
      * @param topics A variable list of topics that the consumer wants to subscribe to
      */
     @Override
-    public void subscribe(String... topics) {
-        if(subscribedPartitions.size() > 0)
-            throw new IllegalStateException("Subcription to topics and partitions is mutually exclusive");
-        for(String topic:topics)
-            subscribedTopics.add(topic);
-        // TODO: trigger a rebalance operation
+    public synchronized void subscribe(String... topics) {
+        ensureNotClosed();
+        log.debug("Subscribed to topic(s): ", Utils.join(topics, ", "));
+        for (String topic : topics)
+            this.subscriptions.subscribe(topic);
+        metadata.addTopics(topics);
     }
 
     /**
-     * Incrementally subscribes to a specific topic partition and does not use the consumer's group management functionality. As such,
-     * there will be no rebalance operation triggered when group membership or cluster and topic metadata change.
+     * Incrementally subscribes to a specific topic partition and does not use the consumer's group management
+     * functionality. As such, there will be no rebalance operation triggered when group membership or cluster and topic
+     * metadata change.
      * <p>
+     * 
      * @param partitions Partitions to incrementally subscribe to
      */
     @Override
-    public void subscribe(TopicPartition... partitions) {
-        if(subscribedTopics.size() > 0)
-            throw new IllegalStateException("Subcription to topics and partitions is mutually exclusive");
-        for(TopicPartition partition:partitions)
-            subscribedPartitions.add(partition);
+    public synchronized void subscribe(TopicPartition... partitions) {
+        ensureNotClosed();
+        log.debug("Subscribed to partitions(s): ", Utils.join(partitions, ", "));
+        for (TopicPartition tp : partitions) {
+            this.subscriptions.subscribe(tp);
+            metadata.addTopics(tp.topic());
+        }
     }
 
     /**
-     * Unsubscribe from the specific topics. This will trigger a rebalance operation and messages for this topic will not be returned 
-     * from the next {@link #poll(long) poll()} onwards
+     * Unsubscribe from the specific topics. This will trigger a rebalance operation and records for this topic will not
+     * be returned from the next {@link #poll(long) poll()} onwards
+     * 
      * @param topics Topics to unsubscribe from
      */
-    public void unsubscribe(String... topics) {   
+    public synchronized void unsubscribe(String... topics) {
+        ensureNotClosed();
+        log.debug("Unsubscribed from topic(s): ", Utils.join(topics, ", "));
         // throw an exception if the topic was never subscribed to
-        for(String topic:topics) {
-            if(!subscribedTopics.contains(topic))
-                throw new IllegalStateException("Topic " + topic + " was never subscribed to. subscribe(" + topic + ") should be called prior" +
-                		" to unsubscribe(" + topic + ")");
-            subscribedTopics.remove(topic);
-        }
-        // TODO trigger a rebalance operation
+        for (String topic : topics)
+            this.subscriptions.unsubscribe(topic);
     }
 
     /**
-     * Unsubscribe from the specific topic partitions. Messages for these partitions will not be returned from the next 
+     * Unsubscribe from the specific topic partitions. records for these partitions will not be returned from the next
      * {@link #poll(long) poll()} onwards
+     * 
      * @param partitions Partitions to unsubscribe from
      */
-    public void unsubscribe(TopicPartition... partitions) {        
+    public synchronized void unsubscribe(TopicPartition... partitions) {
+        ensureNotClosed();
+        log.debug("Unsubscribed from partitions(s): ", Utils.join(partitions, ", "));
         // throw an exception if the partition was never subscribed to
-        for(TopicPartition partition:partitions) {
-            if(!subscribedPartitions.contains(partition))
-                throw new IllegalStateException("Partition " + partition + " was never subscribed to. subscribe(new TopicPartition(" + 
-                                                 partition.topic() + "," + partition.partition() + ") should be called prior" +
-                                                " to unsubscribe(new TopicPartition(" + partition.topic() + "," + partition.partition() + ")");
-            subscribedPartitions.remove(partition);                
-        }
-        // trigger a rebalance operation
+        for (TopicPartition partition : partitions)
+            this.subscriptions.unsubscribe(partition);
     }
-    
+
     /**
-     * Fetches data for the topics or partitions specified using one of the subscribe APIs. It is an error to not have subscribed to
-     * any topics or partitions before polling for data.
-     * <p> 
-     * The offset used for fetching the data is governed by whether or not {@link #seek(Map) seek(offsets)}
-     * is used. If {@link #seek(Map) seek(offsets)} is used, it will use the specified offsets on startup and
-     * on every rebalance, to consume data from that offset sequentially on every poll. If not, it will use the last checkpointed offset 
-     * using {@link #commit(Map, boolean) commit(offsets, sync)} 
-     * for the subscribed list of partitions.
-     * @param timeout  The time, in milliseconds, spent waiting in poll if data is not available. If 0, waits indefinitely. Must not be negative
+     * Fetches data for the topics or partitions specified using one of the subscribe APIs. It is an error to not have
+     * subscribed to any topics or partitions before polling for data.
+     * <p>
+     * The offset used for fetching the data is governed by whether or not {@link #seek(TopicPartition, long)} is used.
+     * If {@link #seek(TopicPartition, long)} is used, it will use the specified offsets on startup and on every
+     * rebalance, to consume data from that offset sequentially on every poll. If not, it will use the last checkpointed
+     * offset using {@link #commit(Map, CommitType) commit(offsets, sync)} for the subscribed list of partitions.
+     * 
+     * @param timeout The time, in milliseconds, spent waiting in poll if data is not available. If 0, waits
+     *            indefinitely. Must not be negative
      * @return map of topic to records since the last fetch for the subscribed list of topics and partitions
+     * 
+     * @throws NoOffsetForPartitionException If there is no stored offset for a subscribed partition and no automatic
+     *             offset reset policy has been configured.
      */
     @Override
-    public Map<String, ConsumerRecords<K,V>> poll(long timeout) {
-        // TODO Auto-generated method stub
-        return null;
+    public synchronized ConsumerRecords<K, V> poll(long timeout) {
+        ensureNotClosed();
+        long now = time.milliseconds();
+
+        if (subscriptions.partitionsAutoAssigned()) {
+            // get partition assignment if needed
+            if (subscriptions.needsPartitionAssignment()) {
+                joinGroup(now);
+            } else if (!heartbeat.isAlive(now)) {
+                log.error("Failed heartbeat check.");
+                coordinatorDead();
+            } else if (heartbeat.shouldHeartbeat(now)) {
+                initiateHeartbeat(now);
+            }
+        }
+
+        // fetch positions if we have partitions we're subscribed to that we
+        // don't know the offset for
+        if (!subscriptions.hasAllFetchPositions())
+            fetchMissingPositionsOrResetThem(this.subscriptions.missingFetchPositions(), now);
+
+        // maybe autocommit position
+        if (shouldAutoCommit(now))
+            commit(CommitType.ASYNC);
+
+        /*
+         * initiate any needed fetches, then block for the timeout the user specified
+         */
+        Cluster cluster = this.metadata.fetch();
+        reinstateFetches(cluster, now);
+        client.poll(timeout, now);
+
+        /*
+         * initiate a fetch request for any nodes that we just got a response from without blocking
+         */
+        reinstateFetches(cluster, now);
+        client.poll(0, now);
+
+        return new ConsumerRecords<K, V>(consumeBufferedRecords());
     }
 
     /**
      * Commits the specified offsets for the specified list of topics and partitions to Kafka.
      * <p>
-     * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after every rebalance
-     * and also on startup. As such, if you need to store offsets in anything other than Kafka, this API should not be used.
-     * @param offsets The list of offsets per partition that should be committed to Kafka. 
-     * @param sync If true, commit will block until the consumer receives an acknowledgment  
-     * @return An {@link OffsetMetadata} object that contains the partition, offset and a corresponding error code. Returns null
-     * if the sync flag is set to false.  
+     * This commits offsets to Kafka. The offsets committed using this API will be used on the first fetch after every
+     * rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
+     * should not be used.
+     * <p>
+     * A non-blocking commit will attempt to commit offsets asychronously. No error will be thrown if the commit fails.
+     * A blocking commit will wait for a response acknowledging the commit. In the event of an error it will retry until
+     * the commit succeeds.
+     * 
+     * @param offsets The list of offsets per partition that should be committed to Kafka.
+     * @param commitType Control whether the commit is blocking
      */
     @Override
-    public OffsetMetadata commit(Map<TopicPartition, Long> offsets, boolean sync) {
-        throw new UnsupportedOperationException();
+    public synchronized void commit(final Map<TopicPartition, Long> offsets, CommitType commitType) {
+        ensureNotClosed();
+        log.debug("Committing offsets ({}): {} ", commitType.toString().toLowerCase(), offsets);
+        long now = time.milliseconds();
+        this.lastCommitAttemptMs = now;
+        if (!offsets.isEmpty()) {
+            Map<TopicPartition, OffsetCommitRequest.PartitionData> offsetData = new HashMap<TopicPartition, OffsetCommitRequest.PartitionData>(offsets.size());
+            for (Map.Entry<TopicPartition, Long> entry : offsets.entrySet())
+                offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(entry.getValue(), now, ""));
+            OffsetCommitRequest req = new OffsetCommitRequest(this.group, this.generation, this.consumerId, offsetData);
+
+            RequestCompletionHandler handler = new RequestCompletionHandler() {
+                public void onComplete(ClientResponse resp) {
+                    if (resp.wasDisconnected()) {
+                        handleDisconnect(resp, time.milliseconds());
+                    } else {
+                        OffsetCommitResponse response = new OffsetCommitResponse(resp.responseBody());
+                        for (Map.Entry<TopicPartition, Short> entry : response.responseData().entrySet()) {
+                            TopicPartition tp = entry.getKey();
+                            short errorCode = entry.getValue();
+                            long offset = offsets.get(tp);
+                            if (errorCode == Errors.NONE.code()) {
+                                log.debug("Committed offset {} for partition {}", offset, tp);
+                                subscriptions.committed(tp, offset);
+                            } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
+                                    || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
+                                coordinatorDead();
+                            } else {
+                                log.error("Error committing partition {} at offset {}: {}",
+                                          tp,
+                                          offset,
+                                          Errors.forCode(errorCode).exception().getMessage());
+                            }
+                        }
+                    }
+                    metrics.commitLatency.record(resp.requestLatencyMs());
+                }
+            };
+
+            if (commitType == CommitType.ASYNC) {
+                this.initiateCoordinatorRequest(ApiKeys.OFFSET_COMMIT, req.toStruct(), handler, now);
+                return;
+            } else {
+                boolean done;
+                do {
+                    ClientResponse response = blockingCoordinatorRequest(ApiKeys.OFFSET_COMMIT,
+                                                                         req.toStruct(),
+                                                                         handler,
+                                                                         now);
+
+                    // check for errors
+                    done = true;
+                    OffsetCommitResponse commitResponse = new OffsetCommitResponse(response.responseBody());
+                    for (short errorCode : commitResponse.responseData().values()) {
+                        if (errorCode != Errors.NONE.code())
+                            done = false;
+                    }
+                    if (!done) {
+                        log.debug("Error in offset commit, backing off for {} ms before retrying again.",
+                                  this.retryBackoffMs);
+                        Utils.sleep(this.retryBackoffMs);
+                    }
+                } while (!done);
+            }
+        }
     }
 
     /**
-     * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and 
-     * partitions. 
+     * Commits offsets returned on the last {@link #poll(long) poll()} for the subscribed list of topics and partitions.
      * <p>
-     * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after every rebalance
-     * and also on startup. As such, if you need to store offsets in anything other than Kafka, this API should not be used.
-     * @param sync If true, commit will block until the consumer receives an acknowledgment  
-     * @return An {@link OffsetMetadata} object that contains the partition, offset and a corresponding error code. Returns null
-     * if the sync flag is set to false.
+     * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after
+     * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API
+     * should not be used.
+     * 
+     * @param commitType Whether or not the commit should block until it is acknowledged.
      */
     @Override
-    public OffsetMetadata commit(boolean sync) {
-        throw new UnsupportedOperationException();
+    public synchronized void commit(CommitType commitType) {
+        ensureNotClosed();
+        commit(this.subscriptions.allConsumed(), commitType);
     }
 
     /**
-     * Overrides the fetch offsets that the consumer will use on the next {@link #poll(long) poll(timeout)}. If this API is invoked
-     * for the same partition more than once, the latest offset will be used on the next poll(). Note that you may lose data if this API is 
-     * arbitrarily used in the middle of consumption, to reset the fetch offsets  
+     * Overrides the fetch offsets that the consumer will use on the next {@link #poll(long) poll(timeout)}. If this API
+     * is invoked for the same partition more than once, the latest offset will be used on the next poll(). Note that
+     * you may lose data if this API is arbitrarily used in the middle of consumption, to reset the fetch offsets
      */
     @Override
-    public void seek(Map<TopicPartition, Long> offsets) {
+    public synchronized void seek(TopicPartition partition, long offset) {
+        ensureNotClosed();
+        log.debug("Seeking to offset {} for partition {}", offset, partition);
+        this.subscriptions.seek(partition, offset);
     }
 
     /**
-     * Returns the fetch position of the <i>next message</i> for the specified topic partition to be used on the next {@link #poll(long) poll()}
-     * @param partitions Partitions for which the fetch position will be returned
-     * @return The position from which data will be fetched for the specified partition on the next {@link #poll(long) poll()}
+     * Seek to the first offset for each of the given partitions
      */
-    public Map<TopicPartition, Long> position(Collection<TopicPartition> partitions) {
-        return null;
+    public synchronized void seekToBeginning(TopicPartition... partitions) {
+        ensureNotClosed();
+        Collection<TopicPartition> parts = partitions.length == 0 ? this.subscriptions.assignedPartitions()
+                : Arrays.asList(partitions);
+        for (TopicPartition tp : parts) {
+            // TODO: list offset call could be optimized by grouping by node
+            seek(tp, listOffset(tp, EARLIEST_OFFSET_TIMESTAMP));
+        }
     }
 
     /**
-     * Fetches the last committed offsets of partitions that the consumer currently consumes. This API is only relevant if Kafka based offset
-     * storage is used. This API can be used in conjunction with {@link #seek(Map) seek(offsets)} to rewind consumption of data.  
-     * @param partitions The list of partitions to return the last committed offset for
-     * @return The list of offsets committed on the last {@link #commit(boolean) commit(sync)} 
+     * Seek to the last offset for each of the given partitions
      */
-    @Override
-    public Map<TopicPartition, Long> committed(Collection<TopicPartition> partitions) {
-        // TODO Auto-generated method stub
-        throw new UnsupportedOperationException();
+    public synchronized void seekToEnd(TopicPartition... partitions) {
+        ensureNotClosed();
+        Collection<TopicPartition> parts = partitions.length == 0 ? this.subscriptions.assignedPartitions()
+                : Arrays.asList(partitions);
+        for (TopicPartition tp : parts) {
+            // TODO: list offset call could be optimized by grouping by node
+            seek(tp, listOffset(tp, LATEST_OFFSET_TIMESTAMP));
+        }
     }
 
     /**
-     * Fetches offsets before a certain timestamp. Note that the offsets returned are approximately computed and do not correspond to the exact
-     * message at the given timestamp. As such, if the consumer is rewound to offsets returned by this API, there may be duplicate messages 
-     * returned by the consumer. 
-     * @param partitions The list of partitions for which the offsets are returned
-     * @param timestamp The unix timestamp. Value -1 indicates earliest available timestamp. Value -2 indicates latest available timestamp. 
-     * @return The offsets per partition before the specified timestamp.
-     */
-    public Map<TopicPartition, Long> offsetsBeforeTime(long timestamp, Collection<TopicPartition> partitions) {
-        return null;
+     * Returns the offset of the <i>next record</i> that will be fetched (if a record with that offset exists).
+     * 
+     * @param partition The partition to get the position for
+     * @return The offset
+     * @throws NoOffsetForPartitionException If a position hasn't been set for a given partition, and no reset policy is
+     *             available.
+     */
+    public synchronized long position(TopicPartition partition) {
+        ensureNotClosed();
+        if (!this.subscriptions.assignedPartitions().contains(partition))
+            throw new IllegalArgumentException("You can only check the position for partitions assigned to this consumer.");
+        Long offset = this.subscriptions.consumed(partition);
+        if (offset == null) {
+            fetchMissingPositionsOrResetThem(Collections.singleton(partition), time.milliseconds());
+            return this.subscriptions.consumed(partition);
+        } else {
+            return offset;
+        }
     }
 
+    /**
+     * Fetches the last committed offset for the given partition (whether the commit happened by this process or
+     * another). This offset will be used as the position for the consumer in the event of a failure.
+     * <p>
+     * This call may block to do a remote call if the partition in question isn't assigned to this consumer or if the
+     * consumer hasn't yet initialized it's cache of committed offsets.
+     * 
+     * @param partition The partition to check
+     * @return The last committed offset or null if no offset has been committed
+     * @throws NoOffsetForPartitionException If no offset has ever been committed by any process for the given
+     *             partition.
+     */
+    @Override
+    public synchronized long committed(TopicPartition partition) {
+        ensureNotClosed();
+        Set<TopicPartition> partitionsToFetch;
+        if (subscriptions.assignedPartitions().contains(partition)) {
+            Long committed = this.subscriptions.committed(partition);
+            if (committed != null)
+                return committed;
+            partitionsToFetch = subscriptions.assignedPartitions();
+        } else {
+            partitionsToFetch = Collections.singleton(partition);
+        }
+        this.refreshCommittedOffsets(time.milliseconds(), partitionsToFetch);
+        Long committed = this.subscriptions.committed(partition);
+        if (committed == null)
+            throw new NoOffsetForPartitionException("No offset has been committed for partition " + partition);
+        return committed;
+    }
+
+    /**
+     * Get the metrics kept by the consumer
+     */
     @Override
     public Map<MetricName, ? extends Metric> metrics() {
-        return Collections.unmodifiableMap(this.metrics.metrics());
+        return Collections.unmodifiableMap(this.metrics.metrics.metrics());
     }
 
+    /**
+     * Get metadata about the partitions for a given topic. This method will issue a remote call to the server if it
+     * does not already have any metadata about the given topic.
+     * 
+     * @param topic The topic to get partition metadata for
+     * @return The list of partitions
+     */
     @Override
-    public void close() {
+    public List<PartitionInfo> partitionsFor(String topic) {
+        Cluster cluster = this.metadata.fetch();
+        List<PartitionInfo> parts = cluster.partitionsForTopic(topic);
+        if (parts == null) {
+            metadata.add(topic);
+            awaitMetadataUpdate();
+            parts = metadata.fetch().partitionsForTopic(topic);
+        }
+        return parts;
+    }
+
+    @Override
+    public synchronized void close() {
         log.trace("Closing the Kafka consumer.");
-        subscribedTopics.clear();
-        subscribedPartitions.clear();
-        this.metrics.close();
+        this.closed = true;
+        this.metrics.metrics.close();
+        this.client.close();
         log.debug("The Kafka consumer has closed.");
     }
+
+    private boolean shouldAutoCommit(long now) {
+        return this.autoCommit && this.lastCommitAttemptMs <= now - this.autoCommitIntervalMs;
+    }
+
+    /*
+     * Request a metadata update and wait until it has occurred
+     */
+    private void awaitMetadataUpdate() {
+        int version = this.metadata.requestUpdate();
+        do {
+            long now = time.milliseconds();
+            this.client.poll(this.retryBackoffMs, now);
+        } while (this.metadata.version() == version);
+    }
+
+    /*
+     * Send a join group request to the controller
+     */
+    private void joinGroup(long now) {
+        log.debug("Joining group {}", group);
+
+        // execute the user's callback
+        try {
+            // TODO: Hmmm, is passing the full Consumer like this actually safe?
+            // Need to think about reentrancy...
+            this.rebalanceCallback.onPartitionsRevoked(this, this.subscriptions.assignedPartitions());
+        } catch (Exception e) {
+            log.error("User provided callback " + this.rebalanceCallback.getClass().getName()
+                    + " failed on partition revocation: ", e);
+        }
+
+        // join the group
+        JoinGroupRequest jgr = new JoinGroupRequest(group,
+                                                    (int) this.sessionTimeoutMs,
+                                                    new ArrayList<String>(this.subscriptions.subscribedTopics()),
+                                                    this.consumerId,
+                                                    this.partitionAssignmentStrategy);
+        ClientResponse resp = this.blockingCoordinatorRequest(ApiKeys.JOIN_GROUP, jgr.toStruct(), null, now);
+        // process the response
+        JoinGroupResponse response = new JoinGroupResponse(resp.responseBody());
+        log.debug("Joined group: {}", response);
+        Errors.forCode(response.errorCode()).maybeThrow();
+        this.consumerId = response.consumerId();
+        this.subscriptions.changePartitionAssignment(response.assignedPartitions());
+        this.heartbeat.receivedResponse(now);
+
+        // execute the callback
+        try {
+            // TODO: Hmmm, is passing the full Consumer like this actually safe?
+            this.rebalanceCallback.onPartitionsAssigned(this, this.subscriptions.assignedPartitions());
+        } catch (Exception e) {
+            log.error("User provided callback " + this.rebalanceCallback.getClass().getName()
+                    + " failed on partition assignment: ", e);
+        }
+
+        // record re-assignment time
+        this.metrics.partitionReassignments.record(time.milliseconds() - now);
+    }
+
+    /*
+     * Empty the record buffer and update the consumed position.
+     */
+    private Map<TopicPartition, List<ConsumerRecord<K, V>>> consumeBufferedRecords() {
+        if (this.subscriptions.needsPartitionAssignment()) {
+            return Collections.emptyMap();
+        } else {
+            Map<TopicPartition, List<ConsumerRecord<K, V>>> drained = new HashMap<TopicPartition, List<ConsumerRecord<K, V>>>();
+            for (PartitionRecords<K, V> part : this.records) {
+                Long consumed = subscriptions.consumed(part.partition);
+                if (this.subscriptions.assignedPartitions().contains(part.partition)
+                        && (consumed == null || part.fetchOffset == consumed)) {
+                    List<ConsumerRecord<K, V>> partRecs = drained.get(part.partition);
+                    if (partRecs == null) {
+                        partRecs = part.records;
+                        drained.put(part.partition, partRecs);
+                    } else {
+                        partRecs.addAll(part.records);
+    

<TRUNCATED>