You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by jg...@apache.org on 2017/05/24 22:28:51 UTC

[1/5] kafka git commit: KAFKA-5259; TransactionalId auth implies ProducerId auth

Repository: kafka
Updated Branches:
  refs/heads/trunk 88200938f -> 38f6cae9e


http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala
new file mode 100644
index 0000000..358e12c
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorTest.scala
@@ -0,0 +1,1492 @@
+/**
+ * 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.group
+
+import kafka.common.OffsetAndMetadata
+import kafka.server.{DelayedOperationPurgatory, KafkaConfig, ReplicaManager}
+import kafka.utils._
+import kafka.utils.timer.MockTimer
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch}
+import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
+import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest, OffsetFetchResponse, TransactionResult}
+import org.easymock.{Capture, EasyMock, IAnswer}
+import java.util.concurrent.TimeUnit
+
+import org.apache.kafka.common.internals.Topic
+import org.junit.Assert._
+import org.junit.{After, Assert, Before, Test}
+import org.scalatest.junit.JUnitSuite
+
+import scala.collection._
+import scala.concurrent.duration.Duration
+import scala.concurrent.{Await, Future, Promise, TimeoutException}
+
+class GroupCoordinatorTest extends JUnitSuite {
+  type JoinGroupCallback = JoinGroupResult => Unit
+  type SyncGroupCallbackParams = (Array[Byte], Errors)
+  type SyncGroupCallback = (Array[Byte], Errors) => Unit
+  type HeartbeatCallbackParams = Errors
+  type HeartbeatCallback = Errors => Unit
+  type CommitOffsetCallbackParams = Map[TopicPartition, Errors]
+  type CommitOffsetCallback = Map[TopicPartition, Errors] => Unit
+  type LeaveGroupCallbackParams = Errors
+  type LeaveGroupCallback = Errors => Unit
+
+  val ClientId = "consumer-test"
+  val ClientHost = "localhost"
+  val ConsumerMinSessionTimeout = 10
+  val ConsumerMaxSessionTimeout = 1000
+  val DefaultRebalanceTimeout = 500
+  val DefaultSessionTimeout = 500
+  val GroupInitialRebalanceDelay = 50
+  var timer: MockTimer = null
+  var groupCoordinator: GroupCoordinator = null
+  var replicaManager: ReplicaManager = null
+  var scheduler: KafkaScheduler = null
+  var zkUtils: ZkUtils = null
+
+  private val groupId = "groupId"
+  private val protocolType = "consumer"
+  private val memberId = "memberId"
+  private val metadata = Array[Byte]()
+  private val protocols = List(("range", metadata))
+  private var groupPartitionId: Int = -1
+
+  // we use this string value since its hashcode % #.partitions is different
+  private val otherGroupId = "otherGroup"
+
+  @Before
+  def setUp() {
+    val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
+    props.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString)
+    props.setProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString)
+    props.setProperty(KafkaConfig.GroupInitialRebalanceDelayMsProp, GroupInitialRebalanceDelay.toString)
+    // make two partitions of the group topic to make sure some partitions are not owned by the coordinator
+    val ret = mutable.Map[String, Map[Int, Seq[Int]]]()
+    ret += (Topic.GROUP_METADATA_TOPIC_NAME -> Map(0 -> Seq(1), 1 -> Seq(1)))
+
+    replicaManager = EasyMock.createNiceMock(classOf[ReplicaManager])
+
+    zkUtils = EasyMock.createNiceMock(classOf[ZkUtils])
+    // make two partitions of the group topic to make sure some partitions are not owned by the coordinator
+    EasyMock.expect(zkUtils.getTopicPartitionCount(Topic.GROUP_METADATA_TOPIC_NAME)).andReturn(Some(2))
+    EasyMock.replay(zkUtils)
+
+    timer = new MockTimer
+
+    val config = KafkaConfig.fromProps(props)
+
+    val heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", timer, config.brokerId, reaperEnabled = false)
+    val joinPurgatory = new DelayedOperationPurgatory[DelayedJoin]("Rebalance", timer, config.brokerId, reaperEnabled = false)
+
+    groupCoordinator = GroupCoordinator(config, zkUtils, replicaManager, heartbeatPurgatory, joinPurgatory, timer.time)
+    groupCoordinator.startup(false)
+
+    // add the partition into the owned partition list
+    groupPartitionId = groupCoordinator.partitionFor(groupId)
+    groupCoordinator.groupManager.addPartitionOwnership(groupPartitionId)
+  }
+
+  @After
+  def tearDown() {
+    EasyMock.reset(replicaManager)
+    if (groupCoordinator != null)
+      groupCoordinator.shutdown()
+  }
+
+  @Test
+  def testOffsetsRetentionMsIntegerOverflow() {
+    val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
+    props.setProperty(KafkaConfig.OffsetsRetentionMinutesProp, Integer.MAX_VALUE.toString)
+    val config = KafkaConfig.fromProps(props)
+    val offsetConfig = GroupCoordinator.offsetConfig(config)
+    assertEquals(offsetConfig.offsetsRetentionMs, Integer.MAX_VALUE * 60L * 1000L)
+  }
+
+  @Test
+  def testJoinGroupWrongCoordinator() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(otherGroupId, memberId, protocolType, protocols)
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.NOT_COORDINATOR, joinGroupError)
+  }
+
+  @Test
+  def testJoinGroupSessionTimeoutTooSmall() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols, sessionTimeout = ConsumerMinSessionTimeout - 1)
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.INVALID_SESSION_TIMEOUT, joinGroupError)
+  }
+
+  @Test
+  def testJoinGroupSessionTimeoutTooLarge() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols, sessionTimeout = ConsumerMaxSessionTimeout + 1)
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.INVALID_SESSION_TIMEOUT, joinGroupError)
+  }
+
+  @Test
+  def testJoinGroupUnknownConsumerNewGroup() {
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.UNKNOWN_MEMBER_ID, joinGroupError)
+  }
+
+  @Test
+  def testInvalidGroupId() {
+    val groupId = ""
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    assertEquals(Errors.INVALID_GROUP_ID, joinGroupResult.error)
+  }
+
+  @Test
+  def testValidJoinGroup() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.NONE, joinGroupError)
+  }
+
+  @Test
+  def testJoinGroupInconsistentProtocolType() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    assertEquals(Errors.NONE, joinGroupResult.error)
+
+    EasyMock.reset(replicaManager)
+    val otherJoinGroupResult = await(sendJoinGroup(groupId, otherMemberId, "connect", protocols), 1)
+    assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL, otherJoinGroupResult.error)
+  }
+
+  @Test
+  def testJoinGroupInconsistentGroupProtocol() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupFuture = sendJoinGroup(groupId, memberId, protocolType, List(("range", metadata)))
+
+    EasyMock.reset(replicaManager)
+    val otherJoinGroupResult = joinGroup(groupId, otherMemberId, protocolType, List(("roundrobin", metadata)))
+
+    val joinGroupResult = await(joinGroupFuture, 1)
+    assertEquals(Errors.NONE, joinGroupResult.error)
+    assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL, otherJoinGroupResult.error)
+  }
+
+  @Test
+  def testJoinGroupUnknownConsumerExistingGroup() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val otherMemberId = "memberId"
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    assertEquals(Errors.NONE, joinGroupResult.error)
+
+    EasyMock.reset(replicaManager)
+    val otherJoinGroupResult = await(sendJoinGroup(groupId, otherMemberId, protocolType, protocols), 1)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID, otherJoinGroupResult.error)
+  }
+
+  @Test
+  def testHeartbeatWrongCoordinator() {
+
+    val heartbeatResult = heartbeat(otherGroupId, memberId, -1)
+    assertEquals(Errors.NOT_COORDINATOR, heartbeatResult)
+  }
+
+  @Test
+  def testHeartbeatUnknownGroup() {
+
+    val heartbeatResult = heartbeat(groupId, memberId, -1)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID, heartbeatResult)
+  }
+
+  @Test
+  def testHeartbeatUnknownConsumerExistingGroup() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val otherMemberId = "memberId"
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    val assignedMemberId = joinGroupResult.memberId
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.NONE, joinGroupError)
+
+    EasyMock.reset(replicaManager)
+    val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]()))
+    val syncGroupError = syncGroupResult._2
+    assertEquals(Errors.NONE, syncGroupError)
+
+    EasyMock.reset(replicaManager)
+    val heartbeatResult = heartbeat(groupId, otherMemberId, 1)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID, heartbeatResult)
+  }
+
+  @Test
+  def testHeartbeatRebalanceInProgress() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    val assignedMemberId = joinGroupResult.memberId
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.NONE, joinGroupError)
+
+    EasyMock.reset(replicaManager)
+    val heartbeatResult = heartbeat(groupId, assignedMemberId, 2)
+    assertEquals(Errors.REBALANCE_IN_PROGRESS, heartbeatResult)
+  }
+
+  @Test
+  def testHeartbeatIllegalGeneration() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    val assignedMemberId = joinGroupResult.memberId
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.NONE, joinGroupError)
+
+    EasyMock.reset(replicaManager)
+    val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]()))
+    val syncGroupError = syncGroupResult._2
+    assertEquals(Errors.NONE, syncGroupError)
+
+    EasyMock.reset(replicaManager)
+    val heartbeatResult = heartbeat(groupId, assignedMemberId, 2)
+    assertEquals(Errors.ILLEGAL_GENERATION, heartbeatResult)
+  }
+
+  @Test
+  def testValidHeartbeat() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    val assignedConsumerId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.NONE, joinGroupError)
+
+    EasyMock.reset(replicaManager)
+    val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
+    val syncGroupError = syncGroupResult._2
+    assertEquals(Errors.NONE, syncGroupError)
+
+    EasyMock.reset(replicaManager)
+    val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
+    assertEquals(Errors.NONE, heartbeatResult)
+  }
+
+  @Test
+  def testSessionTimeout() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    val assignedConsumerId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.NONE, joinGroupError)
+
+    EasyMock.reset(replicaManager)
+    val (_, syncGroupError) = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
+    assertEquals(Errors.NONE, syncGroupError)
+
+    EasyMock.reset(replicaManager)
+    EasyMock.expect(replicaManager.getPartition(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId))).andReturn(None)
+    EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(RecordBatch.MAGIC_VALUE_V1)).anyTimes()
+    EasyMock.replay(replicaManager)
+
+    timer.advanceClock(DefaultSessionTimeout + 100)
+
+    EasyMock.reset(replicaManager)
+    val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID, heartbeatResult)
+  }
+
+  @Test
+  def testHeartbeatMaintainsSession() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val sessionTimeout = 1000
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols,
+      rebalanceTimeout = sessionTimeout, sessionTimeout = sessionTimeout)
+    val assignedConsumerId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.NONE, joinGroupError)
+
+    EasyMock.reset(replicaManager)
+    val (_, syncGroupError) = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
+    assertEquals(Errors.NONE, syncGroupError)
+
+    timer.advanceClock(sessionTimeout / 2)
+
+    EasyMock.reset(replicaManager)
+    var heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
+    assertEquals(Errors.NONE, heartbeatResult)
+
+    timer.advanceClock(sessionTimeout / 2 + 100)
+
+    EasyMock.reset(replicaManager)
+    heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
+    assertEquals(Errors.NONE, heartbeatResult)
+  }
+
+  @Test
+  def testCommitMaintainsSession() {
+    val sessionTimeout = 1000
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val tp = new TopicPartition("topic", 0)
+    val offset = OffsetAndMetadata(0)
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols,
+      rebalanceTimeout = sessionTimeout, sessionTimeout = sessionTimeout)
+    val assignedConsumerId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.NONE, joinGroupError)
+
+    EasyMock.reset(replicaManager)
+    val (_, syncGroupError) = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
+    assertEquals(Errors.NONE, syncGroupError)
+
+    timer.advanceClock(sessionTimeout / 2)
+
+    EasyMock.reset(replicaManager)
+    val commitOffsetResult = commitOffsets(groupId, assignedConsumerId, generationId, immutable.Map(tp -> offset))
+    assertEquals(Errors.NONE, commitOffsetResult(tp))
+
+    timer.advanceClock(sessionTimeout / 2 + 100)
+
+    EasyMock.reset(replicaManager)
+    val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
+    assertEquals(Errors.NONE, heartbeatResult)
+  }
+
+  @Test
+  def testSessionTimeoutDuringRebalance() {
+    // create a group with a single member
+    val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols,
+      rebalanceTimeout = 2000, sessionTimeout = 1000)
+    val firstMemberId = firstJoinResult.memberId
+    val firstGenerationId = firstJoinResult.generationId
+    assertEquals(firstMemberId, firstJoinResult.leaderId)
+    assertEquals(Errors.NONE, firstJoinResult.error)
+
+    EasyMock.reset(replicaManager)
+    val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()))
+    assertEquals(Errors.NONE, firstSyncResult._2)
+
+    // now have a new member join to trigger a rebalance
+    EasyMock.reset(replicaManager)
+    val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
+
+    timer.advanceClock(500)
+
+    EasyMock.reset(replicaManager)
+    var heartbeatResult = heartbeat(groupId, firstMemberId, firstGenerationId)
+    assertEquals(Errors.REBALANCE_IN_PROGRESS, heartbeatResult)
+
+    // letting the session expire should make the member fall out of the group
+    timer.advanceClock(1100)
+
+    EasyMock.reset(replicaManager)
+    heartbeatResult = heartbeat(groupId, firstMemberId, firstGenerationId)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID, heartbeatResult)
+
+    // and the rebalance should complete with only the new member
+    val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100)
+    assertEquals(Errors.NONE, otherJoinResult.error)
+  }
+
+  @Test
+  def testRebalanceCompletesBeforeMemberJoins() {
+    // create a group with a single member
+    val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols,
+      rebalanceTimeout = 1200, sessionTimeout = 1000)
+    val firstMemberId = firstJoinResult.memberId
+    val firstGenerationId = firstJoinResult.generationId
+    assertEquals(firstMemberId, firstJoinResult.leaderId)
+    assertEquals(Errors.NONE, firstJoinResult.error)
+
+    EasyMock.reset(replicaManager)
+    val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()))
+    assertEquals(Errors.NONE, firstSyncResult._2)
+
+    // now have a new member join to trigger a rebalance
+    EasyMock.reset(replicaManager)
+    val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
+
+    // send a couple heartbeats to keep the member alive while the rebalance finishes
+    timer.advanceClock(500)
+    EasyMock.reset(replicaManager)
+    var heartbeatResult = heartbeat(groupId, firstMemberId, firstGenerationId)
+    assertEquals(Errors.REBALANCE_IN_PROGRESS, heartbeatResult)
+
+    timer.advanceClock(500)
+    EasyMock.reset(replicaManager)
+    heartbeatResult = heartbeat(groupId, firstMemberId, firstGenerationId)
+    assertEquals(Errors.REBALANCE_IN_PROGRESS, heartbeatResult)
+
+    // now timeout the rebalance, which should kick the unjoined member out of the group
+    // and let the rebalance finish with only the new member
+    timer.advanceClock(500)
+    val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100)
+    assertEquals(Errors.NONE, otherJoinResult.error)
+  }
+
+  @Test
+  def testSyncGroupEmptyAssignment() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    val assignedConsumerId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.NONE, joinGroupError)
+
+    EasyMock.reset(replicaManager)
+    val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map())
+    val syncGroupError = syncGroupResult._2
+    assertEquals(Errors.NONE, syncGroupError)
+    assertTrue(syncGroupResult._1.isEmpty)
+
+    EasyMock.reset(replicaManager)
+    val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
+    assertEquals(Errors.NONE, heartbeatResult)
+  }
+
+  @Test
+  def testSyncGroupNotCoordinator() {
+    val generation = 1
+
+    val syncGroupResult = syncGroupFollower(otherGroupId, generation, memberId)
+    assertEquals(Errors.NOT_COORDINATOR, syncGroupResult._2)
+  }
+
+  @Test
+  def testSyncGroupFromUnknownGroup() {
+    val generation = 1
+
+    val syncGroupResult = syncGroupFollower(groupId, generation, memberId)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID, syncGroupResult._2)
+  }
+
+  @Test
+  def testSyncGroupFromUnknownMember() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    val assignedConsumerId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    assertEquals(Errors.NONE, joinGroupResult.error)
+
+    EasyMock.reset(replicaManager)
+    val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
+    val syncGroupError = syncGroupResult._2
+    assertEquals(Errors.NONE, syncGroupError)
+
+    EasyMock.reset(replicaManager)
+    val unknownMemberId = "blah"
+    val unknownMemberSyncResult = syncGroupFollower(groupId, generationId, unknownMemberId)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID, unknownMemberSyncResult._2)
+  }
+
+  @Test
+  def testSyncGroupFromIllegalGeneration() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    val assignedConsumerId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    assertEquals(Errors.NONE, joinGroupResult.error)
+
+    EasyMock.reset(replicaManager)
+    // send the sync group with an invalid generation
+    val syncGroupResult = syncGroupLeader(groupId, generationId+1, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
+    assertEquals(Errors.ILLEGAL_GENERATION, syncGroupResult._2)
+  }
+
+  @Test
+  def testJoinGroupFromUnchangedFollowerDoesNotRebalance() {
+    // 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, protocolType, protocols)
+    val firstMemberId = firstJoinResult.memberId
+    val firstGenerationId = firstJoinResult.generationId
+    assertEquals(firstMemberId, firstJoinResult.leaderId)
+    assertEquals(Errors.NONE, firstJoinResult.error)
+
+    EasyMock.reset(replicaManager)
+    val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()))
+    assertEquals(Errors.NONE, firstSyncResult._2)
+
+    EasyMock.reset(replicaManager)
+    val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
+
+    EasyMock.reset(replicaManager)
+    val joinFuture = sendJoinGroup(groupId, firstMemberId, protocolType, protocols)
+
+    val joinResult = await(joinFuture, DefaultSessionTimeout+100)
+    val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100)
+    assertEquals(Errors.NONE, joinResult.error)
+    assertEquals(Errors.NONE, otherJoinResult.error)
+    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(replicaManager)
+    val followerJoinResult = await(sendJoinGroup(groupId, otherJoinResult.memberId, protocolType, protocols), 1)
+
+    assertEquals(Errors.NONE, followerJoinResult.error)
+    assertEquals(nextGenerationId, followerJoinResult.generationId)
+  }
+
+  @Test
+  def testJoinGroupFromUnchangedLeaderShouldRebalance() {
+    val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
+    val firstMemberId = firstJoinResult.memberId
+    val firstGenerationId = firstJoinResult.generationId
+    assertEquals(firstMemberId, firstJoinResult.leaderId)
+    assertEquals(Errors.NONE, firstJoinResult.error)
+
+    EasyMock.reset(replicaManager)
+    val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()))
+    assertEquals(Errors.NONE, 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(replicaManager)
+    val secondJoinResult = await(sendJoinGroup(groupId, firstMemberId, protocolType, protocols), 1)
+
+    assertEquals(Errors.NONE, secondJoinResult.error)
+    assertNotEquals(firstGenerationId, secondJoinResult.generationId)
+  }
+
+  @Test
+  def testLeaderFailureInSyncGroup() {
+    // 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, protocolType, protocols)
+    val firstMemberId = firstJoinResult.memberId
+    val firstGenerationId = firstJoinResult.generationId
+    assertEquals(firstMemberId, firstJoinResult.leaderId)
+    assertEquals(Errors.NONE, firstJoinResult.error)
+
+    EasyMock.reset(replicaManager)
+    val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()))
+    assertEquals(Errors.NONE, firstSyncResult._2)
+
+    EasyMock.reset(replicaManager)
+    val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
+
+    EasyMock.reset(replicaManager)
+    val joinFuture = sendJoinGroup(groupId, firstMemberId, protocolType, protocols)
+
+    val joinResult = await(joinFuture, DefaultSessionTimeout+100)
+    val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100)
+    assertEquals(Errors.NONE, joinResult.error)
+    assertEquals(Errors.NONE, otherJoinResult.error)
+    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(replicaManager)
+    val followerSyncFuture = sendSyncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId)
+
+    timer.advanceClock(DefaultSessionTimeout + 100)
+
+    val followerSyncResult = await(followerSyncFuture, DefaultSessionTimeout+100)
+    assertEquals(Errors.REBALANCE_IN_PROGRESS, followerSyncResult._2)
+  }
+
+  @Test
+  def testSyncGroupFollowerAfterLeader() {
+    // 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, protocolType, protocols)
+    val firstMemberId = firstJoinResult.memberId
+    val firstGenerationId = firstJoinResult.generationId
+    assertEquals(firstMemberId, firstJoinResult.leaderId)
+    assertEquals(Errors.NONE, firstJoinResult.error)
+
+    EasyMock.reset(replicaManager)
+    val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()))
+    assertEquals(Errors.NONE, firstSyncResult._2)
+
+    EasyMock.reset(replicaManager)
+    val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
+
+    EasyMock.reset(replicaManager)
+    val joinFuture = sendJoinGroup(groupId, firstMemberId, protocolType, protocols)
+
+    val joinResult = await(joinFuture, DefaultSessionTimeout+100)
+    val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100)
+    assertEquals(Errors.NONE, joinResult.error)
+    assertEquals(Errors.NONE, otherJoinResult.error)
+    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(replicaManager)
+    val leaderSyncResult = syncGroupLeader(groupId, nextGenerationId, leaderId,
+      Map(leaderId -> leaderAssignment, followerId -> followerAssignment))
+    assertEquals(Errors.NONE, leaderSyncResult._2)
+    assertEquals(leaderAssignment, leaderSyncResult._1)
+
+    EasyMock.reset(replicaManager)
+    val followerSyncResult = syncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId)
+    assertEquals(Errors.NONE, followerSyncResult._2)
+    assertEquals(followerAssignment, followerSyncResult._1)
+  }
+
+  @Test
+  def testSyncGroupLeaderAfterFollower() {
+    // 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, protocolType, protocols)
+    val firstMemberId = joinGroupResult.memberId
+    val firstGenerationId = joinGroupResult.generationId
+    assertEquals(firstMemberId, joinGroupResult.leaderId)
+    assertEquals(Errors.NONE, joinGroupResult.error)
+
+    EasyMock.reset(replicaManager)
+    val syncGroupResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()))
+    val syncGroupError = syncGroupResult._2
+    assertEquals(Errors.NONE, syncGroupError)
+
+    EasyMock.reset(replicaManager)
+    val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
+
+    EasyMock.reset(replicaManager)
+    val joinFuture = sendJoinGroup(groupId, firstMemberId, protocolType, protocols)
+
+    val joinResult = await(joinFuture, DefaultSessionTimeout+100)
+    val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100)
+    assertEquals(Errors.NONE, joinResult.error)
+    assertEquals(Errors.NONE, otherJoinResult.error)
+    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(replicaManager)
+    val followerSyncFuture = sendSyncGroupFollower(groupId, nextGenerationId, followerId)
+
+    EasyMock.reset(replicaManager)
+    val leaderSyncResult = syncGroupLeader(groupId, nextGenerationId, leaderId,
+      Map(leaderId -> leaderAssignment, followerId -> followerAssignment))
+    assertEquals(Errors.NONE, leaderSyncResult._2)
+    assertEquals(leaderAssignment, leaderSyncResult._1)
+
+    val followerSyncResult = await(followerSyncFuture, DefaultSessionTimeout+100)
+    assertEquals(Errors.NONE, followerSyncResult._2)
+    assertEquals(followerAssignment, followerSyncResult._1)
+  }
+
+  @Test
+  def testCommitOffsetFromUnknownGroup() {
+    val generationId = 1
+    val tp = new TopicPartition("topic", 0)
+    val offset = OffsetAndMetadata(0)
+
+    val commitOffsetResult = commitOffsets(groupId, memberId, generationId, immutable.Map(tp -> offset))
+    assertEquals(Errors.ILLEGAL_GENERATION, commitOffsetResult(tp))
+  }
+
+  @Test
+  def testCommitOffsetWithDefaultGeneration() {
+    val tp = new TopicPartition("topic", 0)
+    val offset = OffsetAndMetadata(0)
+
+    val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID,
+      OffsetCommitRequest.DEFAULT_GENERATION_ID, immutable.Map(tp -> offset))
+    assertEquals(Errors.NONE, commitOffsetResult(tp))
+  }
+
+  @Test
+  def testFetchOffsets() {
+    val tp = new TopicPartition("topic", 0)
+    val offset = OffsetAndMetadata(0)
+
+    val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID,
+      OffsetCommitRequest.DEFAULT_GENERATION_ID, immutable.Map(tp -> offset))
+    assertEquals(Errors.NONE, commitOffsetResult(tp))
+
+    val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
+    assertEquals(Errors.NONE, error)
+    assertEquals(Some(0), partitionData.get(tp).map(_.offset))
+  }
+
+  @Test
+  def testBasicFetchTxnOffsets() {
+    val tp = new TopicPartition("topic", 0)
+    val offset = OffsetAndMetadata(0)
+    val producerId = 1000L
+    val producerEpoch : Short = 2
+
+    val commitOffsetResult = commitTransactionalOffsets(groupId, producerId, producerEpoch, immutable.Map(tp -> offset))
+    assertEquals(Errors.NONE, commitOffsetResult(tp))
+
+    val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
+
+    // Validate that the offset isn't materialjzed yet.
+    assertEquals(Errors.NONE, error)
+    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData.get(tp).map(_.offset))
+
+    val offsetsTopic = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId)
+
+    // Send commit marker.
+    groupCoordinator.handleTxnCompletion(producerId, List(offsetsTopic), TransactionResult.COMMIT)
+
+    // Validate that committed offset is materialized.
+    val (secondReqError, secondReqPartitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
+    assertEquals(Errors.NONE, secondReqError)
+    assertEquals(Some(0), secondReqPartitionData.get(tp).map(_.offset))
+  }
+
+  @Test
+  def testFetchTxnOffsetsWithAbort() {
+    val tp = new TopicPartition("topic", 0)
+    val offset = OffsetAndMetadata(0)
+    val producerId = 1000L
+    val producerEpoch : Short = 2
+
+    val commitOffsetResult = commitTransactionalOffsets(groupId, producerId, producerEpoch, immutable.Map(tp -> offset))
+    assertEquals(Errors.NONE, commitOffsetResult(tp))
+
+    val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
+    assertEquals(Errors.NONE, error)
+    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData.get(tp).map(_.offset))
+
+    val offsetsTopic = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId)
+
+    // Validate that the pending commit is discarded.
+    groupCoordinator.handleTxnCompletion(producerId, List(offsetsTopic), TransactionResult.ABORT)
+
+    val (secondReqError, secondReqPartitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
+    assertEquals(Errors.NONE, secondReqError)
+    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), secondReqPartitionData.get(tp).map(_.offset))
+  }
+
+  @Test
+  def testFetchTxnOffsetsIgnoreSpuriousCommit() {
+    val tp = new TopicPartition("topic", 0)
+    val offset = OffsetAndMetadata(0)
+    val producerId = 1000L
+    val producerEpoch : Short = 2
+
+    val commitOffsetResult = commitTransactionalOffsets(groupId, producerId, producerEpoch, immutable.Map(tp -> offset))
+    assertEquals(Errors.NONE, commitOffsetResult(tp))
+
+    val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
+    assertEquals(Errors.NONE, error)
+    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData.get(tp).map(_.offset))
+
+    val offsetsTopic = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId)
+    groupCoordinator.handleTxnCompletion(producerId, List(offsetsTopic), TransactionResult.ABORT)
+
+    val (secondReqError, secondReqPartitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
+    assertEquals(Errors.NONE, secondReqError)
+    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), secondReqPartitionData.get(tp).map(_.offset))
+
+    // Ignore spurious commit.
+    groupCoordinator.handleTxnCompletion(producerId, List(offsetsTopic), TransactionResult.COMMIT)
+
+    val (thirdReqError, thirdReqPartitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
+    assertEquals(Errors.NONE, secondReqError)
+    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), thirdReqPartitionData.get(tp).map(_.offset))
+  }
+
+  @Test
+  def testFetchTxnOffsetsOneProducerMultipleGroups() {
+    // One producer, two groups located on separate offsets topic partitions.
+    // Both group have pending offset commits.
+    // Marker for only one partition is received. That commit should be materialized while the other should not.
+
+    val partitions = List(new TopicPartition("topic1", 0), new TopicPartition("topic2", 0))
+    val offsets = List(OffsetAndMetadata(10), OffsetAndMetadata(15))
+    val producerId = 1000L
+    val producerEpoch: Short = 3
+
+    val groupIds = List(groupId, otherGroupId)
+    val offsetTopicPartitions = List(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupCoordinator.partitionFor(groupId)),
+      new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupCoordinator.partitionFor(otherGroupId)))
+
+    groupCoordinator.groupManager.addPartitionOwnership(offsetTopicPartitions(1).partition)
+    val errors = mutable.ArrayBuffer[Errors]()
+    val partitionData = mutable.ArrayBuffer[Map[TopicPartition, OffsetFetchResponse.PartitionData]]()
+
+    val commitOffsetResults = mutable.ArrayBuffer[CommitOffsetCallbackParams]()
+
+    // Ensure that the two groups map to different partitions.
+    assertNotEquals(offsetTopicPartitions(0), offsetTopicPartitions(1))
+
+    commitOffsetResults.append(commitTransactionalOffsets(groupId, producerId, producerEpoch, immutable.Map(partitions(0) -> offsets(0))))
+    assertEquals(Errors.NONE, commitOffsetResults(0)(partitions(0)))
+    commitOffsetResults.append(commitTransactionalOffsets(otherGroupId, producerId, producerEpoch, immutable.Map(partitions(1) -> offsets(1))))
+    assertEquals(Errors.NONE, commitOffsetResults(1)(partitions(1)))
+
+    // We got a commit for only one __consumer_offsets partition. We should only materialize it's group offsets.
+    groupCoordinator.handleTxnCompletion(producerId, List(offsetTopicPartitions(0)), TransactionResult.COMMIT)
+    groupCoordinator.handleFetchOffsets(groupIds(0), Some(partitions)) match {
+      case (error, partData) =>
+        errors.append(error)
+        partitionData.append(partData)
+      case _ =>
+    }
+
+     groupCoordinator.handleFetchOffsets(groupIds(1), Some(partitions)) match {
+      case (error, partData) =>
+        errors.append(error)
+        partitionData.append(partData)
+      case _ =>
+    }
+
+    assertEquals(2, errors.size)
+    assertEquals(Errors.NONE, errors(0))
+    assertEquals(Errors.NONE, errors(1))
+
+    // Exactly one offset commit should have been materialized.
+    assertEquals(Some(offsets(0).offset), partitionData(0).get(partitions(0)).map(_.offset))
+    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData(0).get(partitions(1)).map(_.offset))
+    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData(1).get(partitions(0)).map(_.offset))
+    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData(1).get(partitions(1)).map(_.offset))
+
+    // Now we receive the other marker.
+    groupCoordinator.handleTxnCompletion(producerId, List(offsetTopicPartitions(1)), TransactionResult.COMMIT)
+    errors.clear()
+    partitionData.clear()
+    groupCoordinator.handleFetchOffsets(groupIds(0), Some(partitions)) match {
+      case (error, partData) =>
+        errors.append(error)
+        partitionData.append(partData)
+      case _ =>
+    }
+
+     groupCoordinator.handleFetchOffsets(groupIds(1), Some(partitions)) match {
+      case (error, partData) =>
+        errors.append(error)
+        partitionData.append(partData)
+      case _ =>
+    }
+    // Two offsets should have been materialized
+    assertEquals(Some(offsets(0).offset), partitionData(0).get(partitions(0)).map(_.offset))
+    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData(0).get(partitions(1)).map(_.offset))
+    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData(1).get(partitions(0)).map(_.offset))
+    assertEquals(Some(offsets(1).offset), partitionData(1).get(partitions(1)).map(_.offset))
+  }
+
+  @Test
+  def testFetchTxnOffsetsMultipleProducersOneGroup() {
+    // One group, two producers
+    // Different producers will commit offsets for different partitions.
+    // Each partition's offsets should be materialized when the corresponding producer's marker is received.
+
+    val partitions = List(new TopicPartition("topic1", 0), new TopicPartition("topic2", 0))
+    val offsets = List(OffsetAndMetadata(10), OffsetAndMetadata(15))
+    val producerIds = List(1000L, 1005L)
+    val producerEpochs: Seq[Short] = List(3, 4)
+
+    val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupCoordinator.partitionFor(groupId))
+
+    val errors = mutable.ArrayBuffer[Errors]()
+    val partitionData = mutable.ArrayBuffer[Map[TopicPartition, OffsetFetchResponse.PartitionData]]()
+
+    val commitOffsetResults = mutable.ArrayBuffer[CommitOffsetCallbackParams]()
+
+    // producer0 commits the offsets for partition0
+    commitOffsetResults.append(commitTransactionalOffsets(groupId, producerIds(0), producerEpochs(0), immutable.Map(partitions(0) -> offsets(0))))
+    assertEquals(Errors.NONE, commitOffsetResults(0)(partitions(0)))
+
+    // producer1 commits the offsets for partition1
+    commitOffsetResults.append(commitTransactionalOffsets(groupId, producerIds(1), producerEpochs(1), immutable.Map(partitions(1) -> offsets(1))))
+    assertEquals(Errors.NONE, commitOffsetResults(1)(partitions(1)))
+
+    // producer0 commits its transaction.
+    groupCoordinator.handleTxnCompletion(producerIds(0), List(offsetTopicPartition), TransactionResult.COMMIT)
+    groupCoordinator.handleFetchOffsets(groupId, Some(partitions)) match {
+      case (error, partData) =>
+        errors.append(error)
+        partitionData.append(partData)
+      case _ =>
+    }
+
+    assertEquals(Errors.NONE, errors(0))
+
+    // We should only see the offset commit for producer0
+    assertEquals(Some(offsets(0).offset), partitionData(0).get(partitions(0)).map(_.offset))
+    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData(0).get(partitions(1)).map(_.offset))
+
+    // producer1 now commits its transaction.
+    groupCoordinator.handleTxnCompletion(producerIds(1), List(offsetTopicPartition), TransactionResult.COMMIT)
+
+    groupCoordinator.handleFetchOffsets(groupId, Some(partitions)) match {
+      case (error, partData) =>
+        errors.append(error)
+        partitionData.append(partData)
+      case _ =>
+    }
+
+    assertEquals(Errors.NONE, errors(1))
+
+    // We should now see the offset commits for both producers.
+    assertEquals(Some(offsets(0).offset), partitionData(1).get(partitions(0)).map(_.offset))
+    assertEquals(Some(offsets(1).offset), partitionData(1).get(partitions(1)).map(_.offset))
+  }
+
+  @Test
+  def testFetchOffsetForUnknownPartition(): Unit = {
+    val tp = new TopicPartition("topic", 0)
+    val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
+    assertEquals(Errors.NONE, error)
+    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData.get(tp).map(_.offset))
+  }
+
+  @Test
+  def testFetchOffsetNotCoordinatorForGroup(): Unit = {
+    val tp = new TopicPartition("topic", 0)
+    val (error, partitionData) = groupCoordinator.handleFetchOffsets(otherGroupId, Some(Seq(tp)))
+    assertEquals(Errors.NOT_COORDINATOR, error)
+    assertTrue(partitionData.isEmpty)
+  }
+
+  @Test
+  def testFetchAllOffsets() {
+    val tp1 = new TopicPartition("topic", 0)
+    val tp2 = new TopicPartition("topic", 1)
+    val tp3 = new TopicPartition("other-topic", 0)
+    val offset1 = OffsetAndMetadata(15)
+    val offset2 = OffsetAndMetadata(16)
+    val offset3 = OffsetAndMetadata(17)
+
+    assertEquals((Errors.NONE, Map.empty), groupCoordinator.handleFetchOffsets(groupId))
+
+    val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID,
+      OffsetCommitRequest.DEFAULT_GENERATION_ID, immutable.Map(tp1 -> offset1, tp2 -> offset2, tp3 -> offset3))
+    assertEquals(Errors.NONE, commitOffsetResult(tp1))
+    assertEquals(Errors.NONE, commitOffsetResult(tp2))
+    assertEquals(Errors.NONE, commitOffsetResult(tp3))
+
+    val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId)
+    assertEquals(Errors.NONE, error)
+    assertEquals(3, partitionData.size)
+    assertTrue(partitionData.forall(_._2.error == Errors.NONE))
+    assertEquals(Some(offset1.offset), partitionData.get(tp1).map(_.offset))
+    assertEquals(Some(offset2.offset), partitionData.get(tp2).map(_.offset))
+    assertEquals(Some(offset3.offset), partitionData.get(tp3).map(_.offset))
+  }
+
+  @Test
+  def testCommitOffsetInAwaitingSync() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val tp = new TopicPartition("topic", 0)
+    val offset = OffsetAndMetadata(0)
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    val assignedMemberId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.NONE, joinGroupError)
+
+    EasyMock.reset(replicaManager)
+    val commitOffsetResult = commitOffsets(groupId, assignedMemberId, generationId, immutable.Map(tp -> offset))
+    assertEquals(Errors.REBALANCE_IN_PROGRESS, commitOffsetResult(tp))
+  }
+
+  @Test
+  def testHeartbeatDuringRebalanceCausesRebalanceInProgress() {
+    // 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, protocolType, protocols)
+    val assignedConsumerId = joinGroupResult.memberId
+    val initialGenerationId = joinGroupResult.generationId
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.NONE, joinGroupError)
+
+    // Then join with a new consumer to trigger a rebalance
+    EasyMock.reset(replicaManager)
+    sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
+
+    // We should be in the middle of a rebalance, so the heartbeat should return rebalance in progress
+    EasyMock.reset(replicaManager)
+    val heartbeatResult = heartbeat(groupId, assignedConsumerId, initialGenerationId)
+    assertEquals(Errors.REBALANCE_IN_PROGRESS, heartbeatResult)
+  }
+
+  @Test
+  def testGenerationIdIncrementsOnRebalance() {
+    val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
+    val initialGenerationId = joinGroupResult.generationId
+    val joinGroupError = joinGroupResult.error
+    val memberId = joinGroupResult.memberId
+    assertEquals(1, initialGenerationId)
+    assertEquals(Errors.NONE, joinGroupError)
+
+    EasyMock.reset(replicaManager)
+    val syncGroupResult = syncGroupLeader(groupId, initialGenerationId, memberId, Map(memberId -> Array[Byte]()))
+    val syncGroupError = syncGroupResult._2
+    assertEquals(Errors.NONE, syncGroupError)
+
+    EasyMock.reset(replicaManager)
+    val joinGroupFuture = sendJoinGroup(groupId, memberId, protocolType, protocols)
+    val otherJoinGroupResult = await(joinGroupFuture, 1)
+
+    val nextGenerationId = otherJoinGroupResult.generationId
+    val otherJoinGroupError = otherJoinGroupResult.error
+    assertEquals(2, nextGenerationId)
+    assertEquals(Errors.NONE, otherJoinGroupError)
+  }
+
+  @Test
+  def testLeaveGroupWrongCoordinator() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val leaveGroupResult = leaveGroup(otherGroupId, memberId)
+    assertEquals(Errors.NOT_COORDINATOR, leaveGroupResult)
+  }
+
+  @Test
+  def testLeaveGroupUnknownGroup() {
+
+    val leaveGroupResult = leaveGroup(groupId, memberId)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID, leaveGroupResult)
+  }
+
+  @Test
+  def testLeaveGroupUnknownConsumerExistingGroup() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val otherMemberId = "consumerId"
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.NONE, joinGroupError)
+
+    EasyMock.reset(replicaManager)
+    val leaveGroupResult = leaveGroup(groupId, otherMemberId)
+    assertEquals(Errors.UNKNOWN_MEMBER_ID, leaveGroupResult)
+  }
+
+  @Test
+  def testValidLeaveGroup() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    val assignedMemberId = joinGroupResult.memberId
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.NONE, joinGroupError)
+
+    EasyMock.reset(replicaManager)
+    val leaveGroupResult = leaveGroup(groupId, assignedMemberId)
+    assertEquals(Errors.NONE, leaveGroupResult)
+  }
+
+  @Test
+  def testListGroupsIncludesStableGroups() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    val assignedMemberId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    assertEquals(Errors.NONE, joinGroupResult.error)
+
+    EasyMock.reset(replicaManager)
+    val syncGroupResult = syncGroupLeader(groupId, generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]()))
+    val syncGroupError = syncGroupResult._2
+    assertEquals(Errors.NONE, syncGroupError)
+
+    val (error, groups) = groupCoordinator.handleListGroups()
+    assertEquals(Errors.NONE, error)
+    assertEquals(1, groups.size)
+    assertEquals(GroupOverview("groupId", "consumer"), groups.head)
+  }
+
+  @Test
+  def testListGroupsIncludesRebalancingGroups() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    assertEquals(Errors.NONE, joinGroupResult.error)
+
+    val (error, groups) = groupCoordinator.handleListGroups()
+    assertEquals(Errors.NONE, error)
+    assertEquals(1, groups.size)
+    assertEquals(GroupOverview("groupId", "consumer"), groups.head)
+  }
+
+  @Test
+  def testDescribeGroupWrongCoordinator() {
+    EasyMock.reset(replicaManager)
+    val (error, _) = groupCoordinator.handleDescribeGroup(otherGroupId)
+    assertEquals(Errors.NOT_COORDINATOR, error)
+  }
+
+  @Test
+  def testDescribeGroupInactiveGroup() {
+    EasyMock.reset(replicaManager)
+    val (error, summary) = groupCoordinator.handleDescribeGroup(groupId)
+    assertEquals(Errors.NONE, error)
+    assertEquals(GroupCoordinator.DeadGroup, summary)
+  }
+
+  @Test
+  def testDescribeGroupStable() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    val assignedMemberId = joinGroupResult.memberId
+    val generationId = joinGroupResult.generationId
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.NONE, joinGroupError)
+
+    EasyMock.reset(replicaManager)
+    val syncGroupResult = syncGroupLeader(groupId, generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]()))
+
+    val syncGroupError = syncGroupResult._2
+    assertEquals(Errors.NONE, syncGroupError)
+
+    EasyMock.reset(replicaManager)
+    val (error, summary) = groupCoordinator.handleDescribeGroup(groupId)
+    assertEquals(Errors.NONE, error)
+    assertEquals(protocolType, summary.protocolType)
+    assertEquals("range", summary.protocol)
+    assertEquals(List(assignedMemberId), summary.members.map(_.memberId))
+  }
+
+  @Test
+  def testDescribeGroupRebalancing() {
+    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
+    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
+    val joinGroupError = joinGroupResult.error
+    assertEquals(Errors.NONE, joinGroupError)
+
+    EasyMock.reset(replicaManager)
+    val (error, summary) = groupCoordinator.handleDescribeGroup(groupId)
+    assertEquals(Errors.NONE, error)
+    assertEquals(protocolType, summary.protocolType)
+    assertEquals(GroupCoordinator.NoProtocol, summary.protocol)
+    assertEquals(AwaitingSync.toString, summary.state)
+    assertTrue(summary.members.map(_.memberId).contains(joinGroupResult.memberId))
+    assertTrue(summary.members.forall(_.metadata.isEmpty))
+    assertTrue(summary.members.forall(_.assignment.isEmpty))
+  }
+
+  @Test
+  def shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() {
+    val firstJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
+    timer.advanceClock(GroupInitialRebalanceDelay / 2)
+    verifyDelayedTaskNotCompleted(firstJoinFuture)
+    timer.advanceClock((GroupInitialRebalanceDelay / 2) + 1)
+    val joinGroupResult = await(firstJoinFuture, 1)
+    assertEquals(Errors.NONE, joinGroupResult.error)
+  }
+
+  private def verifyDelayedTaskNotCompleted(firstJoinFuture: Future[JoinGroupResult]) = {
+    try {
+      await(firstJoinFuture, 1)
+      Assert.fail("should have timed out as rebalance delay not expired")
+    } catch {
+      case _: TimeoutException => // ok
+    }
+  }
+
+  @Test
+  def shouldResetRebalanceDelayWhenNewMemberJoinsGroupInInitialRebalance() {
+    val rebalanceTimeout = GroupInitialRebalanceDelay * 3
+    val firstMemberJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols, rebalanceTimeout)
+    EasyMock.reset(replicaManager)
+    timer.advanceClock(GroupInitialRebalanceDelay - 1)
+    val secondMemberJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols, rebalanceTimeout)
+    EasyMock.reset(replicaManager)
+    timer.advanceClock(2)
+
+    // advance past initial rebalance delay and make sure that tasks
+    // haven't been completed
+    timer.advanceClock(GroupInitialRebalanceDelay / 2 + 1)
+    verifyDelayedTaskNotCompleted(firstMemberJoinFuture)
+    verifyDelayedTaskNotCompleted(secondMemberJoinFuture)
+    // advance clock beyond updated delay and make sure the
+    // tasks have completed
+    timer.advanceClock(GroupInitialRebalanceDelay / 2)
+    val firstResult = await(firstMemberJoinFuture, 1)
+    val secondResult = await(secondMemberJoinFuture, 1)
+    assertEquals(Errors.NONE, firstResult.error)
+    assertEquals(Errors.NONE, secondResult.error)
+  }
+
+  @Test
+  def shouldDelayRebalanceUptoRebalanceTimeout() {
+    val rebalanceTimeout = GroupInitialRebalanceDelay * 2
+    val firstMemberJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols, rebalanceTimeout)
+    EasyMock.reset(replicaManager)
+    val secondMemberJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols, rebalanceTimeout)
+    timer.advanceClock(GroupInitialRebalanceDelay + 1)
+    EasyMock.reset(replicaManager)
+    val thirdMemberJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols, rebalanceTimeout)
+    timer.advanceClock(GroupInitialRebalanceDelay)
+    EasyMock.reset(replicaManager)
+
+    verifyDelayedTaskNotCompleted(firstMemberJoinFuture)
+    verifyDelayedTaskNotCompleted(secondMemberJoinFuture)
+    verifyDelayedTaskNotCompleted(thirdMemberJoinFuture)
+
+    // advance clock beyond rebalanceTimeout
+    timer.advanceClock(1)
+
+    val firstResult = await(firstMemberJoinFuture, 1)
+    val secondResult = await(secondMemberJoinFuture, 1)
+    val thirdResult = await(thirdMemberJoinFuture, 1)
+    assertEquals(Errors.NONE, firstResult.error)
+    assertEquals(Errors.NONE, secondResult.error)
+    assertEquals(Errors.NONE, thirdResult.error)
+  }
+
+  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, error) =>
+      responsePromise.success((assignment, error))
+    (responseFuture, responseCallback)
+  }
+
+  private def setupHeartbeatCallback: (Future[HeartbeatCallbackParams], HeartbeatCallback) = {
+    val responsePromise = Promise[HeartbeatCallbackParams]
+    val responseFuture = responsePromise.future
+    val responseCallback: HeartbeatCallback = error => responsePromise.success(error)
+    (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,
+                            protocolType: String,
+                            protocols: List[(String, Array[Byte])],
+                            rebalanceTimeout: Int = DefaultRebalanceTimeout,
+                            sessionTimeout: Int = DefaultSessionTimeout): Future[JoinGroupResult] = {
+    val (responseFuture, responseCallback) = setupJoinGroupCallback
+
+    EasyMock.replay(replicaManager)
+
+    groupCoordinator.handleJoinGroup(groupId, memberId, "clientId", "clientHost", rebalanceTimeout, sessionTimeout,
+      protocolType, protocols, responseCallback)
+    responseFuture
+  }
+
+
+  private def sendSyncGroupLeader(groupId: String,
+                                  generation: Int,
+                                  leaderId: String,
+                                  assignment: Map[String, Array[Byte]]): Future[SyncGroupCallbackParams] = {
+    val (responseFuture, responseCallback) = setupSyncGroupCallback
+
+    val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
+
+    EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
+      EasyMock.anyShort(),
+      internalTopicsAllowed = EasyMock.eq(true),
+      isFromClient = EasyMock.eq(false),
+      EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
+      EasyMock.capture(capturedArgument),
+      EasyMock.anyObject().asInstanceOf[Option[Object]])).andAnswer(new IAnswer[Unit] {
+      override def answer = capturedArgument.getValue.apply(
+        Map(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId) ->
+          new PartitionResponse(Errors.NONE, 0L, RecordBatch.NO_TIMESTAMP)
+        )
+      )})
+    EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(RecordBatch.MAGIC_VALUE_V1)).anyTimes()
+    EasyMock.replay(replicaManager)
+
+    groupCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback)
+    responseFuture
+  }
+
+  private def sendSyncGroupFollower(groupId: String,
+                                    generation: Int,
+                                    memberId: String): Future[SyncGroupCallbackParams] = {
+    val (responseFuture, responseCallback) = setupSyncGroupCallback
+
+    EasyMock.replay(replicaManager)
+
+    groupCoordinator.handleSyncGroup(groupId, generation, memberId, Map.empty[String, Array[Byte]], responseCallback)
+    responseFuture
+  }
+
+  private def joinGroup(groupId: String,
+                        memberId: String,
+                        protocolType: String,
+                        protocols: List[(String, Array[Byte])],
+                        sessionTimeout: Int = DefaultSessionTimeout,
+                        rebalanceTimeout: Int = DefaultRebalanceTimeout): JoinGroupResult = {
+    val responseFuture = sendJoinGroup(groupId, memberId, protocolType, protocols, rebalanceTimeout, sessionTimeout)
+    timer.advanceClock(GroupInitialRebalanceDelay + 1)
+    // 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(rebalanceTimeout + 100, TimeUnit.MILLISECONDS))
+  }
+
+
+  private def syncGroupFollower(groupId: String,
+                                generationId: Int,
+                                memberId: String,
+                                sessionTimeout: Int = DefaultSessionTimeout): SyncGroupCallbackParams = {
+    val responseFuture = sendSyncGroupFollower(groupId, generationId, memberId)
+    Await.result(responseFuture, Duration(sessionTimeout + 100, TimeUnit.MILLISECONDS))
+  }
+
+  private def syncGroupLeader(groupId: String,
+                              generationId: Int,
+                              memberId: String,
+                              assignment: Map[String, Array[Byte]],
+                              sessionTimeout: Int = DefaultSessionTimeout): SyncGroupCallbackParams = {
+    val responseFuture = sendSyncGroupLeader(groupId, generationId, memberId, assignment)
+    Await.result(responseFuture, Duration(sessionTimeout + 100, TimeUnit.MILLISECONDS))
+  }
+
+  private def heartbeat(groupId: String,
+                        consumerId: String,
+                        generationId: Int): HeartbeatCallbackParams = {
+    val (responseFuture, responseCallback) = setupHeartbeatCallback
+
+    EasyMock.replay(replicaManager)
+
+    groupCoordinator.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: immutable.Map[TopicPartition, OffsetAndMetadata]): CommitOffsetCallbackParams = {
+    val (responseFuture, responseCallback) = setupCommitOffsetsCallback
+
+    val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
+
+    EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
+      EasyMock.anyShort(),
+      internalTopicsAllowed = EasyMock.eq(true),
+      isFromClient = EasyMock.eq(false),
+      EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
+      EasyMock.capture(capturedArgument),
+      EasyMock.anyObject().asInstanceOf[Option[Object]])
+    ).andAnswer(new IAnswer[Unit] {
+      override def answer = capturedArgument.getValue.apply(
+        Map(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId) ->
+          new PartitionResponse(Errors.NONE, 0L, RecordBatch.NO_TIMESTAMP)
+        )
+      )})
+    EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(RecordBatch.MAGIC_VALUE_V1)).anyTimes()
+    EasyMock.replay(replicaManager)
+
+    groupCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback)
+    Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS))
+  }
+
+  private def commitTransactionalOffsets(groupId: String,
+                                         producerId: Long,
+                                         producerEpoch: Short,
+                                         offsets: immutable.Map[TopicPartition, OffsetAndMetadata]): CommitOffsetCallbackParams = {
+    val (responseFuture, responseCallback) = setupCommitOffsetsCallback
+
+    val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
+
+    EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
+      EasyMock.anyShort(),
+      internalTopicsAllowed = EasyMock.eq(true),
+      isFromClient = EasyMock.eq(false),
+      EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
+      EasyMock.capture(capturedArgument),
+      EasyMock.anyObject().asInstanceOf[Option[Object]])
+    ).andAnswer(new IAnswer[Unit] {
+      override def answer = capturedArgument.getValue.apply(
+        Map(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupCoordinator.partitionFor(groupId)) ->
+          new PartitionResponse(Errors.NONE, 0L, RecordBatch.NO_TIMESTAMP)
+        )
+      )})
+    EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(RecordBatch.MAGIC_VALUE_V2)).anyTimes()
+    EasyMock.replay(replicaManager)
+
+    groupCoordinator.handleTxnCommitOffsets(groupId, producerId, producerEpoch, offsets, responseCallback)
+    val result = Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS))
+    EasyMock.reset(replicaManager)
+    result
+  }
+
+  private def leaveGroup(groupId: String, consumerId: String): LeaveGroupCallbackParams = {
+    val (responseFuture, responseCallback) = setupHeartbeatCallback
+
+    EasyMock.expect(replicaManager.getPartition(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId))).andReturn(None)
+    EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(RecordBatch.MAGIC_VALUE_V1)).anyTimes()
+    EasyMock.replay(replicaManager)
+
+    groupCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback)
+    Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS))
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala
index 0e13f89..2db6603 100644
--- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataTest.scala
@@ -455,7 +455,7 @@ class GroupMetadataTest extends JUnitSuite {
     assertTrue(group.hasPendingOffsetCommitsFromProducer(producerId))
     assertTrue(group.hasOffsets)
     assertEquals(None, group.offset(partition))
-    group.failPendingTxnOffsetCommit(producerId, partition, txnOffsetCommit)
+    group.failPendingTxnOffsetCommit(producerId, partition)
     assertFalse(group.hasOffsets)
     assertFalse(group.hasPendingOffsetCommitsFromProducer(producerId))
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
index 7e50049..fa2e55b 100644
--- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
+++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
@@ -250,19 +250,20 @@ class RequestQuotaTest extends BaseRequestTest {
           new OffsetsForLeaderEpochRequest.Builder().add(tp, 0)
 
         case ApiKeys.ADD_PARTITIONS_TO_TXN =>
-          new AddPartitionsToTxnRequest.Builder("txn1", 1, 0, List(tp).asJava)
+          new AddPartitionsToTxnRequest.Builder("test-transactional-id", 1, 0, List(tp).asJava)
 
         case ApiKeys.ADD_OFFSETS_TO_TXN =>
-          new AddOffsetsToTxnRequest.Builder("txn1", 1, 0, "test-txn-group")
+          new AddOffsetsToTxnRequest.Builder("test-transactional-id", 1, 0, "test-txn-group")
 
         case ApiKeys.END_TXN =>
-          new EndTxnRequest.Builder("txn1", 1, 0, TransactionResult.forId(false))
+          new EndTxnRequest.Builder("test-transactional-id", 1, 0, TransactionResult.forId(false))
 
         case ApiKeys.WRITE_TXN_MARKERS =>
           new WriteTxnMarkersRequest.Builder(List.empty.asJava)
 
         case ApiKeys.TXN_OFFSET_COMMIT =>
-          new TxnOffsetCommitRequest.Builder("test-txn-group", 2, 0, Map.empty.asJava)
+          new TxnOffsetCommitRequest.Builder("test-transactional-id", "test-txn-group", 2, 0,
+            Map.empty[TopicPartition, TxnOffsetCommitRequest.CommittedOffset].asJava)
 
         case ApiKeys.DESCRIBE_ACLS =>
           new DescribeAclsRequest.Builder(AclBindingFilter.ANY)

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/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 70c340b..054a4ff 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -1340,21 +1340,19 @@ object TestUtils extends Logging {
   }
 
   // Seeds the given topic with records with keys and values in the range [0..numRecords)
-  def seedTopicWithNumberedRecords(topic: String, numRecords: Int, servers: Seq[KafkaServer]): Int = {
+  def seedTopicWithNumberedRecords(topic: String, numRecords: Int, servers: Seq[KafkaServer]): Unit = {
     val props = new Properties()
     props.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true")
-    var recordsWritten = 0
-    val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers), retries = Integer.MAX_VALUE, acks = -1, props = Some(props))
+    val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers),
+      retries = Integer.MAX_VALUE, acks = -1, props = Some(props))
     try {
       for (i <- 0 until numRecords) {
         producer.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, asBytes(i.toString), asBytes(i.toString)))
-        recordsWritten += 1
       }
       producer.flush()
     } finally {
       producer.close()
     }
-    recordsWritten
   }
 
   private def asString(bytes: Array[Byte]) = new String(bytes, StandardCharsets.UTF_8)
@@ -1404,7 +1402,7 @@ object TestUtils extends Logging {
     offsetsToCommit.toMap
   }
 
-  def pollUntilAtLeastNumRecords(consumer: KafkaConsumer[Array[Byte], Array[Byte]], numRecords: Int) : Seq[ConsumerRecord[Array[Byte], Array[Byte]]] = {
+  def pollUntilAtLeastNumRecords(consumer: KafkaConsumer[Array[Byte], Array[Byte]], numRecords: Int): Seq[ConsumerRecord[Array[Byte], Array[Byte]]] = {
     val records = new ArrayBuffer[ConsumerRecord[Array[Byte], Array[Byte]]]()
     TestUtils.waitUntilTrue(() => {
       records ++= consumer.poll(50)


[2/5] kafka git commit: KAFKA-5259; TransactionalId auth implies ProducerId auth

Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala
deleted file mode 100644
index 0a8209e..0000000
--- a/core/src/test/scala/unit/kafka/coordinator/group/GroupCoordinatorResponseTest.scala
+++ /dev/null
@@ -1,1492 +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.group
-
-import kafka.common.OffsetAndMetadata
-import kafka.server.{DelayedOperationPurgatory, KafkaConfig, ReplicaManager}
-import kafka.utils._
-import kafka.utils.timer.MockTimer
-import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.protocol.Errors
-import org.apache.kafka.common.record.{MemoryRecords, RecordBatch}
-import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
-import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest, OffsetFetchResponse, TransactionResult}
-import org.easymock.{Capture, EasyMock, IAnswer}
-import java.util.concurrent.TimeUnit
-
-import org.apache.kafka.common.internals.Topic
-import org.junit.Assert._
-import org.junit.{After, Assert, Before, Test}
-import org.scalatest.junit.JUnitSuite
-
-import scala.collection._
-import scala.concurrent.duration.Duration
-import scala.concurrent.{Await, Future, Promise, TimeoutException}
-
-class GroupCoordinatorResponseTest extends JUnitSuite {
-  type JoinGroupCallback = JoinGroupResult => Unit
-  type SyncGroupCallbackParams = (Array[Byte], Errors)
-  type SyncGroupCallback = (Array[Byte], Errors) => Unit
-  type HeartbeatCallbackParams = Errors
-  type HeartbeatCallback = Errors => Unit
-  type CommitOffsetCallbackParams = Map[TopicPartition, Errors]
-  type CommitOffsetCallback = Map[TopicPartition, Errors] => Unit
-  type LeaveGroupCallbackParams = Errors
-  type LeaveGroupCallback = Errors => Unit
-
-  val ClientId = "consumer-test"
-  val ClientHost = "localhost"
-  val ConsumerMinSessionTimeout = 10
-  val ConsumerMaxSessionTimeout = 1000
-  val DefaultRebalanceTimeout = 500
-  val DefaultSessionTimeout = 500
-  val GroupInitialRebalanceDelay = 50
-  var timer: MockTimer = null
-  var groupCoordinator: GroupCoordinator = null
-  var replicaManager: ReplicaManager = null
-  var scheduler: KafkaScheduler = null
-  var zkUtils: ZkUtils = null
-
-  private val groupId = "groupId"
-  private val protocolType = "consumer"
-  private val memberId = "memberId"
-  private val metadata = Array[Byte]()
-  private val protocols = List(("range", metadata))
-  private var groupPartitionId: Int = -1
-
-  // we use this string value since its hashcode % #.partitions is different
-  private val otherGroupId = "otherGroup"
-
-  @Before
-  def setUp() {
-    val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
-    props.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, ConsumerMinSessionTimeout.toString)
-    props.setProperty(KafkaConfig.GroupMaxSessionTimeoutMsProp, ConsumerMaxSessionTimeout.toString)
-    props.setProperty(KafkaConfig.GroupInitialRebalanceDelayMsProp, GroupInitialRebalanceDelay.toString)
-    // make two partitions of the group topic to make sure some partitions are not owned by the coordinator
-    val ret = mutable.Map[String, Map[Int, Seq[Int]]]()
-    ret += (Topic.GROUP_METADATA_TOPIC_NAME -> Map(0 -> Seq(1), 1 -> Seq(1)))
-
-    replicaManager = EasyMock.createNiceMock(classOf[ReplicaManager])
-
-    zkUtils = EasyMock.createNiceMock(classOf[ZkUtils])
-    // make two partitions of the group topic to make sure some partitions are not owned by the coordinator
-    EasyMock.expect(zkUtils.getTopicPartitionCount(Topic.GROUP_METADATA_TOPIC_NAME)).andReturn(Some(2))
-    EasyMock.replay(zkUtils)
-
-    timer = new MockTimer
-
-    val config = KafkaConfig.fromProps(props)
-
-    val heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", timer, config.brokerId, reaperEnabled = false)
-    val joinPurgatory = new DelayedOperationPurgatory[DelayedJoin]("Rebalance", timer, config.brokerId, reaperEnabled = false)
-
-    groupCoordinator = GroupCoordinator(config, zkUtils, replicaManager, heartbeatPurgatory, joinPurgatory, timer.time)
-    groupCoordinator.startup(false)
-
-    // add the partition into the owned partition list
-    groupPartitionId = groupCoordinator.partitionFor(groupId)
-    groupCoordinator.groupManager.addPartitionOwnership(groupPartitionId)
-  }
-
-  @After
-  def tearDown() {
-    EasyMock.reset(replicaManager)
-    if (groupCoordinator != null)
-      groupCoordinator.shutdown()
-  }
-
-  @Test
-  def testOffsetsRetentionMsIntegerOverflow() {
-    val props = TestUtils.createBrokerConfig(nodeId = 0, zkConnect = "")
-    props.setProperty(KafkaConfig.OffsetsRetentionMinutesProp, Integer.MAX_VALUE.toString)
-    val config = KafkaConfig.fromProps(props)
-    val offsetConfig = GroupCoordinator.offsetConfig(config)
-    assertEquals(offsetConfig.offsetsRetentionMs, Integer.MAX_VALUE * 60L * 1000L)
-  }
-
-  @Test
-  def testJoinGroupWrongCoordinator() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val joinGroupResult = joinGroup(otherGroupId, memberId, protocolType, protocols)
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.NOT_COORDINATOR, joinGroupError)
-  }
-
-  @Test
-  def testJoinGroupSessionTimeoutTooSmall() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols, sessionTimeout = ConsumerMinSessionTimeout - 1)
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.INVALID_SESSION_TIMEOUT, joinGroupError)
-  }
-
-  @Test
-  def testJoinGroupSessionTimeoutTooLarge() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols, sessionTimeout = ConsumerMaxSessionTimeout + 1)
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.INVALID_SESSION_TIMEOUT, joinGroupError)
-  }
-
-  @Test
-  def testJoinGroupUnknownConsumerNewGroup() {
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.UNKNOWN_MEMBER_ID, joinGroupError)
-  }
-
-  @Test
-  def testInvalidGroupId() {
-    val groupId = ""
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    assertEquals(Errors.INVALID_GROUP_ID, joinGroupResult.error)
-  }
-
-  @Test
-  def testValidJoinGroup() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.NONE, joinGroupError)
-  }
-
-  @Test
-  def testJoinGroupInconsistentProtocolType() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-    val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    assertEquals(Errors.NONE, joinGroupResult.error)
-
-    EasyMock.reset(replicaManager)
-    val otherJoinGroupResult = await(sendJoinGroup(groupId, otherMemberId, "connect", protocols), 1)
-    assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL, otherJoinGroupResult.error)
-  }
-
-  @Test
-  def testJoinGroupInconsistentGroupProtocol() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val joinGroupFuture = sendJoinGroup(groupId, memberId, protocolType, List(("range", metadata)))
-
-    EasyMock.reset(replicaManager)
-    val otherJoinGroupResult = joinGroup(groupId, otherMemberId, protocolType, List(("roundrobin", metadata)))
-
-    val joinGroupResult = await(joinGroupFuture, 1)
-    assertEquals(Errors.NONE, joinGroupResult.error)
-    assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL, otherJoinGroupResult.error)
-  }
-
-  @Test
-  def testJoinGroupUnknownConsumerExistingGroup() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-    val otherMemberId = "memberId"
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    assertEquals(Errors.NONE, joinGroupResult.error)
-
-    EasyMock.reset(replicaManager)
-    val otherJoinGroupResult = await(sendJoinGroup(groupId, otherMemberId, protocolType, protocols), 1)
-    assertEquals(Errors.UNKNOWN_MEMBER_ID, otherJoinGroupResult.error)
-  }
-
-  @Test
-  def testHeartbeatWrongCoordinator() {
-
-    val heartbeatResult = heartbeat(otherGroupId, memberId, -1)
-    assertEquals(Errors.NOT_COORDINATOR, heartbeatResult)
-  }
-
-  @Test
-  def testHeartbeatUnknownGroup() {
-
-    val heartbeatResult = heartbeat(groupId, memberId, -1)
-    assertEquals(Errors.UNKNOWN_MEMBER_ID, heartbeatResult)
-  }
-
-  @Test
-  def testHeartbeatUnknownConsumerExistingGroup() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-    val otherMemberId = "memberId"
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    val assignedMemberId = joinGroupResult.memberId
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.NONE, joinGroupError)
-
-    EasyMock.reset(replicaManager)
-    val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]()))
-    val syncGroupError = syncGroupResult._2
-    assertEquals(Errors.NONE, syncGroupError)
-
-    EasyMock.reset(replicaManager)
-    val heartbeatResult = heartbeat(groupId, otherMemberId, 1)
-    assertEquals(Errors.UNKNOWN_MEMBER_ID, heartbeatResult)
-  }
-
-  @Test
-  def testHeartbeatRebalanceInProgress() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    val assignedMemberId = joinGroupResult.memberId
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.NONE, joinGroupError)
-
-    EasyMock.reset(replicaManager)
-    val heartbeatResult = heartbeat(groupId, assignedMemberId, 2)
-    assertEquals(Errors.REBALANCE_IN_PROGRESS, heartbeatResult)
-  }
-
-  @Test
-  def testHeartbeatIllegalGeneration() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    val assignedMemberId = joinGroupResult.memberId
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.NONE, joinGroupError)
-
-    EasyMock.reset(replicaManager)
-    val syncGroupResult = syncGroupLeader(groupId, joinGroupResult.generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]()))
-    val syncGroupError = syncGroupResult._2
-    assertEquals(Errors.NONE, syncGroupError)
-
-    EasyMock.reset(replicaManager)
-    val heartbeatResult = heartbeat(groupId, assignedMemberId, 2)
-    assertEquals(Errors.ILLEGAL_GENERATION, heartbeatResult)
-  }
-
-  @Test
-  def testValidHeartbeat() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    val assignedConsumerId = joinGroupResult.memberId
-    val generationId = joinGroupResult.generationId
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.NONE, joinGroupError)
-
-    EasyMock.reset(replicaManager)
-    val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
-    val syncGroupError = syncGroupResult._2
-    assertEquals(Errors.NONE, syncGroupError)
-
-    EasyMock.reset(replicaManager)
-    val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
-    assertEquals(Errors.NONE, heartbeatResult)
-  }
-
-  @Test
-  def testSessionTimeout() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    val assignedConsumerId = joinGroupResult.memberId
-    val generationId = joinGroupResult.generationId
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.NONE, joinGroupError)
-
-    EasyMock.reset(replicaManager)
-    val (_, syncGroupError) = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
-    assertEquals(Errors.NONE, syncGroupError)
-
-    EasyMock.reset(replicaManager)
-    EasyMock.expect(replicaManager.getPartition(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId))).andReturn(None)
-    EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(RecordBatch.MAGIC_VALUE_V1)).anyTimes()
-    EasyMock.replay(replicaManager)
-
-    timer.advanceClock(DefaultSessionTimeout + 100)
-
-    EasyMock.reset(replicaManager)
-    val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
-    assertEquals(Errors.UNKNOWN_MEMBER_ID, heartbeatResult)
-  }
-
-  @Test
-  def testHeartbeatMaintainsSession() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-    val sessionTimeout = 1000
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols,
-      rebalanceTimeout = sessionTimeout, sessionTimeout = sessionTimeout)
-    val assignedConsumerId = joinGroupResult.memberId
-    val generationId = joinGroupResult.generationId
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.NONE, joinGroupError)
-
-    EasyMock.reset(replicaManager)
-    val (_, syncGroupError) = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
-    assertEquals(Errors.NONE, syncGroupError)
-
-    timer.advanceClock(sessionTimeout / 2)
-
-    EasyMock.reset(replicaManager)
-    var heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
-    assertEquals(Errors.NONE, heartbeatResult)
-
-    timer.advanceClock(sessionTimeout / 2 + 100)
-
-    EasyMock.reset(replicaManager)
-    heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
-    assertEquals(Errors.NONE, heartbeatResult)
-  }
-
-  @Test
-  def testCommitMaintainsSession() {
-    val sessionTimeout = 1000
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-    val tp = new TopicPartition("topic", 0)
-    val offset = OffsetAndMetadata(0)
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols,
-      rebalanceTimeout = sessionTimeout, sessionTimeout = sessionTimeout)
-    val assignedConsumerId = joinGroupResult.memberId
-    val generationId = joinGroupResult.generationId
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.NONE, joinGroupError)
-
-    EasyMock.reset(replicaManager)
-    val (_, syncGroupError) = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
-    assertEquals(Errors.NONE, syncGroupError)
-
-    timer.advanceClock(sessionTimeout / 2)
-
-    EasyMock.reset(replicaManager)
-    val commitOffsetResult = commitOffsets(groupId, assignedConsumerId, generationId, immutable.Map(tp -> offset))
-    assertEquals(Errors.NONE, commitOffsetResult(tp))
-
-    timer.advanceClock(sessionTimeout / 2 + 100)
-
-    EasyMock.reset(replicaManager)
-    val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
-    assertEquals(Errors.NONE, heartbeatResult)
-  }
-
-  @Test
-  def testSessionTimeoutDuringRebalance() {
-    // create a group with a single member
-    val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols,
-      rebalanceTimeout = 2000, sessionTimeout = 1000)
-    val firstMemberId = firstJoinResult.memberId
-    val firstGenerationId = firstJoinResult.generationId
-    assertEquals(firstMemberId, firstJoinResult.leaderId)
-    assertEquals(Errors.NONE, firstJoinResult.error)
-
-    EasyMock.reset(replicaManager)
-    val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()))
-    assertEquals(Errors.NONE, firstSyncResult._2)
-
-    // now have a new member join to trigger a rebalance
-    EasyMock.reset(replicaManager)
-    val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
-
-    timer.advanceClock(500)
-
-    EasyMock.reset(replicaManager)
-    var heartbeatResult = heartbeat(groupId, firstMemberId, firstGenerationId)
-    assertEquals(Errors.REBALANCE_IN_PROGRESS, heartbeatResult)
-
-    // letting the session expire should make the member fall out of the group
-    timer.advanceClock(1100)
-
-    EasyMock.reset(replicaManager)
-    heartbeatResult = heartbeat(groupId, firstMemberId, firstGenerationId)
-    assertEquals(Errors.UNKNOWN_MEMBER_ID, heartbeatResult)
-
-    // and the rebalance should complete with only the new member
-    val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100)
-    assertEquals(Errors.NONE, otherJoinResult.error)
-  }
-
-  @Test
-  def testRebalanceCompletesBeforeMemberJoins() {
-    // create a group with a single member
-    val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols,
-      rebalanceTimeout = 1200, sessionTimeout = 1000)
-    val firstMemberId = firstJoinResult.memberId
-    val firstGenerationId = firstJoinResult.generationId
-    assertEquals(firstMemberId, firstJoinResult.leaderId)
-    assertEquals(Errors.NONE, firstJoinResult.error)
-
-    EasyMock.reset(replicaManager)
-    val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()))
-    assertEquals(Errors.NONE, firstSyncResult._2)
-
-    // now have a new member join to trigger a rebalance
-    EasyMock.reset(replicaManager)
-    val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
-
-    // send a couple heartbeats to keep the member alive while the rebalance finishes
-    timer.advanceClock(500)
-    EasyMock.reset(replicaManager)
-    var heartbeatResult = heartbeat(groupId, firstMemberId, firstGenerationId)
-    assertEquals(Errors.REBALANCE_IN_PROGRESS, heartbeatResult)
-
-    timer.advanceClock(500)
-    EasyMock.reset(replicaManager)
-    heartbeatResult = heartbeat(groupId, firstMemberId, firstGenerationId)
-    assertEquals(Errors.REBALANCE_IN_PROGRESS, heartbeatResult)
-
-    // now timeout the rebalance, which should kick the unjoined member out of the group
-    // and let the rebalance finish with only the new member
-    timer.advanceClock(500)
-    val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100)
-    assertEquals(Errors.NONE, otherJoinResult.error)
-  }
-
-  @Test
-  def testSyncGroupEmptyAssignment() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    val assignedConsumerId = joinGroupResult.memberId
-    val generationId = joinGroupResult.generationId
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.NONE, joinGroupError)
-
-    EasyMock.reset(replicaManager)
-    val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map())
-    val syncGroupError = syncGroupResult._2
-    assertEquals(Errors.NONE, syncGroupError)
-    assertTrue(syncGroupResult._1.isEmpty)
-
-    EasyMock.reset(replicaManager)
-    val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
-    assertEquals(Errors.NONE, heartbeatResult)
-  }
-
-  @Test
-  def testSyncGroupNotCoordinator() {
-    val generation = 1
-
-    val syncGroupResult = syncGroupFollower(otherGroupId, generation, memberId)
-    assertEquals(Errors.NOT_COORDINATOR, syncGroupResult._2)
-  }
-
-  @Test
-  def testSyncGroupFromUnknownGroup() {
-    val generation = 1
-
-    val syncGroupResult = syncGroupFollower(groupId, generation, memberId)
-    assertEquals(Errors.UNKNOWN_MEMBER_ID, syncGroupResult._2)
-  }
-
-  @Test
-  def testSyncGroupFromUnknownMember() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    val assignedConsumerId = joinGroupResult.memberId
-    val generationId = joinGroupResult.generationId
-    assertEquals(Errors.NONE, joinGroupResult.error)
-
-    EasyMock.reset(replicaManager)
-    val syncGroupResult = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
-    val syncGroupError = syncGroupResult._2
-    assertEquals(Errors.NONE, syncGroupError)
-
-    EasyMock.reset(replicaManager)
-    val unknownMemberId = "blah"
-    val unknownMemberSyncResult = syncGroupFollower(groupId, generationId, unknownMemberId)
-    assertEquals(Errors.UNKNOWN_MEMBER_ID, unknownMemberSyncResult._2)
-  }
-
-  @Test
-  def testSyncGroupFromIllegalGeneration() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    val assignedConsumerId = joinGroupResult.memberId
-    val generationId = joinGroupResult.generationId
-    assertEquals(Errors.NONE, joinGroupResult.error)
-
-    EasyMock.reset(replicaManager)
-    // send the sync group with an invalid generation
-    val syncGroupResult = syncGroupLeader(groupId, generationId+1, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
-    assertEquals(Errors.ILLEGAL_GENERATION, syncGroupResult._2)
-  }
-
-  @Test
-  def testJoinGroupFromUnchangedFollowerDoesNotRebalance() {
-    // 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, protocolType, protocols)
-    val firstMemberId = firstJoinResult.memberId
-    val firstGenerationId = firstJoinResult.generationId
-    assertEquals(firstMemberId, firstJoinResult.leaderId)
-    assertEquals(Errors.NONE, firstJoinResult.error)
-
-    EasyMock.reset(replicaManager)
-    val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()))
-    assertEquals(Errors.NONE, firstSyncResult._2)
-
-    EasyMock.reset(replicaManager)
-    val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
-
-    EasyMock.reset(replicaManager)
-    val joinFuture = sendJoinGroup(groupId, firstMemberId, protocolType, protocols)
-
-    val joinResult = await(joinFuture, DefaultSessionTimeout+100)
-    val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100)
-    assertEquals(Errors.NONE, joinResult.error)
-    assertEquals(Errors.NONE, otherJoinResult.error)
-    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(replicaManager)
-    val followerJoinResult = await(sendJoinGroup(groupId, otherJoinResult.memberId, protocolType, protocols), 1)
-
-    assertEquals(Errors.NONE, followerJoinResult.error)
-    assertEquals(nextGenerationId, followerJoinResult.generationId)
-  }
-
-  @Test
-  def testJoinGroupFromUnchangedLeaderShouldRebalance() {
-    val firstJoinResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
-    val firstMemberId = firstJoinResult.memberId
-    val firstGenerationId = firstJoinResult.generationId
-    assertEquals(firstMemberId, firstJoinResult.leaderId)
-    assertEquals(Errors.NONE, firstJoinResult.error)
-
-    EasyMock.reset(replicaManager)
-    val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()))
-    assertEquals(Errors.NONE, 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(replicaManager)
-    val secondJoinResult = await(sendJoinGroup(groupId, firstMemberId, protocolType, protocols), 1)
-
-    assertEquals(Errors.NONE, secondJoinResult.error)
-    assertNotEquals(firstGenerationId, secondJoinResult.generationId)
-  }
-
-  @Test
-  def testLeaderFailureInSyncGroup() {
-    // 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, protocolType, protocols)
-    val firstMemberId = firstJoinResult.memberId
-    val firstGenerationId = firstJoinResult.generationId
-    assertEquals(firstMemberId, firstJoinResult.leaderId)
-    assertEquals(Errors.NONE, firstJoinResult.error)
-
-    EasyMock.reset(replicaManager)
-    val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()))
-    assertEquals(Errors.NONE, firstSyncResult._2)
-
-    EasyMock.reset(replicaManager)
-    val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
-
-    EasyMock.reset(replicaManager)
-    val joinFuture = sendJoinGroup(groupId, firstMemberId, protocolType, protocols)
-
-    val joinResult = await(joinFuture, DefaultSessionTimeout+100)
-    val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100)
-    assertEquals(Errors.NONE, joinResult.error)
-    assertEquals(Errors.NONE, otherJoinResult.error)
-    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(replicaManager)
-    val followerSyncFuture = sendSyncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId)
-
-    timer.advanceClock(DefaultSessionTimeout + 100)
-
-    val followerSyncResult = await(followerSyncFuture, DefaultSessionTimeout+100)
-    assertEquals(Errors.REBALANCE_IN_PROGRESS, followerSyncResult._2)
-  }
-
-  @Test
-  def testSyncGroupFollowerAfterLeader() {
-    // 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, protocolType, protocols)
-    val firstMemberId = firstJoinResult.memberId
-    val firstGenerationId = firstJoinResult.generationId
-    assertEquals(firstMemberId, firstJoinResult.leaderId)
-    assertEquals(Errors.NONE, firstJoinResult.error)
-
-    EasyMock.reset(replicaManager)
-    val firstSyncResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()))
-    assertEquals(Errors.NONE, firstSyncResult._2)
-
-    EasyMock.reset(replicaManager)
-    val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
-
-    EasyMock.reset(replicaManager)
-    val joinFuture = sendJoinGroup(groupId, firstMemberId, protocolType, protocols)
-
-    val joinResult = await(joinFuture, DefaultSessionTimeout+100)
-    val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100)
-    assertEquals(Errors.NONE, joinResult.error)
-    assertEquals(Errors.NONE, otherJoinResult.error)
-    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(replicaManager)
-    val leaderSyncResult = syncGroupLeader(groupId, nextGenerationId, leaderId,
-      Map(leaderId -> leaderAssignment, followerId -> followerAssignment))
-    assertEquals(Errors.NONE, leaderSyncResult._2)
-    assertEquals(leaderAssignment, leaderSyncResult._1)
-
-    EasyMock.reset(replicaManager)
-    val followerSyncResult = syncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId)
-    assertEquals(Errors.NONE, followerSyncResult._2)
-    assertEquals(followerAssignment, followerSyncResult._1)
-  }
-
-  @Test
-  def testSyncGroupLeaderAfterFollower() {
-    // 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, protocolType, protocols)
-    val firstMemberId = joinGroupResult.memberId
-    val firstGenerationId = joinGroupResult.generationId
-    assertEquals(firstMemberId, joinGroupResult.leaderId)
-    assertEquals(Errors.NONE, joinGroupResult.error)
-
-    EasyMock.reset(replicaManager)
-    val syncGroupResult = syncGroupLeader(groupId, firstGenerationId, firstMemberId, Map(firstMemberId -> Array[Byte]()))
-    val syncGroupError = syncGroupResult._2
-    assertEquals(Errors.NONE, syncGroupError)
-
-    EasyMock.reset(replicaManager)
-    val otherJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
-
-    EasyMock.reset(replicaManager)
-    val joinFuture = sendJoinGroup(groupId, firstMemberId, protocolType, protocols)
-
-    val joinResult = await(joinFuture, DefaultSessionTimeout+100)
-    val otherJoinResult = await(otherJoinFuture, DefaultSessionTimeout+100)
-    assertEquals(Errors.NONE, joinResult.error)
-    assertEquals(Errors.NONE, otherJoinResult.error)
-    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(replicaManager)
-    val followerSyncFuture = sendSyncGroupFollower(groupId, nextGenerationId, followerId)
-
-    EasyMock.reset(replicaManager)
-    val leaderSyncResult = syncGroupLeader(groupId, nextGenerationId, leaderId,
-      Map(leaderId -> leaderAssignment, followerId -> followerAssignment))
-    assertEquals(Errors.NONE, leaderSyncResult._2)
-    assertEquals(leaderAssignment, leaderSyncResult._1)
-
-    val followerSyncResult = await(followerSyncFuture, DefaultSessionTimeout+100)
-    assertEquals(Errors.NONE, followerSyncResult._2)
-    assertEquals(followerAssignment, followerSyncResult._1)
-  }
-
-  @Test
-  def testCommitOffsetFromUnknownGroup() {
-    val generationId = 1
-    val tp = new TopicPartition("topic", 0)
-    val offset = OffsetAndMetadata(0)
-
-    val commitOffsetResult = commitOffsets(groupId, memberId, generationId, immutable.Map(tp -> offset))
-    assertEquals(Errors.ILLEGAL_GENERATION, commitOffsetResult(tp))
-  }
-
-  @Test
-  def testCommitOffsetWithDefaultGeneration() {
-    val tp = new TopicPartition("topic", 0)
-    val offset = OffsetAndMetadata(0)
-
-    val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID,
-      OffsetCommitRequest.DEFAULT_GENERATION_ID, immutable.Map(tp -> offset))
-    assertEquals(Errors.NONE, commitOffsetResult(tp))
-  }
-
-  @Test
-  def testFetchOffsets() {
-    val tp = new TopicPartition("topic", 0)
-    val offset = OffsetAndMetadata(0)
-
-    val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID,
-      OffsetCommitRequest.DEFAULT_GENERATION_ID, immutable.Map(tp -> offset))
-    assertEquals(Errors.NONE, commitOffsetResult(tp))
-
-    val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
-    assertEquals(Errors.NONE, error)
-    assertEquals(Some(0), partitionData.get(tp).map(_.offset))
-  }
-
-  @Test
-  def testBasicFetchTxnOffsets() {
-    val tp = new TopicPartition("topic", 0)
-    val offset = OffsetAndMetadata(0)
-    val producerId = 1000L
-    val producerEpoch : Short = 2
-
-    val commitOffsetResult = commitTransactionalOffsets(groupId, producerId, producerEpoch, immutable.Map(tp -> offset))
-    assertEquals(Errors.NONE, commitOffsetResult(tp))
-
-    val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
-
-    // Validate that the offset isn't materialjzed yet.
-    assertEquals(Errors.NONE, error)
-    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData.get(tp).map(_.offset))
-
-    val offsetsTopic = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId)
-
-    // Send commit marker.
-    groupCoordinator.handleTxnCompletion(producerId, List(offsetsTopic), TransactionResult.COMMIT)
-
-    // Validate that committed offset is materialized.
-    val (secondReqError, secondReqPartitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
-    assertEquals(Errors.NONE, secondReqError)
-    assertEquals(Some(0), secondReqPartitionData.get(tp).map(_.offset))
-  }
-
-  @Test
-  def testFetchTxnOffsetsWithAbort() {
-    val tp = new TopicPartition("topic", 0)
-    val offset = OffsetAndMetadata(0)
-    val producerId = 1000L
-    val producerEpoch : Short = 2
-
-    val commitOffsetResult = commitTransactionalOffsets(groupId, producerId, producerEpoch, immutable.Map(tp -> offset))
-    assertEquals(Errors.NONE, commitOffsetResult(tp))
-
-    val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
-    assertEquals(Errors.NONE, error)
-    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData.get(tp).map(_.offset))
-
-    val offsetsTopic = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId)
-
-    // Validate that the pending commit is discarded.
-    groupCoordinator.handleTxnCompletion(producerId, List(offsetsTopic), TransactionResult.ABORT)
-
-    val (secondReqError, secondReqPartitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
-    assertEquals(Errors.NONE, secondReqError)
-    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), secondReqPartitionData.get(tp).map(_.offset))
-  }
-
-  @Test
-  def testFetchTxnOffsetsIgnoreSpuriousCommit() {
-    val tp = new TopicPartition("topic", 0)
-    val offset = OffsetAndMetadata(0)
-    val producerId = 1000L
-    val producerEpoch : Short = 2
-
-    val commitOffsetResult = commitTransactionalOffsets(groupId, producerId, producerEpoch, immutable.Map(tp -> offset))
-    assertEquals(Errors.NONE, commitOffsetResult(tp))
-
-    val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
-    assertEquals(Errors.NONE, error)
-    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData.get(tp).map(_.offset))
-
-    val offsetsTopic = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId)
-    groupCoordinator.handleTxnCompletion(producerId, List(offsetsTopic), TransactionResult.ABORT)
-
-    val (secondReqError, secondReqPartitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
-    assertEquals(Errors.NONE, secondReqError)
-    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), secondReqPartitionData.get(tp).map(_.offset))
-
-    // Ignore spurious commit.
-    groupCoordinator.handleTxnCompletion(producerId, List(offsetsTopic), TransactionResult.COMMIT)
-
-    val (thirdReqError, thirdReqPartitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
-    assertEquals(Errors.NONE, secondReqError)
-    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), thirdReqPartitionData.get(tp).map(_.offset))
-  }
-
-  @Test
-  def testFetchTxnOffsetsOneProducerMultipleGroups() {
-    // One producer, two groups located on separate offsets topic partitions.
-    // Both group have pending offset commits.
-    // Marker for only one partition is received. That commit should be materialized while the other should not.
-
-    val partitions = List(new TopicPartition("topic1", 0), new TopicPartition("topic2", 0))
-    val offsets = List(OffsetAndMetadata(10), OffsetAndMetadata(15))
-    val producerId = 1000L
-    val producerEpoch: Short = 3
-
-    val groupIds = List(groupId, otherGroupId)
-    val offsetTopicPartitions = List(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupCoordinator.partitionFor(groupId)),
-      new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupCoordinator.partitionFor(otherGroupId)))
-
-    groupCoordinator.groupManager.addPartitionOwnership(offsetTopicPartitions(1).partition)
-    val errors = mutable.ArrayBuffer[Errors]()
-    val partitionData = mutable.ArrayBuffer[Map[TopicPartition, OffsetFetchResponse.PartitionData]]()
-
-    val commitOffsetResults = mutable.ArrayBuffer[CommitOffsetCallbackParams]()
-
-    // Ensure that the two groups map to different partitions.
-    assertNotEquals(offsetTopicPartitions(0), offsetTopicPartitions(1))
-
-    commitOffsetResults.append(commitTransactionalOffsets(groupId, producerId, producerEpoch, immutable.Map(partitions(0) -> offsets(0))))
-    assertEquals(Errors.NONE, commitOffsetResults(0)(partitions(0)))
-    commitOffsetResults.append(commitTransactionalOffsets(otherGroupId, producerId, producerEpoch, immutable.Map(partitions(1) -> offsets(1))))
-    assertEquals(Errors.NONE, commitOffsetResults(1)(partitions(1)))
-
-    // We got a commit for only one __consumer_offsets partition. We should only materialize it's group offsets.
-    groupCoordinator.handleTxnCompletion(producerId, List(offsetTopicPartitions(0)), TransactionResult.COMMIT)
-    groupCoordinator.handleFetchOffsets(groupIds(0), Some(partitions)) match {
-      case (error, partData) =>
-        errors.append(error)
-        partitionData.append(partData)
-      case _ =>
-    }
-
-     groupCoordinator.handleFetchOffsets(groupIds(1), Some(partitions)) match {
-      case (error, partData) =>
-        errors.append(error)
-        partitionData.append(partData)
-      case _ =>
-    }
-
-    assertEquals(2, errors.size)
-    assertEquals(Errors.NONE, errors(0))
-    assertEquals(Errors.NONE, errors(1))
-
-    // Exactly one offset commit should have been materialized.
-    assertEquals(Some(offsets(0).offset), partitionData(0).get(partitions(0)).map(_.offset))
-    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData(0).get(partitions(1)).map(_.offset))
-    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData(1).get(partitions(0)).map(_.offset))
-    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData(1).get(partitions(1)).map(_.offset))
-
-    // Now we receive the other marker.
-    groupCoordinator.handleTxnCompletion(producerId, List(offsetTopicPartitions(1)), TransactionResult.COMMIT)
-    errors.clear()
-    partitionData.clear()
-    groupCoordinator.handleFetchOffsets(groupIds(0), Some(partitions)) match {
-      case (error, partData) =>
-        errors.append(error)
-        partitionData.append(partData)
-      case _ =>
-    }
-
-     groupCoordinator.handleFetchOffsets(groupIds(1), Some(partitions)) match {
-      case (error, partData) =>
-        errors.append(error)
-        partitionData.append(partData)
-      case _ =>
-    }
-    // Two offsets should have been materialized
-    assertEquals(Some(offsets(0).offset), partitionData(0).get(partitions(0)).map(_.offset))
-    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData(0).get(partitions(1)).map(_.offset))
-    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData(1).get(partitions(0)).map(_.offset))
-    assertEquals(Some(offsets(1).offset), partitionData(1).get(partitions(1)).map(_.offset))
-  }
-
-  @Test
-  def testFetchTxnOffsetsMultipleProducersOneGroup() {
-    // One group, two producers
-    // Different producers will commit offsets for different partitions.
-    // Each partition's offsets should be materialized when the corresponding producer's marker is received.
-
-    val partitions = List(new TopicPartition("topic1", 0), new TopicPartition("topic2", 0))
-    val offsets = List(OffsetAndMetadata(10), OffsetAndMetadata(15))
-    val producerIds = List(1000L, 1005L)
-    val producerEpochs: Seq[Short] = List(3, 4)
-
-    val offsetTopicPartition = new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupCoordinator.partitionFor(groupId))
-
-    val errors = mutable.ArrayBuffer[Errors]()
-    val partitionData = mutable.ArrayBuffer[Map[TopicPartition, OffsetFetchResponse.PartitionData]]()
-
-    val commitOffsetResults = mutable.ArrayBuffer[CommitOffsetCallbackParams]()
-
-    // producer0 commits the offsets for partition0
-    commitOffsetResults.append(commitTransactionalOffsets(groupId, producerIds(0), producerEpochs(0), immutable.Map(partitions(0) -> offsets(0))))
-    assertEquals(Errors.NONE, commitOffsetResults(0)(partitions(0)))
-
-    // producer1 commits the offsets for partition1
-    commitOffsetResults.append(commitTransactionalOffsets(groupId, producerIds(1), producerEpochs(1), immutable.Map(partitions(1) -> offsets(1))))
-    assertEquals(Errors.NONE, commitOffsetResults(1)(partitions(1)))
-
-    // producer0 commits its transaction.
-    groupCoordinator.handleTxnCompletion(producerIds(0), List(offsetTopicPartition), TransactionResult.COMMIT)
-    groupCoordinator.handleFetchOffsets(groupId, Some(partitions)) match {
-      case (error, partData) =>
-        errors.append(error)
-        partitionData.append(partData)
-      case _ =>
-    }
-
-    assertEquals(Errors.NONE, errors(0))
-
-    // We should only see the offset commit for producer0
-    assertEquals(Some(offsets(0).offset), partitionData(0).get(partitions(0)).map(_.offset))
-    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData(0).get(partitions(1)).map(_.offset))
-
-    // producer1 now commits its transaction.
-    groupCoordinator.handleTxnCompletion(producerIds(1), List(offsetTopicPartition), TransactionResult.COMMIT)
-
-    groupCoordinator.handleFetchOffsets(groupId, Some(partitions)) match {
-      case (error, partData) =>
-        errors.append(error)
-        partitionData.append(partData)
-      case _ =>
-    }
-
-    assertEquals(Errors.NONE, errors(1))
-
-    // We should now see the offset commits for both producers.
-    assertEquals(Some(offsets(0).offset), partitionData(1).get(partitions(0)).map(_.offset))
-    assertEquals(Some(offsets(1).offset), partitionData(1).get(partitions(1)).map(_.offset))
-  }
-
-  @Test
-  def testFetchOffsetForUnknownPartition(): Unit = {
-    val tp = new TopicPartition("topic", 0)
-    val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp)))
-    assertEquals(Errors.NONE, error)
-    assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData.get(tp).map(_.offset))
-  }
-
-  @Test
-  def testFetchOffsetNotCoordinatorForGroup(): Unit = {
-    val tp = new TopicPartition("topic", 0)
-    val (error, partitionData) = groupCoordinator.handleFetchOffsets(otherGroupId, Some(Seq(tp)))
-    assertEquals(Errors.NOT_COORDINATOR, error)
-    assertTrue(partitionData.isEmpty)
-  }
-
-  @Test
-  def testFetchAllOffsets() {
-    val tp1 = new TopicPartition("topic", 0)
-    val tp2 = new TopicPartition("topic", 1)
-    val tp3 = new TopicPartition("other-topic", 0)
-    val offset1 = OffsetAndMetadata(15)
-    val offset2 = OffsetAndMetadata(16)
-    val offset3 = OffsetAndMetadata(17)
-
-    assertEquals((Errors.NONE, Map.empty), groupCoordinator.handleFetchOffsets(groupId))
-
-    val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID,
-      OffsetCommitRequest.DEFAULT_GENERATION_ID, immutable.Map(tp1 -> offset1, tp2 -> offset2, tp3 -> offset3))
-    assertEquals(Errors.NONE, commitOffsetResult(tp1))
-    assertEquals(Errors.NONE, commitOffsetResult(tp2))
-    assertEquals(Errors.NONE, commitOffsetResult(tp3))
-
-    val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId)
-    assertEquals(Errors.NONE, error)
-    assertEquals(3, partitionData.size)
-    assertTrue(partitionData.forall(_._2.error == Errors.NONE))
-    assertEquals(Some(offset1.offset), partitionData.get(tp1).map(_.offset))
-    assertEquals(Some(offset2.offset), partitionData.get(tp2).map(_.offset))
-    assertEquals(Some(offset3.offset), partitionData.get(tp3).map(_.offset))
-  }
-
-  @Test
-  def testCommitOffsetInAwaitingSync() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-    val tp = new TopicPartition("topic", 0)
-    val offset = OffsetAndMetadata(0)
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    val assignedMemberId = joinGroupResult.memberId
-    val generationId = joinGroupResult.generationId
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.NONE, joinGroupError)
-
-    EasyMock.reset(replicaManager)
-    val commitOffsetResult = commitOffsets(groupId, assignedMemberId, generationId, immutable.Map(tp -> offset))
-    assertEquals(Errors.REBALANCE_IN_PROGRESS, commitOffsetResult(tp))
-  }
-
-  @Test
-  def testHeartbeatDuringRebalanceCausesRebalanceInProgress() {
-    // 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, protocolType, protocols)
-    val assignedConsumerId = joinGroupResult.memberId
-    val initialGenerationId = joinGroupResult.generationId
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.NONE, joinGroupError)
-
-    // Then join with a new consumer to trigger a rebalance
-    EasyMock.reset(replicaManager)
-    sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
-
-    // We should be in the middle of a rebalance, so the heartbeat should return rebalance in progress
-    EasyMock.reset(replicaManager)
-    val heartbeatResult = heartbeat(groupId, assignedConsumerId, initialGenerationId)
-    assertEquals(Errors.REBALANCE_IN_PROGRESS, heartbeatResult)
-  }
-
-  @Test
-  def testGenerationIdIncrementsOnRebalance() {
-    val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
-    val initialGenerationId = joinGroupResult.generationId
-    val joinGroupError = joinGroupResult.error
-    val memberId = joinGroupResult.memberId
-    assertEquals(1, initialGenerationId)
-    assertEquals(Errors.NONE, joinGroupError)
-
-    EasyMock.reset(replicaManager)
-    val syncGroupResult = syncGroupLeader(groupId, initialGenerationId, memberId, Map(memberId -> Array[Byte]()))
-    val syncGroupError = syncGroupResult._2
-    assertEquals(Errors.NONE, syncGroupError)
-
-    EasyMock.reset(replicaManager)
-    val joinGroupFuture = sendJoinGroup(groupId, memberId, protocolType, protocols)
-    val otherJoinGroupResult = await(joinGroupFuture, 1)
-
-    val nextGenerationId = otherJoinGroupResult.generationId
-    val otherJoinGroupError = otherJoinGroupResult.error
-    assertEquals(2, nextGenerationId)
-    assertEquals(Errors.NONE, otherJoinGroupError)
-  }
-
-  @Test
-  def testLeaveGroupWrongCoordinator() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val leaveGroupResult = leaveGroup(otherGroupId, memberId)
-    assertEquals(Errors.NOT_COORDINATOR, leaveGroupResult)
-  }
-
-  @Test
-  def testLeaveGroupUnknownGroup() {
-
-    val leaveGroupResult = leaveGroup(groupId, memberId)
-    assertEquals(Errors.UNKNOWN_MEMBER_ID, leaveGroupResult)
-  }
-
-  @Test
-  def testLeaveGroupUnknownConsumerExistingGroup() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-    val otherMemberId = "consumerId"
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.NONE, joinGroupError)
-
-    EasyMock.reset(replicaManager)
-    val leaveGroupResult = leaveGroup(groupId, otherMemberId)
-    assertEquals(Errors.UNKNOWN_MEMBER_ID, leaveGroupResult)
-  }
-
-  @Test
-  def testValidLeaveGroup() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    val assignedMemberId = joinGroupResult.memberId
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.NONE, joinGroupError)
-
-    EasyMock.reset(replicaManager)
-    val leaveGroupResult = leaveGroup(groupId, assignedMemberId)
-    assertEquals(Errors.NONE, leaveGroupResult)
-  }
-
-  @Test
-  def testListGroupsIncludesStableGroups() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    val assignedMemberId = joinGroupResult.memberId
-    val generationId = joinGroupResult.generationId
-    assertEquals(Errors.NONE, joinGroupResult.error)
-
-    EasyMock.reset(replicaManager)
-    val syncGroupResult = syncGroupLeader(groupId, generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]()))
-    val syncGroupError = syncGroupResult._2
-    assertEquals(Errors.NONE, syncGroupError)
-
-    val (error, groups) = groupCoordinator.handleListGroups()
-    assertEquals(Errors.NONE, error)
-    assertEquals(1, groups.size)
-    assertEquals(GroupOverview("groupId", "consumer"), groups.head)
-  }
-
-  @Test
-  def testListGroupsIncludesRebalancingGroups() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    assertEquals(Errors.NONE, joinGroupResult.error)
-
-    val (error, groups) = groupCoordinator.handleListGroups()
-    assertEquals(Errors.NONE, error)
-    assertEquals(1, groups.size)
-    assertEquals(GroupOverview("groupId", "consumer"), groups.head)
-  }
-
-  @Test
-  def testDescribeGroupWrongCoordinator() {
-    EasyMock.reset(replicaManager)
-    val (error, _) = groupCoordinator.handleDescribeGroup(otherGroupId)
-    assertEquals(Errors.NOT_COORDINATOR, error)
-  }
-
-  @Test
-  def testDescribeGroupInactiveGroup() {
-    EasyMock.reset(replicaManager)
-    val (error, summary) = groupCoordinator.handleDescribeGroup(groupId)
-    assertEquals(Errors.NONE, error)
-    assertEquals(GroupCoordinator.DeadGroup, summary)
-  }
-
-  @Test
-  def testDescribeGroupStable() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    val assignedMemberId = joinGroupResult.memberId
-    val generationId = joinGroupResult.generationId
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.NONE, joinGroupError)
-
-    EasyMock.reset(replicaManager)
-    val syncGroupResult = syncGroupLeader(groupId, generationId, assignedMemberId, Map(assignedMemberId -> Array[Byte]()))
-
-    val syncGroupError = syncGroupResult._2
-    assertEquals(Errors.NONE, syncGroupError)
-
-    EasyMock.reset(replicaManager)
-    val (error, summary) = groupCoordinator.handleDescribeGroup(groupId)
-    assertEquals(Errors.NONE, error)
-    assertEquals(protocolType, summary.protocolType)
-    assertEquals("range", summary.protocol)
-    assertEquals(List(assignedMemberId), summary.members.map(_.memberId))
-  }
-
-  @Test
-  def testDescribeGroupRebalancing() {
-    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
-    val joinGroupResult = joinGroup(groupId, memberId, protocolType, protocols)
-    val joinGroupError = joinGroupResult.error
-    assertEquals(Errors.NONE, joinGroupError)
-
-    EasyMock.reset(replicaManager)
-    val (error, summary) = groupCoordinator.handleDescribeGroup(groupId)
-    assertEquals(Errors.NONE, error)
-    assertEquals(protocolType, summary.protocolType)
-    assertEquals(GroupCoordinator.NoProtocol, summary.protocol)
-    assertEquals(AwaitingSync.toString, summary.state)
-    assertTrue(summary.members.map(_.memberId).contains(joinGroupResult.memberId))
-    assertTrue(summary.members.forall(_.metadata.isEmpty))
-    assertTrue(summary.members.forall(_.assignment.isEmpty))
-  }
-
-  @Test
-  def shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() {
-    val firstJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols)
-    timer.advanceClock(GroupInitialRebalanceDelay / 2)
-    verifyDelayedTaskNotCompleted(firstJoinFuture)
-    timer.advanceClock((GroupInitialRebalanceDelay / 2) + 1)
-    val joinGroupResult = await(firstJoinFuture, 1)
-    assertEquals(Errors.NONE, joinGroupResult.error)
-  }
-
-  private def verifyDelayedTaskNotCompleted(firstJoinFuture: Future[JoinGroupResult]) = {
-    try {
-      await(firstJoinFuture, 1)
-      Assert.fail("should have timed out as rebalance delay not expired")
-    } catch {
-      case _: TimeoutException => // ok
-    }
-  }
-
-  @Test
-  def shouldResetRebalanceDelayWhenNewMemberJoinsGroupInInitialRebalance() {
-    val rebalanceTimeout = GroupInitialRebalanceDelay * 3
-    val firstMemberJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols, rebalanceTimeout)
-    EasyMock.reset(replicaManager)
-    timer.advanceClock(GroupInitialRebalanceDelay - 1)
-    val secondMemberJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols, rebalanceTimeout)
-    EasyMock.reset(replicaManager)
-    timer.advanceClock(2)
-
-    // advance past initial rebalance delay and make sure that tasks
-    // haven't been completed
-    timer.advanceClock(GroupInitialRebalanceDelay / 2 + 1)
-    verifyDelayedTaskNotCompleted(firstMemberJoinFuture)
-    verifyDelayedTaskNotCompleted(secondMemberJoinFuture)
-    // advance clock beyond updated delay and make sure the
-    // tasks have completed
-    timer.advanceClock(GroupInitialRebalanceDelay / 2)
-    val firstResult = await(firstMemberJoinFuture, 1)
-    val secondResult = await(secondMemberJoinFuture, 1)
-    assertEquals(Errors.NONE, firstResult.error)
-    assertEquals(Errors.NONE, secondResult.error)
-  }
-
-  @Test
-  def shouldDelayRebalanceUptoRebalanceTimeout() {
-    val rebalanceTimeout = GroupInitialRebalanceDelay * 2
-    val firstMemberJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols, rebalanceTimeout)
-    EasyMock.reset(replicaManager)
-    val secondMemberJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols, rebalanceTimeout)
-    timer.advanceClock(GroupInitialRebalanceDelay + 1)
-    EasyMock.reset(replicaManager)
-    val thirdMemberJoinFuture = sendJoinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, protocolType, protocols, rebalanceTimeout)
-    timer.advanceClock(GroupInitialRebalanceDelay)
-    EasyMock.reset(replicaManager)
-
-    verifyDelayedTaskNotCompleted(firstMemberJoinFuture)
-    verifyDelayedTaskNotCompleted(secondMemberJoinFuture)
-    verifyDelayedTaskNotCompleted(thirdMemberJoinFuture)
-
-    // advance clock beyond rebalanceTimeout
-    timer.advanceClock(1)
-
-    val firstResult = await(firstMemberJoinFuture, 1)
-    val secondResult = await(secondMemberJoinFuture, 1)
-    val thirdResult = await(thirdMemberJoinFuture, 1)
-    assertEquals(Errors.NONE, firstResult.error)
-    assertEquals(Errors.NONE, secondResult.error)
-    assertEquals(Errors.NONE, thirdResult.error)
-  }
-
-  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, error) =>
-      responsePromise.success((assignment, error))
-    (responseFuture, responseCallback)
-  }
-
-  private def setupHeartbeatCallback: (Future[HeartbeatCallbackParams], HeartbeatCallback) = {
-    val responsePromise = Promise[HeartbeatCallbackParams]
-    val responseFuture = responsePromise.future
-    val responseCallback: HeartbeatCallback = error => responsePromise.success(error)
-    (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,
-                            protocolType: String,
-                            protocols: List[(String, Array[Byte])],
-                            rebalanceTimeout: Int = DefaultRebalanceTimeout,
-                            sessionTimeout: Int = DefaultSessionTimeout): Future[JoinGroupResult] = {
-    val (responseFuture, responseCallback) = setupJoinGroupCallback
-
-    EasyMock.replay(replicaManager)
-
-    groupCoordinator.handleJoinGroup(groupId, memberId, "clientId", "clientHost", rebalanceTimeout, sessionTimeout,
-      protocolType, protocols, responseCallback)
-    responseFuture
-  }
-
-
-  private def sendSyncGroupLeader(groupId: String,
-                                  generation: Int,
-                                  leaderId: String,
-                                  assignment: Map[String, Array[Byte]]): Future[SyncGroupCallbackParams] = {
-    val (responseFuture, responseCallback) = setupSyncGroupCallback
-
-    val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
-
-    EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
-      EasyMock.anyShort(),
-      internalTopicsAllowed = EasyMock.eq(true),
-      isFromClient = EasyMock.eq(false),
-      EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
-      EasyMock.capture(capturedArgument),
-      EasyMock.anyObject().asInstanceOf[Option[Object]])).andAnswer(new IAnswer[Unit] {
-      override def answer = capturedArgument.getValue.apply(
-        Map(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId) ->
-          new PartitionResponse(Errors.NONE, 0L, RecordBatch.NO_TIMESTAMP)
-        )
-      )})
-    EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(RecordBatch.MAGIC_VALUE_V1)).anyTimes()
-    EasyMock.replay(replicaManager)
-
-    groupCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback)
-    responseFuture
-  }
-
-  private def sendSyncGroupFollower(groupId: String,
-                                    generation: Int,
-                                    memberId: String): Future[SyncGroupCallbackParams] = {
-    val (responseFuture, responseCallback) = setupSyncGroupCallback
-
-    EasyMock.replay(replicaManager)
-
-    groupCoordinator.handleSyncGroup(groupId, generation, memberId, Map.empty[String, Array[Byte]], responseCallback)
-    responseFuture
-  }
-
-  private def joinGroup(groupId: String,
-                        memberId: String,
-                        protocolType: String,
-                        protocols: List[(String, Array[Byte])],
-                        sessionTimeout: Int = DefaultSessionTimeout,
-                        rebalanceTimeout: Int = DefaultRebalanceTimeout): JoinGroupResult = {
-    val responseFuture = sendJoinGroup(groupId, memberId, protocolType, protocols, rebalanceTimeout, sessionTimeout)
-    timer.advanceClock(GroupInitialRebalanceDelay + 1)
-    // 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(rebalanceTimeout + 100, TimeUnit.MILLISECONDS))
-  }
-
-
-  private def syncGroupFollower(groupId: String,
-                                generationId: Int,
-                                memberId: String,
-                                sessionTimeout: Int = DefaultSessionTimeout): SyncGroupCallbackParams = {
-    val responseFuture = sendSyncGroupFollower(groupId, generationId, memberId)
-    Await.result(responseFuture, Duration(sessionTimeout + 100, TimeUnit.MILLISECONDS))
-  }
-
-  private def syncGroupLeader(groupId: String,
-                              generationId: Int,
-                              memberId: String,
-                              assignment: Map[String, Array[Byte]],
-                              sessionTimeout: Int = DefaultSessionTimeout): SyncGroupCallbackParams = {
-    val responseFuture = sendSyncGroupLeader(groupId, generationId, memberId, assignment)
-    Await.result(responseFuture, Duration(sessionTimeout + 100, TimeUnit.MILLISECONDS))
-  }
-
-  private def heartbeat(groupId: String,
-                        consumerId: String,
-                        generationId: Int): HeartbeatCallbackParams = {
-    val (responseFuture, responseCallback) = setupHeartbeatCallback
-
-    EasyMock.replay(replicaManager)
-
-    groupCoordinator.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: immutable.Map[TopicPartition, OffsetAndMetadata]): CommitOffsetCallbackParams = {
-    val (responseFuture, responseCallback) = setupCommitOffsetsCallback
-
-    val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
-
-    EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
-      EasyMock.anyShort(),
-      internalTopicsAllowed = EasyMock.eq(true),
-      isFromClient = EasyMock.eq(false),
-      EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
-      EasyMock.capture(capturedArgument),
-      EasyMock.anyObject().asInstanceOf[Option[Object]])
-    ).andAnswer(new IAnswer[Unit] {
-      override def answer = capturedArgument.getValue.apply(
-        Map(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId) ->
-          new PartitionResponse(Errors.NONE, 0L, RecordBatch.NO_TIMESTAMP)
-        )
-      )})
-    EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(RecordBatch.MAGIC_VALUE_V1)).anyTimes()
-    EasyMock.replay(replicaManager)
-
-    groupCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback)
-    Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS))
-  }
-
-  private def commitTransactionalOffsets(groupId: String,
-                                         producerId: Long,
-                                         producerEpoch: Short,
-                                         offsets: immutable.Map[TopicPartition, OffsetAndMetadata]): CommitOffsetCallbackParams = {
-    val (responseFuture, responseCallback) = setupCommitOffsetsCallback
-
-    val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
-
-    EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
-      EasyMock.anyShort(),
-      internalTopicsAllowed = EasyMock.eq(true),
-      isFromClient = EasyMock.eq(false),
-      EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
-      EasyMock.capture(capturedArgument),
-      EasyMock.anyObject().asInstanceOf[Option[Object]])
-    ).andAnswer(new IAnswer[Unit] {
-      override def answer = capturedArgument.getValue.apply(
-        Map(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupCoordinator.partitionFor(groupId)) ->
-          new PartitionResponse(Errors.NONE, 0L, RecordBatch.NO_TIMESTAMP)
-        )
-      )})
-    EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(RecordBatch.MAGIC_VALUE_V2)).anyTimes()
-    EasyMock.replay(replicaManager)
-
-    groupCoordinator.handleTxnCommitOffsets(groupId, producerId, producerEpoch, offsets, responseCallback)
-    val result = Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS))
-    EasyMock.reset(replicaManager)
-    result
-  }
-
-  private def leaveGroup(groupId: String, consumerId: String): LeaveGroupCallbackParams = {
-    val (responseFuture, responseCallback) = setupHeartbeatCallback
-
-    EasyMock.expect(replicaManager.getPartition(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, groupPartitionId))).andReturn(None)
-    EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(RecordBatch.MAGIC_VALUE_V1)).anyTimes()
-    EasyMock.replay(replicaManager)
-
-    groupCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback)
-    Await.result(responseFuture, Duration(40, TimeUnit.MILLISECONDS))
-  }
-
-}


[4/5] kafka git commit: KAFKA-5259; TransactionalId auth implies ProducerId auth

Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
index cc30f4d..719efe9 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java
@@ -25,6 +25,7 @@ import org.apache.kafka.common.Cluster;
 import org.apache.kafka.common.MetricName;
 import org.apache.kafka.common.Node;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.ClusterAuthorizationException;
 import org.apache.kafka.common.internals.ClusterResourceListeners;
 import org.apache.kafka.common.metrics.KafkaMetric;
 import org.apache.kafka.common.metrics.MetricConfig;
@@ -231,20 +232,8 @@ public class SenderTest {
         int maxRetries = 1;
         Metrics m = new Metrics();
         try {
-            Sender sender = new Sender(client,
-                    metadata,
-                    this.accumulator,
-                    false,
-                    MAX_REQUEST_SIZE,
-                    ACKS_ALL,
-                    maxRetries,
-                    m,
-                    time,
-                    REQUEST_TIMEOUT,
-                    50,
-                    null,
-                    apiVersions
-            );
+            Sender sender = new Sender(client, metadata, this.accumulator, false, MAX_REQUEST_SIZE, ACKS_ALL,
+                    maxRetries, m, time, REQUEST_TIMEOUT, 50, null, apiVersions);
             // do a successful retry
             Future<RecordMetadata> future = accumulator.append(tp0, 0L, "key".getBytes(), "value".getBytes(), null, null, MAX_BLOCK_TIMEOUT).future;
             sender.run(time.milliseconds()); // connect
@@ -290,20 +279,8 @@ public class SenderTest {
         int maxRetries = 1;
         Metrics m = new Metrics();
         try {
-            Sender sender = new Sender(client,
-                    metadata,
-                    this.accumulator,
-                    true,
-                    MAX_REQUEST_SIZE,
-                    ACKS_ALL,
-                    maxRetries,
-                    m,
-                    time,
-                    REQUEST_TIMEOUT,
-                    50,
-                    null,
-                    apiVersions
-            );
+            Sender sender = new Sender(client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries,
+                    m, time, REQUEST_TIMEOUT, 50, null, apiVersions);
             // Create a two broker cluster, with partition 0 on broker 0 and partition 1 on broker 1
             Cluster cluster1 = TestUtils.clusterWith(2, "test", 2);
             metadata.update(cluster1, Collections.<String>emptySet(), time.milliseconds());
@@ -375,21 +352,63 @@ public class SenderTest {
     }
 
     @Test
-    public void testInitPidRequest() throws Exception {
+    public void testInitProducerIdRequest() throws Exception {
         final long producerId = 343434L;
         TransactionManager transactionManager = new TransactionManager();
         setupWithTransactionState(transactionManager);
         client.setNode(new Node(1, "localhost", 33343));
+        prepareAndReceiveInitProducerId(producerId, Errors.NONE);
+        assertTrue(transactionManager.hasProducerId());
+        assertEquals(producerId, transactionManager.producerIdAndEpoch().producerId);
+        assertEquals((short) 0, transactionManager.producerIdAndEpoch().epoch);
+    }
+
+    @Test
+    public void testClusterAuthorizationExceptionInInitProducerIdRequest() throws Exception {
+        final long producerId = 343434L;
+        TransactionManager transactionManager = new TransactionManager();
+        setupWithTransactionState(transactionManager);
+        client.setNode(new Node(1, "localhost", 33343));
+        prepareAndReceiveInitProducerId(producerId, Errors.CLUSTER_AUTHORIZATION_FAILED);
+        assertFalse(transactionManager.hasProducerId());
+        assertTrue(transactionManager.isInErrorState());
+        assertTrue(transactionManager.lastError() instanceof ClusterAuthorizationException);
+
+        // cluster authorization is a fatal error for the producer
+        assertSendFailure(ClusterAuthorizationException.class);
+    }
+
+    @Test
+    public void testClusterAuthorizationExceptionInProduceRequest() throws Exception {
+        final long producerId = 343434L;
+        TransactionManager transactionManager = new TransactionManager();
+        setupWithTransactionState(transactionManager);
+
+        client.setNode(new Node(1, "localhost", 33343));
+        prepareAndReceiveInitProducerId(producerId, Errors.NONE);
+        assertTrue(transactionManager.hasProducerId());
+
+        // cluster authorization is a fatal error for the producer
+        Future<RecordMetadata> future = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), "value".getBytes(),
+                null, null, MAX_BLOCK_TIMEOUT).future;
         client.prepareResponse(new MockClient.RequestMatcher() {
             @Override
             public boolean matches(AbstractRequest body) {
-                return body instanceof InitProducerIdRequest;
+                return body instanceof ProduceRequest && ((ProduceRequest) body).isIdempotent();
             }
-        }, new InitProducerIdResponse(0, Errors.NONE, producerId, (short) 0));
+        }, produceResponse(tp0, -1, Errors.CLUSTER_AUTHORIZATION_FAILED, 0));
+
         sender.run(time.milliseconds());
-        assertTrue(transactionManager.hasProducerId());
-        assertEquals(producerId, transactionManager.producerIdAndEpoch().producerId);
-        assertEquals((short) 0, transactionManager.producerIdAndEpoch().epoch);
+        assertTrue(future.isDone());
+        try {
+            future.get();
+            fail("Future should have raised ClusterAuthorizationException");
+        } catch (ExecutionException e) {
+            assertTrue(e.getCause() instanceof ClusterAuthorizationException);
+        }
+
+        // cluster authorization is a fatal error for the producer
+        assertSendFailure(ClusterAuthorizationException.class);
     }
 
     @Test
@@ -402,20 +421,8 @@ public class SenderTest {
 
         int maxRetries = 10;
         Metrics m = new Metrics();
-        Sender sender = new Sender(client,
-                metadata,
-                this.accumulator,
-                true,
-                MAX_REQUEST_SIZE,
-                ACKS_ALL,
-                maxRetries,
-                m,
-                time,
-                REQUEST_TIMEOUT,
-                50,
-                transactionManager,
-                apiVersions
-        );
+        Sender sender = new Sender(client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries,
+                m, time, REQUEST_TIMEOUT, 50, transactionManager, apiVersions);
 
         Future<RecordMetadata> responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), "value".getBytes(), null, null, MAX_BLOCK_TIMEOUT).future;
         client.prepareResponse(new MockClient.RequestMatcher() {
@@ -446,7 +453,7 @@ public class SenderTest {
     }
 
     @Test
-    public void testAbortRetryWhenPidChanges() throws InterruptedException {
+    public void testAbortRetryWhenProducerIdChanges() throws InterruptedException {
         final long producerId = 343434L;
         TransactionManager transactionManager = new TransactionManager();
         transactionManager.setProducerIdAndEpoch(new ProducerIdAndEpoch(producerId, (short) 0));
@@ -455,20 +462,8 @@ public class SenderTest {
 
         int maxRetries = 10;
         Metrics m = new Metrics();
-        Sender sender = new Sender(client,
-                metadata,
-                this.accumulator,
-                true,
-                MAX_REQUEST_SIZE,
-                ACKS_ALL,
-                maxRetries,
-                m,
-                time,
-                REQUEST_TIMEOUT,
-                50,
-                transactionManager,
-                apiVersions
-        );
+        Sender sender = new Sender(client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries,
+                m, time, REQUEST_TIMEOUT, 50, transactionManager, apiVersions);
 
         Future<RecordMetadata> responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), "value".getBytes(), null, null, MAX_BLOCK_TIMEOUT).future;
         sender.run(time.milliseconds());  // connect.
@@ -504,20 +499,8 @@ public class SenderTest {
 
         int maxRetries = 10;
         Metrics m = new Metrics();
-        Sender sender = new Sender(client,
-                metadata,
-                this.accumulator,
-                true,
-                MAX_REQUEST_SIZE,
-                ACKS_ALL,
-                maxRetries,
-                m,
-                time,
-                REQUEST_TIMEOUT,
-                50,
-                transactionManager,
-                apiVersions
-        );
+        Sender sender = new Sender(client, metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries,
+                m, time, REQUEST_TIMEOUT, 50, transactionManager, apiVersions);
 
         Future<RecordMetadata> responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), "value".getBytes(), null, null, MAX_BLOCK_TIMEOUT).future;
         sender.run(time.milliseconds());  // connect.
@@ -642,20 +625,38 @@ public class SenderTest {
         metricTags.put("client-id", CLIENT_ID);
         MetricConfig metricConfig = new MetricConfig().tags(metricTags);
         this.metrics = new Metrics(metricConfig, time);
-        this.accumulator = new RecordAccumulator(batchSize, 1024 * 1024, CompressionType.NONE, 0L, 0L, metrics, time, apiVersions, transactionManager);
-        this.sender = new Sender(this.client,
-                this.metadata,
-                this.accumulator,
-                true,
-                MAX_REQUEST_SIZE,
-                ACKS_ALL,
-                MAX_RETRIES,
-                this.metrics,
-                this.time,
-                REQUEST_TIMEOUT,
-                50,
-                transactionManager,
-                apiVersions);
+        this.accumulator = new RecordAccumulator(batchSize, 1024 * 1024, CompressionType.NONE, 0L, 0L, metrics, time,
+                apiVersions, transactionManager);
+        this.sender = new Sender(this.client, this.metadata, this.accumulator, true, MAX_REQUEST_SIZE, ACKS_ALL,
+                MAX_RETRIES, this.metrics, this.time, REQUEST_TIMEOUT, 50, transactionManager, apiVersions);
         this.metadata.update(this.cluster, Collections.<String>emptySet(), time.milliseconds());
     }
+
+    private void assertSendFailure(Class<? extends RuntimeException> expectedError) throws Exception {
+        Future<RecordMetadata> future = accumulator.append(tp0, time.milliseconds(), "key".getBytes(), "value".getBytes(),
+                null, null, MAX_BLOCK_TIMEOUT).future;
+        sender.run(time.milliseconds());
+        assertTrue(future.isDone());
+        try {
+            future.get();
+            fail("Future should have raised " + expectedError.getSimpleName());
+        } catch (ExecutionException e) {
+            assertTrue(expectedError.isAssignableFrom(e.getCause().getClass()));
+        }
+    }
+
+    private void prepareAndReceiveInitProducerId(long producerId, Errors error) {
+        short producerEpoch = 0;
+        if (error != Errors.NONE)
+            producerEpoch = RecordBatch.NO_PRODUCER_EPOCH;
+
+        client.prepareResponse(new MockClient.RequestMatcher() {
+            @Override
+            public boolean matches(AbstractRequest body) {
+                return body instanceof InitProducerIdRequest && ((InitProducerIdRequest) body).transactionalId() == null;
+            }
+        }, new InitProducerIdResponse(0, error, producerId, producerEpoch));
+        sender.run(time.milliseconds());
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java
index fcf0488..e9363d0 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/TransactionManagerTest.java
@@ -20,12 +20,14 @@ import org.apache.kafka.clients.ApiVersions;
 import org.apache.kafka.clients.Metadata;
 import org.apache.kafka.clients.MockClient;
 import org.apache.kafka.clients.consumer.OffsetAndMetadata;
-import org.apache.kafka.clients.producer.Callback;
 import org.apache.kafka.clients.producer.RecordMetadata;
 import org.apache.kafka.common.Cluster;
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.Node;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.GroupAuthorizationException;
+import org.apache.kafka.common.errors.TopicAuthorizationException;
+import org.apache.kafka.common.errors.TransactionalIdAuthorizationException;
 import org.apache.kafka.common.internals.ClusterResourceListeners;
 import org.apache.kafka.common.metrics.MetricConfig;
 import org.apache.kafka.common.metrics.Metrics;
@@ -43,6 +45,7 @@ import org.apache.kafka.common.requests.AddPartitionsToTxnResponse;
 import org.apache.kafka.common.requests.EndTxnRequest;
 import org.apache.kafka.common.requests.EndTxnResponse;
 import org.apache.kafka.common.requests.FindCoordinatorRequest;
+import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType;
 import org.apache.kafka.common.requests.FindCoordinatorResponse;
 import org.apache.kafka.common.requests.InitProducerIdRequest;
 import org.apache.kafka.common.requests.InitProducerIdResponse;
@@ -65,6 +68,8 @@ import java.util.Map;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 
+import static java.util.Collections.singleton;
+import static java.util.Collections.singletonMap;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -118,6 +123,7 @@ public class TransactionManagerTest {
                 transactionManager,
                 apiVersions);
         this.metadata.update(this.cluster, Collections.<String>emptySet(), time.milliseconds());
+        client.setNode(brokerNode);
     }
 
     @Test(expected = IllegalStateException.class)
@@ -134,7 +140,6 @@ public class TransactionManagerTest {
         assertEquals((int) transactionManager.sequenceNumber(tp0), 3);
     }
 
-
     @Test
     public void testProducerIdReset() {
         TransactionManager transactionManager = new TransactionManager();
@@ -147,23 +152,13 @@ public class TransactionManagerTest {
 
     @Test
     public void testBasicTransaction() throws InterruptedException {
-        client.setNode(brokerNode);
         // This is called from the initTransactions method in the producer as the first order of business.
         // It finds the coordinator and then gets a PID.
         final long pid = 13131L;
         final short epoch = 1;
-        transactionManager.initializeTransactions();
-        prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId);
 
-        sender.run(time.milliseconds());  // find coordinator
-        sender.run(time.milliseconds());
-        assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION));
-
-        prepareInitPidResponse(Errors.NONE, false, pid, epoch);
+        doInitTransactions(pid, epoch);
 
-        sender.run(time.milliseconds());  // get pid.
-
-        assertTrue(transactionManager.hasProducerId());
         transactionManager.beginTransaction();
         transactionManager.maybeAddPartitionToTransaction(tp0);
 
@@ -199,13 +194,13 @@ public class TransactionManagerTest {
         Map<TopicPartition, Errors> txnOffsetCommitResponse = new HashMap<>();
         txnOffsetCommitResponse.put(tp1, Errors.NONE);
 
-        prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.GROUP, consumerGroupId);
+        prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.GROUP, consumerGroupId);
         prepareTxnOffsetCommitResponse(consumerGroupId, pid, epoch, txnOffsetCommitResponse);
 
-        assertEquals(null, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.GROUP));
+        assertEquals(null, transactionManager.coordinator(CoordinatorType.GROUP));
         sender.run(time.milliseconds());  // try to send TxnOffsetCommitRequest, but find we don't have a group coordinator.
         sender.run(time.milliseconds());  // send find coordinator for group request
-        assertNotNull(transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.GROUP));
+        assertNotNull(transactionManager.coordinator(CoordinatorType.GROUP));
         assertTrue(transactionManager.hasPendingOffsetCommits());
 
         sender.run(time.milliseconds());  // send TxnOffsetCommitRequest commit.
@@ -224,42 +219,40 @@ public class TransactionManagerTest {
 
     @Test
     public void testDisconnectAndRetry() {
-        client.setNode(brokerNode);
         // This is called from the initTransactions method in the producer as the first order of business.
         // It finds the coordinator and then gets a PID.
         transactionManager.initializeTransactions();
-        prepareFindCoordinatorResponse(Errors.NONE, true, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId);
+        prepareFindCoordinatorResponse(Errors.NONE, true, CoordinatorType.TRANSACTION, transactionalId);
         sender.run(time.milliseconds());  // find coordinator, connection lost.
 
-        prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId);
+        prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId);
         sender.run(time.milliseconds());  // find coordinator
         sender.run(time.milliseconds());
-        assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION));
+        assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION));
     }
 
     @Test
     public void testCoordinatorLost() {
-        client.setNode(brokerNode);
         // This is called from the initTransactions method in the producer as the first order of business.
         // It finds the coordinator and then gets a PID.
         final long pid = 13131L;
         final short epoch = 1;
         TransactionalRequestResult initPidResult = transactionManager.initializeTransactions();
-        prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId);
+        prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId);
         sender.run(time.milliseconds());  // find coordinator
         sender.run(time.milliseconds());
-        assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION));
+        assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION));
 
         prepareInitPidResponse(Errors.NOT_COORDINATOR, false, pid, epoch);
         sender.run(time.milliseconds());  // send pid, get not coordinator. Should resend the FindCoordinator and InitPid requests
 
-        assertEquals(null, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION));
+        assertEquals(null, transactionManager.coordinator(CoordinatorType.TRANSACTION));
         assertFalse(initPidResult.isCompleted());
         assertFalse(transactionManager.hasProducerId());
 
-        prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId);
+        prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId);
         sender.run(time.milliseconds());
-        assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION));
+        assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION));
         assertFalse(initPidResult.isCompleted());
         prepareInitPidResponse(Errors.NONE, false, pid, epoch);
         sender.run(time.milliseconds());  // get pid and epoch
@@ -271,24 +264,216 @@ public class TransactionManagerTest {
     }
 
     @Test
-    public void testFlushPendingPartitionsOnCommit() throws InterruptedException {
-        client.setNode(brokerNode);
-        // This is called from the initTransactions method in the producer as the first order of business.
-        // It finds the coordinator and then gets a PID.
+    public void testTransactionalIdAuthorizationFailureInFindCoordinator() {
+        TransactionalRequestResult initPidResult = transactionManager.initializeTransactions();
+        prepareFindCoordinatorResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED, false,
+                CoordinatorType.TRANSACTION, transactionalId);
+        sender.run(time.milliseconds());  // find coordinator
+        sender.run(time.milliseconds());
+
+        assertTrue(transactionManager.isInErrorState());
+        assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException);
+
+        sender.run(time.milliseconds()); // one more run to fail the InitProducerId future
+        assertTrue(initPidResult.isCompleted());
+        assertFalse(initPidResult.isSuccessful());
+        assertTrue(initPidResult.error() instanceof TransactionalIdAuthorizationException);
+
+        assertFatalError(TransactionalIdAuthorizationException.class);
+    }
+
+    @Test
+    public void testTransactionalIdAuthorizationFailureInInitProducerId() {
+        final long pid = 13131L;
+        TransactionalRequestResult initPidResult = transactionManager.initializeTransactions();
+        prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId);
+        sender.run(time.milliseconds());  // find coordinator
+        sender.run(time.milliseconds());
+        assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION));
+
+        prepareInitPidResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED, false, pid, RecordBatch.NO_PRODUCER_EPOCH);
+        sender.run(time.milliseconds());
+
+        assertTrue(transactionManager.isInErrorState());
+        assertTrue(initPidResult.isCompleted());
+        assertFalse(initPidResult.isSuccessful());
+        assertTrue(initPidResult.error() instanceof TransactionalIdAuthorizationException);
+
+        assertFatalError(TransactionalIdAuthorizationException.class);
+    }
+
+    @Test
+    public void testGroupAuthorizationFailureInFindCoordinator() {
+        final String consumerGroupId = "consumer";
         final long pid = 13131L;
         final short epoch = 1;
-        transactionManager.initializeTransactions();
-        prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId);
 
-        sender.run(time.milliseconds());  // find coordinator
+        doInitTransactions(pid, epoch);
+
+        transactionManager.beginTransaction();
+        TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction(
+                singletonMap(new TopicPartition("foo", 0), new OffsetAndMetadata(39L)), consumerGroupId);
+
+        prepareAddOffsetsToTxnResponse(Errors.NONE, consumerGroupId, pid, epoch);
+        sender.run(time.milliseconds());  // AddOffsetsToTxn Handled, TxnOffsetCommit Enqueued
+        sender.run(time.milliseconds());  // FindCoordinator Enqueued
+
+        prepareFindCoordinatorResponse(Errors.GROUP_AUTHORIZATION_FAILED, false, CoordinatorType.GROUP, consumerGroupId);
+        sender.run(time.milliseconds());  // FindCoordinator Failed
+        sender.run(time.milliseconds());  // TxnOffsetCommit Aborted
+        assertTrue(transactionManager.isInErrorState());
+        assertTrue(transactionManager.lastError() instanceof GroupAuthorizationException);
+        assertTrue(sendOffsetsResult.isCompleted());
+        assertFalse(sendOffsetsResult.isSuccessful());
+        assertTrue(sendOffsetsResult.error() instanceof GroupAuthorizationException);
+
+        GroupAuthorizationException exception = (GroupAuthorizationException) sendOffsetsResult.error();
+        assertEquals(consumerGroupId, exception.groupId());
+
+        assertAbortableError(GroupAuthorizationException.class);
+    }
+
+    @Test
+    public void testGroupAuthorizationFailureInTxnOffsetCommit() {
+        final String consumerGroupId = "consumer";
+        final long pid = 13131L;
+        final short epoch = 1;
+        final TopicPartition tp = new TopicPartition("foo", 0);
+
+        doInitTransactions(pid, epoch);
+
+        transactionManager.beginTransaction();
+        TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction(
+                singletonMap(tp, new OffsetAndMetadata(39L)), consumerGroupId);
+
+        prepareAddOffsetsToTxnResponse(Errors.NONE, consumerGroupId, pid, epoch);
+        sender.run(time.milliseconds());  // AddOffsetsToTxn Handled, TxnOffsetCommit Enqueued
+        sender.run(time.milliseconds());  // FindCoordinator Enqueued
+
+        prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.GROUP, consumerGroupId);
+        sender.run(time.milliseconds());  // FindCoordinator Returned
+
+        prepareTxnOffsetCommitResponse(consumerGroupId, pid, epoch, singletonMap(tp, Errors.GROUP_AUTHORIZATION_FAILED));
+        sender.run(time.milliseconds());  // TxnOffsetCommit Handled
+
+        assertTrue(transactionManager.isInErrorState());
+        assertTrue(transactionManager.lastError() instanceof GroupAuthorizationException);
+        assertTrue(sendOffsetsResult.isCompleted());
+        assertFalse(sendOffsetsResult.isSuccessful());
+        assertTrue(sendOffsetsResult.error() instanceof GroupAuthorizationException);
+
+        GroupAuthorizationException exception = (GroupAuthorizationException) sendOffsetsResult.error();
+        assertEquals(consumerGroupId, exception.groupId());
+
+        assertAbortableError(GroupAuthorizationException.class);
+    }
+
+    @Test
+    public void testTransactionalIdAuthorizationFailureInAddOffsetsToTxn() {
+        final String consumerGroupId = "consumer";
+        final long pid = 13131L;
+        final short epoch = 1;
+        final TopicPartition tp = new TopicPartition("foo", 0);
+
+        doInitTransactions(pid, epoch);
+
+        transactionManager.beginTransaction();
+        TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction(
+                singletonMap(tp, new OffsetAndMetadata(39L)), consumerGroupId);
+
+        prepareAddOffsetsToTxnResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED, consumerGroupId, pid, epoch);
+        sender.run(time.milliseconds());  // AddOffsetsToTxn Handled
+
+        assertTrue(transactionManager.isInErrorState());
+        assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException);
+        assertTrue(sendOffsetsResult.isCompleted());
+        assertFalse(sendOffsetsResult.isSuccessful());
+        assertTrue(sendOffsetsResult.error() instanceof TransactionalIdAuthorizationException);
+
+        assertFatalError(TransactionalIdAuthorizationException.class);
+    }
+
+    @Test
+    public void testTransactionalIdAuthorizationFailureInTxnOffsetCommit() {
+        final String consumerGroupId = "consumer";
+        final long pid = 13131L;
+        final short epoch = 1;
+        final TopicPartition tp = new TopicPartition("foo", 0);
+
+        doInitTransactions(pid, epoch);
+
+        transactionManager.beginTransaction();
+        TransactionalRequestResult sendOffsetsResult = transactionManager.sendOffsetsToTransaction(
+                singletonMap(tp, new OffsetAndMetadata(39L)), consumerGroupId);
+
+        prepareAddOffsetsToTxnResponse(Errors.NONE, consumerGroupId, pid, epoch);
+        sender.run(time.milliseconds());  // AddOffsetsToTxn Handled, TxnOffsetCommit Enqueued
+        sender.run(time.milliseconds());  // FindCoordinator Enqueued
+
+        prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.GROUP, consumerGroupId);
+        sender.run(time.milliseconds());  // FindCoordinator Returned
+
+        prepareTxnOffsetCommitResponse(consumerGroupId, pid, epoch, singletonMap(tp, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED));
+        sender.run(time.milliseconds());  // TxnOffsetCommit Handled
+
+        assertTrue(transactionManager.isInErrorState());
+        assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException);
+        assertTrue(sendOffsetsResult.isCompleted());
+        assertFalse(sendOffsetsResult.isSuccessful());
+        assertTrue(sendOffsetsResult.error() instanceof TransactionalIdAuthorizationException);
+
+        assertFatalError(TransactionalIdAuthorizationException.class);
+    }
+
+    @Test
+    public void testTopicAuthorizationFailureInAddPartitions() {
+        final long pid = 13131L;
+        final short epoch = 1;
+        final TopicPartition tp = new TopicPartition("foo", 0);
+
+        doInitTransactions(pid, epoch);
+
+        transactionManager.beginTransaction();
+        transactionManager.maybeAddPartitionToTransaction(tp);
+
+        prepareAddPartitionsToTxn(tp, Errors.TOPIC_AUTHORIZATION_FAILED);
         sender.run(time.milliseconds());
-        assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION));
 
-        prepareInitPidResponse(Errors.NONE, false, pid, epoch);
+        assertTrue(transactionManager.isInErrorState());
+        assertTrue(transactionManager.lastError() instanceof TopicAuthorizationException);
 
-        sender.run(time.milliseconds());  // get pid.
+        TopicAuthorizationException exception = (TopicAuthorizationException) transactionManager.lastError();
+        assertEquals(singleton(tp.topic()), exception.unauthorizedTopics());
 
-        assertTrue(transactionManager.hasProducerId());
+        assertAbortableError(TopicAuthorizationException.class);
+    }
+
+    @Test
+    public void testTransactionalIdAuthorizationFailureInAddPartitions() {
+        final long pid = 13131L;
+        final short epoch = 1;
+        final TopicPartition tp = new TopicPartition("foo", 0);
+
+        doInitTransactions(pid, epoch);
+
+        transactionManager.beginTransaction();
+        transactionManager.maybeAddPartitionToTransaction(tp);
+
+        prepareAddPartitionsToTxn(tp, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED);
+        sender.run(time.milliseconds());
+
+        assertTrue(transactionManager.isInErrorState());
+        assertTrue(transactionManager.lastError() instanceof TransactionalIdAuthorizationException);
+
+        assertFatalError(TransactionalIdAuthorizationException.class);
+    }
+
+    @Test
+    public void testFlushPendingPartitionsOnCommit() throws InterruptedException {
+        final long pid = 13131L;
+        final short epoch = 1;
+
+        doInitTransactions(pid, epoch);
 
         transactionManager.beginTransaction();
         transactionManager.maybeAddPartitionToTransaction(tp0);
@@ -329,23 +514,11 @@ public class TransactionManagerTest {
 
     @Test
     public void testMultipleAddPartitionsPerForOneProduce() throws InterruptedException {
-        client.setNode(brokerNode);
-        // This is called from the initTransactions method in the producer as the first order of business.
-        // It finds the coordinator and then gets a PID.
         final long pid = 13131L;
         final short epoch = 1;
-        transactionManager.initializeTransactions();
-        prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId);
-
-        sender.run(time.milliseconds());  // find coordinator
-        sender.run(time.milliseconds());
-        assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION));
-
-        prepareInitPidResponse(Errors.NONE, false, pid, epoch);
 
-        sender.run(time.milliseconds());  // get pid.
+        doInitTransactions(pid, epoch);
 
-        assertTrue(transactionManager.hasProducerId());
         transactionManager.beginTransaction();
         // User does one producer.sed
         transactionManager.maybeAddPartitionToTransaction(tp0);
@@ -392,28 +565,16 @@ public class TransactionManagerTest {
 
     @Test(expected = ExecutionException.class)
     public void testProducerFencedException() throws InterruptedException, ExecutionException {
-        client.setNode(brokerNode);
-        // This is called from the initTransactions method in the producer as the first order of business.
-        // It finds the coordinator and then gets a PID.
         final long pid = 13131L;
         final short epoch = 1;
-        transactionManager.initializeTransactions();
-        prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId);
 
-        sender.run(time.milliseconds());  // find coordinator
-        sender.run(time.milliseconds());
-        assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION));
-
-        prepareInitPidResponse(Errors.NONE, false, pid, epoch);
+        doInitTransactions(pid, epoch);
 
-        sender.run(time.milliseconds());  // get pid.
-
-        assertTrue(transactionManager.hasProducerId());
         transactionManager.beginTransaction();
         transactionManager.maybeAddPartitionToTransaction(tp0);
 
         Future<RecordMetadata> responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(),
-                "value".getBytes(), Record.EMPTY_HEADERS, new MockCallback(transactionManager), MAX_BLOCK_TIMEOUT).future;
+                "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future;
 
         assertFalse(responseFuture.isDone());
         prepareAddPartitionsToTxnResponse(Errors.NONE, tp0, epoch, pid);
@@ -429,28 +590,16 @@ public class TransactionManagerTest {
 
     @Test
     public void testDisallowCommitOnProduceFailure() throws InterruptedException {
-        client.setNode(brokerNode);
-        // This is called from the initTransactions method in the producer as the first order of business.
-        // It finds the coordinator and then gets a PID.
         final long pid = 13131L;
         final short epoch = 1;
-        transactionManager.initializeTransactions();
-        prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId);
 
-        sender.run(time.milliseconds());  // find coordinator
-        sender.run(time.milliseconds());
-        assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION));
-
-        prepareInitPidResponse(Errors.NONE, false, pid, epoch);
+        doInitTransactions(pid, epoch);
 
-        sender.run(time.milliseconds());  // get pid.
-
-        assertTrue(transactionManager.hasProducerId());
         transactionManager.beginTransaction();
         transactionManager.maybeAddPartitionToTransaction(tp0);
 
         Future<RecordMetadata> responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(),
-                "value".getBytes(), Record.EMPTY_HEADERS, new MockCallback(transactionManager), MAX_BLOCK_TIMEOUT).future;
+                "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future;
 
         TransactionalRequestResult commitResult = transactionManager.beginCommittingTransaction();
         assertFalse(responseFuture.isDone());
@@ -483,28 +632,16 @@ public class TransactionManagerTest {
 
     @Test
     public void testAllowAbortOnProduceFailure() throws InterruptedException {
-        client.setNode(brokerNode);
-        // This is called from the initTransactions method in the producer as the first order of business.
-        // It finds the coordinator and then gets a PID.
         final long pid = 13131L;
         final short epoch = 1;
-        transactionManager.initializeTransactions();
-        prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId);
-
-        sender.run(time.milliseconds());  // find coordinator
-        sender.run(time.milliseconds());
-
-        prepareInitPidResponse(Errors.NONE, false, pid, epoch);
-        sender.run(time.milliseconds());  // get pid.
 
-        assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION));
-        assertTrue(transactionManager.hasProducerId());
+        doInitTransactions(pid, epoch);
 
         transactionManager.beginTransaction();
         transactionManager.maybeAddPartitionToTransaction(tp0);
 
         Future<RecordMetadata> responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(),
-                "value".getBytes(), Record.EMPTY_HEADERS, new MockCallback(transactionManager), MAX_BLOCK_TIMEOUT).future;
+                "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future;
 
         TransactionalRequestResult abortResult = transactionManager.beginAbortingTransaction();
         assertFalse(responseFuture.isDone());
@@ -524,28 +661,16 @@ public class TransactionManagerTest {
 
     @Test
     public void testHandlingOfUnknownTopicPartitionErrorOnAddPartitions() throws InterruptedException {
-        client.setNode(brokerNode);
-        // This is called from the initTransactions method in the producer as the first order of business.
-        // It finds the coordinator and then gets a PID.
         final long pid = 13131L;
         final short epoch = 1;
-        transactionManager.initializeTransactions();
-        prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId);
 
-        sender.run(time.milliseconds());  // find coordinator
-        sender.run(time.milliseconds());
-        assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION));
-
-        prepareInitPidResponse(Errors.NONE, false, pid, epoch);
-
-        sender.run(time.milliseconds());  // get pid.
+        doInitTransactions(pid, epoch);
 
-        assertTrue(transactionManager.hasProducerId());
         transactionManager.beginTransaction();
         transactionManager.maybeAddPartitionToTransaction(tp0);
 
         Future<RecordMetadata> responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(),
-                "value".getBytes(), Record.EMPTY_HEADERS, new MockCallback(transactionManager), MAX_BLOCK_TIMEOUT).future;
+                "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future;
 
         assertFalse(responseFuture.isDone());
         prepareAddPartitionsToTxnResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION, tp0, epoch, pid);
@@ -564,23 +689,11 @@ public class TransactionManagerTest {
 
     @Test
     public void testHandlingOfUnknownTopicPartitionErrorOnTxnOffsetCommit() throws InterruptedException {
-        client.setNode(brokerNode);
-        // This is called from the initTransactions method in the producer as the first order of business.
-        // It finds the coordinator and then gets a PID.
         final long pid = 13131L;
         final short epoch = 1;
-        transactionManager.initializeTransactions();
-        prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId);
 
-        sender.run(time.milliseconds());  // find coordinator
-        sender.run(time.milliseconds());
-        assertEquals(brokerNode, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.TRANSACTION));
-
-        prepareInitPidResponse(Errors.NONE, false, pid, epoch);
+        doInitTransactions(pid, epoch);
 
-        sender.run(time.milliseconds());  // get pid.
-
-        assertTrue(transactionManager.hasProducerId());
         transactionManager.beginTransaction();
 
         Map<TopicPartition, OffsetAndMetadata> offsets = new HashMap<>();
@@ -597,13 +710,13 @@ public class TransactionManagerTest {
         Map<TopicPartition, Errors> txnOffsetCommitResponse = new HashMap<>();
         txnOffsetCommitResponse.put(tp1, Errors.UNKNOWN_TOPIC_OR_PARTITION);
 
-        prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.GROUP, consumerGroupId);
+        prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.GROUP, consumerGroupId);
         prepareTxnOffsetCommitResponse(consumerGroupId, pid, epoch, txnOffsetCommitResponse);
 
-        assertEquals(null, transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.GROUP));
+        assertEquals(null, transactionManager.coordinator(CoordinatorType.GROUP));
         sender.run(time.milliseconds());  // try to send TxnOffsetCommitRequest, but find we don't have a group coordinator.
         sender.run(time.milliseconds());  // send find coordinator for group request
-        assertNotNull(transactionManager.coordinator(FindCoordinatorRequest.CoordinatorType.GROUP));
+        assertNotNull(transactionManager.coordinator(CoordinatorType.GROUP));
         assertTrue(transactionManager.hasPendingOffsetCommits());
 
         sender.run(time.milliseconds());  // send TxnOffsetCommitRequest request.
@@ -625,58 +738,35 @@ public class TransactionManagerTest {
     }
 
     private void verifyAddPartitionsFailsWithPartitionLevelError(final Errors error) throws InterruptedException {
-        client.setNode(brokerNode);
-        transactionManager.initializeTransactions();
-        prepareFindCoordinatorResponse(Errors.NONE, false, FindCoordinatorRequest.CoordinatorType.TRANSACTION, transactionalId);
-
-        sender.run(time.milliseconds());  // find coordinator
-        sender.run(time.milliseconds());
-
         final long pid = 1L;
         final short epoch = 1;
 
-        prepareInitPidResponse(Errors.NONE, false, pid, epoch);
+        doInitTransactions(pid, epoch);
 
-        sender.run(time.milliseconds());  // get pid.
-
-        assertTrue(transactionManager.hasProducerId());
         transactionManager.beginTransaction();
         transactionManager.maybeAddPartitionToTransaction(tp0);
 
         Future<RecordMetadata> responseFuture = accumulator.append(tp0, time.milliseconds(), "key".getBytes(),
-                                                                   "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future;
+                "value".getBytes(), Record.EMPTY_HEADERS, null, MAX_BLOCK_TIMEOUT).future;
         assertFalse(responseFuture.isDone());
-        prepareAddPartitionsToTxnPartitionErrorResponse(tp0, error);
+        prepareAddPartitionsToTxn(tp0, error);
         sender.run(time.milliseconds());  // attempt send addPartitions.
         assertTrue(transactionManager.isInErrorState());
         assertFalse(transactionManager.transactionContainsPartition(tp0));
     }
 
-    private void prepareAddPartitionsToTxnPartitionErrorResponse(final TopicPartition tp0, final Errors error) {
+    private void prepareAddPartitionsToTxn(final TopicPartition tp, final Errors error) {
         client.prepareResponse(new MockClient.RequestMatcher() {
             @Override
             public boolean matches(AbstractRequest body) {
-                assertTrue(body instanceof AddPartitionsToTxnRequest);
-                return true;
-            }
-        }, new AddPartitionsToTxnResponse(0, Collections.singletonMap(tp0, error)));
-    }
-
-    private static class MockCallback implements Callback {
-        private final TransactionManager transactionManager;
-        public MockCallback(TransactionManager transactionManager) {
-            this.transactionManager = transactionManager;
-        }
-        @Override
-        public void onCompletion(RecordMetadata metadata, Exception exception) {
-            if (exception != null && transactionManager != null) {
-                transactionManager.setError(exception);
+                return body instanceof AddPartitionsToTxnRequest &&
+                        ((AddPartitionsToTxnRequest) body).partitions().contains(tp);
             }
-        }
+        }, new AddPartitionsToTxnResponse(0, singletonMap(tp, error)));
     }
 
     private void prepareFindCoordinatorResponse(Errors error, boolean shouldDisconnect,
-                                                final FindCoordinatorRequest.CoordinatorType coordinatorType,
+                                                final CoordinatorType coordinatorType,
                                                 final String coordinatorKey) {
         client.prepareResponse(new MockClient.RequestMatcher() {
             @Override
@@ -733,7 +823,7 @@ public class TransactionManagerTest {
                 assertEquals(transactionalId, addPartitionsToTxnRequest.transactionalId());
                 return true;
             }
-        }, new AddPartitionsToTxnResponse(0, Collections.singletonMap(topicPartition, error)));
+        }, new AddPartitionsToTxnResponse(0, singletonMap(topicPartition, error)));
     }
 
     private void prepareEndTxnResponse(Errors error, final TransactionResult result, final long pid, final short epoch) {
@@ -782,9 +872,54 @@ public class TransactionManagerTest {
 
     private ProduceResponse produceResponse(TopicPartition tp, long offset, Errors error, int throttleTimeMs) {
         ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse(error, offset, RecordBatch.NO_TIMESTAMP);
-        Map<TopicPartition, ProduceResponse.PartitionResponse> partResp = Collections.singletonMap(tp, resp);
+        Map<TopicPartition, ProduceResponse.PartitionResponse> partResp = singletonMap(tp, resp);
         return new ProduceResponse(partResp, throttleTimeMs);
     }
 
+    private void doInitTransactions(long pid, short epoch) {
+        transactionManager.initializeTransactions();
+        prepareFindCoordinatorResponse(Errors.NONE, false, CoordinatorType.TRANSACTION, transactionalId);
+        sender.run(time.milliseconds());  // find coordinator
+        sender.run(time.milliseconds());
+        assertEquals(brokerNode, transactionManager.coordinator(CoordinatorType.TRANSACTION));
+
+        prepareInitPidResponse(Errors.NONE, false, pid, epoch);
+        sender.run(time.milliseconds());  // get pid.
+        assertTrue(transactionManager.hasProducerId());
+    }
 
+    private void assertAbortableError(Class<? extends RuntimeException> cause) {
+        try {
+            transactionManager.beginTransaction();
+            fail("Should have raised " + cause.getSimpleName());
+        } catch (KafkaException e) {
+            assertTrue(cause.isAssignableFrom(e.getCause().getClass()));
+            assertTrue(transactionManager.isInErrorState());
+        }
+
+        assertTrue(transactionManager.isInErrorState());
+        transactionManager.beginAbortingTransaction();
+        assertFalse(transactionManager.isInErrorState());
+    }
+
+    private void assertFatalError(Class<? extends RuntimeException> cause) {
+        assertTrue(transactionManager.isInErrorState());
+
+        try {
+            transactionManager.beginAbortingTransaction();
+            fail("Should have raised " + cause.getSimpleName());
+        } catch (KafkaException e) {
+            assertTrue(cause.isAssignableFrom(e.getCause().getClass()));
+            assertTrue(transactionManager.isInErrorState());
+        }
+
+        // Transaction abort cannot clear fatal error state
+        try {
+            transactionManager.beginAbortingTransaction();
+            fail("Should have raised " + cause.getSimpleName());
+        } catch (KafkaException e) {
+            assertTrue(cause.isAssignableFrom(e.getCause().getClass()));
+            assertTrue(transactionManager.isInErrorState());
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/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 9142c90..2e9a688 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
@@ -115,10 +115,10 @@ public class RequestResponseTest {
         checkErrorResponse(createListOffsetRequest(2), new UnknownServerException());
         checkResponse(createListOffsetResponse(2), 2);
         checkRequest(MetadataRequest.Builder.allTopics().build((short) 2));
-        checkRequest(createMetadataRequest(1, asList("topic1")));
-        checkErrorResponse(createMetadataRequest(1, asList("topic1")), new UnknownServerException());
+        checkRequest(createMetadataRequest(1, singletonList("topic1")));
+        checkErrorResponse(createMetadataRequest(1, singletonList("topic1")), new UnknownServerException());
         checkResponse(createMetadataResponse(), 2);
-        checkErrorResponse(createMetadataRequest(2, asList("topic1")), new UnknownServerException());
+        checkErrorResponse(createMetadataRequest(2, singletonList("topic1")), new UnknownServerException());
         checkRequest(createOffsetCommitRequest(2));
         checkErrorResponse(createOffsetCommitRequest(2), new UnknownServerException());
         checkResponse(createOffsetCommitResponse(), 0);
@@ -183,7 +183,7 @@ public class RequestResponseTest {
         checkOlderFetchVersions();
         checkResponse(createMetadataResponse(), 0);
         checkResponse(createMetadataResponse(), 1);
-        checkErrorResponse(createMetadataRequest(1, asList("topic1")), new UnknownServerException());
+        checkErrorResponse(createMetadataRequest(1, singletonList("topic1")), new UnknownServerException());
         checkRequest(createOffsetCommitRequest(0));
         checkErrorResponse(createOffsetCommitRequest(0), new UnknownServerException());
         checkRequest(createOffsetCommitRequest(1));
@@ -984,7 +984,7 @@ public class RequestResponseTest {
         final Map<TopicPartition, TxnOffsetCommitRequest.CommittedOffset> offsets = new HashMap<>();
         offsets.put(new TopicPartition("topic", 73),
                     new TxnOffsetCommitRequest.CommittedOffset(100, null));
-        return new TxnOffsetCommitRequest.Builder("gid", 21L, (short) 42, offsets).build();
+        return new TxnOffsetCommitRequest.Builder("transactionalId", "groupId", 21L, (short) 42, offsets).build();
     }
 
     private TxnOffsetCommitResponse createTxnOffsetCommitResponse() {

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/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 925c407..e02b5dc 100644
--- a/core/src/main/scala/kafka/admin/AclCommand.scala
+++ b/core/src/main/scala/kafka/admin/AclCommand.scala
@@ -34,7 +34,8 @@ object AclCommand {
     Broker -> Set(DescribeConfigs),
     Topic -> Set(Read, Write, Describe, Delete, DescribeConfigs, AlterConfigs, All),
     Group -> Set(Read, Describe, All),
-    Cluster -> Set(Create, ClusterAction, DescribeConfigs, AlterConfigs, All)
+    Cluster -> Set(Create, ClusterAction, DescribeConfigs, AlterConfigs, IdempotentWrite, All),
+    TransactionalId -> Set(Describe, Write, All)
   )
 
   def main(args: Array[String]) {
@@ -88,7 +89,7 @@ object AclCommand {
         CommandLineUtils.printUsageAndDie(opts.parser, "You must specify one of: --allow-principal, --deny-principal when trying to add ACLs.")
 
       for ((resource, acls) <- resourceToAcl) {
-        println(s"Adding ACLs for resource `${resource}`: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline")
+        println(s"Adding ACLs for resource `$resource`: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline")
         authorizer.addAcls(acls, resource)
       }
 
@@ -102,10 +103,10 @@ object AclCommand {
 
       for ((resource, acls) <- resourceToAcl) {
         if (acls.isEmpty) {
-          if (confirmAction(opts, s"Are you sure you want to delete all ACLs for resource `${resource}`? (y/n)"))
+          if (confirmAction(opts, s"Are you sure you want to delete all ACLs for resource `$resource`? (y/n)"))
             authorizer.removeAcls(resource)
         } else {
-          if (confirmAction(opts, s"Are you sure you want to remove ACLs: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline from resource `${resource}`? (y/n)"))
+          if (confirmAction(opts, s"Are you sure you want to remove ACLs: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline from resource `$resource`? (y/n)"))
             authorizer.removeAcls(acls, resource)
         }
       }
@@ -123,7 +124,7 @@ object AclCommand {
         else resources.map(resource => resource -> authorizer.getAcls(resource))
 
       for ((resource, acls) <- resourceToAcls)
-        println(s"Current ACLs for resource `${resource}`: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline")
+        println(s"Current ACLs for resource `$resource`: $Newline ${acls.map("\t" + _).mkString(Newline)} $Newline")
     }
   }
 
@@ -149,12 +150,16 @@ object AclCommand {
 
   private def getProducerResourceToAcls(opts: AclCommandOptions): Map[Resource, Set[Acl]] = {
     val topics: Set[Resource] = getResource(opts).filter(_.resourceType == Topic)
+    val transactionalIds: Set[Resource] = getResource(opts).filter(_.resourceType == TransactionalId)
+    val enableIdempotence = opts.options.has(opts.idempotentOpt)
 
     val acls = getAcl(opts, Set(Write, Describe))
 
-    //Write, Describe permission on topics, Create permission on cluster
-    topics.map(_ -> acls).toMap[Resource, Set[Acl]] +
-      (Resource.ClusterResource -> getAcl(opts, Set(Create)))
+    //Write, Describe permission on topics, Create permission on cluster, Write, Describe on transactionalIds
+    topics.map(_ -> acls).toMap[Resource, Set[Acl]] ++
+      transactionalIds.map(_ -> acls).toMap[Resource, Set[Acl]] +
+      (Resource.ClusterResource -> (getAcl(opts, Set(Create)) ++
+        (if (enableIdempotence) getAcl(opts, Set(IdempotentWrite)) else Set.empty[Acl])))
   }
 
   private def getConsumerResourceToAcls(opts: AclCommandOptions): Map[Resource, Set[Acl]] = {
@@ -184,7 +189,7 @@ object AclCommand {
 
     val allowedHosts = getHosts(opts, opts.allowHostsOpt, opts.allowPrincipalsOpt)
 
-    val deniedHosts = getHosts(opts, opts.denyHostssOpt, opts.denyPrincipalsOpt)
+    val deniedHosts = getHosts(opts, opts.denyHostsOpt, opts.denyPrincipalsOpt)
 
     val acls = new collection.mutable.HashSet[Acl]
     if (allowedHosts.nonEmpty && allowedPrincipals.nonEmpty)
@@ -232,7 +237,7 @@ object AclCommand {
     if (opts.options.has(opts.topicOpt))
       opts.options.valuesOf(opts.topicOpt).asScala.foreach(topic => resources += new Resource(Topic, topic.trim))
 
-    if (opts.options.has(opts.clusterOpt))
+    if (opts.options.has(opts.clusterOpt) || opts.options.has(opts.idempotentOpt))
       resources += Resource.ClusterResource
 
     if (opts.options.has(opts.groupOpt))
@@ -241,6 +246,10 @@ object AclCommand {
     if (opts.options.has(opts.brokerOpt))
       opts.options.valuesOf(opts.brokerOpt).asScala.foreach(broker => resources += new Resource(Broker, broker.toString))
 
+    if (opts.options.has(opts.transactionalIdOpt))
+      opts.options.valuesOf(opts.transactionalIdOpt).asScala.foreach(transactionalId =>
+        resources += new Resource(TransactionalId, transactionalId))
+
     if (resources.isEmpty && dieIfNoResourceFound)
       CommandLineUtils.printUsageAndDie(opts.parser, "You must provide at least one resource: --topic <topic> or --cluster or --group <group>")
 
@@ -295,6 +304,16 @@ object AclCommand {
       .describedAs("broker")
       .ofType(classOf[Int])
 
+    val transactionalIdOpt = parser.accepts("transactional-id", "The transactionalId to which ACLs should " +
+      "be added or removed. A value of * indicates the ACLs should apply to all transactionalIds.")
+      .withRequiredArg
+      .describedAs("transactional-id")
+      .ofType(classOf[String])
+
+    val idempotentOpt = parser.accepts("idempotent", "Enable idempotence for the producer. This should be " +
+      "used in combination with the --producer option. Note that idempotence is enabled automatically if " +
+      "the producer is authorized to a particular transactional-id.")
+
     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 --group <group> or --cluster to specify a resource.")
@@ -329,7 +348,7 @@ object AclCommand {
       .describedAs("allow-host")
       .ofType(classOf[String])
 
-    val denyHostssOpt = parser.accepts("deny-host", "Host from which principals listed in --deny-principal will be denied access. " +
+    val denyHostsOpt = parser.accepts("deny-host", "Host from which principals listed in --deny-principal will be denied access. " +
       "If you have specified --deny-principal then the default for this option will be set to * which denies access from all hosts.")
       .withRequiredArg
       .describedAs("deny-host")
@@ -354,17 +373,20 @@ object AclCommand {
       if (actions != 1)
         CommandLineUtils.printUsageAndDie(parser, "Command must include exactly one action: --list, --add, --remove. ")
 
-      CommandLineUtils.checkInvalidArgs(parser, options, listOpt, Set(producerOpt, consumerOpt, allowHostsOpt, allowPrincipalsOpt, denyHostssOpt, denyPrincipalsOpt))
+      CommandLineUtils.checkInvalidArgs(parser, options, listOpt, Set(producerOpt, consumerOpt, allowHostsOpt, allowPrincipalsOpt, denyHostsOpt, denyPrincipalsOpt))
 
       //when --producer or --consumer is specified , user should not specify operations as they are inferred and we also disallow --deny-principals and --deny-hosts.
-      CommandLineUtils.checkInvalidArgs(parser, options, producerOpt, Set(operationsOpt, denyPrincipalsOpt, denyHostssOpt))
-      CommandLineUtils.checkInvalidArgs(parser, options, consumerOpt, Set(operationsOpt, denyPrincipalsOpt, denyHostssOpt))
+      CommandLineUtils.checkInvalidArgs(parser, options, producerOpt, Set(operationsOpt, denyPrincipalsOpt, denyHostsOpt))
+      CommandLineUtils.checkInvalidArgs(parser, options, consumerOpt, Set(operationsOpt, denyPrincipalsOpt, denyHostsOpt))
 
       if (options.has(producerOpt) && !options.has(topicOpt))
         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 --group and no --cluster option should be specified.")
+      if (options.has(idempotentOpt) && !options.has(producerOpt))
+        CommandLineUtils.printUsageAndDie(parser, "The --idempotent option is only available if --producer is set")
+
+      if (options.has(consumerOpt) && (!options.has(topicOpt) || !options.has(groupOpt) || (!options.has(producerOpt) && (options.has(clusterOpt) || options.has(transactionalIdOpt)))))
+        CommandLineUtils.printUsageAndDie(parser, "With --consumer you must specify a --topic and a --group and no --cluster or --transactional-id option should be specified.")
     }
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala
index 302fcb5..7bde4e2 100644
--- a/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala
+++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadata.scala
@@ -323,6 +323,7 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState
   }
 
   def prepareTxnOffsetCommit(producerId: Long, offsets: Map[TopicPartition, OffsetAndMetadata]) {
+    trace(s"TxnOffsetCommit for producer $producerId and group $groupId with offsets $offsets is pending")
     receivedTransactionalOffsetCommits = true
     val producerOffsets = pendingTransactionalOffsetCommits.getOrElseUpdate(producerId,
       mutable.Map.empty[TopicPartition, CommitRecordMetadataAndOffset])
@@ -339,10 +340,12 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState
   /* Remove a pending transactional offset commit if the actual offset commit record was not written to the log.
    * We will return an error and the client will retry the request, potentially to a different coordinator.
    */
-  def failPendingTxnOffsetCommit(producerId: Long, topicPartition: TopicPartition, offsetAndMetadata: OffsetAndMetadata): Unit = {
+  def failPendingTxnOffsetCommit(producerId: Long, topicPartition: TopicPartition): Unit = {
     pendingTransactionalOffsetCommits.get(producerId) match {
       case Some(pendingOffsets) =>
-        pendingOffsets.remove(topicPartition)
+        val pendingOffsetCommit = pendingOffsets.remove(topicPartition)
+        trace(s"TxnOffsetCommit for producer $producerId and group $groupId with offsets $pendingOffsetCommit failed " +
+          s"to be appended to the log")
         if (pendingOffsets.isEmpty)
           pendingTransactionalOffsetCommits.remove(producerId)
       case _ =>
@@ -366,18 +369,28 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState
    * to the log.
    */
   def completePendingTxnOffsetCommit(producerId: Long, isCommit: Boolean): Unit = {
-    trace(s"Completing transactional offset commit for producer $producerId and group $groupId. isCommit: $isCommit")
+    val pendingOffsetsOpt = pendingTransactionalOffsetCommits.remove(producerId)
     if (isCommit) {
-      val producerOffsets = pendingTransactionalOffsetCommits.getOrElse(producerId, Map.empty[TopicPartition, CommitRecordMetadataAndOffset])
-      producerOffsets.foreach { case (topicPartition, commitRecordMetadataAndOffset) =>
-        if (commitRecordMetadataAndOffset.appendedBatchOffset.isEmpty)
-          throw new IllegalStateException(s"Trying to complete a transactional offset commit for producerId $producerId " +
-            s"and groupId $groupId even though the the offset commit record itself hasn't been appended to the log.")
-        if (!offsets.contains(topicPartition) || offsets(topicPartition).olderThan(commitRecordMetadataAndOffset))
-          offsets.put(topicPartition, commitRecordMetadataAndOffset)
+      pendingOffsetsOpt.foreach { pendingOffsets =>
+        pendingOffsets.foreach { case (topicPartition, commitRecordMetadataAndOffset) =>
+          if (commitRecordMetadataAndOffset.appendedBatchOffset.isEmpty)
+            throw new IllegalStateException(s"Trying to complete a transactional offset commit for producerId $producerId " +
+              s"and groupId $groupId even though the the offset commit record itself hasn't been appended to the log.")
+
+          val currentOffsetOpt = offsets.get(topicPartition)
+          if (currentOffsetOpt.forall(_.olderThan(commitRecordMetadataAndOffset))) {
+            trace(s"TxnOffsetCommit for producer $producerId and group $groupId with offset $commitRecordMetadataAndOffset " +
+              "committed and loaded into the cache.")
+            offsets.put(topicPartition, commitRecordMetadataAndOffset)
+          } else {
+            trace(s"TxnOffsetCommit for producer $producerId and group $groupId with offset $commitRecordMetadataAndOffset " +
+              s"committed, but not loaded since its offset is older than current offset $currentOffsetOpt.")
+          }
+        }
       }
+    } else {
+      trace(s"TxnOffsetCommit for producer $producerId and group $groupId with offsets $pendingOffsetsOpt aborted")
     }
-    pendingTransactionalOffsetCommits.remove(producerId)
   }
 
   def activeProducers = pendingTransactionalOffsetCommits.keySet
@@ -430,7 +443,13 @@ private[group] class GroupMetadata(val groupId: String, initialState: GroupState
   }
 
   override def toString: String = {
-    "[%s,%s,%s,%s]".format(groupId, protocolType, currentState.toString, members)
+    "GroupMetadata(" +
+      s"groupId=$groupId, " +
+      s"generation=$generationId, " +
+      s"protocolType=$protocolType, " +
+      s"currentState=$currentState, " +
+      s"members=$members)"
   }
+
 }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
index a7eb28b..8e5135d 100644
--- a/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
+++ b/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala
@@ -324,7 +324,7 @@ class GroupMetadataManager(brokerId: Int,
                     removeProducerGroup(producerId, group.groupId)
                   filteredOffsetMetadata.foreach { case (topicPartition, offsetAndMetadata) =>
                     if (isTxnOffsetCommit)
-                      group.failPendingTxnOffsetCommit(producerId, topicPartition, offsetAndMetadata)
+                      group.failPendingTxnOffsetCommit(producerId, topicPartition)
                     else
                       group.failPendingOffsetWrite(topicPartition, offsetAndMetadata)
                   }
@@ -536,7 +536,6 @@ class GroupMetadataManager(brokerId: Int,
                     val groupId = groupMetadataKey.key
                     val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, record.value)
                     if (groupMetadata != null) {
-                      trace(s"Loaded group metadata for group $groupId with generation ${groupMetadata.generationId}")
                       removedGroups.remove(groupId)
                       loadedGroups.put(groupId, groupMetadata)
                     } else {
@@ -577,6 +576,7 @@ class GroupMetadataManager(brokerId: Int,
           loadedGroups.values.foreach { group =>
             val offsets = groupOffsets.getOrElse(group.groupId, Map.empty[TopicPartition, CommitRecordMetadataAndOffset])
             val pendingOffsets = pendingGroupOffsets.getOrElse(group.groupId, Map.empty[Long, mutable.Map[TopicPartition, CommitRecordMetadataAndOffset]])
+            trace(s"Loaded group metadata $group with offsets $offsets and pending offsets $pendingOffsets")
             loadGroup(group, offsets, pendingOffsets)
             onGroupLoaded(group)
           }
@@ -587,6 +587,7 @@ class GroupMetadataManager(brokerId: Int,
             val group = new GroupMetadata(groupId)
             val offsets = emptyGroupOffsets.getOrElse(groupId, Map.empty[TopicPartition, CommitRecordMetadataAndOffset])
             val pendingOffsets = pendingEmptyGroupOffsets.getOrElse(groupId, Map.empty[Long, mutable.Map[TopicPartition, CommitRecordMetadataAndOffset]])
+            trace(s"Loaded group metadata $group with offsets $offsets and pending offsets $pendingOffsets")
             loadGroup(group, offsets, pendingOffsets)
             onGroupLoaded(group)
           }

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/core/src/main/scala/kafka/coordinator/group/MemberMetadata.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/group/MemberMetadata.scala b/core/src/main/scala/kafka/coordinator/group/MemberMetadata.scala
index ce542b1..b082b9b 100644
--- a/core/src/main/scala/kafka/coordinator/group/MemberMetadata.scala
+++ b/core/src/main/scala/kafka/coordinator/group/MemberMetadata.scala
@@ -85,7 +85,7 @@ private[group] class MemberMetadata(val memberId: String,
     if (protocols.size != this.supportedProtocols.size)
       return false
 
-    for (i <- 0 until protocols.size) {
+    for (i <- protocols.indices) {
       val p1 = protocols(i)
       val p2 = supportedProtocols(i)
       if (p1._1 != p2._1 || !util.Arrays.equals(p1._2, p2._2))
@@ -114,7 +114,15 @@ private[group] class MemberMetadata(val memberId: String,
     }
   }
 
-  override def toString = {
-    "[%s,%s,%s,%d]".format(memberId, clientId, clientHost, sessionTimeoutMs)
+  override def toString: String = {
+    "MemberMetadata(" +
+      s"memberId=$memberId, " +
+      s"clientId=$clientId, " +
+      s"clientHost=$clientHost, " +
+      s"sessionTimeoutMs=$sessionTimeoutMs, " +
+      s"rebalanceTimeoutMs=$rebalanceTimeoutMs, " +
+      s"supportedProtocols=${supportedProtocols.map(_._1)}, " +
+      ")"
   }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/core/src/main/scala/kafka/security/auth/Operation.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/security/auth/Operation.scala b/core/src/main/scala/kafka/security/auth/Operation.scala
index f65d9f0..420c3eb 100644
--- a/core/src/main/scala/kafka/security/auth/Operation.scala
+++ b/core/src/main/scala/kafka/security/auth/Operation.scala
@@ -65,6 +65,10 @@ case object AlterConfigs extends Operation {
   val name = "AlterConfigs"
   val toJava = AclOperation.ALTER_CONFIGS
 }
+case object IdempotentWrite extends Operation {
+  val name = "IdempotentWrite"
+  val toJava = AclOperation.IDEMPOTENT_WRITE
+}
 case object All extends Operation {
   val name = "All"
   val toJava = AclOperation.ALL
@@ -86,5 +90,5 @@ object Operation {
   }
 
   def values: Seq[Operation] = List(Read, Write, Create, Delete, Alter, Describe, ClusterAction, AlterConfigs,
-     DescribeConfigs, All)
+     DescribeConfigs, IdempotentWrite, All)
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/core/src/main/scala/kafka/security/auth/Resource.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/security/auth/Resource.scala b/core/src/main/scala/kafka/security/auth/Resource.scala
index a0ed9f9..311f5b5 100644
--- a/core/src/main/scala/kafka/security/auth/Resource.scala
+++ b/core/src/main/scala/kafka/security/auth/Resource.scala
@@ -21,7 +21,6 @@ object Resource {
   val ClusterResourceName = "kafka-cluster"
   val ClusterResource = new Resource(Cluster, Resource.ClusterResourceName)
   val ProducerIdResourceName = "producer-id"
-  val ProducerIdResource = new Resource(Cluster, Resource.ProducerIdResourceName)
   val WildCardResource = "*"
 
   def fromString(str: String): Resource = {
@@ -38,7 +37,7 @@ object Resource {
  * @param name name of the resource, for topic this will be topic name , for group it will be group name. For cluster type
  *             it will be a constant string kafka-cluster.
  */
-case class Resource(val resourceType: ResourceType, val name: String) {
+case class Resource(resourceType: ResourceType, name: String) {
 
   override def toString: String = {
     resourceType.name + Resource.Separator + name

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/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 ea7ce3c..9cfe1cd 100644
--- a/core/src/main/scala/kafka/security/auth/ResourceType.scala
+++ b/core/src/main/scala/kafka/security/auth/ResourceType.scala
@@ -19,11 +19,6 @@ package kafka.security.auth
 import kafka.common.{BaseEnum, KafkaException}
 import org.apache.kafka.common.protocol.Errors
 
-/**
- * ResourceTypes.
- */
-
-
 sealed trait ResourceType extends BaseEnum { def error: Errors }
 
 case object Cluster extends ResourceType {
@@ -46,16 +41,11 @@ case object Group extends ResourceType {
   val error = Errors.GROUP_AUTHORIZATION_FAILED
 }
 
-case object ProducerTransactionalId extends ResourceType {
-  val name = "ProducerTransactionalId"
+case object TransactionalId extends ResourceType {
+  val name = "TransactionalId"
   val error = Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED
 }
 
-case object ProducerIdResource extends ResourceType {
-  val name = "ProducerIdResource"
-  val error = Errors.PRODUCER_ID_AUTHORIZATION_FAILED
-}
-
 object ResourceType {
 
   def fromString(resourceType: String): ResourceType = {
@@ -63,5 +53,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, Group, ProducerTransactionalId, ProducerIdResource, Broker)
+  def values: Seq[ResourceType] = List(Cluster, Topic, Group, TransactionalId, Broker)
 }


[3/5] kafka git commit: KAFKA-5259; TransactionalId auth implies ProducerId auth

Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/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 b0d354b..380685f 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -93,7 +93,7 @@ class KafkaApis(val requestChannel: RequestChannel,
       trace("Handling request:%s from connection %s;securityProtocol:%s,principal:%s".
         format(request.requestDesc(true), request.connectionId, request.securityProtocol, request.session.principal))
       ApiKeys.forId(request.requestId) match {
-        case ApiKeys.PRODUCE => handleProducerRequest(request)
+        case ApiKeys.PRODUCE => handleProduceRequest(request)
         case ApiKeys.FETCH => handleFetchRequest(request)
         case ApiKeys.LIST_OFFSETS => handleListOffsetRequest(request)
         case ApiKeys.METADATA => handleTopicMetadataRequest(request)
@@ -368,97 +368,107 @@ class KafkaApis(val requestChannel: RequestChannel,
   /**
    * Handle a produce request
    */
-  def handleProducerRequest(request: RequestChannel.Request) {
+  def handleProduceRequest(request: RequestChannel.Request) {
     val produceRequest = request.body[ProduceRequest]
     val numBytesAppended = request.header.toStruct.sizeOf + request.bodyAndSize.size
 
-    if (produceRequest.isTransactional && !authorize(request.session, Write, new Resource(ProducerTransactionalId, produceRequest.transactionalId())))
-      sendResponseMaybeThrottle(request, (throttleMs: Int) => produceRequest.getErrorResponse(throttleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
-    else if (produceRequest.isIdempotent && !authorize(request.session, Write, Resource.ProducerIdResource))
-      sendResponseMaybeThrottle(request, (throttleMs: Int) => produceRequest.getErrorResponse(throttleMs, Errors.PRODUCER_ID_AUTHORIZATION_FAILED.exception))
-    else {
-      val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) =
-        produceRequest.partitionRecordsOrFail.asScala.partition { case (tp, _) =>
-          authorize(request.session, Describe, new Resource(Topic, tp.topic)) && metadataCache.contains(tp.topic)
-        }
+    def sendErrorResponse(error: Errors): Unit = {
+      sendResponseMaybeThrottle(request, requestThrottleMs =>
+        produceRequest.getErrorResponse(requestThrottleMs, error.exception))
+    }
 
-      val (authorizedRequestInfo, unauthorizedForWriteRequestInfo) = existingAndAuthorizedForDescribeTopics.partition {
-        case (tp, _) => authorize(request.session, Write, new Resource(Topic, tp.topic))
+    if (produceRequest.isTransactional) {
+      if (!authorize(request.session, Write, new Resource(TransactionalId, produceRequest.transactionalId))) {
+        sendErrorResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED)
+        return
       }
+      // Note that authorization to a transactionalId implies ProducerId authorization
 
-      // the callback for sending a produce response
-      def sendResponseCallback(responseStatus: Map[TopicPartition, PartitionResponse]) {
+    } else if (produceRequest.isIdempotent && !authorize(request.session, IdempotentWrite, Resource.ClusterResource)) {
+      sendErrorResponse(Errors.CLUSTER_AUTHORIZATION_FAILED)
+      return
+    }
+
+    val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) =
+      produceRequest.partitionRecordsOrFail.asScala.partition { case (tp, _) =>
+        authorize(request.session, Describe, new Resource(Topic, tp.topic)) && metadataCache.contains(tp.topic)
+      }
 
-        val mergedResponseStatus = responseStatus ++
-            unauthorizedForWriteRequestInfo.mapValues(_ => new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED)) ++
-            nonExistingOrUnauthorizedForDescribeTopics.mapValues(_ => new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION))
+    val (authorizedRequestInfo, unauthorizedForWriteRequestInfo) = existingAndAuthorizedForDescribeTopics.partition {
+      case (tp, _) => authorize(request.session, Write, new Resource(Topic, tp.topic))
+    }
 
-        var errorInResponse = false
+    // the callback for sending a produce response
+    def sendResponseCallback(responseStatus: Map[TopicPartition, PartitionResponse]) {
 
-        mergedResponseStatus.foreach { case (topicPartition, status) =>
-          if (status.error != Errors.NONE) {
-            errorInResponse = true
-            debug("Produce request with correlation id %d from client %s on partition %s failed due to %s".format(
-              request.header.correlationId,
-              request.header.clientId,
-              topicPartition,
-              status.error.exceptionName))
-          }
+      val mergedResponseStatus = responseStatus ++
+        unauthorizedForWriteRequestInfo.mapValues(_ => new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED)) ++
+        nonExistingOrUnauthorizedForDescribeTopics.mapValues(_ => new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION))
+
+      var errorInResponse = false
+
+      mergedResponseStatus.foreach { case (topicPartition, status) =>
+        if (status.error != Errors.NONE) {
+          errorInResponse = true
+          debug("Produce request with correlation id %d from client %s on partition %s failed due to %s".format(
+            request.header.correlationId,
+            request.header.clientId,
+            topicPartition,
+            status.error.exceptionName))
         }
+      }
 
-        def produceResponseCallback(bandwidthThrottleTimeMs: Int) {
-          if (produceRequest.acks == 0) {
-            // no operation needed if producer request.required.acks = 0; however, if there is any error in handling
-            // the request, since no response is expected by the producer, the server will close socket server so that
-            // the producer client will know that some error has happened and will refresh its metadata
-            val action =
-              if (errorInResponse) {
-                val exceptionsSummary = mergedResponseStatus.map { case (topicPartition, status) =>
-                  topicPartition -> status.error.exceptionName
-                }.mkString(", ")
-                info(
-                  s"Closing connection due to error during produce request with correlation id ${request.header.correlationId} " +
-                    s"from client id ${request.header.clientId} with ack=0\n" +
-                    s"Topic and partition to exceptions: $exceptionsSummary"
-                )
-                RequestChannel.CloseConnectionAction
-              } else RequestChannel.NoOpAction
-            sendResponseExemptThrottle(new RequestChannel.Response(request, None, action))
-          } else {
-            sendResponseMaybeThrottle(request, requestThrottleMs =>
-              new ProduceResponse(mergedResponseStatus.asJava, bandwidthThrottleTimeMs + requestThrottleMs)
+      def produceResponseCallback(bandwidthThrottleTimeMs: Int) {
+        if (produceRequest.acks == 0) {
+          // no operation needed if producer request.required.acks = 0; however, if there is any error in handling
+          // the request, since no response is expected by the producer, the server will close socket server so that
+          // the producer client will know that some error has happened and will refresh its metadata
+          val action =
+          if (errorInResponse) {
+            val exceptionsSummary = mergedResponseStatus.map { case (topicPartition, status) =>
+              topicPartition -> status.error.exceptionName
+            }.mkString(", ")
+            info(
+              s"Closing connection due to error during produce request with correlation id ${request.header.correlationId} " +
+                s"from client id ${request.header.clientId} with ack=0\n" +
+                s"Topic and partition to exceptions: $exceptionsSummary"
             )
-          }
+            RequestChannel.CloseConnectionAction
+          } else RequestChannel.NoOpAction
+          sendResponseExemptThrottle(new RequestChannel.Response(request, None, action))
+        } else {
+          sendResponseMaybeThrottle(request, requestThrottleMs =>
+            new ProduceResponse(mergedResponseStatus.asJava, bandwidthThrottleTimeMs + requestThrottleMs))
         }
+      }
 
-        // When this callback is triggered, the remote API call has completed
-        request.apiRemoteCompleteTimeNanos = time.nanoseconds
+      // When this callback is triggered, the remote API call has completed
+      request.apiRemoteCompleteTimeNanos = time.nanoseconds
 
-        quotas.produce.recordAndMaybeThrottle(
-          request.session.sanitizedUser,
-          request.header.clientId,
-          numBytesAppended,
-          produceResponseCallback)
-      }
+      quotas.produce.recordAndMaybeThrottle(
+        request.session.sanitizedUser,
+        request.header.clientId,
+        numBytesAppended,
+        produceResponseCallback)
+    }
 
-      if (authorizedRequestInfo.isEmpty)
-        sendResponseCallback(Map.empty)
-      else {
-        val internalTopicsAllowed = request.header.clientId == AdminUtils.AdminClientId
-
-        // call the replica manager to append messages to the replicas
-        replicaManager.appendRecords(
-          timeout = produceRequest.timeout.toLong,
-          requiredAcks = produceRequest.acks,
-          internalTopicsAllowed = internalTopicsAllowed,
-          isFromClient = true,
-          entriesPerPartition = authorizedRequestInfo,
-          responseCallback = sendResponseCallback)
-
-        // if the request is put into the purgatory, it will have a held reference and hence cannot be garbage collected;
-        // hence we clear its data here inorder to let GC re-claim its memory since it is already appended to log
-        produceRequest.clearPartitionRecords()
-      }
+    if (authorizedRequestInfo.isEmpty)
+      sendResponseCallback(Map.empty)
+    else {
+      val internalTopicsAllowed = request.header.clientId == AdminUtils.AdminClientId
+
+      // call the replica manager to append messages to the replicas
+      replicaManager.appendRecords(
+        timeout = produceRequest.timeout.toLong,
+        requiredAcks = produceRequest.acks,
+        internalTopicsAllowed = internalTopicsAllowed,
+        isFromClient = true,
+        entriesPerPartition = authorizedRequestInfo,
+        responseCallback = sendResponseCallback)
+
+      // if the request is put into the purgatory, it will have a held reference and hence cannot be garbage collected;
+      // hence we clear its data here inorder to let GC re-claim its memory since it is already appended to log
+      produceRequest.clearPartitionRecords()
     }
   }
 
@@ -1052,13 +1062,16 @@ class KafkaApis(val requestChannel: RequestChannel,
   def handleFindCoordinatorRequest(request: RequestChannel.Request) {
     val findCoordinatorRequest = request.body[FindCoordinatorRequest]
 
-    if (findCoordinatorRequest.coordinatorType == FindCoordinatorRequest.CoordinatorType.GROUP &&
-      !authorize(request.session, Describe, new Resource(Group, findCoordinatorRequest.coordinatorKey))) {
-      sendResponseMaybeThrottle(request, requestThrottleMs =>
-        new FindCoordinatorResponse(requestThrottleMs, Errors.GROUP_AUTHORIZATION_FAILED, Node.noNode))
-    } else {
-      // TODO: Authorize by transactional id if coordinator type is TRANSACTION
+    def sendErrorResponse(error: Errors): Unit =
+      sendResponseMaybeThrottle(request, requestThrottleMs => new FindCoordinatorResponse(error, Node.noNode))
 
+    if (findCoordinatorRequest.coordinatorType == FindCoordinatorRequest.CoordinatorType.GROUP &&
+        !authorize(request.session, Describe, new Resource(Group, findCoordinatorRequest.coordinatorKey)))
+      sendErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED)
+    else if (findCoordinatorRequest.coordinatorType == FindCoordinatorRequest.CoordinatorType.TRANSACTION &&
+        !authorize(request.session, Describe, new Resource(TransactionalId, findCoordinatorRequest.coordinatorKey)))
+      sendErrorResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED)
+    else {
       // get metadata (and create the topic if necessary)
       val (partition, topicMetadata) = findCoordinatorRequest.coordinatorType match {
         case FindCoordinatorRequest.CoordinatorType.GROUP =>
@@ -1102,18 +1115,18 @@ class KafkaApis(val requestChannel: RequestChannel,
     val describeRequest = request.body[DescribeGroupsRequest]
 
     val groups = describeRequest.groupIds.asScala.map { groupId =>
-        if (!authorize(request.session, Describe, new Resource(Group, groupId))) {
-          groupId -> DescribeGroupsResponse.GroupMetadata.forError(Errors.GROUP_AUTHORIZATION_FAILED)
-        } else {
-          val (error, summary) = groupCoordinator.handleDescribeGroup(groupId)
-          val members = summary.members.map { member =>
-            val metadata = ByteBuffer.wrap(member.metadata)
-            val assignment = ByteBuffer.wrap(member.assignment)
-            new DescribeGroupsResponse.GroupMember(member.memberId, member.clientId, member.clientHost, metadata, assignment)
-          }
-          groupId -> new DescribeGroupsResponse.GroupMetadata(error, summary.state, summary.protocolType,
-            summary.protocol, members.asJava)
+      if (!authorize(request.session, Describe, new Resource(Group, groupId))) {
+        groupId -> DescribeGroupsResponse.GroupMetadata.forError(Errors.GROUP_AUTHORIZATION_FAILED)
+      } else {
+        val (error, summary) = groupCoordinator.handleDescribeGroup(groupId)
+        val members = summary.members.map { member =>
+          val metadata = ByteBuffer.wrap(member.metadata)
+          val assignment = ByteBuffer.wrap(member.assignment)
+          new DescribeGroupsResponse.GroupMember(member.memberId, member.clientId, member.clientHost, metadata, assignment)
         }
+        groupId -> new DescribeGroupsResponse.GroupMetadata(error, summary.state, summary.protocolType,
+          summary.protocol, members.asJava)
+      }
     }.toMap
 
     sendResponseMaybeThrottle(request, requestThrottleMs => new DescribeGroupsResponse(requestThrottleMs, groups.asJava))
@@ -1414,31 +1427,36 @@ class KafkaApis(val requestChannel: RequestChannel,
     val initProducerIdRequest = request.body[InitProducerIdRequest]
     val transactionalId = initProducerIdRequest.transactionalId
 
+    def sendErrorResponse(error: Errors): Unit = {
+      sendResponseMaybeThrottle(request, requestThrottleMs => new InitProducerIdResponse(requestThrottleMs, error))
+    }
 
-    if (!authorize(request.session, Write, Resource.ProducerIdResource)) {
-      sendResponseMaybeThrottle(request, requestThrottleMs =>
-        new InitProducerIdResponse(requestThrottleMs, Errors.PRODUCER_ID_AUTHORIZATION_FAILED))
-    } else if (transactionalId == null || authorize(request.session, Write, new Resource(ProducerTransactionalId, transactionalId))) {
-      // Send response callback
-      def sendResponseCallback(result: InitProducerIdResult): Unit = {
-        def createResponse(requestThrottleMs: Int): AbstractResponse = {
-          val responseBody = new InitProducerIdResponse(requestThrottleMs, result.error, result.producerId, result.producerEpoch)
-          trace(s"Completed $transactionalId's InitProducerIdRequest with result $result from client ${request.header.clientId}.")
-          responseBody
-        }
-        sendResponseMaybeThrottle(request, createResponse)
+    if (transactionalId != null) {
+      if (!authorize(request.session, Write, new Resource(TransactionalId, transactionalId))) {
+        sendErrorResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED)
+        return
       }
-      txnCoordinator.handleInitProducerId(transactionalId, initProducerIdRequest.transactionTimeoutMs, sendResponseCallback)
-    }else
-      sendResponseMaybeThrottle(request, requestThrottleMs =>
-        new InitProducerIdResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED))
+    } else if (!authorize(request.session, IdempotentWrite, Resource.ClusterResource)) {
+      sendErrorResponse(Errors.CLUSTER_AUTHORIZATION_FAILED)
+      return
+    }
+
+    def sendResponseCallback(result: InitProducerIdResult): Unit = {
+      def createResponse(requestThrottleMs: Int): AbstractResponse = {
+        val responseBody = new InitProducerIdResponse(requestThrottleMs, result.error, result.producerId, result.producerEpoch)
+        trace(s"Completed $transactionalId's InitProducerIdRequest with result $result from client ${request.header.clientId}.")
+        responseBody
+      }
+      sendResponseMaybeThrottle(request, createResponse)
+    }
+    txnCoordinator.handleInitProducerId(transactionalId, initProducerIdRequest.transactionTimeoutMs, sendResponseCallback)
   }
 
   def handleEndTxnRequest(request: RequestChannel.Request): Unit = {
     val endTxnRequest = request.body[EndTxnRequest]
     val transactionalId = endTxnRequest.transactionalId
 
-    if(authorize(request.session, Write, new Resource(ProducerTransactionalId, transactionalId))) {
+    if (authorize(request.session, Write, new Resource(TransactionalId, transactionalId))) {
       def sendResponseCallback(error: Errors) {
         def createResponse(requestThrottleMs: Int): AbstractResponse = {
           val responseBody = new EndTxnResponse(requestThrottleMs, error)
@@ -1471,6 +1489,7 @@ class KafkaApis(val requestChannel: RequestChannel,
     }
 
     def sendResponseCallback(producerId: Long, result: TransactionResult)(responseStatus: Map[TopicPartition, PartitionResponse]): Unit = {
+      trace(s"End transaction marker append for producer id $producerId completed with status: $responseStatus")
       errors.put(producerId, responseStatus.mapValues(_.error).asJava)
 
       val successfulOffsetsPartitions = responseStatus.filter { case (topicPartition, partitionResponse) =>
@@ -1524,9 +1543,9 @@ class KafkaApis(val requestChannel: RequestChannel,
     val transactionalId = addPartitionsToTxnRequest.transactionalId
     val partitionsToAdd = addPartitionsToTxnRequest.partitions
 
-    if(!authorize(request.session, Write, new Resource(ProducerTransactionalId, transactionalId)))
+    if (!authorize(request.session, Write, new Resource(TransactionalId, transactionalId)))
       sendResponseMaybeThrottle(request, requestThrottleMs =>
-        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception()))
+        addPartitionsToTxnRequest.getErrorResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED.exception))
     else {
       val internalTopics = partitionsToAdd.asScala.filter {tp => org.apache.kafka.common.internals.Topic.isInternal(tp.topic())}
 
@@ -1535,24 +1554,24 @@ class KafkaApis(val requestChannel: RequestChannel,
           authorize(request.session, Describe, new Resource(Topic, tp.topic)) && metadataCache.contains(tp.topic)
         }
 
-      val (authorizedRequestInfo, unauthorizedForWriteRequestInfo) = existingAndAuthorizedForDescribeTopics.partition {
-        tp => authorize(request.session, Write, new Resource(Topic, tp.topic))
+      val unauthorizedForWriteRequestInfo = existingAndAuthorizedForDescribeTopics.filterNot { tp =>
+        authorize(request.session, Write, new Resource(Topic, tp.topic))
       }
 
       if (nonExistingOrUnauthorizedForDescribeTopics.nonEmpty
         || unauthorizedForWriteRequestInfo.nonEmpty
         || internalTopics.nonEmpty) {
 
-        // Only send back error responses for the partitions that failed. If there are any partition failures
-        // then the entire request fails
-        val partitionErrors = unauthorizedForWriteRequestInfo.map { tp => (tp, Errors.TOPIC_AUTHORIZATION_FAILED) }.toMap ++
-          nonExistingOrUnauthorizedForDescribeTopics.map { tp => (tp, Errors.UNKNOWN_TOPIC_OR_PARTITION) }.toMap ++
-          internalTopics.map { tp => (tp, Errors.TOPIC_AUTHORIZATION_FAILED) }
+        // Any failed partition check causes the entire request to fail. We only send back error responses
+        // for the partitions that failed to avoid needing to send an ambiguous error code for the partitions
+        // which succeeded.
+        val partitionErrors = (unauthorizedForWriteRequestInfo.map(_ -> Errors.TOPIC_AUTHORIZATION_FAILED) ++
+          nonExistingOrUnauthorizedForDescribeTopics.map(_ -> Errors.UNKNOWN_TOPIC_OR_PARTITION) ++
+          internalTopics.map(_ ->Errors.TOPIC_AUTHORIZATION_FAILED)).toMap
 
         sendResponseMaybeThrottle(request, requestThrottleMs =>
           new AddPartitionsToTxnResponse(requestThrottleMs, partitionErrors.asJava))
       } else {
-        // Send response callback
         def sendResponseCallback(error: Errors): Unit = {
           def createResponse(requestThrottleMs: Int): AbstractResponse = {
             val responseBody: AddPartitionsToTxnResponse = new AddPartitionsToTxnResponse(requestThrottleMs,
@@ -1565,14 +1584,12 @@ class KafkaApis(val requestChannel: RequestChannel,
         }
 
         txnCoordinator.handleAddPartitionsToTransaction(transactionalId,
-          addPartitionsToTxnRequest.producerId(),
-          addPartitionsToTxnRequest.producerEpoch(),
+          addPartitionsToTxnRequest.producerId,
+          addPartitionsToTxnRequest.producerEpoch,
           partitionsToAdd.asScala.toSet,
           sendResponseCallback)
       }
     }
-
-
   }
 
   def handleAddOffsetsToTxnRequest(request: RequestChannel.Request): Unit = {
@@ -1581,44 +1598,47 @@ class KafkaApis(val requestChannel: RequestChannel,
     val groupId = addOffsetsToTxnRequest.consumerGroupId
     val offsetTopicPartition = new TopicPartition(GROUP_METADATA_TOPIC_NAME, groupCoordinator.partitionFor(groupId))
 
-    if (!authorize(request.session, Write, new Resource(ProducerTransactionalId, transactionalId)))
+    if (!authorize(request.session, Write, new Resource(TransactionalId, transactionalId)))
       sendResponseMaybeThrottle(request, requestThrottleMs =>
         new AddOffsetsToTxnResponse(requestThrottleMs, Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED))
     else if (!authorize(request.session, Read, new Resource(Group, groupId)))
       sendResponseMaybeThrottle(request, requestThrottleMs =>
         new AddOffsetsToTxnResponse(requestThrottleMs, Errors.GROUP_AUTHORIZATION_FAILED))
     else {
-        // Send response callback
-        def sendResponseCallback(error: Errors): Unit = {
-          def createResponse(requestThrottleMs: Int): AbstractResponse = {
-            val responseBody: AddOffsetsToTxnResponse = new AddOffsetsToTxnResponse(requestThrottleMs, error)
-            trace(s"Completed $transactionalId's AddOffsetsToTxnRequest for group $groupId as on partition $offsetTopicPartition: errors: $error from client ${request.header.clientId}")
-            responseBody
-          }
-          sendResponseMaybeThrottle(request, createResponse)
+      def sendResponseCallback(error: Errors): Unit = {
+        def createResponse(requestThrottleMs: Int): AbstractResponse = {
+          val responseBody: AddOffsetsToTxnResponse = new AddOffsetsToTxnResponse(requestThrottleMs, error)
+          trace(s"Completed $transactionalId's AddOffsetsToTxnRequest for group $groupId on partition " +
+            s"$offsetTopicPartition: errors: $error from client ${request.header.clientId}")
+          responseBody
         }
-
-        txnCoordinator.handleAddPartitionsToTransaction(transactionalId,
-          addOffsetsToTxnRequest.producerId,
-          addOffsetsToTxnRequest.producerEpoch,
-          Set(offsetTopicPartition),
-          sendResponseCallback)
+        sendResponseMaybeThrottle(request, createResponse)
       }
-    }
 
+      txnCoordinator.handleAddPartitionsToTransaction(transactionalId,
+        addOffsetsToTxnRequest.producerId,
+        addOffsetsToTxnRequest.producerEpoch,
+        Set(offsetTopicPartition),
+        sendResponseCallback)
+    }
+  }
 
   def handleTxnOffsetCommitRequest(request: RequestChannel.Request): Unit = {
     val header = request.header
     val txnOffsetCommitRequest = request.body[TxnOffsetCommitRequest]
-    // reject the request if not authorized to the group
-    if (!authorize(request.session, Read, new Resource(Group, txnOffsetCommitRequest.consumerGroupId))) {
-      val error = Errors.GROUP_AUTHORIZATION_FAILED
-      val results = txnOffsetCommitRequest.offsets.keySet.asScala.map { topicPartition =>
-        (topicPartition, error)
-      }.toMap
+
+    def sendErrorResponse(error: Errors): Unit = {
       sendResponseMaybeThrottle(request, requestThrottleMs =>
-        new TxnOffsetCommitResponse(requestThrottleMs, results.asJava))
-    } else {
+        txnOffsetCommitRequest.getErrorResponse(requestThrottleMs, error.exception))
+    }
+
+    // authorize for the transactionalId and the consumer group. Note that we skip producerId authorization
+    // since it is implied by transactionalId authorization
+    if (!authorize(request.session, Write, new Resource(TransactionalId, txnOffsetCommitRequest.transactionalId)))
+      sendErrorResponse(Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED)
+    else if (!authorize(request.session, Read, new Resource(Group, txnOffsetCommitRequest.consumerGroupId)))
+      sendErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED)
+    else {
       val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = txnOffsetCommitRequest.offsets.asScala.toMap.partition {
         case (topicPartition, _) =>
           val authorizedForDescribe = authorize(request.session, Describe, new Resource(Topic, topicPartition.topic))

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
index 0198d38..c464834 100644
--- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
@@ -42,6 +42,7 @@ import org.apache.kafka.common.KafkaException
 import kafka.admin.AdminUtils
 import kafka.log.LogConfig
 import kafka.network.SocketServer
+import org.apache.kafka.clients.consumer.OffsetAndMetadata
 import org.apache.kafka.common.network.ListenerName
 import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord}
 
@@ -55,6 +56,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
   val createTopic = "topic-new"
   val deleteTopic = "topic-delete"
   val transactionalId = "transactional.id"
+  val producerId = 83392L
   val part = 0
   val correlationId = 0
   val clientId = "client-Id"
@@ -64,22 +66,24 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
   val topicResource = new Resource(Topic, topic)
   val groupResource = new Resource(Group, group)
   val deleteTopicResource = new Resource(Topic, deleteTopic)
-  val producerTransactionalIdResource = new Resource(ProducerTransactionalId, transactionalId)
+  val transactionalIdResource = new Resource(TransactionalId, transactionalId)
 
   val groupReadAcl = Map(groupResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Read)))
+  val groupDescribeAcl = Map(groupResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe)))
   val clusterAcl = Map(Resource.ClusterResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, ClusterAction)))
   val clusterCreateAcl = Map(Resource.ClusterResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Create)))
+  val clusterIdempotentWriteAcl = Map(Resource.ClusterResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, IdempotentWrite)))
   val topicReadAcl = Map(topicResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Read)))
   val topicWriteAcl = Map(topicResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)))
   val topicDescribeAcl = Map(topicResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe)))
   val topicDeleteAcl = Map(deleteTopicResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Delete)))
   val topicDescribeConfigsAcl = Map(topicResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, DescribeConfigs)))
   val topicAlterConfigsAcl = Map(topicResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, AlterConfigs)))
-  val producerTransactionalIdWriteAcl = Map(producerTransactionalIdResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)))
+  val transactionIdWriteAcl = Map(transactionalIdResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)))
+  val transactionalIdDescribeAcl = Map(transactionalIdResource -> Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe)))
 
   val consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]()
   val producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]()
-  var transactionalProducer: KafkaProducer[Array[Byte], Array[Byte]] = _
 
   val producerCount = 1
   val consumerCount = 2
@@ -115,7 +119,13 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
       ApiKeys.DELETE_TOPICS -> classOf[requests.DeleteTopicsResponse],
       ApiKeys.OFFSET_FOR_LEADER_EPOCH -> classOf[OffsetsForLeaderEpochResponse],
       ApiKeys.DESCRIBE_CONFIGS -> classOf[DescribeConfigsResponse],
-      ApiKeys.ALTER_CONFIGS -> classOf[AlterConfigsResponse]
+      ApiKeys.ALTER_CONFIGS -> classOf[AlterConfigsResponse],
+      ApiKeys.INIT_PRODUCER_ID -> classOf[InitProducerIdResponse],
+      ApiKeys.WRITE_TXN_MARKERS -> classOf[WriteTxnMarkersResponse],
+      ApiKeys.ADD_PARTITIONS_TO_TXN -> classOf[AddPartitionsToTxnResponse],
+      ApiKeys.ADD_OFFSETS_TO_TXN -> classOf[AddOffsetsToTxnResponse],
+      ApiKeys.END_TXN -> classOf[EndTxnResponse],
+      ApiKeys.TXN_OFFSET_COMMIT -> classOf[TxnOffsetCommitResponse]
   )
 
   val requestKeyToError = Map[ApiKeys, Nothing => Errors](
@@ -140,17 +150,23 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
     ApiKeys.DESCRIBE_CONFIGS -> ((resp: DescribeConfigsResponse) =>
       resp.configs.get(new RResource(RResourceType.TOPIC, tp.topic)).error.error),
     ApiKeys.ALTER_CONFIGS -> ((resp: AlterConfigsResponse) =>
-      resp.errors.get(new RResource(RResourceType.TOPIC, tp.topic)).error)
+      resp.errors.get(new RResource(RResourceType.TOPIC, tp.topic)).error),
+    ApiKeys.INIT_PRODUCER_ID -> ((resp: InitProducerIdResponse) => resp.error),
+    ApiKeys.WRITE_TXN_MARKERS -> ((resp: WriteTxnMarkersResponse) => resp.errors(producerId).get(tp)),
+    ApiKeys.ADD_PARTITIONS_TO_TXN -> ((resp: AddPartitionsToTxnResponse) => resp.errors.get(tp)),
+    ApiKeys.ADD_OFFSETS_TO_TXN -> ((resp: AddOffsetsToTxnResponse) => resp.error),
+    ApiKeys.END_TXN -> ((resp: EndTxnResponse) => resp.error),
+    ApiKeys.TXN_OFFSET_COMMIT -> ((resp: TxnOffsetCommitResponse) => resp.errors.get(tp))
   )
 
   val requestKeysToAcls = Map[ApiKeys, Map[Resource, Set[Acl]]](
     ApiKeys.METADATA -> topicDescribeAcl,
-    ApiKeys.PRODUCE -> topicWriteAcl,
+    ApiKeys.PRODUCE -> (topicWriteAcl ++ transactionIdWriteAcl ++ clusterIdempotentWriteAcl),
     ApiKeys.FETCH -> topicReadAcl,
     ApiKeys.LIST_OFFSETS -> topicDescribeAcl,
     ApiKeys.OFFSET_COMMIT -> (topicReadAcl ++ groupReadAcl),
     ApiKeys.OFFSET_FETCH -> (topicReadAcl ++ groupReadAcl),
-    ApiKeys.FIND_COORDINATOR -> (topicReadAcl ++ groupReadAcl),
+    ApiKeys.FIND_COORDINATOR -> (topicReadAcl ++ groupDescribeAcl ++ transactionalIdDescribeAcl),
     ApiKeys.UPDATE_METADATA_KEY -> clusterAcl,
     ApiKeys.JOIN_GROUP -> groupReadAcl,
     ApiKeys.SYNC_GROUP -> groupReadAcl,
@@ -163,7 +179,13 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
     ApiKeys.DELETE_TOPICS -> topicDeleteAcl,
     ApiKeys.OFFSET_FOR_LEADER_EPOCH -> clusterAcl,
     ApiKeys.DESCRIBE_CONFIGS -> topicDescribeConfigsAcl,
-    ApiKeys.ALTER_CONFIGS -> topicAlterConfigsAcl
+    ApiKeys.ALTER_CONFIGS -> topicAlterConfigsAcl,
+    ApiKeys.INIT_PRODUCER_ID -> (transactionIdWriteAcl ++ clusterIdempotentWriteAcl),
+    ApiKeys.WRITE_TXN_MARKERS -> clusterAcl,
+    ApiKeys.ADD_PARTITIONS_TO_TXN -> (topicWriteAcl ++ transactionIdWriteAcl),
+    ApiKeys.ADD_OFFSETS_TO_TXN -> (groupReadAcl ++ transactionIdWriteAcl),
+    ApiKeys.END_TXN -> transactionIdWriteAcl,
+    ApiKeys.TXN_OFFSET_COMMIT -> (groupReadAcl ++ transactionIdWriteAcl)
   )
 
   @Before
@@ -177,14 +199,6 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
         maxBlockMs = 3000,
         acks = 1)
 
-    val transactionalProperties = new Properties()
-    transactionalProperties.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true")
-    transactionalProperties.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId)
-    transactionalProducer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers),
-      retries = 3,
-      props = Some(transactionalProperties)
-    )
-
     for (_ <- 0 until consumerCount)
       consumers += TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers), groupId = group, securityProtocol = SecurityProtocol.PLAINTEXT)
 
