Skip to content
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

KAFKA-18060: new coordinator does not handle TxnOffsetCommitRequest with empty member id when using CONSUMER group #17914

Merged
merged 14 commits into from
Dec 3, 2024

Conversation

brandboat
Copy link
Member

@brandboat brandboat commented Nov 22, 2024

There are two issues in KAFKA-18060:

  1. New coordinator can't handle the TxnOffsetCommitRequest with empty member id, and TxnOffsetCommitRequest v0-v2 do definitely has an empty member ID, causing ConsumerGroup#validateOffsetCommit to throw an UnknownMemberIdException. This prevents the old producer from calling sendOffsetsToTransaction. Note that TxnOffsetCommitRequest versions v0-v2 are included in KIP-896, so it seems the new coordinator should support that operations

  2. The deprecated API Producer#sendOffsetsToTransaction does not use v0-v2 to send TxnOffsetCommitRequest with an empty member ID. Unfortunately, it has been released for a while. Therefore, the new coordinator needs to handle TxnOffsetCommitRequest with an empty member ID for all versions.

Taken from the two issues above, we need to handle empty member id in all API versions when new coordinator are dealing with TxnOffsetCommitRequest.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

…th empty member id when using CONSUMER group
@github-actions github-actions bot added the core Kafka Broker label Nov 22, 2024
@dajac dajac added the KIP-848 The Next Generation of the Consumer Rebalance Protocol label Nov 22, 2024
@dajac dajac self-requested a review November 22, 2024 15:52
@dajac dajac changed the title KAFKA-18060 new coordinator does not handle TxnOffsetCommitRequest with empty member id when using CONSUMER group KAFKA-18060: new coordinator does not handle TxnOffsetCommitRequest with empty member id when using CONSUMER group Nov 22, 2024
@dajac
Copy link
Member

dajac commented Nov 22, 2024

@brandboat Thanks for the quick patch. Could you please add a description to the pull request? We usually also use it for the commit message.

Copy link
Member

@dajac dajac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@brandboat Thanks for the patch. I left some comments for consideration.

Comment on lines 518 to 520
// TxnOffsetCommitRequest versions v0-v2 do not include a member ID.
// And we can still send UNKNOWN_GENERATION_ID and UNKNOWN_MEMBER_ID through Producer#sendOffsetsToTransaction.
// Therefore, we should not throw exception in these cases.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would simplify it a bit. What about The TxnOffsetCommit API does not require the member id, the generation id and the group instance id fields. Hence the are only validated if any of them is provided.?

@@ -26,6 +26,7 @@
import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you please extend unit tests in ConsumerGroupTest too?

Comment on lines 99 to 100
adminClient = cluster.admin()
adminClient.createTopics(singleton(new NewTopic(topic, 1, 1.toShort)))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You could actually use createTopic method from the parent class.

Comment on lines 106 to 109
// add this line to avoid error INVALID_TXN_STATE
producer.sendOffsetsToTransaction(
Collections.singletonMap(new TopicPartition("topic", 0), new OffsetAndMetadata(5)),
new ConsumerGroupMetadata(groupId))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I suppose that you must do this in order to add the __consumer_offsets partition to the transaction.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Personally, I prefer to build the AddOffsetsToTxnRequest explicitly. for example:

  protected def addOffsetsToTxn(groupId: String,
                                producerId: Int,
                                producerEpoch: Short,
                                transactionalId: String,
                                version: Short): Unit = {
    val request = new AddOffsetsToTxnRequest.Builder(new AddOffsetsToTxnRequestData()
      .setTransactionalId(transactionalId)
      .setProducerId(producerId).setProducerEpoch(producerEpoch).setGroupId(groupId)
    ).build(version)

    val expectedResponse = new AddOffsetsToTxnResponseData()

    val response = connectAndReceive[AddOffsetsToTxnResponse](request)
    assertEquals(expectedResponse, response.data)
  }

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you both! I believe all comments have been addressed. Please take a look when you have a chance.

Collections.singletonMap(new TopicPartition("topic", 0), new OffsetAndMetadata(5)),
new ConsumerGroupMetadata(groupId))

// verify that the TXN_OFFSET_COMMIT request is processed correctly when member id is UNKNOWN_MEMBER_ID
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Verify....

