KIP-863 introduced a change to ByteBufferDeserializer which is not
properly documented, but should be called out because it could surface
bugs in application code which using ByteBufferDeserializer.
Reviewers: Lianet Magrans <lmagrans@confluent.io>, Kirk True <ktrue@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
This PR upgrades RocksDB from 7.9.2 to 9.7.3 and addresses the following compatibility issues introduced by the RocksDB upgrade:
- Removal of AccessHint: The AccessHint class was completely removed in RocksDB 9.7.3. This required removing all import statements, variable declarations, method parameters, method return types, and static method calls related to AccessHint in RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.java RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapterTest.java Unused methods are removed in RocksDBGenericOptionsToDbOptionsColumnFamilyOptionsAdapter.java
- Removal of NO_FILE_CLOSES: The NO_FILE_CLOSES metric was also removed in RocksDB 9.7.3. The calculation for numberOfOpenFiles in RocksDBMetricsRecorder.java has been adjusted to now track the total number of file opens since the last reset. The previous calculation, which subtracted NO_FILE_CLOSES from NO_FILE_OPENS, is no longer possible. The reason RocksDB removed NO_FILE_CLOSES seems to be that it did not properly work: https://github.com/search?q=repo%3Afacebook%2Frocksdb+NO_FILE_CLOSES&type=issues
- Removal of methods related to compressed block cache configuration in BlockBasedTableConfig
- Change of the signature of org.rocksdb.Options.setLogger()
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias J. Sax <matthias@confluent.io>, Bruno Cadonna <cadonna@apache.org>
In document the shell command base dir is in Kafka, so we should make all command is bin/<tools>.sh
In kafka repo doesn't have controller_static.properties this file, this would misunderstanding user, I change to use controller.properties
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Also remove confusing comment related to connect & Java 17 from build.gradle.
I also updated KIP-1013 to note the implications of KIP-1032.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
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>
This PR covers all the docs for KIP-1112, including the new config and a note about the new APIs in the 4.0 section of the upgrade guide.
This also fixes/updates some unrelated parts of the upgrade guide that were out-of-date, such as the broker compatibility matrix
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>, Matthias Sax <mjsax@apache.org>
This PR introduces the DescribeGroups v6 API as part of KIP-1043. This adds an error message for the described groups so that it is possible to get some context on the error. It also changes the behaviour for when the group ID cannot be found but returning error code GROUP_ID_NOT_FOUND rather than NONE.
Reviewers: David Jacot <djacot@confluent.io>
Documents current ApiVersions fallback mechanism
The broker behaviour when it receives an ApiVersionsRequest ahead of it's supported versions is useful to know as an implementor/manipulator of the protocol.
The behaviour contradicts a point documented in the protocol guide. In the fallback case the protocol version of the response is different from the protocol version of the request.
Also spotted a couple of dead links in the table of contents
Signed-off-by: Robert Young <robeyoun@redhat.com>
Reviewers: Luke Chen <showuon@gmail.com>
This PR implement KIP-1011, kafka-configs.sh now uses incrementalAlterConfigs API to alter broker configurations instead of the deprecated alterConfigs API, and it will fall directly if the broker doesn't support incrementalAlterConfigs.
Reviewers: David Jacot <djacot@confluent.io>, OmniaGM <o.g.h.ibrahim@gmail.com>.
This PR updates the key for storing the KIP-714 client instance id for the global consumer to follow a more consistent pattern of the other embedded Kafka Streams consumer clients.
Reviewers: Matthias Sax <mjsax@apache.org>
These metrics were deprecated in KIP-773 and are being removed in Kafka 4.0.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Divij Vaidya <diviv@amazon.com>
In docs/ops.html, add a section discussion the difference between static and dynamic quorums. This section also discusses how to find out which quorum type you have. Also discuss the current limitations, such as the inability to transition from static quorums to dynamic.
Add a brief section to docs/upgrade.html discussing controller membership change.
Co-authored-by: Federico Valeri <fedevaleri@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
Reviewers: Justine Olshan <jolshan@confluent.io>
KIP-853 adds support for dynamic KRaft quorums. This means that the quorum topology is
no longer statically determined by the controller.quorum.voters configuration. Instead, it
is contained in the storage directories of each controller and broker.
Users of dynamic quorums must format at least one controller storage directory with either
the --initial-controllers or --standalone flags. If they fail to do this, no quorum can be
established. This PR changes the storage tool to warn about the case where a KIP-853 flag has
not been supplied to format a KIP-853 controller. (Note that broker storage directories
can continue to be formatted without a KIP-853 flag.)
There are cases where we don't want to specify initial voters when formatting a controller. One
example is where we format a single controller with --standalone, and then dynamically add 4
more controllers with no initial topology. In this case, we want the 4 later controllers to grab
the quorum topology from the initial one. To support this case, this PR adds the
--no-initial-controllers flag.
Reviewers: José Armando García Sancio <jsancio@apache.org>, Federico Valeri <fvaleri@redhat.com>
This patch updates the upgrades note about the removal of `offsets.commit.required.acks` configuration in 4.0.
Reviewers: Lianet Magrans <lmagrans@confluent.io>
Implements KIP-1087
Reviewers: Matthias J. Sax <matthias@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
Change the configurations under config/kraft to use controller.quorum.bootstrap.servers instead of controller.quorum.voters. Add comments explaining how to use the older static quorum configuration where appropriate.
In docs/ops.html, remove the reference to "tentative timelines for ZooKeeper removal" and "Tiered storage is considered as an early access feature" since they are no longer up-to-date. Add KIP-853 information.
In docs/quickstart.html, move the ZK instructions to be after the KRaft instructions. Update the KRaft instructions to use KIP-853.
In docs/security.html, add an explanation of --bootstrap-controller and document controller.quorum.bootstrap.servers instead of controller.quorum.voters.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>