You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by gu...@apache.org on 2015/10/21 21:08:45 UTC

[1/8] kafka git commit: KAFKA-2464: client-side assignment for new consumer

Repository: kafka
Updated Branches:
  refs/heads/trunk 6e747d429 -> 86eb74d92


http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/unit/kafka/coordinator/GroupMetadataTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataTest.scala
new file mode 100644
index 0000000..0f3e748
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataTest.scala
@@ -0,0 +1,249 @@
+/**
+ * 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.coordinator
+
+import org.junit.Assert._
+import org.junit.{Before, Test}
+import org.scalatest.junit.JUnitSuite
+
+/**
+ * Test group state transitions and other GroupMetadata functionality
+ */
+class GroupMetadataTest extends JUnitSuite {
+  var group: GroupMetadata = null
+
+  @Before
+  def setUp() {
+    group = new GroupMetadata("groupId", "consumer")
+  }
+
+  @Test
+  def testCanRebalanceWhenStable() {
+    assertTrue(group.canRebalance)
+  }
+
+  @Test
+  def testCanRebalanceWhenAwaitingSync(){
+    group.transitionTo(PreparingRebalance)
+    group.transitionTo(AwaitingSync)
+    assertTrue(group.canRebalance)
+  }
+
+  @Test
+  def testCannotRebalanceWhenPreparingRebalance() {
+    group.transitionTo(PreparingRebalance)
+    assertFalse(group.canRebalance)
+  }
+
+  @Test
+  def testCannotRebalanceWhenDead() {
+    group.transitionTo(PreparingRebalance)
+    group.transitionTo(Dead)
+    assertFalse(group.canRebalance)
+  }
+
+  @Test
+  def testStableToPreparingRebalanceTransition() {
+    group.transitionTo(PreparingRebalance)
+    assertState(group, PreparingRebalance)
+  }
+
+  @Test
+  def testAwaitingSyncToPreparingRebalanceTransition() {
+    group.transitionTo(PreparingRebalance)
+    group.transitionTo(AwaitingSync)
+    group.transitionTo(PreparingRebalance)
+    assertState(group, PreparingRebalance)
+  }
+
+  @Test
+  def testPreparingRebalanceToDeadTransition() {
+    group.transitionTo(PreparingRebalance)
+    group.transitionTo(Dead)
+    assertState(group, Dead)
+  }
+
+  @Test
+  def testAwaitingSyncToStableTransition() {
+    group.transitionTo(PreparingRebalance)
+    group.transitionTo(AwaitingSync)
+    group.transitionTo(Stable)
+    assertState(group, Stable)
+  }
+
+  @Test(expected = classOf[IllegalStateException])
+  def testStableToStableIllegalTransition() {
+    group.transitionTo(Stable)
+  }
+
+  @Test(expected = classOf[IllegalStateException])
+  def testStableToAwaitingSyncIllegalTransition() {
+    group.transitionTo(AwaitingSync)
+  }
+
+  @Test(expected = classOf[IllegalStateException])
+  def testStableToDeadIllegalTransition() {
+    group.transitionTo(Dead)
+  }
+
+  @Test(expected = classOf[IllegalStateException])
+  def testPreparingRebalanceToPreparingRebalanceIllegalTransition() {
+    group.transitionTo(PreparingRebalance)
+    group.transitionTo(PreparingRebalance)
+  }
+
+  @Test(expected = classOf[IllegalStateException])
+  def testPreparingRebalanceToStableIllegalTransition() {
+    group.transitionTo(PreparingRebalance)
+    group.transitionTo(Stable)
+  }
+
+  @Test(expected = classOf[IllegalStateException])
+  def testAwaitingSyncToAwaitingSyncIllegalTransition() {
+    group.transitionTo(PreparingRebalance)
+    group.transitionTo(AwaitingSync)
+    group.transitionTo(AwaitingSync)
+  }
+
+  @Test(expected = classOf[IllegalStateException])
+  def testAwaitingSyncToDeadIllegalTransition() {
+    group.transitionTo(PreparingRebalance)
+    group.transitionTo(AwaitingSync)
+    group.transitionTo(Dead)
+  }
+
+  @Test(expected = classOf[IllegalStateException])
+  def testDeadToDeadIllegalTransition() {
+    group.transitionTo(PreparingRebalance)
+    group.transitionTo(Dead)
+    group.transitionTo(Dead)
+  }
+
+  @Test(expected = classOf[IllegalStateException])
+  def testDeadToStableIllegalTransition() {
+    group.transitionTo(PreparingRebalance)
+    group.transitionTo(Dead)
+    group.transitionTo(Stable)
+  }
+
+  @Test(expected = classOf[IllegalStateException])
+  def testDeadToPreparingRebalanceIllegalTransition() {
+    group.transitionTo(PreparingRebalance)
+    group.transitionTo(Dead)
+    group.transitionTo(PreparingRebalance)
+  }
+
+  @Test(expected = classOf[IllegalStateException])
+  def testDeadToAwaitingSyncIllegalTransition() {
+    group.transitionTo(PreparingRebalance)
+    group.transitionTo(Dead)
+    group.transitionTo(AwaitingSync)
+  }
+
+  @Test
+  def testSelectProtocol() {
+    val groupId = "groupId"
+
+    val sessionTimeoutMs = 10000
+
+    val memberId = "memberId"
+    val member = new MemberMetadata(memberId, groupId, sessionTimeoutMs,
+      List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte])))
+
+    group.add(memberId, member)
+    assertEquals("range", group.selectProtocol)
+
+    val otherMemberId = "otherMemberId"
+    val otherMember = new MemberMetadata(otherMemberId, groupId, sessionTimeoutMs,
+      List(("roundrobin", Array.empty[Byte]), ("range", Array.empty[Byte])))
+
+    group.add(otherMemberId, otherMember)
+    // now could be either range or robin since there is no majority preference
+    assertTrue(Set("range", "roundrobin")(group.selectProtocol))
+
+    val lastMemberId = "lastMemberId"
+    val lastMember = new MemberMetadata(lastMemberId, groupId, sessionTimeoutMs,
+      List(("roundrobin", Array.empty[Byte]), ("range", Array.empty[Byte])))
+
+    group.add(lastMemberId, lastMember)
+    // now we should prefer 'roundrobin'
+    assertEquals("roundrobin", group.selectProtocol)
+  }
+
+  @Test(expected = classOf[IllegalStateException])
+  def testSelectProtocolRaisesIfNoMembers() {
+    group.selectProtocol
+    fail()
+  }
+
+  @Test
+  def testSelectProtocolChoosesCompatibleProtocol() {
+    val groupId = "groupId"
+
+    val sessionTimeoutMs = 10000
+
+    val memberId = "memberId"
+    val member = new MemberMetadata(memberId, groupId, sessionTimeoutMs,
+      List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte])))
+
+    val otherMemberId = "otherMemberId"
+    val otherMember = new MemberMetadata(otherMemberId, groupId, sessionTimeoutMs,
+      List(("roundrobin", Array.empty[Byte]), ("blah", Array.empty[Byte])))
+
+    group.add(memberId, member)
+    group.add(otherMemberId, otherMember)
+    assertEquals("roundrobin", group.selectProtocol)
+  }
+
+  @Test
+  def testSupportsProtocols() {
+    val groupId = "groupId"
+
+    val sessionTimeoutMs = 10000
+
+    // by default, the group supports everything
+    assertTrue(group.supportsProtocols(Set("roundrobin", "range")))
+
+    val memberId = "memberId"
+    val member = new MemberMetadata(memberId, groupId, sessionTimeoutMs,
+      List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte])))
+
+    group.add(memberId, member)
+    assertTrue(group.supportsProtocols(Set("roundrobin", "foo")))
+    assertTrue(group.supportsProtocols(Set("range", "foo")))
+    assertFalse(group.supportsProtocols(Set("foo", "bar")))
+
+    val otherMemberId = "otherMemberId"
+    val otherMember = new MemberMetadata(otherMemberId, groupId, sessionTimeoutMs,
+      List(("roundrobin", Array.empty[Byte]), ("blah", Array.empty[Byte])))
+
+    group.add(otherMemberId, otherMember)
+
+    assertTrue(group.supportsProtocols(Set("roundrobin", "foo")))
+    assertFalse(group.supportsProtocols(Set("range", "foo")))
+  }
+
+  private def assertState(group: GroupMetadata, targetState: GroupState) {
+    val states: Set[GroupState] = Set(Stable, PreparingRebalance, AwaitingSync, Dead)
+    val otherStates = states - targetState
+    otherStates.foreach { otherState =>
+      assertFalse(group.is(otherState))
+    }
+    assertTrue(group.is(targetState))
+  }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/unit/kafka/coordinator/MemberMetadataTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/MemberMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/MemberMetadataTest.scala
new file mode 100644
index 0000000..0a5bb3c
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/coordinator/MemberMetadataTest.scala
@@ -0,0 +1,90 @@
+/**
+ * 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.coordinator
+
+import java.util
+
+import org.junit.Assert._
+import org.junit.Test
+import org.scalatest.junit.JUnitSuite
+
+class MemberMetadataTest extends JUnitSuite {
+
+  @Test
+  def testMatchesSupportedProtocols {
+    val groupId = "groupId"
+    val memberId = "memberId"
+    val sessionTimeoutMs = 10000
+    val protocols = List(("range", Array.empty[Byte]))
+
+    val member = new MemberMetadata(memberId, groupId, sessionTimeoutMs, protocols)
+    assertTrue(member.matches(protocols))
+    assertFalse(member.matches(List(("range", Array[Byte](0)))))
+    assertFalse(member.matches(List(("roundrobin", Array.empty[Byte]))))
+    assertFalse(member.matches(List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte]))))
+  }
+
+  @Test
+  def testVoteForPreferredProtocol {
+    val groupId = "groupId"
+    val memberId = "memberId"
+    val sessionTimeoutMs = 10000
+    val protocols = List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte]))
+
+    val member = new MemberMetadata(memberId, groupId, sessionTimeoutMs, protocols)
+    assertEquals("range", member.vote(Set("range", "roundrobin")))
+    assertEquals("roundrobin", member.vote(Set("blah", "roundrobin")))
+  }
+
+  @Test
+  def testMetadata {
+    val groupId = "groupId"
+    val memberId = "memberId"
+    val sessionTimeoutMs = 10000
+    val protocols = List(("range", Array[Byte](0)), ("roundrobin", Array[Byte](1)))
+
+    val member = new MemberMetadata(memberId, groupId, sessionTimeoutMs, protocols)
+    assertTrue(util.Arrays.equals(Array[Byte](0), member.metadata("range")))
+    assertTrue(util.Arrays.equals(Array[Byte](1), member.metadata("roundrobin")))
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testMetadataRaisesOnUnsupportedProtocol {
+    val groupId = "groupId"
+    val memberId = "memberId"
+    val sessionTimeoutMs = 10000
+    val protocols = List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte]))
+
+    val member = new MemberMetadata(memberId, groupId, sessionTimeoutMs, protocols)
+    member.metadata("blah")
+    fail()
+  }
+
+  @Test(expected = classOf[IllegalArgumentException])
+  def testVoteRaisesOnNoSupportedProtocols {
+    val groupId = "groupId"
+    val memberId = "memberId"
+    val sessionTimeoutMs = 10000
+    val protocols = List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte]))
+
+    val member = new MemberMetadata(memberId, groupId, sessionTimeoutMs, protocols)
+    member.vote(Set("blah"))
+    fail()
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/unit/kafka/coordinator/PartitionAssignorTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/PartitionAssignorTest.scala b/core/src/test/scala/unit/kafka/coordinator/PartitionAssignorTest.scala
deleted file mode 100644
index 79c691f..0000000
--- a/core/src/test/scala/unit/kafka/coordinator/PartitionAssignorTest.scala
+++ /dev/null
@@ -1,305 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package kafka.coordinator
-
-import kafka.common.TopicAndPartition
-
-import org.junit.Assert._
-import org.junit.Test
-import org.scalatest.junit.JUnitSuite
-
-class PartitionAssignorTest extends JUnitSuite {
-
-  @Test
-  def testRangeAssignorOneConsumerNoTopic() {
-    val consumer = "consumer"
-    val assignor = new RangeAssignor()
-    val topicsPerConsumer = Map(consumer -> Set.empty[String])
-    val partitionsPerTopic = Map.empty[String, Int]
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(consumer -> Set.empty[TopicAndPartition])
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRangeAssignorOneConsumerNonexistentTopic() {
-    val topic = "topic"
-    val consumer = "consumer"
-    val assignor = new RangeAssignor()
-    val topicsPerConsumer = Map(consumer -> Set(topic))
-    val partitionsPerTopic = Map(topic -> 0)
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(consumer -> Set.empty[TopicAndPartition])
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRangeAssignorOneConsumerOneTopic() {
-    val topic = "topic"
-    val consumer = "consumer"
-    val numPartitions = 3
-    val assignor = new RangeAssignor()
-    val topicsPerConsumer = Map(consumer -> Set(topic))
-    val partitionsPerTopic = Map(topic -> numPartitions)
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(consumer -> topicAndPartitions(Map(topic -> Set(0, 1, 2))))
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRangeAssignorOnlyAssignsPartitionsFromSubscribedTopics() {
-    val subscribedTopic = "topic"
-    val otherTopic = "other"
-    val consumer = "consumer"
-    val subscribedTopicNumPartitions = 3
-    val otherTopicNumPartitions = 3
-    val assignor = new RangeAssignor()
-    val topicsPerConsumer = Map(consumer -> Set(subscribedTopic))
-    val partitionsPerTopic = Map(subscribedTopic -> subscribedTopicNumPartitions, otherTopic -> otherTopicNumPartitions)
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(consumer -> topicAndPartitions(Map(subscribedTopic -> Set(0, 1, 2))))
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRangeAssignorOneConsumerMultipleTopics() {
-    val topic1 = "topic1"
-    val topic2 = "topic2"
-    val consumer = "consumer"
-    val numTopic1Partitions = 1
-    val numTopic2Partitions = 2
-    val assignor = new RangeAssignor()
-    val topicsPerConsumer = Map(consumer -> Set(topic1, topic2))
-    val partitionsPerTopic = Map(topic1 -> numTopic1Partitions, topic2 -> numTopic2Partitions)
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(consumer -> topicAndPartitions(Map(topic1 -> Set(0), topic2 -> Set(0, 1))))
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRangeAssignorTwoConsumersOneTopicOnePartition() {
-    val topic = "topic"
-    val consumer1 = "consumer1"
-    val consumer2 = "consumer2"
-    val numPartitions = 1
-    val assignor = new RangeAssignor()
-    val topicsPerConsumer = Map(consumer1 -> Set(topic), consumer2 -> Set(topic))
-    val partitionsPerTopic = Map(topic -> numPartitions)
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(
-      consumer1 -> topicAndPartitions(Map(topic -> Set(0))),
-      consumer2 -> Set.empty[TopicAndPartition])
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRangeAssignorTwoConsumersOneTopicTwoPartitions() {
-    val topic = "topic"
-    val consumer1 = "consumer1"
-    val consumer2 = "consumer2"
-    val numPartitions = 2
-    val assignor = new RangeAssignor()
-    val topicsPerConsumer = Map(consumer1 -> Set(topic), consumer2 -> Set(topic))
-    val partitionsPerTopic = Map(topic -> numPartitions)
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(
-      consumer1 -> topicAndPartitions(Map(topic -> Set(0))),
-      consumer2 -> topicAndPartitions(Map(topic -> Set(1))))
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRangeAssignorMultipleConsumersMixedTopics() {
-    val topic1 = "topic1"
-    val topic2 = "topic2"
-    val consumer1 = "consumer1"
-    val consumer2 = "consumer2"
-    val consumer3 = "consumer3"
-    val numTopic1Partitions = 3
-    val numTopic2Partitions = 2
-    val assignor = new RangeAssignor()
-    val topicsPerConsumer = Map(consumer1 -> Set(topic1), consumer2 -> Set(topic1, topic2), consumer3 -> Set(topic1))
-    val partitionsPerTopic = Map(topic1 -> numTopic1Partitions, topic2 -> numTopic2Partitions)
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(
-      consumer1 -> topicAndPartitions(Map(topic1 -> Set(0))),
-      consumer2 -> topicAndPartitions(Map(topic1 -> Set(1), topic2 -> Set(0, 1))),
-      consumer3 -> topicAndPartitions(Map(topic1 -> Set(2))))
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRangeAssignorTwoConsumersTwoTopicsSixPartitions() {
-    val topic1 = "topic1"
-    val topic2 = "topic2"
-    val consumer1 = "consumer1"
-    val consumer2 = "consumer2"
-    val numTopic1Partitions = 3
-    val numTopic2Partitions = 3
-    val assignor = new RangeAssignor()
-    val topicsPerConsumer = Map(consumer1 -> Set(topic1, topic2), consumer2 -> Set(topic1, topic2))
-    val partitionsPerTopic = Map(topic1 -> numTopic1Partitions, topic2 -> numTopic2Partitions)
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(
-      consumer1 -> topicAndPartitions(Map(topic1 -> Set(0, 1), topic2 -> Set(0, 1))),
-      consumer2 -> topicAndPartitions(Map(topic1 -> Set(2), topic2 -> Set(2))))
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRoundRobinAssignorOneConsumerNoTopic() {
-    val consumer = "consumer"
-    val assignor = new RoundRobinAssignor()
-    val topicsPerConsumer = Map(consumer -> Set.empty[String])
-    val partitionsPerTopic = Map.empty[String, Int]
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(consumer -> Set.empty[TopicAndPartition])
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRoundRobinAssignorOneConsumerNonexistentTopic() {
-    val topic = "topic"
-    val consumer = "consumer"
-    val assignor = new RoundRobinAssignor()
-    val topicsPerConsumer = Map(consumer -> Set(topic))
-    val partitionsPerTopic = Map(topic -> 0)
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(consumer -> Set.empty[TopicAndPartition])
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRoundRobinAssignorOneConsumerOneTopic() {
-    val topic = "topic"
-    val consumer = "consumer"
-    val numPartitions = 3
-    val assignor = new RoundRobinAssignor()
-    val topicsPerConsumer = Map(consumer -> Set(topic))
-    val partitionsPerTopic = Map(topic -> numPartitions)
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(consumer -> topicAndPartitions(Map(topic -> Set(0, 1, 2))))
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRoundRobinAssignorOnlyAssignsPartitionsFromSubscribedTopics() {
-    val subscribedTopic = "topic"
-    val otherTopic = "other"
-    val consumer = "consumer"
-    val subscribedTopicNumPartitions = 3
-    val otherTopicNumPartitions = 3
-    val assignor = new RoundRobinAssignor()
-    val topicsPerConsumer = Map(consumer -> Set(subscribedTopic))
-    val partitionsPerTopic = Map(subscribedTopic -> subscribedTopicNumPartitions, otherTopic -> otherTopicNumPartitions)
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(consumer -> topicAndPartitions(Map(subscribedTopic -> Set(0, 1, 2))))
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRoundRobinAssignorOneConsumerMultipleTopics() {
-    val topic1 = "topic1"
-    val topic2 = "topic2"
-    val consumer = "consumer"
-    val numTopic1Partitions = 1
-    val numTopic2Partitions = 2
-    val assignor = new RoundRobinAssignor()
-    val topicsPerConsumer = Map(consumer -> Set(topic1, topic2))
-    val partitionsPerTopic = Map(topic1 -> numTopic1Partitions, topic2 -> numTopic2Partitions)
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(consumer -> topicAndPartitions(Map(topic1 -> Set(0), topic2 -> Set(0, 1))))
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRoundRobinAssignorTwoConsumersOneTopicOnePartition() {
-    val topic = "topic"
-    val consumer1 = "consumer1"
-    val consumer2 = "consumer2"
-    val numPartitions = 1
-    val assignor = new RoundRobinAssignor()
-    val topicsPerConsumer = Map(consumer1 -> Set(topic), consumer2 -> Set(topic))
-    val partitionsPerTopic = Map(topic -> numPartitions)
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(
-      consumer1 -> topicAndPartitions(Map(topic -> Set(0))),
-      consumer2 -> Set.empty[TopicAndPartition])
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRoundRobinAssignorTwoConsumersOneTopicTwoPartitions() {
-    val topic = "topic"
-    val consumer1 = "consumer1"
-    val consumer2 = "consumer2"
-    val numPartitions = 2
-    val assignor = new RoundRobinAssignor()
-    val topicsPerConsumer = Map(consumer1 -> Set(topic), consumer2 -> Set(topic))
-    val partitionsPerTopic = Map(topic -> numPartitions)
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(
-      consumer1 -> topicAndPartitions(Map(topic -> Set(0))),
-      consumer2 -> topicAndPartitions(Map(topic -> Set(1))))
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRoundRobinAssignorMultipleConsumersMixedTopics() {
-    val topic1 = "topic1"
-    val topic2 = "topic2"
-    val consumer1 = "consumer1"
-    val consumer2 = "consumer2"
-    val consumer3 = "consumer3"
-    val numTopic1Partitions = 3
-    val numTopic2Partitions = 2
-    val assignor = new RoundRobinAssignor()
-    val topicsPerConsumer = Map(consumer1 -> Set(topic1), consumer2 -> Set(topic1, topic2), consumer3 -> Set(topic1))
-    val partitionsPerTopic = Map(topic1 -> numTopic1Partitions, topic2 -> numTopic2Partitions)
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(
-      consumer1 -> topicAndPartitions(Map(topic1 -> Set(0))),
-      consumer2 -> topicAndPartitions(Map(topic1 -> Set(1), topic2 -> Set(0, 1))),
-      consumer3 -> topicAndPartitions(Map(topic1 -> Set(2))))
-    assertEquals(expected, actual)
-  }
-
-  @Test
-  def testRoundRobinAssignorTwoConsumersTwoTopicsSixPartitions() {
-    val topic1 = "topic1"
-    val topic2 = "topic2"
-    val consumer1 = "consumer1"
-    val consumer2 = "consumer2"
-    val numTopic1Partitions = 3
-    val numTopic2Partitions = 3
-    val assignor = new RoundRobinAssignor()
-    val topicsPerConsumer = Map(consumer1 -> Set(topic1, topic2), consumer2 -> Set(topic1, topic2))
-    val partitionsPerTopic = Map(topic1 -> numTopic1Partitions, topic2 -> numTopic2Partitions)
-    val actual = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    val expected = Map(
-      consumer1 -> topicAndPartitions(Map(topic1 -> Set(0, 2), topic2 -> Set(1))),
-      consumer2 -> topicAndPartitions(Map(topic1 -> Set(1), topic2 -> Set(0, 2))))
-    assertEquals(expected, actual)
-  }
-
-  private def topicAndPartitions(topicPartitions: Map[String, Set[Int]]): Set[TopicAndPartition] = {
-    topicPartitions.flatMap { case (topic, partitions) =>
-      partitions.map(partition => TopicAndPartition(topic, partition))
-    }.toSet
-  }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala
index 85252d0..6238f6d 100644
--- a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala
+++ b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala
@@ -206,8 +206,8 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness {
     val resourceToAcls = Map[Resource, Set[Acl]](
       new Resource(Topic, Resource.WildCardResource) -> Set[Acl](new Acl(user2, Allow, WildCardHost, Read)),
       new Resource(Cluster, Resource.WildCardResource) -> Set[Acl](new Acl(user2, Allow, host1, Read)),
-      new Resource(ConsumerGroup, Resource.WildCardResource) -> acls,
-      new Resource(ConsumerGroup, "test-ConsumerGroup") -> acls
+      new Resource(Group, Resource.WildCardResource) -> acls,
+      new Resource(Group, "test-ConsumerGroup") -> acls
     )
 
     resourceToAcls foreach { case (key, value) => changeAclAndVerify(Set.empty[Acl], value, Set.empty[Acl], key) }

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
index c9f2540..1c3e55d 100755
--- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
@@ -465,8 +465,8 @@ class KafkaConfigTest {
         case KafkaConfig.ControlledShutdownMaxRetriesProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
         case KafkaConfig.ControlledShutdownRetryBackoffMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
         case KafkaConfig.ControlledShutdownEnableProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_boolean", "0")
-        case KafkaConfig.ConsumerMinSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
-        case KafkaConfig.ConsumerMaxSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
+        case KafkaConfig.GroupMinSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
+        case KafkaConfig.GroupMaxSessionTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
         case KafkaConfig.OffsetMetadataMaxSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number")
         case KafkaConfig.OffsetsLoadBufferSizeProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")
         case KafkaConfig.OffsetsTopicReplicationFactorProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "0")

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
index 7440500..4e5e776 100755
--- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
+++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala
@@ -17,7 +17,7 @@
 
 package kafka.server
 
-import kafka.api.{ConsumerMetadataRequest, OffsetCommitRequest, OffsetFetchRequest}
+import kafka.api.{GroupMetadataRequest, OffsetCommitRequest, OffsetFetchRequest}
 import kafka.consumer.SimpleConsumer
 import kafka.common.{OffsetMetadata, OffsetMetadataAndError, OffsetAndMetadata, ErrorMapping, TopicAndPartition}
 import kafka.utils._
@@ -56,7 +56,7 @@ class OffsetCommitTest extends ZooKeeperTestHarness {
     time = new MockTime()
     server = TestUtils.createServer(KafkaConfig.fromProps(config), time)
     simpleConsumer = new SimpleConsumer("localhost", server.boundPort(), 1000000, 64*1024, "test-client")
-    val consumerMetadataRequest = ConsumerMetadataRequest(group)
+    val consumerMetadataRequest = GroupMetadataRequest(group)
     Stream.continually {
       val consumerMetadataResponse = simpleConsumer.send(consumerMetadataRequest)
       consumerMetadataResponse.coordinatorOpt.isDefined

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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 46c88a3..ca17c6b 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -49,12 +49,10 @@ import kafka.utils.ZkUtils._
 
 import org.junit.Assert._
 import org.apache.kafka.clients.producer.KafkaProducer
-import org.apache.kafka.clients.consumer.{ConsumerRebalanceListener, KafkaConsumer}
-import org.apache.kafka.common.serialization.ByteArrayDeserializer
+import org.apache.kafka.clients.consumer.{RangeAssignor, KafkaConsumer}
 import org.apache.kafka.clients.CommonClientConfigs
 import org.apache.kafka.common.network.Mode
 import org.apache.kafka.common.security.ssl.SSLFactory
-import org.apache.kafka.common.config.SSLConfigs
 import org.apache.kafka.test.TestSSLUtils
 
 import scala.collection.Map
@@ -481,7 +479,7 @@ object TestUtils extends Logging {
                         groupId: String,
                         autoOffsetReset: String = "earliest",
                         partitionFetchSize: Long = 4096L,
-                        partitionAssignmentStrategy: String = "blah",
+                        partitionAssignmentStrategy: String = classOf[RangeAssignor].getName,
                         sessionTimeout: Int = 30000,
                         securityProtocol: SecurityProtocol,
                         trustStoreFile: Option[File] = None) : KafkaConsumer[Array[Byte],Array[Byte]] = {


[3/8] kafka git commit: KAFKA-2464: client-side assignment for new consumer

Posted by gu...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
new file mode 100644
index 0000000..ef94289
--- /dev/null
+++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
@@ -0,0 +1,632 @@
+/**
+ * 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.coordinator
+
+import java.util.Properties
+import java.util.concurrent.atomic.AtomicBoolean
+
+import kafka.common.{OffsetAndMetadata, OffsetMetadataAndError, TopicAndPartition}
+import kafka.log.LogConfig
+import kafka.message.UncompressedCodec
+import kafka.server._
+import kafka.utils._
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.JoinGroupRequest
+
+import scala.collection.{Map, Seq, immutable}
+
+case class GroupManagerConfig(groupMinSessionTimeoutMs: Int,
+                              groupMaxSessionTimeoutMs: Int)
+
+case class JoinGroupResult(members: Map[String, Array[Byte]],
+                           memberId: String,
+                           generationId: Int,
+                           subProtocol: String,
+                           leaderId: String,
+                           errorCode: Short)
+
+/**
+ * GroupCoordinator handles general group membership and offset management.
+ *
+ * Each Kafka server instantiates a coordinator which is responsible for a set of
+ * groups. Groups are assigned to coordinators based on their group names.
+ */
+class GroupCoordinator(val brokerId: Int,
+                       val groupConfig: GroupManagerConfig,
+                       val offsetConfig: OffsetManagerConfig,
+                       private val offsetManager: OffsetManager) extends Logging {
+  type JoinCallback = JoinGroupResult => Unit
+  type SyncCallback = (Array[Byte], Short) => Unit
+
+  this.logIdent = "[GroupCoordinator " + brokerId + "]: "
+
+  private val isActive = new AtomicBoolean(false)
+
+  private var heartbeatPurgatory: DelayedOperationPurgatory[DelayedHeartbeat] = null
+  private var joinPurgatory: DelayedOperationPurgatory[DelayedJoin] = null
+  private var coordinatorMetadata: CoordinatorMetadata = null
+
+  def this(brokerId: Int,
+           groupConfig: GroupManagerConfig,
+           offsetConfig: OffsetManagerConfig,
+           replicaManager: ReplicaManager,
+           zkUtils: ZkUtils,
+           scheduler: KafkaScheduler) = this(brokerId, groupConfig, offsetConfig,
+    new OffsetManager(offsetConfig, replicaManager, zkUtils, scheduler))
+
+  def offsetsTopicConfigs: Properties = {
+    val props = new Properties
+    props.put(LogConfig.CleanupPolicyProp, LogConfig.Compact)
+    props.put(LogConfig.SegmentBytesProp, offsetConfig.offsetsTopicSegmentBytes.toString)
+    props.put(LogConfig.CompressionTypeProp, UncompressedCodec.name)
+    props
+  }
+
+  /**
+   * NOTE: If a group lock and metadataLock are simultaneously needed,
+   * be sure to acquire the group lock before metadataLock to prevent deadlock
+   */
+
+  /**
+   * Startup logic executed at the same time when the server starts up.
+   */
+  def startup() {
+    info("Starting up.")
+    heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", brokerId)
+    joinPurgatory = new DelayedOperationPurgatory[DelayedJoin]("Rebalance", brokerId)
+    coordinatorMetadata = new CoordinatorMetadata(brokerId)
+    isActive.set(true)
+    info("Startup complete.")
+  }
+
+  /**
+   * Shutdown logic executed at the same time when server shuts down.
+   * Ordering of actions should be reversed from the startup process.
+   */
+  def shutdown() {
+    info("Shutting down.")
+    isActive.set(false)
+    offsetManager.shutdown()
+    coordinatorMetadata.shutdown()
+    heartbeatPurgatory.shutdown()
+    joinPurgatory.shutdown()
+    info("Shutdown complete.")
+  }
+
+  def handleJoinGroup(groupId: String,
+                      memberId: String,
+                      sessionTimeoutMs: Int,
+                      protocolType: String,
+                      protocols: List[(String, Array[Byte])],
+                      responseCallback: JoinCallback) {
+    if (!isActive.get) {
+      responseCallback(joinError(memberId, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code))
+    } else if (!isCoordinatorForGroup(groupId)) {
+      responseCallback(joinError(memberId,Errors.NOT_COORDINATOR_FOR_GROUP.code))
+    } else if (sessionTimeoutMs < groupConfig.groupMinSessionTimeoutMs ||
+               sessionTimeoutMs > groupConfig.groupMaxSessionTimeoutMs) {
+      responseCallback(joinError(memberId, Errors.INVALID_SESSION_TIMEOUT.code))
+    } else {
+      // only try to create the group if the group is not unknown AND
+      // the member id is UNKNOWN, if member is specified but group does not
+      // exist we should reject the request
+      var group = coordinatorMetadata.getGroup(groupId)
+      if (group == null) {
+        if (memberId != JoinGroupRequest.UNKNOWN_MEMBER_ID) {
+          responseCallback(joinError(memberId, Errors.UNKNOWN_MEMBER_ID.code))
+        } else {
+          group = coordinatorMetadata.addGroup(groupId, protocolType)
+          doJoinGroup(group, memberId, sessionTimeoutMs, protocolType, protocols, responseCallback)
+        }
+      } else {
+        doJoinGroup(group, memberId, sessionTimeoutMs, protocolType, protocols, responseCallback)
+      }
+    }
+  }
+
+  private def doJoinGroup(group: GroupMetadata,
+                          memberId: String,
+                          sessionTimeoutMs: Int,
+                          protocolType: String,
+                          protocols: List[(String, Array[Byte])],
+                          responseCallback: JoinCallback) {
+    group synchronized {
+      if (group.protocolType != protocolType || !group.supportsProtocols(protocols.map(_._1).toSet)) {
+        // if the new member does not support the group protocol, reject it
+        responseCallback(joinError(memberId, Errors.INCONSISTENT_GROUP_PROTOCOL.code))
+      } else if (memberId != JoinGroupRequest.UNKNOWN_MEMBER_ID && !group.has(memberId)) {
+        // if the member trying to register with a un-recognized id, send the response to let
+        // it reset its member id and retry
+        responseCallback(joinError(memberId, Errors.UNKNOWN_MEMBER_ID.code))
+      } else {
+        group.currentState match {
+          case Dead =>
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; this is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // joining without the specified member id,
+            responseCallback(joinError(memberId, Errors.UNKNOWN_MEMBER_ID.code))
+
+          case PreparingRebalance =>
+            if (memberId == JoinGroupRequest.UNKNOWN_MEMBER_ID) {
+              addMemberAndRebalance(sessionTimeoutMs, protocols, group, responseCallback)
+            } else {
+              val member = group.get(memberId)
+              updateMemberAndRebalance(group, member, protocols, responseCallback)
+            }
+
+          case AwaitingSync =>
+            if (memberId == JoinGroupRequest.UNKNOWN_MEMBER_ID) {
+              addMemberAndRebalance(sessionTimeoutMs, protocols, group, responseCallback)
+            } else {
+              val member = group.get(memberId)
+              if (member.matches(protocols)) {
+                // member is joining with the same metadata (which could be because it failed to
+                // receive the initial JoinGroup response), so just return current group information
+                // for the current generation.
+                responseCallback(JoinGroupResult(
+                  members = if (memberId == group.leaderId) {
+                    group.currentMemberMetadata
+                  } else {
+                    Map.empty
+                  },
+                  memberId = memberId,
+                  generationId = group.generationId,
+                  subProtocol = group.protocol,
+                  leaderId = group.leaderId,
+                  errorCode = Errors.NONE.code))
+              } else {
+                // member has changed metadata, so force a rebalance
+                updateMemberAndRebalance(group, member, protocols, responseCallback)
+              }
+            }
+
+          case Stable =>
+            if (memberId == JoinGroupRequest.UNKNOWN_MEMBER_ID) {
+              // if the member id is unknown, register the member to the group
+              addMemberAndRebalance(sessionTimeoutMs, protocols, group, responseCallback)
+            } else {
+              val member = group.get(memberId)
+              if (memberId == group.leaderId || !member.matches(protocols)) {
+                // force a rebalance if a member has changed metadata or if the leader sends JoinGroup.
+                // The latter allows the leader to trigger rebalances for changes affecting assignment
+                // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                updateMemberAndRebalance(group, member, protocols, responseCallback)
+              } else {
+                // for followers with no actual change to their metadata, just return group information
+                // for the current generation which will allow them to issue SyncGroup
+                responseCallback(JoinGroupResult(
+                  members = Map.empty,
+                  memberId = memberId,
+                  generationId = group.generationId,
+                  subProtocol = group.protocol,
+                  leaderId = group.leaderId,
+                  errorCode = Errors.NONE.code))
+              }
+            }
+        }
+
+        if (group.is(PreparingRebalance))
+          joinPurgatory.checkAndComplete(ConsumerGroupKey(group.groupId))
+      }
+    }
+  }
+
+  def handleSyncGroup(groupId: String,
+                      generation: Int,
+                      memberId: String,
+                      groupAssignment: Map[String, Array[Byte]],
+                      responseCallback: SyncCallback) {
+    if (!isActive.get) {
+      responseCallback(Array.empty, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code)
+    } else if (!isCoordinatorForGroup(groupId)) {
+      responseCallback(Array.empty, Errors.NOT_COORDINATOR_FOR_GROUP.code)
+    } else {
+      val group = coordinatorMetadata.getGroup(groupId)
+      if (group == null)
+        responseCallback(Array.empty, Errors.UNKNOWN_MEMBER_ID.code)
+      else
+        doSyncGroup(group, generation, memberId, groupAssignment, responseCallback)
+    }
+  }
+
+  private def doSyncGroup(group: GroupMetadata,
+                          generationId: Int,
+                          memberId: String,
+                          groupAssignment: Map[String, Array[Byte]],
+                          responseCallback: SyncCallback) {
+    group synchronized {
+      if (!group.has(memberId)) {
+        responseCallback(Array.empty, Errors.UNKNOWN_MEMBER_ID.code)
+      } else if (generationId != group.generationId) {
+        responseCallback(Array.empty, Errors.ILLEGAL_GENERATION.code)
+      } else {
+        group.currentState match {
+          case Dead =>
+            responseCallback(Array.empty, Errors.UNKNOWN_MEMBER_ID.code)
+
+          case PreparingRebalance =>
+            responseCallback(Array.empty, Errors.REBALANCE_IN_PROGRESS.code)
+
+          case AwaitingSync =>
+            group.get(memberId).awaitingSyncCallback = responseCallback
+            completeAndScheduleNextHeartbeatExpiration(group, group.get(memberId))
+
+            // if this is the leader, then we can transition to stable and
+            // propagate the assignment to any awaiting members
+            if (memberId == group.leaderId) {
+              group.transitionTo(Stable)
+              propagateAssignment(group, groupAssignment)
+            }
+
+          case Stable =>
+            // if the group is stable, we just return the current assignment
+            val memberMetadata = group.get(memberId)
+            responseCallback(memberMetadata.assignment, Errors.NONE.code)
+            completeAndScheduleNextHeartbeatExpiration(group, group.get(memberId))
+        }
+      }
+    }
+  }
+
+  def handleLeaveGroup(groupId: String, consumerId: String, responseCallback: Short => Unit) {
+    if (!isActive.get) {
+      responseCallback(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code)
+    } else if (!isCoordinatorForGroup(groupId)) {
+      responseCallback(Errors.NOT_COORDINATOR_FOR_GROUP.code)
+    } else {
+      val group = coordinatorMetadata.getGroup(groupId)
+      if (group == null) {
+        // if the group is marked as dead, it means some other thread has just removed the group
+        // from the coordinator metadata; this is likely that the group has migrated to some other
+        // coordinator OR the group is in a transient unstable phase. Let the consumer to retry
+        // joining without specified consumer id,
+        responseCallback(Errors.UNKNOWN_MEMBER_ID.code)
+      } else {
+        group synchronized {
+          if (group.is(Dead)) {
+            responseCallback(Errors.UNKNOWN_MEMBER_ID.code)
+          } else if (!group.has(consumerId)) {
+            responseCallback(Errors.UNKNOWN_MEMBER_ID.code)
+          } else {
+            val member = group.get(consumerId)
+            removeHeartbeatForLeavingMember(group, member)
+            onMemberFailure(group, member)
+            responseCallback(Errors.NONE.code)
+          }
+        }
+      }
+    }
+  }
+
+  def handleHeartbeat(groupId: String,
+                      memberId: String,
+                      generationId: Int,
+                      responseCallback: Short => Unit) {
+    if (!isActive.get) {
+      responseCallback(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code)
+    } else if (!isCoordinatorForGroup(groupId)) {
+      responseCallback(Errors.NOT_COORDINATOR_FOR_GROUP.code)
+    } else {
+      val group = coordinatorMetadata.getGroup(groupId)
+      if (group == null) {
+        // if the group is marked as dead, it means some other thread has just removed the group
+        // from the coordinator metadata; this is likely that the group has migrated to some other
+        // coordinator OR the group is in a transient unstable phase. Let the member retry
+        // joining without the specified member id,
+        responseCallback(Errors.UNKNOWN_MEMBER_ID.code)
+      } else {
+        group synchronized {
+          if (group.is(Dead)) {
+            responseCallback(Errors.UNKNOWN_MEMBER_ID.code)
+          } else if (!group.is(Stable)) {
+            responseCallback(Errors.REBALANCE_IN_PROGRESS.code)
+          } else if (!group.has(memberId)) {
+            responseCallback(Errors.UNKNOWN_MEMBER_ID.code)
+          } else if (generationId != group.generationId) {
+            responseCallback(Errors.ILLEGAL_GENERATION.code)
+          } else {
+            val member = group.get(memberId)
+            completeAndScheduleNextHeartbeatExpiration(group, member)
+            responseCallback(Errors.NONE.code)
+          }
+        }
+      }
+    }
+  }
+
+  def handleCommitOffsets(groupId: String,
+                          memberId: String,
+                          generationId: Int,
+                          offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata],
+                          responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) {
+    if (!isActive.get) {
+      responseCallback(offsetMetadata.mapValues(_ => Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code))
+    } else if (!isCoordinatorForGroup(groupId)) {
+      responseCallback(offsetMetadata.mapValues(_ => Errors.NOT_COORDINATOR_FOR_GROUP.code))
+    } else {
+      val group = coordinatorMetadata.getGroup(groupId)
+      if (group == null) {
+        if (generationId < 0)
+          // the group is not relying on Kafka for partition management, so allow the commit
+          offsetManager.storeOffsets(groupId, memberId, generationId, offsetMetadata, responseCallback)
+        else
+          // the group has failed over to this coordinator (which will be handled in KAFKA-2017),
+          // or this is a request coming from an older generation. either way, reject the commit
+          responseCallback(offsetMetadata.mapValues(_ => Errors.ILLEGAL_GENERATION.code))
+      } else {
+        group synchronized {
+          if (group.is(Dead)) {
+            responseCallback(offsetMetadata.mapValues(_ => Errors.UNKNOWN_MEMBER_ID.code))
+          } else if (group.is(AwaitingSync)) {
+            responseCallback(offsetMetadata.mapValues(_ => Errors.REBALANCE_IN_PROGRESS.code))
+          } else if (!group.has(memberId)) {
+            responseCallback(offsetMetadata.mapValues(_ => Errors.UNKNOWN_MEMBER_ID.code))
+          } else if (generationId != group.generationId) {
+            responseCallback(offsetMetadata.mapValues(_ => Errors.ILLEGAL_GENERATION.code))
+          } else {
+            offsetManager.storeOffsets(groupId, memberId, generationId, offsetMetadata, responseCallback)
+          }
+        }
+      }
+    }
+  }
+
+  def handleFetchOffsets(groupId: String,
+                         partitions: Seq[TopicAndPartition]): Map[TopicAndPartition, OffsetMetadataAndError] = {
+    if (!isActive.get) {
+      partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.GroupCoordinatorNotAvailable)}.toMap
+    } else if (!isCoordinatorForGroup(groupId)) {
+      partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.NotCoordinatorForGroup)}.toMap
+    } else {
+      // return offsets blindly regardless the current group state since the group may be using
+      // Kafka commit storage without automatic group management
+      offsetManager.getOffsets(groupId, partitions)
+    }
+  }
+
+  def handleGroupImmigration(offsetTopicPartitionId: Int) = {
+    // TODO we may need to add more logic in KAFKA-2017
+    offsetManager.loadOffsetsFromLog(offsetTopicPartitionId)
+  }
+
+  def handleGroupEmigration(offsetTopicPartitionId: Int) = {
+    // TODO we may need to add more logic in KAFKA-2017
+    offsetManager.removeOffsetsFromCacheForPartition(offsetTopicPartitionId)
+  }
+
+  private def joinError(memberId: String, errorCode: Short): JoinGroupResult = {
+    JoinGroupResult(
+      members=Map.empty,
+      memberId=memberId,
+      generationId=0,
+      subProtocol=GroupCoordinator.NoProtocol,
+      leaderId=GroupCoordinator.NoLeader,
+      errorCode=errorCode)
+  }
+
+  private def propagateAssignment(group: GroupMetadata,
+                                  assignment: Map[String, Array[Byte]]) {
+    for (member <- group.allMembers) {
+      member.assignment = assignment.getOrElse(member.memberId, Array.empty[Byte])
+      if (member.awaitingSyncCallback != null) {
+        member.awaitingSyncCallback(member.assignment, Errors.NONE.code)
+        member.awaitingSyncCallback = null
+      }
+    }
+  }
+
+  /**
+   * Complete existing DelayedHeartbeats for the given member and schedule the next one
+   */
+  private def completeAndScheduleNextHeartbeatExpiration(group: GroupMetadata, member: MemberMetadata) {
+    // complete current heartbeat expectation
+    member.latestHeartbeat = SystemTime.milliseconds
+    val memberKey = ConsumerKey(member.groupId, member.memberId)
+    heartbeatPurgatory.checkAndComplete(memberKey)
+
+    // reschedule the next heartbeat expiration deadline
+    val newHeartbeatDeadline = member.latestHeartbeat + member.sessionTimeoutMs
+    val delayedHeartbeat = new DelayedHeartbeat(this, group, member, newHeartbeatDeadline, member.sessionTimeoutMs)
+    heartbeatPurgatory.tryCompleteElseWatch(delayedHeartbeat, Seq(memberKey))
+  }
+
+  private def removeHeartbeatForLeavingMember(group: GroupMetadata, member: MemberMetadata) {
+    member.isLeaving = true
+    val consumerKey = ConsumerKey(member.groupId, member.memberId)
+    heartbeatPurgatory.checkAndComplete(consumerKey)
+  }
+
+  private def addMemberAndRebalance(sessionTimeoutMs: Int,
+                                    protocols: List[(String, Array[Byte])],
+                                    group: GroupMetadata,
+                                    callback: JoinCallback) = {
+    val memberId = group.generateNextMemberId
+    val member = new MemberMetadata(memberId, group.groupId, sessionTimeoutMs, protocols)
+    member.awaitingJoinCallback = callback
+    group.add(member.memberId, member)
+    maybePrepareRebalance(group)
+    member
+  }
+
+  private def updateMemberAndRebalance(group: GroupMetadata,
+                                       member: MemberMetadata,
+                                       protocols: List[(String, Array[Byte])],
+                                       callback: JoinCallback) {
+    member.supportedProtocols = protocols
+    member.awaitingJoinCallback = callback
+    maybePrepareRebalance(group)
+  }
+
+  private def maybePrepareRebalance(group: GroupMetadata) {
+    group synchronized {
+      if (group.canRebalance)
+        prepareRebalance(group)
+    }
+  }
+
+  private def prepareRebalance(group: GroupMetadata) {
+    // if any members are awaiting sync, cancel their request and have them rejoin
+    if (group.is(AwaitingSync)) {
+      for (member <- group.allMembers) {
+        if (member.awaitingSyncCallback != null) {
+          member.awaitingSyncCallback(Array.empty, Errors.REBALANCE_IN_PROGRESS.code)
+          member.awaitingSyncCallback = null
+        }
+      }
+    }
+
+    group.allMembers.foreach(_.assignment = null)
+    group.transitionTo(PreparingRebalance)
+    info("Preparing to restabilize group %s with old generation %s".format(group.groupId, group.generationId))
+
+    val rebalanceTimeout = group.rebalanceTimeout
+    val delayedRebalance = new DelayedJoin(this, group, rebalanceTimeout)
+    val consumerGroupKey = ConsumerGroupKey(group.groupId)
+    joinPurgatory.tryCompleteElseWatch(delayedRebalance, Seq(consumerGroupKey))
+  }
+
+  private def onMemberFailure(group: GroupMetadata, member: MemberMetadata) {
+    trace("Member %s in group %s has failed".format(member.memberId, group.groupId))
+    group.remove(member.memberId)
+    group.currentState match {
+      case Dead =>
+      case Stable | AwaitingSync => maybePrepareRebalance(group)
+      case PreparingRebalance => joinPurgatory.checkAndComplete(ConsumerGroupKey(group.groupId))
+    }
+  }
+
+  def tryCompleteJoin(group: GroupMetadata, forceComplete: () => Boolean) = {
+    group synchronized {
+      if (group.notYetRejoinedMembers.isEmpty)
+        forceComplete()
+      else false
+    }
+  }
+
+  def onExpireJoin() {
+    // TODO: add metrics for restabilize timeouts
+  }
+
+  def onCompleteJoin(group: GroupMetadata) {
+    group synchronized {
+      val failedMembers = group.notYetRejoinedMembers
+      if (group.isEmpty || !failedMembers.isEmpty) {
+        failedMembers.foreach { failedMember =>
+          group.remove(failedMember.memberId)
+          // TODO: cut the socket connection to the client
+        }
+
+        if (group.isEmpty) {
+          group.transitionTo(Dead)
+          info("Group %s generation %s is dead and removed".format(group.groupId, group.generationId))
+          coordinatorMetadata.removeGroup(group.groupId)
+        }
+      }
+      if (!group.is(Dead)) {
+        group.initNextGeneration
+        info("Stabilized group %s generation %s".format(group.groupId, group.generationId))
+
+        // trigger the awaiting join group response callback for all the members after rebalancing
+        for (member <- group.allMembers) {
+          assert(member.awaitingJoinCallback != null)
+          val joinResult = JoinGroupResult(
+            members=if (member.memberId == group.leaderId) { group.currentMemberMetadata } else { Map.empty },
+            memberId=member.memberId,
+            generationId=group.generationId,
+            subProtocol=group.protocol,
+            leaderId=group.leaderId,
+            errorCode=Errors.NONE.code)
+
+          member.awaitingJoinCallback(joinResult)
+          member.awaitingJoinCallback = null
+          completeAndScheduleNextHeartbeatExpiration(group, member)
+        }
+      }
+    }
+  }
+
+  def tryCompleteHeartbeat(group: GroupMetadata, member: MemberMetadata, heartbeatDeadline: Long, forceComplete: () => Boolean) = {
+    group synchronized {
+      if (shouldKeepMemberAlive(member, heartbeatDeadline) || member.isLeaving)
+        forceComplete()
+      else false
+    }
+  }
+
+  def onExpireHeartbeat(group: GroupMetadata, member: MemberMetadata, heartbeatDeadline: Long) {
+    group synchronized {
+      if (!shouldKeepMemberAlive(member, heartbeatDeadline))
+        onMemberFailure(group, member)
+    }
+  }
+
+  def onCompleteHeartbeat() {
+    // TODO: add metrics for complete heartbeats
+  }
+
+  def partitionFor(group: String): Int = offsetManager.partitionFor(group)
+
+  private def shouldKeepMemberAlive(member: MemberMetadata, heartbeatDeadline: Long) =
+    member.awaitingJoinCallback != null ||
+      member.awaitingSyncCallback != null ||
+      member.latestHeartbeat + member.sessionTimeoutMs > heartbeatDeadline
+
+  private def isCoordinatorForGroup(groupId: String) = offsetManager.leaderIsLocal(offsetManager.partitionFor(groupId))
+}
+
+object GroupCoordinator {
+
+  val NoProtocol = ""
+  val NoLeader = ""
+  val OffsetsTopicName = "__consumer_offsets"
+
+  def create(config: KafkaConfig,
+             zkUtils: ZkUtils,
+             replicaManager: ReplicaManager,
+             kafkaScheduler: KafkaScheduler): GroupCoordinator = {
+    val offsetConfig = OffsetManagerConfig(maxMetadataSize = config.offsetMetadataMaxSize,
+      loadBufferSize = config.offsetsLoadBufferSize,
+      offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L,
+      offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs,
+      offsetsTopicNumPartitions = config.offsetsTopicPartitions,
+      offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor,
+      offsetCommitTimeoutMs = config.offsetCommitTimeoutMs,
+      offsetCommitRequiredAcks = config.offsetCommitRequiredAcks)
+    val groupConfig = GroupManagerConfig(groupMinSessionTimeoutMs = config.groupMinSessionTimeoutMs,
+      groupMaxSessionTimeoutMs = config.groupMaxSessionTimeoutMs)
+
+    new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, replicaManager, zkUtils, kafkaScheduler)
+  }
+
+  def create(config: KafkaConfig,
+             zkUtils: ZkUtils,
+             offsetManager: OffsetManager): GroupCoordinator = {
+    val offsetConfig = OffsetManagerConfig(maxMetadataSize = config.offsetMetadataMaxSize,
+      loadBufferSize = config.offsetsLoadBufferSize,
+      offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L,
+      offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs,
+      offsetsTopicNumPartitions = config.offsetsTopicPartitions,
+      offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor,
+      offsetCommitTimeoutMs = config.offsetCommitTimeoutMs,
+      offsetCommitRequiredAcks = config.offsetCommitRequiredAcks)
+    val groupConfig = GroupManagerConfig(groupMinSessionTimeoutMs = config.groupMinSessionTimeoutMs,
+      groupMaxSessionTimeoutMs = config.groupMaxSessionTimeoutMs)
+
+    new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, offsetManager)
+  }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/coordinator/GroupMetadata.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadata.scala b/core/src/main/scala/kafka/coordinator/GroupMetadata.scala
new file mode 100644
index 0000000..60ee987
--- /dev/null
+++ b/core/src/main/scala/kafka/coordinator/GroupMetadata.scala
@@ -0,0 +1,209 @@
+/**
+ * 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.coordinator
+
+import kafka.utils.nonthreadsafe
+
+import java.util.UUID
+
+import org.apache.kafka.common.protocol.Errors
+
+import collection.mutable
+
+private[coordinator] sealed trait GroupState { def state: Byte }
+
+/**
+ * Group is preparing to rebalance
+ *
+ * action: respond to heartbeats with REBALANCE_IN_PROGRESS
+ *         respond to sync group with REBALANCE_IN_PROGRESS
+ *         remove member on leave group request
+ *         park join group requests from new or existing members until all expected members have joined
+ *         allow offset commits from previous generation
+ *         allow offset fetch requests
+ * transition: some members have joined by the timeout => AwaitingSync
+ *             all members have left the group => Dead
+ */
+private[coordinator] case object PreparingRebalance extends GroupState { val state: Byte = 1 }
+
+/**
+ * Group is awaiting state assignment from the leader
+ *
+ * action: respond to heartbeats with REBALANCE_IN_PROGRESS
+ *         respond to offset commits with REBALANCE_IN_PROGRESS
+ *         park sync group requests from followers until transition to Stable
+ *         allow offset fetch requests
+ * transition: sync group with state assignment received from leader => Stable
+ *             join group from new member or existing member with updated metadata => PreparingRebalance
+ *             leave group from existing member => PreparingRebalance
+ *             member failure detected => PreparingRebalance
+ */
+private[coordinator] case object AwaitingSync extends GroupState { val state: Byte = 5}
+
+/**
+ * Group is stable
+ *
+ * action: respond to member heartbeats normally
+ *         respond to sync group from any member with current assignment
+ *         respond to join group from followers with matching metadata with current group metadata
+ *         allow offset commits from member of current generation
+ *         allow offset fetch requests
+ * transition: member failure detected via heartbeat => PreparingRebalance
+ *             leave group from existing member => PreparingRebalance
+ *             leader join-group received => PreparingRebalance
+ *             follower join-group with new metadata => PreparingRebalance
+ */
+private[coordinator] case object Stable extends GroupState { val state: Byte = 3 }
+
+/**
+ * Group has no more members
+ *
+ * action: respond to join group with UNKNOWN_MEMBER_ID
+ *         respond to sync group with UNKNOWN_MEMBER_ID
+ *         respond to heartbeat with UNKNOWN_MEMBER_ID
+ *         respond to leave group with UNKNOWN_MEMBER_ID
+ *         respond to offset commit with UNKNOWN_MEMBER_ID
+ *         allow offset fetch requests
+ * transition: Dead is a final state before group metadata is cleaned up, so there are no transitions
+ */
+private[coordinator] case object Dead extends GroupState { val state: Byte = 4 }
+
+
+private object GroupMetadata {
+  private val validPreviousStates: Map[GroupState, Set[GroupState]] =
+    Map(Dead -> Set(PreparingRebalance),
+      AwaitingSync -> Set(PreparingRebalance),
+      Stable -> Set(AwaitingSync),
+      PreparingRebalance -> Set(Stable, AwaitingSync))
+}
+
+/**
+ * Group contains the following metadata:
+ *
+ *  Membership metadata:
+ *  1. Members registered in this group
+ *  2. Current protocol assigned to the group (e.g. partition assignment strategy for consumers)
+ *  3. Protocol metadata associated with group members
+ *
+ *  State metadata:
+ *  1. group state
+ *  2. generation id
+ *  3. leader id
+ */
+@nonthreadsafe
+private[coordinator] class GroupMetadata(val groupId: String, val protocolType: String) {
+
+  private val members = new mutable.HashMap[String, MemberMetadata]
+  private var state: GroupState = Stable
+  var generationId = 0
+  var leaderId: String = null
+  var protocol: String = null
+
+  def is(groupState: GroupState) = state == groupState
+  def not(groupState: GroupState) = state != groupState
+  def has(memberId: String) = members.contains(memberId)
+  def get(memberId: String) = members(memberId)
+
+  def add(memberId: String, member: MemberMetadata) {
+    assert(supportsProtocols(member.protocols))
+
+    if (leaderId == null)
+      leaderId = memberId
+    members.put(memberId, member)
+  }
+
+  def remove(memberId: String) {
+    members.remove(memberId)
+    if (memberId == leaderId) {
+      leaderId = if (members.isEmpty) {
+        null
+      } else {
+        members.keys.head
+      }
+    }
+  }
+
+  def currentState = state
+
+  def isEmpty = members.isEmpty
+
+  def notYetRejoinedMembers = members.values.filter(_.awaitingJoinCallback == null).toList
+
+  def allMembers = members.values.toList
+
+  def rebalanceTimeout = members.values.foldLeft(0) {(timeout, member) =>
+    timeout.max(member.sessionTimeoutMs)
+  }
+
+  // TODO: decide if ids should be predictable or random
+  def generateNextMemberId = UUID.randomUUID().toString
+
+  def canRebalance = state == Stable || state == AwaitingSync
+
+  def transitionTo(groupState: GroupState) {
+    assertValidTransition(groupState)
+    state = groupState
+  }
+
+  def selectProtocol: String = {
+    if (members.isEmpty)
+      throw new IllegalStateException("Cannot select protocol for empty group")
+
+    // select the protocol for this group which is supported by all members
+    val candidates = candidateProtocols
+
+    // let each member vote for one of the protocols and choose the one with the most votes
+    val votes: List[(String, Int)] = allMembers
+      .map(_.vote(candidates))
+      .groupBy(identity)
+      .mapValues(_.size)
+      .toList
+
+    votes.maxBy(_._2)._1
+  }
+
+  private def candidateProtocols = {
+    // get the set of protocols that are commonly supported by all members
+    allMembers
+      .map(_.protocols)
+      .reduceLeft((commonProtocols, protocols) => commonProtocols & protocols)
+  }
+
+  def supportsProtocols(memberProtocols: Set[String]) = {
+    isEmpty || (memberProtocols & candidateProtocols).nonEmpty
+  }
+
+  def initNextGeneration = {
+    assert(notYetRejoinedMembers == List.empty[MemberMetadata])
+    generationId += 1
+    protocol = selectProtocol
+    transitionTo(AwaitingSync)
+  }
+
+  def currentMemberMetadata: Map[String, Array[Byte]] = {
+    if (is(Dead) || is(PreparingRebalance))
+      throw new IllegalStateException("Cannot obtain member metadata for group in state %s".format(state))
+    members.map{ case (memberId, memberMetadata) => (memberId, memberMetadata.metadata(protocol))}.toMap
+  }
+
+  private def assertValidTransition(targetState: GroupState) {
+    if (!GroupMetadata.validPreviousStates(targetState).contains(state))
+      throw new IllegalStateException("Group %s should be in the %s states before moving to %s state. Instead it is in %s state"
+        .format(groupId, GroupMetadata.validPreviousStates(targetState).mkString(","), targetState, state))
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/coordinator/MemberMetadata.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/MemberMetadata.scala b/core/src/main/scala/kafka/coordinator/MemberMetadata.scala
new file mode 100644
index 0000000..7f7df9a
--- /dev/null
+++ b/core/src/main/scala/kafka/coordinator/MemberMetadata.scala
@@ -0,0 +1,99 @@
+/**
+ * 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.coordinator
+
+import java.util
+
+import kafka.utils.nonthreadsafe
+
+import scala.collection.Map
+
+/**
+ * Member metadata contains the following metadata:
+ *
+ * Heartbeat metadata:
+ * 1. negotiated heartbeat session timeout
+ * 2. timestamp of the latest heartbeat
+ *
+ * Protocol metadata:
+ * 1. the list of supported protocols (ordered by preference)
+ * 2. the metadata associated with each protocol
+ *
+ * In addition, it also contains the following state information:
+ *
+ * 1. Awaiting rebalance callback: when the group is in the prepare-rebalance state,
+ *                                 its rebalance callback will be kept in the metadata if the
+ *                                 member has sent the join group request
+ * 2. Awaiting sync callback: when the group is in the awaiting-sync state, its sync callback
+ *                            is kept in metadata until the leader provides the group assignment
+ *                            and the group transitions to stable
+ */
+@nonthreadsafe
+private[coordinator] class MemberMetadata(val memberId: String,
+                                          val groupId: String,
+                                          val sessionTimeoutMs: Int,
+                                          var supportedProtocols: List[(String, Array[Byte])]) {
+
+  var assignment: Array[Byte] = null
+  var awaitingJoinCallback: JoinGroupResult => Unit = null
+  var awaitingSyncCallback: (Array[Byte], Short) => Unit = null
+  var latestHeartbeat: Long = -1
+  var isLeaving: Boolean = false
+
+  def protocols = supportedProtocols.map(_._1).toSet
+
+  /**
+   * Get metadata corresponding to the provided protocol.
+   */
+  def metadata(protocol: String): Array[Byte] = {
+    supportedProtocols.find(_._1 == protocol) match {
+      case Some((_, metadata)) => metadata
+      case None =>
+        throw new IllegalArgumentException("Member does not support protocol")
+    }
+  }
+
+  /**
+   * Check if the provided protocol metadata matches the currently stored metadata.
+   */
+  def matches(protocols: List[(String, Array[Byte])]): Boolean = {
+    if (protocols.size != this.supportedProtocols.size)
+      return false
+
+    for (i <- 0 until protocols.size) {
+      val p1 = protocols(i)
+      val p2 = supportedProtocols(i)
+      if (p1._1 != p2._1 || !util.Arrays.equals(p1._2, p2._2))
+        return false
+    }
+    return true
+  }
+
+  /**
+   * Vote for one of the potential group protocols. This takes into account the protocol preference as
+   * indicated by the order of supported protocols and returns the first one also contained in the set
+   */
+  def vote(candidates: Set[String]): String = {
+    supportedProtocols.find({ case (protocol, _) => candidates.contains(protocol)}) match {
+      case Some((protocol, _)) => protocol
+      case None =>
+        throw new IllegalArgumentException("Member does not support any of the candidate protocols")
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/coordinator/PartitionAssignor.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/PartitionAssignor.scala b/core/src/main/scala/kafka/coordinator/PartitionAssignor.scala
deleted file mode 100644
index 8499bf8..0000000
--- a/core/src/main/scala/kafka/coordinator/PartitionAssignor.scala
+++ /dev/null
@@ -1,125 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package kafka.coordinator
-
-import kafka.common.TopicAndPartition
-import kafka.utils.CoreUtils
-
-private[coordinator] trait PartitionAssignor {
-  /**
-   * Assigns partitions to consumers in a group.
-   * @return A mapping from consumer to assigned partitions.
-   */
-  def assign(topicsPerConsumer: Map[String, Set[String]],
-             partitionsPerTopic: Map[String, Int]): Map[String, Set[TopicAndPartition]]
-
-  protected def fill[K, V](vsPerK: Map[K, Set[V]], expectedKs: Set[K]): Map[K, Set[V]] = {
-    val unfilledKs = expectedKs -- vsPerK.keySet
-    vsPerK ++ unfilledKs.map(k => (k, Set.empty[V]))
-  }
-
-  protected def aggregate[K, V](pairs: Seq[(K, V)]): Map[K, Set[V]] = {
-    pairs
-      .groupBy { case (k, v) => k }
-      .map { case (k, kvPairs) => (k, kvPairs.map(_._2).toSet) }
-  }
-
-  protected def invert[K, V](vsPerK: Map[K, Set[V]]): Map[V, Set[K]] = {
-    val vkPairs = vsPerK.toSeq.flatMap { case (k, vs) => vs.map(v => (v, k)) }
-    aggregate(vkPairs)
-  }
-}
-
-private[coordinator] object PartitionAssignor {
-  val strategies = Set("range", "roundrobin")
-
-  def createInstance(strategy: String) = strategy match {
-    case "roundrobin" => new RoundRobinAssignor()
-    case _ => new RangeAssignor()
-  }
-}
-
-/**
- * The roundrobin assignor lays out all the available partitions and all the available consumers. It
- * then proceeds to do a roundrobin assignment from partition to consumer. If the subscriptions of all consumer
- * instances are identical, then the partitions will be uniformly distributed. (i.e., the partition ownership counts
- * will be within a delta of exactly one across all consumers.)
- *
- * For example, suppose there are two consumers C0 and C1, two topics t0 and t1, and each topic has 3 partitions,
- * resulting in partitions t0p0, t0p1, t0p2, t1p0, t1p1, and t1p2.
- *
- * The assignment will be:
- * C0 -> [t0p0, t0p2, t1p1]
- * C1 -> [t0p1, t1p0, t1p2]
- */
-private[coordinator] class RoundRobinAssignor extends PartitionAssignor {
-  override def assign(topicsPerConsumer: Map[String, Set[String]],
-                      partitionsPerTopic: Map[String, Int]): Map[String, Set[TopicAndPartition]] = {
-    val consumers = topicsPerConsumer.keys.toSeq.sorted
-    val topics = topicsPerConsumer.values.flatten.toSeq.distinct.sorted
-
-    val allTopicPartitions = topics.flatMap { topic =>
-      val numPartitionsForTopic = partitionsPerTopic(topic)
-      (0 until numPartitionsForTopic).map(partition => TopicAndPartition(topic, partition))
-    }
-
-    var consumerAssignor = CoreUtils.circularIterator(consumers)
-    val consumerPartitionPairs = allTopicPartitions.map { topicAndPartition =>
-      consumerAssignor = consumerAssignor.dropWhile(consumerId => !topicsPerConsumer(consumerId).contains(topicAndPartition.topic))
-      val consumer = consumerAssignor.next()
-      (consumer, topicAndPartition)
-    }
-    fill(aggregate(consumerPartitionPairs), topicsPerConsumer.keySet)
-  }
-}
-
-/**
- * The range assignor works on a per-topic basis. For each topic, we lay out the available partitions in numeric order
- * and the consumers in lexicographic order. We then divide the number of partitions by the total number of
- * consumers to determine the number of partitions to assign to each consumer. If it does not evenly
- * divide, then the first few consumers will have one extra partition.
- *
- * For example, suppose there are two consumers C0 and C1, two topics t0 and t1, and each topic has 3 partitions,
- * resulting in partitions t0p0, t0p1, t0p2, t1p0, t1p1, and t1p2.
- *
- * The assignment will be:
- * C0 -> [t0p0, t0p1, t1p0, t1p1]
- * C1 -> [t0p2, t1p2]
- */
-private[coordinator] class RangeAssignor extends PartitionAssignor {
-  override def assign(topicsPerConsumer: Map[String, Set[String]],
-                      partitionsPerTopic: Map[String, Int]): Map[String, Set[TopicAndPartition]] = {
-    val consumersPerTopic = invert(topicsPerConsumer)
-    val consumerPartitionPairs = consumersPerTopic.toSeq.flatMap { case (topic, consumersForTopic) =>
-      val numPartitionsForTopic = partitionsPerTopic(topic)
-
-      val numPartitionsPerConsumer = numPartitionsForTopic / consumersForTopic.size
-      val consumersWithExtraPartition = numPartitionsForTopic % consumersForTopic.size
-
-      consumersForTopic.toSeq.sorted.zipWithIndex.flatMap { case (consumerForTopic, consumerIndex) =>
-        val startPartition = numPartitionsPerConsumer * consumerIndex + consumerIndex.min(consumersWithExtraPartition)
-        val numPartitions = numPartitionsPerConsumer + (if (consumerIndex + 1 > consumersWithExtraPartition) 0 else 1)
-
-        // The first few consumers pick up an extra partition, if any.
-        (startPartition until startPartition + numPartitions)
-          .map(partition => (consumerForTopic, TopicAndPartition(topic, partition)))
-      }
-    }
-    fill(aggregate(consumerPartitionPairs), topicsPerConsumer.keySet)
-  }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala b/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala
deleted file mode 100644
index 4345a8e..0000000
--- a/core/src/main/scala/kafka/javaapi/ConsumerMetadataResponse.scala
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package kafka.javaapi
-
-import java.nio.ByteBuffer
-import kafka.cluster.BrokerEndPoint
-
-class ConsumerMetadataResponse(private val underlying: kafka.api.ConsumerMetadataResponse) {
-
-  def errorCode = underlying.errorCode
-
-  def coordinator: BrokerEndPoint = {
-    import kafka.javaapi.Implicits._
-    underlying.coordinatorOpt
-  }
-
-  override def equals(other: Any) = canEqual(other) && {
-    val otherConsumerMetadataResponse = other.asInstanceOf[kafka.javaapi.ConsumerMetadataResponse]
-    this.underlying.equals(otherConsumerMetadataResponse.underlying)
-  }
-
-  def canEqual(other: Any) = other.isInstanceOf[kafka.javaapi.ConsumerMetadataResponse]
-
-  override def hashCode = underlying.hashCode
-
-  override def toString = underlying.toString
-
-}
-
-object ConsumerMetadataResponse {
-  def readFrom(buffer: ByteBuffer) = new ConsumerMetadataResponse(kafka.api.ConsumerMetadataResponse.readFrom(buffer))
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/javaapi/GroupMetadataResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/javaapi/GroupMetadataResponse.scala b/core/src/main/scala/kafka/javaapi/GroupMetadataResponse.scala
new file mode 100644
index 0000000..b94aa01
--- /dev/null
+++ b/core/src/main/scala/kafka/javaapi/GroupMetadataResponse.scala
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.javaapi
+
+import java.nio.ByteBuffer
+import kafka.cluster.BrokerEndPoint
+
+class GroupMetadataResponse(private val underlying: kafka.api.GroupMetadataResponse) {
+
+  def errorCode = underlying.errorCode
+
+  def coordinator: BrokerEndPoint = {
+    import kafka.javaapi.Implicits._
+    underlying.coordinatorOpt
+  }
+
+  override def equals(other: Any) = canEqual(other) && {
+    val otherConsumerMetadataResponse = other.asInstanceOf[kafka.javaapi.GroupMetadataResponse]
+    this.underlying.equals(otherConsumerMetadataResponse.underlying)
+  }
+
+  def canEqual(other: Any) = other.isInstanceOf[kafka.javaapi.GroupMetadataResponse]
+
+  override def hashCode = underlying.hashCode
+
+  override def toString = underlying.toString
+
+}
+
+object GroupMetadataResponse {
+  def readFrom(buffer: ByteBuffer) = new GroupMetadataResponse(kafka.api.GroupMetadataResponse.readFrom(buffer))
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/security/auth/ResourceType.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/security/auth/ResourceType.scala b/core/src/main/scala/kafka/security/auth/ResourceType.scala
index 3b8312d..ceb6348 100644
--- a/core/src/main/scala/kafka/security/auth/ResourceType.scala
+++ b/core/src/main/scala/kafka/security/auth/ResourceType.scala
@@ -33,8 +33,8 @@ case object Topic extends ResourceType {
   val name = "Topic"
 }
 
-case object ConsumerGroup extends ResourceType {
-  val name = "ConsumerGroup"
+case object Group extends ResourceType {
+  val name = "Group"
 }
 
 
@@ -45,5 +45,5 @@ object ResourceType {
     rType.getOrElse(throw new KafkaException(resourceType + " not a valid resourceType name. The valid names are " + values.mkString(",")))
   }
 
-  def values: Seq[ResourceType] = List(Cluster, Topic, ConsumerGroup)
+  def values: Seq[ResourceType] = List(Cluster, Topic, Group)
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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 6acab8d..c80bd46 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -17,24 +17,26 @@
 
 package kafka.server
 
-import kafka.message.MessageSet
-import kafka.security.auth.Topic
-import org.apache.kafka.common.metrics.Metrics
-import org.apache.kafka.common.protocol.SecurityProtocol
-import org.apache.kafka.common.TopicPartition
-import kafka.api._
+import java.nio.ByteBuffer
+
 import kafka.admin.AdminUtils
+import kafka.api._
 import kafka.common._
 import kafka.controller.KafkaController
-import kafka.coordinator.ConsumerCoordinator
+import kafka.coordinator.{GroupCoordinator, JoinGroupResult}
 import kafka.log._
+import kafka.message.MessageSet
 import kafka.network._
 import kafka.network.RequestChannel.{Session, Response}
-import org.apache.kafka.common.requests.{JoinGroupRequest, JoinGroupResponse, HeartbeatRequest, HeartbeatResponse, LeaveGroupRequest, LeaveGroupResponse, ResponseHeader, ResponseSend}
-import kafka.utils.{ZkUtils, ZKGroupTopicDirs, SystemTime, Logging}
-import scala.collection._
+import kafka.security.auth.{Authorizer, ClusterAction, Group, Create, Describe, Operation, Read, Resource, Topic, Write}
+import kafka.utils.{Logging, SystemTime, ZKGroupTopicDirs, ZkUtils}
 import org.I0Itec.zkclient.ZkClient
-import kafka.security.auth.{Authorizer, Read, Write, Create, ClusterAction, Describe, Resource, Topic, Operation, ConsumerGroup}
+import org.apache.kafka.common.metrics.Metrics
+import org.apache.kafka.common.protocol.SecurityProtocol
+import org.apache.kafka.common.requests.{HeartbeatRequest, HeartbeatResponse, JoinGroupRequest, JoinGroupResponse, LeaveGroupRequest, LeaveGroupResponse, ResponseHeader, ResponseSend, SyncGroupRequest, SyncGroupResponse}
+import org.apache.kafka.common.utils.Utils
+
+import scala.collection._
 
 
 /**
@@ -42,7 +44,7 @@ import kafka.security.auth.{Authorizer, Read, Write, Create, ClusterAction, Desc
  */
 class KafkaApis(val requestChannel: RequestChannel,
                 val replicaManager: ReplicaManager,
-                val coordinator: ConsumerCoordinator,
+                val coordinator: GroupCoordinator,
                 val controller: KafkaController,
                 val zkUtils: ZkUtils,
                 val brokerId: Int,
@@ -73,10 +75,11 @@ class KafkaApis(val requestChannel: RequestChannel,
         case RequestKeys.ControlledShutdownKey => handleControlledShutdownRequest(request)
         case RequestKeys.OffsetCommitKey => handleOffsetCommitRequest(request)
         case RequestKeys.OffsetFetchKey => handleOffsetFetchRequest(request)
-        case RequestKeys.ConsumerMetadataKey => handleConsumerMetadataRequest(request)
+        case RequestKeys.GroupMetadataKey => handleGroupMetadataRequest(request)
         case RequestKeys.JoinGroupKey => handleJoinGroupRequest(request)
         case RequestKeys.HeartbeatKey => handleHeartbeatRequest(request)
         case RequestKeys.LeaveGroupKey => handleLeaveGroupRequest(request)
+        case RequestKeys.SyncGroupKey => handleSyncGroupRequest(request)
         case requestId => throw new KafkaException("Unknown api code " + requestId)
       }
     } catch {
@@ -114,12 +117,12 @@ class KafkaApis(val requestChannel: RequestChannel,
       // for each new leader or follower, call coordinator to handle
       // consumer group migration
       result.updatedLeaders.foreach { case partition =>
-        if (partition.topic == ConsumerCoordinator.OffsetsTopicName)
+        if (partition.topic == GroupCoordinator.OffsetsTopicName)
           coordinator.handleGroupImmigration(partition.partitionId)
       }
       result.updatedFollowers.foreach { case partition =>
         partition.leaderReplicaIdOpt.foreach { leaderReplica =>
-          if (partition.topic == ConsumerCoordinator.OffsetsTopicName &&
+          if (partition.topic == GroupCoordinator.OffsetsTopicName &&
               leaderReplica == brokerId)
             coordinator.handleGroupEmigration(partition.partitionId)
         }
@@ -188,7 +191,7 @@ class KafkaApis(val requestChannel: RequestChannel,
     val (authorizedRequestInfo, unauthorizedRequestInfo) =  filteredRequestInfo.partition {
       case (topicAndPartition, offsetMetadata) =>
         authorize(request.session, Read, new Resource(Topic, topicAndPartition.topic)) &&
-          authorize(request.session, Read, new Resource(ConsumerGroup, offsetCommitRequest.groupId))
+          authorize(request.session, Read, new Resource(Group, offsetCommitRequest.groupId))
     }
 
     // the callback for sending an offset commit response
@@ -268,7 +271,7 @@ class KafkaApis(val requestChannel: RequestChannel,
       // call coordinator to handle commit offset
       coordinator.handleCommitOffsets(
         offsetCommitRequest.groupId,
-        offsetCommitRequest.consumerId,
+        offsetCommitRequest.memberId,
         offsetCommitRequest.groupGenerationId,
         offsetData,
         sendResponseCallback)
@@ -526,9 +529,9 @@ class KafkaApis(val requestChannel: RequestChannel,
     if (topics.size > 0 && topicResponses.size != topics.size) {
       val nonExistentTopics = topics -- topicResponses.map(_.topic).toSet
       val responsesForNonExistentTopics = nonExistentTopics.map { topic =>
-        if (topic == ConsumerCoordinator.OffsetsTopicName || config.autoCreateTopicsEnable) {
+        if (topic == GroupCoordinator.OffsetsTopicName || config.autoCreateTopicsEnable) {
           try {
-            if (topic == ConsumerCoordinator.OffsetsTopicName) {
+            if (topic == GroupCoordinator.OffsetsTopicName) {
               val aliveBrokers = metadataCache.getAliveBrokers
               val offsetsTopicReplicationFactor =
                 if (aliveBrokers.length > 0)
@@ -610,7 +613,7 @@ class KafkaApis(val requestChannel: RequestChannel,
 
     val (authorizedTopicPartitions, unauthorizedTopicPartitions) = offsetFetchRequest.requestInfo.partition { topicAndPartition =>
       authorize(request.session, Describe, new Resource(Topic, topicAndPartition.topic)) &&
-        authorize(request.session, Read, new Resource(ConsumerGroup, offsetFetchRequest.groupId))
+        authorize(request.session, Read, new Resource(Group, offsetFetchRequest.groupId))
     }
 
     val authorizationError = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, ErrorMapping.AuthorizationCode)
@@ -659,29 +662,29 @@ class KafkaApis(val requestChannel: RequestChannel,
   /*
    * Handle a consumer metadata request
    */
-  def handleConsumerMetadataRequest(request: RequestChannel.Request) {
-    val consumerMetadataRequest = request.requestObj.asInstanceOf[ConsumerMetadataRequest]
+  def handleGroupMetadataRequest(request: RequestChannel.Request) {
+    val groupMetadataRequest = request.requestObj.asInstanceOf[GroupMetadataRequest]
 
-    if (!authorize(request.session, Read, new Resource(ConsumerGroup, consumerMetadataRequest.group))) {
-      val response = ConsumerMetadataResponse(None, ErrorMapping.AuthorizationCode, consumerMetadataRequest.correlationId)
+    if (!authorize(request.session, Read, new Resource(Group, groupMetadataRequest.group))) {
+      val response = GroupMetadataResponse(None, ErrorMapping.AuthorizationCode, groupMetadataRequest.correlationId)
       requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, response)))
     } else {
-      val partition = coordinator.partitionFor(consumerMetadataRequest.group)
+      val partition = coordinator.partitionFor(groupMetadataRequest.group)
 
-      //get metadata (and create the topic if necessary)
-      val offsetsTopicMetadata = getTopicMetadata(Set(ConsumerCoordinator.OffsetsTopicName), request.securityProtocol).head
+      // get metadata (and create the topic if necessary)
+      val offsetsTopicMetadata = getTopicMetadata(Set(GroupCoordinator.OffsetsTopicName), request.securityProtocol).head
 
-      val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, consumerMetadataRequest.correlationId)
+      val errorResponse = GroupMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, groupMetadataRequest.correlationId)
 
       val response =
         offsetsTopicMetadata.partitionsMetadata.find(_.partitionId == partition).map { partitionMetadata =>
           partitionMetadata.leader.map { leader =>
-            ConsumerMetadataResponse(Some(leader), ErrorMapping.NoError, consumerMetadataRequest.correlationId)
+            GroupMetadataResponse(Some(leader), ErrorMapping.NoError, groupMetadataRequest.correlationId)
           }.getOrElse(errorResponse)
         }.getOrElse(errorResponse)
 
       trace("Sending consumer metadata %s for correlation id %d to client %s."
-        .format(response, consumerMetadataRequest.correlationId, consumerMetadataRequest.clientId))
+        .format(response, groupMetadataRequest.correlationId, groupMetadataRequest.clientId))
       requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, response)))
     }
   }
@@ -690,39 +693,65 @@ class KafkaApis(val requestChannel: RequestChannel,
     import JavaConversions._
 
     val joinGroupRequest = request.body.asInstanceOf[JoinGroupRequest]
-    val respHeader = new ResponseHeader(request.header.correlationId)
+    val responseHeader = new ResponseHeader(request.header.correlationId)
 
     // the callback for sending a join-group response
-    def sendResponseCallback(partitions: Set[TopicAndPartition], consumerId: String, generationId: Int, errorCode: Short) {
-      val partitionList = if (errorCode == ErrorMapping.NoError)
-        partitions.map(tp => new TopicPartition(tp.topic, tp.partition)).toBuffer
-      else
-        List.empty.toBuffer
-
-      val responseBody = new JoinGroupResponse(errorCode, generationId, consumerId, partitionList)
-
+    def sendResponseCallback(joinResult: JoinGroupResult) {
+      val members = joinResult.members map { case (memberId, metadataArray) => (memberId, ByteBuffer.wrap(metadataArray)) }
+      val responseBody = new JoinGroupResponse(joinResult.errorCode, joinResult.generationId, joinResult.subProtocol,
+        joinResult.memberId, joinResult.leaderId, members)
       trace("Sending join group response %s for correlation id %d to client %s."
         .format(responseBody, request.header.correlationId, request.header.clientId))
-      requestChannel.sendResponse(new Response(request, new ResponseSend(request.connectionId, respHeader, responseBody)))
+      requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, responseHeader, responseBody)))
     }
 
-    // ensure that the client is authorized to join the group and read from all subscribed topics
-    if (!authorize(request.session, Read, new Resource(ConsumerGroup, joinGroupRequest.groupId())) ||
-        joinGroupRequest.topics().exists(topic => !authorize(request.session, Read, new Resource(Topic, topic)))) {
-      val responseBody = new JoinGroupResponse(ErrorMapping.AuthorizationCode, 0, joinGroupRequest.consumerId(), List.empty[TopicPartition])
-      requestChannel.sendResponse(new Response(request, new ResponseSend(request.connectionId, respHeader, responseBody)))
+    if (!authorize(request.session, Read, new Resource(Group, joinGroupRequest.groupId()))) {
+      val responseBody = new JoinGroupResponse(
+        ErrorMapping.AuthorizationCode,
+        JoinGroupResponse.UNKNOWN_GENERATION_ID,
+        JoinGroupResponse.UNKNOWN_PROTOCOL,
+        JoinGroupResponse.UNKNOWN_MEMBER_ID, // memberId
+        JoinGroupResponse.UNKNOWN_MEMBER_ID, // leaderId
+        Map.empty[String, ByteBuffer])
+      requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, responseHeader, responseBody)))
     } else {
       // let the coordinator to handle join-group
+      val protocols = joinGroupRequest.groupProtocols().map(protocol =>
+        (protocol.name, Utils.toArray(protocol.metadata))).toList
       coordinator.handleJoinGroup(
         joinGroupRequest.groupId(),
-        joinGroupRequest.consumerId(),
-        joinGroupRequest.topics().toSet,
+        joinGroupRequest.memberId(),
         joinGroupRequest.sessionTimeout(),
-        joinGroupRequest.strategy(),
+        joinGroupRequest.protocolType(),
+        protocols,
         sendResponseCallback)
     }
   }
 
+  def handleSyncGroupRequest(request: RequestChannel.Request) {
+    import JavaConversions._
+
+    val syncGroupRequest = request.body.asInstanceOf[SyncGroupRequest]
+
+    def sendResponseCallback(memberState: Array[Byte], errorCode: Short) {
+      val responseBody = new SyncGroupResponse(errorCode, ByteBuffer.wrap(memberState))
+      val responseHeader = new ResponseHeader(request.header.correlationId)
+      requestChannel.sendResponse(new Response(request, new ResponseSend(request.connectionId, responseHeader, responseBody)))
+    }
+
+    if (!authorize(request.session, Read, new Resource(Group, syncGroupRequest.groupId()))) {
+      sendResponseCallback(Array[Byte](), ErrorMapping.AuthorizationCode)
+    } else {
+      coordinator.handleSyncGroup(
+        syncGroupRequest.groupId(),
+        syncGroupRequest.generationId(),
+        syncGroupRequest.memberId(),
+        syncGroupRequest.groupAssignment().mapValues(Utils.toArray(_)),
+        sendResponseCallback
+      )
+    }
+  }
+
   def handleHeartbeatRequest(request: RequestChannel.Request) {
     val heartbeatRequest = request.body.asInstanceOf[HeartbeatRequest]
     val respHeader = new ResponseHeader(request.header.correlationId)
@@ -735,7 +764,7 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, respHeader, response)))
     }
 
-    if (!authorize(request.session, Read, new Resource(ConsumerGroup, heartbeatRequest.groupId))) {
+    if (!authorize(request.session, Read, new Resource(Group, heartbeatRequest.groupId))) {
       val heartbeatResponse = new HeartbeatResponse(ErrorMapping.AuthorizationCode)
       requestChannel.sendResponse(new Response(request, new ResponseSend(request.connectionId, respHeader, heartbeatResponse)))
     }
@@ -743,7 +772,7 @@ class KafkaApis(val requestChannel: RequestChannel,
       // let the coordinator to handle heartbeat
       coordinator.handleHeartbeat(
         heartbeatRequest.groupId(),
-        heartbeatRequest.consumerId(),
+        heartbeatRequest.memberId(),
         heartbeatRequest.groupGenerationId(),
         sendResponseCallback)
     }
@@ -788,11 +817,16 @@ class KafkaApis(val requestChannel: RequestChannel,
       requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, respHeader, response)))
     }
 
-    // let the coordinator to handle leave-group
-    coordinator.handleLeaveGroup(
-      leaveGroupRequest.groupId(),
-      leaveGroupRequest.consumerId(),
-      sendResponseCallback)
+    if (!authorize(request.session, Read, new Resource(Group, leaveGroupRequest.groupId))) {
+      val leaveGroupResponse = new LeaveGroupResponse(ErrorMapping.AuthorizationCode)
+      requestChannel.sendResponse(new Response(request, new ResponseSend(request.connectionId, respHeader, leaveGroupResponse)))
+    } else {
+      // let the coordinator to handle leave-group
+      coordinator.handleLeaveGroup(
+        leaveGroupRequest.groupId(),
+        leaveGroupRequest.consumerId(),
+        sendResponseCallback)
+    }
   }
 
   def close() {

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/server/KafkaConfig.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index 194ee9c..b054f48 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -277,9 +277,9 @@ object KafkaConfig {
   val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries"
   val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms"
   val ControlledShutdownEnableProp = "controlled.shutdown.enable"
-  /** ********* Consumer coordinator configuration ***********/
-  val ConsumerMinSessionTimeoutMsProp = "consumer.min.session.timeout.ms"
-  val ConsumerMaxSessionTimeoutMsProp = "consumer.max.session.timeout.ms"
+  /** ********* Group coordinator configuration ***********/
+  val GroupMinSessionTimeoutMsProp = "group.min.session.timeout.ms"
+  val GroupMaxSessionTimeoutMsProp = "group.max.session.timeout.ms"
   /** ********* Offset management configuration ***********/
   val OffsetMetadataMaxSizeProp = "offset.metadata.max.bytes"
   val OffsetsLoadBufferSizeProp = "offsets.load.buffer.size"
@@ -619,8 +619,8 @@ object KafkaConfig {
       .define(ControlledShutdownEnableProp, BOOLEAN, Defaults.ControlledShutdownEnable, MEDIUM, ControlledShutdownEnableDoc)
 
       /** ********* Consumer coordinator configuration ***********/
-      .define(ConsumerMinSessionTimeoutMsProp, INT, Defaults.ConsumerMinSessionTimeoutMs, MEDIUM, ConsumerMinSessionTimeoutMsDoc)
-      .define(ConsumerMaxSessionTimeoutMsProp, INT, Defaults.ConsumerMaxSessionTimeoutMs, MEDIUM, ConsumerMaxSessionTimeoutMsDoc)
+      .define(GroupMinSessionTimeoutMsProp, INT, Defaults.ConsumerMinSessionTimeoutMs, MEDIUM, ConsumerMinSessionTimeoutMsDoc)
+      .define(GroupMaxSessionTimeoutMsProp, INT, Defaults.ConsumerMaxSessionTimeoutMs, MEDIUM, ConsumerMaxSessionTimeoutMsDoc)
 
       /** ********* Offset management configuration ***********/
       .define(OffsetMetadataMaxSizeProp, INT, Defaults.OffsetMetadataMaxSize, HIGH, OffsetMetadataMaxSizeDoc)
@@ -799,9 +799,9 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka
   val controlledShutdownRetryBackoffMs = getLong(KafkaConfig.ControlledShutdownRetryBackoffMsProp)
   val controlledShutdownEnable = getBoolean(KafkaConfig.ControlledShutdownEnableProp)
 
-  /** ********* Consumer coordinator configuration ***********/
-  val consumerMinSessionTimeoutMs = getInt(KafkaConfig.ConsumerMinSessionTimeoutMsProp)
-  val consumerMaxSessionTimeoutMs = getInt(KafkaConfig.ConsumerMaxSessionTimeoutMsProp)
+  /** ********* Group coordinator configuration ***********/
+  val groupMinSessionTimeoutMs = getInt(KafkaConfig.GroupMinSessionTimeoutMsProp)
+  val groupMaxSessionTimeoutMs = getInt(KafkaConfig.GroupMaxSessionTimeoutMsProp)
 
   /** ********* Offset management configuration ***********/
   val offsetMetadataMaxSize = getInt(KafkaConfig.OffsetMetadataMaxSizeProp)

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/server/KafkaServer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
index beea83a..84d48cb 100755
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -50,7 +50,7 @@ import kafka.common.{ErrorMapping, InconsistentBrokerIdException, GenerateBroker
 import kafka.network.{BlockingChannel, SocketServer}
 import kafka.metrics.KafkaMetricsGroup
 import com.yammer.metrics.core.Gauge
-import kafka.coordinator.{ConsumerCoordinator}
+import kafka.coordinator.{GroupManagerConfig, GroupCoordinator}
 
 object KafkaServer {
   // Copy the subset of properties that are relevant to Logs
@@ -119,7 +119,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
   var dynamicConfigHandlers: Map[String, ConfigHandler] = null
   var dynamicConfigManager: DynamicConfigManager = null
 
-  var consumerCoordinator: ConsumerCoordinator = null
+  var consumerCoordinator: GroupCoordinator = null
 
   var kafkaController: KafkaController = null
 
@@ -187,7 +187,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
         kafkaController.startup()
 
         /* start kafka coordinator */
-        consumerCoordinator = ConsumerCoordinator.create(config, zkUtils, replicaManager, kafkaScheduler)
+        consumerCoordinator = GroupCoordinator.create(config, zkUtils, replicaManager, kafkaScheduler)
         consumerCoordinator.startup()
 
         /* Get the authorizer and initialize it if one is specified.*/

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/server/OffsetManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/OffsetManager.scala b/core/src/main/scala/kafka/server/OffsetManager.scala
index bdc3bb6..967dc6f 100755
--- a/core/src/main/scala/kafka/server/OffsetManager.scala
+++ b/core/src/main/scala/kafka/server/OffsetManager.scala
@@ -32,7 +32,7 @@ import kafka.metrics.KafkaMetricsGroup
 import kafka.common.TopicAndPartition
 import kafka.tools.MessageFormatter
 import kafka.api.ProducerResponseStatus
-import kafka.coordinator.ConsumerCoordinator
+import kafka.coordinator.GroupCoordinator
 
 import scala.Some
 import scala.collection._
@@ -144,9 +144,9 @@ class OffsetManager(val config: OffsetManagerConfig,
       // Append the tombstone messages to the offset partitions. It is okay if the replicas don't receive these (say,
       // if we crash or leaders move) since the new leaders will get rid of expired offsets during their own purge cycles.
       tombstonesForPartition.flatMap { case (offsetsPartition, tombstones) =>
-        val partitionOpt = replicaManager.getPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition)
+        val partitionOpt = replicaManager.getPartition(GroupCoordinator.OffsetsTopicName, offsetsPartition)
         partitionOpt.map { partition =>
-          val appendPartition = TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition)
+          val appendPartition = TopicAndPartition(GroupCoordinator.OffsetsTopicName, offsetsPartition)
           val messages = tombstones.map(_._2).toSeq
 
           trace("Marked %d offsets in %s for deletion.".format(messages.size, appendPartition))
@@ -225,7 +225,7 @@ class OffsetManager(val config: OffsetManagerConfig,
       )
     }.toSeq
 
-    val offsetTopicPartition = TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, partitionFor(groupId))
+    val offsetTopicPartition = TopicAndPartition(GroupCoordinator.OffsetsTopicName, partitionFor(groupId))
 
     val offsetsAndMetadataMessageSet = Map(offsetTopicPartition ->
       new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*))
@@ -336,7 +336,7 @@ class OffsetManager(val config: OffsetManagerConfig,
    */
   def loadOffsetsFromLog(offsetsPartition: Int) {
 
-    val topicPartition = TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition)
+    val topicPartition = TopicAndPartition(GroupCoordinator.OffsetsTopicName, offsetsPartition)
 
     loadingPartitions synchronized {
       if (loadingPartitions.contains(offsetsPartition)) {
@@ -408,7 +408,7 @@ class OffsetManager(val config: OffsetManagerConfig,
   }
 
   private def getHighWatermark(partitionId: Int): Long = {
-    val partitionOpt = replicaManager.getPartition(ConsumerCoordinator.OffsetsTopicName, partitionId)
+    val partitionOpt = replicaManager.getPartition(GroupCoordinator.OffsetsTopicName, partitionId)
 
     val hw = partitionOpt.map { partition =>
       partition.leaderReplicaIfLocal().map(_.highWatermark.messageOffset).getOrElse(-1L)
@@ -436,7 +436,7 @@ class OffsetManager(val config: OffsetManagerConfig,
     }
 
     if (numRemoved > 0) info("Removed %d cached offsets for %s on follower transition."
-                             .format(numRemoved, TopicAndPartition(ConsumerCoordinator.OffsetsTopicName, offsetsPartition)))
+                             .format(numRemoved, TopicAndPartition(GroupCoordinator.OffsetsTopicName, offsetsPartition)))
   }
 
   def shutdown() {
@@ -448,7 +448,7 @@ class OffsetManager(val config: OffsetManagerConfig,
    * If the topic does not exist, the configured partition count is returned.
    */
   private def getOffsetsTopicPartitionCount = {
-    val topic = ConsumerCoordinator.OffsetsTopicName
+    val topic = GroupCoordinator.OffsetsTopicName
     val topicData = zkUtils.getPartitionAssignmentForTopics(Seq(topic))
     if (topicData(topic).nonEmpty)
       topicData(topic).size

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
index 84bebef..f99f0d8 100644
--- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
+++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
@@ -29,7 +29,7 @@ import org.junit.{Test, Before}
 
 import scala.collection.JavaConversions._
 import scala.collection.JavaConverters._
-import kafka.coordinator.ConsumerCoordinator
+import kafka.coordinator.GroupCoordinator
 
 /**
  * Integration tests for the new consumer that cover basic usage as well as server failures
@@ -50,7 +50,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging {
   this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown
   this.serverConfig.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset
   this.serverConfig.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1")
-  this.serverConfig.setProperty(KafkaConfig.ConsumerMinSessionTimeoutMsProp, "100") // set small enough session timeout
+  this.serverConfig.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, "100") // set small enough session timeout
   this.producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "all")
   this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test")
   this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
@@ -154,7 +154,16 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging {
     val numRecords = 10000
     sendRecords(numRecords)
 
-    consumer0.subscribe(List(topic))
+    val rebalanceListener = new ConsumerRebalanceListener {
+      override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]) = {
+        // keep partitions paused in this test so that we can verify the commits based on specific seeks
+        partitions.foreach(consumer0.pause(_))
+      }
+
+      override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]) = {}
+    }
+
+    consumer0.subscribe(List(topic), rebalanceListener)
 
     val assignment = Set(tp, tp2)
     TestUtils.waitUntilTrue(() => {
@@ -166,11 +175,11 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging {
     consumer0.seek(tp2, 500)
 
     // change subscription to trigger rebalance
-    consumer0.subscribe(List(topic, topic2))
+    consumer0.subscribe(List(topic, topic2), rebalanceListener)
 
     val newAssignment = Set(tp, tp2, new TopicPartition(topic2, 0), new TopicPartition(topic2, 1))
     TestUtils.waitUntilTrue(() => {
-      consumer0.poll(50)
+      val records = consumer0.poll(50)
       consumer0.assignment() == newAssignment.asJava
     }, s"Expected partitions ${newAssignment.asJava} but actually got ${consumer0.assignment()}")
 
@@ -421,9 +430,9 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging {
       consumer0.poll(50)
     
     // get metadata for the topic
-    var parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName).asScala
+    var parts = consumer0.partitionsFor(GroupCoordinator.OffsetsTopicName).asScala
     while(parts == null)
-      parts = consumer0.partitionsFor(ConsumerCoordinator.OffsetsTopicName).asScala
+      parts = consumer0.partitionsFor(GroupCoordinator.OffsetsTopicName).asScala
     assertEquals(1, parts.size)
     assertNotNull(parts(0).leader())
     
@@ -436,6 +445,8 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging {
       consumer0.poll(50)
 
     assertEquals(2, listener.callsToAssigned)
+
+    // only expect one revocation since revoke is not invoked on initial membership
     assertEquals(2, listener.callsToRevoked)
 
     consumer0.close()

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
index db610c1..f2b0f85 100644
--- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
+++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala
@@ -19,13 +19,15 @@ import kafka.server.KafkaConfig
 import kafka.utils.{Logging, ShutdownableThread, TestUtils}
 import org.apache.kafka.clients.consumer._
 import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
-import org.apache.kafka.common.errors.{IllegalGenerationException, UnknownConsumerIdException}
+import org.apache.kafka.common.errors.{IllegalGenerationException, UnknownMemberIdException}
 import org.apache.kafka.common.TopicPartition
 import org.junit.Assert._
 import org.junit.{Test, Before}
 
 import scala.collection.JavaConversions._
 
+
+
 /**
  * Integration tests for the new consumer that cover basic usage as well as server failures
  */
@@ -43,7 +45,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging {
   this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown
   this.serverConfig.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset
   this.serverConfig.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1")
-  this.serverConfig.setProperty(KafkaConfig.ConsumerMinSessionTimeoutMsProp, "10") // set small enough session timeout
+  this.serverConfig.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, "10") // set small enough session timeout
   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)
@@ -108,7 +110,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging {
       } catch {
         // TODO: should be no need to catch these exceptions once KAFKA-2017 is
         // merged since coordinator fail-over will not cause a rebalance
-        case _: UnknownConsumerIdException | _: IllegalGenerationException =>
+        case _: UnknownMemberIdException | _: IllegalGenerationException =>
       }
     }
     scheduler.shutdown()
@@ -176,4 +178,6 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging {
     }
     futures.map(_.get)
   }
+
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
index 2ec59fb..5741ce2 100644
--- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
+++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
@@ -18,16 +18,16 @@
 package kafka.api
 
 import org.apache.kafka.clients.producer.ProducerConfig
-import org.apache.kafka.clients.consumer.ConsumerConfig
+import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer}
 import kafka.utils.TestUtils
 import java.util.Properties
-import org.apache.kafka.clients.consumer.KafkaConsumer
 import org.apache.kafka.clients.producer.KafkaProducer
 import kafka.server.KafkaConfig
 import kafka.integration.KafkaServerTestHarness
+
 import org.junit.{After, Before}
 import scala.collection.mutable.Buffer
-import kafka.coordinator.ConsumerCoordinator
+import kafka.coordinator.GroupCoordinator
 
 /**
  * A helper class for writing integration tests that involve producers, consumers, and servers
@@ -60,14 +60,14 @@ trait IntegrationTestHarness extends KafkaServerTestHarness {
     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])
-    consumerConfig.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range")
     for(i <- 0 until producerCount)
       producers += new KafkaProducer(producerConfig)
-    for(i <- 0 until consumerCount)
+    for(i <- 0 until consumerCount) {
       consumers += new KafkaConsumer(consumerConfig)
+    }
 
     // create the consumer offset topic
-    TestUtils.createTopic(zkUtils, ConsumerCoordinator.OffsetsTopicName,
+    TestUtils.createTopic(zkUtils, GroupCoordinator.OffsetsTopicName,
       serverConfig.getProperty(KafkaConfig.OffsetsTopicPartitionsProp).toInt,
       serverConfig.getProperty(KafkaConfig.OffsetsTopicReplicationFactorProp).toInt,
       servers,

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/integration/kafka/api/QuotasTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala
index bdf7e49..735a3b2 100644
--- a/core/src/test/scala/integration/kafka/api/QuotasTest.scala
+++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala
@@ -101,7 +101,6 @@ class QuotasTest extends KafkaServerTestHarness {
                       classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer])
     consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG,
                       classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer])
-    consumerProps.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range")
 
     consumerProps.put(ConsumerConfig.CLIENT_ID_CONFIG, consumerId1)
     consumers += new KafkaConsumer(consumerProps)

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/integration/kafka/api/SslConsumerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/SslConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SslConsumerTest.scala
deleted file mode 100644
index 1d13d88..0000000
--- a/core/src/test/scala/integration/kafka/api/SslConsumerTest.scala
+++ /dev/null
@@ -1,22 +0,0 @@
-/**
-  * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
-  * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
-  * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
-  * License. You may obtain a copy of the License at
-  *
-  * http://www.apache.org/licenses/LICENSE-2.0
-  *
-  * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
-  * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
-  * specific language governing permissions and limitations under the License.
-  */
-package kafka.api
-
-import java.io.File
-
-import org.apache.kafka.common.protocol.SecurityProtocol
-
-class SslConsumerTest extends BaseConsumerTest {
-  override protected def securityProtocol = SecurityProtocol.SSL
-  override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks"))
-}


[6/8] kafka git commit: KAFKA-2464: client-side assignment for new consumer

Posted by gu...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
index 9f8e981..36094b0 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
@@ -20,7 +20,12 @@ import org.apache.kafka.common.protocol.types.ArrayOf;
 import org.apache.kafka.common.protocol.types.Field;
 import org.apache.kafka.common.protocol.types.Schema;
 
-import static org.apache.kafka.common.protocol.types.Type.*;
+import static org.apache.kafka.common.protocol.types.Type.BYTES;
+import static org.apache.kafka.common.protocol.types.Type.INT16;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+import static org.apache.kafka.common.protocol.types.Type.INT8;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
 
 public class Protocol {
 
@@ -180,31 +185,31 @@ public class Protocol {
 
     public static final Schema OFFSET_COMMIT_REQUEST_V0 = new Schema(new Field("group_id",
                                                                                STRING,
-                                                                               "The consumer group id."),
+                                                                               "The group id."),
                                                                      new Field("topics",
                                                                                new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V0),
                                                                                "Topics to commit offsets."));
 
     public static final Schema OFFSET_COMMIT_REQUEST_V1 = new Schema(new Field("group_id",
                                                                                STRING,
-                                                                               "The consumer group id."),
+                                                                               "The group id."),
                                                                      new Field("group_generation_id",
                                                                                INT32,
-                                                                               "The generation of the consumer group."),
-                                                                     new Field("consumer_id",
+                                                                               "The generation of the group."),
+                                                                     new Field("member_id",
                                                                                STRING,
-                                                                               "The consumer id assigned by the group coordinator."),
+                                                                               "The member id assigned by the group coordinator."),
                                                                      new Field("topics",
                                                                                new ArrayOf(OFFSET_COMMIT_REQUEST_TOPIC_V1),
                                                                                "Topics to commit offsets."));
 
     public static final Schema OFFSET_COMMIT_REQUEST_V2 = new Schema(new Field("group_id",
                                                                                STRING,
-                                                                               "The consumer group id."),
+                                                                               "The group id."),
                                                                      new Field("group_generation_id",
                                                                                INT32,
                                                                                "The generation of the consumer group."),
-                                                                     new Field("consumer_id",
+                                                                     new Field("member_id",
                                                                                STRING,
                                                                                "The consumer id assigned by the group coordinator."),
                                                                      new Field("retention_time",
@@ -384,17 +389,17 @@ public class Protocol {
     public static final Schema[] FETCH_RESPONSE = new Schema[] {FETCH_RESPONSE_V0, FETCH_RESPONSE_V1};
 
     /* Consumer metadata api */
-    public static final Schema CONSUMER_METADATA_REQUEST_V0 = new Schema(new Field("group_id",
-                                                                                   STRING,
-                                                                                   "The consumer group id."));
+    public static final Schema GROUP_METADATA_REQUEST_V0 = new Schema(new Field("group_id",
+                                                                                STRING,
+                                                                                "The unique group id."));
 
-    public static final Schema CONSUMER_METADATA_RESPONSE_V0 = new Schema(new Field("error_code", INT16),
-                                                                          new Field("coordinator",
-                                                                                    BROKER,
-                                                                                    "Host and port information for the coordinator for a consumer group."));
+    public static final Schema GROUP_METADATA_RESPONSE_V0 = new Schema(new Field("error_code", INT16),
+                                                                       new Field("coordinator",
+                                                                                 BROKER,
+                                                                                 "Host and port information for the coordinator for a consumer group."));
 
-    public static final Schema[] CONSUMER_METADATA_REQUEST = new Schema[] {CONSUMER_METADATA_REQUEST_V0};
-    public static final Schema[] CONSUMER_METADATA_RESPONSE = new Schema[] {CONSUMER_METADATA_RESPONSE_V0};
+    public static final Schema[] GROUP_METADATA_REQUEST = new Schema[] {GROUP_METADATA_REQUEST_V0};
+    public static final Schema[] GROUP_METADATA_RESPONSE = new Schema[] {GROUP_METADATA_RESPONSE_V0};
 
     /* Controlled shutdown api */
     public static final Schema CONTROLLED_SHUTDOWN_REQUEST_V1 = new Schema(new Field("broker_id",
@@ -416,45 +421,67 @@ public class Protocol {
     public static final Schema[] CONTROLLED_SHUTDOWN_RESPONSE = new Schema[] {null, CONTROLLED_SHUTDOWN_RESPONSE_V1};
 
     /* Join group api */
+    public static final Schema JOIN_GROUP_REQUEST_PROTOCOL_V0 = new Schema(new Field("protocol_name", STRING),
+                                                                           new Field("protocol_metadata", BYTES));
+
     public static final Schema JOIN_GROUP_REQUEST_V0 = new Schema(new Field("group_id",
                                                                             STRING,
-                                                                            "The consumer group id."),
+                                                                            "The group id."),
                                                                   new Field("session_timeout",
                                                                             INT32,
                                                                             "The coordinator considers the consumer dead if it receives no heartbeat after this timeout in ms."),
-                                                                  new Field("topics",
-                                                                            new ArrayOf(STRING),
-                                                                            "An array of topics to subscribe to."),
-                                                                  new Field("consumer_id",
+                                                                  new Field("member_id",
                                                                             STRING,
                                                                             "The assigned consumer id or an empty string for a new consumer."),
-                                                                  new Field("partition_assignment_strategy",
+                                                                  new Field("protocol_type",
                                                                             STRING,
-                                                                            "The strategy for the coordinator to assign partitions."));
+                                                                            "Unique name for class of protocols implemented by group"),
+                                                                  new Field("group_protocols",
+                                                                            new ArrayOf(JOIN_GROUP_REQUEST_PROTOCOL_V0),
+                                                                            "List of protocols that the member supports"));
+
 
-    public static final Schema JOIN_GROUP_RESPONSE_TOPIC_V0 = new Schema(new Field("topic", STRING),
-                                                                         new Field("partitions", new ArrayOf(INT32)));
+    public static final Schema JOIN_GROUP_RESPONSE_MEMBER_V0 = new Schema(new Field("member_id", STRING),
+                                                                          new Field("member_metadata", BYTES));
     public static final Schema JOIN_GROUP_RESPONSE_V0 = new Schema(new Field("error_code", INT16),
-                                                                   new Field("group_generation_id",
+                                                                   new Field("generation_id",
                                                                              INT32,
                                                                              "The generation of the consumer group."),
-                                                                   new Field("consumer_id",
+                                                                   new Field("group_protocol",
+                                                                             STRING,
+                                                                             "The group protocol selected by the coordinator"),
+                                                                   new Field("leader_id",
+                                                                             STRING,
+                                                                             "The leader of the group"),
+                                                                   new Field("member_id",
                                                                              STRING,
                                                                              "The consumer id assigned by the group coordinator."),
-                                                                   new Field("assigned_partitions",
-                                                                             new ArrayOf(JOIN_GROUP_RESPONSE_TOPIC_V0)));
+                                                                   new Field("members",
+                                                                             new ArrayOf(JOIN_GROUP_RESPONSE_MEMBER_V0)));
 
     public static final Schema[] JOIN_GROUP_REQUEST = new Schema[] {JOIN_GROUP_REQUEST_V0};
     public static final Schema[] JOIN_GROUP_RESPONSE = new Schema[] {JOIN_GROUP_RESPONSE_V0};
 
+    /* SyncGroup api */
+    public static final Schema SYNC_GROUP_REQUEST_MEMBER_V0 = new Schema(new Field("member_id", STRING),
+                                                                         new Field("member_assignment", BYTES));
+    public static final Schema SYNC_GROUP_REQUEST_V0 = new Schema(new Field("group_id", STRING),
+                                                                  new Field("generation_id", INT32),
+                                                                  new Field("member_id", STRING),
+                                                                  new Field("group_assignment", new ArrayOf(SYNC_GROUP_REQUEST_MEMBER_V0)));
+    public static final Schema SYNC_GROUP_RESPONSE_V0 = new Schema(new Field("error_code", INT16),
+                                                                   new Field("member_assignment", BYTES));
+    public static final Schema[] SYNC_GROUP_REQUEST = new Schema[] {SYNC_GROUP_REQUEST_V0};
+    public static final Schema[] SYNC_GROUP_RESPONSE = new Schema[] {SYNC_GROUP_RESPONSE_V0};
+
     /* Heartbeat api */
     public static final Schema HEARTBEAT_REQUEST_V0 = new Schema(new Field("group_id", STRING, "The consumer group id."),
                                                                  new Field("group_generation_id",
                                                                            INT32,
                                                                            "The generation of the consumer group."),
-                                                                 new Field("consumer_id",
+                                                                 new Field("member_id",
                                                                            STRING,
-                                                                           "The consumer id assigned by the group coordinator."));
+                                                                           "The member id assigned by the group coordinator."));
 
     public static final Schema HEARTBEAT_RESPONSE_V0 = new Schema(new Field("error_code", INT16));
 
@@ -589,10 +616,11 @@ public class Protocol {
         REQUESTS[ApiKeys.CONTROLLED_SHUTDOWN_KEY.id] = CONTROLLED_SHUTDOWN_REQUEST;
         REQUESTS[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_REQUEST;
         REQUESTS[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_REQUEST;
-        REQUESTS[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_REQUEST;
+        REQUESTS[ApiKeys.GROUP_METADATA.id] = GROUP_METADATA_REQUEST;
         REQUESTS[ApiKeys.JOIN_GROUP.id] = JOIN_GROUP_REQUEST;
         REQUESTS[ApiKeys.HEARTBEAT.id] = HEARTBEAT_REQUEST;
         REQUESTS[ApiKeys.LEAVE_GROUP.id] = LEAVE_GROUP_REQUEST;
+        REQUESTS[ApiKeys.SYNC_GROUP.id] = SYNC_GROUP_REQUEST;
 
 
         RESPONSES[ApiKeys.PRODUCE.id] = PRODUCE_RESPONSE;
@@ -605,10 +633,11 @@ public class Protocol {
         RESPONSES[ApiKeys.CONTROLLED_SHUTDOWN_KEY.id] = CONTROLLED_SHUTDOWN_RESPONSE;
         RESPONSES[ApiKeys.OFFSET_COMMIT.id] = OFFSET_COMMIT_RESPONSE;
         RESPONSES[ApiKeys.OFFSET_FETCH.id] = OFFSET_FETCH_RESPONSE;
-        RESPONSES[ApiKeys.CONSUMER_METADATA.id] = CONSUMER_METADATA_RESPONSE;
+        RESPONSES[ApiKeys.GROUP_METADATA.id] = GROUP_METADATA_RESPONSE;
         RESPONSES[ApiKeys.JOIN_GROUP.id] = JOIN_GROUP_RESPONSE;
         RESPONSES[ApiKeys.HEARTBEAT.id] = HEARTBEAT_RESPONSE;
         RESPONSES[ApiKeys.LEAVE_GROUP.id] = LEAVE_GROUP_RESPONSE;
+        RESPONSES[ApiKeys.SYNC_GROUP.id] = SYNC_GROUP_RESPONSE;
 
         /* set the maximum version of each api */
         for (ApiKeys api : ApiKeys.values())

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
index 095cd52..03e77a5 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
@@ -49,14 +49,16 @@ public abstract class AbstractRequest extends AbstractRequestResponse {
                 return OffsetCommitRequest.parse(buffer, versionId);
             case OFFSET_FETCH:
                 return OffsetFetchRequest.parse(buffer, versionId);
-            case CONSUMER_METADATA:
-                return ConsumerMetadataRequest.parse(buffer, versionId);
+            case GROUP_METADATA:
+                return GroupMetadataRequest.parse(buffer, versionId);
             case JOIN_GROUP:
                 return JoinGroupRequest.parse(buffer, versionId);
             case HEARTBEAT:
                 return HeartbeatRequest.parse(buffer, versionId);
             case LEAVE_GROUP:
                 return LeaveGroupRequest.parse(buffer, versionId);
+            case SYNC_GROUP:
+                return SyncGroupRequest.parse(buffer, versionId);
             case STOP_REPLICA:
                 return StopReplicaRequest.parse(buffer, versionId);
             case CONTROLLED_SHUTDOWN_KEY:

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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
deleted file mode 100644
index 5b3e04a..0000000
--- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataRequest.java
+++ /dev/null
@@ -1,65 +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.common.requests;
-
-import org.apache.kafka.common.Node;
-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 java.nio.ByteBuffer;
-
-public class ConsumerMetadataRequest extends AbstractRequest {
-    
-    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(CURRENT_SCHEMA));
-
-        struct.set(GROUP_ID_KEY_NAME, groupId);
-        this.groupId = groupId;
-    }
-
-    public ConsumerMetadataRequest(Struct struct) {
-        super(struct);
-        groupId = struct.getString(GROUP_ID_KEY_NAME);
-    }
-
-    @Override
-    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
-        switch (versionId) {
-            case 0:
-                return new ConsumerMetadataResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code(), Node.noNode());
-            default:
-                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
-                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.CONSUMER_METADATA.id)));
-        }
-    }
-
-    public String groupId() {
-        return groupId;
-    }
-
-    public static ConsumerMetadataRequest parse(ByteBuffer buffer, int versionId) {
-        return new ConsumerMetadataRequest(ProtoUtils.parseRequest(ApiKeys.CONSUMER_METADATA.id, versionId, buffer));
-    }
-
-    public static ConsumerMetadataRequest parse(ByteBuffer buffer) {
-        return new ConsumerMetadataRequest((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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
deleted file mode 100644
index 0c250c3..0000000
--- a/clients/src/main/java/org/apache/kafka/common/requests/ConsumerMetadataResponse.java
+++ /dev/null
@@ -1,70 +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.common.requests;
-
-import org.apache.kafka.common.Node;
-import org.apache.kafka.common.protocol.ApiKeys;
-import org.apache.kafka.common.protocol.ProtoUtils;
-import org.apache.kafka.common.protocol.types.Schema;
-import org.apache.kafka.common.protocol.types.Struct;
-
-import java.nio.ByteBuffer;
-
-public class ConsumerMetadataResponse extends AbstractRequestResponse {
-    
-    private static 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 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(CURRENT_SCHEMA));
-        struct.set(ERROR_CODE_KEY_NAME, errorCode);
-        Struct coordinator = struct.instance(COORDINATOR_KEY_NAME);
-        coordinator.set(NODE_ID_KEY_NAME, node.id());
-        coordinator.set(HOST_KEY_NAME, node.host());
-        coordinator.set(PORT_KEY_NAME, node.port());
-        struct.set(COORDINATOR_KEY_NAME, coordinator);
-        this.errorCode = errorCode;
-        this.node = node;
-    }
-
-    public ConsumerMetadataResponse(Struct struct) {
-        super(struct);
-        errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
-        Struct broker = (Struct) struct.get(COORDINATOR_KEY_NAME);
-        int nodeId = broker.getInt(NODE_ID_KEY_NAME);
-        String host = broker.getString(HOST_KEY_NAME);
-        int port = broker.getInt(PORT_KEY_NAME);
-        node = new Node(nodeId, host, port);
-    }
-
-    public short errorCode() {
-        return errorCode;
-    }
-
-    public Node node() {
-        return node;
-    }
-
-    public static ConsumerMetadataResponse parse(ByteBuffer buffer) {
-        return new ConsumerMetadataResponse((Struct) CURRENT_SCHEMA.read(buffer));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/common/requests/GroupMetadataRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/GroupMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/GroupMetadataRequest.java
new file mode 100644
index 0000000..fd54c5a
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/GroupMetadataRequest.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.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 java.nio.ByteBuffer;
+
+public class GroupMetadataRequest extends AbstractRequest {
+    
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.GROUP_METADATA.id);
+    private static final String GROUP_ID_KEY_NAME = "group_id";
+
+    private final String groupId;
+
+    public GroupMetadataRequest(String groupId) {
+        super(new Struct(CURRENT_SCHEMA));
+
+        struct.set(GROUP_ID_KEY_NAME, groupId);
+        this.groupId = groupId;
+    }
+
+    public GroupMetadataRequest(Struct struct) {
+        super(struct);
+        groupId = struct.getString(GROUP_ID_KEY_NAME);
+    }
+
+    @Override
+    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
+        switch (versionId) {
+            case 0:
+                return new GroupMetadataResponse(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code(), Node.noNode());
+            default:
+                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
+                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.GROUP_METADATA.id)));
+        }
+    }
+
+    public String groupId() {
+        return groupId;
+    }
+
+    public static GroupMetadataRequest parse(ByteBuffer buffer, int versionId) {
+        return new GroupMetadataRequest(ProtoUtils.parseRequest(ApiKeys.GROUP_METADATA.id, versionId, buffer));
+    }
+
+    public static GroupMetadataRequest parse(ByteBuffer buffer) {
+        return new GroupMetadataRequest((Struct) CURRENT_SCHEMA.read(buffer));
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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 89719f1..74be3ed 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
@@ -25,27 +25,27 @@ public class HeartbeatRequest extends AbstractRequest {
     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 static final String MEMBER_ID_KEY_NAME = "member_id";
 
     private final String groupId;
     private final int groupGenerationId;
-    private final String consumerId;
+    private final String memberId;
 
-    public HeartbeatRequest(String groupId, int groupGenerationId, String consumerId) {
+    public HeartbeatRequest(String groupId, int groupGenerationId, String memberId) {
         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);
+        struct.set(MEMBER_ID_KEY_NAME, memberId);
         this.groupId = groupId;
         this.groupGenerationId = groupGenerationId;
-        this.consumerId = consumerId;
+        this.memberId = memberId;
     }
 
     public HeartbeatRequest(Struct struct) {
         super(struct);
         groupId = struct.getString(GROUP_ID_KEY_NAME);
         groupGenerationId = struct.getInt(GROUP_GENERATION_ID_KEY_NAME);
-        consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
+        memberId = struct.getString(MEMBER_ID_KEY_NAME);
     }
 
     @Override
@@ -67,8 +67,8 @@ public class HeartbeatRequest extends AbstractRequest {
         return groupGenerationId;
     }
 
-    public String consumerId() {
-        return consumerId;
+    public String memberId() {
+        return memberId;
     }
 
     public static HeartbeatRequest parse(ByteBuffer buffer, int versionId) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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 96e6ab0..48cb4c0 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
@@ -27,10 +27,10 @@ public class HeartbeatResponse extends AbstractRequestResponse {
     /**
      * Possible error code:
      *
-     * CONSUMER_COORDINATOR_NOT_AVAILABLE (15)
-     * NOT_COORDINATOR_FOR_CONSUMER (16)
+     * GROUP_COORDINATOR_NOT_AVAILABLE (15)
+     * NOT_COORDINATOR_FOR_GROUP (16)
      * ILLEGAL_GENERATION (22)
-     * UNKNOWN_CONSUMER_ID (25)
+     * UNKNOWN_MEMBER_ID (25)
      */
 
     private final short errorCode;

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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 1ffe076..91a698c 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
@@ -12,7 +12,6 @@
  */
 package org.apache.kafka.common.requests;
 
-import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.protocol.ProtoUtils;
@@ -29,42 +28,79 @@ public class JoinGroupRequest extends AbstractRequest {
     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 static final String MEMBER_ID_KEY_NAME = "member_id";
+    private static final String PROTOCOL_TYPE_KEY_NAME = "protocol_type";
+    private static final String GROUP_PROTOCOLS_KEY_NAME = "group_protocols";
+    private static final String PROTOCOL_NAME_KEY_NAME = "protocol_name";
+    private static final String PROTOCOL_METADATA_KEY_NAME = "protocol_metadata";
 
-    public static final String UNKNOWN_CONSUMER_ID = "";
+    public static final String UNKNOWN_MEMBER_ID = "";
 
     private final String groupId;
     private final int sessionTimeout;
-    private final List<String> topics;
-    private final String consumerId;
-    private final String strategy;
+    private final String memberId;
+    private final String protocolType;
+    private final List<GroupProtocol> groupProtocols;
 
-    public JoinGroupRequest(String groupId, int sessionTimeout, List<String> topics, String consumerId, String strategy) {
+    public static class GroupProtocol {
+        private final String name;
+        private final ByteBuffer metadata;
+
+        public GroupProtocol(String name, ByteBuffer metadata) {
+            this.name = name;
+            this.metadata = metadata;
+        }
+
+        public String name() {
+            return name;
+        }
+
+        public ByteBuffer metadata() {
+            return metadata;
+        }
+    }
+
+    public JoinGroupRequest(String groupId,
+                            int sessionTimeout,
+                            String memberId,
+                            String protocolType,
+                            List<GroupProtocol> groupProtocols) {
         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());
-        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
-        struct.set(STRATEGY_KEY_NAME, strategy);
+        struct.set(MEMBER_ID_KEY_NAME, memberId);
+        struct.set(PROTOCOL_TYPE_KEY_NAME, protocolType);
+
+        List<Struct> groupProtocolsList = new ArrayList<>();
+        for (GroupProtocol protocol : groupProtocols) {
+            Struct protocolStruct = struct.instance(GROUP_PROTOCOLS_KEY_NAME);
+            protocolStruct.set(PROTOCOL_NAME_KEY_NAME, protocol.name);
+            protocolStruct.set(PROTOCOL_METADATA_KEY_NAME, protocol.metadata);
+            groupProtocolsList.add(protocolStruct);
+        }
+
+        struct.set(GROUP_PROTOCOLS_KEY_NAME, groupProtocolsList.toArray());
         this.groupId = groupId;
         this.sessionTimeout = sessionTimeout;
-        this.topics = topics;
-        this.consumerId = consumerId;
-        this.strategy = strategy;
+        this.memberId = memberId;
+        this.protocolType = protocolType;
+        this.groupProtocols = groupProtocols;
     }
 
     public JoinGroupRequest(Struct struct) {
         super(struct);
         groupId = struct.getString(GROUP_ID_KEY_NAME);
         sessionTimeout = struct.getInt(SESSION_TIMEOUT_KEY_NAME);
-        Object[] topicsArray = struct.getArray(TOPICS_KEY_NAME);
-        topics = new ArrayList<String>();
-        for (Object topic: topicsArray)
-            topics.add((String) topic);
-        consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
-        strategy = struct.getString(STRATEGY_KEY_NAME);
+        memberId = struct.getString(MEMBER_ID_KEY_NAME);
+        protocolType = struct.getString(PROTOCOL_TYPE_KEY_NAME);
+
+        groupProtocols = new ArrayList<>();
+        for (Object groupProtocolObj : struct.getArray(GROUP_PROTOCOLS_KEY_NAME)) {
+            Struct groupProtocolStruct = (Struct) groupProtocolObj;
+            String name = groupProtocolStruct.getString(PROTOCOL_NAME_KEY_NAME);
+            ByteBuffer metadata = groupProtocolStruct.getBytes(PROTOCOL_METADATA_KEY_NAME);
+            groupProtocols.add(new GroupProtocol(name, metadata));
+        }
     }
 
     @Override
@@ -74,8 +110,10 @@ public class JoinGroupRequest extends AbstractRequest {
                 return new JoinGroupResponse(
                         Errors.forException(e).code(),
                         JoinGroupResponse.UNKNOWN_GENERATION_ID,
-                        JoinGroupResponse.UNKNOWN_CONSUMER_ID,
-                        Collections.<TopicPartition>emptyList());
+                        JoinGroupResponse.UNKNOWN_PROTOCOL,
+                        JoinGroupResponse.UNKNOWN_MEMBER_ID, // memberId
+                        JoinGroupResponse.UNKNOWN_MEMBER_ID, // leaderId
+                        Collections.<String, ByteBuffer>emptyMap());
             default:
                 throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
                         versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.JOIN_GROUP.id)));
@@ -90,16 +128,16 @@ public class JoinGroupRequest extends AbstractRequest {
         return sessionTimeout;
     }
 
-    public List<String> topics() {
-        return topics;
+    public String memberId() {
+        return memberId;
     }
 
-    public String consumerId() {
-        return consumerId;
+    public List<GroupProtocol> groupProtocols() {
+        return groupProtocols;
     }
 
-    public String strategy() {
-        return strategy;
+    public String protocolType() {
+        return protocolType;
     }
 
     public static JoinGroupRequest parse(ByteBuffer buffer, int versionId) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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 7bf544e..c65a4bb 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
@@ -12,15 +12,16 @@
  */
 package org.apache.kafka.common.requests;
 
-import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.ProtoUtils;
 import org.apache.kafka.common.protocol.types.Schema;
 import org.apache.kafka.common.protocol.types.Struct;
-import org.apache.kafka.common.utils.CollectionUtils;
 
 import java.nio.ByteBuffer;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 
 public class JoinGroupResponse extends AbstractRequestResponse {
     
@@ -30,63 +31,78 @@ public class JoinGroupResponse extends AbstractRequestResponse {
     /**
      * Possible error code:
      *
-     * CONSUMER_COORDINATOR_NOT_AVAILABLE (15)
-     * NOT_COORDINATOR_FOR_CONSUMER (16)
-     * INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY (23)
-     * UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY (24)
-     * UNKNOWN_CONSUMER_ID (25)
+     * GROUP_COORDINATOR_NOT_AVAILABLE (15)
+     * NOT_COORDINATOR_FOR_GROUP (16)
+     * INCONSISTENT_GROUP_PROTOCOL (23)
+     * UNKNOWN_MEMBER_ID (25)
      * INVALID_SESSION_TIMEOUT (26)
      */
 
-    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";
+    private static final String GENERATION_ID_KEY_NAME = "generation_id";
+    private static final String GROUP_PROTOCOL_KEY_NAME = "group_protocol";
+    private static final String LEADER_ID_KEY_NAME = "leader_id";
+    private static final String MEMBER_ID_KEY_NAME = "member_id";
+    private static final String MEMBERS_KEY_NAME = "members";
 
+    private static final String MEMBER_METADATA_KEY_NAME = "member_metadata";
+
+    public static final String UNKNOWN_PROTOCOL = "";
     public static final int UNKNOWN_GENERATION_ID = -1;
-    public static final String UNKNOWN_CONSUMER_ID = "";
+    public static final String UNKNOWN_MEMBER_ID = "";
 
     private final short errorCode;
     private final int generationId;
-    private final String consumerId;
-    private final List<TopicPartition> assignedPartitions;
-
-    public JoinGroupResponse(short errorCode, int generationId, String consumerId, List<TopicPartition> assignedPartitions) {
+    private final String groupProtocol;
+    private final String memberId;
+    private final String leaderId;
+    private final Map<String, ByteBuffer> members;
+
+    public JoinGroupResponse(short errorCode,
+                             int generationId,
+                             String groupProtocol,
+                             String memberId,
+                             String leaderId,
+                             Map<String, ByteBuffer> groupMembers) {
         super(new Struct(CURRENT_SCHEMA));
 
-        Map<String, List<Integer>> partitionsByTopic = CollectionUtils.groupDataByTopic(assignedPartitions);
-
         struct.set(ERROR_CODE_KEY_NAME, errorCode);
         struct.set(GENERATION_ID_KEY_NAME, generationId);
-        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
-        List<Struct> topicArray = new ArrayList<Struct>();
-        for (Map.Entry<String, List<Integer>> entries: partitionsByTopic.entrySet()) {
-            Struct topicData = struct.instance(ASSIGNED_PARTITIONS_KEY_NAME);
-            topicData.set(TOPIC_KEY_NAME, entries.getKey());
-            topicData.set(PARTITIONS_KEY_NAME, entries.getValue().toArray());
-            topicArray.add(topicData);
+        struct.set(GROUP_PROTOCOL_KEY_NAME, groupProtocol);
+        struct.set(MEMBER_ID_KEY_NAME, memberId);
+        struct.set(LEADER_ID_KEY_NAME, leaderId);
+
+        List<Struct> memberArray = new ArrayList<>();
+        for (Map.Entry<String, ByteBuffer> entries: groupMembers.entrySet()) {
+            Struct memberData = struct.instance(MEMBERS_KEY_NAME);
+            memberData.set(MEMBER_ID_KEY_NAME, entries.getKey());
+            memberData.set(MEMBER_METADATA_KEY_NAME, entries.getValue());
+            memberArray.add(memberData);
         }
-        struct.set(ASSIGNED_PARTITIONS_KEY_NAME, topicArray.toArray());
+        struct.set(MEMBERS_KEY_NAME, memberArray.toArray());
 
         this.errorCode = errorCode;
         this.generationId = generationId;
-        this.consumerId = consumerId;
-        this.assignedPartitions = assignedPartitions;
+        this.groupProtocol = groupProtocol;
+        this.memberId = memberId;
+        this.leaderId = leaderId;
+        this.members = groupMembers;
     }
 
     public JoinGroupResponse(Struct struct) {
         super(struct);
-        assignedPartitions = new ArrayList<TopicPartition>();
-        for (Object topicDataObj : struct.getArray(ASSIGNED_PARTITIONS_KEY_NAME)) {
-            Struct topicData = (Struct) topicDataObj;
-            String topic = topicData.getString(TOPIC_KEY_NAME);
-            for (Object partitionObj : topicData.getArray(PARTITIONS_KEY_NAME))
-                assignedPartitions.add(new TopicPartition(topic, (Integer) partitionObj));
+        members = new HashMap<>();
+
+        for (Object memberDataObj : struct.getArray(MEMBERS_KEY_NAME)) {
+            Struct memberData = (Struct) memberDataObj;
+            String memberId = memberData.getString(MEMBER_ID_KEY_NAME);
+            ByteBuffer memberMetadata = memberData.getBytes(MEMBER_METADATA_KEY_NAME);
+            members.put(memberId, memberMetadata);
         }
         errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
         generationId = struct.getInt(GENERATION_ID_KEY_NAME);
-        consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
+        groupProtocol = struct.getString(GROUP_PROTOCOL_KEY_NAME);
+        memberId = struct.getString(MEMBER_ID_KEY_NAME);
+        leaderId = struct.getString(LEADER_ID_KEY_NAME);
     }
 
     public short errorCode() {
@@ -97,12 +113,24 @@ public class JoinGroupResponse extends AbstractRequestResponse {
         return generationId;
     }
 
-    public String consumerId() {
-        return consumerId;
+    public String groupProtocol() {
+        return groupProtocol;
+    }
+
+    public String memberId() {
+        return memberId;
+    }
+
+    public String leaderId() {
+        return leaderId;
+    }
+
+    public boolean isLeader() {
+        return memberId.equals(leaderId);
     }
 
-    public List<TopicPartition> assignedPartitions() {
-        return assignedPartitions;
+    public Map<String, ByteBuffer> members() {
+        return members;
     }
 
     public static JoinGroupResponse parse(ByteBuffer buffer) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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 03df1e7..8721efa 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
@@ -34,7 +34,7 @@ public class OffsetCommitRequest extends AbstractRequest {
     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 MEMBER_ID_KEY_NAME = "member_id";
     private static final String TOPICS_KEY_NAME = "topics";
     private static final String RETENTION_TIME_KEY_NAME = "retention_time";
 
@@ -52,7 +52,7 @@ public class OffsetCommitRequest extends AbstractRequest {
 
     // default values for the current version
     public static final int DEFAULT_GENERATION_ID = -1;
-    public static final String DEFAULT_CONSUMER_ID = "";
+    public static final String DEFAULT_MEMBER_ID = "";
     public static final long DEFAULT_RETENTION_TIME = -1L;
 
     // default values for old versions,
@@ -61,7 +61,7 @@ public class OffsetCommitRequest extends AbstractRequest {
     public static final long DEFAULT_TIMESTAMP = -1L;            // for V0, V1
 
     private final String groupId;
-    private final String consumerId;
+    private final String memberId;
     private final int generationId;
     private final long retentionTime;
     private final Map<TopicPartition, PartitionData> offsetData;
@@ -97,7 +97,7 @@ public class OffsetCommitRequest extends AbstractRequest {
         initCommonFields(groupId, offsetData);
         this.groupId = groupId;
         this.generationId = DEFAULT_GENERATION_ID;
-        this.consumerId = DEFAULT_CONSUMER_ID;
+        this.memberId = DEFAULT_MEMBER_ID;
         this.retentionTime = DEFAULT_RETENTION_TIME;
         this.offsetData = offsetData;
     }
@@ -106,19 +106,19 @@ public class OffsetCommitRequest extends AbstractRequest {
      * Constructor for version 1.
      * @param groupId
      * @param generationId
-     * @param consumerId
+     * @param memberId
      * @param offsetData
      */
     @Deprecated
-    public OffsetCommitRequest(String groupId, int generationId, String consumerId, Map<TopicPartition, PartitionData> offsetData) {
+    public OffsetCommitRequest(String groupId, int generationId, String memberId, Map<TopicPartition, PartitionData> offsetData) {
         super(new Struct(ProtoUtils.requestSchema(ApiKeys.OFFSET_COMMIT.id, 1)));
 
         initCommonFields(groupId, offsetData);
         struct.set(GENERATION_ID_KEY_NAME, generationId);
-        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
+        struct.set(MEMBER_ID_KEY_NAME, memberId);
         this.groupId = groupId;
         this.generationId = generationId;
-        this.consumerId = consumerId;
+        this.memberId = memberId;
         this.retentionTime = DEFAULT_RETENTION_TIME;
         this.offsetData = offsetData;
     }
@@ -127,20 +127,20 @@ public class OffsetCommitRequest extends AbstractRequest {
      * Constructor for version 2.
      * @param groupId
      * @param generationId
-     * @param consumerId
+     * @param memberId
      * @param retentionTime
      * @param offsetData
      */
-    public OffsetCommitRequest(String groupId, int generationId, String consumerId, long retentionTime, Map<TopicPartition, PartitionData> offsetData) {
+    public OffsetCommitRequest(String groupId, int generationId, String memberId, long retentionTime, Map<TopicPartition, PartitionData> offsetData) {
         super(new Struct(CURRENT_SCHEMA));
 
         initCommonFields(groupId, offsetData);
         struct.set(GENERATION_ID_KEY_NAME, generationId);
-        struct.set(CONSUMER_ID_KEY_NAME, consumerId);
+        struct.set(MEMBER_ID_KEY_NAME, memberId);
         struct.set(RETENTION_TIME_KEY_NAME, retentionTime);
         this.groupId = groupId;
         this.generationId = generationId;
-        this.consumerId = consumerId;
+        this.memberId = memberId;
         this.retentionTime = retentionTime;
         this.offsetData = offsetData;
     }
@@ -183,10 +183,10 @@ public class OffsetCommitRequest extends AbstractRequest {
             generationId = DEFAULT_GENERATION_ID;
 
         // This field only exists in v1.
-        if (struct.hasField(CONSUMER_ID_KEY_NAME))
-            consumerId = struct.getString(CONSUMER_ID_KEY_NAME);
+        if (struct.hasField(MEMBER_ID_KEY_NAME))
+            memberId = struct.getString(MEMBER_ID_KEY_NAME);
         else
-            consumerId = DEFAULT_CONSUMER_ID;
+            memberId = DEFAULT_MEMBER_ID;
 
         // This field only exists in v2
         if (struct.hasField(RETENTION_TIME_KEY_NAME))
@@ -243,8 +243,8 @@ public class OffsetCommitRequest extends AbstractRequest {
         return generationId;
     }
 
-    public String consumerId() {
-        return consumerId;
+    public String memberId() {
+        return memberId;
     }
 
     public long retentionTime() {

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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 a163333..dae9c37 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
@@ -42,10 +42,10 @@ public class OffsetCommitResponse extends AbstractRequestResponse {
      * Possible error code:
      *
      * OFFSET_METADATA_TOO_LARGE (12)
-     * CONSUMER_COORDINATOR_NOT_AVAILABLE (15)
-     * NOT_COORDINATOR_FOR_CONSUMER (16)
+     * GROUP_COORDINATOR_NOT_AVAILABLE (15)
+     * NOT_COORDINATOR_FOR_GROUP (16)
      * ILLEGAL_GENERATION (22)
-     * UNKNOWN_CONSUMER_ID (25)
+     * UNKNOWN_MEMBER_ID (25)
      * COMMITTING_PARTITIONS_NOT_ASSIGNED (27)
      * INVALID_COMMIT_OFFSET_SIZE (28)
      */

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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 3dc8521..09ac74a 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
@@ -49,9 +49,9 @@ public class OffsetFetchResponse extends AbstractRequestResponse {
      *
      *  UNKNOWN_TOPIC_OR_PARTITION (3)  <- only for request v0
      *  OFFSET_LOAD_IN_PROGRESS (14)
-     *  NOT_COORDINATOR_FOR_CONSUMER (16)
+     *  NOT_COORDINATOR_FOR_GROUP (16)
      *  ILLEGAL_GENERATION (22)
-     *  UNKNOWN_CONSUMER_ID (25)
+     *  UNKNOWN_MEMBER_ID (25)
      */
 
     private final Map<TopicPartition, PartitionData> responseData;

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java
new file mode 100644
index 0000000..606584b
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupRequest.java
@@ -0,0 +1,118 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.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 java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class SyncGroupRequest extends AbstractRequest {
+
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.SYNC_GROUP.id);
+    public static final String GROUP_ID_KEY_NAME = "group_id";
+    public static final String GENERATION_ID_KEY_NAME = "generation_id";
+    public static final String MEMBER_ID_KEY_NAME = "member_id";
+    public static final String MEMBER_ASSIGNMENT_KEY_NAME = "member_assignment";
+    public static final String GROUP_ASSIGNMENT_KEY_NAME = "group_assignment";
+
+    private final String groupId;
+    private final int generationId;
+    private final String memberId;
+    private final Map<String, ByteBuffer> groupAssignment;
+
+    public SyncGroupRequest(String groupId,
+                            int generationId,
+                            String memberId,
+                            Map<String, ByteBuffer> groupAssignment) {
+        super(new Struct(CURRENT_SCHEMA));
+        struct.set(GROUP_ID_KEY_NAME, groupId);
+        struct.set(GENERATION_ID_KEY_NAME, generationId);
+        struct.set(MEMBER_ID_KEY_NAME, memberId);
+
+        List<Struct> memberArray = new ArrayList<>();
+        for (Map.Entry<String, ByteBuffer> entries: groupAssignment.entrySet()) {
+            Struct memberData = struct.instance(GROUP_ASSIGNMENT_KEY_NAME);
+            memberData.set(MEMBER_ID_KEY_NAME, entries.getKey());
+            memberData.set(MEMBER_ASSIGNMENT_KEY_NAME, entries.getValue());
+            memberArray.add(memberData);
+        }
+        struct.set(GROUP_ASSIGNMENT_KEY_NAME, memberArray.toArray());
+
+        this.groupId = groupId;
+        this.generationId = generationId;
+        this.memberId = memberId;
+        this.groupAssignment = groupAssignment;
+    }
+
+    public SyncGroupRequest(Struct struct) {
+        super(struct);
+        this.groupId = struct.getString(GROUP_ID_KEY_NAME);
+        this.generationId = struct.getInt(GENERATION_ID_KEY_NAME);
+        this.memberId = struct.getString(MEMBER_ID_KEY_NAME);
+
+        groupAssignment = new HashMap<>();
+
+        for (Object memberDataObj : struct.getArray(GROUP_ASSIGNMENT_KEY_NAME)) {
+            Struct memberData = (Struct) memberDataObj;
+            String memberId = memberData.getString(MEMBER_ID_KEY_NAME);
+            ByteBuffer memberMetadata = memberData.getBytes(MEMBER_ASSIGNMENT_KEY_NAME);
+            groupAssignment.put(memberId, memberMetadata);
+        }
+    }
+
+    @Override
+    public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) {
+        switch (versionId) {
+            case 0:
+                return new SyncGroupResponse(
+                        Errors.forException(e).code(),
+                        ByteBuffer.wrap(new byte[]{}));
+            default:
+                throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d",
+                        versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.JOIN_GROUP.id)));
+        }
+    }
+
+    public String groupId() {
+        return groupId;
+    }
+
+    public int generationId() {
+        return generationId;
+    }
+
+    public Map<String, ByteBuffer> groupAssignment() {
+        return groupAssignment;
+    }
+
+    public String memberId() {
+        return memberId;
+    }
+
+    public static SyncGroupRequest parse(ByteBuffer buffer, int versionId) {
+        return new SyncGroupRequest(ProtoUtils.parseRequest(ApiKeys.SYNC_GROUP.id, versionId, buffer));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java
new file mode 100644
index 0000000..a96b7e5
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/requests/SyncGroupResponse.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.common.requests;
+
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+
+import java.nio.ByteBuffer;
+
+public class SyncGroupResponse extends AbstractRequestResponse {
+
+    private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.SYNC_GROUP.id);
+    public static final String ERROR_CODE_KEY_NAME = "error_code";
+    public static final String MEMBER_ASSIGNMENT_KEY_NAME = "member_assignment";
+
+    /**
+     * Possible error codes:
+     *
+     * GROUP_COORDINATOR_NOT_AVAILABLE (15)
+     * NOT_COORDINATOR_FOR_GROUP (16)
+     * ILLEGAL_GENERATION (22)
+     * UNKNOWN_MEMBER_ID (25)
+     * REBALANCE_IN_PROGRESS (30)
+     *
+     */
+
+    private final short errorCode;
+    private final ByteBuffer memberState;
+
+    public SyncGroupResponse(short errorCode, ByteBuffer memberState) {
+        super(new Struct(CURRENT_SCHEMA));
+
+        struct.set(ERROR_CODE_KEY_NAME, errorCode);
+        struct.set(MEMBER_ASSIGNMENT_KEY_NAME, memberState);
+
+        this.errorCode = errorCode;
+        this.memberState = memberState;
+    }
+
+    public SyncGroupResponse(Struct struct) {
+        super(struct);
+
+        this.errorCode = struct.getShort(ERROR_CODE_KEY_NAME);
+        this.memberState = struct.getBytes(MEMBER_ASSIGNMENT_KEY_NAME);
+    }
+
+    public short errorCode() {
+        return errorCode;
+    }
+
+    public ByteBuffer memberAssignment() {
+        return memberState;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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 a7a2968..bc0e645 100755
--- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
@@ -23,6 +23,7 @@ import java.io.StringWriter;
 import java.io.PrintWriter;
 import java.nio.ByteBuffer;
 import java.nio.MappedByteBuffer;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -52,6 +53,18 @@ public class Utils {
     private static final Logger log = LoggerFactory.getLogger(Utils.class);
 
     /**
+     * Get a sorted list representation of a collection.
+     * @param collection The collection to sort
+     * @param <T> The class of objects in the collection
+     * @return An unmodifiable sorted list with the contents of the collection
+     */
+    public static <T extends Comparable<? super T>> List<T> sorted(Collection<T> collection) {
+        List<T> res = new ArrayList<>(collection);
+        Collections.sort(res);
+        return Collections.unmodifiableList(res);
+    }
+
+    /**
      * Turn the given UTF8 byte array into a string
      *
      * @param bytes The byte array
@@ -114,6 +127,21 @@ public class Utils {
     }
 
     /**
+     * Get the little-endian value of an integer as a byte array.
+     * @param val The value to convert to a litte-endian array
+     * @return The little-endian encoded array of bytes for the value
+     */
+    public static byte[] toArrayLE(int val) {
+        return new byte[] {
+            (byte) (val >> 8 * 0),
+            (byte) (val >> 8 * 1),
+            (byte) (val >> 8 * 2),
+            (byte) (val >> 8 * 3)
+        };
+    }
+
+
+    /**
      * Read an unsigned integer stored in little-endian format from a byte array
      * at a given offset.
      *

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java
index b7160a1..55d7608 100644
--- a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java
@@ -195,7 +195,6 @@ public class MetadataTest {
             new HashSet<>(Arrays.asList("topic", "topic1")), topics);
     }
 
-
     private Thread asyncFetch(final String topic) {
         Thread thread = new Thread() {
             public void run() {

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/test/java/org/apache/kafka/clients/consumer/RangeAssignorTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/RangeAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/RangeAssignorTest.java
new file mode 100644
index 0000000..13cce13
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/RangeAssignorTest.java
@@ -0,0 +1,217 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class RangeAssignorTest {
+
+    private RangeAssignor assignor = new RangeAssignor();
+
+
+    @Test
+    public void testOneConsumerNoTopic() {
+        String consumerId = "consumer";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic,
+                Collections.singletonMap(consumerId, Collections.<String>emptyList()));
+
+        assertEquals(Collections.singleton(consumerId), assignment.keySet());
+        assertTrue(assignment.get(consumerId).isEmpty());
+    }
+
+    @Test
+    public void testOneConsumerNonexistentTopic() {
+        String topic = "topic";
+        String consumerId = "consumer";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        partitionsPerTopic.put(topic, 0);
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic,
+                Collections.singletonMap(consumerId, Arrays.asList(topic)));
+        assertEquals(Collections.singleton(consumerId), assignment.keySet());
+        assertTrue(assignment.get(consumerId).isEmpty());
+    }
+
+    @Test
+    public void testOneConsumerOneTopic() {
+        String topic = "topic";
+        String consumerId = "consumer";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        partitionsPerTopic.put(topic, 3);
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic,
+                Collections.singletonMap(consumerId, Arrays.asList(topic)));
+
+        assertEquals(Collections.singleton(consumerId), assignment.keySet());
+        assertAssignment(Arrays.asList(
+                new TopicPartition(topic, 0),
+                new TopicPartition(topic, 1),
+                new TopicPartition(topic, 2)), assignment.get(consumerId));
+    }
+
+    @Test
+    public void testOnlyAssignsPartitionsFromSubscribedTopics() {
+        String topic = "topic";
+        String otherTopic = "other";
+        String consumerId = "consumer";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        partitionsPerTopic.put(topic, 3);
+        partitionsPerTopic.put(otherTopic, 3);
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic,
+                Collections.singletonMap(consumerId, Arrays.asList(topic)));
+        assertEquals(Collections.singleton(consumerId), assignment.keySet());
+        assertAssignment(Arrays.asList(
+                new TopicPartition(topic, 0),
+                new TopicPartition(topic, 1),
+                new TopicPartition(topic, 2)), assignment.get(consumerId));
+    }
+
+    @Test
+    public void testOneConsumerMultipleTopics() {
+        String topic1 = "topic1";
+        String topic2 = "topic2";
+        String consumerId = "consumer";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        partitionsPerTopic.put(topic1, 1);
+        partitionsPerTopic.put(topic2, 2);
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic,
+                Collections.singletonMap(consumerId, Arrays.asList(topic1, topic2)));
+
+        assertEquals(Collections.singleton(consumerId), assignment.keySet());
+        assertAssignment(Arrays.asList(
+                new TopicPartition(topic1, 0),
+                new TopicPartition(topic2, 0),
+                new TopicPartition(topic2, 1)), assignment.get(consumerId));
+    }
+
+    @Test
+    public void testTwoConsumersOneTopicOnePartition() {
+        String topic = "topic";
+        String consumer1 = "consumer1";
+        String consumer2 = "consumer2";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        partitionsPerTopic.put(topic, 1);
+
+        Map<String, List<String>> consumers = new HashMap<>();
+        consumers.put(consumer1, Arrays.asList(topic));
+        consumers.put(consumer2, Arrays.asList(topic));
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, consumers);
+        assertAssignment(Arrays.asList(new TopicPartition(topic, 0)), assignment.get(consumer1));
+        assertAssignment(Collections.<TopicPartition>emptyList(), assignment.get(consumer2));
+    }
+
+
+    @Test
+    public void testTwoConsumersOneTopicTwoPartitions() {
+        String topic = "topic";
+        String consumer1 = "consumer1";
+        String consumer2 = "consumer2";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        partitionsPerTopic.put(topic, 2);
+
+        Map<String, List<String>> consumers = new HashMap<>();
+        consumers.put(consumer1, Arrays.asList(topic));
+        consumers.put(consumer2, Arrays.asList(topic));
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, consumers);
+        assertAssignment(Arrays.asList(new TopicPartition(topic, 0)), assignment.get(consumer1));
+        assertAssignment(Arrays.asList(new TopicPartition(topic, 1)), assignment.get(consumer2));
+    }
+
+    @Test
+    public void testMultipleConsumersMixedTopics() {
+        String topic1 = "topic1";
+        String topic2 = "topic2";
+        String consumer1 = "consumer1";
+        String consumer2 = "consumer2";
+        String consumer3 = "consumer3";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        partitionsPerTopic.put(topic1, 3);
+        partitionsPerTopic.put(topic2, 2);
+
+        Map<String, List<String>> consumers = new HashMap<>();
+        consumers.put(consumer1, Arrays.asList(topic1));
+        consumers.put(consumer2, Arrays.asList(topic1, topic2));
+        consumers.put(consumer3, Arrays.asList(topic1));
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, consumers);
+        assertAssignment(Arrays.asList(
+                new TopicPartition(topic1, 0)), assignment.get(consumer1));
+        assertAssignment(Arrays.asList(
+                new TopicPartition(topic1, 1),
+                new TopicPartition(topic2, 0),
+                new TopicPartition(topic2, 1)), assignment.get(consumer2));
+        assertAssignment(Arrays.asList(
+                new TopicPartition(topic1, 2)), assignment.get(consumer3));
+    }
+
+    @Test
+    public void testTwoConsumersTwoTopicsSixPartitions() {
+        String topic1 = "topic1";
+        String topic2 = "topic2";
+        String consumer1 = "consumer1";
+        String consumer2 = "consumer2";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        partitionsPerTopic.put(topic1, 3);
+        partitionsPerTopic.put(topic2, 3);
+
+        Map<String, List<String>> consumers = new HashMap<>();
+        consumers.put(consumer1, Arrays.asList(topic1, topic2));
+        consumers.put(consumer2, Arrays.asList(topic1, topic2));
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, consumers);
+        assertAssignment(Arrays.asList(
+                new TopicPartition(topic1, 0),
+                new TopicPartition(topic1, 1),
+                new TopicPartition(topic2, 0),
+                new TopicPartition(topic2, 1)), assignment.get(consumer1));
+        assertAssignment(Arrays.asList(
+                new TopicPartition(topic1, 2),
+                new TopicPartition(topic2, 2)), assignment.get(consumer2));
+    }
+
+    private void assertAssignment(List<TopicPartition> expected, List<TopicPartition> actual) {
+        // order doesn't matter for assignment, so convert to a set
+        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/test/java/org/apache/kafka/clients/consumer/RoundRobinAssignorTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/RoundRobinAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/RoundRobinAssignorTest.java
new file mode 100644
index 0000000..31598cd
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/RoundRobinAssignorTest.java
@@ -0,0 +1,209 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class RoundRobinAssignorTest {
+
+    private RoundRobinAssignor assignor = new RoundRobinAssignor();
+
+
+    @Test
+    public void testOneConsumerNoTopic() {
+        String consumerId = "consumer";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic,
+                Collections.singletonMap(consumerId, Collections.<String>emptyList()));
+        assertEquals(Collections.singleton(consumerId), assignment.keySet());
+        assertTrue(assignment.get(consumerId).isEmpty());
+    }
+
+    @Test
+    public void testOneConsumerNonexistentTopic() {
+        String topic = "topic";
+        String consumerId = "consumer";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        partitionsPerTopic.put(topic, 0);
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic,
+                Collections.singletonMap(consumerId, Arrays.asList(topic)));
+
+        assertEquals(Collections.singleton(consumerId), assignment.keySet());
+        assertTrue(assignment.get(consumerId).isEmpty());
+    }
+
+    @Test
+    public void testOneConsumerOneTopic() {
+        String topic = "topic";
+        String consumerId = "consumer";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        partitionsPerTopic.put(topic, 3);
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic,
+                Collections.singletonMap(consumerId, Arrays.asList(topic)));
+        assertEquals(Arrays.asList(
+                new TopicPartition(topic, 0),
+                new TopicPartition(topic, 1),
+                new TopicPartition(topic, 2)), assignment.get(consumerId));
+    }
+
+    @Test
+    public void testOnlyAssignsPartitionsFromSubscribedTopics() {
+        String topic = "topic";
+        String otherTopic = "other";
+        String consumerId = "consumer";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        partitionsPerTopic.put(topic, 3);
+        partitionsPerTopic.put(otherTopic, 3);
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic,
+                Collections.singletonMap(consumerId, Arrays.asList(topic)));
+        assertEquals(Arrays.asList(
+                new TopicPartition(topic, 0),
+                new TopicPartition(topic, 1),
+                new TopicPartition(topic, 2)), assignment.get(consumerId));
+    }
+
+    @Test
+    public void testOneConsumerMultipleTopics() {
+        String topic1 = "topic1";
+        String topic2 = "topic2";
+        String consumerId = "consumer";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        partitionsPerTopic.put(topic1, 1);
+        partitionsPerTopic.put(topic2, 2);
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic,
+                Collections.singletonMap(consumerId, Arrays.asList(topic1, topic2)));
+        assertEquals(Arrays.asList(
+                new TopicPartition(topic1, 0),
+                new TopicPartition(topic2, 0),
+                new TopicPartition(topic2, 1)), assignment.get(consumerId));
+    }
+
+    @Test
+    public void testTwoConsumersOneTopicOnePartition() {
+        String topic = "topic";
+        String consumer1 = "consumer1";
+        String consumer2 = "consumer2";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        partitionsPerTopic.put(topic, 1);
+
+        Map<String, List<String>> consumers = new HashMap<>();
+        consumers.put(consumer1, Arrays.asList(topic));
+        consumers.put(consumer2, Arrays.asList(topic));
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, consumers);
+        assertEquals(Arrays.asList(new TopicPartition(topic, 0)), assignment.get(consumer1));
+        assertEquals(Collections.<TopicPartition>emptyList(), assignment.get(consumer2));
+    }
+
+    @Test
+    public void testTwoConsumersOneTopicTwoPartitions() {
+        String topic = "topic";
+        String consumer1 = "consumer1";
+        String consumer2 = "consumer2";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        partitionsPerTopic.put(topic, 2);
+
+        Map<String, List<String>> consumers = new HashMap<>();
+        consumers.put(consumer1, Arrays.asList(topic));
+        consumers.put(consumer2, Arrays.asList(topic));
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, consumers);
+        assertEquals(Arrays.asList(new TopicPartition(topic, 0)), assignment.get(consumer1));
+        assertEquals(Arrays.asList(new TopicPartition(topic, 1)), assignment.get(consumer2));
+    }
+
+    @Test
+    public void testMultipleConsumersMixedTopics() {
+        String topic1 = "topic1";
+        String topic2 = "topic2";
+        String consumer1 = "consumer1";
+        String consumer2 = "consumer2";
+        String consumer3 = "consumer3";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        partitionsPerTopic.put(topic1, 3);
+        partitionsPerTopic.put(topic2, 2);
+
+        Map<String, List<String>> consumers = new HashMap<>();
+        consumers.put(consumer1, Arrays.asList(topic1));
+        consumers.put(consumer2, Arrays.asList(topic1, topic2));
+        consumers.put(consumer3, Arrays.asList(topic1));
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, consumers);
+        assertEquals(Arrays.asList(
+                new TopicPartition(topic1, 0)), assignment.get(consumer1));
+        assertEquals(Arrays.asList(
+                new TopicPartition(topic1, 1),
+                new TopicPartition(topic2, 0),
+                new TopicPartition(topic2, 1)), assignment.get(consumer2));
+        assertEquals(Arrays.asList(
+                new TopicPartition(topic1, 2)), assignment.get(consumer3));
+    }
+
+    @Test
+    public void testTwoConsumersTwoTopicsSixPartitions() {
+        String topic1 = "topic1";
+        String topic2 = "topic2";
+        String consumer1 = "consumer1";
+        String consumer2 = "consumer2";
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        partitionsPerTopic.put(topic1, 3);
+        partitionsPerTopic.put(topic2, 3);
+
+        Map<String, List<String>> consumers = new HashMap<>();
+        consumers.put(consumer1, Arrays.asList(topic1, topic2));
+        consumers.put(consumer2, Arrays.asList(topic1, topic2));
+
+        Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, consumers);
+        assertEquals(Arrays.asList(
+                new TopicPartition(topic1, 0),
+                new TopicPartition(topic1, 2),
+                new TopicPartition(topic2, 1)), assignment.get(consumer1));
+        assertEquals(Arrays.asList(
+                new TopicPartition(topic1, 1),
+                new TopicPartition(topic2, 0),
+                new TopicPartition(topic2, 2)), assignment.get(consumer2));
+    }
+
+    public static List<String> topics(String... topics) {
+        return Arrays.asList(topics);
+    }
+
+    public static TopicPartition tp(String topic, int partition) {
+        return new TopicPartition(topic, partition);
+    }
+
+}


[8/8] kafka git commit: KAFKA-2464: client-side assignment for new consumer

Posted by gu...@apache.org.
KAFKA-2464: client-side assignment for new consumer

Author: Jason Gustafson <ja...@confluent.io>

Reviewers: Jiangjie Qin, Onur Karaman, Ewen Cheslack-Postava, Guozhang Wang

Closes #165 from hachikuji/KAFKA-2464


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

Branch: refs/heads/trunk
Commit: 86eb74d9236c586af5889fe79f4b9e066c9c2af3
Parents: 6e747d4
Author: Jason Gustafson <ja...@confluent.io>
Authored: Wed Oct 21 12:13:42 2015 -0700
Committer: Guozhang Wang <wa...@gmail.com>
Committed: Wed Oct 21 12:13:42 2015 -0700

----------------------------------------------------------------------
 .../kafka/clients/consumer/ConsumerConfig.java  |   9 +-
 .../kafka/clients/consumer/KafkaConsumer.java   |  47 +-
 .../kafka/clients/consumer/RangeAssignor.java   |  97 ++
 .../clients/consumer/RoundRobinAssignor.java    | 114 +++
 .../consumer/internals/AbstractCoordinator.java | 638 +++++++++++++
 .../internals/AbstractPartitionAssignor.java    |  90 ++
 .../consumer/internals/ConsumerCoordinator.java | 595 ++++++++++++
 .../internals/ConsumerNetworkClient.java        |   9 +
 .../consumer/internals/ConsumerProtocol.java    | 162 ++++
 .../clients/consumer/internals/Coordinator.java | 848 -----------------
 .../clients/consumer/internals/Fetcher.java     |  31 +-
 .../consumer/internals/PartitionAssignor.java   | 117 +++
 .../consumer/internals/RequestFuture.java       |  16 +-
 .../consumer/internals/SubscriptionState.java   |  44 +-
 .../java/org/apache/kafka/common/Cluster.java   |  10 +
 ...onsumerCoordinatorNotAvailableException.java |  40 -
 .../GroupCoordinatorNotAvailableException.java  |  40 +
 .../NotCoordinatorForConsumerException.java     |  40 -
 .../errors/NotCoordinatorForGroupException.java |  40 +
 .../errors/UnknownConsumerIdException.java      |  33 -
 .../common/errors/UnknownMemberIdException.java |  33 +
 .../apache/kafka/common/protocol/ApiKeys.java   |   5 +-
 .../apache/kafka/common/protocol/Errors.java    |  22 +-
 .../apache/kafka/common/protocol/Protocol.java  |  97 +-
 .../kafka/common/requests/AbstractRequest.java  |   6 +-
 .../requests/ConsumerMetadataRequest.java       |  65 --
 .../requests/ConsumerMetadataResponse.java      |  70 --
 .../common/requests/GroupMetadataRequest.java   |  65 ++
 .../common/requests/GroupMetadataResponse.java  |  70 ++
 .../kafka/common/requests/HeartbeatRequest.java |  16 +-
 .../common/requests/HeartbeatResponse.java      |   6 +-
 .../kafka/common/requests/JoinGroupRequest.java |  96 +-
 .../common/requests/JoinGroupResponse.java      | 110 ++-
 .../common/requests/OffsetCommitRequest.java    |  34 +-
 .../common/requests/OffsetCommitResponse.java   |   6 +-
 .../common/requests/OffsetFetchResponse.java    |   4 +-
 .../kafka/common/requests/SyncGroupRequest.java | 118 +++
 .../common/requests/SyncGroupResponse.java      |  71 ++
 .../org/apache/kafka/common/utils/Utils.java    |  28 +
 .../org/apache/kafka/clients/MetadataTest.java  |   1 -
 .../clients/consumer/RangeAssignorTest.java     | 217 +++++
 .../consumer/RoundRobinAssignorTest.java        | 209 +++++
 .../internals/ConsumerCoordinatorTest.java      | 749 +++++++++++++++
 .../internals/ConsumerNetworkClientTest.java    |   2 +-
 .../internals/ConsumerProtocolTest.java         | 118 +++
 .../consumer/internals/CoordinatorTest.java     | 635 -------------
 .../clients/consumer/internals/FetcherTest.java |   4 +-
 .../internals/MockPartitionAssignor.java        |  49 +
 .../common/requests/RequestResponseTest.java    |  16 +-
 .../kafka/copycat/util/KafkaBasedLogTest.java   |   2 +-
 .../src/main/scala/kafka/admin/AclCommand.scala |  22 +-
 .../main/scala/kafka/admin/TopicCommand.scala   |   7 +-
 .../kafka/api/ConsumerMetadataRequest.scala     |  80 --
 .../kafka/api/ConsumerMetadataResponse.scala    |  58 --
 .../scala/kafka/api/GroupMetadataRequest.scala  |  80 ++
 .../scala/kafka/api/GroupMetadataResponse.scala |  58 ++
 .../scala/kafka/api/OffsetCommitRequest.scala   |  16 +-
 core/src/main/scala/kafka/api/RequestKeys.scala |   5 +-
 .../main/scala/kafka/client/ClientUtils.scala   |   4 +-
 .../kafka/common/OffsetMetadataAndError.scala   |   5 +-
 core/src/main/scala/kafka/common/Topic.scala    |   4 +-
 .../scala/kafka/consumer/SimpleConsumer.scala   |   4 +-
 .../kafka/coordinator/ConsumerCoordinator.scala | 535 -----------
 .../coordinator/ConsumerGroupMetadata.scala     | 133 ---
 .../kafka/coordinator/ConsumerMetadata.scala    |  50 -
 .../kafka/coordinator/CoordinatorMetadata.scala | 160 +---
 .../kafka/coordinator/DelayedHeartbeat.scala    |  12 +-
 .../scala/kafka/coordinator/DelayedJoin.scala   |  40 +
 .../kafka/coordinator/DelayedRebalance.scala    |  40 -
 .../kafka/coordinator/GroupCoordinator.scala    | 632 +++++++++++++
 .../scala/kafka/coordinator/GroupMetadata.scala | 209 +++++
 .../kafka/coordinator/MemberMetadata.scala      |  99 ++
 .../kafka/coordinator/PartitionAssignor.scala   | 125 ---
 .../javaapi/ConsumerMetadataResponse.scala      |  47 -
 .../kafka/javaapi/GroupMetadataResponse.scala   |  47 +
 .../kafka/security/auth/ResourceType.scala      |   6 +-
 .../src/main/scala/kafka/server/KafkaApis.scala | 144 +--
 .../main/scala/kafka/server/KafkaConfig.scala   |  16 +-
 .../main/scala/kafka/server/KafkaServer.scala   |   6 +-
 .../main/scala/kafka/server/OffsetManager.scala |  16 +-
 .../kafka/api/BaseConsumerTest.scala            |  25 +-
 .../kafka/api/ConsumerBounceTest.scala          |  10 +-
 .../kafka/api/IntegrationTestHarness.scala      |  12 +-
 .../integration/kafka/api/QuotasTest.scala      |   1 -
 .../integration/kafka/api/SslConsumerTest.scala |  22 -
 .../scala/other/kafka/TestOffsetManager.scala   |   4 +-
 .../scala/unit/kafka/admin/AclCommandTest.scala |   8 +-
 .../unit/kafka/admin/TopicCommandTest.scala     |   8 +-
 .../api/RequestResponseSerializationTest.scala  |  10 +-
 .../unit/kafka/consumer/TopicFilterTest.scala   |  10 +-
 .../ConsumerCoordinatorResponseTest.scala       | 447 ---------
 .../coordinator/ConsumerGroupMetadataTest.scala | 172 ----
 .../coordinator/CoordinatorMetadataTest.scala   | 160 +---
 .../GroupCoordinatorResponseTest.scala          | 907 +++++++++++++++++++
 .../kafka/coordinator/GroupMetadataTest.scala   | 249 +++++
 .../kafka/coordinator/MemberMetadataTest.scala  |  90 ++
 .../coordinator/PartitionAssignorTest.scala     | 305 -------
 .../security/auth/SimpleAclAuthorizerTest.scala |   4 +-
 .../unit/kafka/server/KafkaConfigTest.scala     |   4 +-
 .../unit/kafka/server/OffsetCommitTest.scala    |   4 +-
 .../test/scala/unit/kafka/utils/TestUtils.scala |   6 +-
 101 files changed, 6634 insertions(+), 4428 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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 1894822..5cc0419 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
@@ -17,9 +17,9 @@ import org.apache.kafka.common.config.AbstractConfig;
 import org.apache.kafka.common.config.ConfigDef;
 import org.apache.kafka.common.config.ConfigDef.Importance;
 import org.apache.kafka.common.config.ConfigDef.Type;
-import org.apache.kafka.common.serialization.Deserializer;
 import org.apache.kafka.common.config.SSLConfigs;
 import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.serialization.Deserializer;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -78,7 +78,7 @@ public class ConsumerConfig extends AbstractConfig {
      * <code>partition.assignment.strategy</code>
      */
     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";
+    private static final String PARTITION_ASSIGNMENT_STRATEGY_DOC = "The class name of the partition assignment strategy that the client will use to distribute partition ownership amongst consumer instances when group management is used";
 
     /**
      * <code>auto.offset.reset</code>
@@ -182,9 +182,8 @@ public class ConsumerConfig extends AbstractConfig {
                                         Importance.HIGH,
                                         HEARTBEAT_INTERVAL_MS_DOC)
                                 .define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
-                                        Type.STRING,
-                                        "range",
-                                        in("range", "roundrobin"),
+                                        Type.LIST,
+                                        RangeAssignor.class.getName(),
                                         Importance.MEDIUM,
                                         PARTITION_ASSIGNMENT_STRATEGY_DOC)
                                 .define(METADATA_MAX_AGE_CONFIG,

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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 2f7f153..cd166f0 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
@@ -16,9 +16,10 @@ import org.apache.kafka.clients.ClientUtils;
 import org.apache.kafka.clients.Metadata;
 import org.apache.kafka.clients.NetworkClient;
 import org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient;
-import org.apache.kafka.clients.consumer.internals.Coordinator;
+import org.apache.kafka.clients.consumer.internals.ConsumerCoordinator;
 import org.apache.kafka.clients.consumer.internals.Fetcher;
 import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.internals.PartitionAssignor;
 import org.apache.kafka.clients.consumer.internals.SubscriptionState;
 import org.apache.kafka.common.Cluster;
 import org.apache.kafka.common.KafkaException;
@@ -43,7 +44,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.net.InetSocketAddress;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -403,7 +403,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
     private static final String JMX_PREFIX = "kafka.consumer";
 
     private String clientId;
-    private final Coordinator coordinator;
+    private final ConsumerCoordinator coordinator;
     private final Deserializer<K> keyDeserializer;
     private final Deserializer<V> valueDeserializer;
     private final Fetcher<K, V> fetcher;
@@ -416,7 +416,6 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
     private final long retryBackoffMs;
     private long requestTimeoutMs;
     private boolean closed = false;
-    private Metadata.Listener metadataListener;
 
     // currentThread holds the threadId of the current thread accessing KafkaConsumer
     // and is used to prevent multi-threaded access
@@ -531,11 +530,15 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
             this.client = new ConsumerNetworkClient(netClient, metadata, time, retryBackoffMs);
             OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase());
             this.subscriptions = new SubscriptionState(offsetResetStrategy);
-            this.coordinator = new Coordinator(this.client,
+            List<PartitionAssignor> assignors = config.getConfiguredInstances(
+                    ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG,
+                    PartitionAssignor.class);
+            this.coordinator = new ConsumerCoordinator(this.client,
                     config.getString(ConsumerConfig.GROUP_ID_CONFIG),
                     config.getInt(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG),
                     config.getInt(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG),
-                    config.getString(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG),
+                    assignors,
+                    this.metadata,
                     this.subscriptions,
                     metrics,
                     metricGrpPrefix,
@@ -543,7 +546,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
                     this.time,
                     requestTimeoutMs,
                     retryBackoffMs,
-                    new Coordinator.DefaultOffsetCommitCallback(),
+                    new ConsumerCoordinator.DefaultOffsetCommitCallback(),
                     config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG),
                     config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG));
             if (keyDeserializer == null) {
@@ -652,7 +655,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
         try {
             log.debug("Subscribed to topic(s): {}", Utils.join(topics, ", "));
             this.subscriptions.subscribe(topics, listener);
-            metadata.setTopics(topics);
+            metadata.setTopics(subscriptions.groupSubscription());
         } finally {
             release();
         }
@@ -699,22 +702,8 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
         acquire();
         try {
             log.debug("Subscribed to pattern: {}", pattern);
-            metadataListener = new Metadata.Listener() {
-                @Override
-                public void onMetadataUpdate(Cluster cluster) {
-                    final List<String> topicsToSubscribe = new ArrayList<>();
-
-                    for (String topic : cluster.topics())
-                        if (subscriptions.getSubscribedPattern().matcher(topic).matches())
-                            topicsToSubscribe.add(topic);
-
-                    subscriptions.changeSubscription(topicsToSubscribe);
-                    metadata.setTopics(topicsToSubscribe);
-                }
-            };
             this.subscriptions.subscribe(pattern, listener);
             this.metadata.needMetadataForAllTopics(true);
-            this.metadata.addListener(metadataListener);
         } finally {
             release();
         }
@@ -729,7 +718,6 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
             this.subscriptions.unsubscribe();
             this.coordinator.resetGeneration();
             this.metadata.needMetadataForAllTopics(false);
-            this.metadata.removeListener(metadataListener);
         } finally {
             release();
         }
@@ -1079,12 +1067,11 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
         try {
             Cluster cluster = this.metadata.fetch();
             List<PartitionInfo> parts = cluster.partitionsForTopic(topic);
-            if (parts == null) {
-                metadata.add(topic);
-                client.awaitMetadataUpdate();
-                parts = metadata.fetch().partitionsForTopic(topic);
-            }
-            return parts;
+            if (parts != null)
+                return parts;
+
+            Map<String, List<PartitionInfo>> topicMetadata = fetcher.getTopicMetadata(Collections.singletonList(topic), requestTimeoutMs);
+            return topicMetadata.get(topic);
         } finally {
             release();
         }
@@ -1101,7 +1088,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
     public Map<String, List<PartitionInfo>> listTopics() {
         acquire();
         try {
-            return fetcher.getAllTopics(requestTimeoutMs);
+            return fetcher.getAllTopicMetadata(requestTimeoutMs);
         } finally {
             release();
         }

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java
new file mode 100644
index 0000000..f23151c
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/RangeAssignor.java
@@ -0,0 +1,97 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.clients.consumer.internals.AbstractPartitionAssignor;
+import org.apache.kafka.common.TopicPartition;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * The range assignor works on a per-topic basis. For each topic, we lay out the available partitions in numeric order
+ * and the consumers in lexicographic order. We then divide the number of partitions by the total number of
+ * consumers to determine the number of partitions to assign to each consumer. If it does not evenly
+ * divide, then the first few consumers will have one extra partition.
+ *
+ * For example, suppose there are two consumers C0 and C1, two topics t0 and t1, and each topic has 3 partitions,
+ * resulting in partitions t0p0, t0p1, t0p2, t1p0, t1p1, and t1p2.
+ *
+ * The assignment will be:
+ * C0: [t0p0, t0p1, t1p0, t1p1]
+ * C1: [t0p2, t1p2]
+ */
+public class RangeAssignor extends AbstractPartitionAssignor {
+
+    @Override
+    public String name() {
+        return "range";
+    }
+
+    private List<TopicPartition> partitions(String topic,
+                                            int numPartitions) {
+        List<TopicPartition> partitions = new ArrayList<>();
+        for (int i = 0; i < numPartitions; i++)
+            partitions.add(new TopicPartition(topic, i));
+        return partitions;
+    }
+
+    private Map<String, List<String>> consumersPerTopic(Map<String, List<String>> consumerMetadata) {
+        Map<String, List<String>> res = new HashMap<>();
+        for (Map.Entry<String, List<String>> subscriptionEntry : consumerMetadata.entrySet()) {
+            String consumerId = subscriptionEntry.getKey();
+            for (String topic : subscriptionEntry.getValue())
+                put(res, topic, consumerId);
+        }
+        return res;
+    }
+
+    @Override
+    public Map<String, List<TopicPartition>> assign(Map<String, Integer> partitionsPerTopic,
+                                                    Map<String, List<String>> subscriptions) {
+        Map<String, List<String>> consumersPerTopic = consumersPerTopic(subscriptions);
+        Map<String, List<TopicPartition>> assignment = new HashMap<>();
+        for (String memberId : subscriptions.keySet())
+            assignment.put(memberId, new ArrayList<TopicPartition>());
+
+        for (Map.Entry<String, List<String>> topicEntry : consumersPerTopic.entrySet()) {
+            String topic = topicEntry.getKey();
+            List<String> consumersForTopic = topicEntry.getValue();
+
+            Integer numPartitionsForTopic = partitionsPerTopic.get(topic);
+            if (numPartitionsForTopic == null)
+                continue;
+
+            Collections.sort(consumersForTopic);
+
+            int numPartitionsPerConsumer = numPartitionsForTopic / consumersForTopic.size();
+            int consumersWithExtraPartition = numPartitionsForTopic % consumersForTopic.size();
+
+            List<TopicPartition> partitions = partitions(topic, numPartitionsForTopic);
+            for (int i = 0, n = consumersForTopic.size(); i < n; i++) {
+                int start = numPartitionsPerConsumer * i + Math.min(i, consumersWithExtraPartition);
+                int length = numPartitionsPerConsumer + (i + 1 > consumersWithExtraPartition ? 0 : 1);
+                assignment.get(consumersForTopic.get(i)).addAll(partitions.subList(start, start + length));
+            }
+        }
+        return assignment;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java
new file mode 100644
index 0000000..c5ea2bb
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/RoundRobinAssignor.java
@@ -0,0 +1,114 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.clients.consumer.internals.AbstractPartitionAssignor;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.Utils;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.TreeSet;
+
+/**
+ * The roundrobin assignor lays out all the available partitions and all the available consumers. It
+ * then proceeds to do a roundrobin assignment from partition to consumer. If the subscriptions of all consumer
+ * instances are identical, then the partitions will be uniformly distributed. (i.e., the partition ownership counts
+ * will be within a delta of exactly one across all consumers.)
+ *
+ * For example, suppose there are two consumers C0 and C1, two topics t0 and t1, and each topic has 3 partitions,
+ * resulting in partitions t0p0, t0p1, t0p2, t1p0, t1p1, and t1p2.
+ *
+ * The assignment will be:
+ * C0: [t0p0, t0p2, t1p1]
+ * C1: [t0p1, t1p0, t1p2]
+ */
+public class RoundRobinAssignor extends AbstractPartitionAssignor {
+
+    @Override
+    public Map<String, List<TopicPartition>> assign(Map<String, Integer> partitionsPerTopic,
+                                                    Map<String, List<String>> subscriptions) {
+        Map<String, List<TopicPartition>> assignment = new HashMap<>();
+        for (String memberId : subscriptions.keySet())
+            assignment.put(memberId, new ArrayList<TopicPartition>());
+
+        CircularIterator<String> assigner = new CircularIterator<>(Utils.sorted(subscriptions.keySet()));
+        for (TopicPartition partition : allPartitionsSorted(partitionsPerTopic, subscriptions)) {
+            final String topic = partition.topic();
+            while (!subscriptions.get(assigner.peek()).contains(topic))
+                assigner.next();
+            assignment.get(assigner.next()).add(partition);
+        }
+        return assignment;
+    }
+
+
+    public List<TopicPartition> allPartitionsSorted(Map<String, Integer> partitionsPerTopic,
+                                                    Map<String, List<String>> subscriptions) {
+        SortedSet<String> topics = new TreeSet<>();
+        for (List<String> subscription : subscriptions.values())
+            topics.addAll(subscription);
+
+        List<TopicPartition> allPartitions = new ArrayList<>();
+        for (String topic : topics) {
+            Integer partitions = partitionsPerTopic.get(topic);
+            for (int partition = 0; partition < partitions; partition++) {
+                allPartitions.add(new TopicPartition(topic, partition));
+            }
+        }
+        return allPartitions;
+    }
+
+    @Override
+    public String name() {
+        return "roundrobin";
+    }
+
+    private static class CircularIterator<T> implements Iterator<T> {
+        int i = 0;
+        private List<T> list;
+
+        public CircularIterator(List<T> list) {
+            if (list.isEmpty()) {
+                throw new IllegalArgumentException("CircularIterator can only be used on non-empty lists");
+            }
+            this.list = list;
+        }
+
+        @Override
+        public boolean hasNext() {
+            return true;
+        }
+
+        @Override
+        public T next() {
+            T next = list.get(i);
+            i = (i + 1) % list.size();
+            return next;
+        }
+
+        public T peek() {
+            return list.get(i);
+        }
+
+        @Override
+        public void remove() {
+            throw new UnsupportedOperationException();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
new file mode 100644
index 0000000..1ffd2bb
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java
@@ -0,0 +1,638 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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 org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.errors.DisconnectException;
+import org.apache.kafka.common.errors.UnknownMemberIdException;
+import org.apache.kafka.common.metrics.Measurable;
+import org.apache.kafka.common.metrics.MetricConfig;
+import org.apache.kafka.common.metrics.Metrics;
+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.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.requests.GroupMetadataRequest;
+import org.apache.kafka.common.requests.GroupMetadataResponse;
+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.OffsetCommitRequest;
+import org.apache.kafka.common.requests.SyncGroupRequest;
+import org.apache.kafka.common.requests.SyncGroupResponse;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * AbstractCoordinator implements group management for a single group member by interacting with
+ * a designated Kafka broker (the coordinator). Group semantics are provided by extending this class.
+ * See {@link ConsumerCoordinator} for example usage.
+ *
+ * From a high level, Kafka's group management protocol consists of the following sequence of actions:
+ *
+ * <ol>
+ *     <li>Group Registration: Group members register with the coordinator providing their own metadata
+ *         (such as the set of topics they are interested in).</li>
+ *     <li>Group/Leader Selection: The coordinator select the members of the group and chooses one member
+ *         as the leader.</li>
+ *     <li>State Assignment: The leader collects the metadata from all the members of the group and
+ *         assigns state.</li>
+ *     <li>Group Stabilization: Each member receives the state assigned by the leader and begins
+ *         processing.</li>
+ * </ol>
+ *
+ * To leverage this protocol, an implementation must define the format of metadata provided by each
+ * member for group registration in {@link #metadata()} and the format of the state assignment provided
+ * by the leader in {@link #doSync(String, String, Map)} and becomes available to members in
+ * {@link #onJoin(int, String, String, ByteBuffer)}.
+ *
+ */
+public abstract class AbstractCoordinator {
+
+    private static final Logger log = LoggerFactory.getLogger(AbstractCoordinator.class);
+
+    private final Heartbeat heartbeat;
+    private final HeartbeatTask heartbeatTask;
+    private final int sessionTimeoutMs;
+    private final GroupCoordinatorMetrics sensors;
+    protected final String groupId;
+    protected final ConsumerNetworkClient client;
+    protected final Time time;
+    protected final long retryBackoffMs;
+    protected final long requestTimeoutMs;
+
+    private boolean rejoinNeeded = true;
+    protected Node coordinator;
+    protected String memberId;
+    protected String protocol;
+    protected int generation;
+
+    /**
+     * Initialize the coordination manager.
+     */
+    public AbstractCoordinator(ConsumerNetworkClient client,
+                               String groupId,
+                               int sessionTimeoutMs,
+                               int heartbeatIntervalMs,
+                               Metrics metrics,
+                               String metricGrpPrefix,
+                               Map<String, String> metricTags,
+                               Time time,
+                               long requestTimeoutMs,
+                               long retryBackoffMs) {
+        this.client = client;
+        this.time = time;
+        this.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID;
+        this.memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID;
+        this.groupId = groupId;
+        this.coordinator = null;
+        this.sessionTimeoutMs = sessionTimeoutMs;
+        this.heartbeat = new Heartbeat(this.sessionTimeoutMs, heartbeatIntervalMs, time.milliseconds());
+        this.heartbeatTask = new HeartbeatTask();
+        this.sensors = new GroupCoordinatorMetrics(metrics, metricGrpPrefix, metricTags);
+        this.requestTimeoutMs = requestTimeoutMs;
+        this.retryBackoffMs = retryBackoffMs;
+    }
+
+    /**
+     * Unique identifier for the class of protocols implements (e.g. "consumer" or "copycat").
+     * @return Non-null protocol type namej
+     */
+    protected abstract String protocolType();
+
+    /**
+     * Get the current list of protocols and their associated metadata supported
+     * by the local member. The order of the protocols in the map indicates the preference
+     * of the protocol (the first entry is the most preferred). The coordinator takes this
+     * preference into account when selecting the generation protocol (generally more preferred
+     * protocols will be selected as long as all members support them and there is no disagreement
+     * on the preference).
+     * @return Non-empty map of supported protocols and metadata
+     */
+    protected abstract LinkedHashMap<String, ByteBuffer> metadata();
+
+    /**
+     * Invoked when a group member has successfully joined a group.
+     * @param generation The generation that was joined
+     * @param memberId The identifier for the local member in the group
+     * @param protocol The protocol selected by the coordinator
+     * @param memberAssignment The assignment propagated from the group leader
+     */
+    protected abstract void onJoin(int generation,
+                                   String memberId,
+                                   String protocol,
+                                   ByteBuffer memberAssignment);
+
+    /**
+     * Perform synchronization for the group. This is used by the leader to push state to all the members
+     * of the group (e.g. to push partition assignments in the case of the new consumer)
+     * @param leaderId The id of the leader (which is this member)
+     * @param allMemberMetadata Metadata from all members of the group
+     * @return A map from each member to their state assignment
+     */
+    protected abstract Map<String, ByteBuffer> doSync(String leaderId,
+                                                      String protocol,
+                                                      Map<String, ByteBuffer> allMemberMetadata);
+
+    /**
+     * Invoked when the group is left (whether because of shutdown, metadata change, stale generation, etc.)
+     * @param generation The generation that was left
+     * @param memberId The identifier of the local member in the group
+     */
+    protected abstract void onLeave(int generation, String memberId);
+
+
+    /**
+     * Block until the coordinator for this group is known.
+     */
+    public void ensureCoordinatorKnown() {
+        while (coordinatorUnknown()) {
+            RequestFuture<Void> future = sendGroupMetadataRequest();
+            client.poll(future, requestTimeoutMs);
+
+            if (future.failed())
+                client.awaitMetadataUpdate();
+        }
+    }
+
+    /**
+     * Check whether the group should be rejoined (e.g. if metadata changes)
+     * @return true if it should, false otherwise
+     */
+    protected boolean needRejoin() {
+        return rejoinNeeded;
+    }
+
+    /**
+     * Reset the generation/memberId tracked by this member
+     */
+    public void resetGeneration() {
+        this.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID;
+        this.memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID;
+        rejoinNeeded = true;
+    }
+
+    /**
+     * Ensure that the group is active (i.e. joined and synced)
+     */
+    public void ensureActiveGroup() {
+        if (!needRejoin())
+            return;
+
+        // onLeave only invoked if we have a valid current generation
+        onLeave(generation, memberId);
+
+        while (needRejoin()) {
+            ensureCoordinatorKnown();
+
+            // ensure that there are no pending requests to the coordinator. This is important
+            // in particular to avoid resending a pending JoinGroup request.
+            if (client.pendingRequestCount(this.coordinator) > 0) {
+                client.awaitPendingRequests(this.coordinator);
+                continue;
+            }
+
+            RequestFuture<ByteBuffer> future = sendJoinGroupRequest();
+            client.poll(future);
+
+            if (future.succeeded()) {
+                onJoin(generation, memberId, protocol, future.value());
+                heartbeatTask.reset();
+            } else {
+                if (future.exception() instanceof UnknownMemberIdException)
+                    continue;
+                else if (!future.isRetriable())
+                    throw future.exception();
+                Utils.sleep(retryBackoffMs);
+            }
+        }
+    }
+
+    private class HeartbeatTask implements DelayedTask {
+
+        public void reset() {
+            // start or restart the heartbeat task to be executed at the next chance
+            long now = time.milliseconds();
+            heartbeat.resetSessionTimeout(now);
+            client.unschedule(this);
+            client.schedule(this, now);
+        }
+
+        @Override
+        public void run(final long now) {
+            if (generation < 0 || needRejoin() || coordinatorUnknown()) {
+                // no need to send the heartbeat we're not using auto-assignment or if we are
+                // awaiting a rebalance
+                return;
+            }
+
+            if (heartbeat.sessionTimeoutExpired(now)) {
+                // we haven't received a successful heartbeat in one session interval
+                // so mark the coordinator dead
+                coordinatorDead();
+                return;
+            }
+
+            if (!heartbeat.shouldHeartbeat(now)) {
+                // we don't need to heartbeat now, so reschedule for when we do
+                client.schedule(this, now + heartbeat.timeToNextHeartbeat(now));
+            } else {
+                heartbeat.sentHeartbeat(now);
+                RequestFuture<Void> future = sendHeartbeatRequest();
+                future.addListener(new RequestFutureListener<Void>() {
+                    @Override
+                    public void onSuccess(Void value) {
+                        long now = time.milliseconds();
+                        heartbeat.receiveHeartbeat(now);
+                        long nextHeartbeatTime = now + heartbeat.timeToNextHeartbeat(now);
+                        client.schedule(HeartbeatTask.this, nextHeartbeatTime);
+                    }
+
+                    @Override
+                    public void onFailure(RuntimeException e) {
+                        client.schedule(HeartbeatTask.this, time.milliseconds() + retryBackoffMs);
+                    }
+                });
+            }
+        }
+    }
+
+    /**
+     * Send a request to get a new partition assignment. This is a non-blocking call which sends
+     * a JoinGroup request to the coordinator (if it is available). The returned future must
+     * be polled to see if the request completed successfully.
+     * @return A request future whose completion indicates the result of the JoinGroup request.
+     */
+    private RequestFuture<ByteBuffer> sendJoinGroupRequest() {
+        if (coordinatorUnknown())
+            return RequestFuture.coordinatorNotAvailable();
+
+        // send a join group request to the coordinator
+        log.debug("(Re-)joining group {}", groupId);
+
+        List<JoinGroupRequest.GroupProtocol> protocols = new ArrayList<>();
+        for (Map.Entry<String, ByteBuffer> metadataEntry : metadata().entrySet())
+            protocols.add(new JoinGroupRequest.GroupProtocol(metadataEntry.getKey(), metadataEntry.getValue()));
+
+        JoinGroupRequest request = new JoinGroupRequest(
+                groupId,
+                this.sessionTimeoutMs,
+                this.memberId,
+                protocolType(),
+                protocols);
+
+        // create the request for the coordinator
+        log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.coordinator.id());
+        return client.send(coordinator, ApiKeys.JOIN_GROUP, request)
+                .compose(new JoinGroupResponseHandler());
+    }
+
+
+    private class JoinGroupResponseHandler extends CoordinatorResponseHandler<JoinGroupResponse, ByteBuffer> {
+
+        @Override
+        public JoinGroupResponse parse(ClientResponse response) {
+            return new JoinGroupResponse(response.responseBody());
+        }
+
+        @Override
+        public void handle(JoinGroupResponse joinResponse, RequestFuture<ByteBuffer> future) {
+            // process the response
+            short errorCode = joinResponse.errorCode();
+            if (errorCode == Errors.NONE.code()) {
+                log.debug("Joined group: {}", joinResponse.toStruct());
+                AbstractCoordinator.this.memberId = joinResponse.memberId();
+                AbstractCoordinator.this.generation = joinResponse.generationId();
+                AbstractCoordinator.this.rejoinNeeded = false;
+                AbstractCoordinator.this.protocol = joinResponse.groupProtocol();
+                sensors.joinLatency.record(response.requestLatencyMs());
+                performSync(joinResponse).chain(future);
+            } else if (errorCode == Errors.UNKNOWN_MEMBER_ID.code()) {
+                // reset the member id and retry immediately
+                AbstractCoordinator.this.memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID;
+                log.info("Attempt to join group {} failed due to unknown member id, resetting and retrying.",
+                        groupId);
+                future.raise(Errors.UNKNOWN_MEMBER_ID);
+            } else if (errorCode == Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()
+                    || errorCode == Errors.NOT_COORDINATOR_FOR_GROUP.code()) {
+                // re-discover the coordinator and retry with backoff
+                coordinatorDead();
+                log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.",
+                        groupId);
+                future.raise(Errors.forCode(errorCode));
+            } else if (errorCode == Errors.INCONSISTENT_GROUP_PROTOCOL.code()
+                    || errorCode == Errors.INVALID_SESSION_TIMEOUT.code()) {
+                // log the error and re-throw the exception
+                Errors error = Errors.forCode(errorCode);
+                log.error("Attempt to join group {} failed due to: {}",
+                        groupId, error.exception().getMessage());
+                future.raise(error);
+            } else {
+                // unexpected error, throw the exception
+                future.raise(new KafkaException("Unexpected error in join group response: "
+                        + Errors.forCode(joinResponse.errorCode()).exception().getMessage()));
+            }
+        }
+    }
+
+    private RequestFuture<ByteBuffer> performSync(JoinGroupResponse joinResponse) {
+        if (joinResponse.isLeader()) {
+            try {
+                // perform the leader synchronization and send back the assignment for the group
+                Map<String, ByteBuffer> groupAssignment = doSync(joinResponse.leaderId(), joinResponse.groupProtocol(),
+                        joinResponse.members());
+
+                SyncGroupRequest request = new SyncGroupRequest(groupId, generation, memberId, groupAssignment);
+                log.debug("Issuing leader SyncGroup ({}: {}) to coordinator {}", ApiKeys.SYNC_GROUP, request, this.coordinator.id());
+                return sendSyncGroupRequest(request);
+            } catch (RuntimeException e) {
+                return RequestFuture.failure(e);
+            }
+        } else {
+            // send follower's sync group with an empty assignment
+            SyncGroupRequest request = new SyncGroupRequest(groupId, generation,
+                    memberId, Collections.<String, ByteBuffer>emptyMap());
+            log.debug("Issuing follower SyncGroup ({}: {}) to coordinator {}", ApiKeys.SYNC_GROUP, request, this.coordinator.id());
+            return sendSyncGroupRequest(request);
+        }
+    }
+
+    private RequestFuture<ByteBuffer> sendSyncGroupRequest(SyncGroupRequest request) {
+        if (coordinatorUnknown())
+            return RequestFuture.coordinatorNotAvailable();
+        return client.send(coordinator, ApiKeys.SYNC_GROUP, request)
+                .compose(new SyncGroupRequestHandler());
+    }
+
+    private class SyncGroupRequestHandler extends CoordinatorResponseHandler<SyncGroupResponse, ByteBuffer> {
+
+        @Override
+        public SyncGroupResponse parse(ClientResponse response) {
+            return new SyncGroupResponse(response.responseBody());
+        }
+
+        @Override
+        public void handle(SyncGroupResponse syncResponse,
+                           RequestFuture<ByteBuffer> future) {
+            short errorCode = syncResponse.errorCode();
+            if (errorCode == Errors.NONE.code()) {
+                try {
+                    future.complete(syncResponse.memberAssignment());
+                    sensors.syncLatency.record(response.requestLatencyMs());
+                } catch (SchemaException e) {
+                    future.raise(e);
+                }
+            } else {
+                AbstractCoordinator.this.rejoinNeeded = true;
+                future.raise(Errors.forCode(errorCode));
+            }
+        }
+    }
+
+    /**
+     * Discover the current coordinator for the group. Sends a GroupMetadata request to
+     * one of the brokers. The returned future should be polled to get the result of the request.
+     * @return A request future which indicates the completion of the metadata request
+     */
+    private RequestFuture<Void> sendGroupMetadataRequest() {
+        // initiate the group metadata request
+        // find a node to ask about the coordinator
+        Node node = this.client.leastLoadedNode();
+        if (node == null) {
+            // TODO: If there are no brokers left, perhaps we should use the bootstrap set
+            // from configuration?
+            return RequestFuture.noBrokersAvailable();
+        } else {
+            // create a group  metadata request
+            log.debug("Issuing group metadata request to broker {}", node.id());
+            GroupMetadataRequest metadataRequest = new GroupMetadataRequest(this.groupId);
+            return client.send(node, ApiKeys.GROUP_METADATA, metadataRequest)
+                    .compose(new RequestFutureAdapter<ClientResponse, Void>() {
+                        @Override
+                        public void onSuccess(ClientResponse response, RequestFuture<Void> future) {
+                            handleGroupMetadataResponse(response, future);
+                        }
+                    });
+        }
+    }
+
+    private void handleGroupMetadataResponse(ClientResponse resp, RequestFuture<Void> future) {
+        log.debug("Group metadata response {}", resp);
+
+        // parse the response to get the coordinator info if it is not disconnected,
+        // otherwise we need to request metadata update
+        if (resp.wasDisconnected()) {
+            future.raise(new DisconnectException());
+        } else if (!coordinatorUnknown()) {
+            // We already found the coordinator, so ignore the request
+            future.complete(null);
+        } else {
+            GroupMetadataResponse groupMetadataResponse = new GroupMetadataResponse(resp.responseBody());
+            // use MAX_VALUE - node.id as the coordinator id to mimic separate connections
+            // for the coordinator in the underlying network client layer
+            // TODO: this needs to be better handled in KAFKA-1935
+            if (groupMetadataResponse.errorCode() == Errors.NONE.code()) {
+                this.coordinator = new Node(Integer.MAX_VALUE - groupMetadataResponse.node().id(),
+                        groupMetadataResponse.node().host(),
+                        groupMetadataResponse.node().port());
+
+                // start sending heartbeats only if we have a valid generation
+                if (generation > 0)
+                    heartbeatTask.reset();
+                future.complete(null);
+            } else {
+                future.raise(Errors.forCode(groupMetadataResponse.errorCode()));
+            }
+        }
+    }
+
+    /**
+     * Check if we know who the coordinator is.
+     * @return true if the coordinator is unknown
+     */
+    public boolean coordinatorUnknown() {
+        return this.coordinator == null;
+    }
+
+
+    /**
+     * Mark the current coordinator as dead.
+     */
+    protected void coordinatorDead() {
+        if (this.coordinator != null) {
+            log.info("Marking the coordinator {} dead.", this.coordinator.id());
+            this.coordinator = null;
+        }
+    }
+
+    /**
+     * Send a heartbeat request now (visible only for testing).
+     */
+    public RequestFuture<Void> sendHeartbeatRequest() {
+        HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.memberId);
+        return client.send(coordinator, ApiKeys.HEARTBEAT, req)
+                .compose(new HeartbeatCompletionHandler());
+    }
+
+    private class HeartbeatCompletionHandler extends CoordinatorResponseHandler<HeartbeatResponse, Void> {
+        @Override
+        public HeartbeatResponse parse(ClientResponse response) {
+            return new HeartbeatResponse(response.responseBody());
+        }
+
+        @Override
+        public void handle(HeartbeatResponse heartbeatResponse, RequestFuture<Void> future) {
+            sensors.heartbeatLatency.record(response.requestLatencyMs());
+            short error = heartbeatResponse.errorCode();
+            if (error == Errors.NONE.code()) {
+                log.debug("Received successful heartbeat response.");
+                future.complete(null);
+            } else if (error == Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()
+                    || error == Errors.NOT_COORDINATOR_FOR_GROUP.code()) {
+                log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead.");
+                coordinatorDead();
+                future.raise(Errors.forCode(error));
+            } else if (error == Errors.REBALANCE_IN_PROGRESS.code()) {
+                log.info("Attempt to heart beat failed since the group is rebalancing, try to re-join group.");
+                AbstractCoordinator.this.rejoinNeeded = true;
+                future.raise(Errors.REBALANCE_IN_PROGRESS);
+            } else if (error == Errors.ILLEGAL_GENERATION.code()) {
+                log.info("Attempt to heart beat failed since generation id is not legal, try to re-join group.");
+                AbstractCoordinator.this.rejoinNeeded = true;
+                future.raise(Errors.ILLEGAL_GENERATION);
+            } else if (error == Errors.UNKNOWN_MEMBER_ID.code()) {
+                log.info("Attempt to heart beat failed since member id is not valid, reset it and try to re-join group.");
+                memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID;
+                AbstractCoordinator.this.rejoinNeeded = true;
+                future.raise(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                future.raise(new KafkaException("Unexpected error in heartbeat response: "
+                        + Errors.forCode(error).exception().getMessage()));
+            }
+        }
+    }
+
+    protected abstract class CoordinatorResponseHandler<R, T>
+            extends RequestFutureAdapter<ClientResponse, T> {
+        protected ClientResponse response;
+
+        public abstract R parse(ClientResponse response);
+
+        public abstract void handle(R response, RequestFuture<T> future);
+
+        @Override
+        public void onSuccess(ClientResponse clientResponse, RequestFuture<T> future) {
+            this.response = clientResponse;
+
+            if (clientResponse.wasDisconnected()) {
+                int correlation = response.request().request().header().correlationId();
+                log.debug("Cancelled request {} with correlation id {} due to coordinator {} being disconnected",
+                        response.request(),
+                        correlation,
+                        response.request().request().destination());
+
+                // mark the coordinator as dead
+                coordinatorDead();
+                future.raise(new DisconnectException());
+                return;
+            }
+
+            R response = parse(clientResponse);
+            handle(response, future);
+        }
+
+    }
+
+    private class GroupCoordinatorMetrics {
+        public final Metrics metrics;
+        public final String metricGrpName;
+
+        public final Sensor heartbeatLatency;
+        public final Sensor joinLatency;
+        public final Sensor syncLatency;
+
+        public GroupCoordinatorMetrics(Metrics metrics, String metricGrpPrefix, Map<String, String> tags) {
+            this.metrics = metrics;
+            this.metricGrpName = metricGrpPrefix + "-coordinator-metrics";
+
+            this.heartbeatLatency = metrics.sensor("heartbeat-latency");
+            this.heartbeatLatency.add(new MetricName("heartbeat-response-time-max",
+                this.metricGrpName,
+                "The max time taken to receive a response to a heartbeat request",
+                tags), new Max());
+            this.heartbeatLatency.add(new MetricName("heartbeat-rate",
+                this.metricGrpName,
+                "The average number of heartbeats per second",
+                tags), new Rate(new Count()));
+
+            this.joinLatency = metrics.sensor("join-latency");
+            this.joinLatency.add(new MetricName("join-time-avg",
+                    this.metricGrpName,
+                    "The average time taken for a group rejoin",
+                    tags), new Avg());
+            this.joinLatency.add(new MetricName("join-time-max",
+                    this.metricGrpName,
+                    "The max time taken for a group rejoin",
+                    tags), new Avg());
+            this.joinLatency.add(new MetricName("join-rate",
+                    this.metricGrpName,
+                    "The number of group joins per second",
+                    tags), new Rate(new Count()));
+
+            this.syncLatency = metrics.sensor("sync-latency");
+            this.syncLatency.add(new MetricName("sync-time-avg",
+                    this.metricGrpName,
+                    "The average time taken for a group sync",
+                    tags), new Avg());
+            this.syncLatency.add(new MetricName("sync-time-max",
+                    this.metricGrpName,
+                    "The max time taken for a group sync",
+                    tags), new Avg());
+            this.syncLatency.add(new MetricName("sync-rate",
+                    this.metricGrpName,
+                    "The number of group syncs per second",
+                    tags), new Rate(new Count()));
+
+            Measurable lastHeartbeat =
+                new Measurable() {
+                    public double measure(MetricConfig config, long now) {
+                        return TimeUnit.SECONDS.convert(now - heartbeat.lastHeartbeatSend(), TimeUnit.MILLISECONDS);
+                    }
+                };
+            metrics.addMetric(new MetricName("last-heartbeat-seconds-ago",
+                this.metricGrpName,
+                "The number of seconds since the last controller heartbeat",
+                tags),
+                lastHeartbeat);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java
new file mode 100644
index 0000000..12fa913
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java
@@ -0,0 +1,90 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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 org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.TopicPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Abstract assignor implementation which does some common grunt work (in particular collecting
+ * partition counts which are always needed in assignors).
+ */
+public abstract class AbstractPartitionAssignor implements PartitionAssignor {
+    private static final Logger log = LoggerFactory.getLogger(AbstractPartitionAssignor.class);
+
+    /**
+     * Perform the group assignment given the partition counts and member subscriptions
+     * @param partitionsPerTopic The number of partitions for each subscribed topic (may be empty for some topics)
+     * @param subscriptions Map from the memberId to their respective topic subscription
+     * @return Map from each member to the
+     */
+    public abstract Map<String, List<TopicPartition>> assign(Map<String, Integer> partitionsPerTopic,
+                                                             Map<String, List<String>> subscriptions);
+
+    @Override
+    public Subscription subscription(Set<String> topics) {
+        return new Subscription(new ArrayList<>(topics));
+    }
+
+    @Override
+    public Map<String, Assignment> assign(Cluster metadata, Map<String, Subscription> subscriptions) {
+        Set<String> allSubscribedTopics = new HashSet<>();
+        Map<String, List<String>> topicSubscriptions = new HashMap<>();
+        for (Map.Entry<String, Subscription> subscriptionEntry : subscriptions.entrySet()) {
+            List<String> topics = subscriptionEntry.getValue().topics();
+            allSubscribedTopics.addAll(topics);
+            topicSubscriptions.put(subscriptionEntry.getKey(), topics);
+        }
+
+        Map<String, Integer> partitionsPerTopic = new HashMap<>();
+        for (String topic : allSubscribedTopics) {
+            Integer numPartitions = metadata.partitionCountForTopic(topic);
+            if (numPartitions != null)
+                partitionsPerTopic.put(topic, numPartitions);
+            else
+                log.debug("Skipping assignment for topic {} since no metadata is available", topic);
+        }
+
+        Map<String, List<TopicPartition>> rawAssignments = assign(partitionsPerTopic, topicSubscriptions);
+
+        // this class has maintains no user data, so just wrap the results
+        Map<String, Assignment> assignments = new HashMap<>();
+        for (Map.Entry<String, List<TopicPartition>> assignmentEntry : rawAssignments.entrySet())
+            assignments.put(assignmentEntry.getKey(), new Assignment(assignmentEntry.getValue()));
+        return assignments;
+    }
+
+    @Override
+    public void onAssignment(Assignment assignment) {
+        // this assignor maintains no internal state, so nothing to do
+    }
+
+    protected static <K, V> void put(Map<K, List<V>> map, K key, V value) {
+        List<V> list = map.get(key);
+        if (list == null) {
+            list = new ArrayList<>();
+            map.put(key, list);
+        }
+        list.add(value);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
new file mode 100644
index 0000000..fc7e819
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
@@ -0,0 +1,595 @@
+/**
+ * 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 org.apache.kafka.clients.ClientResponse;
+import org.apache.kafka.clients.Metadata;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.ConsumerWakeupException;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.clients.consumer.internals.PartitionAssignor.Assignment;
+import org.apache.kafka.clients.consumer.internals.PartitionAssignor.Subscription;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.metrics.Measurable;
+import org.apache.kafka.common.metrics.MetricConfig;
+import org.apache.kafka.common.metrics.Metrics;
+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.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+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.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Closeable;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This class manages the coordination process with the consumer coordinator.
+ */
+public final class ConsumerCoordinator extends AbstractCoordinator implements Closeable {
+
+    private static final Logger log = LoggerFactory.getLogger(ConsumerCoordinator.class);
+
+    private final Map<String, PartitionAssignor> protocolMap;
+    private final org.apache.kafka.clients.Metadata metadata;
+    private final MetadataSnapshot metadataSnapshot;
+    private final ConsumerCoordinatorMetrics sensors;
+    private final SubscriptionState subscriptions;
+    private final OffsetCommitCallback defaultOffsetCommitCallback;
+    private final boolean autoCommitEnabled;
+
+    /**
+     * Initialize the coordination manager.
+     */
+    public ConsumerCoordinator(ConsumerNetworkClient client,
+                               String groupId,
+                               int sessionTimeoutMs,
+                               int heartbeatIntervalMs,
+                               List<PartitionAssignor> assignors,
+                               Metadata metadata,
+                               SubscriptionState subscriptions,
+                               Metrics metrics,
+                               String metricGrpPrefix,
+                               Map<String, String> metricTags,
+                               Time time,
+                               long requestTimeoutMs,
+                               long retryBackoffMs,
+                               OffsetCommitCallback defaultOffsetCommitCallback,
+                               boolean autoCommitEnabled,
+                               long autoCommitIntervalMs) {
+        super(client,
+                groupId,
+                sessionTimeoutMs,
+                heartbeatIntervalMs,
+                metrics,
+                metricGrpPrefix,
+                metricTags,
+                time,
+                requestTimeoutMs,
+                retryBackoffMs);
+        this.metadata = metadata;
+
+        this.metadata.requestUpdate();
+        this.metadataSnapshot = new MetadataSnapshot();
+        this.subscriptions = subscriptions;
+        this.defaultOffsetCommitCallback = defaultOffsetCommitCallback;
+        this.autoCommitEnabled = autoCommitEnabled;
+
+        this.protocolMap = new HashMap<>();
+        for (PartitionAssignor assignor : assignors)
+            this.protocolMap.put(assignor.name(), assignor);
+
+        addMetadataListener();
+
+        if (autoCommitEnabled)
+            scheduleAutoCommitTask(autoCommitIntervalMs);
+
+        this.sensors = new ConsumerCoordinatorMetrics(metrics, metricGrpPrefix, metricTags);
+    }
+
+    @Override
+    public String protocolType() {
+        return "consumer";
+    }
+
+    @Override
+    public LinkedHashMap<String, ByteBuffer> metadata() {
+        LinkedHashMap<String, ByteBuffer> metadata = new LinkedHashMap<>();
+        for (PartitionAssignor assignor : protocolMap.values()) {
+            Subscription subscription = assignor.subscription(subscriptions.subscription());
+            metadata.put(assignor.name(), ConsumerProtocol.serializeSubscription(subscription));
+        }
+        return metadata;
+    }
+
+    private void addMetadataListener() {
+        this.metadata.addListener(new Metadata.Listener() {
+            @Override
+            public void onMetadataUpdate(Cluster cluster) {
+                if (subscriptions.hasPatternSubscription()) {
+                    final List<String> topicsToSubscribe = new ArrayList<>();
+
+                    for (String topic : cluster.topics())
+                        if (subscriptions.getSubscribedPattern().matcher(topic).matches())
+                            topicsToSubscribe.add(topic);
+
+                    subscriptions.changeSubscription(topicsToSubscribe);
+                    metadata.setTopics(subscriptions.groupSubscription());
+                }
+
+                // check if there are any changes to the metadata which should trigger a rebalance
+                if (metadataSnapshot.update(subscriptions, cluster) && subscriptions.partitionsAutoAssigned())
+                    subscriptions.needReassignment();
+            }
+        });
+    }
+
+    @Override
+    protected void onJoin(int generation,
+                          String memberId,
+                          String assignmentStrategy,
+                          ByteBuffer assignmentBuffer) {
+        PartitionAssignor assignor = protocolMap.get(assignmentStrategy);
+        if (assignor == null)
+            throw new IllegalStateException("Coordinator selected invalid assignment protocol: " + assignmentStrategy);
+
+        Assignment assignment = ConsumerProtocol.deserializeAssignment(assignmentBuffer);
+
+        // set the flag to refresh last committed offsets
+        subscriptions.needRefreshCommits();
+
+        // update partition assignment
+        subscriptions.changePartitionAssignment(assignment.partitions());
+
+        // give the assignor a chance to update internal state based on the received assignment
+        assignor.onAssignment(assignment);
+
+        // execute the user's callback after rebalance
+        ConsumerRebalanceListener listener = subscriptions.listener();
+        log.debug("Setting newly assigned partitions {}", subscriptions.assignedPartitions());
+        try {
+            Set<TopicPartition> assigned = new HashSet<>(subscriptions.assignedPartitions());
+            listener.onPartitionsAssigned(assigned);
+        } catch (Exception e) {
+            log.error("User provided listener " + listener.getClass().getName()
+                    + " failed on partition assignment: ", e);
+        }
+    }
+
+    @Override
+    protected Map<String, ByteBuffer> doSync(String leaderId,
+                                             String assignmentStrategy,
+                                             Map<String, ByteBuffer> allSubscriptions) {
+        PartitionAssignor assignor = protocolMap.get(protocol);
+        if (assignor == null)
+            throw new IllegalStateException("Coordinator selected invalid assignment protocol: " + assignmentStrategy);
+
+        Set<String> allSubscribedTopics = new HashSet<>();
+        Map<String, Subscription> subscriptions = new HashMap<>();
+        for (Map.Entry<String, ByteBuffer> subscriptionEntry : allSubscriptions.entrySet()) {
+            Subscription subscription = ConsumerProtocol.deserializeSubscription(subscriptionEntry.getValue());
+            subscriptions.put(subscriptionEntry.getKey(), subscription);
+            allSubscribedTopics.addAll(subscription.topics());
+        }
+
+        // the leader will begin watching for changes to any of the topics the group is interested in,
+        // which ensures that all metadata changes will eventually be seen
+        this.subscriptions.groupSubscribe(allSubscribedTopics);
+        metadata.setTopics(this.subscriptions.groupSubscription());
+        client.ensureFreshMetadata();
+
+        log.debug("Performing {} assignment for subscriptions {}", assignor.name(), subscriptions);
+
+        Map<String, Assignment> assignment = assignor.assign(metadata.fetch(), subscriptions);
+
+        log.debug("Finished assignment: {}", assignment);
+
+        Map<String, ByteBuffer> groupAssignment = new HashMap<>();
+        for (Map.Entry<String, Assignment> assignmentEntry : assignment.entrySet()) {
+            ByteBuffer buffer = ConsumerProtocol.serializeAssignment(assignmentEntry.getValue());
+            groupAssignment.put(assignmentEntry.getKey(), buffer);
+        }
+
+        return groupAssignment;
+    }
+
+    @Override
+    protected void onLeave(int generation, String memberId) {
+        // commit offsets prior to rebalance if auto-commit enabled
+        maybeAutoCommitOffsetsSync();
+
+        // execute the user's callback before rebalance
+        ConsumerRebalanceListener listener = subscriptions.listener();
+        log.debug("Revoking previously assigned partitions {}", subscriptions.assignedPartitions());
+        try {
+            Set<TopicPartition> revoked = new HashSet<>(subscriptions.assignedPartitions());
+            listener.onPartitionsRevoked(revoked);
+        } catch (Exception e) {
+            log.error("User provided listener " + listener.getClass().getName()
+                    + " failed on partition revocation: ", e);
+        }
+
+        subscriptions.needReassignment();
+    }
+
+    @Override
+    public boolean needRejoin() {
+        return subscriptions.partitionsAutoAssigned() &&
+                (super.needRejoin() || subscriptions.partitionAssignmentNeeded());
+    }
+
+    /**
+     * Refresh the committed offsets for provided partitions.
+     */
+    public void refreshCommittedOffsetsIfNeeded() {
+        if (subscriptions.refreshCommitsNeeded()) {
+            Map<TopicPartition, OffsetAndMetadata> offsets = fetchCommittedOffsets(subscriptions.assignedPartitions());
+            for (Map.Entry<TopicPartition, OffsetAndMetadata> entry : offsets.entrySet()) {
+                TopicPartition tp = entry.getKey();
+                // verify assignment is still active
+                if (subscriptions.isAssigned(tp))
+                    this.subscriptions.committed(tp, entry.getValue());
+            }
+            this.subscriptions.commitsRefreshed();
+        }
+    }
+
+    /**
+     * Fetch the current committed offsets from the coordinator for a set of partitions.
+     * @param partitions The partitions to fetch offsets for
+     * @return A map from partition to the committed offset
+     */
+    public Map<TopicPartition, OffsetAndMetadata> fetchCommittedOffsets(Set<TopicPartition> partitions) {
+        while (true) {
+            ensureCoordinatorKnown();
+
+            // contact coordinator to fetch committed offsets
+            RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future = sendOffsetFetchRequest(partitions);
+            client.poll(future);
+
+            if (future.succeeded())
+                return future.value();
+
+            if (!future.isRetriable())
+                throw future.exception();
+
+            Utils.sleep(retryBackoffMs);
+        }
+    }
+
+    /**
+     * Ensure that we have a valid partition assignment from the coordinator.
+     */
+    public void ensurePartitionAssignment() {
+        if (subscriptions.partitionsAutoAssigned())
+            ensureActiveGroup();
+    }
+
+    @Override
+    public void close() {
+        // commit offsets prior to closing if auto-commit enabled
+        while (true) {
+            try {
+                maybeAutoCommitOffsetsSync();
+                return;
+            } catch (ConsumerWakeupException e) {
+                // ignore wakeups while closing to ensure we have a chance to commit
+                continue;
+            }
+        }
+    }
+
+    public void commitOffsetsAsync(final Map<TopicPartition, OffsetAndMetadata> offsets, OffsetCommitCallback callback) {
+        this.subscriptions.needRefreshCommits();
+        RequestFuture<Void> future = sendOffsetCommitRequest(offsets);
+        final OffsetCommitCallback cb = callback == null ? defaultOffsetCommitCallback : callback;
+        future.addListener(new RequestFutureListener<Void>() {
+            @Override
+            public void onSuccess(Void value) {
+                cb.onComplete(offsets, null);
+            }
+
+            @Override
+            public void onFailure(RuntimeException e) {
+                cb.onComplete(offsets, e);
+            }
+        });
+    }
+
+    public void commitOffsetsSync(Map<TopicPartition, OffsetAndMetadata> offsets) {
+        if (offsets.isEmpty())
+            return;
+
+        while (true) {
+            ensureCoordinatorKnown();
+
+            RequestFuture<Void> future = sendOffsetCommitRequest(offsets);
+            client.poll(future);
+
+            if (future.succeeded()) {
+                return;
+            }
+
+            if (!future.isRetriable()) {
+                throw future.exception();
+            }
+
+            Utils.sleep(retryBackoffMs);
+        }
+    }
+
+    private void scheduleAutoCommitTask(final long interval) {
+        DelayedTask task = new DelayedTask() {
+            public void run(long now) {
+                commitOffsetsAsync(subscriptions.allConsumed(), new OffsetCommitCallback() {
+                    @Override
+                    public void onComplete(Map<TopicPartition, OffsetAndMetadata> offsets, Exception exception) {
+                        if (exception != null)
+                            log.error("Auto offset commit failed.", exception);
+                    }
+                });
+                client.schedule(this, now + interval);
+            }
+        };
+        client.schedule(task, time.milliseconds() + interval);
+    }
+
+    private void maybeAutoCommitOffsetsSync() {
+        if (autoCommitEnabled) {
+            try {
+                commitOffsetsSync(subscriptions.allConsumed());
+            } catch (ConsumerWakeupException e) {
+                // rethrow wakeups since they are triggered by the user
+                throw e;
+            } catch (Exception e) {
+                // consistent with async auto-commit failures, we do not propagate the exception
+                log.error("Auto offset commit failed.", e);
+            }
+        }
+    }
+
+    /**
+     * Commit offsets for the specified list of topics and partitions. This is a non-blocking call
+     * which returns a request future that can be polled in the case of a synchronous commit or ignored in the
+     * asynchronous case.
+     *
+     * @param offsets The list of offsets per partition that should be committed.
+     * @return A request future whose value indicates whether the commit was successful or not
+     */
+    private RequestFuture<Void> sendOffsetCommitRequest(final Map<TopicPartition, OffsetAndMetadata> offsets) {
+        if (coordinatorUnknown())
+            return RequestFuture.coordinatorNotAvailable();
+
+        if (offsets.isEmpty())
+            return RequestFuture.voidSuccess();
+
+        // create the offset commit request
+        Map<TopicPartition, OffsetCommitRequest.PartitionData> offsetData = new HashMap<>(offsets.size());
+        for (Map.Entry<TopicPartition, OffsetAndMetadata> entry : offsets.entrySet()) {
+            OffsetAndMetadata offsetAndMetadata = entry.getValue();
+            offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(
+                    offsetAndMetadata.offset(), offsetAndMetadata.metadata()));
+        }
+
+        OffsetCommitRequest req = new OffsetCommitRequest(this.groupId,
+                this.generation,
+                this.memberId,
+                OffsetCommitRequest.DEFAULT_RETENTION_TIME,
+                offsetData);
+
+        return client.send(coordinator, ApiKeys.OFFSET_COMMIT, req)
+                .compose(new OffsetCommitResponseHandler(offsets));
+    }
+
+    public static class DefaultOffsetCommitCallback implements OffsetCommitCallback {
+        @Override
+        public void onComplete(Map<TopicPartition, OffsetAndMetadata> offsets, Exception exception) {
+            if (exception != null)
+                log.error("Offset commit failed.", exception);
+        }
+    }
+
+    private class OffsetCommitResponseHandler extends CoordinatorResponseHandler<OffsetCommitResponse, Void> {
+
+        private final Map<TopicPartition, OffsetAndMetadata> offsets;
+
+        public OffsetCommitResponseHandler(Map<TopicPartition, OffsetAndMetadata> offsets) {
+            this.offsets = offsets;
+        }
+
+        @Override
+        public OffsetCommitResponse parse(ClientResponse response) {
+            return new OffsetCommitResponse(response.responseBody());
+        }
+
+        @Override
+        public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> future) {
+            sensors.commitLatency.record(response.requestLatencyMs());
+            for (Map.Entry<TopicPartition, Short> entry : commitResponse.responseData().entrySet()) {
+                TopicPartition tp = entry.getKey();
+                OffsetAndMetadata offsetAndMetadata = this.offsets.get(tp);
+                long offset = offsetAndMetadata.offset();
+
+                short errorCode = entry.getValue();
+                if (errorCode == Errors.NONE.code()) {
+                    log.debug("Committed offset {} for partition {}", offset, tp);
+                    if (subscriptions.isAssigned(tp))
+                        // update the local cache only if the partition is still assigned
+                        subscriptions.committed(tp, offsetAndMetadata);
+                } else {
+                    if (errorCode == Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()
+                            || errorCode == Errors.NOT_COORDINATOR_FOR_GROUP.code()) {
+                        coordinatorDead();
+                    } else if (errorCode == Errors.UNKNOWN_MEMBER_ID.code()
+                            || errorCode == Errors.ILLEGAL_GENERATION.code()) {
+                        // need to re-join group
+                        subscriptions.needReassignment();
+                    }
+
+                    log.error("Error committing partition {} at offset {}: {}",
+                            tp,
+                            offset,
+                            Errors.forCode(errorCode).exception().getMessage());
+
+                    future.raise(Errors.forCode(errorCode));
+                    return;
+                }
+            }
+
+            future.complete(null);
+        }
+    }
+
+    /**
+     * Fetch the committed offsets for a set of partitions. This is a non-blocking call. The
+     * returned future can be polled to get the actual offsets returned from the broker.
+     *
+     * @param partitions The set of partitions to get offsets for.
+     * @return A request future containing the committed offsets.
+     */
+    private RequestFuture<Map<TopicPartition, OffsetAndMetadata>> sendOffsetFetchRequest(Set<TopicPartition> partitions) {
+        if (coordinatorUnknown())
+            return RequestFuture.coordinatorNotAvailable();
+
+        log.debug("Fetching committed offsets for partitions: {}",  Utils.join(partitions, ", "));
+        // construct the request
+        OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList<TopicPartition>(partitions));
+
+        // send the request with a callback
+        return client.send(coordinator, ApiKeys.OFFSET_FETCH, request)
+                .compose(new OffsetFetchResponseHandler());
+    }
+
+    private class OffsetFetchResponseHandler extends CoordinatorResponseHandler<OffsetFetchResponse, Map<TopicPartition, OffsetAndMetadata>> {
+
+        @Override
+        public OffsetFetchResponse parse(ClientResponse response) {
+            return new OffsetFetchResponse(response.responseBody());
+        }
+
+        @Override
+        public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future) {
+            Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>(response.responseData().size());
+            for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry : response.responseData().entrySet()) {
+                TopicPartition tp = entry.getKey();
+                OffsetFetchResponse.PartitionData data = entry.getValue();
+                if (data.hasError()) {
+                    log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode)
+                            .exception()
+                            .getMessage());
+                    if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) {
+                        // just retry
+                        future.raise(Errors.OFFSET_LOAD_IN_PROGRESS);
+                    } else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_GROUP.code()) {
+                        // re-discover the coordinator and retry
+                        coordinatorDead();
+                        future.raise(Errors.NOT_COORDINATOR_FOR_GROUP);
+                    } else if (data.errorCode == Errors.UNKNOWN_MEMBER_ID.code()
+                            || data.errorCode == Errors.ILLEGAL_GENERATION.code()) {
+                        // need to re-join group
+                        subscriptions.needReassignment();
+                        future.raise(Errors.forCode(data.errorCode));
+                    } else {
+                        future.raise(new KafkaException("Unexpected error in fetch offset response: "
+                                + Errors.forCode(data.errorCode).exception().getMessage()));
+                    }
+                    return;
+                } else if (data.offset >= 0) {
+                    // record the position with the offset (-1 indicates no committed offset to fetch)
+                    offsets.put(tp, new OffsetAndMetadata(data.offset, data.metadata));
+                } else {
+                    log.debug("No committed offset for partition " + tp);
+                }
+            }
+
+            future.complete(offsets);
+        }
+    }
+
+    private class ConsumerCoordinatorMetrics {
+        public final Metrics metrics;
+        public final String metricGrpName;
+
+        public final Sensor commitLatency;
+
+        public ConsumerCoordinatorMetrics(Metrics metrics, String metricGrpPrefix, Map<String, String> tags) {
+            this.metrics = metrics;
+            this.metricGrpName = metricGrpPrefix + "-coordinator-metrics";
+
+            this.commitLatency = metrics.sensor("commit-latency");
+            this.commitLatency.add(new MetricName("commit-latency-avg",
+                this.metricGrpName,
+                "The average time taken for a commit request",
+                tags), new Avg());
+            this.commitLatency.add(new MetricName("commit-latency-max",
+                this.metricGrpName,
+                "The max time taken for a commit request",
+                tags), new Max());
+            this.commitLatency.add(new MetricName("commit-rate",
+                this.metricGrpName,
+                "The number of commit calls per second",
+                tags), new Rate(new Count()));
+
+            Measurable numParts =
+                new Measurable() {
+                    public double measure(MetricConfig config, long now) {
+                        return subscriptions.assignedPartitions().size();
+                    }
+                };
+            metrics.addMetric(new MetricName("assigned-partitions",
+                this.metricGrpName,
+                "The number of partitions currently assigned to this consumer",
+                tags),
+                numParts);
+        }
+    }
+
+    private static class MetadataSnapshot {
+        private Map<String, Integer> partitionsPerTopic = new HashMap<>();
+
+        public boolean update(SubscriptionState subscription, Cluster cluster) {
+            Map<String, Integer> partitionsPerTopic = new HashMap<>();
+            for (String topic : subscription.groupSubscription())
+                partitionsPerTopic.put(topic, cluster.partitionCountForTopic(topic));
+
+            if (!partitionsPerTopic.equals(this.partitionsPerTopic)) {
+                this.partitionsPerTopic = partitionsPerTopic;
+                return true;
+            }
+
+            return false;
+        }
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
index 4153eb3..fbfe54a 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java
@@ -127,6 +127,15 @@ public class ConsumerNetworkClient implements Closeable {
     }
 
     /**
+     * Ensure our metadata is fresh (if an update is expected, this will block
+     * until it has completed).
+     */
+    public void ensureFreshMetadata() {
+        if (this.metadata.timeToNextUpdate(time.milliseconds()) == 0)
+            awaitMetadataUpdate();
+    }
+
+    /**
      * Wakeup an active poll. This will cause the polling thread to throw an exception either
      * on the current poll if one is active, or the next poll.
      */


[4/8] kafka git commit: KAFKA-2464: client-side assignment for new consumer

Posted by gu...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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
deleted file mode 100644
index ea1c0d0..0000000
--- a/core/src/main/scala/kafka/api/ConsumerMetadataResponse.scala
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package kafka.api
-
-import java.nio.ByteBuffer
-import kafka.cluster.BrokerEndPoint
-import kafka.common.ErrorMapping
-
-object ConsumerMetadataResponse {
-  val CurrentVersion = 0
-
-  private val NoBrokerEndpointOpt = Some(BrokerEndPoint(id = -1, host = "", port = -1))
-  
-  def readFrom(buffer: ByteBuffer) = {
-    val correlationId = buffer.getInt
-    val errorCode = buffer.getShort
-    val broker = BrokerEndPoint.readFrom(buffer)
-    val coordinatorOpt = if (errorCode == ErrorMapping.NoError)
-      Some(broker)
-    else
-      None
-
-    ConsumerMetadataResponse(coordinatorOpt, errorCode, correlationId)
-  }
-  
-}
-
-case class ConsumerMetadataResponse (coordinatorOpt: Option[BrokerEndPoint], errorCode: Short, correlationId: Int)
-  extends RequestOrResponse() {
-
-  def sizeInBytes =
-    4 + /* correlationId */
-    2 + /* error code */
-    coordinatorOpt.orElse(ConsumerMetadataResponse.NoBrokerEndpointOpt).get.sizeInBytes
-
-  def writeTo(buffer: ByteBuffer) {
-    buffer.putInt(correlationId)
-    buffer.putShort(errorCode)
-    coordinatorOpt.orElse(ConsumerMetadataResponse.NoBrokerEndpointOpt).foreach(_.writeTo(buffer))
-  }
-
-  def describe(details: Boolean) = toString
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/api/GroupMetadataRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/GroupMetadataRequest.scala b/core/src/main/scala/kafka/api/GroupMetadataRequest.scala
new file mode 100644
index 0000000..075ddb5
--- /dev/null
+++ b/core/src/main/scala/kafka/api/GroupMetadataRequest.scala
@@ -0,0 +1,80 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.api
+
+import java.nio.ByteBuffer
+
+import kafka.common.ErrorMapping
+import kafka.network.{RequestOrResponseSend, RequestChannel}
+import kafka.network.RequestChannel.Response
+
+object GroupMetadataRequest {
+  val CurrentVersion = 0.shortValue
+  val DefaultClientId = ""
+
+  def readFrom(buffer: ByteBuffer) = {
+    // envelope
+    val versionId = buffer.getShort
+    val correlationId = buffer.getInt
+    val clientId = ApiUtils.readShortString(buffer)
+
+    // request
+    val group = ApiUtils.readShortString(buffer)
+    GroupMetadataRequest(group, versionId, correlationId, clientId)
+  }
+
+}
+
+case class GroupMetadataRequest(group: String,
+                                versionId: Short = GroupMetadataRequest.CurrentVersion,
+                                correlationId: Int = 0,
+                                clientId: String = GroupMetadataRequest.DefaultClientId)
+  extends RequestOrResponse(Some(RequestKeys.GroupMetadataKey)) {
+
+  def sizeInBytes =
+    2 + /* versionId */
+    4 + /* correlationId */
+    ApiUtils.shortStringLength(clientId) +
+    ApiUtils.shortStringLength(group)
+
+  def writeTo(buffer: ByteBuffer) {
+    // envelope
+    buffer.putShort(versionId)
+    buffer.putInt(correlationId)
+    ApiUtils.writeShortString(buffer, clientId)
+
+    // consumer metadata request
+    ApiUtils.writeShortString(buffer, group)
+  }
+
+  override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
+    // return ConsumerCoordinatorNotAvailable for all uncaught errors
+    val errorResponse = GroupMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, correlationId)
+    requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse)))
+  }
+
+  def describe(details: Boolean) = {
+    val consumerMetadataRequest = new StringBuilder
+    consumerMetadataRequest.append("Name: " + this.getClass.getSimpleName)
+    consumerMetadataRequest.append("; Version: " + versionId)
+    consumerMetadataRequest.append("; CorrelationId: " + correlationId)
+    consumerMetadataRequest.append("; ClientId: " + clientId)
+    consumerMetadataRequest.append("; Group: " + group)
+    consumerMetadataRequest.toString()
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/api/GroupMetadataResponse.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/GroupMetadataResponse.scala b/core/src/main/scala/kafka/api/GroupMetadataResponse.scala
new file mode 100644
index 0000000..2d65917
--- /dev/null
+++ b/core/src/main/scala/kafka/api/GroupMetadataResponse.scala
@@ -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 kafka.api
+
+import java.nio.ByteBuffer
+import kafka.cluster.BrokerEndPoint
+import kafka.common.ErrorMapping
+
+object GroupMetadataResponse {
+  val CurrentVersion = 0
+
+  private val NoBrokerEndpointOpt = Some(BrokerEndPoint(id = -1, host = "", port = -1))
+  
+  def readFrom(buffer: ByteBuffer) = {
+    val correlationId = buffer.getInt
+    val errorCode = buffer.getShort
+    val broker = BrokerEndPoint.readFrom(buffer)
+    val coordinatorOpt = if (errorCode == ErrorMapping.NoError)
+      Some(broker)
+    else
+      None
+
+    GroupMetadataResponse(coordinatorOpt, errorCode, correlationId)
+  }
+  
+}
+
+case class GroupMetadataResponse (coordinatorOpt: Option[BrokerEndPoint], errorCode: Short, correlationId: Int)
+  extends RequestOrResponse() {
+
+  def sizeInBytes =
+    4 + /* correlationId */
+    2 + /* error code */
+    coordinatorOpt.orElse(GroupMetadataResponse.NoBrokerEndpointOpt).get.sizeInBytes
+
+  def writeTo(buffer: ByteBuffer) {
+    buffer.putInt(correlationId)
+    buffer.putShort(errorCode)
+    coordinatorOpt.orElse(GroupMetadataResponse.NoBrokerEndpointOpt).foreach(_.writeTo(buffer))
+  }
+
+  def describe(details: Boolean) = toString
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/api/OffsetCommitRequest.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala
index 5b362ef..75067cf 100644
--- a/core/src/main/scala/kafka/api/OffsetCommitRequest.scala
+++ b/core/src/main/scala/kafka/api/OffsetCommitRequest.scala
@@ -41,7 +41,7 @@ object OffsetCommitRequest extends Logging {
     val clientId = readShortString(buffer)
 
     // Read the OffsetRequest 
-    val consumerGroupId = readShortString(buffer)
+    val groupId = readShortString(buffer)
 
     // version 1 and 2 specific fields
     val groupGenerationId: Int =
@@ -50,11 +50,11 @@ object OffsetCommitRequest extends Logging {
       else
         org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID
 
-    val consumerId: String =
+    val memberId: String =
       if (versionId >= 1)
         readShortString(buffer)
       else
-        org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID
+        org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_MEMBER_ID
 
     // version 2 specific fields
     val retentionMs: Long =
@@ -83,7 +83,7 @@ object OffsetCommitRequest extends Logging {
       })
     })
 
-    OffsetCommitRequest(consumerGroupId, immutable.Map(pairs:_*), versionId, correlationId, clientId, groupGenerationId, consumerId, retentionMs)
+    OffsetCommitRequest(groupId, immutable.Map(pairs:_*), versionId, correlationId, clientId, groupGenerationId, memberId, retentionMs)
   }
 }
 
@@ -93,7 +93,7 @@ case class OffsetCommitRequest(groupId: String,
                                correlationId: Int = 0,
                                clientId: String = OffsetCommitRequest.DefaultClientId,
                                groupGenerationId: Int = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_GENERATION_ID,
-                               consumerId: String =  org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_CONSUMER_ID,
+                               memberId: String =  org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_MEMBER_ID,
                                retentionMs: Long = org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_RETENTION_TIME)
     extends RequestOrResponse(Some(RequestKeys.OffsetCommitKey)) {
 
@@ -114,7 +114,7 @@ case class OffsetCommitRequest(groupId: String,
     // version 1 and 2 specific data
     if (versionId >= 1) {
       buffer.putInt(groupGenerationId)
-      writeShortString(buffer, consumerId)
+      writeShortString(buffer, memberId)
     }
 
     // version 2 or above specific data
@@ -142,7 +142,7 @@ case class OffsetCommitRequest(groupId: String,
     4 + /* correlationId */
     shortStringLength(clientId) +
     shortStringLength(groupId) +
-    (if (versionId >= 1) 4 /* group generation id */ + shortStringLength(consumerId) else 0) +
+    (if (versionId >= 1) 4 /* group generation id */ + shortStringLength(memberId) else 0) +
     (if (versionId >= 2) 8 /* retention time */ else 0) +
     4 + /* topic count */
     requestInfoGroupedByTopic.foldLeft(0)((count, topicAndOffsets) => {
@@ -175,7 +175,7 @@ case class OffsetCommitRequest(groupId: String,
     offsetCommitRequest.append("; ClientId: " + clientId)
     offsetCommitRequest.append("; GroupId: " + groupId)
     offsetCommitRequest.append("; GroupGenerationId: " + groupGenerationId)
-    offsetCommitRequest.append("; ConsumerId: " + consumerId)
+    offsetCommitRequest.append("; MemberId: " + memberId)
     offsetCommitRequest.append("; RetentionMs: " + retentionMs)
     if(details)
       offsetCommitRequest.append("; RequestInfo: " + requestInfo.mkString(","))

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/api/RequestKeys.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/RequestKeys.scala b/core/src/main/scala/kafka/api/RequestKeys.scala
index 8a22c1a..669b63a 100644
--- a/core/src/main/scala/kafka/api/RequestKeys.scala
+++ b/core/src/main/scala/kafka/api/RequestKeys.scala
@@ -33,10 +33,11 @@ object RequestKeys {
   val ControlledShutdownKey: Short = 7
   val OffsetCommitKey: Short = 8
   val OffsetFetchKey: Short = 9
-  val ConsumerMetadataKey: Short = 10
+  val GroupMetadataKey: Short = 10
   val JoinGroupKey: Short = 11
   val HeartbeatKey: Short = 12
   val LeaveGroupKey: Short = 13
+  val SyncGroupKey: Short = 14
 
   val keyToNameAndDeserializerMap: Map[Short, (String, (ByteBuffer) => RequestOrResponse)]=
     Map(ProduceKey -> ("Produce", ProducerRequest.readFrom),
@@ -49,7 +50,7 @@ object RequestKeys {
         ControlledShutdownKey -> ("ControlledShutdown", ControlledShutdownRequest.readFrom),
         OffsetCommitKey -> ("OffsetCommit", OffsetCommitRequest.readFrom),
         OffsetFetchKey -> ("OffsetFetch", OffsetFetchRequest.readFrom),
-        ConsumerMetadataKey -> ("ConsumerMetadata", ConsumerMetadataRequest.readFrom)
+        GroupMetadataKey -> ("GroupMetadata", GroupMetadataRequest.readFrom)
     )
 
   def nameForKey(key: Short): String = {

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/client/ClientUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala
index 6ae0347..36b5b3b 100755
--- a/core/src/main/scala/kafka/client/ClientUtils.scala
+++ b/core/src/main/scala/kafka/client/ClientUtils.scala
@@ -151,9 +151,9 @@ object ClientUtils extends Logging{
            if (!queryChannel.isConnected)
              queryChannel = channelToAnyBroker(zkUtils)
            debug("Querying %s:%d to locate offset manager for %s.".format(queryChannel.host, queryChannel.port, group))
-           queryChannel.send(ConsumerMetadataRequest(group))
+           queryChannel.send(GroupMetadataRequest(group))
            val response = queryChannel.receive()
-           val consumerMetadataResponse =  ConsumerMetadataResponse.readFrom(response.payload())
+           val consumerMetadataResponse =  GroupMetadataResponse.readFrom(response.payload())
            debug("Consumer metadata response: " + consumerMetadataResponse.toString)
            if (consumerMetadataResponse.errorCode == ErrorMapping.NoError)
              coordinatorOpt = consumerMetadataResponse.coordinatorOpt

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala
index deb48b1..bbee894 100644
--- a/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala
+++ b/core/src/main/scala/kafka/common/OffsetMetadataAndError.scala
@@ -64,8 +64,9 @@ case class OffsetMetadataAndError(offsetMetadata: OffsetMetadata, error: Short =
 object OffsetMetadataAndError {
   val NoOffset = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.NONE.code)
   val OffsetsLoading = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.OFFSET_LOAD_IN_PROGRESS.code)
-  val UnknownConsumer = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.UNKNOWN_CONSUMER_ID.code)
-  val NotCoordinatorForGroup = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.NOT_COORDINATOR_FOR_CONSUMER.code)
+  val UnknownMember = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.UNKNOWN_MEMBER_ID.code)
+  val NotCoordinatorForGroup = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.NOT_COORDINATOR_FOR_GROUP.code)
+  val GroupCoordinatorNotAvailable = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code)
   val UnknownTopicOrPartition = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.UNKNOWN_TOPIC_OR_PARTITION.code)
   val IllegalGroupGenerationId = OffsetMetadataAndError(OffsetMetadata.InvalidOffsetMetadata, Errors.ILLEGAL_GENERATION.code)
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/common/Topic.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/common/Topic.scala b/core/src/main/scala/kafka/common/Topic.scala
index db75d4b..ca41eba 100644
--- a/core/src/main/scala/kafka/common/Topic.scala
+++ b/core/src/main/scala/kafka/common/Topic.scala
@@ -18,7 +18,7 @@
 package kafka.common
 
 import util.matching.Regex
-import kafka.coordinator.ConsumerCoordinator
+import kafka.coordinator.GroupCoordinator
 
 
 object Topic {
@@ -26,7 +26,7 @@ object Topic {
   private val maxNameLength = 255
   private val rgx = new Regex(legalChars + "+")
 
-  val InternalTopics = Set(ConsumerCoordinator.OffsetsTopicName)
+  val InternalTopics = Set(GroupCoordinator.OffsetsTopicName)
 
   def validate(topic: String) {
     if (topic.length <= 0)

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/consumer/SimpleConsumer.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala
index b7af6d6..5b1aead 100644
--- a/core/src/main/scala/kafka/consumer/SimpleConsumer.scala
+++ b/core/src/main/scala/kafka/consumer/SimpleConsumer.scala
@@ -112,9 +112,9 @@ class SimpleConsumer(val host: String,
     TopicMetadataResponse.readFrom(response.payload())
   }
 
-  def send(request: ConsumerMetadataRequest): ConsumerMetadataResponse = {
+  def send(request: GroupMetadataRequest): GroupMetadataResponse = {
     val response = sendRequest(request)
-    ConsumerMetadataResponse.readFrom(response.payload())
+    GroupMetadataResponse.readFrom(response.payload())
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala b/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala
deleted file mode 100644
index bf23e9b..0000000
--- a/core/src/main/scala/kafka/coordinator/ConsumerCoordinator.scala
+++ /dev/null
@@ -1,535 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package kafka.coordinator
-
-import kafka.common.{OffsetMetadataAndError, OffsetAndMetadata, TopicAndPartition}
-import kafka.message.UncompressedCodec
-import kafka.log.LogConfig
-import kafka.server._
-import kafka.utils._
-import org.apache.kafka.common.protocol.Errors
-import org.apache.kafka.common.requests.JoinGroupRequest
-
-import org.I0Itec.zkclient.ZkClient
-import java.util.concurrent.atomic.AtomicBoolean
-import java.util.Properties
-import scala.collection.{Map, Seq, immutable}
-
-case class GroupManagerConfig(consumerMinSessionTimeoutMs: Int,
-                              consumerMaxSessionTimeoutMs: Int)
-
-/**
- * ConsumerCoordinator handles consumer group and consumer offset management.
- *
- * Each Kafka server instantiates a coordinator which is responsible for a set of
- * consumer groups. Consumer groups are assigned to coordinators based on their
- * group names.
- */
-class ConsumerCoordinator(val brokerId: Int,
-                          val groupConfig: GroupManagerConfig,
-                          val offsetConfig: OffsetManagerConfig,
-                          private val offsetManager: OffsetManager,
-                          zkUtils: ZkUtils) extends Logging {
-
-  this.logIdent = "[ConsumerCoordinator " + brokerId + "]: "
-
-  private val isActive = new AtomicBoolean(false)
-
-  private var heartbeatPurgatory: DelayedOperationPurgatory[DelayedHeartbeat] = null
-  private var rebalancePurgatory: DelayedOperationPurgatory[DelayedRebalance] = null
-  private var coordinatorMetadata: CoordinatorMetadata = null
-
-  def this(brokerId: Int,
-           groupConfig: GroupManagerConfig,
-           offsetConfig: OffsetManagerConfig,
-           replicaManager: ReplicaManager,
-           zkUtils: ZkUtils,
-           scheduler: KafkaScheduler) = this(brokerId, groupConfig, offsetConfig,
-    new OffsetManager(offsetConfig, replicaManager, zkUtils, scheduler), zkUtils)
-
-  def offsetsTopicConfigs: Properties = {
-    val props = new Properties
-    props.put(LogConfig.CleanupPolicyProp, LogConfig.Compact)
-    props.put(LogConfig.SegmentBytesProp, offsetConfig.offsetsTopicSegmentBytes.toString)
-    props.put(LogConfig.CompressionTypeProp, UncompressedCodec.name)
-    props
-  }
-
-  /**
-   * NOTE: If a group lock and metadataLock are simultaneously needed,
-   * be sure to acquire the group lock before metadataLock to prevent deadlock
-   */
-
-  /**
-   * Startup logic executed at the same time when the server starts up.
-   */
-  def startup() {
-    info("Starting up.")
-    heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", brokerId)
-    rebalancePurgatory = new DelayedOperationPurgatory[DelayedRebalance]("Rebalance", brokerId)
-    coordinatorMetadata = new CoordinatorMetadata(brokerId, zkUtils, maybePrepareRebalance)
-    isActive.set(true)
-    info("Startup complete.")
-  }
-
-  /**
-   * Shutdown logic executed at the same time when server shuts down.
-   * Ordering of actions should be reversed from the startup process.
-   */
-  def shutdown() {
-    info("Shutting down.")
-    isActive.set(false)
-    offsetManager.shutdown()
-    coordinatorMetadata.shutdown()
-    heartbeatPurgatory.shutdown()
-    rebalancePurgatory.shutdown()
-    info("Shutdown complete.")
-  }
-
-  def handleJoinGroup(groupId: String,
-                      consumerId: String,
-                      topics: Set[String],
-                      sessionTimeoutMs: Int,
-                      partitionAssignmentStrategy: String,
-                      responseCallback:(Set[TopicAndPartition], String, Int, Short) => Unit) {
-    if (!isActive.get) {
-      responseCallback(Set.empty, consumerId, 0, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code)
-    } else if (!isCoordinatorForGroup(groupId)) {
-      responseCallback(Set.empty, consumerId, 0, Errors.NOT_COORDINATOR_FOR_CONSUMER.code)
-    } else if (!PartitionAssignor.strategies.contains(partitionAssignmentStrategy)) {
-      responseCallback(Set.empty, consumerId, 0, Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code)
-    } else if (sessionTimeoutMs < groupConfig.consumerMinSessionTimeoutMs ||
-               sessionTimeoutMs > groupConfig.consumerMaxSessionTimeoutMs) {
-      responseCallback(Set.empty, consumerId, 0, Errors.INVALID_SESSION_TIMEOUT.code)
-    } else {
-      // only try to create the group if the group is not unknown AND
-      // the consumer id is UNKNOWN, if consumer is specified but group does not
-      // exist we should reject the request
-      var group = coordinatorMetadata.getGroup(groupId)
-      if (group == null) {
-        if (consumerId != JoinGroupRequest.UNKNOWN_CONSUMER_ID) {
-          responseCallback(Set.empty, consumerId, 0, Errors.UNKNOWN_CONSUMER_ID.code)
-        } else {
-          group = coordinatorMetadata.addGroup(groupId, partitionAssignmentStrategy)
-          doJoinGroup(group, consumerId, topics, sessionTimeoutMs, partitionAssignmentStrategy, responseCallback)
-        }
-      } else {
-        doJoinGroup(group, consumerId, topics, sessionTimeoutMs, partitionAssignmentStrategy, responseCallback)
-      }
-    }
-  }
-
-  private def doJoinGroup(group: ConsumerGroupMetadata,
-                          consumerId: String,
-                          topics: Set[String],
-                          sessionTimeoutMs: Int,
-                          partitionAssignmentStrategy: String,
-                          responseCallback:(Set[TopicAndPartition], String, Int, Short) => Unit) {
-    group synchronized {
-      if (group.is(Dead)) {
-        // if the group is marked as dead, it means some other thread has just removed the group
-        // from the coordinator metadata; this is likely that the group has migrated to some other
-        // coordinator OR the group is in a transient unstable phase. Let the consumer to retry
-        // joining without specified consumer id,
-        responseCallback(Set.empty, consumerId, 0, Errors.UNKNOWN_CONSUMER_ID.code)
-      } else if (partitionAssignmentStrategy != group.partitionAssignmentStrategy) {
-        responseCallback(Set.empty, consumerId, 0, Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code)
-      } else if (consumerId != JoinGroupRequest.UNKNOWN_CONSUMER_ID && !group.has(consumerId)) {
-        // if the consumer trying to register with a un-recognized id, send the response to let
-        // it reset its consumer id and retry
-        responseCallback(Set.empty, consumerId, 0, Errors.UNKNOWN_CONSUMER_ID.code)
-      } else if (group.has(consumerId) && group.is(Stable) && topics == group.get(consumerId).topics) {
-        /*
-         * if an existing consumer sends a JoinGroupRequest with no changes while the group is stable,
-         * just treat it like a heartbeat and return their currently assigned partitions.
-         */
-        val consumer = group.get(consumerId)
-        completeAndScheduleNextHeartbeatExpiration(group, consumer)
-        responseCallback(consumer.assignedTopicPartitions, consumerId, group.generationId, Errors.NONE.code)
-      } else {
-        val consumer = if (consumerId == JoinGroupRequest.UNKNOWN_CONSUMER_ID) {
-          // if the consumer id is unknown, register this consumer to the group
-          val generatedConsumerId = group.generateNextConsumerId
-          val consumer = addConsumer(generatedConsumerId, topics, sessionTimeoutMs, group)
-          maybePrepareRebalance(group)
-          consumer
-        } else {
-          val consumer = group.get(consumerId)
-          if (topics != consumer.topics) {
-            // existing consumer changed its subscribed topics
-            updateConsumer(group, consumer, topics)
-            maybePrepareRebalance(group)
-            consumer
-          } else {
-            // existing consumer rejoining a group due to rebalance
-            consumer
-          }
-        }
-
-        consumer.awaitingRebalanceCallback = responseCallback
-
-        if (group.is(PreparingRebalance))
-          rebalancePurgatory.checkAndComplete(ConsumerGroupKey(group.groupId))
-      }
-    }
-  }
-
-  def handleLeaveGroup(groupId: String, consumerId: String, responseCallback: Short => Unit) {
-    if (!isActive.get) {
-      responseCallback(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code)
-    } else if (!isCoordinatorForGroup(groupId)) {
-      responseCallback(Errors.NOT_COORDINATOR_FOR_CONSUMER.code)
-    } else {
-      val group = coordinatorMetadata.getGroup(groupId)
-      if (group == null) {
-        // if the group is marked as dead, it means some other thread has just removed the group
-        // from the coordinator metadata; this is likely that the group has migrated to some other
-        // coordinator OR the group is in a transient unstable phase. Let the consumer to retry
-        // joining without specified consumer id,
-        responseCallback(Errors.UNKNOWN_CONSUMER_ID.code)
-      } else {
-        group synchronized {
-          if (group.is(Dead)) {
-            responseCallback(Errors.UNKNOWN_CONSUMER_ID.code)
-          } else if (!group.has(consumerId)) {
-            responseCallback(Errors.UNKNOWN_CONSUMER_ID.code)
-          } else {
-            val consumer = group.get(consumerId)
-            removeHeartbeatForLeavingConsumer(group, consumer)
-            onConsumerFailure(group, consumer)
-            responseCallback(Errors.NONE.code)
-            if (group.is(PreparingRebalance))
-              rebalancePurgatory.checkAndComplete(ConsumerGroupKey(group.groupId))
-          }
-        }
-      }
-    }
-  }
-
-  def handleHeartbeat(groupId: String,
-                      consumerId: String,
-                      generationId: Int,
-                      responseCallback: Short => Unit) {
-    if (!isActive.get) {
-      responseCallback(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code)
-    } else if (!isCoordinatorForGroup(groupId)) {
-      responseCallback(Errors.NOT_COORDINATOR_FOR_CONSUMER.code)
-    } else {
-      val group = coordinatorMetadata.getGroup(groupId)
-      if (group == null) {
-        // if the group is marked as dead, it means some other thread has just removed the group
-        // from the coordinator metadata; this is likely that the group has migrated to some other
-        // coordinator OR the group is in a transient unstable phase. Let the consumer to retry
-        // joining without specified consumer id,
-        responseCallback(Errors.UNKNOWN_CONSUMER_ID.code)
-      } else {
-        group synchronized {
-          if (group.is(Dead)) {
-            responseCallback(Errors.UNKNOWN_CONSUMER_ID.code)
-          } else if (!group.has(consumerId)) {
-            responseCallback(Errors.UNKNOWN_CONSUMER_ID.code)
-          } else if (generationId != group.generationId) {
-            responseCallback(Errors.ILLEGAL_GENERATION.code)
-          } else if (!group.is(Stable)) {
-            responseCallback(Errors.REBALANCE_IN_PROGRESS.code)
-          } else {
-            val consumer = group.get(consumerId)
-            completeAndScheduleNextHeartbeatExpiration(group, consumer)
-            responseCallback(Errors.NONE.code)
-          }
-        }
-      }
-    }
-  }
-
-  def handleCommitOffsets(groupId: String,
-                          consumerId: String,
-                          generationId: Int,
-                          offsetMetadata: immutable.Map[TopicAndPartition, OffsetAndMetadata],
-                          responseCallback: immutable.Map[TopicAndPartition, Short] => Unit) {
-    if (!isActive.get) {
-      responseCallback(offsetMetadata.mapValues(_ => Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code))
-    } else if (!isCoordinatorForGroup(groupId)) {
-      responseCallback(offsetMetadata.mapValues(_ => Errors.NOT_COORDINATOR_FOR_CONSUMER.code))
-    } else {
-      val group = coordinatorMetadata.getGroup(groupId)
-      if (group == null) {
-        if (generationId < 0)
-          // the group is not relying on Kafka for partition management, so allow the commit
-          offsetManager.storeOffsets(groupId, consumerId, generationId, offsetMetadata, responseCallback)
-        else
-          // the group has failed over to this coordinator (which will be handled in KAFKA-2017),
-          // or this is a request coming from an older generation. either way, reject the commit
-          responseCallback(offsetMetadata.mapValues(_ => Errors.ILLEGAL_GENERATION.code))
-      } else {
-        group synchronized {
-          if (group.is(Dead)) {
-            responseCallback(offsetMetadata.mapValues(_ => Errors.UNKNOWN_CONSUMER_ID.code))
-          } else if (!group.has(consumerId)) {
-            responseCallback(offsetMetadata.mapValues(_ => Errors.UNKNOWN_CONSUMER_ID.code))
-          } else if (generationId != group.generationId) {
-            responseCallback(offsetMetadata.mapValues(_ => Errors.ILLEGAL_GENERATION.code))
-          } else if (!offsetMetadata.keySet.subsetOf(group.get(consumerId).assignedTopicPartitions)) {
-            responseCallback(offsetMetadata.mapValues(_ => Errors.COMMITTING_PARTITIONS_NOT_ASSIGNED.code))
-          } else {
-            offsetManager.storeOffsets(groupId, consumerId, generationId, offsetMetadata, responseCallback)
-          }
-        }
-      }
-    }
-  }
-
-  def handleFetchOffsets(groupId: String,
-                         partitions: Seq[TopicAndPartition]): Map[TopicAndPartition, OffsetMetadataAndError] = {
-    if (!isActive.get) {
-      partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.NotCoordinatorForGroup)}.toMap
-    } else if (!isCoordinatorForGroup(groupId)) {
-      partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.NotCoordinatorForGroup)}.toMap
-    } else {
-      val group = coordinatorMetadata.getGroup(groupId)
-      if (group == null) {
-        // if the group does not exist, it means this group is not relying
-        // on Kafka for partition management, and hence never send join-group
-        // request to the coordinator before; in this case blindly fetch the offsets
-        offsetManager.getOffsets(groupId, partitions)
-      } else {
-        group synchronized {
-          if (group.is(Dead)) {
-            partitions.map {case topicAndPartition => (topicAndPartition, OffsetMetadataAndError.UnknownConsumer)}.toMap
-          } else {
-            offsetManager.getOffsets(groupId, partitions)
-          }
-        }
-      }
-    }
-  }
-
-  def handleGroupImmigration(offsetTopicPartitionId: Int) = {
-    // TODO we may need to add more logic in KAFKA-2017
-    offsetManager.loadOffsetsFromLog(offsetTopicPartitionId)
-  }
-
-  def handleGroupEmigration(offsetTopicPartitionId: Int) = {
-    // TODO we may need to add more logic in KAFKA-2017
-    offsetManager.removeOffsetsFromCacheForPartition(offsetTopicPartitionId)
-  }
-
-  /**
-   * Complete existing DelayedHeartbeats for the given consumer and schedule the next one
-   */
-  private def completeAndScheduleNextHeartbeatExpiration(group: ConsumerGroupMetadata, consumer: ConsumerMetadata) {
-    // complete current heartbeat expectation
-    consumer.latestHeartbeat = SystemTime.milliseconds
-    val consumerKey = ConsumerKey(consumer.groupId, consumer.consumerId)
-    heartbeatPurgatory.checkAndComplete(consumerKey)
-
-    // reschedule the next heartbeat expiration deadline
-    val newHeartbeatDeadline = consumer.latestHeartbeat + consumer.sessionTimeoutMs
-    val delayedHeartbeat = new DelayedHeartbeat(this, group, consumer, newHeartbeatDeadline, consumer.sessionTimeoutMs)
-    heartbeatPurgatory.tryCompleteElseWatch(delayedHeartbeat, Seq(consumerKey))
-  }
-
-  private def removeHeartbeatForLeavingConsumer(group: ConsumerGroupMetadata, consumer: ConsumerMetadata) {
-    consumer.isLeaving = true
-    val consumerKey = ConsumerKey(consumer.groupId, consumer.consumerId)
-    heartbeatPurgatory.checkAndComplete(consumerKey)
-  }
-
-  private def addConsumer(consumerId: String,
-                          topics: Set[String],
-                          sessionTimeoutMs: Int,
-                          group: ConsumerGroupMetadata) = {
-    val consumer = new ConsumerMetadata(consumerId, group.groupId, topics, sessionTimeoutMs)
-    val topicsToBind = topics -- group.topics
-    group.add(consumer.consumerId, consumer)
-    coordinatorMetadata.bindGroupToTopics(group.groupId, topicsToBind)
-    consumer
-  }
-
-  private def removeConsumer(group: ConsumerGroupMetadata, consumer: ConsumerMetadata) {
-    group.remove(consumer.consumerId)
-    val topicsToUnbind = consumer.topics -- group.topics
-    coordinatorMetadata.unbindGroupFromTopics(group.groupId, topicsToUnbind)
-  }
-
-  private def updateConsumer(group: ConsumerGroupMetadata, consumer: ConsumerMetadata, topics: Set[String]) {
-    val topicsToBind = topics -- group.topics
-    group.remove(consumer.consumerId)
-    val topicsToUnbind = consumer.topics -- (group.topics ++ topics)
-    group.add(consumer.consumerId, consumer)
-    consumer.topics = topics
-    coordinatorMetadata.bindAndUnbindGroupFromTopics(group.groupId, topicsToBind, topicsToUnbind)
-  }
-
-  private def maybePrepareRebalance(group: ConsumerGroupMetadata) {
-    group synchronized {
-      if (group.canRebalance)
-        prepareRebalance(group)
-    }
-  }
-
-  private def prepareRebalance(group: ConsumerGroupMetadata) {
-    group.transitionTo(PreparingRebalance)
-    info("Preparing to rebalance group %s with old generation %s".format(group.groupId, group.generationId))
-
-    val rebalanceTimeout = group.rebalanceTimeout
-    val delayedRebalance = new DelayedRebalance(this, group, rebalanceTimeout)
-    val consumerGroupKey = ConsumerGroupKey(group.groupId)
-    rebalancePurgatory.tryCompleteElseWatch(delayedRebalance, Seq(consumerGroupKey))
-  }
-
-  private def rebalance(group: ConsumerGroupMetadata) {
-    assert(group.notYetRejoinedConsumers == List.empty[ConsumerMetadata])
-
-    group.transitionTo(Rebalancing)
-    group.generationId += 1
-
-    info("Rebalancing group %s with new generation %s".format(group.groupId, group.generationId))
-
-    val assignedPartitionsPerConsumer = reassignPartitions(group)
-    trace("Rebalance for group %s generation %s has assigned partitions: %s"
-          .format(group.groupId, group.generationId, assignedPartitionsPerConsumer))
-
-    group.transitionTo(Stable)
-    info("Stabilized group %s generation %s".format(group.groupId, group.generationId))
-  }
-
-  private def onConsumerFailure(group: ConsumerGroupMetadata, consumer: ConsumerMetadata) {
-    trace("Consumer %s in group %s has failed".format(consumer.consumerId, group.groupId))
-    removeConsumer(group, consumer)
-    maybePrepareRebalance(group)
-  }
-
-  private def reassignPartitions(group: ConsumerGroupMetadata) = {
-    val assignor = PartitionAssignor.createInstance(group.partitionAssignmentStrategy)
-    val topicsPerConsumer = group.topicsPerConsumer
-    val partitionsPerTopic = coordinatorMetadata.partitionsPerTopic
-    val assignedPartitionsPerConsumer = assignor.assign(topicsPerConsumer, partitionsPerTopic)
-    assignedPartitionsPerConsumer.foreach { case (consumerId, partitions) =>
-      group.get(consumerId).assignedTopicPartitions = partitions
-    }
-    assignedPartitionsPerConsumer
-  }
-
-  def tryCompleteRebalance(group: ConsumerGroupMetadata, forceComplete: () => Boolean) = {
-    group synchronized {
-      if (group.notYetRejoinedConsumers.isEmpty)
-        forceComplete()
-      else false
-    }
-  }
-
-  def onExpirationRebalance() {
-    // TODO: add metrics for rebalance timeouts
-  }
-
-  def onCompleteRebalance(group: ConsumerGroupMetadata) {
-    group synchronized {
-      val failedConsumers = group.notYetRejoinedConsumers
-      if (group.isEmpty || !failedConsumers.isEmpty) {
-        failedConsumers.foreach { failedConsumer =>
-          removeConsumer(group, failedConsumer)
-          // TODO: cut the socket connection to the consumer
-        }
-
-        if (group.isEmpty) {
-          group.transitionTo(Dead)
-          info("Group %s generation %s is dead and removed".format(group.groupId, group.generationId))
-          coordinatorMetadata.removeGroup(group.groupId, group.topics)
-        }
-      }
-      if (!group.is(Dead)) {
-        // assign partitions to existing consumers of the group according to the partitioning strategy
-        rebalance(group)
-
-        // trigger the awaiting join group response callback for all the consumers after rebalancing
-        for (consumer <- group.allConsumers) {
-          assert(consumer.awaitingRebalanceCallback != null)
-          consumer.awaitingRebalanceCallback(consumer.assignedTopicPartitions, consumer.consumerId, group.generationId, Errors.NONE.code)
-          consumer.awaitingRebalanceCallback = null
-          completeAndScheduleNextHeartbeatExpiration(group, consumer)
-        }
-      }
-    }
-  }
-
-  def tryCompleteHeartbeat(group: ConsumerGroupMetadata, consumer: ConsumerMetadata, heartbeatDeadline: Long, forceComplete: () => Boolean) = {
-    group synchronized {
-      if (shouldKeepConsumerAlive(consumer, heartbeatDeadline) || consumer.isLeaving)
-        forceComplete()
-      else false
-    }
-  }
-
-  def onExpirationHeartbeat(group: ConsumerGroupMetadata, consumer: ConsumerMetadata, heartbeatDeadline: Long) {
-    group synchronized {
-      if (!shouldKeepConsumerAlive(consumer, heartbeatDeadline))
-        onConsumerFailure(group, consumer)
-    }
-  }
-
-  def onCompleteHeartbeat() {
-    // TODO: add metrics for complete heartbeats
-  }
-
-  def partitionFor(group: String): Int = offsetManager.partitionFor(group)
-
-  private def shouldKeepConsumerAlive(consumer: ConsumerMetadata, heartbeatDeadline: Long) =
-    consumer.awaitingRebalanceCallback != null || consumer.latestHeartbeat + consumer.sessionTimeoutMs > heartbeatDeadline
-
-  private def isCoordinatorForGroup(groupId: String) = offsetManager.leaderIsLocal(offsetManager.partitionFor(groupId))
-}
-
-object ConsumerCoordinator {
-
-  val OffsetsTopicName = "__consumer_offsets"
-
-  def create(config: KafkaConfig,
-             zkUtils: ZkUtils,
-             replicaManager: ReplicaManager,
-             kafkaScheduler: KafkaScheduler): ConsumerCoordinator = {
-    val offsetConfig = OffsetManagerConfig(maxMetadataSize = config.offsetMetadataMaxSize,
-      loadBufferSize = config.offsetsLoadBufferSize,
-      offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L,
-      offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs,
-      offsetsTopicNumPartitions = config.offsetsTopicPartitions,
-      offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor,
-      offsetCommitTimeoutMs = config.offsetCommitTimeoutMs,
-      offsetCommitRequiredAcks = config.offsetCommitRequiredAcks)
-    val groupConfig = GroupManagerConfig(consumerMinSessionTimeoutMs = config.consumerMinSessionTimeoutMs,
-      consumerMaxSessionTimeoutMs = config.consumerMaxSessionTimeoutMs)
-
-    new ConsumerCoordinator(config.brokerId, groupConfig, offsetConfig, replicaManager, zkUtils, kafkaScheduler)
-  }
-
-  def create(config: KafkaConfig,
-             zkUtils: ZkUtils,
-             offsetManager: OffsetManager): ConsumerCoordinator = {
-    val offsetConfig = OffsetManagerConfig(maxMetadataSize = config.offsetMetadataMaxSize,
-      loadBufferSize = config.offsetsLoadBufferSize,
-      offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L,
-      offsetsRetentionCheckIntervalMs = config.offsetsRetentionCheckIntervalMs,
-      offsetsTopicNumPartitions = config.offsetsTopicPartitions,
-      offsetsTopicReplicationFactor = config.offsetsTopicReplicationFactor,
-      offsetCommitTimeoutMs = config.offsetCommitTimeoutMs,
-      offsetCommitRequiredAcks = config.offsetCommitRequiredAcks)
-    val groupConfig = GroupManagerConfig(consumerMinSessionTimeoutMs = config.consumerMinSessionTimeoutMs,
-      consumerMaxSessionTimeoutMs = config.consumerMaxSessionTimeoutMs)
-
-    new ConsumerCoordinator(config.brokerId, groupConfig, offsetConfig, offsetManager, zkUtils)
-  }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/coordinator/ConsumerGroupMetadata.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/ConsumerGroupMetadata.scala b/core/src/main/scala/kafka/coordinator/ConsumerGroupMetadata.scala
deleted file mode 100644
index 0e3657f..0000000
--- a/core/src/main/scala/kafka/coordinator/ConsumerGroupMetadata.scala
+++ /dev/null
@@ -1,133 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package kafka.coordinator
-
-import kafka.utils.nonthreadsafe
-
-import java.util.UUID
-
-import collection.mutable
-
-private[coordinator] sealed trait GroupState { def state: Byte }
-
-/**
- * Consumer group is preparing to rebalance
- *
- * action: respond to heartbeats with an ILLEGAL GENERATION error code
- * transition: some consumers have joined by the timeout => Rebalancing
- *             all consumers have left the group => Dead
- */
-private[coordinator] case object PreparingRebalance extends GroupState { val state: Byte = 1 }
-
-/**
- * Consumer group is rebalancing
- *
- * action: compute the group's partition assignment
- *         send the join-group response with new partition assignment when rebalance is complete
- * transition: partition assignment has been computed => Stable
- */
-private[coordinator] case object Rebalancing extends GroupState { val state: Byte = 2 }
-
-/**
- * Consumer group is stable
- *
- * action: respond to consumer heartbeats normally
- * transition: consumer failure detected via heartbeat => PreparingRebalance
- *             consumer join-group received => PreparingRebalance
- *             zookeeper topic watcher fired => PreparingRebalance
- */
-private[coordinator] case object Stable extends GroupState { val state: Byte = 3 }
-
-/**
- * Consumer group has no more members
- *
- * action: none
- * transition: none
- */
-private[coordinator] case object Dead extends GroupState { val state: Byte = 4 }
-
-
-private object ConsumerGroupMetadata {
-  private val validPreviousStates: Map[GroupState, Set[GroupState]] =
-    Map(Dead -> Set(PreparingRebalance),
-      Stable -> Set(Rebalancing),
-      PreparingRebalance -> Set(Stable),
-      Rebalancing -> Set(PreparingRebalance))
-}
-
-/**
- * Group contains the following metadata:
- *
- *  Membership metadata:
- *  1. Consumers registered in this group
- *  2. Partition assignment strategy for this group
- *
- *  State metadata:
- *  1. group state
- *  2. generation id
- */
-@nonthreadsafe
-private[coordinator] class ConsumerGroupMetadata(val groupId: String,
-                                                 val partitionAssignmentStrategy: String) {
-
-  private val consumers = new mutable.HashMap[String, ConsumerMetadata]
-  private var state: GroupState = Stable
-  var generationId = 0
-
-  def is(groupState: GroupState) = state == groupState
-  def has(consumerId: String) = consumers.contains(consumerId)
-  def get(consumerId: String) = consumers(consumerId)
-
-  def add(consumerId: String, consumer: ConsumerMetadata) {
-    consumers.put(consumerId, consumer)
-  }
-
-  def remove(consumerId: String) {
-    consumers.remove(consumerId)
-  }
-
-  def isEmpty = consumers.isEmpty
-
-  def topicsPerConsumer = consumers.mapValues(_.topics).toMap
-
-  def topics = consumers.values.flatMap(_.topics).toSet
-
-  def notYetRejoinedConsumers = consumers.values.filter(_.awaitingRebalanceCallback == null).toList
-
-  def allConsumers = consumers.values.toList
-
-  def rebalanceTimeout = consumers.values.foldLeft(0) {(timeout, consumer) =>
-    timeout.max(consumer.sessionTimeoutMs)
-  }
-
-  // TODO: decide if ids should be predictable or random
-  def generateNextConsumerId = UUID.randomUUID().toString
-
-  def canRebalance = state == Stable
-
-  def transitionTo(groupState: GroupState) {
-    assertValidTransition(groupState)
-    state = groupState
-  }
-
-  private def assertValidTransition(targetState: GroupState) {
-    if (!ConsumerGroupMetadata.validPreviousStates(targetState).contains(state))
-      throw new IllegalStateException("Group %s should be in the %s states before moving to %s state. Instead it is in %s state"
-        .format(groupId, ConsumerGroupMetadata.validPreviousStates(targetState).mkString(","), targetState, state))
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/coordinator/ConsumerMetadata.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/ConsumerMetadata.scala b/core/src/main/scala/kafka/coordinator/ConsumerMetadata.scala
deleted file mode 100644
index 64ed4a5..0000000
--- a/core/src/main/scala/kafka/coordinator/ConsumerMetadata.scala
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package kafka.coordinator
-
-import kafka.common.TopicAndPartition
-import kafka.utils.nonthreadsafe
-
-/**
- * Consumer metadata contains the following metadata:
- *
- * Heartbeat metadata:
- * 1. negotiated heartbeat session timeout
- * 2. timestamp of the latest heartbeat
- *
- * Subscription metadata:
- * 1. subscribed topics
- * 2. assigned partitions for the subscribed topics
- *
- * In addition, it also contains the following state information:
- *
- * 1. Awaiting rebalance callback: when the consumer group is in the prepare-rebalance state,
- *                                 its rebalance callback will be kept in the metadata if the
- *                                 consumer has sent the join group request
- */
-@nonthreadsafe
-private[coordinator] class ConsumerMetadata(val consumerId: String,
-                                            val groupId: String,
-                                            var topics: Set[String],
-                                            val sessionTimeoutMs: Int) {
-
-  var awaitingRebalanceCallback: (Set[TopicAndPartition], String, Int, Short) => Unit = null
-  var assignedTopicPartitions = Set.empty[TopicAndPartition]
-  var latestHeartbeat: Long = -1
-  var isLeaving: Boolean = false
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala b/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala
index a33231a..2279924 100644
--- a/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala
+++ b/core/src/main/scala/kafka/coordinator/CoordinatorMetadata.scala
@@ -17,11 +17,8 @@
 
 package kafka.coordinator
 
-import kafka.server.KafkaConfig
 import kafka.utils.CoreUtils.{inReadLock, inWriteLock}
-import kafka.utils.{threadsafe, ZkUtils, Logging}
-import kafka.utils.ZkUtils._
-import org.I0Itec.zkclient.{ZkClient, IZkDataListener}
+import kafka.utils.threadsafe
 
 import java.util.concurrent.locks.ReentrantReadWriteLock
 
@@ -32,9 +29,7 @@ import scala.collection.mutable
  * It delegates all group logic to the callers.
  */
 @threadsafe
-private[coordinator] class CoordinatorMetadata(brokerId: Int,
-                                               zkUtils: ZkUtils,
-                                               maybePrepareRebalance: ConsumerGroupMetadata => Unit) {
+private[coordinator] class CoordinatorMetadata(brokerId: Int) {
 
   /**
    * NOTE: If a group lock and metadataLock are simultaneously needed,
@@ -45,24 +40,11 @@ private[coordinator] class CoordinatorMetadata(brokerId: Int,
   /**
    * These should be guarded by metadataLock
    */
-  private val groups = new mutable.HashMap[String, ConsumerGroupMetadata]
-  private val groupsPerTopic = new mutable.HashMap[String, Set[String]]
-  private val topicPartitionCounts = new mutable.HashMap[String, Int]
-  private val topicPartitionChangeListeners = new mutable.HashMap[String, TopicPartitionChangeListener]
+  private val groups = new mutable.HashMap[String, GroupMetadata]
 
   def shutdown() {
     inWriteLock(metadataLock) {
-      topicPartitionChangeListeners.keys.foreach(deregisterTopicPartitionChangeListener)
-      topicPartitionChangeListeners.clear()
       groups.clear()
-      groupsPerTopic.clear()
-      topicPartitionCounts.clear()
-    }
-  }
-
-  def partitionsPerTopic = {
-    inReadLock(metadataLock) {
-      topicPartitionCounts.toMap
     }
   }
 
@@ -78,148 +60,22 @@ private[coordinator] class CoordinatorMetadata(brokerId: Int,
   /**
    * Add a group or get the group associated with the given groupId if it already exists
    */
-  def addGroup(groupId: String, partitionAssignmentStrategy: String) = {
+  def addGroup(groupId: String, protocolType: String) = {
     inWriteLock(metadataLock) {
-      groups.getOrElseUpdate(groupId, new ConsumerGroupMetadata(groupId, partitionAssignmentStrategy))
+      groups.getOrElseUpdate(groupId, new GroupMetadata(groupId, protocolType))
     }
   }
 
   /**
    * Remove all metadata associated with the group, including its topics
    * @param groupId the groupId of the group we are removing
-   * @param topicsForGroup topics that consumers in the group were subscribed to
    */
-  def removeGroup(groupId: String, topicsForGroup: Set[String]) {
+  def removeGroup(groupId: String) {
     inWriteLock(metadataLock) {
-      topicsForGroup.foreach(topic => unbindGroupFromTopics(groupId, topicsForGroup))
+      if (!groups.contains(groupId))
+        throw new IllegalArgumentException("Cannot remove non-existing group")
       groups.remove(groupId)
     }
   }
 
-  /**
-   * Add the given group to the set of groups interested in
-   * topic partition changes for the given topics
-   */
-  def bindGroupToTopics(groupId: String, topics: Set[String]) {
-    inWriteLock(metadataLock) {
-      require(groups.contains(groupId), "CoordinatorMetadata can only bind existing groups")
-      topics.foreach(topic => bindGroupToTopic(groupId, topic))
-    }
-  }
-
-  /**
-   * Remove the given group from the set of groups interested in
-   * topic partition changes for the given topics
-   */
-  def unbindGroupFromTopics(groupId: String, topics: Set[String]) {
-    inWriteLock(metadataLock) {
-      require(groups.contains(groupId), "CoordinatorMetadata can only unbind existing groups")
-      topics.foreach(topic => unbindGroupFromTopic(groupId, topic))
-    }
-  }
-
-  /**
-   * Add the given group to the set of groups interested in the topicsToBind and
-   * remove the given group from the set of groups interested in the topicsToUnbind
-   */
-  def bindAndUnbindGroupFromTopics(groupId: String, topicsToBind: Set[String], topicsToUnbind: Set[String]) {
-    inWriteLock(metadataLock) {
-      require(groups.contains(groupId), "CoordinatorMetadata can only update topic bindings for existing groups")
-      topicsToBind.foreach(topic => bindGroupToTopic(groupId, topic))
-      topicsToUnbind.foreach(topic => unbindGroupFromTopic(groupId, topic))
-    }
-  }
-
-  private def isListeningToTopic(topic: String) = topicPartitionChangeListeners.contains(topic)
-
-  private def bindGroupToTopic(groupId: String, topic: String) {
-    if (isListeningToTopic(topic)) {
-      val currentGroupsForTopic = groupsPerTopic(topic)
-      groupsPerTopic.put(topic, currentGroupsForTopic + groupId)
-    }
-    else {
-      groupsPerTopic.put(topic, Set(groupId))
-      topicPartitionCounts.put(topic, getTopicPartitionCountFromZK(topic))
-      registerTopicPartitionChangeListener(topic)
-    }
-  }
-
-  private def unbindGroupFromTopic(groupId: String, topic: String) {
-    if (isListeningToTopic(topic)) {
-      val remainingGroupsForTopic = groupsPerTopic(topic) - groupId
-      if (remainingGroupsForTopic.isEmpty) {
-        // no other group cares about the topic, so erase all metadata associated with the topic
-        groupsPerTopic.remove(topic)
-        topicPartitionCounts.remove(topic)
-        deregisterTopicPartitionChangeListener(topic)
-      } else {
-        groupsPerTopic.put(topic, remainingGroupsForTopic)
-      }
-    }
-  }
-
-  private def getTopicPartitionCountFromZK(topic: String) = {
-    val topicData = zkUtils.getPartitionAssignmentForTopics(Seq(topic))
-    topicData(topic).size
-  }
-
-  private def registerTopicPartitionChangeListener(topic: String) {
-    val listener = new TopicPartitionChangeListener
-    topicPartitionChangeListeners.put(topic, listener)
-    zkUtils.zkClient.subscribeDataChanges(getTopicPath(topic), listener)
-  }
-
-  private def deregisterTopicPartitionChangeListener(topic: String) {
-    val listener = topicPartitionChangeListeners(topic)
-    zkUtils.zkClient.unsubscribeDataChanges(getTopicPath(topic), listener)
-    topicPartitionChangeListeners.remove(topic)
-  }
-
-  /**
-   * Zookeeper listener to handle topic partition changes
-   */
-  class TopicPartitionChangeListener extends IZkDataListener with Logging {
-    this.logIdent = "[TopicPartitionChangeListener on Coordinator " + brokerId + "]: "
-
-    override def handleDataChange(dataPath: String, data: Object) {
-      info("Handling data change for path: %s data: %s".format(dataPath, data))
-      val topic = topicFromDataPath(dataPath)
-      val numPartitions = getTopicPartitionCountFromZK(topic)
-
-      val groupsToRebalance = inWriteLock(metadataLock) {
-        /*
-         * This condition exists because a consumer can leave and modify CoordinatorMetadata state
-         * while ZkClient begins handling the data change but before we acquire the metadataLock.
-         */
-        if (isListeningToTopic(topic)) {
-          topicPartitionCounts.put(topic, numPartitions)
-          groupsPerTopic(topic).map(groupId => groups(groupId))
-        }
-        else Set.empty[ConsumerGroupMetadata]
-      }
-      groupsToRebalance.foreach(maybePrepareRebalance)
-    }
-
-    override def handleDataDeleted(dataPath: String) {
-      info("Handling data delete for path: %s".format(dataPath))
-      val topic = topicFromDataPath(dataPath)
-      val groupsToRebalance = inWriteLock(metadataLock) {
-        /*
-         * This condition exists because a consumer can leave and modify CoordinatorMetadata state
-         * while ZkClient begins handling the data delete but before we acquire the metadataLock.
-         */
-        if (isListeningToTopic(topic)) {
-          topicPartitionCounts.put(topic, 0)
-          groupsPerTopic(topic).map(groupId => groups(groupId))
-        }
-        else Set.empty[ConsumerGroupMetadata]
-      }
-      groupsToRebalance.foreach(maybePrepareRebalance)
-    }
-
-    private def topicFromDataPath(dataPath: String) = {
-      val nodes = dataPath.split("/")
-      nodes.last
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala b/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala
index 70a710c..8e250c3 100644
--- a/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala
+++ b/core/src/main/scala/kafka/coordinator/DelayedHeartbeat.scala
@@ -23,13 +23,13 @@ import kafka.server.DelayedOperation
  * Delayed heartbeat operations that are added to the purgatory for session timeout checking.
  * Heartbeats are paused during rebalance.
  */
-private[coordinator] class DelayedHeartbeat(consumerCoordinator: ConsumerCoordinator,
-                                            group: ConsumerGroupMetadata,
-                                            consumer: ConsumerMetadata,
+private[coordinator] class DelayedHeartbeat(coordinator: GroupCoordinator,
+                                            group: GroupMetadata,
+                                            member: MemberMetadata,
                                             heartbeatDeadline: Long,
                                             sessionTimeout: Long)
   extends DelayedOperation(sessionTimeout) {
-  override def tryComplete(): Boolean = consumerCoordinator.tryCompleteHeartbeat(group, consumer, heartbeatDeadline, forceComplete)
-  override def onExpiration() = consumerCoordinator.onExpirationHeartbeat(group, consumer, heartbeatDeadline)
-  override def onComplete() = consumerCoordinator.onCompleteHeartbeat()
+  override def tryComplete(): Boolean = coordinator.tryCompleteHeartbeat(group, member, heartbeatDeadline, forceComplete)
+  override def onExpiration() = coordinator.onExpireHeartbeat(group, member, heartbeatDeadline)
+  override def onComplete() = coordinator.onCompleteHeartbeat()
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/coordinator/DelayedJoin.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/DelayedJoin.scala b/core/src/main/scala/kafka/coordinator/DelayedJoin.scala
new file mode 100644
index 0000000..ae96e15
--- /dev/null
+++ b/core/src/main/scala/kafka/coordinator/DelayedJoin.scala
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package kafka.coordinator
+
+import kafka.server.DelayedOperation
+
+/**
+ * Delayed rebalance operations that are added to the purgatory when group is preparing for rebalance
+ *
+ * Whenever a join-group request is received, check if all known group members have requested
+ * to re-join the group; if yes, complete this operation to proceed rebalance.
+ *
+ * When the operation has expired, any known members that have not requested to re-join
+ * the group are marked as failed, and complete this operation to proceed rebalance with
+ * the rest of the group.
+ */
+private[coordinator] class DelayedJoin(coordinator: GroupCoordinator,
+                                            group: GroupMetadata,
+                                            sessionTimeout: Long)
+  extends DelayedOperation(sessionTimeout) {
+
+  override def tryComplete(): Boolean = coordinator.tryCompleteJoin(group, forceComplete)
+  override def onExpiration() = coordinator.onExpireJoin()
+  override def onComplete() = coordinator.onCompleteJoin(group)
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/coordinator/DelayedRebalance.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/DelayedRebalance.scala b/core/src/main/scala/kafka/coordinator/DelayedRebalance.scala
deleted file mode 100644
index 8247d33..0000000
--- a/core/src/main/scala/kafka/coordinator/DelayedRebalance.scala
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package kafka.coordinator
-
-import kafka.server.DelayedOperation
-
-/**
- * Delayed rebalance operations that are added to the purgatory when group is preparing for rebalance
- *
- * Whenever a join-group request is received, check if all known consumers have requested
- * to re-join the group; if yes, complete this operation to proceed rebalance.
- *
- * When the operation has expired, any known consumers that have not requested to re-join
- * the group are marked as failed, and complete this operation to proceed rebalance with
- * the rest of the group.
- */
-private[coordinator] class DelayedRebalance(consumerCoordinator: ConsumerCoordinator,
-                                            group: ConsumerGroupMetadata,
-                                            sessionTimeout: Long)
-  extends DelayedOperation(sessionTimeout) {
-
-  override def tryComplete(): Boolean = consumerCoordinator.tryCompleteRebalance(group, forceComplete)
-  override def onExpiration() = consumerCoordinator.onExpirationRebalance()
-  override def onComplete() = consumerCoordinator.onCompleteRebalance(group)
-}


[2/8] kafka git commit: KAFKA-2464: client-side assignment for new consumer

Posted by gu...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/other/kafka/TestOffsetManager.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/other/kafka/TestOffsetManager.scala b/core/src/test/scala/other/kafka/TestOffsetManager.scala
index 549a96b..a77979a 100644
--- a/core/src/test/scala/other/kafka/TestOffsetManager.scala
+++ b/core/src/test/scala/other/kafka/TestOffsetManager.scala
@@ -135,8 +135,8 @@ object TestOffsetManager {
       val id = random.nextInt().abs % numGroups
       val group = "group-" + id
       try {
-        metadataChannel.send(ConsumerMetadataRequest(group))
-        val coordinatorId = ConsumerMetadataResponse.readFrom(metadataChannel.receive().payload()).coordinatorOpt.map(_.id).getOrElse(-1)
+        metadataChannel.send(GroupMetadataRequest(group))
+        val coordinatorId = GroupMetadataResponse.readFrom(metadataChannel.receive().payload()).coordinatorOpt.map(_.id).getOrElse(-1)
 
         val channel = if (channels.contains(coordinatorId))
           channels(coordinatorId)

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala
index e2a75e2..1266598 100644
--- a/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala
@@ -35,18 +35,18 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging {
   private val HostsString = Hosts.mkString(AclCommand.Delimiter.toString)
 
   private val TopicResources = Set(new Resource(Topic, "test-1"), new Resource(Topic, "test-2"))
-  private val ConsumerGroupResources = Set(new Resource(ConsumerGroup, "testGroup-1"), new Resource(ConsumerGroup, "testGroup-2"))
+  private val GroupResources = Set(new Resource(Group, "testGroup-1"), new Resource(Group, "testGroup-2"))
 
   private val ResourceToCommand = Map[Set[Resource], Array[String]](
     TopicResources -> Array("--topic", "test-1,test-2"),
     Set(Resource.ClusterResource) -> Array("--cluster"),
-    ConsumerGroupResources -> Array("--consumer-group", "testGroup-1,testGroup-2")
+    GroupResources -> Array("--group", "testGroup-1,testGroup-2")
   )
 
   private val ResourceToOperations = Map[Set[Resource], (Set[Operation], Array[String])](
     TopicResources -> (Set(Read, Write, Describe), Array("--operations", "Read,Write,Describe")),
     Set(Resource.ClusterResource) -> (Set(Create, ClusterAction), Array("--operations", "Create,ClusterAction")),
-    ConsumerGroupResources -> (Set(Read).toSet[Operation], Array("--operations", "Read"))
+    GroupResources -> (Set(Read).toSet[Operation], Array("--operations", "Read"))
   )
 
   private val ProducerResourceToAcls = Map[Set[Resource], Set[Acl]](
@@ -56,7 +56,7 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging {
 
   private val ConsumerResourceToAcls = Map[Set[Resource], Set[Acl]](
     TopicResources -> AclCommand.getAcls(Users, Allow, Set(Read, Describe), Hosts),
-    ConsumerGroupResources -> AclCommand.getAcls(Users, Allow, Set(Read), Hosts)
+    GroupResources -> AclCommand.getAcls(Users, Allow, Set(Read), Hosts)
   )
 
   private val CmdToResourcesToAcl = Map[Array[String], Map[Set[Resource], Set[Acl]]](

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
index cab4813..820a825 100644
--- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
@@ -24,7 +24,7 @@ import kafka.zk.ZooKeeperTestHarness
 import kafka.server.ConfigType
 import kafka.admin.TopicCommand.TopicCommandOptions
 import kafka.utils.ZkUtils._
-import kafka.coordinator.ConsumerCoordinator
+import kafka.coordinator.GroupCoordinator
 
 class TopicCommandTest extends ZooKeeperTestHarness with Logging {
 
@@ -85,12 +85,12 @@ class TopicCommandTest extends ZooKeeperTestHarness with Logging {
     // create the offset topic
     val createOffsetTopicOpts = new TopicCommandOptions(Array("--partitions", numPartitionsOriginal.toString,
       "--replication-factor", "1",
-      "--topic", ConsumerCoordinator.OffsetsTopicName))
+      "--topic", GroupCoordinator.OffsetsTopicName))
     TopicCommand.createTopic(zkUtils, createOffsetTopicOpts)
 
     // try to delete the OffsetManager.OffsetsTopicName and make sure it doesn't
-    val deleteOffsetTopicOpts = new TopicCommandOptions(Array("--topic", ConsumerCoordinator.OffsetsTopicName))
-    val deleteOffsetTopicPath = getDeleteTopicPath(ConsumerCoordinator.OffsetsTopicName)
+    val deleteOffsetTopicOpts = new TopicCommandOptions(Array("--topic", GroupCoordinator.OffsetsTopicName))
+    val deleteOffsetTopicPath = getDeleteTopicPath(GroupCoordinator.OffsetsTopicName)
     assertFalse("Delete path for topic shouldn't exist before deletion.", zkUtils.zkClient.exists(deleteOffsetTopicPath))
     intercept[AdminOperationException] {
         TopicCommand.deleteTopic(zkUtils, deleteOffsetTopicOpts)

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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 b7e7967..09e9ce3 100644
--- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
+++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala
@@ -231,12 +231,12 @@ object SerializationTestUtils {
     ))
   }
 
-  def createConsumerMetadataRequest: ConsumerMetadataRequest = {
-    ConsumerMetadataRequest("group 1", clientId = "client 1")
+  def createConsumerMetadataRequest: GroupMetadataRequest = {
+    GroupMetadataRequest("group 1", clientId = "client 1")
   }
 
-  def createConsumerMetadataResponse: ConsumerMetadataResponse = {
-    ConsumerMetadataResponse(Some(brokers.head.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), ErrorMapping.NoError, 0)
+  def createConsumerMetadataResponse: GroupMetadataResponse = {
+    GroupMetadataResponse(Some(brokers.head.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)), ErrorMapping.NoError, 0)
   }
 
   def createUpdateMetadataRequest(versionId: Short): UpdateMetadataRequest = {
@@ -276,7 +276,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, 0)
+  private val consumerMetadataResponseNoCoordinator = GroupMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, 0)
   private val updateMetadataRequestV0 = SerializationTestUtils.createUpdateMetadataRequest(0)
   private val updateMetadataRequestV1 = SerializationTestUtils.createUpdateMetadataRequest(1)
   private val updateMetdataResponse = SerializationTestUtils.createUpdateMetadataResponse

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
index 2e18e92..24fba45 100644
--- a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
+++ b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
@@ -22,7 +22,7 @@ import org.junit.Assert._
 import org.scalatest.junit.JUnitSuite
 import org.junit.Test
 import kafka.server.OffsetManager
-import kafka.coordinator.ConsumerCoordinator
+import kafka.coordinator.GroupCoordinator
 
 
 class TopicFilterTest extends JUnitSuite {
@@ -38,8 +38,8 @@ class TopicFilterTest extends JUnitSuite {
 
     val topicFilter2 = new Whitelist(".+")
     assertTrue(topicFilter2.isTopicAllowed("alltopics", excludeInternalTopics = true))
-    assertFalse(topicFilter2.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = true))
-    assertTrue(topicFilter2.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = false))
+    assertFalse(topicFilter2.isTopicAllowed(GroupCoordinator.OffsetsTopicName, excludeInternalTopics = true))
+    assertTrue(topicFilter2.isTopicAllowed(GroupCoordinator.OffsetsTopicName, excludeInternalTopics = false))
 
     val topicFilter3 = new Whitelist("white_listed-topic.+")
     assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1", excludeInternalTopics = true))
@@ -58,8 +58,8 @@ class TopicFilterTest extends JUnitSuite {
     assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = true))
     assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = false))
 
-    assertFalse(topicFilter1.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = true))
-    assertTrue(topicFilter1.isTopicAllowed(ConsumerCoordinator.OffsetsTopicName, excludeInternalTopics = false))
+    assertFalse(topicFilter1.isTopicAllowed(GroupCoordinator.OffsetsTopicName, excludeInternalTopics = true))
+    assertTrue(topicFilter1.isTopicAllowed(GroupCoordinator.OffsetsTopicName, excludeInternalTopics = false))
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala
deleted file mode 100644
index c108955..0000000
--- a/core/src/test/scala/unit/kafka/coordinator/ConsumerCoordinatorResponseTest.scala
+++ /dev/null
@@ -1,447 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package kafka.coordinator
-
-
-import java.util.concurrent.TimeUnit
-
-import org.junit.Assert._
-import kafka.common.{OffsetAndMetadata, TopicAndPartition}
-import kafka.server.{OffsetManager, KafkaConfig}
-import kafka.utils.TestUtils
-import org.apache.kafka.common.protocol.Errors
-import org.apache.kafka.common.requests.{OffsetCommitRequest, JoinGroupRequest}
-import org.easymock.{IAnswer, EasyMock}
-import org.junit.{After, Before, Test}
-import org.scalatest.junit.JUnitSuite
-
-import scala.concurrent.duration.Duration
-import scala.concurrent.{Await, Future, Promise}
-
-/**
- * Test ConsumerCoordinator responses
- */
-class ConsumerCoordinatorResponseTest extends JUnitSuite {
-  type JoinGroupCallbackParams = (Set[TopicAndPartition], String, Int, Short)
-  type JoinGroupCallback = (Set[TopicAndPartition], String, Int, Short) => Unit
-  type HeartbeatCallbackParams = Short
-  type HeartbeatCallback = Short => Unit
-  type CommitOffsetCallbackParams = Map[TopicAndPartition, Short]
-  type CommitOffsetCallback = Map[TopicAndPartition, Short] => Unit
-  type LeaveGroupCallbackParams = Short
-  type LeaveGroupCallback = Short => Unit
-
-  val ConsumerMinSessionTimeout = 10
-  val ConsumerMaxSessionTimeout = 200
-  val DefaultSessionTimeout = 100
-  var consumerCoordinator: ConsumerCoordinator = null
-  var offsetManager : OffsetManager = null
-
-  @Before
-  def setUp() {
-    val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
-    props.setProperty(KafkaConfig.ConsumerMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString)
-    props.setProperty(KafkaConfig.ConsumerMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString)
-    offsetManager = EasyMock.createStrictMock(classOf[OffsetManager])
-    consumerCoordinator = ConsumerCoordinator.create(KafkaConfig.fromProps(props), null, offsetManager)
-    consumerCoordinator.startup()
-  }
-
-  @After
-  def tearDown() {
-    EasyMock.reset(offsetManager)
-    consumerCoordinator.shutdown()
-  }
-
-  @Test
-  def testJoinGroupWrongCoordinator() {
-    val groupId = "groupId"
-    val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID
-    val partitionAssignmentStrategy = "range"
-
-    val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = false)
-    val joinGroupErrorCode = joinGroupResult._4
-    assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.code, joinGroupErrorCode)
-  }
-
-  @Test
-  def testJoinGroupUnknownPartitionAssignmentStrategy() {
-    val groupId = "groupId"
-    val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID
-    val partitionAssignmentStrategy = "foo"
-
-    val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true)
-    val joinGroupErrorCode = joinGroupResult._4
-    assertEquals(Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code, joinGroupErrorCode)
-  }
-
-  @Test
-  def testJoinGroupSessionTimeoutTooSmall() {
-    val groupId = "groupId"
-    val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID
-    val partitionAssignmentStrategy = "range"
-
-    val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, ConsumerMinSessionTimeout - 1, isCoordinatorForGroup = true)
-    val joinGroupErrorCode = joinGroupResult._4
-    assertEquals(Errors.INVALID_SESSION_TIMEOUT.code, joinGroupErrorCode)
-  }
-
-  @Test
-  def testJoinGroupSessionTimeoutTooLarge() {
-    val groupId = "groupId"
-    val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID
-    val partitionAssignmentStrategy = "range"
-
-    val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, ConsumerMaxSessionTimeout + 1, isCoordinatorForGroup = true)
-    val joinGroupErrorCode = joinGroupResult._4
-    assertEquals(Errors.INVALID_SESSION_TIMEOUT.code, joinGroupErrorCode)
-  }
-
-  @Test
-  def testJoinGroupUnknownConsumerNewGroup() {
-    val groupId = "groupId"
-    val consumerId = "consumerId"
-    val partitionAssignmentStrategy = "range"
-
-    val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true)
-    val joinGroupErrorCode = joinGroupResult._4
-    assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, joinGroupErrorCode)
-  }
-
-  @Test
-  def testValidJoinGroup() {
-    val groupId = "groupId"
-    val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID
-    val partitionAssignmentStrategy = "range"
-
-    val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true)
-    val joinGroupErrorCode = joinGroupResult._4
-    assertEquals(Errors.NONE.code, joinGroupErrorCode)
-  }
-
-  @Test
-  def testJoinGroupInconsistentPartitionAssignmentStrategy() {
-    val groupId = "groupId"
-    val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID
-    val otherConsumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID
-    val partitionAssignmentStrategy = "range"
-    val otherPartitionAssignmentStrategy = "roundrobin"
-
-    val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true)
-    val joinGroupErrorCode = joinGroupResult._4
-    assertEquals(Errors.NONE.code, joinGroupErrorCode)
-
-    EasyMock.reset(offsetManager)
-    val otherJoinGroupResult = joinGroup(groupId, otherConsumerId, otherPartitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true)
-    val otherJoinGroupErrorCode = otherJoinGroupResult._4
-    assertEquals(Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code, otherJoinGroupErrorCode)
-  }
-
-  @Test
-  def testJoinGroupUnknownConsumerExistingGroup() {
-    val groupId = "groupId"
-    val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID
-    val otherConsumerId = "consumerId"
-    val partitionAssignmentStrategy = "range"
-
-    val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true)
-    val joinGroupErrorCode = joinGroupResult._4
-    assertEquals(Errors.NONE.code, joinGroupErrorCode)
-
-    EasyMock.reset(offsetManager)
-    val otherJoinGroupResult = joinGroup(groupId, otherConsumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true)
-    val otherJoinGroupErrorCode = otherJoinGroupResult._4
-    assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, otherJoinGroupErrorCode)
-  }
-
-  @Test
-  def testHeartbeatWrongCoordinator() {
-    val groupId = "groupId"
-    val consumerId = "consumerId"
-
-    val heartbeatResult = heartbeat(groupId, consumerId, -1, isCoordinatorForGroup = false)
-    assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.code, heartbeatResult)
-  }
-
-  @Test
-  def testHeartbeatUnknownGroup() {
-    val groupId = "groupId"
-    val consumerId = "consumerId"
-
-    val heartbeatResult = heartbeat(groupId, consumerId, -1, isCoordinatorForGroup = true)
-    assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, heartbeatResult)
-  }
-
-  @Test
-  def testHeartbeatUnknownConsumerExistingGroup() {
-    val groupId = "groupId"
-    val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID
-    val otherConsumerId = "consumerId"
-    val partitionAssignmentStrategy = "range"
-
-    val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true)
-    val joinGroupErrorCode = joinGroupResult._4
-    assertEquals(Errors.NONE.code, joinGroupErrorCode)
-
-    EasyMock.reset(offsetManager)
-    val heartbeatResult = heartbeat(groupId, otherConsumerId, 1, isCoordinatorForGroup = true)
-    assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, heartbeatResult)
-  }
-
-  @Test
-  def testHeartbeatIllegalGeneration() {
-    val groupId = "groupId"
-    val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID
-    val partitionAssignmentStrategy = "range"
-
-    val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true)
-    val assignedConsumerId = joinGroupResult._2
-    val joinGroupErrorCode = joinGroupResult._4
-    assertEquals(Errors.NONE.code, joinGroupErrorCode)
-
-    EasyMock.reset(offsetManager)
-    val heartbeatResult = heartbeat(groupId, assignedConsumerId, 2, isCoordinatorForGroup = true)
-    assertEquals(Errors.ILLEGAL_GENERATION.code, heartbeatResult)
-  }
-
-  @Test
-  def testValidHeartbeat() {
-    val groupId = "groupId"
-    val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID
-    val partitionAssignmentStrategy = "range"
-
-    val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true)
-    val assignedConsumerId = joinGroupResult._2
-    val joinGroupErrorCode = joinGroupResult._4
-    assertEquals(Errors.NONE.code, joinGroupErrorCode)
-
-    EasyMock.reset(offsetManager)
-    val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1, isCoordinatorForGroup = true)
-    assertEquals(Errors.NONE.code, heartbeatResult)
-  }
-
-  @Test
-  def testCommitOffsetFromUnknownGroup() {
-    val groupId = "groupId"
-    val consumerId = "consumer"
-    val generationId = 1
-    val tp = new TopicAndPartition("topic", 0)
-    val offset = OffsetAndMetadata(0)
-
-    val commitOffsetResult = commitOffsets(groupId, consumerId, generationId, Map(tp -> offset), true)
-    assertEquals(Errors.ILLEGAL_GENERATION.code, commitOffsetResult(tp))
-  }
-
-  @Test
-  def testCommitOffsetWithDefaultGeneration() {
-    val groupId = "groupId"
-    val tp = new TopicAndPartition("topic", 0)
-    val offset = OffsetAndMetadata(0)
-
-    val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_CONSUMER_ID,
-      OffsetCommitRequest.DEFAULT_GENERATION_ID, Map(tp -> offset), true)
-    assertEquals(Errors.NONE.code, commitOffsetResult(tp))
-  }
-
-  @Test
-  def testHeartbeatDuringRebalanceCausesRebalanceInProgress() {
-    val groupId = "groupId"
-    val partitionAssignmentStrategy = "range"
-
-    // First start up a group (with a slightly larger timeout to give us time to heartbeat when the rebalance starts)
-    val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_CONSUMER_ID, partitionAssignmentStrategy,
-      DefaultSessionTimeout, isCoordinatorForGroup = true)
-    val assignedConsumerId = joinGroupResult._2
-    val initialGenerationId = joinGroupResult._3
-    val joinGroupErrorCode = joinGroupResult._4
-    assertEquals(Errors.NONE.code, joinGroupErrorCode)
-
-    // Then join with a new consumer to trigger a rebalance
-    EasyMock.reset(offsetManager)
-    sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_CONSUMER_ID, partitionAssignmentStrategy,
-      DefaultSessionTimeout, isCoordinatorForGroup = true)
-
-    // We should be in the middle of a rebalance, so the heartbeat should return rebalance in progress
-    EasyMock.reset(offsetManager)
-    val heartbeatResult = heartbeat(groupId, assignedConsumerId, initialGenerationId, isCoordinatorForGroup = true)
-    assertEquals(Errors.REBALANCE_IN_PROGRESS.code, heartbeatResult)
-  }
-
-  @Test
-  def testGenerationIdIncrementsOnRebalance() {
-    val groupId = "groupId"
-    val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID
-    val otherConsumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID
-    val partitionAssignmentStrategy = "range"
-
-    val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true)
-    val initialGenerationId = joinGroupResult._3
-    val joinGroupErrorCode = joinGroupResult._4
-    assertEquals(1, initialGenerationId)
-    assertEquals(Errors.NONE.code, joinGroupErrorCode)
-
-    EasyMock.reset(offsetManager)
-    val otherJoinGroupResult = joinGroup(groupId, otherConsumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true)
-    val nextGenerationId = otherJoinGroupResult._3
-    val otherJoinGroupErrorCode = otherJoinGroupResult._4
-    assertEquals(2, nextGenerationId)
-    assertEquals(Errors.NONE.code, otherJoinGroupErrorCode)
-  }
-
-  @Test
-  def testLeaveGroupWrongCoordinator() {
-    val groupId = "groupId"
-    val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID
-
-    val leaveGroupResult = leaveGroup(groupId, consumerId, isCoordinatorForGroup = false)
-    assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.code, leaveGroupResult)
-  }
-
-  @Test
-  def testLeaveGroupUnknownGroup() {
-    val groupId = "groupId"
-    val consumerId = "consumerId"
-
-    val leaveGroupResult = leaveGroup(groupId, consumerId, isCoordinatorForGroup = true)
-    assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, leaveGroupResult)
-  }
-
-  @Test
-  def testLeaveGroupUnknownConsumerExistingGroup() {
-    val groupId = "groupId"
-    val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID
-    val otherConsumerId = "consumerId"
-    val partitionAssignmentStrategy = "range"
-
-    val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true)
-    val joinGroupErrorCode = joinGroupResult._4
-    assertEquals(Errors.NONE.code, joinGroupErrorCode)
-
-    EasyMock.reset(offsetManager)
-    val leaveGroupResult = leaveGroup(groupId, otherConsumerId, isCoordinatorForGroup = true)
-    assertEquals(Errors.UNKNOWN_CONSUMER_ID.code, leaveGroupResult)
-  }
-
-  @Test
-  def testValidLeaveGroup() {
-    val groupId = "groupId"
-    val consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID
-    val partitionAssignmentStrategy = "range"
-
-    val joinGroupResult = joinGroup(groupId, consumerId, partitionAssignmentStrategy, DefaultSessionTimeout, isCoordinatorForGroup = true)
-    val assignedConsumerId = joinGroupResult._2
-    val joinGroupErrorCode = joinGroupResult._4
-    assertEquals(Errors.NONE.code, joinGroupErrorCode)
-
-    EasyMock.reset(offsetManager)
-    val leaveGroupResult = leaveGroup(groupId, assignedConsumerId, isCoordinatorForGroup = true)
-    assertEquals(Errors.NONE.code, leaveGroupResult)
-  }
-
-  private def setupJoinGroupCallback: (Future[JoinGroupCallbackParams], JoinGroupCallback) = {
-    val responsePromise = Promise[JoinGroupCallbackParams]
-    val responseFuture = responsePromise.future
-    val responseCallback: JoinGroupCallback = (partitions, consumerId, generationId, errorCode) =>
-      responsePromise.success((partitions, consumerId, generationId, errorCode))
-    (responseFuture, responseCallback)
-  }
-
-  private def setupHeartbeatCallback: (Future[HeartbeatCallbackParams], HeartbeatCallback) = {
-    val responsePromise = Promise[HeartbeatCallbackParams]
-    val responseFuture = responsePromise.future
-    val responseCallback: HeartbeatCallback = errorCode => responsePromise.success(errorCode)
-    (responseFuture, responseCallback)
-  }
-
-  private def setupCommitOffsetsCallback: (Future[CommitOffsetCallbackParams], CommitOffsetCallback) = {
-    val responsePromise = Promise[CommitOffsetCallbackParams]
-    val responseFuture = responsePromise.future
-    val responseCallback: CommitOffsetCallback = offsets => responsePromise.success(offsets)
-    (responseFuture, responseCallback)
-  }
-
-  private def setupLeaveGroupCallback: (Future[LeaveGroupCallbackParams], LeaveGroupCallback) = {
-    val responsePromise = Promise[LeaveGroupCallbackParams]
-    val responseFuture = responsePromise.future
-    val responseCallback: LeaveGroupCallback = errorCode => responsePromise.success(errorCode)
-    (responseFuture, responseCallback)
-  }
-
-  private def sendJoinGroup(groupId: String,
-                            consumerId: String,
-                            partitionAssignmentStrategy: String,
-                            sessionTimeout: Int,
-                            isCoordinatorForGroup: Boolean): Future[JoinGroupCallbackParams] = {
-    val (responseFuture, responseCallback) = setupJoinGroupCallback
-    EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1)
-    EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup)
-    EasyMock.replay(offsetManager)
-    consumerCoordinator.handleJoinGroup(groupId, consumerId, Set.empty, sessionTimeout, partitionAssignmentStrategy, responseCallback)
-    responseFuture
-  }
-
-  private def joinGroup(groupId: String,
-                        consumerId: String,
-                        partitionAssignmentStrategy: String,
-                        sessionTimeout: Int,
-                        isCoordinatorForGroup: Boolean): JoinGroupCallbackParams = {
-    val responseFuture = sendJoinGroup(groupId, consumerId, partitionAssignmentStrategy, sessionTimeout, isCoordinatorForGroup)
-    // should only have to wait as long as session timeout, but allow some extra time in case of an unexpected delay
-    Await.result(responseFuture, Duration(sessionTimeout+100, TimeUnit.MILLISECONDS))
-  }
-
-  private def heartbeat(groupId: String,
-                        consumerId: String,
-                        generationId: Int,
-                        isCoordinatorForGroup: Boolean): HeartbeatCallbackParams = {
-    val (responseFuture, responseCallback) = setupHeartbeatCallback
-    EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1)
-    EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup)
-    EasyMock.replay(offsetManager)
-    consumerCoordinator.handleHeartbeat(groupId, consumerId, generationId, responseCallback)
-    Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS))
-  }
-
-  private def commitOffsets(groupId: String,
-                            consumerId: String,
-                            generationId: Int,
-                            offsets: Map[TopicAndPartition, OffsetAndMetadata],
-                            isCoordinatorForGroup: Boolean): CommitOffsetCallbackParams = {
-    val (responseFuture, responseCallback) = setupCommitOffsetsCallback
-    EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1)
-    EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup)
-    val storeOffsetAnswer = new IAnswer[Unit] {
-      override def answer = responseCallback.apply(offsets.mapValues(_ => Errors.NONE.code))
-    }
-    EasyMock.expect(offsetManager.storeOffsets(groupId, consumerId, generationId, offsets, responseCallback))
-      .andAnswer(storeOffsetAnswer)
-    EasyMock.replay(offsetManager)
-    consumerCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback)
-    Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS))
-    Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS))
-  }
-
-  private def leaveGroup(groupId: String, consumerId: String, isCoordinatorForGroup: Boolean): LeaveGroupCallbackParams = {
-    val (responseFuture, responseCallback) = setupHeartbeatCallback
-    EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1)
-    EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup)
-    EasyMock.replay(offsetManager)
-    consumerCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback)
-    Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS))
-  }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/unit/kafka/coordinator/ConsumerGroupMetadataTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/ConsumerGroupMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/ConsumerGroupMetadataTest.scala
deleted file mode 100644
index 5d812c2..0000000
--- a/core/src/test/scala/unit/kafka/coordinator/ConsumerGroupMetadataTest.scala
+++ /dev/null
@@ -1,172 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package kafka.coordinator
-
-import org.junit.Assert._
-import org.junit.{Before, Test}
-import org.scalatest.junit.JUnitSuite
-
-/**
- * Test group state transitions
- */
-class ConsumerGroupMetadataTest extends JUnitSuite {
-  var group: ConsumerGroupMetadata = null
-
-  @Before
-  def setUp() {
-    group = new ConsumerGroupMetadata("test", "range")
-  }
-
-  @Test
-  def testCanRebalanceWhenStable() {
-    assertTrue(group.canRebalance)
-  }
-
-  @Test
-  def testCannotRebalanceWhenPreparingRebalance() {
-    group.transitionTo(PreparingRebalance)
-    assertFalse(group.canRebalance)
-  }
-
-  @Test
-  def testCannotRebalanceWhenRebalancing() {
-    group.transitionTo(PreparingRebalance)
-    group.transitionTo(Rebalancing)
-    assertFalse(group.canRebalance)
-  }
-
-  @Test
-  def testCannotRebalanceWhenDead() {
-    group.transitionTo(PreparingRebalance)
-    group.transitionTo(Dead)
-    assertFalse(group.canRebalance)
-  }
-
-  @Test
-  def testStableToPreparingRebalanceTransition() {
-    group.transitionTo(PreparingRebalance)
-    assertState(group, PreparingRebalance)
-  }
-
-  @Test
-  def testPreparingRebalanceToRebalancingTransition() {
-    group.transitionTo(PreparingRebalance)
-    group.transitionTo(Rebalancing)
-    assertState(group, Rebalancing)
-  }
-
-  @Test
-  def testPreparingRebalanceToDeadTransition() {
-    group.transitionTo(PreparingRebalance)
-    group.transitionTo(Dead)
-    assertState(group, Dead)
-  }
-
-  @Test
-  def testRebalancingToStableTransition() {
-    group.transitionTo(PreparingRebalance)
-    group.transitionTo(Rebalancing)
-    group.transitionTo(Stable)
-    assertState(group, Stable)
-  }
-
-  @Test(expected = classOf[IllegalStateException])
-  def testStableToStableIllegalTransition() {
-    group.transitionTo(Stable)
-  }
-
-  @Test(expected = classOf[IllegalStateException])
-  def testStableToRebalancingIllegalTransition() {
-    group.transitionTo(Rebalancing)
-  }
-
-  @Test(expected = classOf[IllegalStateException])
-  def testStableToDeadIllegalTransition() {
-    group.transitionTo(Dead)
-  }
-
-  @Test(expected = classOf[IllegalStateException])
-  def testPreparingRebalanceToPreparingRebalanceIllegalTransition() {
-    group.transitionTo(PreparingRebalance)
-    group.transitionTo(PreparingRebalance)
-  }
-
-  @Test(expected = classOf[IllegalStateException])
-  def testPreparingRebalanceToStableIllegalTransition() {
-    group.transitionTo(PreparingRebalance)
-    group.transitionTo(Stable)
-  }
-
-  @Test(expected = classOf[IllegalStateException])
-  def testRebalancingToRebalancingIllegalTransition() {
-    group.transitionTo(PreparingRebalance)
-    group.transitionTo(Rebalancing)
-    group.transitionTo(Rebalancing)
-  }
-
-  @Test(expected = classOf[IllegalStateException])
-  def testRebalancingToPreparingRebalanceTransition() {
-    group.transitionTo(PreparingRebalance)
-    group.transitionTo(Rebalancing)
-    group.transitionTo(PreparingRebalance)
-  }
-
-  @Test(expected = classOf[IllegalStateException])
-  def testRebalancingToDeadIllegalTransition() {
-    group.transitionTo(PreparingRebalance)
-    group.transitionTo(Rebalancing)
-    group.transitionTo(Dead)
-  }
-
-  @Test(expected = classOf[IllegalStateException])
-  def testDeadToDeadIllegalTransition() {
-    group.transitionTo(PreparingRebalance)
-    group.transitionTo(Dead)
-    group.transitionTo(Dead)
-  }
-
-  @Test(expected = classOf[IllegalStateException])
-  def testDeadToStableIllegalTransition() {
-    group.transitionTo(PreparingRebalance)
-    group.transitionTo(Dead)
-    group.transitionTo(Stable)
-  }
-
-  @Test(expected = classOf[IllegalStateException])
-  def testDeadToPreparingRebalanceIllegalTransition() {
-    group.transitionTo(PreparingRebalance)
-    group.transitionTo(Dead)
-    group.transitionTo(PreparingRebalance)
-  }
-
-  @Test(expected = classOf[IllegalStateException])
-  def testDeadToRebalancingIllegalTransition() {
-    group.transitionTo(PreparingRebalance)
-    group.transitionTo(Dead)
-    group.transitionTo(Rebalancing)
-  }
-
-  private def assertState(group: ConsumerGroupMetadata, targetState: GroupState) {
-    val states: Set[GroupState] = Set(Stable, PreparingRebalance, Rebalancing, Dead)
-    val otherStates = states - targetState
-    otherStates.foreach { otherState =>
-      assertFalse(group.is(otherState))
-    }
-    assertTrue(group.is(targetState))
-  }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala
index 3bc37e5..49a237b 100644
--- a/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/CoordinatorMetadataTest.scala
@@ -18,13 +18,9 @@
 package kafka.coordinator
 
 import kafka.server.KafkaConfig
-import kafka.utils.{TestUtils, ZkUtils}
-import kafka.utils.ZkUtils._
+import kafka.utils.TestUtils
 
 import org.junit.Assert._
-import org.I0Itec.zkclient.{IZkDataListener, ZkClient}
-import org.apache.zookeeper.data.Stat
-import org.easymock.EasyMock
 import org.junit.{Before, Test}
 import org.scalatest.junit.JUnitSuite
 
@@ -34,15 +30,12 @@ import org.scalatest.junit.JUnitSuite
 class CoordinatorMetadataTest extends JUnitSuite {
   val DefaultNumPartitions = 8
   val DefaultNumReplicas = 2
-  var zkUtils: ZkUtils = null
   var coordinatorMetadata: CoordinatorMetadata = null
 
   @Before
   def setUp() {
     val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
-    val zkClient = EasyMock.createStrictMock(classOf[ZkClient])
-    zkUtils = ZkUtils(zkClient, false)
-    coordinatorMetadata = new CoordinatorMetadata(KafkaConfig.fromProps(props).brokerId, zkUtils, null)
+    coordinatorMetadata = new CoordinatorMetadata(KafkaConfig.fromProps(props).brokerId)
   }
 
   @Test
@@ -53,7 +46,8 @@ class CoordinatorMetadataTest extends JUnitSuite {
   @Test
   def testGetGroup() {
     val groupId = "group"
-    val expected = coordinatorMetadata.addGroup(groupId, "range")
+    val protocolType = "consumer"
+    val expected = coordinatorMetadata.addGroup(groupId, protocolType)
     val actual = coordinatorMetadata.getGroup(groupId)
     assertEquals(expected, actual)
   }
@@ -61,155 +55,17 @@ class CoordinatorMetadataTest extends JUnitSuite {
   @Test
   def testAddGroupReturnsPreexistingGroupIfItAlreadyExists() {
     val groupId = "group"
-    val group1 = coordinatorMetadata.addGroup(groupId, "range")
-    val group2 = coordinatorMetadata.addGroup(groupId, "range")
+    val protocolType = "consumer"
+    val group1 = coordinatorMetadata.addGroup(groupId, protocolType)
+    val group2 = coordinatorMetadata.addGroup(groupId, protocolType)
     assertEquals(group1, group2)
   }
 
   @Test(expected = classOf[IllegalArgumentException])
-  def testBindNonexistentGroupToTopics() {
-    val groupId = "group"
-    val topics = Set("a")
-    coordinatorMetadata.bindGroupToTopics(groupId, topics)
-  }
-
-  @Test
-  def testBindGroupToTopicsNotListenedOn() {
-    val groupId = "group"
-    val topics = Set("a")
-    coordinatorMetadata.addGroup(groupId, "range")
-
-    expectZkClientSubscribeDataChanges(zkUtils, topics)
-    EasyMock.replay(zkUtils.zkClient)
-    coordinatorMetadata.bindGroupToTopics(groupId, topics)
-    assertEquals(Map("a" -> DefaultNumPartitions), coordinatorMetadata.partitionsPerTopic)
-  }
-
-  @Test
-  def testBindGroupToTopicsAlreadyListenedOn() {
-    val group1 = "group1"
-    val group2 = "group2"
-    val topics = Set("a")
-    coordinatorMetadata.addGroup(group1, "range")
-    coordinatorMetadata.addGroup(group2, "range")
-
-    expectZkClientSubscribeDataChanges(zkUtils, topics)
-    EasyMock.replay(zkUtils.zkClient)
-    coordinatorMetadata.bindGroupToTopics(group1, topics)
-    coordinatorMetadata.bindGroupToTopics(group2, topics)
-    assertEquals(Map("a" -> DefaultNumPartitions), coordinatorMetadata.partitionsPerTopic)
-  }
-
-  @Test(expected = classOf[IllegalArgumentException])
-  def testUnbindNonexistentGroupFromTopics() {
-    val groupId = "group"
-    val topics = Set("a")
-    coordinatorMetadata.unbindGroupFromTopics(groupId, topics)
-  }
-
-  @Test
-  def testUnbindGroupFromTopicsNotListenedOn() {
-    val groupId = "group"
-    val topics = Set("a")
-    coordinatorMetadata.addGroup(groupId, "range")
-
-    expectZkClientSubscribeDataChanges(zkUtils, topics)
-    EasyMock.replay(zkUtils.zkClient)
-    coordinatorMetadata.bindGroupToTopics(groupId, topics)
-    coordinatorMetadata.unbindGroupFromTopics(groupId, Set("b"))
-    assertEquals(Map("a" -> DefaultNumPartitions), coordinatorMetadata.partitionsPerTopic)
-  }
-
-  @Test
-  def testUnbindGroupFromTopicsListenedOnByOtherGroups() {
-    val group1 = "group1"
-    val group2 = "group2"
-    val topics = Set("a")
-    coordinatorMetadata.addGroup(group1, "range")
-    coordinatorMetadata.addGroup(group2, "range")
-
-    expectZkClientSubscribeDataChanges(zkUtils, topics)
-    EasyMock.replay(zkUtils.zkClient)
-    coordinatorMetadata.bindGroupToTopics(group1, topics)
-    coordinatorMetadata.bindGroupToTopics(group2, topics)
-    coordinatorMetadata.unbindGroupFromTopics(group1, topics)
-    assertEquals(Map("a" -> DefaultNumPartitions), coordinatorMetadata.partitionsPerTopic)
-  }
-
-  @Test
-  def testUnbindGroupFromTopicsListenedOnByNoOtherGroup() {
-    val groupId = "group"
-    val topics = Set("a")
-    coordinatorMetadata.addGroup(groupId, "range")
-
-    expectZkClientSubscribeDataChanges(zkUtils, topics)
-    expectZkClientUnsubscribeDataChanges(zkUtils.zkClient, topics)
-    EasyMock.replay(zkUtils.zkClient)
-    coordinatorMetadata.bindGroupToTopics(groupId, topics)
-    coordinatorMetadata.unbindGroupFromTopics(groupId, topics)
-    assertEquals(Map.empty[String, Int], coordinatorMetadata.partitionsPerTopic)
-  }
-
-  @Test(expected = classOf[IllegalArgumentException])
   def testRemoveNonexistentGroup() {
     val groupId = "group"
     val topics = Set("a")
-    coordinatorMetadata.removeGroup(groupId, topics)
-  }
-
-  @Test
-  def testRemoveGroupWithOtherGroupsBoundToItsTopics() {
-    val group1 = "group1"
-    val group2 = "group2"
-    val topics = Set("a")
-    coordinatorMetadata.addGroup(group1, "range")
-    coordinatorMetadata.addGroup(group2, "range")
-
-    expectZkClientSubscribeDataChanges(zkUtils, topics)
-    EasyMock.replay(zkUtils.zkClient)
-    coordinatorMetadata.bindGroupToTopics(group1, topics)
-    coordinatorMetadata.bindGroupToTopics(group2, topics)
-    coordinatorMetadata.removeGroup(group1, topics)
-    assertNull(coordinatorMetadata.getGroup(group1))
-    assertNotNull(coordinatorMetadata.getGroup(group2))
-    assertEquals(Map("a" -> DefaultNumPartitions), coordinatorMetadata.partitionsPerTopic)
-  }
-
-  @Test
-  def testRemoveGroupWithNoOtherGroupsBoundToItsTopics() {
-    val groupId = "group"
-    val topics = Set("a")
-    coordinatorMetadata.addGroup(groupId, "range")
-
-    expectZkClientSubscribeDataChanges(zkUtils, topics)
-    expectZkClientUnsubscribeDataChanges(zkUtils.zkClient, topics)
-    EasyMock.replay(zkUtils.zkClient)
-    coordinatorMetadata.bindGroupToTopics(groupId, topics)
-    coordinatorMetadata.removeGroup(groupId, topics)
-    assertNull(coordinatorMetadata.getGroup(groupId))
-    assertEquals(Map.empty[String, Int], coordinatorMetadata.partitionsPerTopic)
+    coordinatorMetadata.removeGroup(groupId)
   }
 
-  private def expectZkClientSubscribeDataChanges(zkUtils: ZkUtils, topics: Set[String]) {
-    topics.foreach(topic => expectZkClientSubscribeDataChange(zkUtils.zkClient, topic))
-  }
-
-  private def expectZkClientUnsubscribeDataChanges(zkClient: ZkClient, topics: Set[String]) {
-    topics.foreach(topic => expectZkClientUnsubscribeDataChange(zkClient, topic))
-  }
-
-  private def expectZkClientSubscribeDataChange(zkClient: ZkClient, topic: String) {
-    val replicaAssignment =
-      (0 until DefaultNumPartitions)
-      .map(partition => partition.toString -> (0 until DefaultNumReplicas).toSeq).toMap
-    val topicPath = getTopicPath(topic)
-    EasyMock.expect(zkClient.readData(topicPath, new Stat()))
-      .andReturn(zkUtils.replicaAssignmentZkData(replicaAssignment))
-    zkClient.subscribeDataChanges(EasyMock.eq(topicPath), EasyMock.isA(classOf[IZkDataListener]))
-  }
-
-  private def expectZkClientUnsubscribeDataChange(zkClient: ZkClient, topic: String) {
-    val topicPath = getTopicPath(topic)
-    zkClient.unsubscribeDataChanges(EasyMock.eq(topicPath), EasyMock.isA(classOf[IZkDataListener]))
-  }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
new file mode 100644
index 0000000..cdd78ef
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
@@ -0,0 +1,907 @@
+/**
+ * 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.coordinator
+
+
+import java.util.concurrent.TimeUnit
+
+import org.junit.Assert._
+import kafka.common.{OffsetAndMetadata, TopicAndPartition}
+import kafka.server.{OffsetManager, KafkaConfig}
+import kafka.utils.TestUtils
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.requests.{OffsetCommitRequest, JoinGroupRequest}
+import org.easymock.{IAnswer, EasyMock}
+import org.junit.{After, Before, Test}
+import org.scalatest.junit.JUnitSuite
+
+import scala.concurrent.duration.Duration
+import scala.concurrent.{Await, Future, Promise}
+
+/**
+ * Test GroupCoordinator responses
+ */
+class GroupCoordinatorResponseTest extends JUnitSuite {
+  type JoinGroupCallback = JoinGroupResult => Unit
+  type SyncGroupCallbackParams = (Array[Byte], Short)
+  type SyncGroupCallback = (Array[Byte], Short) => Unit
+  type HeartbeatCallbackParams = Short
+  type HeartbeatCallback = Short => Unit
+  type CommitOffsetCallbackParams = Map[TopicAndPartition, Short]
+  type CommitOffsetCallback = Map[TopicAndPartition, Short] => Unit
+  type LeaveGroupCallbackParams = Short
+  type LeaveGroupCallback = Short => Unit
+
+  val ConsumerMinSessionTimeout = 10
+  val ConsumerMaxSessionTimeout = 1000
+  val DefaultSessionTimeout = 500
+  var consumerCoordinator: GroupCoordinator = null
+  var offsetManager : OffsetManager = null
+
+  @Before
+  def setUp() {
+    val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
+    props.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString)
+    props.setProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString)
+    offsetManager = EasyMock.createStrictMock(classOf[OffsetManager])
+    consumerCoordinator = GroupCoordinator.create(KafkaConfig.fromProps(props), null, offsetManager)
+    consumerCoordinator.startup()
+  }
+
+  @After
+  def tearDown() {
+    EasyMock.reset(offsetManager)
+    consumerCoordinator.shutdown()
+  }
+
+  @Test
+  def testJoinGroupWrongCoordinator() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType,
+      protocols, isCoordinatorForGroup = false)
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(Errors.NOT_COORDINATOR_FOR_GROUP.code, joinGroupErrorCode)
+  }
+
+  @Test
+  def testJoinGroupSessionTimeoutTooSmall() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+
+    val joinGroupResult = joinGroup(groupId, memberId, ConsumerMinSessionTimeout - 1, protocolType, protocols,
+      isCoordinatorForGroup = true)
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(Errors.INVALID_SESSION_TIMEOUT.code, joinGroupErrorCode)
+  }
+
+  @Test
+  def testJoinGroupSessionTimeoutTooLarge() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+
+    val joinGroupResult = joinGroup(groupId, memberId, ConsumerMaxSessionTimeout + 1, protocolType, protocols,
+      isCoordinatorForGroup = true)
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(Errors.INVALID_SESSION_TIMEOUT.code, joinGroupErrorCode)
+  }
+
+  @Test
+  def testJoinGroupUnknownConsumerNewGroup() {
+    val groupId = "groupId"
+    val memberId = "memberId"
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(Errors.UNKNOWN_MEMBER_ID.code, joinGroupErrorCode)
+  }
+
+  @Test
+  def testValidJoinGroup() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType,
+      protocols, isCoordinatorForGroup = true)
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(Errors.NONE.code, joinGroupErrorCode)
+  }
+
+  @Test
+  def testJoinGroupInconsistentProtocolType() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val metadata = Array[Byte]()
+
+    val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, "consumer", List(("range", metadata)),
+      isCoordinatorForGroup = true)
+    assertEquals(Errors.NONE.code, joinGroupResult.errorCode)
+
+    EasyMock.reset(offsetManager)
+    val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, "copycat",
+      List(("range", metadata)), isCoordinatorForGroup = true)
+    assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code, otherJoinGroupResult.errorCode)
+  }
+
+  @Test
+  def testJoinGroupInconsistentGroupProtocol() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val protocolType = "consumer"
+    val metadata = Array[Byte]()
+
+    val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, List(("range", metadata)),
+      isCoordinatorForGroup = true)
+    assertEquals(Errors.NONE.code, joinGroupResult.errorCode)
+
+    EasyMock.reset(offsetManager)
+    val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType,
+      List(("roundrobin", metadata)), isCoordinatorForGroup = true)
+    assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code, otherJoinGroupResult.errorCode)
+  }
+
+  @Test
+  def testJoinGroupUnknownConsumerExistingGroup() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val otherMemberId = "memberId"
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+    assertEquals(Errors.NONE.code, joinGroupResult.errorCode)
+
+    EasyMock.reset(offsetManager)
+    val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID.code, otherJoinGroupResult.errorCode)
+  }
+
+  @Test
+  def testHeartbeatWrongCoordinator() {
+    val groupId = "groupId"
+    val consumerId = "memberId"
+
+    val heartbeatResult = heartbeat(groupId, consumerId, -1, isCoordinatorForGroup = false)
+    assertEquals(Errors.NOT_COORDINATOR_FOR_GROUP.code, heartbeatResult)
+  }
+
+  @Test
+  def testHeartbeatUnknownGroup() {
+    val groupId = "groupId"
+    val consumerId = "memberId"
+
+    val heartbeatResult = heartbeat(groupId, consumerId, -1, isCoordinatorForGroup = true)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID.code, heartbeatResult)
+  }
+
+  @Test
+  def testHeartbeatUnknownConsumerExistingGroup() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val otherMemberId = "memberId"
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(Errors.NONE.code, joinGroupErrorCode)
+
+    EasyMock.reset(offsetManager)
+    val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, joinGroupResult.memberId, Map.empty, true)
+    val syncGroupErrorCode = syncGroupResult._2
+    assertEquals(Errors.NONE.code, syncGroupErrorCode)
+
+    EasyMock.reset(offsetManager)
+    val heartbeatResult = heartbeat(groupId, otherMemberId, 1, isCoordinatorForGroup = true)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID.code, heartbeatResult)
+  }
+
+  @Test
+  def testHeartbeatRebalanceInProgress() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+    val assignedMemberId = joinGroupResult.memberId
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(Errors.NONE.code, joinGroupErrorCode)
+
+    EasyMock.reset(offsetManager)
+    val heartbeatResult = heartbeat(groupId, assignedMemberId, 2, isCoordinatorForGroup = true)
+    assertEquals(Errors.REBALANCE_IN_PROGRESS.code, heartbeatResult)
+  }
+
+  @Test
+  def testHeartbeatIllegalGeneration() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+    val assignedMemberId = joinGroupResult.memberId
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(Errors.NONE.code, joinGroupErrorCode)
+
+    EasyMock.reset(offsetManager)
+    val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, assignedMemberId, Map.empty, true)
+    val syncGroupErrorCode = syncGroupResult._2
+    assertEquals(Errors.NONE.code, syncGroupErrorCode)
+
+    EasyMock.reset(offsetManager)
+    val heartbeatResult = heartbeat(groupId, assignedMemberId, 2, isCoordinatorForGroup = true)
+    assertEquals(Errors.ILLEGAL_GENERATION.code, heartbeatResult)
+  }
+
+  @Test
+  def testValidHeartbeat() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+    val assignedConsumerId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(Errors.NONE.code, joinGroupErrorCode)
+
+    EasyMock.reset(offsetManager)
+    val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map.empty, true)
+    val syncGroupErrorCode = syncGroupResult._2
+    assertEquals(Errors.NONE.code, syncGroupErrorCode)
+
+    EasyMock.reset(offsetManager)
+    val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1, isCoordinatorForGroup = true)
+    assertEquals(Errors.NONE.code, heartbeatResult)
+  }
+
+  @Test
+  def testSyncGroupNotCoordinator() {
+    val groupId = "groupId"
+    val memberId = "member"
+    val generation = 1
+
+    val syncGroupResult = syncGroupFollower(groupId, generation, memberId, false)
+    assertEquals(Errors.NOT_COORDINATOR_FOR_GROUP.code, syncGroupResult._2)
+  }
+
+  @Test
+  def testSyncGroupFromUnknownGroup() {
+    val groupId = "groupId"
+    val memberId = "member"
+    val generation = 1
+
+    val syncGroupResult = syncGroupFollower(groupId, generation, memberId, true)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID.code, syncGroupResult._2)
+  }
+
+  @Test
+  def testSyncGroupFromUnknownMember() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+    val assignedConsumerId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    assertEquals(Errors.NONE.code, joinGroupResult.errorCode)
+
+    EasyMock.reset(offsetManager)
+    val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map.empty, true)
+    val syncGroupErrorCode = syncGroupResult._2
+    assertEquals(Errors.NONE.code, syncGroupErrorCode)
+
+    EasyMock.reset(offsetManager)
+    val unknownMemberId = "blah"
+    val unknownMemberSyncResult = syncGroupFollower(groupId, generationId, unknownMemberId, true)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID.code, unknownMemberSyncResult._2)
+  }
+
+  @Test
+  def testSyncGroupFromIllegalGeneration() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+    val assignedConsumerId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    assertEquals(Errors.NONE.code, joinGroupResult.errorCode)
+
+    EasyMock.reset(offsetManager)
+    // send the sync group with an invalid generation
+    val syncGroupResult = syncGroupLeader(groupId, generationId+1, assignedConsumerId, Map.empty, true)
+    assertEquals(Errors.ILLEGAL_GENERATION.code, syncGroupResult._2)
+  }
+
+  @Test
+  def testJoinGroupFromUnchangedFollowerDoesNotRebalance() {
+    val groupId = "groupId"
+    val protocolType = "consumer"
+    val protocols = List(("range", Array[Byte]()))
+
+    // to get a group of two members:
+    // 1. join and sync with a single member (because we can't immediately join with two members)
+    // 2. join and sync with the first member and a new member
+
+    val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout,
+      protocolType, protocols, isCoordinatorForGroup = true)
+    val firstMemberId = firstJoinResult.memberId
+    val firstGenerationId = firstJoinResult.generationId
+    assertEquals(firstMemberId, firstJoinResult.leaderId)
+    assertEquals(Errors.NONE.code, firstJoinResult.errorCode)
+
+    EasyMock.reset(offsetManager)
+    val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true)
+    assertEquals(Errors.NONE.code, firstSyncResult._2)
+
+    EasyMock.reset(offsetManager)
+    val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout,
+      protocolType, protocols, isCoordinatorForGroup = true)
+
+    EasyMock.reset(offsetManager)
+    val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true);
+
+    val joinResult = await(joinFuture, DefaultSessionTimeout+100)
+    val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100)
+    assertEquals(Errors.NONE.code, joinResult.errorCode)
+    assertEquals(Errors.NONE.code, otherJoinResult.errorCode)
+    assertTrue(joinResult.generationId == otherJoinResult.generationId)
+
+    assertEquals(firstMemberId, joinResult.leaderId)
+    assertEquals(firstMemberId, otherJoinResult.leaderId)
+
+    val nextGenerationId = joinResult.generationId
+
+    // this shouldn't cause a rebalance since protocol information hasn't changed
+    EasyMock.reset(offsetManager)
+    val followerJoinResult = joinGroup(groupId, otherJoinResult.memberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+
+    assertEquals(Errors.NONE.code, followerJoinResult.errorCode)
+    assertEquals(nextGenerationId, followerJoinResult.generationId)
+  }
+
+  @Test
+  def testJoinGroupFromUnchangedLeaderShouldRebalance() {
+    val groupId = "groupId"
+    val protocolType = "consumer"
+    val protocols = List(("range", Array[Byte]()))
+
+    val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout,
+      protocolType, protocols, isCoordinatorForGroup = true)
+    val firstMemberId = firstJoinResult.memberId
+    val firstGenerationId = firstJoinResult.generationId
+    assertEquals(firstMemberId, firstJoinResult.leaderId)
+    assertEquals(Errors.NONE.code, firstJoinResult.errorCode)
+
+    EasyMock.reset(offsetManager)
+    val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true)
+    assertEquals(Errors.NONE.code, firstSyncResult._2)
+
+    // join groups from the leader should force the group to rebalance, which allows the
+    // leader to push new assignments when local metadata changes
+
+    EasyMock.reset(offsetManager)
+    val secondJoinResult = joinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+
+    assertEquals(Errors.NONE.code, secondJoinResult.errorCode)
+    assertNotEquals(firstGenerationId, secondJoinResult.generationId)
+  }
+
+  @Test
+  def testLeaderFailureInSyncGroup() {
+    val groupId = "groupId"
+    val protocolType = "consumer"
+    val protocols = List(("range", Array[Byte]()))
+
+    // to get a group of two members:
+    // 1. join and sync with a single member (because we can't immediately join with two members)
+    // 2. join and sync with the first member and a new member
+
+    val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout,
+      protocolType, protocols, isCoordinatorForGroup = true)
+    val firstMemberId = firstJoinResult.memberId
+    val firstGenerationId = firstJoinResult.generationId
+    assertEquals(firstMemberId, firstJoinResult.leaderId)
+    assertEquals(Errors.NONE.code, firstJoinResult.errorCode)
+
+    EasyMock.reset(offsetManager)
+    val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true)
+    assertEquals(Errors.NONE.code, firstSyncResult._2)
+
+    EasyMock.reset(offsetManager)
+    val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout,
+      protocolType, protocols, isCoordinatorForGroup = true)
+
+    EasyMock.reset(offsetManager)
+    val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true);
+
+    val joinResult = await(joinFuture, DefaultSessionTimeout+100)
+    val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100)
+    assertEquals(Errors.NONE.code, joinResult.errorCode)
+    assertEquals(Errors.NONE.code, otherJoinResult.errorCode)
+    assertTrue(joinResult.generationId == otherJoinResult.generationId)
+
+    assertEquals(firstMemberId, joinResult.leaderId)
+    assertEquals(firstMemberId, otherJoinResult.leaderId)
+
+    val nextGenerationId = joinResult.generationId
+
+    // with no leader SyncGroup, the follower's request should failure with an error indicating
+    // that it should rejoin
+    EasyMock.reset(offsetManager)
+    val followerSyncFuture= sendSyncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId,
+      isCoordinatorForGroup = true)
+    val followerSyncResult = await(followerSyncFuture, DefaultSessionTimeout+100)
+    assertEquals(Errors.REBALANCE_IN_PROGRESS.code, followerSyncResult._2)
+  }
+
+  @Test
+  def testSyncGroupFollowerAfterLeader() {
+    val groupId = "groupId"
+    val protocolType = "consumer"
+    val protocols = List(("range", Array[Byte]()))
+
+    // to get a group of two members:
+    // 1. join and sync with a single member (because we can't immediately join with two members)
+    // 2. join and sync with the first member and a new member
+
+    val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout,
+      protocolType, protocols, isCoordinatorForGroup = true)
+    val firstMemberId = firstJoinResult.memberId
+    val firstGenerationId = firstJoinResult.generationId
+    assertEquals(firstMemberId, firstJoinResult.leaderId)
+    assertEquals(Errors.NONE.code, firstJoinResult.errorCode)
+
+    EasyMock.reset(offsetManager)
+    val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true)
+    assertEquals(Errors.NONE.code, firstSyncResult._2)
+
+    EasyMock.reset(offsetManager)
+    val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout,
+      protocolType, protocols, isCoordinatorForGroup = true)
+
+    EasyMock.reset(offsetManager)
+    val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true);
+
+    val joinResult = await(joinFuture, DefaultSessionTimeout+100)
+    val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100)
+    assertEquals(Errors.NONE.code, joinResult.errorCode)
+    assertEquals(Errors.NONE.code, otherJoinResult.errorCode)
+    assertTrue(joinResult.generationId == otherJoinResult.generationId)
+
+    assertEquals(firstMemberId, joinResult.leaderId)
+    assertEquals(firstMemberId, otherJoinResult.leaderId)
+
+    val nextGenerationId = joinResult.generationId
+    val leaderId = firstMemberId
+    val leaderAssignment = Array[Byte](0)
+    val followerId = otherJoinResult.memberId
+    val followerAssignment = Array[Byte](1)
+
+    EasyMock.reset(offsetManager)
+    val leaderSyncResult = syncGroupLeader(groupId, nextGenerationId, leaderId,
+      Map(leaderId -> leaderAssignment, followerId -> followerAssignment), isCoordinatorForGroup = true)
+    assertEquals(Errors.NONE.code, leaderSyncResult._2)
+    assertEquals(leaderAssignment, leaderSyncResult._1)
+
+    EasyMock.reset(offsetManager)
+    val followerSyncResult = syncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId,
+      isCoordinatorForGroup = true)
+    assertEquals(Errors.NONE.code, followerSyncResult._2)
+    assertEquals(followerAssignment, followerSyncResult._1)
+  }
+
+  @Test
+  def testSyncGroupLeaderAfterFollower() {
+    val groupId = "groupId"
+    val protocolType = "consumer"
+    val protocols = List(("range", Array[Byte]()))
+
+    // to get a group of two members:
+    // 1. join and sync with a single member (because we can't immediately join with two members)
+    // 2. join and sync with the first member and a new member
+
+    val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout,
+      protocolType, protocols, isCoordinatorForGroup = true)
+    val firstMemberId = joinGroupResult.memberId
+    val firstGenerationId = joinGroupResult.generationId
+    assertEquals(firstMemberId, joinGroupResult.leaderId)
+    assertEquals(Errors.NONE.code, joinGroupResult.errorCode)
+
+    EasyMock.reset(offsetManager)
+    val syncGroupResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map.empty, true)
+    val syncGroupErrorCode = syncGroupResult._2
+    assertEquals(Errors.NONE.code, syncGroupErrorCode)
+
+    EasyMock.reset(offsetManager)
+    val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout,
+      protocolType, protocols, isCoordinatorForGroup = true)
+
+    EasyMock.reset(offsetManager)
+    val joinFuture = sendJoinGroup(groupId, firstMemberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true);
+
+    val joinResult = await(joinFuture, DefaultSessionTimeout+100)
+    val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100)
+    assertEquals(Errors.NONE.code, joinResult.errorCode)
+    assertEquals(Errors.NONE.code, otherJoinResult.errorCode)
+    assertTrue(joinResult.generationId == otherJoinResult.generationId)
+
+    val nextGenerationId = joinResult.generationId
+    val leaderId = joinResult.leaderId
+    val leaderAssignment = Array[Byte](0)
+    val followerId = otherJoinResult.memberId
+    val followerAssignment = Array[Byte](1)
+
+    assertEquals(firstMemberId, joinResult.leaderId)
+    assertEquals(firstMemberId, otherJoinResult.leaderId)
+
+    EasyMock.reset(offsetManager)
+    val followerSyncFuture = sendSyncGroupFollower(groupId, nextGenerationId, followerId, isCoordinatorForGroup = true)
+
+    EasyMock.reset(offsetManager)
+    val leaderSyncResult = syncGroupLeader(groupId, nextGenerationId, leaderId,
+      Map(leaderId -> leaderAssignment, followerId -> followerAssignment), isCoordinatorForGroup = true)
+    assertEquals(Errors.NONE.code, leaderSyncResult._2)
+    assertEquals(leaderAssignment, leaderSyncResult._1)
+
+    val followerSyncResult = await(followerSyncFuture, DefaultSessionTimeout+100)
+    assertEquals(Errors.NONE.code, followerSyncResult._2)
+    assertEquals(followerAssignment, followerSyncResult._1)
+  }
+
+  @Test
+  def testCommitOffsetFromUnknownGroup() {
+    val groupId = "groupId"
+    val consumerId = "consumer"
+    val generationId = 1
+    val tp = new TopicAndPartition("topic", 0)
+    val offset = OffsetAndMetadata(0)
+
+    val commitOffsetResult = commitOffsets(groupId, consumerId, generationId, Map(tp -> offset), true)
+    assertEquals(Errors.ILLEGAL_GENERATION.code, commitOffsetResult(tp))
+  }
+
+  @Test
+  def testCommitOffsetWithDefaultGeneration() {
+    val groupId = "groupId"
+    val tp = new TopicAndPartition("topic", 0)
+    val offset = OffsetAndMetadata(0)
+
+    val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID,
+      OffsetCommitRequest.DEFAULT_GENERATION_ID, Map(tp -> offset), true)
+    assertEquals(Errors.NONE.code, commitOffsetResult(tp))
+  }
+
+  @Test
+  def testCommitOffsetInAwaitingSync() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+    val tp = new TopicAndPartition("topic", 0)
+    val offset = OffsetAndMetadata(0)
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+    val assignedMemberId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(Errors.NONE.code, joinGroupErrorCode)
+
+    EasyMock.reset(offsetManager)
+    val commitOffsetResult = commitOffsets(groupId, assignedMemberId, generationId, Map(tp -> offset), true)
+    assertEquals(Errors.REBALANCE_IN_PROGRESS.code, commitOffsetResult(tp))
+  }
+
+  @Test
+  def testHeartbeatDuringRebalanceCausesRebalanceInProgress() {
+    val groupId = "groupId"
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+
+    // First start up a group (with a slightly larger timeout to give us time to heartbeat when the rebalance starts)
+    val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout,
+      protocolType, protocols, isCoordinatorForGroup = true)
+    val assignedConsumerId = joinGroupResult.memberId
+    val initialGenerationId = joinGroupResult.generationId
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(Errors.NONE.code, joinGroupErrorCode)
+
+    // Then join with a new consumer to trigger a rebalance
+    EasyMock.reset(offsetManager)
+    sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+
+    // We should be in the middle of a rebalance, so the heartbeat should return rebalance in progress
+    EasyMock.reset(offsetManager)
+    val heartbeatResult = heartbeat(groupId, assignedConsumerId, initialGenerationId, isCoordinatorForGroup = true)
+    assertEquals(Errors.REBALANCE_IN_PROGRESS.code, heartbeatResult)
+  }
+
+  @Test
+  def testGenerationIdIncrementsOnRebalance() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+    val initialGenerationId = joinGroupResult.generationId
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(1, initialGenerationId)
+    assertEquals(Errors.NONE.code, joinGroupErrorCode)
+
+    EasyMock.reset(offsetManager)
+    val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+    val nextGenerationId = otherJoinGroupResult.generationId
+    val otherJoinGroupErrorCode = otherJoinGroupResult.errorCode
+    assertEquals(2, nextGenerationId)
+    assertEquals(Errors.NONE.code, otherJoinGroupErrorCode)
+  }
+
+  @Test
+  def testLeaveGroupWrongCoordinator() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val leaveGroupResult = leaveGroup(groupId, memberId, isCoordinatorForGroup = false)
+    assertEquals(Errors.NOT_COORDINATOR_FOR_GROUP.code, leaveGroupResult)
+  }
+
+  @Test
+  def testLeaveGroupUnknownGroup() {
+    val groupId = "groupId"
+    val memberId = "consumerId"
+
+    val leaveGroupResult = leaveGroup(groupId, memberId, isCoordinatorForGroup = true)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID.code, leaveGroupResult)
+  }
+
+  @Test
+  def testLeaveGroupUnknownConsumerExistingGroup() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val otherMemberId = "consumerId"
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(Errors.NONE.code, joinGroupErrorCode)
+
+    EasyMock.reset(offsetManager)
+    val leaveGroupResult = leaveGroup(groupId, otherMemberId, isCoordinatorForGroup = true)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID.code, leaveGroupResult)
+  }
+
+  @Test
+  def testValidLeaveGroup() {
+    val groupId = "groupId"
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val metadata = Array[Byte]()
+    val protocolType = "consumer"
+    val protocols = List(("range", metadata))
+
+    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols,
+      isCoordinatorForGroup = true)
+    val assignedMemberId = joinGroupResult.memberId
+    val joinGroupErrorCode = joinGroupResult.errorCode
+    assertEquals(Errors.NONE.code, joinGroupErrorCode)
+
+    EasyMock.reset(offsetManager)
+    val leaveGroupResult = leaveGroup(groupId, assignedMemberId, isCoordinatorForGroup = true)
+    assertEquals(Errors.NONE.code, leaveGroupResult)
+  }
+
+  private def setupJoinGroupCallback: (Future[JoinGroupResult], JoinGroupCallback) = {
+    val responsePromise = Promise[JoinGroupResult]
+    val responseFuture = responsePromise.future
+    val responseCallback: JoinGroupCallback = responsePromise.success(_)
+    (responseFuture, responseCallback)
+  }
+
+  private def setupSyncGroupCallback: (Future[SyncGroupCallbackParams], SyncGroupCallback) = {
+    val responsePromise = Promise[SyncGroupCallbackParams]
+    val responseFuture = responsePromise.future
+    val responseCallback: SyncGroupCallback = (assignment, errorCode) =>
+      responsePromise.success((assignment, errorCode))
+    (responseFuture, responseCallback)
+  }
+
+  private def setupHeartbeatCallback: (Future[HeartbeatCallbackParams], HeartbeatCallback) = {
+    val responsePromise = Promise[HeartbeatCallbackParams]
+    val responseFuture = responsePromise.future
+    val responseCallback: HeartbeatCallback = errorCode => responsePromise.success(errorCode)
+    (responseFuture, responseCallback)
+  }
+
+  private def setupCommitOffsetsCallback: (Future[CommitOffsetCallbackParams], CommitOffsetCallback) = {
+    val responsePromise = Promise[CommitOffsetCallbackParams]
+    val responseFuture = responsePromise.future
+    val responseCallback: CommitOffsetCallback = offsets => responsePromise.success(offsets)
+    (responseFuture, responseCallback)
+  }
+
+  private def sendJoinGroup(groupId: String,
+                            memberId: String,
+                            sessionTimeout: Int,
+                            protocolType: String,
+                            protocols: List[(String, Array[Byte])],
+                            isCoordinatorForGroup: Boolean): Future[JoinGroupResult] = {
+    val (responseFuture, responseCallback) = setupJoinGroupCallback
+    EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1)
+    EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup)
+    EasyMock.replay(offsetManager)
+    consumerCoordinator.handleJoinGroup(groupId, memberId, sessionTimeout, protocolType, protocols, responseCallback)
+    responseFuture
+  }
+
+
+  private def sendSyncGroupLeader(groupId: String,
+                                  generation: Int,
+                                  leaderId: String,
+                                  assignment: Map[String, Array[Byte]],
+                                  isCoordinatorForGroup: Boolean): Future[SyncGroupCallbackParams] = {
+    val (responseFuture, responseCallback) = setupSyncGroupCallback
+    EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1)
+    EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup)
+    EasyMock.replay(offsetManager)
+    consumerCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback)
+    responseFuture
+  }
+
+  private def sendSyncGroupFollower(groupId: String,
+                                    generation: Int,
+                                    memberId: String,
+                                    isCoordinatorForGroup: Boolean): Future[SyncGroupCallbackParams] = {
+    val (responseFuture, responseCallback) = setupSyncGroupCallback
+    EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1)
+    EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup)
+    EasyMock.replay(offsetManager)
+    consumerCoordinator.handleSyncGroup(groupId, generation, memberId, Map.empty[String, Array[Byte]], responseCallback)
+    responseFuture
+  }
+
+  private def joinGroup(groupId: String,
+                        memberId: String,
+                        sessionTimeout: Int,
+                        protocolType: String,
+                        protocols: List[(String, Array[Byte])],
+                        isCoordinatorForGroup: Boolean): JoinGroupResult = {
+    val responseFuture = sendJoinGroup(groupId, memberId, sessionTimeout, protocolType, protocols, isCoordinatorForGroup)
+    // should only have to wait as long as session timeout, but allow some extra time in case of an unexpected delay
+    Await.result(responseFuture, Duration(sessionTimeout+100, TimeUnit.MILLISECONDS))
+  }
+
+
+  private def syncGroupFollower(groupId: String,
+                                generationId: Int,
+                                memberId: String,
+                                isCoordinatorForGroup: Boolean): SyncGroupCallbackParams = {
+    val responseFuture = sendSyncGroupFollower(groupId, generationId, memberId, isCoordinatorForGroup)
+    Await.result(responseFuture, Duration(DefaultSessionTimeout+100, TimeUnit.MILLISECONDS))
+  }
+
+  private def syncGroupLeader(groupId: String,
+                              generationId: Int,
+                              memberId: String,
+                              assignment: Map[String, Array[Byte]],
+                              isCoordinatorForGroup: Boolean): SyncGroupCallbackParams = {
+    val responseFuture = sendSyncGroupLeader(groupId, generationId, memberId, assignment, isCoordinatorForGroup)
+    Await.result(responseFuture, Duration(DefaultSessionTimeout+100, TimeUnit.MILLISECONDS))
+  }
+
+  private def heartbeat(groupId: String,
+                        consumerId: String,
+                        generationId: Int,
+                        isCoordinatorForGroup: Boolean): HeartbeatCallbackParams = {
+    val (responseFuture, responseCallback) = setupHeartbeatCallback
+    EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1)
+    EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup)
+    EasyMock.replay(offsetManager)
+    consumerCoordinator.handleHeartbeat(groupId, consumerId, generationId, responseCallback)
+    Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS))
+  }
+
+  private def await[T](future: Future[T], millis: Long): T = {
+    Await.result(future, Duration(millis, TimeUnit.MILLISECONDS))
+  }
+
+  private def commitOffsets(groupId: String,
+                            consumerId: String,
+                            generationId: Int,
+                            offsets: Map[TopicAndPartition, OffsetAndMetadata],
+                            isCoordinatorForGroup: Boolean): CommitOffsetCallbackParams = {
+    val (responseFuture, responseCallback) = setupCommitOffsetsCallback
+    EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1)
+    EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup)
+    val storeOffsetAnswer = new IAnswer[Unit] {
+      override def answer = responseCallback.apply(offsets.mapValues(_ => Errors.NONE.code))
+    }
+    EasyMock.expect(offsetManager.storeOffsets(groupId, consumerId, generationId, offsets, responseCallback))
+      .andAnswer(storeOffsetAnswer)
+    EasyMock.replay(offsetManager)
+    consumerCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback)
+    Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS))
+  }
+
+  private def leaveGroup(groupId: String, consumerId: String, isCoordinatorForGroup: Boolean): LeaveGroupCallbackParams = {
+    val (responseFuture, responseCallback) = setupHeartbeatCallback
+    EasyMock.expect(offsetManager.partitionFor(groupId)).andReturn(1)
+    EasyMock.expect(offsetManager.leaderIsLocal(1)).andReturn(isCoordinatorForGroup)
+    EasyMock.replay(offsetManager)
+    consumerCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback)
+    Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS))
+  }
+}


[5/8] kafka git commit: KAFKA-2464: client-side assignment for new consumer

Posted by gu...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
new file mode 100644
index 0000000..93994d7
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
@@ -0,0 +1,749 @@
+/**
+ * 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 org.apache.kafka.clients.ClientRequest;
+import org.apache.kafka.clients.Metadata;
+import org.apache.kafka.clients.MockClient;
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.clients.consumer.OffsetCommitCallback;
+import org.apache.kafka.clients.consumer.OffsetResetStrategy;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.DisconnectException;
+import org.apache.kafka.common.errors.OffsetMetadataTooLarge;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.requests.GroupMetadataResponse;
+import org.apache.kafka.common.requests.HeartbeatResponse;
+import org.apache.kafka.common.requests.JoinGroupResponse;
+import org.apache.kafka.common.requests.OffsetCommitRequest;
+import org.apache.kafka.common.requests.OffsetCommitResponse;
+import org.apache.kafka.common.requests.OffsetFetchResponse;
+import org.apache.kafka.common.requests.SyncGroupRequest;
+import org.apache.kafka.common.requests.SyncGroupResponse;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+public class ConsumerCoordinatorTest {
+
+    private String topicName = "test";
+    private String groupId = "test-group";
+    private TopicPartition tp = new TopicPartition(topicName, 0);
+    private int sessionTimeoutMs = 10;
+    private int heartbeatIntervalMs = 2;
+    private long retryBackoffMs = 100;
+    private long requestTimeoutMs = 5000;
+    private boolean autoCommitEnabled = false;
+    private long autoCommitIntervalMs = 5000;
+    private MockPartitionAssignor partitionAssignor = new MockPartitionAssignor();
+    private List<PartitionAssignor> assignors = Arrays.<PartitionAssignor>asList(partitionAssignor);
+    private MockTime time;
+    private MockClient client;
+    private Cluster cluster = TestUtils.singletonCluster(topicName, 1);
+    private Node node = cluster.nodes().get(0);
+    private SubscriptionState subscriptions;
+    private Metadata metadata;
+    private Metrics metrics;
+    private Map<String, String> metricTags = new LinkedHashMap<>();
+    private ConsumerNetworkClient consumerClient;
+    private MockRebalanceListener rebalanceListener;
+    private MockCommitCallback defaultOffsetCommitCallback;
+    private ConsumerCoordinator coordinator;
+
+    @Before
+    public void setup() {
+        this.time = new MockTime();
+        this.client = new MockClient(time);
+        this.subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST);
+        this.metadata = new Metadata(0, Long.MAX_VALUE);
+        this.metadata.update(cluster, time.milliseconds());
+        this.consumerClient = new ConsumerNetworkClient(client, metadata, time, 100);
+        this.metrics = new Metrics(time);
+        this.rebalanceListener = new MockRebalanceListener();
+        this.defaultOffsetCommitCallback = new MockCommitCallback();
+        this.partitionAssignor.clear();
+
+        client.setNode(node);
+
+        this.coordinator = new ConsumerCoordinator(
+                consumerClient,
+                groupId,
+                sessionTimeoutMs,
+                heartbeatIntervalMs,
+                assignors,
+                metadata,
+                subscriptions,
+                metrics,
+                "consumer" + groupId,
+                metricTags,
+                time,
+                requestTimeoutMs,
+                retryBackoffMs,
+                defaultOffsetCommitCallback,
+                autoCommitEnabled,
+                autoCommitIntervalMs);
+    }
+
+    @After
+    public void teardown() {
+        this.metrics.close();
+    }
+
+    @Test
+    public void testNormalHeartbeat() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // normal heartbeat
+        time.sleep(sessionTimeoutMs);
+        RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
+        assertEquals(1, consumerClient.pendingRequestCount());
+        assertFalse(future.isDone());
+
+        client.prepareResponse(heartbeatResponse(Errors.NONE.code()));
+        consumerClient.poll(0);
+
+        assertTrue(future.isDone());
+        assertTrue(future.succeeded());
+    }
+
+    @Test
+    public void testCoordinatorNotAvailable() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // GROUP_COORDINATOR_NOT_AVAILABLE will mark coordinator as unknown
+        time.sleep(sessionTimeoutMs);
+        RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
+        assertEquals(1, consumerClient.pendingRequestCount());
+        assertFalse(future.isDone());
+
+        client.prepareResponse(heartbeatResponse(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()));
+        time.sleep(sessionTimeoutMs);
+        consumerClient.poll(0);
+
+        assertTrue(future.isDone());
+        assertTrue(future.failed());
+        assertEquals(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.exception(), future.exception());
+        assertTrue(coordinator.coordinatorUnknown());
+    }
+
+    @Test
+    public void testNotCoordinator() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // not_coordinator will mark coordinator as unknown
+        time.sleep(sessionTimeoutMs);
+        RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
+        assertEquals(1, consumerClient.pendingRequestCount());
+        assertFalse(future.isDone());
+
+        client.prepareResponse(heartbeatResponse(Errors.NOT_COORDINATOR_FOR_GROUP.code()));
+        time.sleep(sessionTimeoutMs);
+        consumerClient.poll(0);
+
+        assertTrue(future.isDone());
+        assertTrue(future.failed());
+        assertEquals(Errors.NOT_COORDINATOR_FOR_GROUP.exception(), future.exception());
+        assertTrue(coordinator.coordinatorUnknown());
+    }
+
+    @Test
+    public void testIllegalGeneration() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // illegal_generation will cause re-partition
+        subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener);
+        subscriptions.changePartitionAssignment(Collections.singletonList(tp));
+
+        time.sleep(sessionTimeoutMs);
+        RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
+        assertEquals(1, consumerClient.pendingRequestCount());
+        assertFalse(future.isDone());
+
+        client.prepareResponse(heartbeatResponse(Errors.ILLEGAL_GENERATION.code()));
+        time.sleep(sessionTimeoutMs);
+        consumerClient.poll(0);
+
+        assertTrue(future.isDone());
+        assertTrue(future.failed());
+        assertEquals(Errors.ILLEGAL_GENERATION.exception(), future.exception());
+        assertTrue(coordinator.needRejoin());
+    }
+
+    @Test
+    public void testUnknownConsumerId() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // illegal_generation will cause re-partition
+        subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener);
+        subscriptions.changePartitionAssignment(Collections.singletonList(tp));
+
+        time.sleep(sessionTimeoutMs);
+        RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
+        assertEquals(1, consumerClient.pendingRequestCount());
+        assertFalse(future.isDone());
+
+        client.prepareResponse(heartbeatResponse(Errors.UNKNOWN_MEMBER_ID.code()));
+        time.sleep(sessionTimeoutMs);
+        consumerClient.poll(0);
+
+        assertTrue(future.isDone());
+        assertTrue(future.failed());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.exception(), future.exception());
+        assertTrue(coordinator.needRejoin());
+    }
+
+    @Test
+    public void testCoordinatorDisconnect() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // coordinator disconnect will mark coordinator as unknown
+        time.sleep(sessionTimeoutMs);
+        RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
+        assertEquals(1, consumerClient.pendingRequestCount());
+        assertFalse(future.isDone());
+
+        client.prepareResponse(heartbeatResponse(Errors.NONE.code()), true); // return disconnected
+        time.sleep(sessionTimeoutMs);
+        consumerClient.poll(0);
+
+        assertTrue(future.isDone());
+        assertTrue(future.failed());
+        assertTrue(future.exception() instanceof DisconnectException);
+        assertTrue(coordinator.coordinatorUnknown());
+    }
+
+    @Test
+    public void testNormalJoinGroupLeader() {
+        final String consumerId = "leader";
+
+        subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener);
+        subscriptions.needReassignment();
+
+        // ensure metadata is up-to-date for leader
+        metadata.setTopics(Arrays.asList(topicName));
+        metadata.update(cluster, time.milliseconds());
+
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // normal join group
+        Map<String, List<String>> memberSubscriptions = Collections.singletonMap(consumerId, Arrays.asList(topicName));
+        partitionAssignor.prepare(Collections.singletonMap(consumerId, Arrays.asList(tp)));
+
+        client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code()));
+        client.prepareResponse(new MockClient.RequestMatcher() {
+            @Override
+            public boolean matches(ClientRequest request) {
+                SyncGroupRequest sync = new SyncGroupRequest(request.request().body());
+                return sync.memberId().equals(consumerId) &&
+                        sync.generationId() == 1 &&
+                        sync.groupAssignment().containsKey(consumerId);
+            }
+        }, syncGroupResponse(Arrays.asList(tp), Errors.NONE.code()));
+        coordinator.ensurePartitionAssignment();
+
+        assertFalse(subscriptions.partitionAssignmentNeeded());
+        assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions());
+        assertEquals(1, rebalanceListener.revokedCount);
+        assertEquals(Collections.emptySet(), rebalanceListener.revoked);
+        assertEquals(1, rebalanceListener.assignedCount);
+        assertEquals(Collections.singleton(tp), rebalanceListener.assigned);
+    }
+
+    @Test
+    public void testNormalJoinGroupFollower() {
+        final String consumerId = "consumer";
+
+        subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener);
+        subscriptions.needReassignment();
+
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // normal join group
+        client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code()));
+        client.prepareResponse(new MockClient.RequestMatcher() {
+            @Override
+            public boolean matches(ClientRequest request) {
+                SyncGroupRequest sync = new SyncGroupRequest(request.request().body());
+                return sync.memberId().equals(consumerId) &&
+                        sync.generationId() == 1 &&
+                        sync.groupAssignment().isEmpty();
+            }
+        }, syncGroupResponse(Arrays.asList(tp), Errors.NONE.code()));
+
+        coordinator.ensurePartitionAssignment();
+
+        assertFalse(subscriptions.partitionAssignmentNeeded());
+        assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions());
+        assertEquals(1, rebalanceListener.revokedCount);
+        assertEquals(1, rebalanceListener.assignedCount);
+        assertEquals(Collections.singleton(tp), rebalanceListener.assigned);
+    }
+
+
+    @Test
+    public void testMetadataChangeTriggersRebalance() {
+        final String consumerId = "consumer";
+
+        subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener);
+        subscriptions.needReassignment();
+
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code()));
+        client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code()));
+
+        coordinator.ensurePartitionAssignment();
+
+        assertFalse(subscriptions.partitionAssignmentNeeded());
+
+        // a new partition is added to the topic
+        metadata.update(TestUtils.singletonCluster(topicName, 2), time.milliseconds());
+
+        // we should detect the change and ask for reassignment
+        assertTrue(subscriptions.partitionAssignmentNeeded());
+    }
+
+    @Test
+    public void testRejoinGroup() {
+        subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener);
+        subscriptions.needReassignment();
+
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // join the group once
+        client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code()));
+        client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code()));
+        coordinator.ensurePartitionAssignment();
+
+        assertEquals(1, rebalanceListener.revokedCount);
+        assertEquals(1, rebalanceListener.assignedCount);
+
+        // and join the group again
+        subscriptions.needReassignment();
+        client.prepareResponse(joinGroupFollowerResponse(2, "consumer", "leader", Errors.NONE.code()));
+        client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code()));
+        coordinator.ensurePartitionAssignment();
+
+        assertEquals(2, rebalanceListener.revokedCount);
+        assertEquals(Collections.singleton(tp), rebalanceListener.revoked);
+        assertEquals(2, rebalanceListener.assignedCount);
+        assertEquals(Collections.singleton(tp), rebalanceListener.assigned);
+    }
+
+    @Test
+    public void testDisconnectInJoin() {
+        subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener);
+        subscriptions.needReassignment();
+
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // disconnected from original coordinator will cause re-discover and join again
+        client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code()), true);
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code()));
+        client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code()));
+        coordinator.ensurePartitionAssignment();
+        assertFalse(subscriptions.partitionAssignmentNeeded());
+        assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions());
+        assertEquals(1, rebalanceListener.revokedCount);
+        assertEquals(1, rebalanceListener.assignedCount);
+        assertEquals(Collections.singleton(tp), rebalanceListener.assigned);
+    }
+
+    @Test(expected = ApiException.class)
+    public void testInvalidSessionTimeout() {
+        subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener);
+        subscriptions.needReassignment();
+
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // coordinator doesn't like the session timeout
+        client.prepareResponse(joinGroupFollowerResponse(0, "consumer", "", Errors.INVALID_SESSION_TIMEOUT.code()));
+        coordinator.ensurePartitionAssignment();
+    }
+
+    @Test
+    public void testCommitOffsetOnly() {
+        subscriptions.assign(Arrays.asList(tp));
+
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
+
+        AtomicBoolean success = new AtomicBoolean(false);
+        coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), callback(success));
+        consumerClient.poll(0);
+        assertTrue(success.get());
+
+        assertEquals(100L, subscriptions.committed(tp).offset());
+    }
+
+    @Test
+    public void testCommitOffsetMetadata() {
+        subscriptions.assign(Arrays.asList(tp));
+
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
+
+        AtomicBoolean success = new AtomicBoolean(false);
+        coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "hello")), callback(success));
+        consumerClient.poll(0);
+        assertTrue(success.get());
+
+        assertEquals(100L, subscriptions.committed(tp).offset());
+        assertEquals("hello", subscriptions.committed(tp).metadata());
+    }
+
+    @Test
+    public void testCommitOffsetAsyncWithDefaultCallback() {
+        int invokedBeforeTest = defaultOffsetCommitCallback.invoked;
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
+        coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), null);
+        consumerClient.poll(0);
+        assertEquals(invokedBeforeTest + 1, defaultOffsetCommitCallback.invoked);
+        assertNull(defaultOffsetCommitCallback.exception);
+    }
+
+    @Test
+    public void testResetGeneration() {
+        // enable auto-assignment
+        subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener);
+
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code()));
+        client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code()));
+        coordinator.ensurePartitionAssignment();
+
+        // now switch to manual assignment
+        subscriptions.unsubscribe();
+        coordinator.resetGeneration();
+        subscriptions.assign(Arrays.asList(tp));
+
+        // the client should not reuse generation/memberId from auto-subscribed generation
+        client.prepareResponse(new MockClient.RequestMatcher() {
+            @Override
+            public boolean matches(ClientRequest request) {
+                OffsetCommitRequest commitRequest = new OffsetCommitRequest(request.request().body());
+                return commitRequest.memberId().equals(OffsetCommitRequest.DEFAULT_MEMBER_ID) &&
+                        commitRequest.generationId() == OffsetCommitRequest.DEFAULT_GENERATION_ID;
+            }
+        }, offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
+
+        AtomicBoolean success = new AtomicBoolean(false);
+        coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), callback(success));
+        consumerClient.poll(0);
+        assertTrue(success.get());
+    }
+
+    @Test
+    public void testCommitOffsetAsyncFailedWithDefaultCallback() {
+        int invokedBeforeTest = defaultOffsetCommitCallback.invoked;
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code())));
+        coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), null);
+        consumerClient.poll(0);
+        assertEquals(invokedBeforeTest + 1, defaultOffsetCommitCallback.invoked);
+        assertEquals(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.exception(), defaultOffsetCommitCallback.exception);
+    }
+
+    @Test
+    public void testCommitOffsetAsyncCoordinatorNotAvailable() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // async commit with coordinator not available
+        MockCommitCallback cb = new MockCommitCallback();
+        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code())));
+        coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb);
+        consumerClient.poll(0);
+
+        assertTrue(coordinator.coordinatorUnknown());
+        assertEquals(1, cb.invoked);
+        assertEquals(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.exception(), cb.exception);
+    }
+
+    @Test
+    public void testCommitOffsetAsyncNotCoordinator() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // async commit with not coordinator
+        MockCommitCallback cb = new MockCommitCallback();
+        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_GROUP.code())));
+        coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb);
+        consumerClient.poll(0);
+
+        assertTrue(coordinator.coordinatorUnknown());
+        assertEquals(1, cb.invoked);
+        assertEquals(Errors.NOT_COORDINATOR_FOR_GROUP.exception(), cb.exception);
+    }
+
+    @Test
+    public void testCommitOffsetAsyncDisconnected() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // async commit with coordinator disconnected
+        MockCommitCallback cb = new MockCommitCallback();
+        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true);
+        coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb);
+        consumerClient.poll(0);
+
+        assertTrue(coordinator.coordinatorUnknown());
+        assertEquals(1, cb.invoked);
+        assertTrue(cb.exception instanceof DisconnectException);
+    }
+
+    @Test
+    public void testCommitOffsetSyncNotCoordinator() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request)
+        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_GROUP.code())));
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
+        coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)));
+    }
+
+    @Test
+    public void testCommitOffsetSyncCoordinatorNotAvailable() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request)
+        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code())));
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
+        coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)));
+    }
+
+    @Test
+    public void testCommitOffsetSyncCoordinatorDisconnected() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request)
+        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true);
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
+        coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)));
+    }
+
+    @Test(expected = OffsetMetadataTooLarge.class)
+    public void testCommitOffsetMetadataTooLarge() {
+        // since offset metadata is provided by the user, we have to propagate the exception so they can handle it
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.OFFSET_METADATA_TOO_LARGE.code())));
+        coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "metadata")));
+    }
+
+    @Test(expected = ApiException.class)
+    public void testCommitOffsetSyncCallbackWithNonRetriableException() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        // sync commit with invalid partitions should throw if we have no callback
+        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.UNKNOWN.code())), false);
+        coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)));
+    }
+
+    @Test
+    public void testRefreshOffset() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        subscriptions.assign(Arrays.asList(tp));
+        subscriptions.needRefreshCommits();
+        client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L));
+        coordinator.refreshCommittedOffsetsIfNeeded();
+        assertFalse(subscriptions.refreshCommitsNeeded());
+        assertEquals(100L, subscriptions.committed(tp).offset());
+    }
+
+    @Test
+    public void testRefreshOffsetLoadInProgress() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        subscriptions.assign(Arrays.asList(tp));
+        subscriptions.needRefreshCommits();
+        client.prepareResponse(offsetFetchResponse(tp, Errors.OFFSET_LOAD_IN_PROGRESS.code(), "", 100L));
+        client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L));
+        coordinator.refreshCommittedOffsetsIfNeeded();
+        assertFalse(subscriptions.refreshCommitsNeeded());
+        assertEquals(100L, subscriptions.committed(tp).offset());
+    }
+
+    @Test
+    public void testRefreshOffsetNotCoordinatorForConsumer() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        subscriptions.assign(Arrays.asList(tp));
+        subscriptions.needRefreshCommits();
+        client.prepareResponse(offsetFetchResponse(tp, Errors.NOT_COORDINATOR_FOR_GROUP.code(), "", 100L));
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L));
+        coordinator.refreshCommittedOffsetsIfNeeded();
+        assertFalse(subscriptions.refreshCommitsNeeded());
+        assertEquals(100L, subscriptions.committed(tp).offset());
+    }
+
+    @Test
+    public void testRefreshOffsetWithNoFetchableOffsets() {
+        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
+        coordinator.ensureCoordinatorKnown();
+
+        subscriptions.assign(Arrays.asList(tp));
+        subscriptions.needRefreshCommits();
+        client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L));
+        coordinator.refreshCommittedOffsetsIfNeeded();
+        assertFalse(subscriptions.refreshCommitsNeeded());
+        assertEquals(null, subscriptions.committed(tp));
+    }
+
+    private Struct consumerMetadataResponse(Node node, short error) {
+        GroupMetadataResponse response = new GroupMetadataResponse(error, node);
+        return response.toStruct();
+    }
+
+    private Struct heartbeatResponse(short error) {
+        HeartbeatResponse response = new HeartbeatResponse(error);
+        return response.toStruct();
+    }
+
+    private Struct joinGroupLeaderResponse(int generationId, String memberId,
+                                           Map<String, List<String>> subscriptions,
+                                           short error) {
+        Map<String, ByteBuffer> metadata = new HashMap<>();
+        for (Map.Entry<String, List<String>> subscriptionEntry : subscriptions.entrySet()) {
+            PartitionAssignor.Subscription subscription = new PartitionAssignor.Subscription(subscriptionEntry.getValue());
+            ByteBuffer buf = ConsumerProtocol.serializeSubscription(subscription);
+            metadata.put(subscriptionEntry.getKey(), buf);
+        }
+        return new JoinGroupResponse(error, generationId, partitionAssignor.name(), memberId, memberId, metadata).toStruct();
+    }
+
+    private Struct joinGroupFollowerResponse(int generationId, String memberId, String leaderId, short error) {
+        return new JoinGroupResponse(error, generationId, partitionAssignor.name(), memberId, leaderId,
+                Collections.<String, ByteBuffer>emptyMap()).toStruct();
+    }
+
+    private Struct syncGroupResponse(List<TopicPartition> partitions, short error) {
+        ByteBuffer buf = ConsumerProtocol.serializeAssignment(new PartitionAssignor.Assignment(partitions));
+        return new SyncGroupResponse(error, buf).toStruct();
+    }
+
+    private Struct offsetCommitResponse(Map<TopicPartition, Short> responseData) {
+        OffsetCommitResponse response = new OffsetCommitResponse(responseData);
+        return response.toStruct();
+    }
+
+    private Struct offsetFetchResponse(TopicPartition tp, Short error, String metadata, long offset) {
+        OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, metadata, error);
+        OffsetFetchResponse response = new OffsetFetchResponse(Collections.singletonMap(tp, data));
+        return response.toStruct();
+    }
+
+    private OffsetCommitCallback callback(final AtomicBoolean success) {
+        return new OffsetCommitCallback() {
+            @Override
+            public void onComplete(Map<TopicPartition, OffsetAndMetadata> offsets, Exception exception) {
+                if (exception == null)
+                    success.set(true);
+            }
+        };
+    }
+
+    private static class MockCommitCallback implements OffsetCommitCallback {
+        public int invoked = 0;
+        public Exception exception = null;
+
+        @Override
+        public void onComplete(Map<TopicPartition, OffsetAndMetadata> offsets, Exception exception) {
+            invoked++;
+            this.exception = exception;
+        }
+    }
+
+    private static class MockRebalanceListener implements ConsumerRebalanceListener {
+        public Collection<TopicPartition> revoked;
+        public Collection<TopicPartition> assigned;
+        public int revokedCount = 0;
+        public int assignedCount = 0;
+
+
+        @Override
+        public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
+            this.assigned = partitions;
+            assignedCount++;
+        }
+
+        @Override
+        public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
+            this.revoked = partitions;
+            revokedCount++;
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java
index 9de1cee..6a42058 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java
@@ -106,7 +106,7 @@ public class ConsumerNetworkClientTest {
 
 
     private HeartbeatRequest heartbeatRequest() {
-        return new HeartbeatRequest("group", 1, "consumerId");
+        return new HeartbeatRequest("group", 1, "memberId");
     }
 
     private Struct heartbeatResponse(short error) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocolTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocolTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocolTest.java
new file mode 100644
index 0000000..8113770
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocolTest.java
@@ -0,0 +1,118 @@
+/**
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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 org.apache.kafka.clients.consumer.internals.PartitionAssignor.Subscription;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.Assert.assertEquals;
+
+public class ConsumerProtocolTest {
+
+    @Test
+    public void serializeDeserializeMetadata() {
+        Subscription subscription = new Subscription(Arrays.asList("foo", "bar"));
+
+        ByteBuffer buffer = ConsumerProtocol.serializeSubscription(subscription);
+        Subscription parsedSubscription = ConsumerProtocol.deserializeSubscription(buffer);
+        assertEquals(subscription.topics(), parsedSubscription.topics());
+    }
+
+    @Test
+    public void deserializeNewSubscriptionVersion() {
+        // verify that a new version which adds a field is still parseable
+        short version = 100;
+
+        Schema subscriptionSchemaV100 = new Schema(
+                new Field(ConsumerProtocol.TOPICS_KEY_NAME, new ArrayOf(Type.STRING)),
+                new Field(ConsumerProtocol.USER_DATA_KEY_NAME, Type.BYTES),
+                new Field("foo", Type.STRING));
+
+        Struct subscriptionV100 = new Struct(subscriptionSchemaV100);
+        subscriptionV100.set(ConsumerProtocol.TOPICS_KEY_NAME, new Object[]{"topic"});
+        subscriptionV100.set(ConsumerProtocol.USER_DATA_KEY_NAME, ByteBuffer.wrap(new byte[0]));
+        subscriptionV100.set("foo", "bar");
+
+        Struct headerV100 = new Struct(ConsumerProtocol.CONSUMER_PROTOCOL_HEADER_SCHEMA);
+        headerV100.set(ConsumerProtocol.VERSION_KEY_NAME, version);
+
+        ByteBuffer buffer = ByteBuffer.allocate(subscriptionV100.sizeOf() + headerV100.sizeOf());
+        headerV100.writeTo(buffer);
+        subscriptionV100.writeTo(buffer);
+
+        buffer.flip();
+
+        Subscription subscription = ConsumerProtocol.deserializeSubscription(buffer);
+        assertEquals(Arrays.asList("topic"), subscription.topics());
+    }
+
+    @Test
+    public void serializeDeserializeAssignment() {
+        List<TopicPartition> partitions = Arrays.asList(new TopicPartition("foo", 0), new TopicPartition("bar", 2));
+        ByteBuffer buffer = ConsumerProtocol.serializeAssignment(new PartitionAssignor.Assignment(partitions));
+        PartitionAssignor.Assignment parsedAssignment = ConsumerProtocol.deserializeAssignment(buffer);
+        assertEquals(toSet(partitions), toSet(parsedAssignment.partitions()));
+    }
+
+    @Test
+    public void deserializeNewAssignmentVersion() {
+        // verify that a new version which adds a field is still parseable
+        short version = 100;
+
+        Schema assignmentSchemaV100 = new Schema(
+                new Field(ConsumerProtocol.TOPIC_PARTITIONS_KEY_NAME, new ArrayOf(ConsumerProtocol.TOPIC_ASSIGNMENT_V0)),
+                new Field(ConsumerProtocol.USER_DATA_KEY_NAME, Type.BYTES),
+                new Field("foo", Type.STRING));
+
+        Struct assignmentV100 = new Struct(assignmentSchemaV100);
+        assignmentV100.set(ConsumerProtocol.TOPIC_PARTITIONS_KEY_NAME,
+                new Object[]{new Struct(ConsumerProtocol.TOPIC_ASSIGNMENT_V0)
+                        .set(ConsumerProtocol.TOPIC_KEY_NAME, "foo")
+                        .set(ConsumerProtocol.PARTITIONS_KEY_NAME, new Object[]{1})});
+        assignmentV100.set(ConsumerProtocol.USER_DATA_KEY_NAME, ByteBuffer.wrap(new byte[0]));
+        assignmentV100.set("foo", "bar");
+
+        Struct headerV100 = new Struct(ConsumerProtocol.CONSUMER_PROTOCOL_HEADER_SCHEMA);
+        headerV100.set(ConsumerProtocol.VERSION_KEY_NAME, version);
+
+        ByteBuffer buffer = ByteBuffer.allocate(assignmentV100.sizeOf() + headerV100.sizeOf());
+        headerV100.writeTo(buffer);
+        assignmentV100.writeTo(buffer);
+
+        buffer.flip();
+
+        PartitionAssignor.Assignment assignment = ConsumerProtocol.deserializeAssignment(buffer);
+        assertEquals(toSet(Arrays.asList(new TopicPartition("foo", 1))), toSet(assignment.partitions()));
+    }
+
+    private static <T> Set<T> toSet(Collection<T> collection) {
+        return new HashSet<>(collection);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java
deleted file mode 100644
index 66b2e32..0000000
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CoordinatorTest.java
+++ /dev/null
@@ -1,635 +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.internals;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.kafka.clients.ClientRequest;
-import org.apache.kafka.clients.Metadata;
-import org.apache.kafka.clients.MockClient;
-import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.clients.consumer.OffsetCommitCallback;
-import org.apache.kafka.clients.consumer.OffsetResetStrategy;
-import org.apache.kafka.common.Cluster;
-import org.apache.kafka.common.Node;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.ApiException;
-import org.apache.kafka.common.errors.DisconnectException;
-import org.apache.kafka.common.errors.OffsetMetadataTooLarge;
-import org.apache.kafka.common.metrics.Metrics;
-import org.apache.kafka.common.protocol.Errors;
-import org.apache.kafka.common.protocol.types.Struct;
-import org.apache.kafka.common.requests.ConsumerMetadataResponse;
-import org.apache.kafka.common.requests.HeartbeatResponse;
-import org.apache.kafka.common.requests.JoinGroupResponse;
-import org.apache.kafka.common.requests.OffsetCommitRequest;
-import org.apache.kafka.common.requests.OffsetCommitResponse;
-import org.apache.kafka.common.requests.OffsetFetchResponse;
-import org.apache.kafka.common.utils.MockTime;
-import org.apache.kafka.test.TestUtils;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-
-public class CoordinatorTest {
-
-    private String topicName = "test";
-    private String groupId = "test-group";
-    private TopicPartition tp = new TopicPartition(topicName, 0);
-    private int sessionTimeoutMs = 10;
-    private int heartbeatIntervalMs = 2;
-    private long retryBackoffMs = 100;
-    private long requestTimeoutMs = 5000;
-    private boolean autoCommitEnabled = false;
-    private long autoCommitIntervalMs = 5000;
-    private String rebalanceStrategy = "not-matter";
-    private MockTime time;
-    private MockClient client;
-    private Cluster cluster = TestUtils.singletonCluster(topicName, 1);
-    private Node node = cluster.nodes().get(0);
-    private SubscriptionState subscriptions;
-    private Metadata metadata;
-    private Metrics metrics;
-    private Map<String, String> metricTags = new LinkedHashMap<String, String>();
-    private ConsumerNetworkClient consumerClient;
-    private MockRebalanceListener subscriptionListener;
-    private MockCommitCallback defaultOffsetCommitCallback;
-    private Coordinator coordinator;
-
-    @Before
-    public void setup() {
-        this.time = new MockTime();
-        this.client = new MockClient(time);
-        this.subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST);
-        this.metadata = new Metadata(0, Long.MAX_VALUE);
-        this.consumerClient = new ConsumerNetworkClient(client, metadata, time, 100);
-        this.metrics = new Metrics(time);
-        this.subscriptionListener = new MockRebalanceListener();
-        this.defaultOffsetCommitCallback = new MockCommitCallback();
-
-        client.setNode(node);
-
-        this.coordinator = new Coordinator(consumerClient,
-                groupId,
-                sessionTimeoutMs,
-                heartbeatIntervalMs,
-                rebalanceStrategy,
-                subscriptions,
-                metrics,
-                "consumer" + groupId,
-                metricTags,
-                time,
-                requestTimeoutMs,
-                retryBackoffMs,
-                defaultOffsetCommitCallback,
-                autoCommitEnabled,
-                autoCommitIntervalMs);
-    }
-
-    @After
-    public void teardown() {
-        this.metrics.close();
-    }
-
-    @Test
-    public void testNormalHeartbeat() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // normal heartbeat
-        time.sleep(sessionTimeoutMs);
-        RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
-        assertEquals(1, consumerClient.pendingRequestCount());
-        assertFalse(future.isDone());
-
-        client.prepareResponse(heartbeatResponse(Errors.NONE.code()));
-        consumerClient.poll(0);
-
-        assertTrue(future.isDone());
-        assertTrue(future.succeeded());
-    }
-
-    @Test
-    public void testCoordinatorNotAvailable() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // consumer_coordinator_not_available will mark coordinator as unknown
-        time.sleep(sessionTimeoutMs);
-        RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
-        assertEquals(1, consumerClient.pendingRequestCount());
-        assertFalse(future.isDone());
-
-        client.prepareResponse(heartbeatResponse(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()));
-        time.sleep(sessionTimeoutMs);
-        consumerClient.poll(0);
-
-        assertTrue(future.isDone());
-        assertTrue(future.failed());
-        assertEquals(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception(), future.exception());
-        assertTrue(coordinator.coordinatorUnknown());
-    }
-
-    @Test
-    public void testNotCoordinator() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // not_coordinator will mark coordinator as unknown
-        time.sleep(sessionTimeoutMs);
-        RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
-        assertEquals(1, consumerClient.pendingRequestCount());
-        assertFalse(future.isDone());
-
-        client.prepareResponse(heartbeatResponse(Errors.NOT_COORDINATOR_FOR_CONSUMER.code()));
-        time.sleep(sessionTimeoutMs);
-        consumerClient.poll(0);
-
-        assertTrue(future.isDone());
-        assertTrue(future.failed());
-        assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.exception(), future.exception());
-        assertTrue(coordinator.coordinatorUnknown());
-    }
-
-    @Test
-    public void testIllegalGeneration() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // illegal_generation will cause re-partition
-        subscriptions.subscribe(Arrays.asList(topicName), subscriptionListener);
-        subscriptions.changePartitionAssignment(Collections.singletonList(tp));
-
-        time.sleep(sessionTimeoutMs);
-        RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
-        assertEquals(1, consumerClient.pendingRequestCount());
-        assertFalse(future.isDone());
-
-        client.prepareResponse(heartbeatResponse(Errors.ILLEGAL_GENERATION.code()));
-        time.sleep(sessionTimeoutMs);
-        consumerClient.poll(0);
-
-        assertTrue(future.isDone());
-        assertTrue(future.failed());
-        assertEquals(Errors.ILLEGAL_GENERATION.exception(), future.exception());
-        assertTrue(subscriptions.partitionAssignmentNeeded());
-    }
-
-    @Test
-    public void testUnknownConsumerId() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // illegal_generation will cause re-partition
-        subscriptions.subscribe(Arrays.asList(topicName), subscriptionListener);
-        subscriptions.changePartitionAssignment(Collections.singletonList(tp));
-
-        time.sleep(sessionTimeoutMs);
-        RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
-        assertEquals(1, consumerClient.pendingRequestCount());
-        assertFalse(future.isDone());
-
-        client.prepareResponse(heartbeatResponse(Errors.UNKNOWN_CONSUMER_ID.code()));
-        time.sleep(sessionTimeoutMs);
-        consumerClient.poll(0);
-
-        assertTrue(future.isDone());
-        assertTrue(future.failed());
-        assertEquals(Errors.UNKNOWN_CONSUMER_ID.exception(), future.exception());
-        assertTrue(subscriptions.partitionAssignmentNeeded());
-    }
-
-    @Test
-    public void testCoordinatorDisconnect() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // coordinator disconnect will mark coordinator as unknown
-        time.sleep(sessionTimeoutMs);
-        RequestFuture<Void> future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat
-        assertEquals(1, consumerClient.pendingRequestCount());
-        assertFalse(future.isDone());
-
-        client.prepareResponse(heartbeatResponse(Errors.NONE.code()), true); // return disconnected
-        time.sleep(sessionTimeoutMs);
-        consumerClient.poll(0);
-
-        assertTrue(future.isDone());
-        assertTrue(future.failed());
-        assertTrue(future.exception() instanceof DisconnectException);
-        assertTrue(coordinator.coordinatorUnknown());
-    }
-
-    @Test
-    public void testNormalJoinGroup() {
-        subscriptions.subscribe(Arrays.asList(topicName), subscriptionListener);
-        subscriptions.needReassignment();
-
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // normal join group
-        client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code()));
-        coordinator.ensurePartitionAssignment();
-
-        assertFalse(subscriptions.partitionAssignmentNeeded());
-        assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions());
-        assertEquals(1, subscriptionListener.revokedCount);
-        assertEquals(Collections.emptySet(), subscriptionListener.revoked);
-        assertEquals(1, subscriptionListener.assignedCount);
-        assertEquals(Collections.singleton(tp), subscriptionListener.assigned);
-    }
-
-    @Test
-    public void testReJoinGroup() {
-        subscriptions.subscribe(Arrays.asList(topicName), subscriptionListener);
-        subscriptions.needReassignment();
-
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // disconnected from original coordinator will cause re-discover and join again
-        client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code()), true);
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code()));
-        coordinator.ensurePartitionAssignment();
-        assertFalse(subscriptions.partitionAssignmentNeeded());
-        assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions());
-        assertEquals(1, subscriptionListener.revokedCount);
-        assertEquals(Collections.emptySet(), subscriptionListener.revoked);
-        assertEquals(1, subscriptionListener.assignedCount);
-        assertEquals(Collections.singleton(tp), subscriptionListener.assigned);
-    }
-
-    @Test(expected = ApiException.class)
-    public void testUnknownPartitionAssignmentStrategy() {
-        subscriptions.subscribe(Arrays.asList(topicName), subscriptionListener);
-        subscriptions.needReassignment();
-
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // coordinator doesn't like our assignment strategy
-        client.prepareResponse(joinGroupResponse(0, "consumer", Collections.<TopicPartition>emptyList(), Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code()));
-        coordinator.ensurePartitionAssignment();
-    }
-
-    @Test(expected = ApiException.class)
-    public void testInvalidSessionTimeout() {
-        subscriptions.subscribe(Arrays.asList(topicName), subscriptionListener);
-        subscriptions.needReassignment();
-
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // coordinator doesn't like our assignment strategy
-        client.prepareResponse(joinGroupResponse(0, "consumer", Collections.<TopicPartition>emptyList(), Errors.INVALID_SESSION_TIMEOUT.code()));
-        coordinator.ensurePartitionAssignment();
-    }
-
-    @Test
-    public void testCommitOffsetOnly() {
-        subscriptions.assign(Arrays.asList(tp));
-
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
-
-        AtomicBoolean success = new AtomicBoolean(false);
-        coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), callback(success));
-        consumerClient.poll(0);
-        assertTrue(success.get());
-
-        assertEquals(100L, subscriptions.committed(tp).offset());
-    }
-
-    @Test
-    public void testCommitOffsetMetadata() {
-        subscriptions.assign(Arrays.asList(tp));
-
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
-
-        AtomicBoolean success = new AtomicBoolean(false);
-        coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "hello")), callback(success));
-        consumerClient.poll(0);
-        assertTrue(success.get());
-
-        assertEquals(100L, subscriptions.committed(tp).offset());
-        assertEquals("hello", subscriptions.committed(tp).metadata());
-    }
-
-    @Test
-    public void testCommitOffsetAsyncWithDefaultCallback() {
-        int invokedBeforeTest = defaultOffsetCommitCallback.invoked;
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
-        coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), null);
-        consumerClient.poll(0);
-        assertEquals(invokedBeforeTest + 1, defaultOffsetCommitCallback.invoked);
-        assertNull(defaultOffsetCommitCallback.exception);
-    }
-
-    @Test
-    public void testResetGeneration() {
-        // enable auto-assignment
-        subscriptions.subscribe(Arrays.asList(topicName), subscriptionListener);
-
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        client.prepareResponse(joinGroupResponse(1, "consumer", Collections.singletonList(tp), Errors.NONE.code()));
-        coordinator.ensurePartitionAssignment();
-
-        // now switch to manual assignment
-        subscriptions.unsubscribe();
-        coordinator.resetGeneration();
-        subscriptions.assign(Arrays.asList(tp));
-
-        // the client should not reuse generation/consumerId from auto-subscribed generation
-        client.prepareResponse(new MockClient.RequestMatcher() {
-            @Override
-            public boolean matches(ClientRequest request) {
-                OffsetCommitRequest commitRequest = new OffsetCommitRequest(request.request().body());
-                return commitRequest.consumerId().equals(OffsetCommitRequest.DEFAULT_CONSUMER_ID) &&
-                        commitRequest.generationId() == OffsetCommitRequest.DEFAULT_GENERATION_ID;
-            }
-        }, offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
-
-        AtomicBoolean success = new AtomicBoolean(false);
-        coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), callback(success));
-        consumerClient.poll(0);
-        assertTrue(success.get());
-    }
-
-    @Test
-    public void testCommitOffsetAsyncFailedWithDefaultCallback() {
-        int invokedBeforeTest = defaultOffsetCommitCallback.invoked;
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code())));
-        coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), null);
-        consumerClient.poll(0);
-        assertEquals(invokedBeforeTest + 1, defaultOffsetCommitCallback.invoked);
-        assertEquals(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception(), defaultOffsetCommitCallback.exception);
-    }
-
-    @Test
-    public void testCommitOffsetAsyncCoordinatorNotAvailable() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // async commit with coordinator not available
-        MockCommitCallback cb = new MockCommitCallback();
-        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code())));
-        coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb);
-        consumerClient.poll(0);
-
-        assertTrue(coordinator.coordinatorUnknown());
-        assertEquals(1, cb.invoked);
-        assertEquals(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception(), cb.exception);
-    }
-
-    @Test
-    public void testCommitOffsetAsyncNotCoordinator() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // async commit with not coordinator
-        MockCommitCallback cb = new MockCommitCallback();
-        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code())));
-        coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb);
-        consumerClient.poll(0);
-
-        assertTrue(coordinator.coordinatorUnknown());
-        assertEquals(1, cb.invoked);
-        assertEquals(Errors.NOT_COORDINATOR_FOR_CONSUMER.exception(), cb.exception);
-    }
-
-    @Test
-    public void testCommitOffsetAsyncDisconnected() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // async commit with coordinator disconnected
-        MockCommitCallback cb = new MockCommitCallback();
-        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true);
-        coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb);
-        consumerClient.poll(0);
-
-        assertTrue(coordinator.coordinatorUnknown());
-        assertEquals(1, cb.invoked);
-        assertTrue(cb.exception instanceof DisconnectException);
-    }
-
-    @Test
-    public void testCommitOffsetSyncNotCoordinator() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request)
-        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code())));
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
-        coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)));
-    }
-
-    @Test
-    public void testCommitOffsetSyncCoordinatorNotAvailable() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request)
-        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code())));
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
-        coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)));
-    }
-
-    @Test
-    public void testCommitOffsetSyncCoordinatorDisconnected() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request)
-        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true);
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())));
-        coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)));
-    }
-
-    @Test(expected = OffsetMetadataTooLarge.class)
-    public void testCommitOffsetMetadataTooLarge() {
-        // since offset metadata is provided by the user, we have to propagate the exception so they can handle it
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.OFFSET_METADATA_TOO_LARGE.code())));
-        coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "metadata")));
-    }
-
-    @Test(expected = ApiException.class)
-    public void testCommitOffsetSyncCallbackWithNonRetriableException() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        // sync commit with invalid partitions should throw if we have no callback
-        client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.COMMITTING_PARTITIONS_NOT_ASSIGNED.code())), false);
-        coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)));
-    }
-
-    @Test
-    public void testRefreshOffset() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        subscriptions.assign(Arrays.asList(tp));
-        subscriptions.needRefreshCommits();
-        client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L));
-        coordinator.refreshCommittedOffsetsIfNeeded();
-        assertFalse(subscriptions.refreshCommitsNeeded());
-        assertEquals(100L, subscriptions.committed(tp).offset());
-    }
-
-    @Test
-    public void testRefreshOffsetLoadInProgress() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        subscriptions.assign(Arrays.asList(tp));
-        subscriptions.needRefreshCommits();
-        client.prepareResponse(offsetFetchResponse(tp, Errors.OFFSET_LOAD_IN_PROGRESS.code(), "", 100L));
-        client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L));
-        coordinator.refreshCommittedOffsetsIfNeeded();
-        assertFalse(subscriptions.refreshCommitsNeeded());
-        assertEquals(100L, subscriptions.committed(tp).offset());
-    }
-
-    @Test
-    public void testRefreshOffsetNotCoordinatorForConsumer() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        subscriptions.assign(Arrays.asList(tp));
-        subscriptions.needRefreshCommits();
-        client.prepareResponse(offsetFetchResponse(tp, Errors.NOT_COORDINATOR_FOR_CONSUMER.code(), "", 100L));
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", 100L));
-        coordinator.refreshCommittedOffsetsIfNeeded();
-        assertFalse(subscriptions.refreshCommitsNeeded());
-        assertEquals(100L, subscriptions.committed(tp).offset());
-    }
-
-    @Test
-    public void testRefreshOffsetWithNoFetchableOffsets() {
-        client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code()));
-        coordinator.ensureCoordinatorKnown();
-
-        subscriptions.assign(Arrays.asList(tp));
-        subscriptions.needRefreshCommits();
-        client.prepareResponse(offsetFetchResponse(tp, Errors.NONE.code(), "", -1L));
-        coordinator.refreshCommittedOffsetsIfNeeded();
-        assertFalse(subscriptions.refreshCommitsNeeded());
-        assertEquals(null, subscriptions.committed(tp));
-    }
-
-    private Struct consumerMetadataResponse(Node node, short error) {
-        ConsumerMetadataResponse response = new ConsumerMetadataResponse(error, node);
-        return response.toStruct();
-    }
-
-    private Struct heartbeatResponse(short error) {
-        HeartbeatResponse response = new HeartbeatResponse(error);
-        return response.toStruct();
-    }
-
-    private Struct joinGroupResponse(int generationId, String consumerId, List<TopicPartition> assignedPartitions, short error) {
-        JoinGroupResponse response = new JoinGroupResponse(error, generationId, consumerId, assignedPartitions);
-        return response.toStruct();
-    }
-
-    private Struct offsetCommitResponse(Map<TopicPartition, Short> responseData) {
-        OffsetCommitResponse response = new OffsetCommitResponse(responseData);
-        return response.toStruct();
-    }
-
-    private Struct offsetFetchResponse(TopicPartition tp, Short error, String metadata, long offset) {
-        OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, metadata, error);
-        OffsetFetchResponse response = new OffsetFetchResponse(Collections.singletonMap(tp, data));
-        return response.toStruct();
-    }
-
-    private OffsetCommitCallback callback(final AtomicBoolean success) {
-        return new OffsetCommitCallback() {
-            @Override
-            public void onComplete(Map<TopicPartition, OffsetAndMetadata> offsets, Exception exception) {
-                if (exception == null)
-                    success.set(true);
-            }
-        };
-    }
-
-    private static class MockCommitCallback implements OffsetCommitCallback {
-        public int invoked = 0;
-        public Exception exception = null;
-
-        @Override
-        public void onComplete(Map<TopicPartition, OffsetAndMetadata> offsets, Exception exception) {
-            invoked++;
-            this.exception = exception;
-        }
-    }
-
-    private static class MockRebalanceListener implements ConsumerRebalanceListener {
-        public Collection<TopicPartition> revoked;
-        public Collection<TopicPartition> assigned;
-        public int revokedCount = 0;
-        public int assignedCount = 0;
-
-
-        @Override
-        public void onPartitionsAssigned(Collection<TopicPartition> partitions) {
-            this.assigned = partitions;
-            assignedCount++;
-        }
-
-        @Override
-        public void onPartitionsRevoked(Collection<TopicPartition> partitions) {
-            this.revoked = partitions;
-            revokedCount++;
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
index 4929449..8773f8c 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
@@ -350,11 +350,11 @@ public class FetcherTest {
 
     @Test
     public void testGetAllTopics() throws InterruptedException {
-        // sending response before request, as getAllTopics is a blocking call
+        // sending response before request, as getTopicMetadata is a blocking call
         client.prepareResponse(
             new MetadataResponse(cluster, Collections.<String, Errors>emptyMap()).toStruct());
 
-        Map<String, List<PartitionInfo>> allTopics = fetcher.getAllTopics(5000L);
+        Map<String, List<PartitionInfo>> allTopics = fetcher.getAllTopicMetadata(5000L);
 
         assertEquals(cluster.topics().size(), allTopics.size());
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MockPartitionAssignor.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MockPartitionAssignor.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MockPartitionAssignor.java
new file mode 100644
index 0000000..40ae661
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/MockPartitionAssignor.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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 org.apache.kafka.common.TopicPartition;
+
+import java.util.List;
+import java.util.Map;
+
+public class MockPartitionAssignor extends AbstractPartitionAssignor {
+
+    private Map<String, List<TopicPartition>> result = null;
+
+    @Override
+    public Map<String, List<TopicPartition>> assign(Map<String, Integer> partitionsPerTopic,
+                                                    Map<String, List<String>> subscriptions) {
+        if (result == null)
+            throw new IllegalStateException("Call to assign with no result prepared");
+        return result;
+    }
+
+    @Override
+    public String name() {
+        return "consumer-mock-assignor";
+    }
+
+    public void clear() {
+        this.result = null;
+    }
+
+    public void prepare(Map<String, List<TopicPartition>> result) {
+        this.result = result;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
index cabf591..fb21802 100644
--- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java
@@ -29,9 +29,9 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
-import java.util.HashSet;
 import java.util.Set;
 
 import static org.junit.Assert.assertEquals;
@@ -151,11 +151,11 @@ public class RequestResponseTest {
     }
 
     private AbstractRequest createConsumerMetadataRequest() {
-        return new ConsumerMetadataRequest("test-group");
+        return new GroupMetadataRequest("test-group");
     }
 
     private AbstractRequestResponse createConsumerMetadataResponse() {
-        return new ConsumerMetadataResponse((short) 1, new Node(10, "host1", 2014));
+        return new GroupMetadataResponse(Errors.NONE.code(), new Node(10, "host1", 2014));
     }
 
     private AbstractRequest createFetchRequest() {
@@ -180,11 +180,17 @@ public class RequestResponseTest {
     }
 
     private AbstractRequest createJoinGroupRequest() {
-        return new JoinGroupRequest("group1", 30000, Arrays.asList("topic1"), "consumer1", "strategy1");
+        ByteBuffer metadata = ByteBuffer.wrap(new byte[] {});
+        List<JoinGroupRequest.GroupProtocol> protocols = new ArrayList<>();
+        protocols.add(new JoinGroupRequest.GroupProtocol("consumer-range", metadata));
+        return new JoinGroupRequest("group1", 30000, "consumer1", "consumer", protocols);
     }
 
     private AbstractRequestResponse createJoinGroupResponse() {
-        return new JoinGroupResponse(Errors.NONE.code(), 1, "consumer1", Arrays.asList(new TopicPartition("test11", 1), new TopicPartition("test2", 1)));
+        Map<String, ByteBuffer> members = new HashMap<>();
+        members.put("consumer1", ByteBuffer.wrap(new byte[]{}));
+        members.put("consumer2", ByteBuffer.wrap(new byte[]{}));
+        return new JoinGroupResponse(Errors.NONE.code(), 1, "range", "consumer1", "leader", members);
     }
 
     private AbstractRequest createLeaveGroupRequest() {

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/KafkaBasedLogTest.java
----------------------------------------------------------------------
diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/KafkaBasedLogTest.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/KafkaBasedLogTest.java
index 1ff5e73..eb62c9e 100644
--- a/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/KafkaBasedLogTest.java
+++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/KafkaBasedLogTest.java
@@ -340,7 +340,7 @@ public class KafkaBasedLogTest {
                 consumer.schedulePollTask(new Runnable() {
                     @Override
                     public void run() {
-                        consumer.setException(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception());
+                        consumer.setException(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.exception());
                     }
                 });
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/admin/AclCommand.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/AclCommand.scala b/core/src/main/scala/kafka/admin/AclCommand.scala
index 6a8a8a2..fd6d420 100644
--- a/core/src/main/scala/kafka/admin/AclCommand.scala
+++ b/core/src/main/scala/kafka/admin/AclCommand.scala
@@ -31,7 +31,7 @@ object AclCommand {
   val Newline = scala.util.Properties.lineSeparator
   val ResourceTypeToValidOperations = Map[ResourceType, Set[Operation]] (
     Topic -> Set(Read, Write, Describe),
-    ConsumerGroup -> Set(Read),
+    Group -> Set(Read),
     Cluster -> Set(Create, ClusterAction)
   )
 
@@ -146,14 +146,14 @@ object AclCommand {
     val resources = getResource(opts)
 
     val topics: Set[Resource] = getResource(opts).filter(_.resourceType == Topic)
-    val consumerGroups: Set[Resource] = resources.filter(_.resourceType == ConsumerGroup)
+    val groups: Set[Resource] = resources.filter(_.resourceType == Group)
 
     //Read,Describe on topic, Read on consumerGroup + Create on cluster
 
     val acls = getAcl(opts, Set(Read, Describe))
 
     topics.map(_ -> acls).toMap[Resource, Set[Acl]] ++
-      consumerGroups.map(_ -> getAcl(opts, Set(Read))).toMap[Resource, Set[Acl]]
+      groups.map(_ -> getAcl(opts, Set(Read))).toMap[Resource, Set[Acl]]
   }
 
   private def getCliResourceToAcls(opts: AclCommandOptions): Map[Resource, Set[Acl]] = {
@@ -221,10 +221,10 @@ object AclCommand {
       resources += Resource.ClusterResource
 
     if (opts.options.has(opts.groupOpt))
-      opts.options.valuesOf(opts.groupOpt).asScala.foreach(consumerGroup => resources += new Resource(ConsumerGroup, consumerGroup.trim))
+      opts.options.valuesOf(opts.groupOpt).asScala.foreach(group => resources += new Resource(Group, group.trim))
 
     if (resources.isEmpty && dieIfNoResourceFound)
-      CommandLineUtils.printUsageAndDie(opts.parser, "You must provide at least one resource: --topic <topic> or --cluster or --consumer-group <group>")
+      CommandLineUtils.printUsageAndDie(opts.parser, "You must provide at least one resource: --topic <topic> or --cluster or --group <group>")
 
     resources
   }
@@ -266,16 +266,16 @@ object AclCommand {
       .withValuesSeparatedBy(Delimiter)
 
     val clusterOpt = parser.accepts("cluster", "Add/Remove cluster acls.")
-    val groupOpt = parser.accepts("consumer-group", "Comma separated list of consumer groups to which the acls should be added or removed. " +
-      "A value of * indicates the acls should apply to all consumer-groups.")
+    val groupOpt = parser.accepts("group", "Comma separated list of groups to which the acls should be added or removed. " +
+      "A value of * indicates the acls should apply to all groups.")
       .withRequiredArg
-      .describedAs("consumer-group")
+      .describedAs("group")
       .ofType(classOf[String])
       .withValuesSeparatedBy(Delimiter)
 
     val addOpt = parser.accepts("add", "Indicates you are trying to add acls.")
     val removeOpt = parser.accepts("remove", "Indicates you are trying to remove acls.")
-    val listOpt = parser.accepts("list", "List acls for the specified resource, use --topic <topic> or --consumer-group <group> or --cluster to specify a resource.")
+    val listOpt = parser.accepts("list", "List acls for the specified resource, use --topic <topic> or --group <group> or --cluster to specify a resource.")
 
     val operationsOpt = parser.accepts("operations", "Comma separated list of operations, default is All. Valid operation names are: " + Newline +
       Operation.values.map("\t" + _).mkString(Newline) + Newline)
@@ -320,7 +320,7 @@ object AclCommand {
       "This will generate acls that allows WRITE,DESCRIBE on topic and CREATE on cluster. ")
 
     val consumerOpt = parser.accepts("consumer", "Convenience option to add/remove acls for consumer role. " +
-      "This will generate acls that allows READ,DESCRIBE on topic and READ on consumer-group.")
+      "This will generate acls that allows READ,DESCRIBE on topic and READ on group.")
 
     val helpOpt = parser.accepts("help", "Print usage information.")
 
@@ -343,7 +343,7 @@ object AclCommand {
         CommandLineUtils.printUsageAndDie(parser, "With --producer you must specify a --topic")
 
       if (options.has(consumerOpt) && (!options.has(topicOpt) || !options.has(groupOpt) || (!options.has(producerOpt) && options.has(clusterOpt))))
-        CommandLineUtils.printUsageAndDie(parser, "With --consumer you must specify a --topic and a --consumer-group and no --cluster option should be specified.")
+        CommandLineUtils.printUsageAndDie(parser, "With --consumer you must specify a --topic and a --group and no --cluster option should be specified.")
     }
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/core/src/main/scala/kafka/admin/TopicCommand.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala
index e6ca112..ed54aee 100755
--- a/core/src/main/scala/kafka/admin/TopicCommand.scala
+++ b/core/src/main/scala/kafka/admin/TopicCommand.scala
@@ -23,16 +23,15 @@ import kafka.common.{Topic, AdminCommandFailedException}
 import kafka.utils.CommandLineUtils
 import kafka.utils._
 import kafka.utils.ZkUtils._
-import org.I0Itec.zkclient.ZkClient
 import org.I0Itec.zkclient.exception.ZkNodeExistsException
 import scala.collection._
 import scala.collection.JavaConversions._
 import kafka.log.{Defaults, LogConfig}
 import kafka.consumer.{ConsumerConfig, Whitelist}
-import kafka.server.{ConfigType, OffsetManager}
+import kafka.server.ConfigType
 import org.apache.kafka.common.utils.Utils
 import org.apache.kafka.common.security.JaasUtils
-import kafka.coordinator.ConsumerCoordinator
+import kafka.coordinator.GroupCoordinator
 
 
 object TopicCommand extends Logging {
@@ -130,7 +129,7 @@ object TopicCommand extends Logging {
       }
 
       if(opts.options.has(opts.partitionsOpt)) {
-        if (topic == ConsumerCoordinator.OffsetsTopicName) {
+        if (topic == GroupCoordinator.OffsetsTopicName) {
           throw new IllegalArgumentException("The number of partitions for the offsets topic cannot be changed.")
         }
         println("WARNING: If partitions are increased for a topic that has a key, the partition " +

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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
deleted file mode 100644
index 258d5fe..0000000
--- a/core/src/main/scala/kafka/api/ConsumerMetadataRequest.scala
+++ /dev/null
@@ -1,80 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package kafka.api
-
-import java.nio.ByteBuffer
-
-import kafka.common.ErrorMapping
-import kafka.network.{RequestOrResponseSend, RequestChannel}
-import kafka.network.RequestChannel.Response
-
-object ConsumerMetadataRequest {
-  val CurrentVersion = 0.shortValue
-  val DefaultClientId = ""
-
-  def readFrom(buffer: ByteBuffer) = {
-    // envelope
-    val versionId = buffer.getShort
-    val correlationId = buffer.getInt
-    val clientId = ApiUtils.readShortString(buffer)
-
-    // request
-    val group = ApiUtils.readShortString(buffer)
-    ConsumerMetadataRequest(group, versionId, correlationId, clientId)
-  }
-
-}
-
-case class ConsumerMetadataRequest(group: String,
-                                   versionId: Short = ConsumerMetadataRequest.CurrentVersion,
-                                   correlationId: Int = 0,
-                                   clientId: String = ConsumerMetadataRequest.DefaultClientId)
-  extends RequestOrResponse(Some(RequestKeys.ConsumerMetadataKey)) {
-
-  def sizeInBytes =
-    2 + /* versionId */
-    4 + /* correlationId */
-    ApiUtils.shortStringLength(clientId) +
-    ApiUtils.shortStringLength(group)
-
-  def writeTo(buffer: ByteBuffer) {
-    // envelope
-    buffer.putShort(versionId)
-    buffer.putInt(correlationId)
-    ApiUtils.writeShortString(buffer, clientId)
-
-    // consumer metadata request
-    ApiUtils.writeShortString(buffer, group)
-  }
-
-  override def handleError(e: Throwable, requestChannel: RequestChannel, request: RequestChannel.Request): Unit = {
-    // return ConsumerCoordinatorNotAvailable for all uncaught errors
-    val errorResponse = ConsumerMetadataResponse(None, ErrorMapping.ConsumerCoordinatorNotAvailableCode, correlationId)
-    requestChannel.sendResponse(new Response(request, new RequestOrResponseSend(request.connectionId, errorResponse)))
-  }
-
-  def describe(details: Boolean) = {
-    val consumerMetadataRequest = new StringBuilder
-    consumerMetadataRequest.append("Name: " + this.getClass.getSimpleName)
-    consumerMetadataRequest.append("; Version: " + versionId)
-    consumerMetadataRequest.append("; CorrelationId: " + correlationId)
-    consumerMetadataRequest.append("; ClientId: " + clientId)
-    consumerMetadataRequest.append("; Group: " + group)
-    consumerMetadataRequest.toString()
-  }
-}
\ No newline at end of file


[7/8] kafka git commit: KAFKA-2464: client-side assignment for new consumer

Posted by gu...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java
new file mode 100644
index 0000000..0020993
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java
@@ -0,0 +1,162 @@
+/**
+ * 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 org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * ConsumerProtocol contains the schemas for consumer subscriptions and assignments for use with
+ * Kafka's generalized group management protocol. Below is the version 0 format:
+ *
+ * <pre>
+ * Subscription => Version Topics
+ *   Version    => Int16
+ *   Topics     => [String]
+ *   UserData   => Bytes
+ *
+ * Assignment => Version TopicPartitions
+ *   Version         => int16
+ *   TopicPartitions => [Topic Partitions]
+ *     Topic         => String
+ *     Partitions    => [int32]
+ * </pre>
+ *
+ * The current implementation assumes that future versions will not break compatibility. When
+ * it encounters a newer version, it parses it using the current format. This basically means
+ * that new versions cannot remove or reorder any of the existing fields.
+ */
+public class ConsumerProtocol {
+
+    public static final String VERSION_KEY_NAME = "version";
+    public static final String TOPICS_KEY_NAME = "topics";
+    public static final String TOPIC_KEY_NAME = "topic";
+    public static final String PARTITIONS_KEY_NAME = "partitions";
+    public static final String TOPIC_PARTITIONS_KEY_NAME = "topic_partitions";
+    public static final String USER_DATA_KEY_NAME = "user_data";
+
+    public static final short CONSUMER_PROTOCOL_V0 = 0;
+    public static final Schema CONSUMER_PROTOCOL_HEADER_SCHEMA = new Schema(
+            new Field(VERSION_KEY_NAME, Type.INT16));
+    private static final Struct CONSUMER_PROTOCOL_HEADER_V0 = new Struct(CONSUMER_PROTOCOL_HEADER_SCHEMA)
+            .set(VERSION_KEY_NAME, CONSUMER_PROTOCOL_V0);
+
+    public static final Schema SUBSCRIPTION_V0 = new Schema(
+            new Field(TOPICS_KEY_NAME, new ArrayOf(Type.STRING)),
+            new Field(USER_DATA_KEY_NAME, Type.BYTES));
+    public static final Schema TOPIC_ASSIGNMENT_V0 = new Schema(
+            new Field(TOPIC_KEY_NAME, Type.STRING),
+            new Field(PARTITIONS_KEY_NAME, new ArrayOf(Type.INT32)));
+    public static final Schema ASSIGNMENT_V0 = new Schema(
+            new Field(TOPIC_PARTITIONS_KEY_NAME, new ArrayOf(TOPIC_ASSIGNMENT_V0)),
+            new Field(USER_DATA_KEY_NAME, Type.BYTES));
+
+    public static ByteBuffer serializeSubscription(PartitionAssignor.Subscription subscription) {
+        Struct struct = new Struct(SUBSCRIPTION_V0);
+        struct.set(USER_DATA_KEY_NAME, subscription.userData());
+        struct.set(TOPICS_KEY_NAME, subscription.topics().toArray());
+        ByteBuffer buffer = ByteBuffer.allocate(CONSUMER_PROTOCOL_HEADER_V0.sizeOf() + SUBSCRIPTION_V0.sizeOf(struct));
+        CONSUMER_PROTOCOL_HEADER_V0.writeTo(buffer);
+        SUBSCRIPTION_V0.write(buffer, struct);
+        buffer.flip();
+        return buffer;
+    }
+
+    public static PartitionAssignor.Subscription deserializeSubscription(ByteBuffer buffer) {
+        Struct header = (Struct) CONSUMER_PROTOCOL_HEADER_SCHEMA.read(buffer);
+        Short version = header.getShort(VERSION_KEY_NAME);
+        checkVersionCompatibility(version);
+        Struct struct = (Struct) SUBSCRIPTION_V0.read(buffer);
+        ByteBuffer userData = struct.getBytes(USER_DATA_KEY_NAME);
+        List<String> topics = new ArrayList<>();
+        for (Object topicObj : struct.getArray(TOPICS_KEY_NAME))
+            topics.add((String) topicObj);
+        return new PartitionAssignor.Subscription(topics, userData);
+    }
+
+    public static PartitionAssignor.Assignment deserializeAssignment(ByteBuffer buffer) {
+        Struct header = (Struct) CONSUMER_PROTOCOL_HEADER_SCHEMA.read(buffer);
+        Short version = header.getShort(VERSION_KEY_NAME);
+        checkVersionCompatibility(version);
+        Struct struct = (Struct) ASSIGNMENT_V0.read(buffer);
+        ByteBuffer userData = struct.getBytes(USER_DATA_KEY_NAME);
+        List<TopicPartition> partitions = new ArrayList<>();
+        for (Object structObj : struct.getArray(TOPIC_PARTITIONS_KEY_NAME)) {
+            Struct assignment = (Struct) structObj;
+            String topic = assignment.getString(TOPIC_KEY_NAME);
+            for (Object partitionObj : assignment.getArray(PARTITIONS_KEY_NAME)) {
+                Integer partition = (Integer) partitionObj;
+                partitions.add(new TopicPartition(topic, partition));
+            }
+        }
+        return new PartitionAssignor.Assignment(partitions, userData);
+    }
+
+    public static ByteBuffer serializeAssignment(PartitionAssignor.Assignment assignment) {
+        Struct struct = new Struct(ASSIGNMENT_V0);
+        struct.set(USER_DATA_KEY_NAME, assignment.userData());
+        List<Struct> topicAssignments = new ArrayList<>();
+        for (Map.Entry<String, List<Integer>> topicEntry : asMap(assignment.partitions()).entrySet()) {
+            Struct topicAssignment = new Struct(TOPIC_ASSIGNMENT_V0);
+            topicAssignment.set(TOPIC_KEY_NAME, topicEntry.getKey());
+            topicAssignment.set(PARTITIONS_KEY_NAME, topicEntry.getValue().toArray());
+            topicAssignments.add(topicAssignment);
+        }
+        struct.set(TOPIC_PARTITIONS_KEY_NAME, topicAssignments.toArray());
+        ByteBuffer buffer = ByteBuffer.allocate(CONSUMER_PROTOCOL_HEADER_V0.sizeOf() + ASSIGNMENT_V0.sizeOf(struct));
+        CONSUMER_PROTOCOL_HEADER_V0.writeTo(buffer);
+        ASSIGNMENT_V0.write(buffer, struct);
+        buffer.flip();
+        return buffer;
+    }
+
+    private static void checkVersionCompatibility(short version) {
+        // check for invalid versions
+        if (version < CONSUMER_PROTOCOL_V0)
+            throw new SchemaException("Unsupported subscription version: " + version);
+
+        // otherwise, assume versions can be parsed as V0
+    }
+
+
+    private static Map<String, List<Integer>> asMap(Collection<TopicPartition> partitions) {
+        Map<String, List<Integer>> partitionMap = new HashMap<>();
+        for (TopicPartition partition : partitions) {
+            String topic = partition.topic();
+            List<Integer> topicPartitions = partitionMap.get(topic);
+            if (topicPartitions == null) {
+                topicPartitions = new ArrayList<>();
+                partitionMap.put(topic, topicPartitions);
+            }
+            topicPartitions.add(partition.partition());
+        }
+        return partitionMap;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
deleted file mode 100644
index 98193e8..0000000
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Coordinator.java
+++ /dev/null
@@ -1,848 +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.internals;
-
-import org.apache.kafka.clients.ClientResponse;
-import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
-import org.apache.kafka.clients.consumer.ConsumerWakeupException;
-import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.clients.consumer.OffsetCommitCallback;
-import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.MetricName;
-import org.apache.kafka.common.Node;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.errors.DisconnectException;
-import org.apache.kafka.common.errors.UnknownConsumerIdException;
-import org.apache.kafka.common.metrics.Measurable;
-import org.apache.kafka.common.metrics.MetricConfig;
-import org.apache.kafka.common.metrics.Metrics;
-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.protocol.ApiKeys;
-import org.apache.kafka.common.protocol.Errors;
-import org.apache.kafka.common.requests.ConsumerMetadataRequest;
-import org.apache.kafka.common.requests.ConsumerMetadataResponse;
-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.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.utils.Time;
-import org.apache.kafka.common.utils.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Closeable;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
-
-/**
- * This class manages the coordination process with the consumer coordinator.
- */
-public final class Coordinator implements Closeable {
-
-    private static final Logger log = LoggerFactory.getLogger(Coordinator.class);
-
-    private final ConsumerNetworkClient client;
-    private final Time time;
-    private final String groupId;
-    private final Heartbeat heartbeat;
-    private final HeartbeatTask heartbeatTask;
-    private final int sessionTimeoutMs;
-    private final String assignmentStrategy;
-    private final SubscriptionState subscriptions;
-    private final CoordinatorMetrics sensors;
-    private final long requestTimeoutMs;
-    private final long retryBackoffMs;
-    private final OffsetCommitCallback defaultOffsetCommitCallback;
-    private final boolean autoCommitEnabled;
-
-    private Node consumerCoordinator;
-    private String consumerId;
-    private int generation;
-
-    /**
-     * Initialize the coordination manager.
-     */
-    public Coordinator(ConsumerNetworkClient client,
-                       String groupId,
-                       int sessionTimeoutMs,
-                       int heartbeatIntervalMs,
-                       String assignmentStrategy,
-                       SubscriptionState subscriptions,
-                       Metrics metrics,
-                       String metricGrpPrefix,
-                       Map<String, String> metricTags,
-                       Time time,
-                       long requestTimeoutMs,
-                       long retryBackoffMs,
-                       OffsetCommitCallback defaultOffsetCommitCallback,
-                       boolean autoCommitEnabled,
-                       long autoCommitIntervalMs) {
-        this.client = client;
-        this.time = time;
-        this.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID;
-        this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
-        this.groupId = groupId;
-        this.consumerCoordinator = null;
-        this.subscriptions = subscriptions;
-        this.sessionTimeoutMs = sessionTimeoutMs;
-        this.assignmentStrategy = assignmentStrategy;
-        this.heartbeat = new Heartbeat(this.sessionTimeoutMs, heartbeatIntervalMs, time.milliseconds());
-        this.heartbeatTask = new HeartbeatTask();
-        this.sensors = new CoordinatorMetrics(metrics, metricGrpPrefix, metricTags);
-        this.requestTimeoutMs = requestTimeoutMs;
-        this.retryBackoffMs = retryBackoffMs;
-        this.defaultOffsetCommitCallback = defaultOffsetCommitCallback;
-        this.autoCommitEnabled = autoCommitEnabled;
-
-        if (autoCommitEnabled)
-            scheduleAutoCommitTask(autoCommitIntervalMs);
-    }
-
-    /**
-     * Refresh the committed offsets for provided partitions.
-     */
-    public void refreshCommittedOffsetsIfNeeded() {
-        if (subscriptions.refreshCommitsNeeded()) {
-            Map<TopicPartition, OffsetAndMetadata> offsets = fetchCommittedOffsets(subscriptions.assignedPartitions());
-            for (Map.Entry<TopicPartition, OffsetAndMetadata> entry : offsets.entrySet()) {
-                TopicPartition tp = entry.getKey();
-                // verify assignment is still active
-                if (subscriptions.isAssigned(tp))
-                    this.subscriptions.committed(tp, entry.getValue());
-            }
-            this.subscriptions.commitsRefreshed();
-        }
-    }
-
-    /**
-     * Fetch the current committed offsets from the coordinator for a set of partitions.
-     * @param partitions The partitions to fetch offsets for
-     * @return A map from partition to the committed offset
-     */
-    public Map<TopicPartition, OffsetAndMetadata> fetchCommittedOffsets(Set<TopicPartition> partitions) {
-        while (true) {
-            ensureCoordinatorKnown();
-
-            // contact coordinator to fetch committed offsets
-            RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future = sendOffsetFetchRequest(partitions);
-            client.poll(future);
-
-            if (future.succeeded())
-                return future.value();
-
-            if (!future.isRetriable())
-                throw future.exception();
-
-            Utils.sleep(retryBackoffMs);
-        }
-    }
-
-    /**
-     * Ensure that we have a valid partition assignment from the coordinator.
-     */
-    public void ensurePartitionAssignment() {
-        if (!subscriptions.partitionAssignmentNeeded())
-            return;
-
-        // commit offsets prior to rebalance if auto-commit enabled
-        maybeAutoCommitOffsetsSync();
-
-        ConsumerRebalanceListener listener = subscriptions.listener();
-
-        // execute the user's listener before rebalance
-        log.debug("Revoking previously assigned partitions {}", this.subscriptions.assignedPartitions());
-        try {
-            Set<TopicPartition> revoked = new HashSet<>(subscriptions.assignedPartitions());
-            listener.onPartitionsRevoked(revoked);
-        } catch (Exception e) {
-            log.error("User provided listener " + listener.getClass().getName()
-                    + " failed on partition revocation: ", e);
-        }
-
-        reassignPartitions();
-
-        // execute the user's listener after rebalance
-        log.debug("Setting newly assigned partitions {}", this.subscriptions.assignedPartitions());
-        try {
-            Set<TopicPartition> assigned = new HashSet<>(subscriptions.assignedPartitions());
-            listener.onPartitionsAssigned(assigned);
-        } catch (Exception e) {
-            log.error("User provided listener " + listener.getClass().getName()
-                    + " failed on partition assignment: ", e);
-        }
-    }
-
-    private void reassignPartitions() {
-        while (subscriptions.partitionAssignmentNeeded()) {
-            ensureCoordinatorKnown();
-
-            // ensure that there are no pending requests to the coordinator. This is important
-            // in particular to avoid resending a pending JoinGroup request.
-            if (client.pendingRequestCount(this.consumerCoordinator) > 0) {
-                client.awaitPendingRequests(this.consumerCoordinator);
-                continue;
-            }
-
-            RequestFuture<Void> future = sendJoinGroupRequest();
-            client.poll(future);
-
-            if (future.failed()) {
-                if (future.exception() instanceof UnknownConsumerIdException)
-                    continue;
-                else if (!future.isRetriable())
-                    throw future.exception();
-                Utils.sleep(retryBackoffMs);
-            }
-        }
-    }
-
-    /**
-     * Block until the coordinator for this group is known.
-     */
-    public void ensureCoordinatorKnown() {
-        while (coordinatorUnknown()) {
-            RequestFuture<Void> future = sendConsumerMetadataRequest();
-            client.poll(future, requestTimeoutMs);
-
-            if (future.failed())
-                client.awaitMetadataUpdate();
-        }
-    }
-
-
-    @Override
-    public void close() {
-        // commit offsets prior to closing if auto-commit enabled
-        while (true) {
-            try {
-                maybeAutoCommitOffsetsSync();
-                return;
-            } catch (ConsumerWakeupException e) {
-                // ignore wakeups while closing to ensure we have a chance to commit
-                continue;
-            }
-        }
-    }
-
-    private class HeartbeatTask implements DelayedTask {
-
-        public void reset() {
-            // start or restart the heartbeat task to be executed at the next chance
-            long now = time.milliseconds();
-            heartbeat.resetSessionTimeout(now);
-            client.unschedule(this);
-            client.schedule(this, now);
-        }
-
-        @Override
-        public void run(final long now) {
-            if (!subscriptions.partitionsAutoAssigned() ||
-                    subscriptions.partitionAssignmentNeeded() ||
-                    coordinatorUnknown())
-                // no need to send if we're not using auto-assignment or if we are
-                // awaiting a rebalance
-                return;
-
-            if (heartbeat.sessionTimeoutExpired(now)) {
-                // we haven't received a successful heartbeat in one session interval
-                // so mark the coordinator dead
-                coordinatorDead();
-                return;
-            }
-
-            if (!heartbeat.shouldHeartbeat(now)) {
-                // we don't need to heartbeat now, so reschedule for when we do
-                client.schedule(this, now + heartbeat.timeToNextHeartbeat(now));
-            } else {
-                heartbeat.sentHeartbeat(now);
-                RequestFuture<Void> future = sendHeartbeatRequest();
-                future.addListener(new RequestFutureListener<Void>() {
-                    @Override
-                    public void onSuccess(Void value) {
-                        long now = time.milliseconds();
-                        heartbeat.receiveHeartbeat(now);
-                        long nextHeartbeatTime = now + heartbeat.timeToNextHeartbeat(now);
-                        client.schedule(HeartbeatTask.this, nextHeartbeatTime);
-                    }
-
-                    @Override
-                    public void onFailure(RuntimeException e) {
-                        client.schedule(HeartbeatTask.this, time.milliseconds() + retryBackoffMs);
-                    }
-                });
-            }
-        }
-    }
-
-    /**
-     * Send a request to get a new partition assignment. This is a non-blocking call which sends
-     * a JoinGroup request to the coordinator (if it is available). The returned future must
-     * be polled to see if the request completed successfully.
-     * @return A request future whose completion indicates the result of the JoinGroup request.
-     */
-    private RequestFuture<Void> sendJoinGroupRequest() {
-        if (coordinatorUnknown())
-            return RequestFuture.coordinatorNotAvailable();
-
-        // send a join group request to the coordinator
-        List<String> subscribedTopics = new ArrayList<String>(subscriptions.subscription());
-        log.debug("(Re-)joining group {} with subscribed topics {}", groupId, subscribedTopics);
-
-        JoinGroupRequest request = new JoinGroupRequest(groupId,
-                this.sessionTimeoutMs,
-                subscribedTopics,
-                this.consumerId,
-                this.assignmentStrategy);
-
-        // create the request for the coordinator
-        log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.consumerCoordinator.id());
-        return client.send(consumerCoordinator, ApiKeys.JOIN_GROUP, request)
-                .compose(new JoinGroupResponseHandler());
-    }
-
-    private class JoinGroupResponseHandler extends CoordinatorResponseHandler<JoinGroupResponse, Void> {
-
-        @Override
-        public JoinGroupResponse parse(ClientResponse response) {
-            return new JoinGroupResponse(response.responseBody());
-        }
-
-        @Override
-        public void handle(JoinGroupResponse joinResponse, RequestFuture<Void> future) {
-            // process the response
-            short errorCode = joinResponse.errorCode();
-
-            if (errorCode == Errors.NONE.code()) {
-                Coordinator.this.consumerId = joinResponse.consumerId();
-                Coordinator.this.generation = joinResponse.generationId();
-
-                // set the flag to refresh last committed offsets
-                subscriptions.needRefreshCommits();
-
-                log.debug("Joined group: {}", joinResponse.toStruct());
-
-                // record re-assignment time
-                sensors.partitionReassignments.record(response.requestLatencyMs());
-
-                // update partition assignment
-                subscriptions.changePartitionAssignment(joinResponse.assignedPartitions());
-                heartbeatTask.reset();
-                future.complete(null);
-            } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()) {
-                // reset the consumer id and retry immediately
-                Coordinator.this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
-                log.info("Attempt to join group {} failed due to unknown consumer id, resetting and retrying.",
-                        groupId);
-                future.raise(Errors.UNKNOWN_CONSUMER_ID);
-            } else if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
-                    || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
-                // re-discover the coordinator and retry with backoff
-                coordinatorDead();
-                log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.",
-                        groupId);
-                future.raise(Errors.forCode(errorCode));
-            } else if (errorCode == Errors.UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY.code()
-                    || errorCode == Errors.INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY.code()
-                    || errorCode == Errors.INVALID_SESSION_TIMEOUT.code()) {
-                // log the error and re-throw the exception
-                Errors error = Errors.forCode(errorCode);
-                log.error("Attempt to join group {} failed due to: {}",
-                        groupId, error.exception().getMessage());
-                future.raise(error);
-            } else {
-                // unexpected error, throw the exception
-                future.raise(new KafkaException("Unexpected error in join group response: "
-                        + Errors.forCode(joinResponse.errorCode()).exception().getMessage()));
-            }
-        }
-    }
-
-    public void commitOffsetsAsync(final Map<TopicPartition, OffsetAndMetadata> offsets, OffsetCommitCallback callback) {
-        this.subscriptions.needRefreshCommits();
-        RequestFuture<Void> future = sendOffsetCommitRequest(offsets);
-        final OffsetCommitCallback cb = callback == null ? defaultOffsetCommitCallback : callback;
-        future.addListener(new RequestFutureListener<Void>() {
-            @Override
-            public void onSuccess(Void value) {
-                cb.onComplete(offsets, null);
-            }
-
-            @Override
-            public void onFailure(RuntimeException e) {
-                cb.onComplete(offsets, e);
-            }
-        });
-    }
-
-    public void commitOffsetsSync(Map<TopicPartition, OffsetAndMetadata> offsets) {
-        if (offsets.isEmpty())
-            return;
-
-        while (true) {
-            ensureCoordinatorKnown();
-
-            RequestFuture<Void> future = sendOffsetCommitRequest(offsets);
-            client.poll(future);
-
-            if (future.succeeded()) {
-                return;
-            }
-
-            if (!future.isRetriable()) {
-                throw future.exception();
-            }
-
-            Utils.sleep(retryBackoffMs);
-        }
-    }
-
-    private void scheduleAutoCommitTask(final long interval) {
-        DelayedTask task = new DelayedTask() {
-            public void run(long now) {
-                commitOffsetsAsync(subscriptions.allConsumed(), new OffsetCommitCallback() {
-                    @Override
-                    public void onComplete(Map<TopicPartition, OffsetAndMetadata> offsets, Exception exception) {
-                        if (exception != null)
-                            log.error("Auto offset commit failed.", exception);
-                    }
-                });
-                client.schedule(this, now + interval);
-            }
-        };
-        client.schedule(task, time.milliseconds() + interval);
-    }
-
-    private void maybeAutoCommitOffsetsSync() {
-        if (autoCommitEnabled) {
-            try {
-                commitOffsetsSync(subscriptions.allConsumed());
-            } catch (ConsumerWakeupException e) {
-                // rethrow wakeups since they are triggered by the user
-                throw e;
-            } catch (Exception e) {
-                // consistent with async auto-commit failures, we do not propagate the exception
-                log.error("Auto offset commit failed.", e);
-            }
-        }
-    }
-
-    /**
-     * Reset the generation/consumerId tracked by this consumer.
-     */
-    public void resetGeneration() {
-        this.generation = OffsetCommitRequest.DEFAULT_GENERATION_ID;
-        this.consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
-    }
-
-    /**
-     * Commit offsets for the specified list of topics and partitions. This is a non-blocking call
-     * which returns a request future that can be polled in the case of a synchronous commit or ignored in the
-     * asynchronous case.
-     *
-     * @param offsets The list of offsets per partition that should be committed.
-     * @return A request future whose value indicates whether the commit was successful or not
-     */
-    private RequestFuture<Void> sendOffsetCommitRequest(final Map<TopicPartition, OffsetAndMetadata> offsets) {
-        if (coordinatorUnknown())
-            return RequestFuture.coordinatorNotAvailable();
-
-        if (offsets.isEmpty())
-            return RequestFuture.voidSuccess();
-
-        // create the offset commit request
-        Map<TopicPartition, OffsetCommitRequest.PartitionData> offsetData = new HashMap<>(offsets.size());
-        for (Map.Entry<TopicPartition, OffsetAndMetadata> entry : offsets.entrySet()) {
-            OffsetAndMetadata offsetAndMetadata = entry.getValue();
-            offsetData.put(entry.getKey(), new OffsetCommitRequest.PartitionData(
-                    offsetAndMetadata.offset(), offsetAndMetadata.metadata()));
-        }
-
-        OffsetCommitRequest req = new OffsetCommitRequest(this.groupId,
-                this.generation,
-                this.consumerId,
-                OffsetCommitRequest.DEFAULT_RETENTION_TIME,
-                offsetData);
-
-        return client.send(consumerCoordinator, ApiKeys.OFFSET_COMMIT, req)
-                .compose(new OffsetCommitResponseHandler(offsets));
-    }
-
-    public static class DefaultOffsetCommitCallback implements OffsetCommitCallback {
-        @Override
-        public void onComplete(Map<TopicPartition, OffsetAndMetadata> offsets, Exception exception) {
-            if (exception != null)
-                log.error("Offset commit failed.", exception);
-        }
-    }
-
-    private class OffsetCommitResponseHandler extends CoordinatorResponseHandler<OffsetCommitResponse, Void> {
-
-        private final Map<TopicPartition, OffsetAndMetadata> offsets;
-
-        public OffsetCommitResponseHandler(Map<TopicPartition, OffsetAndMetadata> offsets) {
-            this.offsets = offsets;
-        }
-
-        @Override
-        public OffsetCommitResponse parse(ClientResponse response) {
-            return new OffsetCommitResponse(response.responseBody());
-        }
-
-        @Override
-        public void handle(OffsetCommitResponse commitResponse, RequestFuture<Void> future) {
-            sensors.commitLatency.record(response.requestLatencyMs());
-            for (Map.Entry<TopicPartition, Short> entry : commitResponse.responseData().entrySet()) {
-                TopicPartition tp = entry.getKey();
-                OffsetAndMetadata offsetAndMetadata = this.offsets.get(tp);
-                long offset = offsetAndMetadata.offset();
-
-                short errorCode = entry.getValue();
-                if (errorCode == Errors.NONE.code()) {
-                    log.debug("Committed offset {} for partition {}", offset, tp);
-                    if (subscriptions.isAssigned(tp))
-                        // update the local cache only if the partition is still assigned
-                        subscriptions.committed(tp, offsetAndMetadata);
-                } else {
-                    if (errorCode == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
-                            || errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
-                        coordinatorDead();
-                    } else if (errorCode == Errors.UNKNOWN_CONSUMER_ID.code()
-                            || errorCode == Errors.ILLEGAL_GENERATION.code()) {
-                        // need to re-join group
-                        subscriptions.needReassignment();
-                    }
-
-                    log.error("Error committing partition {} at offset {}: {}",
-                            tp,
-                            offset,
-                            Errors.forCode(errorCode).exception().getMessage());
-
-                    future.raise(Errors.forCode(errorCode));
-                    return;
-                }
-            }
-
-            future.complete(null);
-        }
-    }
-
-    /**
-     * Fetch the committed offsets for a set of partitions. This is a non-blocking call. The
-     * returned future can be polled to get the actual offsets returned from the broker.
-     *
-     * @param partitions The set of partitions to get offsets for.
-     * @return A request future containing the committed offsets.
-     */
-    private RequestFuture<Map<TopicPartition, OffsetAndMetadata>> sendOffsetFetchRequest(Set<TopicPartition> partitions) {
-        if (coordinatorUnknown())
-            return RequestFuture.coordinatorNotAvailable();
-
-        log.debug("Fetching committed offsets for partitions: {}",  Utils.join(partitions, ", "));
-        // construct the request
-        OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList<TopicPartition>(partitions));
-
-        // send the request with a callback
-        return client.send(consumerCoordinator, ApiKeys.OFFSET_FETCH, request)
-                .compose(new OffsetFetchResponseHandler());
-    }
-
-    private class OffsetFetchResponseHandler extends CoordinatorResponseHandler<OffsetFetchResponse, Map<TopicPartition, OffsetAndMetadata>> {
-
-        @Override
-        public OffsetFetchResponse parse(ClientResponse response) {
-            return new OffsetFetchResponse(response.responseBody());
-        }
-
-        @Override
-        public void handle(OffsetFetchResponse response, RequestFuture<Map<TopicPartition, OffsetAndMetadata>> future) {
-            Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>(response.responseData().size());
-            for (Map.Entry<TopicPartition, OffsetFetchResponse.PartitionData> entry : response.responseData().entrySet()) {
-                TopicPartition tp = entry.getKey();
-                OffsetFetchResponse.PartitionData data = entry.getValue();
-                if (data.hasError()) {
-                    log.debug("Error fetching offset for topic-partition {}: {}", tp, Errors.forCode(data.errorCode)
-                            .exception()
-                            .getMessage());
-                    if (data.errorCode == Errors.OFFSET_LOAD_IN_PROGRESS.code()) {
-                        // just retry
-                        future.raise(Errors.OFFSET_LOAD_IN_PROGRESS);
-                    } else if (data.errorCode == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
-                        // re-discover the coordinator and retry
-                        coordinatorDead();
-                        future.raise(Errors.NOT_COORDINATOR_FOR_CONSUMER);
-                    } else if (data.errorCode == Errors.UNKNOWN_CONSUMER_ID.code()
-                            || data.errorCode == Errors.ILLEGAL_GENERATION.code()) {
-                        // need to re-join group
-                        subscriptions.needReassignment();
-                        future.raise(Errors.forCode(data.errorCode));
-                    } else {
-                        future.raise(new KafkaException("Unexpected error in fetch offset response: "
-                                + Errors.forCode(data.errorCode).exception().getMessage()));
-                    }
-                    return;
-                } else if (data.offset >= 0) {
-                    // record the position with the offset (-1 indicates no committed offset to fetch)
-                    offsets.put(tp, new OffsetAndMetadata(data.offset, data.metadata));
-                } else {
-                    log.debug("No committed offset for partition " + tp);
-                }
-            }
-
-            future.complete(offsets);
-        }
-    }
-
-    /**
-     * Send a heartbeat request now (visible only for testing).
-     */
-    public RequestFuture<Void> sendHeartbeatRequest() {
-        HeartbeatRequest req = new HeartbeatRequest(this.groupId, this.generation, this.consumerId);
-        return client.send(consumerCoordinator, ApiKeys.HEARTBEAT, req)
-                .compose(new HeartbeatCompletionHandler());
-    }
-
-    public boolean coordinatorUnknown() {
-        return this.consumerCoordinator == null;
-    }
-
-    /**
-     * Discover the current coordinator for the consumer group. Sends a ConsumerMetadata request to
-     * one of the brokers. The returned future should be polled to get the result of the request.
-     * @return A request future which indicates the completion of the metadata request
-     */
-    private RequestFuture<Void> sendConsumerMetadataRequest() {
-        // initiate the consumer metadata request
-        // find a node to ask about the coordinator
-        Node node = this.client.leastLoadedNode();
-        if (node == null) {
-            // TODO: If there are no brokers left, perhaps we should use the bootstrap set
-            // from configuration?
-            return RequestFuture.noBrokersAvailable();
-        } else {
-            // create a consumer metadata request
-            log.debug("Issuing consumer metadata request to broker {}", node.id());
-            ConsumerMetadataRequest metadataRequest = new ConsumerMetadataRequest(this.groupId);
-            return client.send(node, ApiKeys.CONSUMER_METADATA, metadataRequest)
-                    .compose(new RequestFutureAdapter<ClientResponse, Void>() {
-                        @Override
-                        public void onSuccess(ClientResponse response, RequestFuture<Void> future) {
-                            handleConsumerMetadataResponse(response, future);
-                        }
-                    });
-        }
-    }
-
-    private void handleConsumerMetadataResponse(ClientResponse resp, RequestFuture<Void> future) {
-        log.debug("Consumer metadata response {}", resp);
-
-        // parse the response to get the coordinator info if it is not disconnected,
-        // otherwise we need to request metadata update
-        if (resp.wasDisconnected()) {
-            future.raise(new DisconnectException());
-        } else if (!coordinatorUnknown()) {
-            // We already found the coordinator, so ignore the request
-            future.complete(null);
-        } else {
-            ConsumerMetadataResponse consumerMetadataResponse = new ConsumerMetadataResponse(resp.responseBody());
-            // use MAX_VALUE - node.id as the coordinator id to mimic separate connections
-            // for the coordinator in the underlying network client layer
-            // TODO: this needs to be better handled in KAFKA-1935
-            if (consumerMetadataResponse.errorCode() == Errors.NONE.code()) {
-                this.consumerCoordinator = new Node(Integer.MAX_VALUE - consumerMetadataResponse.node().id(),
-                        consumerMetadataResponse.node().host(),
-                        consumerMetadataResponse.node().port());
-                heartbeatTask.reset();
-                future.complete(null);
-            } else {
-                future.raise(Errors.forCode(consumerMetadataResponse.errorCode()));
-            }
-        }
-    }
-
-    /**
-     * Mark the current coordinator as dead.
-     */
-    private void coordinatorDead() {
-        if (this.consumerCoordinator != null) {
-            log.info("Marking the coordinator {} dead.", this.consumerCoordinator.id());
-            this.consumerCoordinator = null;
-        }
-    }
-
-    private class HeartbeatCompletionHandler extends CoordinatorResponseHandler<HeartbeatResponse, Void> {
-        @Override
-        public HeartbeatResponse parse(ClientResponse response) {
-            return new HeartbeatResponse(response.responseBody());
-        }
-
-        @Override
-        public void handle(HeartbeatResponse heartbeatResponse, RequestFuture<Void> future) {
-            sensors.heartbeatLatency.record(response.requestLatencyMs());
-            short error = heartbeatResponse.errorCode();
-            if (error == Errors.NONE.code()) {
-                log.debug("Received successful heartbeat response.");
-                future.complete(null);
-            } else if (error == Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.code()
-                    || error == Errors.NOT_COORDINATOR_FOR_CONSUMER.code()) {
-                log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead.");
-                coordinatorDead();
-                future.raise(Errors.forCode(error));
-            } else if (error == Errors.REBALANCE_IN_PROGRESS.code()) {
-                log.info("Attempt to heart beat failed since the group is rebalancing, try to re-join group.");
-                subscriptions.needReassignment();
-                future.raise(Errors.REBALANCE_IN_PROGRESS);
-            } else if (error == Errors.ILLEGAL_GENERATION.code()) {
-                log.info("Attempt to heart beat failed since generation id is not legal, try to re-join group.");
-                subscriptions.needReassignment();
-                future.raise(Errors.ILLEGAL_GENERATION);
-            } else if (error == Errors.UNKNOWN_CONSUMER_ID.code()) {
-                log.info("Attempt to heart beat failed since consumer id is not valid, reset it and try to re-join group.");
-                consumerId = JoinGroupRequest.UNKNOWN_CONSUMER_ID;
-                subscriptions.needReassignment();
-                future.raise(Errors.UNKNOWN_CONSUMER_ID);
-            } else {
-                future.raise(new KafkaException("Unexpected error in heartbeat response: "
-                        + Errors.forCode(error).exception().getMessage()));
-            }
-        }
-    }
-
-    private abstract class CoordinatorResponseHandler<R, T>
-            extends RequestFutureAdapter<ClientResponse, T> {
-        protected ClientResponse response;
-
-        public abstract R parse(ClientResponse response);
-
-        public abstract void handle(R response, RequestFuture<T> future);
-
-        @Override
-        public void onSuccess(ClientResponse clientResponse, RequestFuture<T> future) {
-            this.response = clientResponse;
-
-            if (clientResponse.wasDisconnected()) {
-                int correlation = response.request().request().header().correlationId();
-                log.debug("Cancelled request {} with correlation id {} due to coordinator {} being disconnected",
-                        response.request(),
-                        correlation,
-                        response.request().request().destination());
-
-                // mark the coordinator as dead
-                coordinatorDead();
-                future.raise(new DisconnectException());
-                return;
-            }
-
-            R response = parse(clientResponse);
-            handle(response, future);
-        }
-
-        @Override
-        public void onFailure(RuntimeException e, RequestFuture<T> future) {
-            if (e instanceof DisconnectException) {
-                log.debug("Coordinator request failed", e);
-                coordinatorDead();
-            }
-            future.raise(e);
-        }
-    }
-
-
-    private class CoordinatorMetrics {
-        public final Metrics metrics;
-        public final String metricGrpName;
-
-        public final Sensor commitLatency;
-        public final Sensor heartbeatLatency;
-        public final Sensor partitionReassignments;
-
-        public CoordinatorMetrics(Metrics metrics, String metricGrpPrefix, Map<String, String> tags) {
-            this.metrics = metrics;
-            this.metricGrpName = metricGrpPrefix + "-coordinator-metrics";
-
-            this.commitLatency = metrics.sensor("commit-latency");
-            this.commitLatency.add(new MetricName("commit-latency-avg",
-                this.metricGrpName,
-                "The average time taken for a commit request",
-                tags), new Avg());
-            this.commitLatency.add(new MetricName("commit-latency-max",
-                this.metricGrpName,
-                "The max time taken for a commit request",
-                tags), new Max());
-            this.commitLatency.add(new MetricName("commit-rate",
-                this.metricGrpName,
-                "The number of commit calls per second",
-                tags), new Rate(new Count()));
-
-            this.heartbeatLatency = metrics.sensor("heartbeat-latency");
-            this.heartbeatLatency.add(new MetricName("heartbeat-response-time-max",
-                this.metricGrpName,
-                "The max time taken to receive a response to a hearbeat request",
-                tags), new Max());
-            this.heartbeatLatency.add(new MetricName("heartbeat-rate",
-                this.metricGrpName,
-                "The average number of heartbeats per second",
-                tags), new Rate(new Count()));
-
-            this.partitionReassignments = metrics.sensor("reassignment-latency");
-            this.partitionReassignments.add(new MetricName("reassignment-time-avg",
-                this.metricGrpName,
-                "The average time taken for a partition reassignment",
-                tags), new Avg());
-            this.partitionReassignments.add(new MetricName("reassignment-time-max",
-                this.metricGrpName,
-                "The max time taken for a partition reassignment",
-                tags), new Avg());
-            this.partitionReassignments.add(new MetricName("reassignment-rate",
-                this.metricGrpName,
-                "The number of partition reassignments per second",
-                tags), new Rate(new Count()));
-
-            Measurable numParts =
-                new Measurable() {
-                    public double measure(MetricConfig config, long now) {
-                        return subscriptions.assignedPartitions().size();
-                    }
-                };
-            metrics.addMetric(new MetricName("assigned-partitions",
-                this.metricGrpName,
-                "The number of partitions currently assigned to this consumer",
-                tags),
-                numParts);
-
-            Measurable lastHeartbeat =
-                new Measurable() {
-                    public double measure(MetricConfig config, long now) {
-                        return TimeUnit.SECONDS.convert(now - heartbeat.lastHeartbeatSend(), TimeUnit.MILLISECONDS);
-                    }
-                };
-            metrics.addMetric(new MetricName("last-heartbeat-seconds-ago",
-                this.metricGrpName,
-                "The number of seconds since the last controller heartbeat",
-                tags),
-                lastHeartbeat);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
index 7e55d46..f119552 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
@@ -167,20 +167,31 @@ public class Fetcher<K, V> {
         }
     }
 
-
+    /**
+     * Get topic metadata for all topics in the cluster
+     * @param timeout time for which getting topic metadata is attempted
+     * @return The map of topics with their partition information
+     */
+    public Map<String, List<PartitionInfo>> getAllTopicMetadata(long timeout) {
+        return getTopicMetadata(null, timeout);
+    }
 
     /**
      * Get metadata for all topics present in Kafka cluster
      *
-     * @param timeout time for which getting all topics is attempted
-     * @return The map of topics and its partitions
+     * @param topics The list of topics to fetch or null to fetch all
+     * @param timeout time for which getting topic metadata is attempted
+     * @return The map of topics with their partition information
      */
-    public Map<String, List<PartitionInfo>> getAllTopics(long timeout) {
+    public Map<String, List<PartitionInfo>> getTopicMetadata(List<String> topics, long timeout) {
+        if (topics != null && topics.isEmpty())
+            return Collections.emptyMap();
+
         final HashMap<String, List<PartitionInfo>> topicsPartitionInfos = new HashMap<>();
         long startTime = time.milliseconds();
 
         while (time.milliseconds() - startTime < timeout) {
-            RequestFuture<ClientResponse> requestFuture = sendMetadataRequest();
+            RequestFuture<ClientResponse> requestFuture = sendMetadataRequest(topics);
             if (requestFuture != null) {
                 client.poll(requestFuture);
 
@@ -209,11 +220,12 @@ public class Fetcher<K, V> {
      * Send Metadata Request to least loaded node in Kafka cluster asynchronously
      * @return A future that indicates result of sent metadata request
      */
-    public RequestFuture<ClientResponse> sendMetadataRequest() {
+    public RequestFuture<ClientResponse> sendMetadataRequest(List<String> topics) {
+        if (topics == null)
+            topics = Collections.emptyList();
         final Node node = client.leastLoadedNode();
         return node == null ? null :
-            client.send(
-                node, ApiKeys.METADATA, new MetadataRequest(Collections.<String>emptyList()));
+            client.send(node, ApiKeys.METADATA, new MetadataRequest(topics));
     }
 
     /**
@@ -448,8 +460,9 @@ public class Fetcher<K, V> {
                 long fetched = this.subscriptions.fetched(partition);
                 long consumed = this.subscriptions.consumed(partition);
                 // Only fetch data for partitions whose previously fetched data has been consumed
-                if (consumed == fetched)
+                if (consumed == fetched) {
                     fetch.put(partition, new FetchRequest.PartitionData(fetched, this.fetchSize));
+                }
             }
         }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignor.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignor.java
new file mode 100644
index 0000000..46bfa75
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignor.java
@@ -0,0 +1,117 @@
+/**
+ * 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 org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.TopicPartition;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This interface is used to define custom partition assignment for use in
+ * {@link org.apache.kafka.clients.consumer.KafkaConsumer}. Members of the consumer group subscribe
+ * to the topics they are interested in and forward their subscriptions to a Kafka broker serving
+ * as the group coordinator. The coordinator selects one member to perform the group assignment and
+ * propagates the subscriptions of all members to it. Then {@link #assign(Cluster, Map)} is called
+ * to perform the assignment and the results are forwarded back to each respective members
+ *
+ * In some cases, it is useful to forward additional metadata to the assignor in order to make
+ * assignment decisions. For this, you can override {@link #subscription(Set)} and provide custom
+ * userData in the returned Subscription. For example, to have a rack-aware assignor, an implementation
+ * can use this user data to forward the rackId belonging to each member.
+ */
+public interface PartitionAssignor {
+
+    /**
+     * Return a serializable object representing the local member's subscription. This can include
+     * additional information as well (e.g. local host/rack information) which can be leveraged in
+     * {@link #assign(Cluster, Map)}.
+     * @param topics Topics subscribed to through {@link org.apache.kafka.clients.consumer.KafkaConsumer#subscribe(List)}
+     *               and variants
+     * @return Non-null subscription with optional user data
+     */
+    Subscription subscription(Set<String> topics);
+
+    /**
+     * Perform the group assignment given the member subscriptions and current cluster metadata.
+     * @param metadata Current topic/broker metadata known by consumer
+     * @param subscriptions Subscriptions from all members provided through {@link #subscription(Set)}
+     * @return A map from the members to their respective assignment. This should have one entry
+     *         for all members who in the input subscription map.
+     */
+    Map<String, Assignment> assign(Cluster metadata, Map<String, Subscription> subscriptions);
+
+
+    /**
+     * Callback which is invoked when a group member receives its assignment from the leader.
+     * @param assignment The local member's assignment as provided by the leader in {@link #assign(Cluster, Map)}
+     */
+    void onAssignment(Assignment assignment);
+
+
+    /**
+     * Unique name for this assignor (e.g. "range" or "roundrobin")
+     * @return non-null unique name
+     */
+    String name();
+
+    class Subscription {
+        private final List<String> topics;
+        private final ByteBuffer userData;
+
+        public Subscription(List<String> topics, ByteBuffer userData) {
+            this.topics = topics;
+            this.userData = userData;
+        }
+
+        public Subscription(List<String> topics) {
+            this(topics, ByteBuffer.wrap(new byte[0]));
+        }
+
+        public List<String> topics() {
+            return topics;
+        }
+
+        public ByteBuffer userData() {
+            return userData;
+        }
+
+    }
+
+    class Assignment {
+        private final List<TopicPartition> partitions;
+        private final ByteBuffer userData;
+
+        public Assignment(List<TopicPartition> partitions, ByteBuffer userData) {
+            this.partitions = partitions;
+            this.userData = userData;
+        }
+
+        public Assignment(List<TopicPartition> partitions) {
+            this(partitions, ByteBuffer.wrap(new byte[0]));
+        }
+
+        public List<TopicPartition> partitions() {
+            return partitions;
+        }
+
+        public ByteBuffer userData() {
+            return userData;
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java
index f5c1afc..7be99bd 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java
@@ -175,6 +175,20 @@ public class RequestFuture<T> {
         return adapted;
     }
 
+    public void chain(final RequestFuture<T> future) {
+        addListener(new RequestFutureListener<T>() {
+            @Override
+            public void onSuccess(T value) {
+                future.complete(value);
+            }
+
+            @Override
+            public void onFailure(RuntimeException e) {
+                future.raise(e);
+            }
+        });
+    }
+
     public static <T> RequestFuture<T> failure(RuntimeException e) {
         RequestFuture<T> future = new RequestFuture<T>();
         future.raise(e);
@@ -188,7 +202,7 @@ public class RequestFuture<T> {
     }
 
     public static <T> RequestFuture<T> coordinatorNotAvailable() {
-        return failure(Errors.CONSUMER_COORDINATOR_NOT_AVAILABLE.exception());
+        return failure(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.exception());
     }
 
     public static <T> RequestFuture<T> leaderNotAvailable() {

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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
index 25a0e90..6e79a7f 100644
--- 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
@@ -29,7 +29,7 @@ import java.util.regex.Pattern;
 /**
  * A class for tracking the topics, partitions, and offsets for the consumer. A partition
  * is "assigned" either directly with {@link #assign(List)} (manual assignment)
- * or with {@link #changePartitionAssignment(List)} (automatic assignment).
+ * or with {@link #changePartitionAssignment(Collection)} (automatic assignment).
  *
  * Once assigned, the partition is not considered "fetchable" until its initial position has
  * been set with {@link #seek(TopicPartition, long)}. Fetchable partitions track a fetch
@@ -54,6 +54,9 @@ public class SubscriptionState {
     /* the list of topics the user has requested */
     private final Set<String> subscription;
 
+    /* the list of topics the group has subscribed to (set only for the leader on join group completion) */
+    private final Set<String> groupSubscription;
+
     /* the list of partitions the user has requested */
     private final Set<TopicPartition> userAssignment;
 
@@ -80,6 +83,7 @@ public class SubscriptionState {
         this.subscription = new HashSet<>();
         this.userAssignment = new HashSet<>();
         this.assignment = new HashMap<>();
+        this.groupSubscription = new HashSet<>();
         this.needsPartitionAssignment = false;
         this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up
         this.subscribedPattern = null;
@@ -101,6 +105,7 @@ public class SubscriptionState {
         if (!this.subscription.equals(new HashSet<>(topicsToSubscribe))) {
             this.subscription.clear();
             this.subscription.addAll(topicsToSubscribe);
+            this.groupSubscription.addAll(topicsToSubscribe);
             this.needsPartitionAssignment = true;
 
             // Remove any assigned partitions which are no longer subscribed to
@@ -110,10 +115,22 @@ public class SubscriptionState {
                     it.remove();
             }
         }
+    }
 
+    /**
+     * Add topics to the current group subscription. This is used by the group leader to ensure
+     * that it receives metadata updates for all topics that the group is interested in.
+     * @param topics The topics to add to the group subscription
+     */
+    public void groupSubscribe(Collection<String> topics) {
+        if (!this.userAssignment.isEmpty())
+            throw new IllegalStateException(SUBSCRIPTION_EXCEPTION_MESSAGE);
+        this.groupSubscription.addAll(topics);
     }
 
     public void needReassignment() {
+        //
+        this.groupSubscription.retainAll(subscription);
         this.needsPartitionAssignment = true;
     }
 
@@ -142,6 +159,10 @@ public class SubscriptionState {
         this.subscribedPattern = pattern;
     }
 
+    public boolean hasPatternSubscription() {
+        return subscribedPattern != null;
+    }
+
     public void unsubscribe() {
         this.subscription.clear();
         this.assignment.clear();
@@ -154,15 +175,24 @@ public class SubscriptionState {
         return this.subscribedPattern;
     }
 
-    public void clearAssignment() {
-        this.assignment.clear();
-        this.needsPartitionAssignment = !subscription().isEmpty();
-    }
-
     public Set<String> subscription() {
         return this.subscription;
     }
 
+    /**
+     * Get the subscription for the group. For the leader, this will include the union of the
+     * subscriptions of all group members. For followers, it is just that member's subscription.
+     * This is used when querying topic metadata to detect the metadata changes which would
+     * require rebalancing. The leader fetches metadata for all topics in the group so that it
+     * can do the partition assignment (which requires at least partition counts for all topics
+     * to be assigned).
+     * @return The union of all subscribed topics in the group if this member is the leader
+     *   of the current generation; otherwise it returns the same set as {@link #subscription()}
+     */
+    public Set<String> groupSubscription() {
+        return this.groupSubscription;
+    }
+
     public Long fetched(TopicPartition tp) {
         return assignedState(tp).fetched;
     }
@@ -280,7 +310,7 @@ public class SubscriptionState {
         for (TopicPartition tp : assignments)
             if (!this.subscription.contains(tp.topic()))
                 throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic.");
-        this.clearAssignment();
+        this.assignment.clear();
         for (TopicPartition tp: assignments)
             addAssignedPartition(tp);
         this.needsPartitionAssignment = false;

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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 60594a7..e6a2e43 100644
--- a/clients/src/main/java/org/apache/kafka/common/Cluster.java
+++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java
@@ -173,6 +173,16 @@ public final class Cluster {
     }
 
     /**
+     * Get the number of partitions for the given topic
+     * @param topic The topic to get the number of partitions for
+     * @return The number of partitions or null if there is no corresponding metadata
+     */
+    public Integer partitionCountForTopic(String topic) {
+        List<PartitionInfo> partitionInfos = this.partitionsByTopic.get(topic);
+        return partitionInfos == null ? null : partitionInfos.size();
+    }
+
+    /**
      * Get all topics.
      * @return a set of all topics
      */

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.java b/clients/src/main/java/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.java
deleted file mode 100644
index ba9ce82..0000000
--- a/clients/src/main/java/org/apache/kafka/common/errors/ConsumerCoordinatorNotAvailableException.java
+++ /dev/null
@@ -1,40 +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.common.errors;
-
-/**
- * The broker returns this error code for consumer metadata requests or offset commit requests if the offsets topic has
- * not yet been created.
- */
-public class ConsumerCoordinatorNotAvailableException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public ConsumerCoordinatorNotAvailableException() {
-        super();
-    }
-
-    public ConsumerCoordinatorNotAvailableException(String message) {
-        super(message);
-    }
-
-    public ConsumerCoordinatorNotAvailableException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public ConsumerCoordinatorNotAvailableException(Throwable cause) {
-        super(cause);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java b/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java
new file mode 100644
index 0000000..c0949e3
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+
+package org.apache.kafka.common.errors;
+
+/**
+ * The broker returns this error code for consumer metadata requests or offset commit requests if the offsets topic has
+ * not yet been created.
+ */
+public class GroupCoordinatorNotAvailableException extends RetriableException {
+
+    private static final long serialVersionUID = 1L;
+
+    public GroupCoordinatorNotAvailableException() {
+        super();
+    }
+
+    public GroupCoordinatorNotAvailableException(String message) {
+        super(message);
+    }
+
+    public GroupCoordinatorNotAvailableException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public GroupCoordinatorNotAvailableException(Throwable cause) {
+        super(cause);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.java
deleted file mode 100644
index b6c83b4..0000000
--- a/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForConsumerException.java
+++ /dev/null
@@ -1,40 +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.common.errors;
-
-/**
- * The broker returns this error code if it receives an offset fetch or commit request for a consumer group that it is
- * not a coordinator for.
- */
-public class NotCoordinatorForConsumerException extends RetriableException {
-
-    private static final long serialVersionUID = 1L;
-
-    public NotCoordinatorForConsumerException() {
-        super();
-    }
-
-    public NotCoordinatorForConsumerException(String message) {
-        super(message);
-    }
-
-    public NotCoordinatorForConsumerException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public NotCoordinatorForConsumerException(Throwable cause) {
-        super(cause);
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForGroupException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForGroupException.java b/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForGroupException.java
new file mode 100644
index 0000000..bc56eb0
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/errors/NotCoordinatorForGroupException.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+
+package org.apache.kafka.common.errors;
+
+/**
+ * The broker returns this error code if it receives an offset fetch or commit request for a consumer group that it is
+ * not a coordinator for.
+ */
+public class NotCoordinatorForGroupException extends RetriableException {
+
+    private static final long serialVersionUID = 1L;
+
+    public NotCoordinatorForGroupException() {
+        super();
+    }
+
+    public NotCoordinatorForGroupException(String message) {
+        super(message);
+    }
+
+    public NotCoordinatorForGroupException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public NotCoordinatorForGroupException(Throwable cause) {
+        super(cause);
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/common/errors/UnknownConsumerIdException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnknownConsumerIdException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnknownConsumerIdException.java
deleted file mode 100644
index 28bfd72..0000000
--- a/clients/src/main/java/org/apache/kafka/common/errors/UnknownConsumerIdException.java
+++ /dev/null
@@ -1,33 +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.common.errors;
-
-public class UnknownConsumerIdException extends ApiException {
-    private static final long serialVersionUID = 1L;
-
-    public UnknownConsumerIdException() {
-        super();
-    }
-
-    public UnknownConsumerIdException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public UnknownConsumerIdException(String message) {
-        super(message);
-    }
-
-    public UnknownConsumerIdException(Throwable cause) {
-        super(cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/common/errors/UnknownMemberIdException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnknownMemberIdException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnknownMemberIdException.java
new file mode 100644
index 0000000..f8eab90
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/errors/UnknownMemberIdException.java
@@ -0,0 +1,33 @@
+/**
+ * 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.errors;
+
+public class UnknownMemberIdException extends ApiException {
+    private static final long serialVersionUID = 1L;
+
+    public UnknownMemberIdException() {
+        super();
+    }
+
+    public UnknownMemberIdException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public UnknownMemberIdException(String message) {
+        super(message);
+    }
+
+    public UnknownMemberIdException(Throwable cause) {
+        super(cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
index fab8b02..af7b266 100644
--- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
@@ -30,10 +30,11 @@ public enum ApiKeys {
     CONTROLLED_SHUTDOWN_KEY(7, "ControlledShutdown"),
     OFFSET_COMMIT(8, "OffsetCommit"),
     OFFSET_FETCH(9, "OffsetFetch"),
-    CONSUMER_METADATA(10, "ConsumerMetadata"),
+    GROUP_METADATA(10, "GroupMetadata"),
     JOIN_GROUP(11, "JoinGroup"),
     HEARTBEAT(12, "Heartbeat"),
-    LEAVE_GROUP(13, "LeaveGroup");
+    LEAVE_GROUP(13, "LeaveGroup"),
+    SYNC_GROUP(14, "SyncGroup");
 
     private static ApiKeys[] codeToType;
     public static final int MAX_API_KEY;

http://git-wip-us.apache.org/repos/asf/kafka/blob/86eb74d9/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 220132f..3191636 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
@@ -60,10 +60,10 @@ public enum Errors {
             new NetworkException("The server disconnected before a response was received.")),
     OFFSET_LOAD_IN_PROGRESS(14,
             new OffsetLoadInProgressException("The coordinator is loading offsets and can't process requests.")),
-    CONSUMER_COORDINATOR_NOT_AVAILABLE(15,
-            new ConsumerCoordinatorNotAvailableException("The coordinator is not available.")),
-    NOT_COORDINATOR_FOR_CONSUMER(16,
-            new NotCoordinatorForConsumerException("This is not the correct coordinator for this consumer.")),
+    GROUP_COORDINATOR_NOT_AVAILABLE(15,
+            new GroupCoordinatorNotAvailableException("The group coordinator is not available.")),
+    NOT_COORDINATOR_FOR_GROUP(16,
+            new NotCoordinatorForGroupException("This is not the correct coordinator for this group.")),
     INVALID_TOPIC_EXCEPTION(17,
             new InvalidTopicException("The request attempted to perform an operation on an invalid topic.")),
     RECORD_LIST_TOO_LARGE(18,
@@ -75,17 +75,13 @@ public enum Errors {
     INVALID_REQUIRED_ACKS(21,
             new InvalidRequiredAcksException("Produce request specified an invalid value for required acks.")),
     ILLEGAL_GENERATION(22,
-            new IllegalGenerationException("Specified consumer generation id is not valid.")),
-    INCONSISTENT_PARTITION_ASSIGNMENT_STRATEGY(23,
-            new ApiException("The request partition assignment strategy does not match that of the group.")),
-    UNKNOWN_PARTITION_ASSIGNMENT_STRATEGY(24,
-            new ApiException("The request partition assignment strategy is unknown to the broker.")),
-    UNKNOWN_CONSUMER_ID(25,
-            new UnknownConsumerIdException("The coordinator is not aware of this consumer.")),
+            new IllegalGenerationException("Specified group generation id is not valid.")),
+    INCONSISTENT_GROUP_PROTOCOL(23,
+            new ApiException("The group member's supported protocols are incompatible with those of existing members.")),
+    UNKNOWN_MEMBER_ID(25,
+            new UnknownMemberIdException("The coordinator is not aware of this member.")),
     INVALID_SESSION_TIMEOUT(26,
             new ApiException("The session timeout is not within an acceptable range.")),
-    COMMITTING_PARTITIONS_NOT_ASSIGNED(27,
-            new ApiException("Some of the committing partitions are not assigned the committer")),
     INVALID_COMMIT_OFFSET_SIZE(28,
             new ApiException("The committing offset data size is not valid")),
     AUTHORIZATION_FAILED(29, new ApiException("Request is not authorized.")),