producer.close()
}
}
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

While we are here, do you mind adding a test which verifies that the fields are correctly validated when provided?

Comment on lines 42 to 45
new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"),
new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1"),
new ClusterConfigProperty(key = TransactionLogConfig.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, value = "1"),
new ClusterConfigProperty(key = TransactionLogConfig.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1"),
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: You can define all the common configs at the class level in ClusterTestDefaults.

Copy link
Member

@dajac dajac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@brandboat Thanks for the update. I left a few more comments for consideration.

Comment on lines 243 to 247
protected def addOffsetsToTxn(groupId: String,
producerId: Int,
producerEpoch: Short,
transactionalId: String,
version: Short): Unit = {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Let's follow the format of the other methods please.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry, that comes from my auto format... I forgot to set the scala code style, thanks

version: Short): Unit = {
val request = new AddOffsetsToTxnRequest.Builder(new AddOffsetsToTxnRequestData()
.setTransactionalId(transactionalId)
.setProducerId(producerId).setProducerEpoch(producerEpoch).setGroupId(groupId)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: Let's put setProducerEpoch and setGroupId on new lines to be consistent.

.setProducerId(producerId).setProducerEpoch(producerEpoch).setGroupId(groupId)
).build(version)

val expectedResponse = new AddOffsetsToTxnResponseData()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: We could inline it.

Comment on lines 95 to 97
producer = cluster.producer(Collections.singletonMap(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId))
producer.initTransactions()
producer.beginTransaction()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Now that we have addOffsetsToTxn, I wonder if we should also add one to init the transaction so we could remove the producer. It is a bit weird to use both because you actually don't know whether the producer id/epoch passes to addOffsetsToTxn are actually correct.

Comment on lines 110 to 111
memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID,
generationId = JoinGroupRequest.UNKNOWN_GENERATION_ID,
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you please add a similar test but which sets member id/generation id too?

Comment on lines 122 to 138
TestUtils.waitUntilTrue(() => {
consumer.poll(Duration.ofSeconds(5)).asScala
.filter(record => record.key() != null && record.value() != null)
.map(record => new GroupCoordinatorRecordSerde()
.deserialize(ByteBuffer.wrap(record.key().get()), ByteBuffer.wrap(record.value().get())))
.filter(coordinatorRecord =>
coordinatorRecord.key().message().isInstanceOf[OffsetCommitKey] &&
coordinatorRecord.value().message().isInstanceOf[OffsetCommitValue])
.exists(coordinatorRecord => {
val offsetCommitKey = coordinatorRecord.key().message().asInstanceOf[OffsetCommitKey]
val offsetCommitValue = coordinatorRecord.value().message().asInstanceOf[OffsetCommitValue]
offsetCommitKey.group() == groupId &&
offsetCommitKey.topic() == topic &&
offsetCommitKey.partition == partition &&
offsetCommitValue.offset() == 100 + version
})
}, "Txn offset commit not found")
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is a bit extreme. I would rather prefer to use fetchOffset to validate the committed offsets.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the review, I have addressed all comments in the latest commits. Also loop the integration test 247 times and all passed.

I=0; while ./gradlew cleanTest core:test --tests kafka.server.TxnOffsetCommitRequestTest --rerun --fail-fast; do (( I=$I+1 )); echo "Completed run: $I"; sleep 1; done

Copy link
Member

@dajac dajac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@brandboat Thanks for the update and your patience. I left a few more minor suggestions.

Comment on lines 170 to 181
// Verify that the TXN_OFFSET_COMMIT request is processed correctly when the member ID
// and generation ID are known. This validation starts from version 3, as the member ID
// must not be empty from version 3 onwards.
if (version >= 3) {
verifyTxnCommitAndFetch(
groupId = groupId,
memberId = memberId,
generationId = memberEpoch,
offset = 200 + version,
version = version.toShort
)
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Would it be possible to add a negative cases too? We could add them in this test or in another one.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Perhaps we can put the negative case to TxnOffsetCommitRequestTest.java ? I thought the TxnOffsetCommitRequestTest should take care the scenario that GroupMetadata is defined while version < 3.
Let me know if you have any concern. Thanks.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

By negative cases, I meant a request with invalid memberId and/or generationId to ensure that the request is rejected with the correct error. Is it worth it?

Copy link
Member Author

@brandboat brandboat Dec 2, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agree, I added those test in the latest commit, thanks for the suggestion !

Copy link
Member

@dajac dajac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@brandboat Thanks for the update and your patience. We're almost there! I replied to one of the open comments and left a small nit.

Comment on lines 170 to 181
// Verify that the TXN_OFFSET_COMMIT request is processed correctly when the member ID
// and generation ID are known. This validation starts from version 3, as the member ID
// must not be empty from version 3 onwards.
if (version >= 3) {
verifyTxnCommitAndFetch(
groupId = groupId,
memberId = memberId,
generationId = memberEpoch,
offset = 200 + version,
version = version.toShort
)
}
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

By negative cases, I meant a request with invalid memberId and/or generationId to ensure that the request is rejected with the correct error. Is it worth it?


createTopic(topic, 1)

def verifyTxnCommitAndFetch(
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: This method is pretty long to be inlined now. Would it make sense to extract it?

Copy link
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@brandboat thanks for this great patch. some trivial comments are left. PTAL

expectedTxnCommitError: Errors
): Unit = {
var producerIdAndEpoch: ProducerIdAndEpoch = null
// Wait until ALLOCATE_PRODUCER_ID request finished
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems that a more accurate comment would be "Wait until the coordinator finishes loading."

val topicRecord = groupIdRecord.topics.asScala.find(_.name == topic).head
val partitionRecord = topicRecord.partitions.asScala.find(_.partitionIndex == partition).head
assertEquals(offset, partitionRecord.committedOffset)
true
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

offset == partitionRecord.committedOffset

expectedError = Errors.NONE
)

if (expectedTxnCommitError != Errors.NONE) return
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you please confirm that committed offsets remain unchanged after a failed TXN_OFFSET_COMMIT?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed in the latest commit, thanks.

Copy link
Member

@dajac dajac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@brandboat Thanks for the update. Overall, LGTM. I left a few nits.


for (version <- 0 to ApiKeys.TXN_OFFSET_COMMIT.latestVersion(isUnstableApiEnabled)) {
// Verify that the TXN_OFFSET_COMMIT request is processed correctly when member id is UNKNOWN_MEMBER_ID
// and generation id is UNKNOWN_GENERATION_ID under all api versions
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: . at the end of the comment.

expectedTxnCommitError = Errors.NONE
)

// Verify TXN_OFFSET_COMMIT request failed with incorrect memberId
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto.

expectedTxnCommitError = Errors.UNKNOWN_MEMBER_ID
)

// Verify TXN_OFFSET_COMMIT request failed with incorrect generationId
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto.

Copy link
Member

@dajac dajac left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM, thanks.

@dajac
Copy link
Member

dajac commented Dec 3, 2024

@chia7712 Do you want to take another look at it? Otherwise, I can merge it tomorrow.

@chia7712 chia7712 merged commit ac8b3df into apache:trunk Dec 3, 2024
8 checks passed
@brandboat brandboat deleted the KAFKA-18060 branch December 4, 2024 00:55
tedyu pushed a commit to tedyu/kafka that referenced this pull request Jan 6, 2025
…th empty member id when using CONSUMER group (apache#17914)

There are two issues in KAFKA-18060:

1) New coordinator can't handle the TxnOffsetCommitRequest with empty member id, and TxnOffsetCommitRequest v0-v2 do definitely has an empty member ID, causing ConsumerGroup#validateOffsetCommit to throw an UnknownMemberIdException. This prevents the old producer from calling sendOffsetsToTransaction. Note that TxnOffsetCommitRequest versions v0-v2 are included in KIP-896, so it seems the new coordinator should support that operations

2) The deprecated API Producer#sendOffsetsToTransaction does not use v0-v2 to send TxnOffsetCommitRequest with an empty member ID. Unfortunately, it has been released for a while. Therefore, the new coordinator needs to handle TxnOffsetCommitRequest with an empty member ID for all versions.

Taken from the two issues above, we need to handle empty member id in all API versions when new coordinator are dealing with TxnOffsetCommitRequest.

Reviewers: David Jacot <[email protected]>, Chia-Ping Tsai <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
clients core Kafka Broker KIP-848 The Next Generation of the Consumer Rebalance Protocol
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants