Skip to content

MINOR: Add tests on TxnOffsetCommit and EndTxnMarker protection against invalid producer epoch when TV2 is used #20024

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 9 commits into
base: trunk
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,16 @@ package kafka.server
import kafka.network.SocketServer
import kafka.utils.TestUtils
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata}
import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.{TopicCollection, TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.common.message.DeleteGroupsResponseData.{DeletableGroupResult, DeletableGroupResultCollection}
import org.apache.kafka.common.message.LeaveGroupRequestData.MemberIdentity
import org.apache.kafka.common.message.LeaveGroupResponseData.MemberResponse
import org.apache.kafka.common.message.SyncGroupRequestData.SyncGroupRequestAssignment
import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AddOffsetsToTxnResponseData, ConsumerGroupDescribeRequestData, ConsumerGroupDescribeResponseData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, DeleteGroupsRequestData, DeleteGroupsResponseData, DescribeGroupsRequestData, DescribeGroupsResponseData, EndTxnRequestData, HeartbeatRequestData, HeartbeatResponseData, InitProducerIdRequestData, JoinGroupRequestData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsRequestData, ListGroupsResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteRequestData, OffsetDeleteResponseData, OffsetFetchRequestData, OffsetFetchResponseData, ShareGroupDescribeRequestData, ShareGroupDescribeResponseData, ShareGroupHeartbeatRequestData, ShareGroupHeartbeatResponseData, SyncGroupRequestData, SyncGroupResponseData, TxnOffsetCommitRequestData, TxnOffsetCommitResponseData}
import org.apache.kafka.common.message.WriteTxnMarkersRequestData.{WritableTxnMarker, WritableTxnMarkerTopic}
import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AddOffsetsToTxnResponseData, ConsumerGroupDescribeRequestData, ConsumerGroupDescribeResponseData, ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData, DeleteGroupsRequestData, DeleteGroupsResponseData, DescribeGroupsRequestData, DescribeGroupsResponseData, EndTxnRequestData, HeartbeatRequestData, HeartbeatResponseData, InitProducerIdRequestData, JoinGroupRequestData, JoinGroupResponseData, LeaveGroupResponseData, ListGroupsRequestData, ListGroupsResponseData, OffsetCommitRequestData, OffsetCommitResponseData, OffsetDeleteRequestData, OffsetDeleteResponseData, OffsetFetchRequestData, OffsetFetchResponseData, ShareGroupDescribeRequestData, ShareGroupDescribeResponseData, ShareGroupHeartbeatRequestData, ShareGroupHeartbeatResponseData, SyncGroupRequestData, SyncGroupResponseData, TxnOffsetCommitRequestData, TxnOffsetCommitResponseData, WriteTxnMarkersRequestData}
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, AddOffsetsToTxnRequest, AddOffsetsToTxnResponse, ConsumerGroupDescribeRequest, ConsumerGroupDescribeResponse, ConsumerGroupHeartbeatRequest, ConsumerGroupHeartbeatResponse, DeleteGroupsRequest, DeleteGroupsResponse, DescribeGroupsRequest, DescribeGroupsResponse, EndTxnRequest, EndTxnResponse, HeartbeatRequest, HeartbeatResponse, InitProducerIdRequest, InitProducerIdResponse, JoinGroupRequest, JoinGroupResponse, LeaveGroupRequest, LeaveGroupResponse, ListGroupsRequest, ListGroupsResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetDeleteRequest, OffsetDeleteResponse, OffsetFetchRequest, OffsetFetchResponse, ShareGroupDescribeRequest, ShareGroupDescribeResponse, ShareGroupHeartbeatRequest, ShareGroupHeartbeatResponse, SyncGroupRequest, SyncGroupResponse, TxnOffsetCommitRequest, TxnOffsetCommitResponse}
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, AddOffsetsToTxnRequest, AddOffsetsToTxnResponse, ConsumerGroupDescribeRequest, ConsumerGroupDescribeResponse, ConsumerGroupHeartbeatRequest, ConsumerGroupHeartbeatResponse, DeleteGroupsRequest, DeleteGroupsResponse, DescribeGroupsRequest, DescribeGroupsResponse, EndTxnRequest, EndTxnResponse, HeartbeatRequest, HeartbeatResponse, InitProducerIdRequest, InitProducerIdResponse, JoinGroupRequest, JoinGroupResponse, LeaveGroupRequest, LeaveGroupResponse, ListGroupsRequest, ListGroupsResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetDeleteRequest, OffsetDeleteResponse, OffsetFetchRequest, OffsetFetchResponse, ShareGroupDescribeRequest, ShareGroupDescribeResponse, ShareGroupHeartbeatRequest, ShareGroupHeartbeatResponse, SyncGroupRequest, SyncGroupResponse, TxnOffsetCommitRequest, TxnOffsetCommitResponse, WriteTxnMarkersRequest, WriteTxnMarkersResponse}
import org.apache.kafka.common.serialization.StringSerializer
import org.apache.kafka.common.test.ClusterInstance
import org.apache.kafka.common.utils.ProducerIdAndEpoch
Expand Down Expand Up @@ -351,6 +353,35 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) {
assertEquals(expectedError, connectAndReceive[EndTxnResponse](request).error)
}

protected def writeTxnMarkers(
producerId: Long,
producerEpoch: Short,
committed: Boolean,
expectedError: Errors = Errors.NONE,
version: Short = ApiKeys.WRITE_TXN_MARKERS.latestVersion(isUnstableApiEnabled)
): Unit = {
val request = new WriteTxnMarkersRequest.Builder(
new WriteTxnMarkersRequestData()
.setMarkers(List(
new WritableTxnMarker()
.setProducerId(producerId)
.setProducerEpoch(producerEpoch)
.setTransactionResult(committed)
.setTopics(List(
new WritableTxnMarkerTopic()
.setName(Topic.GROUP_METADATA_TOPIC_NAME)
.setPartitionIndexes(List[Integer](0).asJava)
).asJava)
.setCoordinatorEpoch(0)
).asJava)
).build(version)

assertEquals(
expectedError.code,
connectAndReceive[WriteTxnMarkersResponse](request).data.markers.get(0).topics.get(0).partitions.get(0).errorCode
)
}

protected def fetchOffsets(
groups: List[OffsetFetchRequestData.OffsetFetchRequestGroup],
requireStable: Boolean,
Expand Down Expand Up @@ -421,6 +452,27 @@ class GroupCoordinatorBaseRequestTest(cluster: ClusterInstance) {
groupResponse
}

protected def fetchOffset(
groupId: String,
topic: String,
partition: Int
): Long = {
val groupIdRecord = fetchOffsets(
group = new OffsetFetchRequestData.OffsetFetchRequestGroup()
.setGroupId(groupId)
.setTopics(List(
new OffsetFetchRequestData.OffsetFetchRequestTopics()
.setName(topic)
.setPartitionIndexes(List[Integer](partition).asJava)
).asJava),
requireStable = true,
version = 9
)
val topicRecord = groupIdRecord.topics.asScala.find(_.name == topic).head
val partitionRecord = topicRecord.partitions.asScala.find(_.partitionIndex == partition).head
partitionRecord.committedOffset
}

protected def deleteOffset(
groupId: String,
topic: String,
Expand Down
141 changes: 112 additions & 29 deletions core/src/test/scala/unit/kafka/server/TxnOffsetCommitRequestTest.scala
Original file line number Diff line number Diff line change
Expand Up @@ -16,19 +16,16 @@
*/
package kafka.server

import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type}
import kafka.utils.TestUtils
import org.apache.kafka.common.errors.UnsupportedVersionException
import org.apache.kafka.common.message.OffsetFetchRequestData
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
import org.apache.kafka.common.requests.JoinGroupRequest
import org.apache.kafka.common.requests.{EndTxnRequest, JoinGroupRequest}
import org.apache.kafka.common.test.ClusterInstance
import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type}
import org.apache.kafka.common.utils.ProducerIdAndEpoch
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
import org.junit.jupiter.api.Assertions.{assertThrows, assertTrue}

import scala.jdk.CollectionConverters._
import org.junit.jupiter.api.Assertions.{assertNotEquals, assertThrows}

@ClusterTestDefaults(
types = Array(Type.KRAFT),
Expand All @@ -51,6 +48,16 @@ class TxnOffsetCommitRequestTest(cluster:ClusterInstance) extends GroupCoordinat
testTxnOffsetCommit(false)
}

@ClusterTest
def testDelayedTxnOffsetCommitWithBumpedEpochIsRejectedWithNewConsumerGroupProtocol(): Unit = {
testDelayedTxnOffsetCommitWithBumpedEpochIsRejected(true)
}

@ClusterTest
def testDelayedTxnOffsetCommitWithBumpedEpochIsRejectedWithOldConsumerGroupProtocol(): Unit = {
testDelayedTxnOffsetCommitWithBumpedEpochIsRejected(false)
}

private def testTxnOffsetCommit(useNewProtocol: Boolean): Unit = {
val topic = "topic"
val partition = 0
Expand All @@ -65,8 +72,8 @@ class TxnOffsetCommitRequestTest(cluster:ClusterInstance) extends GroupCoordinat
// Join the consumer group. Note that we don't heartbeat here so we must use
// a session long enough for the duration of the test.
val (memberId: String, memberEpoch: Int) = joinConsumerGroup(groupId, useNewProtocol)
assertTrue(memberId != JoinGroupRequest.UNKNOWN_MEMBER_ID)
assertTrue(memberEpoch != JoinGroupRequest.UNKNOWN_GENERATION_ID)
assertNotEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, memberId)
assertNotEquals(JoinGroupRequest.UNKNOWN_GENERATION_ID, memberEpoch)

createTopic(topic, 1)

Expand Down Expand Up @@ -178,7 +185,7 @@ class TxnOffsetCommitRequestTest(cluster:ClusterInstance) extends GroupCoordinat
transactionalId = transactionalId
)

val originalOffset = fetchOffset(topic, partition, groupId)
val originalOffset = fetchOffset(groupId, topic, partition)

commitTxnOffset(
groupId = groupId,
Expand Down Expand Up @@ -207,31 +214,107 @@ class TxnOffsetCommitRequestTest(cluster:ClusterInstance) extends GroupCoordinat

TestUtils.waitUntilTrue(() =>
try {
fetchOffset(topic, partition, groupId) == expectedOffset
fetchOffset(groupId, topic, partition) == expectedOffset
} catch {
case _: Throwable => false
}, "txn commit offset validation failed"
)
}

private def fetchOffset(
topic: String,
partition: Int,
groupId: String
): Long = {
val groupIdRecord = fetchOffsets(
group = new OffsetFetchRequestData.OffsetFetchRequestGroup()
.setGroupId(groupId)
.setTopics(List(
new OffsetFetchRequestData.OffsetFetchRequestTopics()
.setName(topic)
.setPartitionIndexes(List[Integer](partition).asJava)
).asJava),
requireStable = true,
version = 9
)
val topicRecord = groupIdRecord.topics.asScala.find(_.name == topic).head
val partitionRecord = topicRecord.partitions.asScala.find(_.partitionIndex == partition).head
partitionRecord.committedOffset
private def testDelayedTxnOffsetCommitWithBumpedEpochIsRejected(useNewProtocol: Boolean): Unit = {
val topic = "topic"
val partition = 0
val transactionalId = "txn"
val groupId = "group"
val offset = 100L

// Creates the __consumer_offsets and __transaction_state topics because it won't be created automatically
// in this test because it does not use FindCoordinator API.
createOffsetsTopic()
createTransactionStateTopic()

// Join the consumer group. Note that we don't heartbeat here so we must use
// a session long enough for the duration of the test.
val (memberId: String, memberEpoch: Int) = joinConsumerGroup(groupId, useNewProtocol)
assertNotEquals(JoinGroupRequest.UNKNOWN_MEMBER_ID, memberId)
assertNotEquals(JoinGroupRequest.UNKNOWN_GENERATION_ID, memberEpoch)

createTopic(topic, 1)

for (version <- ApiKeys.TXN_OFFSET_COMMIT.oldestVersion to ApiKeys.TXN_OFFSET_COMMIT.latestVersion(isUnstableApiEnabled)) {
val useTV2 = version > EndTxnRequest.LAST_STABLE_VERSION_BEFORE_TRANSACTION_V2

// Initialize producer. Wait until the coordinator finishes loading.
var producerIdAndEpoch: ProducerIdAndEpoch = null
TestUtils.waitUntilTrue(() =>
try {
producerIdAndEpoch = initProducerId(
transactionalId = transactionalId,
producerIdAndEpoch = ProducerIdAndEpoch.NONE,
expectedError = Errors.NONE
)
true
} catch {
case _: Throwable => false
}, "initProducerId request failed"
)

addOffsetsToTxn(
groupId = groupId,
producerId = producerIdAndEpoch.producerId,
producerEpoch = producerIdAndEpoch.epoch,
transactionalId = transactionalId
)

// Complete the transaction.
endTxn(
producerId = producerIdAndEpoch.producerId,
producerEpoch = producerIdAndEpoch.epoch,
transactionalId = transactionalId,
isTransactionV2Enabled = useTV2,
committed = true,
expectedError = Errors.NONE
)

// Start a new transaction. Wait for the previous transaction to complete.
TestUtils.waitUntilTrue(() =>
try {
addOffsetsToTxn(
groupId = groupId,
producerId = producerIdAndEpoch.producerId,
producerEpoch = if (useTV2) (producerIdAndEpoch.epoch + 1).toShort else producerIdAndEpoch.epoch,
transactionalId = transactionalId
)
true
} catch {
case _: Throwable => false
}, "addOffsetsToTxn request failed"
)

// Committing offset with old epoch succeeds for TV1 and fails for TV2.
commitTxnOffset(
groupId = groupId,
memberId = if (version >= 3) memberId else JoinGroupRequest.UNKNOWN_MEMBER_ID,
generationId = if (version >= 3) 1 else JoinGroupRequest.UNKNOWN_GENERATION_ID,
producerId = producerIdAndEpoch.producerId,
producerEpoch = producerIdAndEpoch.epoch,
transactionalId = transactionalId,
topic = topic,
partition = partition,
offset = offset,
expectedError = if (useTV2) Errors.INVALID_PRODUCER_EPOCH else Errors.NONE,
version = version.toShort
)

// Complete the transaction.
endTxn(
producerId = producerIdAndEpoch.producerId,
producerEpoch = if (useTV2) (producerIdAndEpoch.epoch + 1).toShort else producerIdAndEpoch.epoch,
transactionalId = transactionalId,
isTransactionV2Enabled = useTV2,
committed = true,
expectedError = Errors.NONE
)
}
}
}
Loading