2019-01-12 08:40:21 +08:00
|
|
|
// 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.
|
|
|
|
|
|
|
|
{
|
|
|
|
"apiKey": 8,
|
|
|
|
"type": "request",
|
2025-01-24 21:53:32 +08:00
|
|
|
"listeners": ["broker"],
|
2019-01-12 08:40:21 +08:00
|
|
|
"name": "OffsetCommitRequest",
|
KAFKA-18269: Remove deprecated protocol APIs support (KIP-896, KIP-724) (#18218)
Included in this change:
1. Remove deprecated protocol api versions from json files.
3. Remove fields that are no longer used from json files (affects ListOffsets, OffsetCommit, DescribeConfigs).
4. Remove record down-conversion support from KafkaApis.
5. No longer return `Errors.UNSUPPORTED_COMPRESSION_TYPE` on the fetch path[1].
6. Deprecate `TopicConfig. MESSAGE_DOWNCONVERSION_ENABLE_CONFIG` and made the relevant
configs (`message.downconversion.enable` and `log.message.downcoversion.enable`) no-ops since
down-conversion is no longer supported. It was an oversight not to deprecate this via KIP-724.
7. Fix `shouldRetainsBufferReference` to handle null request schemas for a given version.
8. Simplify producer logic since it only supports the v2 record format now.
9. Fix tests so they don't exercise protocol api versions that have been removed.
10. Add upgrade note.
Testing:
1. System tests have a lot of failures, but those tests fail for trunk too and I didn't see any issues specific to this change - it's hard to be sure given the number of failing tests, but let's not block on that given the other testing that has been done (see below).
3. Java producers and consumers with version 0.9-0.10.1 don't have api versions support and hence they fail in an ungraceful manner: the broker disconnects and the clients reconnect until the relevant timeout is triggered.
4. Same thing seems to happen for the console producer 0.10.2 although it's unclear why since api versions should be supported. I will look into this separately, it's unlikely to be related to this PR.
5. Console consumer 0.10.2 fails with the expected error and a reasonable message[2].
6. Console producer and consumer 0.11.0 works fine, newer versions should naturally also work fine.
7. kcat 1.5.0 (based on librdkafka 1.1.0) produce and consume fail with a reasonable message[3][4].
8. kcat 1.6.0-1.7.0 (based on librdkafka 1.5.0 and 1.7.0 respectively) consume fails with a reasonable message[5].
9. kcat 1.6.0-1.7.0 produce works fine.
10. kcat 1.7.1 (based on librdkafka 1.8.2) works fine for consumer and produce.
11. confluent-go-client (librdkafka based) 1.8.2 works fine for consumer and produce.
12. I will test more clients, but I don't think we need to block the PR on that.
Note that this also completes part of KIP-724: produce v2 and lower as well as fetch v3 and lower are no longer supported.
Future PRs will remove conditional code that is no longer needed (some of that has been done in KafkaApis,
but only what was required due to the schema changes). We can probably do that in master only as it does
not change behavior.
Note that I did not touch `ignorable` fields even though some of them could have been
changed. The reasoning is that this could result in incompatible changes for clients
that use new protocol versions without setting such fields _if_ we don't manually
validate their presence. I will file a JIRA ticket to look into this carefully for each
case (i.e. if we do validate their presence for the appropriate versions, we can
set them to ignorable=false in the json file).
[1] We would return this error if a fetch < v10 was used and the compression topic config was set
to zstd, but we would not do the same for the case where zstd was compressed at the producer
level (the most common case). Since there is no efficient way to do the check for the common
case, I made it consistent for both by having no checks.
[2] ```org.apache.kafka.common.errors.UnsupportedVersionException: The broker is too new to support JOIN_GROUP version 1```
[3]```METADATA|rdkafka#producer-1| [thrd:main]: localhost:9092/bootstrap: Metadata request failed: connected: Local: Required feature not supported by broker (0ms): Permanent```
[4]```METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9092/bootstrap: Metadata request failed: connected: Local: Required feature not supported by broker (0ms): Permanent```
[5] `ERROR: Topic test-topic [0] error: Failed to query logical offset END: Local: Required feature not supported by broker`
Reviewers: David Arthur <mumrah@gmail.com>
2024-12-21 11:52:00 +08:00
|
|
|
// Versions 0-1 were removed in Apache Kafka 4.0, Version 2 is the new baseline.
|
|
|
|
//
|
2019-01-12 08:40:21 +08:00
|
|
|
// Version 1 adds timestamp and group membership information, as well as the commit timestamp.
|
|
|
|
//
|
|
|
|
// Version 2 adds retention time. It removes the commit timestamp added in version 1.
|
|
|
|
//
|
|
|
|
// Version 3 and 4 are the same as version 2.
|
|
|
|
//
|
|
|
|
// Version 5 removes the retention time, which is now controlled only by a broker configuration.
|
|
|
|
//
|
|
|
|
// Version 6 adds the leader epoch for fencing.
|
2019-10-07 12:13:23 +08:00
|
|
|
//
|
KAFKA-8225 & KIP-345 part-2: fencing static member instances with conflicting group.instance.id (#6650)
For static members join/rejoin, we encode the current timestamp in the new member.id. The format looks like group.instance.id-timestamp.
During consumer/broker interaction logic (Join, Sync, Heartbeat, Commit), we shall check the whether group.instance.id is known on group. If yes, we shall match the member.id stored on static membership map with the request member.id. If mismatching, this indicates a conflict consumer has used same group.instance.id, and it will receive a fatal exception to shut down.
Right now the only missing part is the system test. Will work on it offline while getting the major logic changes reviewed.
Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2019-05-18 22:28:37 +08:00
|
|
|
// version 7 adds a new field called groupInstanceId to indicate member identity across restarts.
|
2019-10-07 12:13:23 +08:00
|
|
|
//
|
|
|
|
// Version 8 is the first flexible version.
|
2023-07-22 02:08:06 +08:00
|
|
|
//
|
|
|
|
// Version 9 is the first version that can be used with the new consumer group protocol (KIP-848). The
|
|
|
|
// request is the same as version 8.
|
KAFKA-18269: Remove deprecated protocol APIs support (KIP-896, KIP-724) (#18218)
Included in this change:
1. Remove deprecated protocol api versions from json files.
3. Remove fields that are no longer used from json files (affects ListOffsets, OffsetCommit, DescribeConfigs).
4. Remove record down-conversion support from KafkaApis.
5. No longer return `Errors.UNSUPPORTED_COMPRESSION_TYPE` on the fetch path[1].
6. Deprecate `TopicConfig. MESSAGE_DOWNCONVERSION_ENABLE_CONFIG` and made the relevant
configs (`message.downconversion.enable` and `log.message.downcoversion.enable`) no-ops since
down-conversion is no longer supported. It was an oversight not to deprecate this via KIP-724.
7. Fix `shouldRetainsBufferReference` to handle null request schemas for a given version.
8. Simplify producer logic since it only supports the v2 record format now.
9. Fix tests so they don't exercise protocol api versions that have been removed.
10. Add upgrade note.
Testing:
1. System tests have a lot of failures, but those tests fail for trunk too and I didn't see any issues specific to this change - it's hard to be sure given the number of failing tests, but let's not block on that given the other testing that has been done (see below).
3. Java producers and consumers with version 0.9-0.10.1 don't have api versions support and hence they fail in an ungraceful manner: the broker disconnects and the clients reconnect until the relevant timeout is triggered.
4. Same thing seems to happen for the console producer 0.10.2 although it's unclear why since api versions should be supported. I will look into this separately, it's unlikely to be related to this PR.
5. Console consumer 0.10.2 fails with the expected error and a reasonable message[2].
6. Console producer and consumer 0.11.0 works fine, newer versions should naturally also work fine.
7. kcat 1.5.0 (based on librdkafka 1.1.0) produce and consume fail with a reasonable message[3][4].
8. kcat 1.6.0-1.7.0 (based on librdkafka 1.5.0 and 1.7.0 respectively) consume fails with a reasonable message[5].
9. kcat 1.6.0-1.7.0 produce works fine.
10. kcat 1.7.1 (based on librdkafka 1.8.2) works fine for consumer and produce.
11. confluent-go-client (librdkafka based) 1.8.2 works fine for consumer and produce.
12. I will test more clients, but I don't think we need to block the PR on that.
Note that this also completes part of KIP-724: produce v2 and lower as well as fetch v3 and lower are no longer supported.
Future PRs will remove conditional code that is no longer needed (some of that has been done in KafkaApis,
but only what was required due to the schema changes). We can probably do that in master only as it does
not change behavior.
Note that I did not touch `ignorable` fields even though some of them could have been
changed. The reasoning is that this could result in incompatible changes for clients
that use new protocol versions without setting such fields _if_ we don't manually
validate their presence. I will file a JIRA ticket to look into this carefully for each
case (i.e. if we do validate their presence for the appropriate versions, we can
set them to ignorable=false in the json file).
[1] We would return this error if a fetch < v10 was used and the compression topic config was set
to zstd, but we would not do the same for the case where zstd was compressed at the producer
level (the most common case). Since there is no efficient way to do the check for the common
case, I made it consistent for both by having no checks.
[2] ```org.apache.kafka.common.errors.UnsupportedVersionException: The broker is too new to support JOIN_GROUP version 1```
[3]```METADATA|rdkafka#producer-1| [thrd:main]: localhost:9092/bootstrap: Metadata request failed: connected: Local: Required feature not supported by broker (0ms): Permanent```
[4]```METADATA|rdkafka#consumer-1| [thrd:main]: localhost:9092/bootstrap: Metadata request failed: connected: Local: Required feature not supported by broker (0ms): Permanent```
[5] `ERROR: Topic test-topic [0] error: Failed to query logical offset END: Local: Required feature not supported by broker`
Reviewers: David Arthur <mumrah@gmail.com>
2024-12-21 11:52:00 +08:00
|
|
|
"validVersions": "2-9",
|
2019-10-07 12:13:23 +08:00
|
|
|
"flexibleVersions": "8+",
|
2019-01-12 08:40:21 +08:00
|
|
|
"fields": [
|
2019-05-08 12:35:17 +08:00
|
|
|
{ "name": "GroupId", "type": "string", "versions": "0+", "entityType": "groupId",
|
2019-01-12 08:40:21 +08:00
|
|
|
"about": "The unique group identifier." },
|
2023-07-22 02:08:06 +08:00
|
|
|
{ "name": "GenerationIdOrMemberEpoch", "type": "int32", "versions": "1+", "default": "-1", "ignorable": true,
|
2023-12-22 05:39:17 +08:00
|
|
|
"about": "The generation of the group if using the classic group protocol or the member epoch if using the consumer protocol." },
|
2019-01-12 08:40:21 +08:00
|
|
|
{ "name": "MemberId", "type": "string", "versions": "1+", "ignorable": true,
|
|
|
|
"about": "The member ID assigned by the group coordinator." },
|
2019-05-22 13:51:56 +08:00
|
|
|
{ "name": "GroupInstanceId", "type": "string", "versions": "7+",
|
|
|
|
"nullableVersions": "7+", "default": "null",
|
KAFKA-8225 & KIP-345 part-2: fencing static member instances with conflicting group.instance.id (#6650)
For static members join/rejoin, we encode the current timestamp in the new member.id. The format looks like group.instance.id-timestamp.
During consumer/broker interaction logic (Join, Sync, Heartbeat, Commit), we shall check the whether group.instance.id is known on group. If yes, we shall match the member.id stored on static membership map with the request member.id. If mismatching, this indicates a conflict consumer has used same group.instance.id, and it will receive a fatal exception to shut down.
Right now the only missing part is the system test. Will work on it offline while getting the major logic changes reviewed.
Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
2019-05-18 22:28:37 +08:00
|
|
|
"about": "The unique identifier of the consumer instance provided by end user." },
|
2019-01-12 08:40:21 +08:00
|
|
|
{ "name": "RetentionTimeMs", "type": "int64", "versions": "2-4", "default": "-1", "ignorable": true,
|
|
|
|
"about": "The time period in ms to retain the offset." },
|
|
|
|
{ "name": "Topics", "type": "[]OffsetCommitRequestTopic", "versions": "0+",
|
|
|
|
"about": "The topics to commit offsets for.", "fields": [
|
2019-05-08 12:35:17 +08:00
|
|
|
{ "name": "Name", "type": "string", "versions": "0+", "entityType": "topicName",
|
2019-01-12 08:40:21 +08:00
|
|
|
"about": "The topic name." },
|
|
|
|
{ "name": "Partitions", "type": "[]OffsetCommitRequestPartition", "versions": "0+",
|
|
|
|
"about": "Each partition to commit offsets for.", "fields": [
|
|
|
|
{ "name": "PartitionIndex", "type": "int32", "versions": "0+",
|
|
|
|
"about": "The partition index." },
|
|
|
|
{ "name": "CommittedOffset", "type": "int64", "versions": "0+",
|
|
|
|
"about": "The message offset to be committed." },
|
|
|
|
{ "name": "CommittedLeaderEpoch", "type": "int32", "versions": "6+", "default": "-1", "ignorable": true,
|
|
|
|
"about": "The leader epoch of this partition." },
|
|
|
|
{ "name": "CommittedMetadata", "type": "string", "versions": "0+", "nullableVersions": "0+",
|
|
|
|
"about": "Any associated metadata the client wants to keep." }
|
|
|
|
]}
|
|
|
|
]}
|
|
|
|
]
|
|
|
|
}
|