@@ -204,7 +218,6 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
     producers.foreach(_.close())
     consumers.foreach(_.wakeup())
     consumers.foreach(_.close())
-    transactionalProducer.close()
     removeAllAcls()
     super.tearDown()
   }
@@ -338,17 +351,17 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
     )
 
     for ((key, request) <- requestKeyToRequest) {
-      removeAllAcls
+      removeAllAcls()
       val resources = requestKeysToAcls(key).map(_._1.resourceType).toSet
       sendRequestAndVerifyResponseError(key, request, resources, isAuthorized = false, isAuthorizedTopicDescribe = false)
 
       val resourceToAcls = requestKeysToAcls(key)
-      resourceToAcls.get(topicResource).map { acls =>
+      resourceToAcls.get(topicResource).foreach { acls =>
         val describeAcls = topicDescribeAcl(topicResource)
         val isAuthorized =  describeAcls == acls
         addAndVerifyAcls(describeAcls, topicResource)
         sendRequestAndVerifyResponseError(key, request, resources, isAuthorized = isAuthorized, isAuthorizedTopicDescribe = true)
-        removeAllAcls
+        removeAllAcls()
       }
 
       for ((resource, acls) <- resourceToAcls)
@@ -377,17 +390,17 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
     )
 
     for ((key, request) <- requestKeyToRequest) {
-      removeAllAcls
+      removeAllAcls()
       val resources = requestKeysToAcls(key).map(_._1.resourceType).toSet
       sendRequestAndVerifyResponseError(key, request, resources, isAuthorized = false, isAuthorizedTopicDescribe = false, topicExists = false)
 
       val resourceToAcls = requestKeysToAcls(key)
-      resourceToAcls.get(topicResource).map { acls =>
+      resourceToAcls.get(topicResource).foreach { acls =>
         val describeAcls = topicDescribeAcl(topicResource)
         val isAuthorized =  describeAcls == acls
         addAndVerifyAcls(describeAcls, topicResource)
         sendRequestAndVerifyResponseError(key, request, resources, isAuthorized = isAuthorized, isAuthorizedTopicDescribe = true, topicExists = false)
-        removeAllAcls
+        removeAllAcls()
       }
 
       for ((resource, acls) <- resourceToAcls)
@@ -852,44 +865,182 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
   }
 
   @Test(expected = classOf[TransactionalIdAuthorizationException])
-  def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnInitTransactions(): Unit = {
-    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), Resource.ProducerIdResource)
-    transactionalProducer.initTransactions()
+  def testTransactionalProducerInitTransactionsNoWriteTransactionalIdAcl(): Unit = {
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe)), transactionalIdResource)
+    val producer = buildTransactionalProducer()
+    producer.initTransactions()
+  }
+
+  @Test(expected = classOf[TransactionalIdAuthorizationException])
+  def testTransactionalProducerInitTransactionsNoDescribeTransactionalIdAcl(): Unit = {
+    val producer = buildTransactionalProducer()
+    producer.initTransactions()
+  }
+
+  @Test
+  def testSendOffsetsWithNoConsumerGroupDescribeAccess(): Unit = {
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, ClusterAction)), Resource.ClusterResource)
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), topicResource)
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), transactionalIdResource)
+    val producer = buildTransactionalProducer()
+    producer.initTransactions()
+    producer.beginTransaction()
+    try {
+      producer.sendOffsetsToTransaction(Map(new TopicPartition(topic, 0) -> new OffsetAndMetadata(0L)).asJava, group)
+      fail("Should have raised GroupAuthorizationException")
+    } catch {
+      case e: GroupAuthorizationException =>
+    }
+  }
+
+  @Test
+  def testSendOffsetsWithNoConsumerGroupWriteAccess(): Unit = {
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), transactionalIdResource)
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe)), groupResource)
+    val producer = buildTransactionalProducer()
+    producer.initTransactions()
+    producer.beginTransaction()
+    try {
+      producer.sendOffsetsToTransaction(Map(new TopicPartition(topic, 0) -> new OffsetAndMetadata(0L)).asJava, group)
+      fail("Should have raised GroupAuthorizationException")
+    } catch {
+      case e: GroupAuthorizationException =>
+    }
+  }
+
+  @Test
+  def testIdempotentProducerNoIdempotentWriteAclInInitProducerId(): Unit = {
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), topicResource)
+    val producer = buildIdempotentProducer()
+    try {
+      // the InitProducerId is sent asynchronously, so we expect the error either in the callback
+      // or raised from send itself
+      producer.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, "hi".getBytes)).get()
+      fail("Should have raised ClusterAuthorizationException")
+    } catch {
+      case e: ExecutionException =>
+        assertTrue(e.getCause.isInstanceOf[ClusterAuthorizationException])
+    }
+    try {
+      // the second time, the call to send itself should fail (the producer becomes unusable
+      // if no producerId can be obtained)
+      producer.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, "hi".getBytes))
+      fail("Should have raised ClusterAuthorizationException")
+    } catch {
+      case e: KafkaException =>
+        assertTrue(e.getCause.isInstanceOf[ClusterAuthorizationException])
+    }
+  }
+
+  @Test
+  def testIdempotentProducerNoIdempotentWriteAclInProduce(): Unit = {
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), topicResource)
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, IdempotentWrite)), Resource.ClusterResource)
+
+    val producer = buildIdempotentProducer()
+
+    // first send should be fine since we have permission to get a ProducerId
+    producer.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, "hi".getBytes)).get()
+
+    // revoke the IdempotentWrite permission
+    removeAllAcls()
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), topicResource)
+
+    try {
+      // the send should now fail with a cluster auth error
+      producer.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, "hi".getBytes)).get()
+      fail("Should have raised ClusterAuthorizationException")
+    } catch {
+      case e: ExecutionException =>
+        assertTrue(e.getCause.isInstanceOf[ClusterAuthorizationException])
+    }
+    try {
+      // the second time, the call to send itself should fail (the producer becomes unusable
+      // if no producerId can be obtained)
+      producer.send(new ProducerRecord[Array[Byte], Array[Byte]](topic, "hi".getBytes))
+      fail("Should have raised ClusterAuthorizationException")
+    } catch {
+      case e: KafkaException =>
+        assertTrue(e.getCause.isInstanceOf[ClusterAuthorizationException])
+    }
   }
 
   @Test
   def shouldInitTransactionsWhenAclSet(): Unit = {
-    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), producerTransactionalIdResource)
-    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), Resource.ProducerIdResource)
-    transactionalProducer.initTransactions()
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), transactionalIdResource)
+    val producer = buildTransactionalProducer()
+    producer.initTransactions()
   }
 
+  @Test
+  def testTransactionalProducerTopicAuthorizationExceptionInSendCallback(): Unit = {
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), transactionalIdResource)
+    // add describe access so that we can fetch metadata
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe)), topicResource)
+    val producer = buildTransactionalProducer()
+    producer.initTransactions()
+    producer.beginTransaction()
+    try {
+      producer.send(new ProducerRecord(tp.topic, tp.partition, "1".getBytes, "1".getBytes)).get
+      Assert.fail("expected TopicAuthorizationException")
+    } catch {
+      case e: ExecutionException =>
+        e.getCause match {
+          case cause: TopicAuthorizationException =>
+            assertEquals(Set(topic), cause.unauthorizedTopics().asScala)
+          case other =>
+            fail("Unexpected failure cause in send callback")
+        }
+    }
+  }
+
+  @Test
+  def testTransactionalProducerTopicAuthorizationExceptionInCommit(): Unit = {
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), transactionalIdResource)
+    // add describe access so that we can fetch metadata
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe)), topicResource)
+    val producer = buildTransactionalProducer()
+    producer.initTransactions()
+    producer.beginTransaction()
+    try {
+      producer.send(new ProducerRecord(tp.topic, tp.partition, "1".getBytes, "1".getBytes))
+      producer.commitTransaction()
+      Assert.fail("expected TopicAuthorizationException")
+    } catch {
+      case e: TopicAuthorizationException =>
+        assertEquals(Set(topic), e.unauthorizedTopics().asScala)
+    }
+  }
 
   @Test
   def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessDuringSend(): Unit = {
-    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), producerTransactionalIdResource)
-    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), Resource.ProducerIdResource)
-    transactionalProducer.initTransactions()
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), transactionalIdResource)
+    val producer = buildTransactionalProducer()
+    producer.initTransactions()
     removeAllAcls()
     addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), topicResource)
     try {
-      transactionalProducer.beginTransaction()
-      transactionalProducer.send(new ProducerRecord(tp.topic(), tp.partition(), "1".getBytes, "1".getBytes)).get
+      producer.beginTransaction()
+      producer.send(new ProducerRecord(tp.topic, tp.partition, "1".getBytes, "1".getBytes)).get
       Assert.fail("expected TransactionalIdAuthorizationException")
     } catch {
-      case e: ExecutionException => assertTrue(s"expected TransactionalIdAuthorizationException, but got ${e.getCause}", e.getCause.isInstanceOf[TransactionalIdAuthorizationException])
+      case e: ExecutionException => assertTrue(s"expected TransactionalIdAuthorizationException, but got ${e.getCause}",
+        e.getCause.isInstanceOf[TransactionalIdAuthorizationException])
     }
   }
 
   @Test
   def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnEndTransaction(): Unit = {
-    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), producerTransactionalIdResource)
-    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), Resource.ProducerIdResource)
-    transactionalProducer.initTransactions()
-    transactionalProducer.beginTransaction()
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), transactionalIdResource)
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), topicResource)
+    val producer = buildTransactionalProducer()
+    producer.initTransactions()
+    producer.beginTransaction()
+    producer.send(new ProducerRecord(tp.topic, tp.partition, "1".getBytes, "1".getBytes)).get
+    producer.flush()
     removeAllAcls()
     try {
-      transactionalProducer.commitTransaction()
+      producer.commitTransaction()
       Assert.fail("expected TransactionalIdAuthorizationException")
     } catch {
       case _: TransactionalIdAuthorizationException => // ok
@@ -898,50 +1049,27 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
 
   @Test
   def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnSendOffsetsToTxn(): Unit = {
-    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), producerTransactionalIdResource)
-    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), Resource.ProducerIdResource)
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), transactionalIdResource)
     addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), groupResource)
