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