-    transactionalProducer.initTransactions()
-    transactionalProducer.beginTransaction()
+    val producer = buildTransactionalProducer()
+    producer.initTransactions()
+    producer.beginTransaction()
     removeAllAcls()
     try {
-      val offsets: util.Map[TopicPartition, OffsetAndMetadata] = Map(new TopicPartition(topicAndPartition.topic, topicAndPartition.partition) -> new OffsetAndMetadata(1L)).asJava
-      transactionalProducer.sendOffsetsToTransaction(offsets, group)
+      val offsets: util.Map[TopicPartition, OffsetAndMetadata] = Map(new TopicPartition(tp.topic, tp.partition) -> new OffsetAndMetadata(1L)).asJava
+      producer.sendOffsetsToTransaction(offsets, group)
       Assert.fail("expected TransactionalIdAuthorizationException")
     } catch {
       case _: TransactionalIdAuthorizationException => // ok
     }
   }
 
-
-  @Test
-  def shouldThrowProducerIdAuthorizationExceptionWhenAclNotSet(): Unit = {
-    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), topicResource)
-    val idempotentProperties = new Properties()
-    idempotentProperties.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true")
-    val idempotentProducer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers),
-      retries = 3,
-      props = Some(idempotentProperties)
-    )
-    try {
-      idempotentProducer.send(new ProducerRecord(tp.topic(), tp.partition(), "1".getBytes, "1".getBytes)).get
-      Assert.fail("expected ProducerIdAuthorizationException")
-    } catch {
-      case e: ExecutionException => assertTrue(s"expected ProducerIdAuthorizationException, but got ${e.getCause}", e.getCause.isInstanceOf[ProducerIdAuthorizationException])
-    }
-  }
-
   @Test
   def shouldSendSuccessfullyWhenIdempotentAndHasCorrectACL(): Unit = {
-    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), Resource.ProducerIdResource)
+    addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, IdempotentWrite)), Resource.ClusterResource)
     addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Write)), topicResource)
-    val idempotentProperties = new Properties()
-    idempotentProperties.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true")
-    val idempotentProducer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers),
-      retries = 3,
-      props = Some(idempotentProperties)
-    )
-    idempotentProducer.send(new ProducerRecord(tp.topic(), tp.partition(), "1".getBytes, "1".getBytes)).get
+    val producer = buildIdempotentProducer()
+    producer.send(new ProducerRecord(tp.topic, tp.partition, "1".getBytes, "1".getBytes)).get
   }
 
   def removeAllAcls() = {
@@ -1032,4 +1160,24 @@ class AuthorizerIntegrationTest extends BaseRequestTest {
     requests.OffsetFetchResponse.parse(response, request.version)
   }
 
+  private def buildTransactionalProducer(): KafkaProducer[Array[Byte], Array[Byte]] = {
+    val transactionalProperties = new Properties()
+    transactionalProperties.setProperty(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId)
+    val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers),
+      retries = 3,
+      props = Some(transactionalProperties))
+    producers += producer
+    producer
+  }
+
+  private def buildIdempotentProducer(): KafkaProducer[Array[Byte], Array[Byte]] = {
+    val idempotentProperties = new Properties()
+    idempotentProperties.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "true")
+    val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers),
+      retries = 3,
+      props = Some(idempotentProperties))
+    producers += producer
+    producer
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala
index 5d46348..1b88f40 100644
--- a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala
+++ b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala
@@ -77,17 +77,17 @@ class TransactionsBounceTest extends KafkaServerTestHarness {
   def testBrokerFailure() {
     // basic idea is to seed a topic with 10000 records, and copy it transactionally while bouncing brokers
     // constantly through the period.
-    val consumerGroup=  "myGroup"
+    val consumerGroup = "myGroup"
     val numInputRecords = 5000
     createTopics()
 
     TestUtils.seedTopicWithNumberedRecords(inputTopic, numInputRecords, servers)
-
     val consumer = createConsumerAndSubscribeToTopics(consumerGroup, List(inputTopic))
-    val producer = TestUtils.createTransactionalProducer("my-test-producer-t.id", servers)
+    val producer = TestUtils.createTransactionalProducer("test-txn", servers)
 
-    val scheduler = new BounceScheduler
     producer.initTransactions()
+
+    val scheduler = new BounceScheduler
     scheduler.start()
 
     var numMessagesProcessed = 0
@@ -97,16 +97,17 @@ class TransactionsBounceTest extends KafkaServerTestHarness {
         val toRead = Math.min(200, numInputRecords - numMessagesProcessed)
         trace(s"$iteration: About to read $toRead messages, processed $numMessagesProcessed so far..")
         val records = TestUtils.pollUntilAtLeastNumRecords(consumer, toRead)
-        trace(s"received ${records.size} messages. sending them transactionally to $outputTopic")
+        trace(s"Received ${records.size} messages, sending them transactionally to $outputTopic")
+
         producer.beginTransaction()
-        val shouldAbort = iteration % 2 == 0
-        records.zipWithIndex.foreach { case (record, i) =>
-          producer.send(
-            TestUtils.producerRecordWithExpectedTransactionStatus(outputTopic, record.key, record.value, !shouldAbort),
-            new ErrorLoggingCallback(outputTopic, record.key, record.value, true))
+        val shouldAbort = iteration % 3 == 0
+        records.foreach { record =>
+          producer.send(TestUtils.producerRecordWithExpectedTransactionStatus(outputTopic, record.key, record.value,
+            !shouldAbort), new ErrorLoggingCallback(outputTopic, record.key, record.value, true))
         }
         trace(s"Sent ${records.size} messages. Committing offsets.")
         producer.sendOffsetsToTransaction(TestUtils.consumerPositions(consumer), consumerGroup)
+
         if (shouldAbort) {
           trace(s"Committed offsets. Aborting transaction of ${records.size} messages.")
           producer.abortTransaction()
@@ -125,8 +126,8 @@ class TransactionsBounceTest extends KafkaServerTestHarness {
 
     scheduler.shutdown()
 
-    val verifyingConsumer = createConsumerAndSubscribeToTopics("randoGroup", List(outputTopic), readCommitted = true)
-    val outputRecords = TestUtils.pollUntilAtLeastNumRecords(verifyingConsumer, numInputRecords).map { case(record) =>
+    val verifyingConsumer = createConsumerAndSubscribeToTopics("randomGroup", List(outputTopic), readCommitted = true)
+    val outputRecords = TestUtils.pollUntilAtLeastNumRecords(verifyingConsumer, numInputRecords).map { record =>
       TestUtils.assertCommittedAndGetValue(record).toInt
     }
     val recordSet = outputRecords.toSet
@@ -142,7 +143,7 @@ class TransactionsBounceTest extends KafkaServerTestHarness {
     val props = new Properties()
     if (readCommitted)
       props.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed")
-    props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, "200")
+    props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, "2000")
     props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false")
     props.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "10000")
     props.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "3000")
@@ -157,8 +158,8 @@ class TransactionsBounceTest extends KafkaServerTestHarness {
   private def createTopics() =  {
     val topicConfig = new Properties()
     topicConfig.put(KafkaConfig.MinInSyncReplicasProp, 2.toString)
-    TestUtils.createTopic(zkUtils, inputTopic, numPartitions, numServers, servers, topicConfig)
-    TestUtils.createTopic(zkUtils, outputTopic, numPartitions, numServers, servers, topicConfig)
+    TestUtils.createTopic(zkUtils, inputTopic, numPartitions, 3, servers, topicConfig)
+    TestUtils.createTopic(zkUtils, outputTopic, numPartitions, 3, servers, topicConfig)
   }
 
   private class BounceScheduler extends ShutdownableThread("daemon-broker-bouncer", false) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/core/src/test/scala/integration/kafka/api/TransactionsTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala
index ec6b3ea..fd9d884 100644
--- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala
+++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala
@@ -120,15 +120,15 @@ class TransactionsTest extends KafkaServerTestHarness {
     consumer.subscribe(List(topic1))
     producer.initTransactions()
 
-    val random = new Random()
     var shouldCommit = false
     var recordsProcessed = 0
     try {
       while (recordsProcessed < numSeedMessages) {
+        val records = TestUtils.pollUntilAtLeastNumRecords(consumer, Math.min(10, numSeedMessages - recordsProcessed))
+
         producer.beginTransaction()
         shouldCommit = !shouldCommit
 
-        val records = TestUtils.pollUntilAtLeastNumRecords(consumer, Math.min(10, numSeedMessages - recordsProcessed))
         records.zipWithIndex.foreach { case (record, i) =>
           val key = new String(record.key(), "UTF-8")
           val value = new String(record.value(), "UTF-8")
@@ -153,7 +153,7 @@ class TransactionsTest extends KafkaServerTestHarness {
       consumer.close()
     }
 
-    // Inspite of random aborts, we should still have exactly 1000 messages in topic2. Ie. we should not
+    // In spite of random aborts, we should still have exactly 1000 messages in topic2. Ie. we should not
     // re-copy or miss any messages from topic1, since the consumed offsets were committed transactionally.
     val verifyingConsumer = transactionalConsumer("foobargroup")
     verifyingConsumer.subscribe(List(topic2))
@@ -334,7 +334,6 @@ class TransactionsTest extends KafkaServerTestHarness {
         fail("Should not be able to send messages from a fenced producer.")
       } catch {
         case e : ProducerFencedException =>
-          producer1.close()
         case e : ExecutionException =>
           assertTrue(e.getCause.isInstanceOf[ProducerFencedException])
         case e : Exception =>

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/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 f5b0a06..f379585 100644
--- a/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala
@@ -36,26 +36,33 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging {
   private val TopicResources = Set(new Resource(Topic, "test-1"), new Resource(Topic, "test-2"))
   private val GroupResources = Set(new Resource(Group, "testGroup-1"), new Resource(Group, "testGroup-2"))
   private val BrokerResources = Set(new Resource(Broker, "0"), new Resource(Broker, "1"))
+  private val TransactionalIdResources = Set(new Resource(TransactionalId, "t0"), new Resource(TransactionalId, "t1"))
 
   private val ResourceToCommand = Map[Set[Resource], Array[String]](
     TopicResources -> Array("--topic", "test-1", "--topic", "test-2"),
     Set(Resource.ClusterResource) -> Array("--cluster"),
     GroupResources -> Array("--group", "testGroup-1", "--group", "testGroup-2"),
-    BrokerResources -> Array("--broker", "0", "--broker", "1")
+    BrokerResources -> Array("--broker", "0", "--broker", "1"),
+    TransactionalIdResources -> Array("--transactional-id", "t0", "--transactional-id", "t1")
   )
 
   private val ResourceToOperations = Map[Set[Resource], (Set[Operation], Array[String])](
     TopicResources -> (Set(Read, Write, Describe, Delete, DescribeConfigs, AlterConfigs),
       Array("--operation", "Read" , "--operation", "Write", "--operation", "Describe", "--operation", "Delete",
         "--operation", "DescribeConfigs", "--operation", "AlterConfigs")),
-    Set(Resource.ClusterResource) -> (Set(Create, ClusterAction), Array("--operation", "Create", "--operation", "ClusterAction")),
+    Set(Resource.ClusterResource) -> (Set(Create, ClusterAction, DescribeConfigs, AlterConfigs, IdempotentWrite),
+      Array("--operation", "Create", "--operation", "ClusterAction", "--operation", "DescribeConfigs",
+        "--operation", "AlterConfigs", "--operation", "IdempotentWrite")),
     GroupResources -> (Set(Read, Describe), Array("--operation", "Read", "--operation", "Describe")),
-    BrokerResources -> (Set(DescribeConfigs), Array("--operation", "DescribeConfigs"))
+    BrokerResources -> (Set(DescribeConfigs), Array("--operation", "DescribeConfigs")),
+    TransactionalIdResources -> (Set(Describe, Write), Array("--operation", "Describe", "--operation", "Write"))
   )
 
-  private val ProducerResourceToAcls = Map[Set[Resource], Set[Acl]](
+  private def ProducerResourceToAcls(enableIdempotence: Boolean = false) = Map[Set[Resource], Set[Acl]](
     TopicResources -> AclCommand.getAcls(Users, Allow, Set(Write, Describe), Hosts),
-    Set(Resource.ClusterResource) -> AclCommand.getAcls(Users, Allow, Set(Create), Hosts)
+    TransactionalIdResources -> AclCommand.getAcls(Users, Allow, Set(Write, Describe), Hosts),
+    Set(Resource.ClusterResource) -> AclCommand.getAcls(Users, Allow, Set(Some(Create),
+      if (enableIdempotence) Some(IdempotentWrite) else None).flatten, Hosts)
   )
 
   private val ConsumerResourceToAcls = Map[Set[Resource], Set[Acl]](
@@ -64,10 +71,13 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging {
   )
 
   private val CmdToResourcesToAcl = Map[Array[String], Map[Set[Resource], Set[Acl]]](
-    Array[String]("--producer") -> ProducerResourceToAcls,
+    Array[String]("--producer") -> ProducerResourceToAcls(),
+    Array[String]("--producer", "--idempotent") -> ProducerResourceToAcls(enableIdempotence = true),
     Array[String]("--consumer") -> ConsumerResourceToAcls,
     Array[String]("--producer", "--consumer") -> ConsumerResourceToAcls.map { case (k, v) => k -> (v ++
-      ProducerResourceToAcls.getOrElse(k, Set.empty[Acl])) }
+      ProducerResourceToAcls().getOrElse(k, Set.empty[Acl])) },
+    Array[String]("--producer", "--idempotent", "--consumer") -> ConsumerResourceToAcls.map { case (k, v) => k -> (v ++
+      ProducerResourceToAcls(enableIdempotence = true).getOrElse(k, Set.empty[Acl])) }
   )
 
   @Test
@@ -108,11 +118,11 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging {
           }
         }
       }
-      testRemove(resourcesToAcls.keys.flatten.toSet, resourceCommand, args, brokerProps)
+      testRemove(resourcesToAcls.keys.flatten.toSet, resourceCommand ++ cmd, args, brokerProps)
     }
   }
 
-  @Test (expected = classOf[IllegalArgumentException])
+  @Test(expected = classOf[IllegalArgumentException])
   def testInvalidAuthorizerProperty() {
     val args = Array("--authorizer-properties", "zookeeper.connect " + zkConnect)
     AclCommand.withAuthorizer(new AclCommandOptions(args))(null)
@@ -120,10 +130,10 @@ class AclCommandTest extends ZooKeeperTestHarness with Logging {
 
   private def testRemove(resources: Set[Resource], resourceCmd: Array[String], args: Array[String], brokerProps: Properties) {
     for (resource <- resources) {
-        AclCommand.main(args ++ resourceCmd :+ "--remove" :+ "--force")
-        withAuthorizer(brokerProps) { authorizer =>
-          TestUtils.waitAndVerifyAcls(Set.empty[Acl], authorizer, resource)
-        }
+      AclCommand.main(args ++ resourceCmd :+ "--remove" :+ "--force")
+      withAuthorizer(brokerProps) { authorizer =>
+        TestUtils.waitAndVerifyAcls(Set.empty[Acl], authorizer, resource)
+      }
     }
   }
 


[5/5] kafka git commit: KAFKA-5259; TransactionalId auth implies ProducerId auth

Posted by jg...@apache.org.
KAFKA-5259; TransactionalId auth implies ProducerId auth

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

Reviewers: Apurva Mehta <ap...@confluent.io>, Jun Rao <ju...@gmail.com>

Closes #3075 from hachikuji/KAFKA-5259-FIXED


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

Branch: refs/heads/trunk
Commit: 38f6cae9e879baa35c5dbc5829bf09ecd59930c2
Parents: 8820093
Author: Jason Gustafson <ja...@confluent.io>
Authored: Wed May 24 15:26:46 2017 -0700
Committer: Jason Gustafson <ja...@confluent.io>
Committed: Wed May 24 15:26:46 2017 -0700

----------------------------------------------------------------------
 .../apache/kafka/clients/ClientResponse.java    |    7 +-
 .../kafka/clients/admin/AclOperation.java       |    7 +-
 .../clients/consumer/internals/Fetcher.java     |    2 +-
 .../kafka/clients/producer/KafkaProducer.java   |   35 +-
 .../clients/producer/internals/Sender.java      |  169 +-
 .../producer/internals/TransactionManager.java  |  268 ++--
 .../ProducerIdAuthorizationException.java       |   23 -
 .../TransactionalIdAuthorizationException.java  |    2 +-
 .../apache/kafka/common/protocol/Errors.java    |   13 +-
 .../apache/kafka/common/protocol/Protocol.java  |    3 +
 .../common/requests/AddOffsetsToTxnRequest.java |   11 +
 .../requests/AddOffsetsToTxnResponse.java       |    5 +-
 .../requests/AddPartitionsToTxnRequest.java     |   11 +
 .../requests/AddPartitionsToTxnResponse.java    |    3 +-
 .../kafka/common/requests/EndTxnRequest.java    |   11 +
 .../kafka/common/requests/EndTxnResponse.java   |    3 +-
 .../common/requests/InitProducerIdResponse.java |   12 +-
 .../kafka/common/requests/ProduceRequest.java   |    5 +-
 .../kafka/common/requests/ProduceResponse.java  |    3 +
 .../common/requests/TxnOffsetCommitRequest.java |   31 +-
 .../requests/TxnOffsetCommitResponse.java       |    1 +
 .../requests/WriteTxnMarkersResponse.java       |    1 +
 .../kafka/clients/admin/AclOperationTest.java   |    3 +-
 .../clients/producer/internals/SenderTest.java  |  183 +--
 .../internals/TransactionManagerTest.java       |  445 ++++--
 .../common/requests/RequestResponseTest.java    |   10 +-
 .../src/main/scala/kafka/admin/AclCommand.scala |   54 +-
 .../kafka/coordinator/group/GroupMetadata.scala |   43 +-
 .../group/GroupMetadataManager.scala            |    5 +-
 .../coordinator/group/MemberMetadata.scala      |   14 +-
 .../scala/kafka/security/auth/Operation.scala   |    6 +-
 .../scala/kafka/security/auth/Resource.scala    |    3 +-
 .../kafka/security/auth/ResourceType.scala      |   16 +-
 .../src/main/scala/kafka/server/KafkaApis.scala |  316 ++--
 .../kafka/api/AuthorizerIntegrationTest.scala   |  292 +++-
 .../kafka/api/TransactionsBounceTest.scala      |   31 +-
 .../kafka/api/TransactionsTest.scala            |    7 +-
 .../scala/unit/kafka/admin/AclCommandTest.scala |   36 +-
 .../group/GroupCoordinatorResponseTest.scala    | 1492 ------------------
 .../group/GroupCoordinatorTest.scala            | 1492 ++++++++++++++++++
 .../coordinator/group/GroupMetadataTest.scala   |    2 +-
 .../unit/kafka/server/RequestQuotaTest.scala    |    9 +-
 .../test/scala/unit/kafka/utils/TestUtils.scala |   10 +-
 43 files changed, 2771 insertions(+), 2324 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java b/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java
index 715eae7..0ff30e9 100644
--- a/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java
+++ b/clients/src/main/java/org/apache/kafka/clients/ClientResponse.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.clients;
 
+import org.apache.kafka.common.errors.UnsupportedVersionException;
 import org.apache.kafka.common.requests.AbstractResponse;
 import org.apache.kafka.common.requests.RequestHeader;
 
@@ -31,7 +32,7 @@ public class ClientResponse {
     private final long receivedTimeMs;
     private final long latencyMs;
     private final boolean disconnected;
-    private final RuntimeException versionMismatch;
+    private final UnsupportedVersionException versionMismatch;
     private final AbstractResponse responseBody;
 
     /**
@@ -51,7 +52,7 @@ public class ClientResponse {
                           long createdTimeMs,
                           long receivedTimeMs,
                           boolean disconnected,
-                          RuntimeException versionMismatch,
+                          UnsupportedVersionException versionMismatch,
                           AbstractResponse responseBody) {
         this.requestHeader = requestHeader;
         this.callback = callback;
@@ -71,7 +72,7 @@ public class ClientResponse {
         return disconnected;
     }
 
-    public RuntimeException versionMismatch() {
+    public UnsupportedVersionException versionMismatch() {
         return versionMismatch;
     }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/clients/admin/AclOperation.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AclOperation.java b/clients/src/main/java/org/apache/kafka/clients/admin/AclOperation.java
index 062e5e3..0c3ff50 100644
--- a/clients/src/main/java/org/apache/kafka/clients/admin/AclOperation.java
+++ b/clients/src/main/java/org/apache/kafka/clients/admin/AclOperation.java
@@ -83,7 +83,12 @@ public enum AclOperation {
     /**
      * ALTER_CONFIGS operation.
      */
-    ALTER_CONFIGS((byte) 11);
+    ALTER_CONFIGS((byte) 11),
+
+    /**
+     * IDEMPOTENT_WRITE operation.
+     */
+    IDEMPOTENT_WRITE((byte) 12);
 
     private final static HashMap<Byte, AclOperation> CODE_TO_VALUE = new HashMap<>();
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/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 509993f..6917a1d 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
@@ -416,7 +416,7 @@ public class Fetcher<K, V> implements SubscriptionState.Listener, Closeable {
             }
             // we might lose the assignment while fetching the offset, so check it is still active
             if (subscriptions.isAssigned(partition)) {
-                log.debug("Resetting offset for partition {} to {} offset.", partition, offsetData.offset);
+                log.debug("Resetting offset for partition {} to offset {}.", partition, offsetData.offset);
                 this.subscriptions.seek(partition, offsetData.offset);
             }
         }

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
index ac0169a..c11ecc7 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
@@ -51,7 +51,6 @@ import org.apache.kafka.common.metrics.MetricsReporter;
 import org.apache.kafka.common.metrics.Sensor;
 import org.apache.kafka.common.network.ChannelBuilder;
 import org.apache.kafka.common.network.Selector;
-import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.record.AbstractRecords;
 import org.apache.kafka.common.record.CompressionType;
 import org.apache.kafka.common.record.RecordBatch;
@@ -607,7 +606,9 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
      * Implementation of asynchronously send a record to a topic.
      */
     private Future<RecordMetadata> doSend(ProducerRecord<K, V> record, Callback callback) {
-        ensureProperTransactionalState();
+        if (transactionManager != null)
+            ensureProperTransactionalState();
+
         TopicPartition tp = null;
         try {
             // first make sure the metadata for the topic is available
@@ -642,9 +643,9 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
             long timestamp = record.timestamp() == null ? time.milliseconds() : record.timestamp();
             log.trace("Sending record {} with callback {} to topic {} partition {}", record, callback, record.topic(), partition);
             // producer callback will make sure to call both 'callback' and interceptor callback
-            Callback interceptCallback = new InterceptorCallback<>(callback, this.interceptors, tp, transactionManager);
+            Callback interceptCallback = this.interceptors == null ? callback : new InterceptorCallback<>(callback, this.interceptors, tp);
 
-            if (transactionManager != null)
+            if (transactionManager != null && transactionManager.isTransactional())
                 transactionManager.maybeAddPartitionToTransaction(tp);
 
             RecordAccumulator.RecordAppendResult result = accumulator.append(tp, timestamp, serializedKey,
@@ -690,27 +691,17 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
     }
 
     private void ensureProperTransactionalState() {
-        if (transactionManager == null)
-            return;
-
         if (transactionManager.isTransactional() && !transactionManager.hasProducerId())
-            throw new IllegalStateException("Cannot perform a 'send' before completing a call to initTransactions when transactions are enabled.");
-
-        if (transactionManager.isFenced())
-            throw Errors.INVALID_PRODUCER_EPOCH.exception();
+            throw new IllegalStateException("Cannot perform a 'send' before completing a call to initTransactions " +
+                    "when transactions are enabled.");
 
         if (transactionManager.isInErrorState()) {
-            String errorMessage =
-                    "Cannot perform send because at least one previous transactional or idempotent request has failed with errors.";
             Exception lastError = transactionManager.lastError();
-            if (lastError != null)
-                throw new KafkaException(errorMessage, lastError);
-            else
-                throw new KafkaException(errorMessage);
+            throw new KafkaException("Cannot perform send because at least one previous transactional or " +
+                    "idempotent request has failed with errors.", lastError);
         }
         if (transactionManager.isCompletingTransaction())
             throw new IllegalStateException("Cannot call send while a commit or abort is in progress.");
-
     }
 
     private void setReadOnly(Headers headers) {
@@ -1013,14 +1004,11 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
         private final Callback userCallback;
         private final ProducerInterceptors<K, V> interceptors;
         private final TopicPartition tp;
-        private final TransactionManager transactionManager;
 
-        public InterceptorCallback(Callback userCallback, ProducerInterceptors<K, V> interceptors,
-                                   TopicPartition tp, TransactionManager transactionManager) {
+        private InterceptorCallback(Callback userCallback, ProducerInterceptors<K, V> interceptors, TopicPartition tp) {
             this.userCallback = userCallback;
             this.interceptors = interceptors;
             this.tp = tp;
-            this.transactionManager = transactionManager;
         }
 
         public void onCompletion(RecordMetadata metadata, Exception exception) {
@@ -1034,9 +1022,6 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
             }
             if (this.userCallback != null)
                 this.userCallback.onCompletion(metadata, exception);
-
-            if (exception != null && transactionManager != null)
-                transactionManager.setError(exception);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
index 4c3b99d..116a1c5 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
@@ -24,15 +24,18 @@ import org.apache.kafka.clients.Metadata;
 import org.apache.kafka.clients.NetworkClientUtils;
 import org.apache.kafka.clients.RequestCompletionHandler;
 import org.apache.kafka.common.Cluster;
-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.ClusterAuthorizationException;
 import org.apache.kafka.common.errors.InvalidMetadataException;
 import org.apache.kafka.common.errors.OutOfOrderSequenceException;
+import org.apache.kafka.common.errors.ProducerFencedException;
 import org.apache.kafka.common.errors.RetriableException;
 import org.apache.kafka.common.errors.TopicAuthorizationException;
+import org.apache.kafka.common.errors.TransactionalIdAuthorizationException;
 import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
+import org.apache.kafka.common.errors.UnsupportedVersionException;
 import org.apache.kafka.common.metrics.Measurable;
 import org.apache.kafka.common.metrics.MetricConfig;
 import org.apache.kafka.common.metrics.Metrics;
@@ -189,28 +192,34 @@ public class Sender implements Runnable {
      * @param now The current POSIX time in milliseconds
      */
     void run(long now) {
-        long pollTimeout = retryBackoffMs;
-        if (!maybeSendTransactionalRequest(now)) {
-            pollTimeout = sendProducerData(now);
+        if (transactionManager != null) {
+            if (!transactionManager.isTransactional()) {
+                // this is an idempotent producer, so make sure we have a producer id
+                maybeWaitForProducerId();
+            } else if (transactionManager.hasInflightRequest() || maybeSendTransactionalRequest(now)) {
+                // as long as there are outstanding transactional requests, we simply wait for them to return
+                client.poll(retryBackoffMs, now);
+                return;
+            }
+
+            // do not continue sending if the transaction manager is in a failed state or if there
+            // is no producer id (for the idempotent case).
+            if (transactionManager.isInErrorState() || !transactionManager.hasProducerId()) {
+                RuntimeException lastError = transactionManager.lastError();
+                if (lastError != null)
+                    maybeAbortBatches(lastError);
+                client.poll(retryBackoffMs, now);
+                return;
+            }
         }
 
+        long pollTimeout = sendProducerData(now);
         log.trace("waiting {}ms in poll", pollTimeout);
-        this.client.poll(pollTimeout, now);
+        client.poll(pollTimeout, now);
     }
 
-
     private long sendProducerData(long now) {
         Cluster cluster = metadata.fetch();
-        maybeWaitForProducerId();
-
-        if (transactionManager != null && transactionManager.isInErrorState()) {
-            final KafkaException exception = transactionManager.lastError() instanceof KafkaException
-                    ? (KafkaException) transactionManager.lastError()
-                    : new KafkaException(transactionManager.lastError());
-            log.error("aborting producer batches because the transaction manager is in an error state.", exception);
-            this.accumulator.abortBatches(exception);
-            return Long.MAX_VALUE;
-        }
         // get the list of partitions with data ready to send
         RecordAccumulator.ReadyCheckResult result = this.accumulator.ready(cluster, now);
 
@@ -286,22 +295,13 @@ public class Sender implements Runnable {
     }
 
     private boolean maybeSendTransactionalRequest(long now) {
-        if (transactionManager == null || !transactionManager.isTransactional())
-            return false;
-
-        if (transactionManager.hasInflightRequest()) {
-            log.trace("TransactionalId: {} -- There is already an inflight transactional request. Going to wait for the response.",
+        TransactionManager.TxnRequestHandler nextRequestHandler = transactionManager.nextRequestHandler();
+        if (nextRequestHandler == null) {
+            log.trace("TransactionalId: {} -- There are no pending transactional requests to send",
                     transactionManager.transactionalId());
-            return true;
-        }
-
-        if (!transactionManager.hasPendingTransactionalRequests()) {
-            log.trace("TransactionalId: {} -- There are no pending transactional requests to send", transactionManager.transactionalId());
             return false;
         }
 
-        TransactionManager.TxnRequestHandler nextRequestHandler = transactionManager.nextRequestHandler();
-
         if (nextRequestHandler.isEndTxn() && transactionManager.isCompletingTransaction() && accumulator.hasUnflushedBatches()) {
             if (!accumulator.flushInProgress())
                 accumulator.beginFlush();
@@ -311,15 +311,11 @@ public class Sender implements Runnable {
             return false;
         }
 
-        if (transactionManager.maybeTerminateRequestWithError(nextRequestHandler)) {
-            log.trace("TransactionalId: {} -- Not sending a transactional request because we are in an error state",
-                    transactionManager.transactionalId());
-            return false;
-        }
-
-        Node targetNode = null;
+        log.debug("TransactionalId: {} -- Sending transactional request {}", transactionManager.transactionalId(),
+                nextRequestHandler.requestBuilder());
 
-        while (targetNode == null) {
+        while (true) {
+            Node targetNode = null;
             try {
                 if (nextRequestHandler.needsCoordinator()) {
                     targetNode = transactionManager.coordinator(nextRequestHandler.coordinatorType());
@@ -340,8 +336,8 @@ public class Sender implements Runnable {
                                 transactionManager.transactionalId(), retryBackoffMs, nextRequestHandler.requestBuilder());
                         time.sleep(retryBackoffMs);
                     }
-                    ClientRequest clientRequest = client.newClientRequest(targetNode.idString(), nextRequestHandler.requestBuilder(),
-                            now, true, nextRequestHandler);
+                    ClientRequest clientRequest = client.newClientRequest(targetNode.idString(),
+                            nextRequestHandler.requestBuilder(), now, true, nextRequestHandler);
                     transactionManager.setInFlightRequestCorrelationId(clientRequest.correlationId());
                     log.trace("TransactionalId: {} -- Sending transactional request {} to node {}", transactionManager.transactionalId(),
                             nextRequestHandler.requestBuilder(), clientRequest.destination());
@@ -349,9 +345,9 @@ public class Sender implements Runnable {
                     return true;
                 }
             } catch (IOException e) {
-                targetNode = null;
-                log.warn("TransactionalId: " + transactionManager.transactionalId() + " -- Got an exception when trying " +
-                        "to find a node to send transactional request " + nextRequestHandler.requestBuilder() + ". Going to back off and retry", e);
+                log.debug("TransactionalId: {} -- Disconnect from {} while trying to send transactional " +
+                                "request {}. Going to back off and retry", transactionManager.transactionalId(),
+                        targetNode, nextRequestHandler.requestBuilder());
             }
             log.trace("TransactionalId: {}. About to wait for {}ms before trying to send another transactional request.",
                     transactionManager.transactionalId(), retryBackoffMs);
@@ -364,6 +360,13 @@ public class Sender implements Runnable {
         return true;
     }
 
+    private void maybeAbortBatches(RuntimeException exception) {
+        if (accumulator.hasUnflushedBatches()) {
+            log.error("Aborting producer batches due to fatal error", exception);
+            accumulator.abortBatches(exception);
+        }
+    }
+
     /**
      * Start closing the sender (won't actually complete until all data is sent out)
      */
@@ -383,7 +386,7 @@ public class Sender implements Runnable {
         initiateClose();
     }
 
-    private ClientResponse sendAndAwaitInitPidRequest(Node node) throws IOException {
+    private ClientResponse sendAndAwaitInitProducerIdRequest(Node node) throws IOException {
         String nodeId = node.idString();
         InitProducerIdRequest.Builder builder = new InitProducerIdRequest.Builder(null);
         ClientRequest request = client.newClientRequest(nodeId, builder, time.milliseconds(), true, null);
@@ -399,43 +402,37 @@ public class Sender implements Runnable {
     }
 
     private void maybeWaitForProducerId() {
-        // If this is a transactional producer, the producer id will be received when recovering transactions in the
-        // initTransactions() method of the producer.
-        if (transactionManager == null || transactionManager.isTransactional())
-            return;
-
         while (!transactionManager.hasProducerId() && !transactionManager.isInErrorState()) {
             try {
                 Node node = awaitLeastLoadedNodeReady(requestTimeout);
                 if (node != null) {
-                    ClientResponse response = sendAndAwaitInitPidRequest(node);
-
-                    if (response.hasResponse() && (response.responseBody() instanceof InitProducerIdResponse)) {
-                        InitProducerIdResponse initProducerIdResponse = (InitProducerIdResponse) response.responseBody();
-                        Exception exception = initProducerIdResponse.error().exception();
-                        if (exception != null && !(exception instanceof  RetriableException)) {
-                            transactionManager.setError(exception);
-                            return;
-                        }
+                    ClientResponse response = sendAndAwaitInitProducerIdRequest(node);
+                    InitProducerIdResponse initProducerIdResponse = (InitProducerIdResponse) response.responseBody();
+                    Errors error = initProducerIdResponse.error();
+                    if (error == Errors.NONE) {
                         ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(
                                 initProducerIdResponse.producerId(), initProducerIdResponse.epoch());
                         transactionManager.setProducerIdAndEpoch(producerIdAndEpoch);
+                    } else if (error.exception() instanceof RetriableException) {
+                        log.debug("Retriable error from InitProducerId response", error.message());
                     } else {
-                        log.error("Received an unexpected response type for an InitProducerIdRequest from {}. " +
-                                "We will back off and try again.", node);
+                        transactionManager.transitionToFatalError(error.exception());
+                        break;
                     }
                 } else {
                     log.debug("Could not find an available broker to send InitProducerIdRequest to. " +
                             "We will back off and try again.");
                 }
-            } catch (Exception e) {
-                log.warn("Received an exception while trying to get a producer id. Will back off and retry.", e);
+            } catch (UnsupportedVersionException e) {
+                transactionManager.transitionToFatalError(e);
+                break;
+            } catch (IOException e) {
+                log.debug("Broker {} disconnected while awaiting InitProducerId response", e);
             }
             log.trace("Retry InitProducerIdRequest in {}ms.", retryBackoffMs);
             time.sleep(retryBackoffMs);
             metadata.requestUpdate();
         }
-
     }
 
     /**
@@ -507,9 +504,9 @@ public class Sender implements Runnable {
                         error);
                 if (transactionManager == null) {
                     reenqueueBatch(batch, now);
-                } else if (transactionManager.producerIdAndEpoch().producerId == batch.producerId() &&
-                        transactionManager.producerIdAndEpoch().epoch == batch.producerEpoch()) {
-                    // If idempotence is enabled only retry the request if the current producer id is the same as the producer id of the batch.
+                } else if (transactionManager.hasProducerIdAndEpoch(batch.producerId(), batch.producerEpoch())) {
+                    // If idempotence is enabled only retry the request if the current producer id is the same as
+                    // the producer id of the batch.
                     log.debug("Retrying batch to topic-partition {}. Sequence number : {}", batch.topicPartition,
                             transactionManager.sequenceNumber(batch.topicPartition));
                     reenqueueBatch(batch, now);
@@ -523,12 +520,10 @@ public class Sender implements Runnable {
                 final RuntimeException exception;
                 if (error == Errors.TOPIC_AUTHORIZATION_FAILED)
                     exception = new TopicAuthorizationException(batch.topicPartition.topic());
+                else if (error == Errors.CLUSTER_AUTHORIZATION_FAILED)
+                    exception = new ClusterAuthorizationException("The producer is not authorized to do idempotent sends");
                 else
                     exception = error.exception();
-                if (error == Errors.OUT_OF_ORDER_SEQUENCE_NUMBER && batch.producerId() == transactionManager.producerIdAndEpoch().producerId)
-                    log.error("The broker received an out of order sequence number for correlation id {}, topic-partition " +
-                                    "{} at offset {}. This indicates data loss on the broker, and should be investigated.",
-                            correlationId, batch.topicPartition, response.baseOffset);
                 // tell the user the result of their request
                 failBatch(batch, response, exception);
                 this.sensors.recordErrors(batch.topicPartition.topic(), batch.recordCount);
@@ -543,12 +538,6 @@ public class Sender implements Runnable {
         } else {
             completeBatch(batch, response);
 
-            if (transactionManager != null && transactionManager.producerIdAndEpoch().producerId == batch.producerId()
-                    && transactionManager.producerIdAndEpoch().epoch == batch.producerEpoch()) {
-                transactionManager.incrementSequenceNumber(batch.topicPartition, batch.recordCount);
-                log.debug("Incremented sequence number for topic-partition {} to {}", batch.topicPartition,
-                        transactionManager.sequenceNumber(batch.topicPartition));
-            }
         }
 
         // Unmute the completed partition.
@@ -562,18 +551,38 @@ public class Sender implements Runnable {
     }
 
     private void completeBatch(ProducerBatch batch, ProduceResponse.PartitionResponse response) {
+        if (transactionManager != null && transactionManager.hasProducerIdAndEpoch(batch.producerId(), batch.producerEpoch())) {
+            transactionManager.incrementSequenceNumber(batch.topicPartition, batch.recordCount);
+            log.debug("Incremented sequence number for topic-partition {} to {}", batch.topicPartition,
+                    transactionManager.sequenceNumber(batch.topicPartition));
+        }
+
         batch.done(response.baseOffset, response.logAppendTime, null);
         this.accumulator.deallocate(batch);
     }
 
     private void failBatch(ProducerBatch batch, ProduceResponse.PartitionResponse response, RuntimeException exception) {
-        if (transactionManager != null && !transactionManager.isTransactional()
-                && batch.producerId() == transactionManager.producerIdAndEpoch().producerId) {
-            // Reset the transaction state since we have hit an irrecoverable exception and cannot make any guarantees
-            // about the previously committed message. Note that this will discard the producer id and sequence
-            // numbers for all existing partitions.
-            transactionManager.resetProducerId();
+        if (transactionManager != null) {
+            if (exception instanceof OutOfOrderSequenceException
+                    && !transactionManager.isTransactional()
+                    && transactionManager.hasProducerId(batch.producerId())) {
+                log.error("The broker received an out of order sequence number for topic-partition " +
+                                "{} at offset {}. This indicates data loss on the broker, and should be investigated.",
+                        batch.topicPartition, response.baseOffset);
+
+                // Reset the transaction state since we have hit an irrecoverable exception and cannot make any guarantees
+                // about the previously committed message. Note that this will discard the producer id and sequence
+                // numbers for all existing partitions.
+                transactionManager.resetProducerId();
+            } else if (exception instanceof ClusterAuthorizationException
+                    || exception instanceof TransactionalIdAuthorizationException
+                    || exception instanceof ProducerFencedException) {
+                transactionManager.transitionToFatalError(exception);
+            } else if (transactionManager.isTransactional()) {
+                transactionManager.transitionToAbortableError(exception);
+            }
         }
+
         batch.done(response.baseOffset, response.logAppendTime, exception);
         this.accumulator.deallocate(batch);
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
index d84a88e..d674697 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/TransactionManager.java
@@ -23,7 +23,7 @@ import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.Node;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.errors.GroupAuthorizationException;
-import org.apache.kafka.common.errors.ProducerFencedException;
+import org.apache.kafka.common.errors.TopicAuthorizationException;
 import org.apache.kafka.common.protocol.Errors;
 import org.apache.kafka.common.requests.AbstractRequest;
 import org.apache.kafka.common.requests.AbstractResponse;
@@ -77,7 +77,7 @@ public class TransactionManager {
     private Node consumerGroupCoordinator;
 
     private volatile State currentState = State.UNINITIALIZED;
-    private volatile Exception lastError = null;
+    private volatile RuntimeException lastError = null;
     private volatile ProducerIdAndEpoch producerIdAndEpoch;
 
     private enum State {
@@ -87,32 +87,34 @@ public class TransactionManager {
         IN_TRANSACTION,
         COMMITTING_TRANSACTION,
         ABORTING_TRANSACTION,
-        FENCED,
-        ERROR;
+        ABORTABLE_ERROR,
+        FATAL_ERROR;
 
         private boolean isTransitionValid(State source, State target) {
             switch (target) {
                 case INITIALIZING:
-                    return source == UNINITIALIZED || source == ERROR;
+                    return source == UNINITIALIZED;
                 case READY:
                     return source == INITIALIZING || source == COMMITTING_TRANSACTION
-                            || source == ABORTING_TRANSACTION || source == ERROR;
+                            || source == ABORTING_TRANSACTION || source == ABORTABLE_ERROR;
                 case IN_TRANSACTION:
                     return source == READY;
                 case COMMITTING_TRANSACTION:
                     return source == IN_TRANSACTION;
                 case ABORTING_TRANSACTION:
-                    return source == IN_TRANSACTION || source == ERROR;
+                    return source == IN_TRANSACTION || source == ABORTABLE_ERROR;
+                case ABORTABLE_ERROR:
+                    return source == IN_TRANSACTION || source == COMMITTING_TRANSACTION || source == ABORTING_TRANSACTION;
+                case FATAL_ERROR:
                 default:
-                    // We can transition to FENCED or ERROR unconditionally.
-                    // FENCED is never a valid starting state for any transition. So the only option is to close the
+                    // We can transition to FATAL_ERROR unconditionally.
+                    // FATAL_ERROR is never a valid starting state for any transition. So the only option is to close the
                     // producer or do purely non transactional requests.
                     return true;
             }
         }
     }
 
-
     // We use the priority to determine the order in which requests need to be sent out. For instance, if we have
     // a pending FindCoordinator request, that must always go first. Next, If we need a producer id, that must go second.
     // The endTxn request must always go last.
@@ -149,7 +151,7 @@ public class TransactionManager {
     }
 
     TransactionManager() {
-        this("", 0);
+        this(null, 0);
     }
 
     public synchronized TransactionalRequestResult initializeTransactions() {
@@ -178,8 +180,8 @@ public class TransactionManager {
 
     public synchronized TransactionalRequestResult beginAbortingTransaction() {
         ensureTransactional();
-        if (isFenced())
-            throw Errors.INVALID_PRODUCER_EPOCH.exception();
+        if (currentState != State.ABORTABLE_ERROR)
+            maybeFailWithError();
         transitionTo(State.ABORTING_TRANSACTION);
         return beginCompletingTransaction(false);
     }
@@ -213,12 +215,16 @@ public class TransactionManager {
     }
 
     public synchronized void maybeAddPartitionToTransaction(TopicPartition topicPartition) {
-        if (!isInTransaction() || partitionsInTransaction.contains(topicPartition))
+        if (!isInTransaction())
+            throw new IllegalArgumentException("Cannot add partitions to a transaction in state " + currentState);
+
+        if (partitionsInTransaction.contains(topicPartition))
             return;
+
         newPartitionsToBeAddedToTransaction.add(topicPartition);
     }
 
-    public Exception lastError() {
+    public RuntimeException lastError() {
         return lastError;
     }
 
@@ -231,11 +237,7 @@ public class TransactionManager {
     }
 
     public boolean isTransactional() {
-        return transactionalId != null && !transactionalId.isEmpty();
-    }
-
-    public boolean isFenced() {
-        return currentState == State.FENCED;
+        return transactionalId != null;
     }
 
     public boolean isCompletingTransaction() {
@@ -247,31 +249,15 @@ public class TransactionManager {
     }
 
     public boolean isInErrorState() {
-        return currentState == State.ERROR || currentState == State.FENCED;
-    }
-
-    public synchronized void setError(Exception exception) {
-        if (exception instanceof ProducerFencedException)
-            transitionTo(State.FENCED, exception);
-        else
-            transitionTo(State.ERROR, exception);
-    }
-
-    boolean maybeTerminateRequestWithError(TxnRequestHandler requestHandler) {
-        if (isInErrorState() && requestHandler.isEndTxn()) {
-            // We shouldn't terminate abort requests from error states.
-            EndTxnHandler endTxnHandler = (EndTxnHandler) requestHandler;
-            if (endTxnHandler.builder.result() == TransactionResult.ABORT)
-                return false;
-            String errorMessage = "Cannot commit transaction because at least one previous transactional request " +
-                    "was not completed successfully.";
-            if (lastError != null)
-                requestHandler.fatal(new KafkaException(errorMessage, lastError));
-            else
-                requestHandler.fatal(new KafkaException(errorMessage));
-            return true;
-        }
-        return false;
+        return currentState == State.ABORTABLE_ERROR || currentState == State.FATAL_ERROR;
+    }
+
+    public synchronized void transitionToAbortableError(RuntimeException exception) {
+        transitionTo(State.ABORTABLE_ERROR, exception);
+    }
+
+    public synchronized void transitionToFatalError(RuntimeException exception) {
+        transitionTo(State.FATAL_ERROR, exception);
     }
 
     /**
@@ -284,6 +270,15 @@ public class TransactionManager {
         return producerIdAndEpoch;
     }
 
+    boolean hasProducerId(long producerId) {
+        return producerIdAndEpoch.producerId == producerId;
+    }
+
+    boolean hasProducerIdAndEpoch(long producerId, short producerEpoch) {
+        ProducerIdAndEpoch idAndEpoch = this.producerIdAndEpoch;
+        return idAndEpoch.producerId == producerId && idAndEpoch.epoch == producerEpoch;
+    }
+
     /**
      * Set the producer id and epoch atomically.
      */
@@ -337,26 +332,26 @@ public class TransactionManager {
         sequenceNumbers.put(topicPartition, currentSequenceNumber);
     }
 
-    boolean hasPendingTransactionalRequests() {
-        return !(pendingRequests.isEmpty() && newPartitionsToBeAddedToTransaction.isEmpty());
-    }
-
-    TxnRequestHandler nextRequestHandler() {
-        if (!hasPendingTransactionalRequests())
-            return null;
-
+    synchronized TxnRequestHandler nextRequestHandler() {
         if (!newPartitionsToBeAddedToTransaction.isEmpty())
             pendingRequests.add(addPartitionsToTransactionHandler());
 
-        return pendingRequests.poll();
+        TxnRequestHandler nextRequestHandler = pendingRequests.poll();
+        if (nextRequestHandler != null && maybeTerminateRequestWithError(nextRequestHandler)) {
+            log.trace("TransactionalId: {} -- Not sending transactional request {} because we are in an error state",
+                    transactionalId, nextRequestHandler.requestBuilder());
+            return null;
+        }
+
+        return nextRequestHandler;
     }
 
-    void retry(TxnRequestHandler request) {
+    synchronized void retry(TxnRequestHandler request) {
         request.setRetry();
         pendingRequests.add(request);
     }
 
-    void reenqueue(TxnRequestHandler request) {
+    synchronized void reenqueue(TxnRequestHandler request) {
         pendingRequests.add(request);
     }
 
@@ -406,15 +401,21 @@ public class TransactionManager {
         transitionTo(target, null);
     }
 
-    private synchronized void transitionTo(State target, Exception error) {
-        if (currentState.isTransitionValid(currentState, target)) {
-            currentState = target;
-            if (target == State.ERROR && error != null)
-                lastError = error;
-        } else {
+    private synchronized void transitionTo(State target, RuntimeException error) {
+        if (!currentState.isTransitionValid(currentState, target))
             throw new KafkaException("Invalid transition attempted from state " + currentState.name() +
                     " to state " + target.name());
+
+        if (target == State.FATAL_ERROR || target == State.ABORTABLE_ERROR) {
+            if (error == null)
+                throw new IllegalArgumentException("Cannot transition to " + target + " with an null exception");
+            lastError = error;
+        } else {
+            lastError = null;
         }
+
+        log.debug("TransactionalId {} -- Transition from state {} to {}", transactionalId, currentState, target);
+        currentState = target;
     }
 
     private void ensureTransactional() {
@@ -423,15 +424,23 @@ public class TransactionManager {
     }
 
     private void maybeFailWithError() {
-        if (isFenced())
-            throw Errors.INVALID_PRODUCER_EPOCH.exception();
+        if (isInErrorState())
+            throw new KafkaException("Cannot execute transactional method because we are in an error state", lastError);
+    }
+
+    private boolean maybeTerminateRequestWithError(TxnRequestHandler requestHandler) {
         if (isInErrorState()) {
-            String errorMessage = "Cannot execute transactional method because we are in an error state.";
-            if (lastError != null)
-                throw new KafkaException(errorMessage, lastError);
-            else
-                throw new KafkaException(errorMessage);
+            if (requestHandler instanceof EndTxnHandler) {
+                // we allow abort requests to break out of the error state. The state and the last error
+                // will be cleared when the request returns
+                EndTxnHandler endTxnHandler = (EndTxnHandler) requestHandler;
+                if (endTxnHandler.builder.result() == TransactionResult.ABORT)
+                    return false;
+            }
+            requestHandler.fail(lastError);
+            return true;
         }
+        return false;
     }
 
     private void lookupCoordinator(FindCoordinatorRequest.CoordinatorType type, String coordinatorKey) {
@@ -443,12 +452,11 @@ public class TransactionManager {
                 transactionCoordinator = null;
                 break;
             default:
-                throw new IllegalStateException("Got an invalid coordinator type: " + type);
+                throw new IllegalStateException("Invalid coordinator type: " + type);
         }
 
         FindCoordinatorRequest.Builder builder = new FindCoordinatorRequest.Builder(type, coordinatorKey);
-        FindCoordinatorHandler request = new FindCoordinatorHandler(builder);
-        pendingRequests.add(request);
+        pendingRequests.add(new FindCoordinatorHandler(builder));
     }
 
     private void completeTransaction() {
@@ -473,9 +481,8 @@ public class TransactionManager {
             CommittedOffset committedOffset = new CommittedOffset(offsetAndMetadata.offset(), offsetAndMetadata.metadata());
             pendingTxnOffsetCommits.put(entry.getKey(), committedOffset);
         }
-        TxnOffsetCommitRequest.Builder builder = new TxnOffsetCommitRequest.Builder(consumerGroupId,
-                producerIdAndEpoch.producerId, producerIdAndEpoch.epoch,
-                pendingTxnOffsetCommits);
+        TxnOffsetCommitRequest.Builder builder = new TxnOffsetCommitRequest.Builder(transactionalId, consumerGroupId,
+                producerIdAndEpoch.producerId, producerIdAndEpoch.epoch, pendingTxnOffsetCommits);
         return new TxnOffsetCommitHandler(result, builder);
     }
 
@@ -491,19 +498,20 @@ public class TransactionManager {
             this(new TransactionalRequestResult());
         }
 
-        void fatal(RuntimeException e) {
+        void fatalError(RuntimeException e) {
+            result.setError(e);
+            transitionToFatalError(e);
+            result.done();
+        }
+
+        void abortableError(RuntimeException e) {
             result.setError(e);
-            transitionTo(State.ERROR, e);
+            transitionToAbortableError(e);
             result.done();
         }
 
-        void fenced() {
-            log.error("Producer has become invalid, which typically means another producer with the same " +
-                            "transactional.id has been started: producerId: {}. epoch: {}.",
-                    producerIdAndEpoch.producerId, producerIdAndEpoch.epoch);
-            result.setError(Errors.INVALID_PRODUCER_EPOCH.exception());
-            lastError = Errors.INVALID_PRODUCER_EPOCH.exception();
-            transitionTo(State.FENCED, Errors.INVALID_PRODUCER_EPOCH.exception());
+        void fail(RuntimeException e) {
+            result.setError(e);
             result.done();
         }
 
@@ -516,19 +524,19 @@ public class TransactionManager {
         @SuppressWarnings("unchecked")
         public void onComplete(ClientResponse response) {
             if (response.requestHeader().correlationId() != inFlightRequestCorrelationId) {
-                fatal(new RuntimeException("Detected more than one in-flight transactional request."));
+                fatalError(new RuntimeException("Detected more than one in-flight transactional request."));
             } else {
                 clearInFlightRequestCorrelationId();
                 if (response.wasDisconnected()) {
                     log.trace("disconnected from " + response.destination() + ". Will retry.");
                     reenqueue();
                 } else if (response.versionMismatch() != null) {
-                    fatal(response.versionMismatch());
+                    fatalError(response.versionMismatch());
                 } else if (response.hasResponse()) {
                     log.trace("Got transactional response for request:" + requestBuilder());
                     handleResponse(response.responseBody());
                 } else {
-                    fatal(new KafkaException("Could not execute transactional request for unknown reasons"));
+                    fatalError(new KafkaException("Could not execute transactional request for unknown reasons"));
                 }
             }
         }
@@ -585,6 +593,10 @@ public class TransactionManager {
         public void handleResponse(AbstractResponse response) {
             InitProducerIdResponse initProducerIdResponse = (InitProducerIdResponse) response;
             Errors error = initProducerIdResponse.error();
+
+            log.debug("TransactionalId {} -- Received InitProducerId response with error {}",
+                    transactionalId, error);
+
             if (error == Errors.NONE) {
                 ProducerIdAndEpoch producerIdAndEpoch = new ProducerIdAndEpoch(initProducerIdResponse.producerId(), initProducerIdResponse.epoch());
                 setProducerIdAndEpoch(producerIdAndEpoch);
@@ -597,9 +609,9 @@ public class TransactionManager {
             } else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.CONCURRENT_TRANSACTIONS) {
                 reenqueue();
             } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) {
-                fatal(error.exception());
+                fatalError(error.exception());
             } else {
-                fatal(new KafkaException("Unexpected error in InitProducerIdResponse; " + error.message()));
+                fatalError(new KafkaException("Unexpected error in InitProducerIdResponse; " + error.message()));
             }
         }
     }
@@ -626,6 +638,11 @@ public class TransactionManager {
             AddPartitionsToTxnResponse addPartitionsToTxnResponse = (AddPartitionsToTxnResponse) response;
             Map<TopicPartition, Errors> errors = addPartitionsToTxnResponse.errors();
             boolean hasPartitionErrors = false;
+            Set<String> unauthorizedTopics = new HashSet<>();
+
+            log.debug("TransactionalId {} -- Received AddPartitionsToTxn response with errors {}",
+                    transactionalId, errors);
+
             for (TopicPartition topicPartition : pendingPartitionsToBeAddedToTransaction) {
                 final Errors error = errors.get(topicPartition);
                 if (error == Errors.NONE || error == null) {
@@ -640,23 +657,28 @@ public class TransactionManager {
                     reenqueue();
                     return;
                 } else if (error == Errors.INVALID_PRODUCER_EPOCH) {
-                    fenced();
+                    fatalError(error.exception());
                     return;
                 } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) {
-                    fatal(error.exception());
+                    fatalError(error.exception());
                     return;
                 } else if (error == Errors.INVALID_PRODUCER_ID_MAPPING
                         || error == Errors.INVALID_TXN_STATE) {
-                    fatal(new KafkaException(error.exception()));
+                    fatalError(new KafkaException(error.exception()));
                     return;
+                } else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) {
+                    unauthorizedTopics.add(topicPartition.topic());
                 } else {
-                    log.error("Could not add partitions to transaction due to partition error. partition={}, error={}", topicPartition, error);
+                    log.error("TransactionalId: {} -- Could not add partition {} due to unexpected error {}",
+                            transactionalId, topicPartition, error);
                     hasPartitionErrors = true;
                 }
             }
 
-            if (hasPartitionErrors) {
-                fatal(new KafkaException("Could not add partitions to transaction due to partition level errors"));
+            if (!unauthorizedTopics.isEmpty()) {
+                abortableError(new TopicAuthorizationException(unauthorizedTopics));
+            } else if (hasPartitionErrors) {
+                abortableError(new KafkaException("Could not add partitions to transaction due to partition level errors"));
             } else {
                 partitionsInTransaction.addAll(pendingPartitionsToBeAddedToTransaction);
                 pendingPartitionsToBeAddedToTransaction.clear();
@@ -695,7 +717,12 @@ public class TransactionManager {
         @Override
         public void handleResponse(AbstractResponse response) {
             FindCoordinatorResponse findCoordinatorResponse = (FindCoordinatorResponse) response;
-            if (findCoordinatorResponse.error() == Errors.NONE) {
+            Errors error = findCoordinatorResponse.error();
+
+            log.debug("TransactionalId {} -- Received FindCoordinator response with error {}",
+                    transactionalId, error);
+
+            if (error == Errors.NONE) {
                 Node node = findCoordinatorResponse.node();
                 switch (builder.coordinatorType()) {
                     case GROUP:
@@ -705,12 +732,14 @@ public class TransactionManager {
                         transactionCoordinator = node;
                 }
                 result.done();
-            } else if (findCoordinatorResponse.error() == Errors.COORDINATOR_NOT_AVAILABLE) {
+            } else if (error == Errors.COORDINATOR_NOT_AVAILABLE) {
                 reenqueue();
+            } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) {
+                fatalError(error.exception());
             } else if (findCoordinatorResponse.error() == Errors.GROUP_AUTHORIZATION_FAILED) {
-                fatal(new GroupAuthorizationException("Not authorized to commit offsets " + builder.coordinatorKey()));
+                abortableError(new GroupAuthorizationException(builder.coordinatorKey()));
             } else {
-                fatal(new KafkaException(String.format("Could not find a coordinator with type %s with key %s due to" +
+                fatalError(new KafkaException(String.format("Could not find a coordinator with type %s with key %s due to" +
                         "unexpected error: %s", builder.coordinatorType(), builder.coordinatorKey(),
                         findCoordinatorResponse.error().message())));
             }
@@ -743,6 +772,10 @@ public class TransactionManager {
         public void handleResponse(AbstractResponse response) {
             EndTxnResponse endTxnResponse = (EndTxnResponse) response;
             Errors error = endTxnResponse.error();
+
+            log.debug("TransactionalId {} -- Received EndTxn response with error {}",
+                    transactionalId, error);
+
             if (error == Errors.NONE) {
                 completeTransaction();
                 result.done();
@@ -752,11 +785,13 @@ public class TransactionManager {
             } else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.CONCURRENT_TRANSACTIONS) {
                 reenqueue();
             } else if (error == Errors.INVALID_PRODUCER_EPOCH) {
-                fenced();
+                fatalError(error.exception());
             } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) {
-                fatal(error.exception());
+                fatalError(error.exception());
+            } else if (error == Errors.INVALID_TXN_STATE) {
+                fatalError(error.exception());
             } else {
-                fatal(new KafkaException("Unhandled error in EndTxnResponse: " + error.message()));
+                fatalError(new KafkaException("Unhandled error in EndTxnResponse: " + error.message()));
             }
         }
     }
@@ -785,6 +820,10 @@ public class TransactionManager {
         public void handleResponse(AbstractResponse response) {
             AddOffsetsToTxnResponse addOffsetsToTxnResponse = (AddOffsetsToTxnResponse) response;
             Errors error = addOffsetsToTxnResponse.error();
+
+            log.debug("TransactionalId {} -- Received AddOffsetsToTxn response with error {}",
+                    transactionalId, error);
+
             if (error == Errors.NONE) {
                 // note the result is not completed until the TxnOffsetCommit returns
                 pendingRequests.add(txnOffsetCommitHandler(result, offsets, builder.consumerGroupId()));
@@ -794,11 +833,13 @@ public class TransactionManager {
             } else if (error == Errors.COORDINATOR_LOAD_IN_PROGRESS || error == Errors.CONCURRENT_TRANSACTIONS) {
                 reenqueue();
             } else if (error == Errors.INVALID_PRODUCER_EPOCH) {
-                fenced();
+                fatalError(error.exception());
             } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) {
-                fatal(error.exception());
+                fatalError(error.exception());
+            } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) {
+                abortableError(new GroupAuthorizationException(builder.consumerGroupId()));
             } else {
-                fatal(new KafkaException("Unexpected error in AddOffsetsToTxnResponse: " + error.message()));
+                fatalError(new KafkaException("Unexpected error in AddOffsetsToTxnResponse: " + error.message()));
             }
         }
     }
@@ -837,7 +878,12 @@ public class TransactionManager {
             TxnOffsetCommitResponse txnOffsetCommitResponse = (TxnOffsetCommitResponse) response;
             boolean coordinatorReloaded = false;
             boolean hadFailure = false;
-            for (Map.Entry<TopicPartition, Errors> entry : txnOffsetCommitResponse.errors().entrySet()) {
+            Map<TopicPartition, Errors> errors = txnOffsetCommitResponse.errors();
+
+            log.debug("TransactionalId {} -- Received TxnOffsetCommit response with errors {}",
+                    transactionalId, errors);
+
+            for (Map.Entry<TopicPartition, Errors> entry : errors.entrySet()) {
                 TopicPartition topicPartition = entry.getKey();
                 Errors error = entry.getValue();
                 if (error == Errors.NONE) {
@@ -850,11 +896,17 @@ public class TransactionManager {
                     }
                 } else if (error == Errors.UNKNOWN_TOPIC_OR_PARTITION) {
                     hadFailure = true;
+                } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) {
+                    abortableError(new GroupAuthorizationException(builder.consumerGroupId()));
+                    return;
+                } else if (error == Errors.TRANSACTIONAL_ID_AUTHORIZATION_FAILED) {
+                    fatalError(error.exception());
+                    return;
                 } else if (error == Errors.INVALID_PRODUCER_EPOCH) {
-                    fenced();
+                    fatalError(error.exception());
                     return;
                 } else {
-                    fatal(new KafkaException("Unexpected error in TxnOffsetCommitResponse: " + error.message()));
+                    fatalError(new KafkaException("Unexpected error in TxnOffsetCommitResponse: " + error.message()));
                     return;
                 }
             }

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/common/errors/ProducerIdAuthorizationException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ProducerIdAuthorizationException.java b/clients/src/main/java/org/apache/kafka/common/errors/ProducerIdAuthorizationException.java
deleted file mode 100644
index 2da9158..0000000
--- a/clients/src/main/java/org/apache/kafka/common/errors/ProducerIdAuthorizationException.java
+++ /dev/null
@@ -1,23 +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 ProducerIdAuthorizationException extends ApiException {
-    public ProducerIdAuthorizationException(final String message) {
-        super(message);
-    }
-}

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/common/errors/TransactionalIdAuthorizationException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/errors/TransactionalIdAuthorizationException.java b/clients/src/main/java/org/apache/kafka/common/errors/TransactionalIdAuthorizationException.java
index 9bf1fbb..3f85513 100644
--- a/clients/src/main/java/org/apache/kafka/common/errors/TransactionalIdAuthorizationException.java
+++ b/clients/src/main/java/org/apache/kafka/common/errors/TransactionalIdAuthorizationException.java
@@ -16,7 +16,7 @@
  */
 package org.apache.kafka.common.errors;
 
-public class TransactionalIdAuthorizationException extends ApiException {
+public class TransactionalIdAuthorizationException extends AuthorizationException {
     public TransactionalIdAuthorizationException(final String message) {
         super(message);
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/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 f94fb4d..9444eb5 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
@@ -57,7 +57,6 @@ import org.apache.kafka.common.errors.OffsetOutOfRangeException;
 import org.apache.kafka.common.errors.OutOfOrderSequenceException;
 import org.apache.kafka.common.errors.PolicyViolationException;
 import org.apache.kafka.common.errors.ProducerFencedException;
-import org.apache.kafka.common.errors.ProducerIdAuthorizationException;
 import org.apache.kafka.common.errors.RebalanceInProgressException;
 import org.apache.kafka.common.errors.RecordBatchTooLargeException;
 import org.apache.kafka.common.errors.RecordTooLargeException;
@@ -483,21 +482,13 @@ public enum Errors {
             return new TransactionalIdAuthorizationException(message);
         }
     }),
-    PRODUCER_ID_AUTHORIZATION_FAILED(54, "Producer is not authorized to use producer Ids, " +
-            "which is required to write idempotent data.",
-                                             new ApiExceptionBuilder() {
-        @Override
-        public ApiException build(String message) {
-            return new ProducerIdAuthorizationException(message);
-        }
-    }),
-    SECURITY_DISABLED(55, "Security features are disabled.", new ApiExceptionBuilder() {
+    SECURITY_DISABLED(54, "Security features are disabled.", new ApiExceptionBuilder() {
         @Override
         public ApiException build(String message) {
             return new SecurityDisabledException(message);
         }
     }),
-    BROKER_AUTHORIZATION_FAILED(56, "Broker authorization failed", new ApiExceptionBuilder() {
+    BROKER_AUTHORIZATION_FAILED(55, "Broker authorization failed", new ApiExceptionBuilder() {
         @Override
         public ApiException build(String message) {
             return new BrokerAuthorizationException(message);

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/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 d5ce469..91391e9 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
@@ -1516,6 +1516,9 @@ public class Protocol {
     );
 
     public static final Schema TXN_OFFSET_COMMIT_REQUEST_V0 = new Schema(
+            new Field("transactional_id",
+                    STRING,
+                    "The transactional id corresponding to the transaction."),
             new Field("consumer_group_id",
                     STRING,
                     "Id of the associated consumer group to commit offsets for."),

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java
index 4bf8b3e..3339470 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnRequest.java
@@ -50,6 +50,17 @@ public class AddOffsetsToTxnRequest extends AbstractRequest {
         public AddOffsetsToTxnRequest build(short version) {
             return new AddOffsetsToTxnRequest(version, transactionalId, producerId, producerEpoch, consumerGroupId);
         }
+
+        @Override
+        public String toString() {
+            StringBuilder bld = new StringBuilder();
+            bld.append("(transactionalId=").append(transactionalId).
+                    append(", producerId=").append(producerId).
+                    append(", producerEpoch=").append(producerEpoch).
+                    append(", consumerGroupId=").append(consumerGroupId).
+                    append(")");
+            return bld.toString();
+        }
     }
 
     private final String transactionalId;

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java
index 8b3a589..754f242 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AddOffsetsToTxnResponse.java
@@ -30,10 +30,11 @@ public class AddOffsetsToTxnResponse extends AbstractResponse {
     //   NotCoordinator
     //   CoordinatorNotAvailable
     //   CoordinatorLoadInProgress
-    //   InvalidPidMapping
+    //   InvalidProducerIdMapping
+    //   InvalidProducerEpoch
     //   InvalidTxnState
     //   GroupAuthorizationFailed
-    //   InvalidProducerEpoch
+    //   TransactionalIdAuthorizationFailed
 
     private final Errors error;
     private final int throttleTimeMs;

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java
index 148ebec..e24fa5a 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnRequest.java
@@ -54,6 +54,17 @@ public class AddPartitionsToTxnRequest extends AbstractRequest {
         public AddPartitionsToTxnRequest build(short version) {
             return new AddPartitionsToTxnRequest(version, transactionalId, producerId, producerEpoch, partitions);
         }
+
+        @Override
+        public String toString() {
+            StringBuilder bld = new StringBuilder();
+            bld.append("(transactionalId=").append(transactionalId).
+                    append(", producerId=").append(producerId).
+                    append(", producerEpoch=").append(producerEpoch).
+                    append(", partitions=").append(partitions).
+                    append(")");
+            return bld.toString();
+        }
     }
 
     private final String transactionalId;

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java
index 697142b..39172ee 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/AddPartitionsToTxnResponse.java
@@ -43,11 +43,12 @@ public class AddPartitionsToTxnResponse extends AbstractResponse {
     //   CoordinatorNotAvailable
     //   CoordinatorLoadInProgress
     //   InvalidTxnState
-    //   InvalidPidMapping
+    //   InvalidProducerIdMapping
     //   TopicAuthorizationFailed
     //   InvalidProducerEpoch
     //   UnknownTopicOrPartition
     //   TopicAuthorizationFailed
+    //   TransactionalIdAuthorizationFailed
     private final Map<TopicPartition, Errors> errors;
 
     public AddPartitionsToTxnResponse(int throttleTimeMs, Map<TopicPartition, Errors> errors) {

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java
index 77ec137..b9f052c 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnRequest.java
@@ -50,6 +50,17 @@ public class EndTxnRequest extends AbstractRequest {
         public EndTxnRequest build(short version) {
             return new EndTxnRequest(version, transactionalId, producerId, producerEpoch, result);
         }
+
+        @Override
+        public String toString() {
+            StringBuilder bld = new StringBuilder();
+            bld.append("(transactionalId=").append(transactionalId).
+                    append(", producerId=").append(producerId).
+                    append(", producerEpoch=").append(producerEpoch).
+                    append(", result=").append(result).
+                    append(")");
+            return bld.toString();
+        }
     }
 
     private final String transactionalId;

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java
index 99e4e8c..17cf68d 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/EndTxnResponse.java
@@ -31,8 +31,9 @@ public class EndTxnResponse extends AbstractResponse {
     //   CoordinatorNotAvailable
     //   CoordinatorLoadInProgress
     //   InvalidTxnState
-    //   InvalidPidMapping
+    //   InvalidProducerIdMapping
     //   InvalidProducerEpoch
+    //   TransactionalIdAuthorizationFailed
 
     private final Errors error;
     private final int throttleTimeMs;

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java
index 7c8a6e5..96e1cdf 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/InitProducerIdResponse.java
@@ -24,11 +24,13 @@ import org.apache.kafka.common.record.RecordBatch;
 import java.nio.ByteBuffer;
 
 public class InitProducerIdResponse extends AbstractResponse {
-    /**
-     * Possible Error codes:
-     * OK
-     *
-     */
+    // Possible error codes:
+    //   NotCoordinator
+    //   CoordinatorNotAvailable
+    //   CoordinatorLoadInProgress
+    //   TransactionalIdAuthorizationFailed
+    //   ClusterAuthorizationFailed
+
     private static final String THROTTLE_TIME_KEY_NAME = "throttle_time_ms";
     private static final String PRODUCER_ID_KEY_NAME = "producer_id";
     private static final String EPOCH_KEY_NAME = "producer_epoch";

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
index 3377f91..3d696c1 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java
@@ -228,13 +228,14 @@ public class ProduceRequest extends AbstractRequest {
     }
 
     @Override
-    public AbstractResponse getErrorResponse(int throttleTimeMs, Throwable e) {
+    public ProduceResponse getErrorResponse(int throttleTimeMs, Throwable e) {
         /* In case the producer doesn't actually want any response */
         if (acks == 0)
             return null;
 
+        Errors error = Errors.forException(e);
         Map<TopicPartition, ProduceResponse.PartitionResponse> responseMap = new HashMap<>();
-        ProduceResponse.PartitionResponse partitionResponse = new ProduceResponse.PartitionResponse(Errors.forException(e));
+        ProduceResponse.PartitionResponse partitionResponse = new ProduceResponse.PartitionResponse(error);
 
         for (TopicPartition tp : partitions())
             responseMap.put(tp, partitionResponse);

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
index 42ae434..55332f6 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java
@@ -61,6 +61,9 @@ public class ProduceResponse extends AbstractResponse {
      * INVALID_REQUIRED_ACKS (21)
      * TOPIC_AUTHORIZATION_FAILED (29)
      * UNSUPPORTED_FOR_MESSAGE_FORMAT (43)
+     * INVALID_PRODUCER_EPOCH (47)
+     * CLUSTER_AUTHORIZATION_FAILED (31)
+     * TRANSACTIONAL_ID_AUTHORIZATION_FAILED (53)
      */
 
     private static final String BASE_OFFSET_KEY_NAME = "base_offset";

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java
index f5334f2..68fa3d2 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitRequest.java
@@ -27,6 +27,7 @@ import java.util.HashMap;
 import java.util.Map;
 
 public class TxnOffsetCommitRequest extends AbstractRequest {
+    private static final String TRANSACTIONAL_ID_KEY_NAME = "transactional_id";
     private static final String CONSUMER_GROUP_ID_KEY_NAME = "consumer_group_id";
     private static final String PRODUCER_ID_KEY_NAME = "producer_id";
     private static final String PRODUCER_EPOCH_KEY_NAME = "producer_epoch";
@@ -38,14 +39,16 @@ public class TxnOffsetCommitRequest extends AbstractRequest {
     private static final String METADATA_KEY_NAME = "metadata";
 
     public static class Builder extends AbstractRequest.Builder<TxnOffsetCommitRequest> {
+        private final String transactionalId;
         private final String consumerGroupId;
         private final long producerId;
         private final short producerEpoch;
         private final Map<TopicPartition, CommittedOffset> offsets;
 
-        public Builder(String consumerGroupId, long producerId, short producerEpoch,
+        public Builder(String transactionalId, String consumerGroupId, long producerId, short producerEpoch,
                        Map<TopicPartition, CommittedOffset> offsets) {
             super(ApiKeys.TXN_OFFSET_COMMIT);
+            this.transactionalId = transactionalId;
             this.consumerGroupId = consumerGroupId;
             this.producerId = producerId;
             this.producerEpoch = producerEpoch;
@@ -58,18 +61,32 @@ public class TxnOffsetCommitRequest extends AbstractRequest {
 
         @Override
         public TxnOffsetCommitRequest build(short version) {
-            return new TxnOffsetCommitRequest(version, consumerGroupId, producerId, producerEpoch, offsets);
+            return new TxnOffsetCommitRequest(version, transactionalId, consumerGroupId, producerId, producerEpoch, offsets);
+        }
+
+        @Override
+        public String toString() {
+            StringBuilder bld = new StringBuilder();
+            bld.append("(transactionalId=").append(transactionalId).
+                    append(", producerId=").append(producerId).
+                    append(", producerEpoch=").append(producerEpoch).
+                    append(", consumerGroupId=").append(consumerGroupId).
+                    append(", offsets=").append(offsets).
+                    append(")");
+            return bld.toString();
         }
     }
 
+    private final String transactionalId;
     private final String consumerGroupId;
     private final long producerId;
     private final short producerEpoch;
     private final Map<TopicPartition, CommittedOffset> offsets;
 
-    public TxnOffsetCommitRequest(short version, String consumerGroupId, long producerId, short producerEpoch,
-                                  Map<TopicPartition, CommittedOffset> offsets) {
+    public TxnOffsetCommitRequest(short version, String transactionalId, String consumerGroupId, long producerId,
+                                  short producerEpoch, Map<TopicPartition, CommittedOffset> offsets) {
         super(version);
+        this.transactionalId = transactionalId;
         this.consumerGroupId = consumerGroupId;
         this.producerId = producerId;
         this.producerEpoch = producerEpoch;
@@ -78,6 +95,7 @@ public class TxnOffsetCommitRequest extends AbstractRequest {
 
     public TxnOffsetCommitRequest(Struct struct, short version) {
         super(version);
+        this.transactionalId = struct.getString(TRANSACTIONAL_ID_KEY_NAME);
         this.consumerGroupId = struct.getString(CONSUMER_GROUP_ID_KEY_NAME);
         this.producerId = struct.getLong(PRODUCER_ID_KEY_NAME);
         this.producerEpoch = struct.getShort(PRODUCER_EPOCH_KEY_NAME);
@@ -98,6 +116,10 @@ public class TxnOffsetCommitRequest extends AbstractRequest {
         this.offsets = offsets;
     }
 
+    public String transactionalId() {
+        return transactionalId;
+    }
+
     public String consumerGroupId() {
         return consumerGroupId;
     }
@@ -117,6 +139,7 @@ public class TxnOffsetCommitRequest extends AbstractRequest {
     @Override
     protected Struct toStruct() {
         Struct struct = new Struct(ApiKeys.TXN_OFFSET_COMMIT.requestSchema(version()));
+        struct.set(TRANSACTIONAL_ID_KEY_NAME, transactionalId);
         struct.set(CONSUMER_GROUP_ID_KEY_NAME, consumerGroupId);
         struct.set(PRODUCER_ID_KEY_NAME, producerId);
         struct.set(PRODUCER_EPOCH_KEY_NAME, producerEpoch);

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java
index 37b9a50..a62568f 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/TxnOffsetCommitResponse.java
@@ -42,6 +42,7 @@ public class TxnOffsetCommitResponse extends AbstractResponse {
     //   OffsetMetadataTooLarge
     //   GroupAuthorizationFailed
     //   InvalidCommitOffsetSize
+    //   TransactionalIdAuthorizationFailed
 
     private final Map<TopicPartition, Errors> errors;
     private final int throttleTimeMs;

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java
index 06f6662..ddddc42 100644
--- a/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/WriteTxnMarkersResponse.java
@@ -48,6 +48,7 @@ public class WriteTxnMarkersResponse extends AbstractResponse {
     //   InvalidRequiredAcks
     //   TransactionCoordinatorFenced
     //   RequestTimeout
+    //   ClusterAuthorizationFailed
 
     private final Map<Long, Map<TopicPartition, Errors>> errors;
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/38f6cae9/clients/src/test/java/org/apache/kafka/clients/admin/AclOperationTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/AclOperationTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/AclOperationTest.java
index 06ace63..0e3441f 100644
--- a/clients/src/test/java/org/apache/kafka/clients/admin/AclOperationTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/admin/AclOperationTest.java
@@ -47,7 +47,8 @@ public class AclOperationTest {
         new AclOperationTestInfo(AclOperation.DESCRIBE, 8, "describe", false),
         new AclOperationTestInfo(AclOperation.CLUSTER_ACTION, 9, "cluster_action", false),
         new AclOperationTestInfo(AclOperation.DESCRIBE_CONFIGS, 10, "describe_configs", false),
-        new AclOperationTestInfo(AclOperation.ALTER_CONFIGS, 11, "alter_configs", false)
+        new AclOperationTestInfo(AclOperation.ALTER_CONFIGS, 11, "alter_configs", false),
+        new AclOperationTestInfo(AclOperation.IDEMPOTENT_WRITE, 12, "idempotent_write", false)
     };
 
     @Test