After MirrorMaker 1 removal, there are no other modules dependencies for these classes, so we can safely move them to tools module.
Signed-off-by: Federico Valeri <fedevaleri@gmail.com>
Reviewers: Mickael Maison <mickael.maison@gmail.com>
I've added the release-version flag to the upgrade and downgrade commands. I've also added tests.
While working on this, I realized that we reveal non-production features to be returned in the version-mapping and dependencies commands. I have changed this to only return production features (except in tests) and added tests for this.
Reviewers: Jun Rao <jun@confluent.io>
Adding some missing input checks and fixing a formatting issue.
Signed-off-by: Federico Valeri <fedevaleri@gmail.com>
Reviewers: Luke Chen <showuon@gmail.com>
This patch completely removes the compile-time dependency on core for both test and main sources by introducing two new modules.
1) `test-common` include all the common test implementation code (including dependency on :core for BrokerServer, ControllerServer, etc)
2) `test-common:api` new sub-module that just includes interfaces including our junit extension
Reviewers: David Arthur <mumrah@gmail.com>
49d7ea6 updated the behavior of the UpdateFeaturesRequest/Response, but the MockAdminClient did not reflect those changes.
Now if any feature fails, all the features fail and the correct message is written in the result. Also only update the features if all features are successful and the command is not validate only.
Reviewers: Jun Rao <jun@confluent.io>
This change includes:
1. Dependency checking when updating the feature (all request versions)
2. Returning top level error and no feature level errors if any feature failed to update and using this error for all the features in the response. (all request versions)
3. Returning only top level none error for v2 and beyond
Reviewers: Jun Rao <jun@confluent.io>
Implements KIP-1087
Reviewers: Matthias J. Sax <matthias@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
This patch belongs to the ongoing efforts of implementing KIP-1022.
Added feature-dependencies command to look up dependencies for a given feature version supplied by --feature flag. If the feature is not known or the version not yet defined, we throw an error.
Examples :
bin/kafka-feature feature-dependencies --feature transaction.version=2
transaction.version=2 requires:
metadata.version=4 (3.3-IV0) (listing any other version dependencies)
bin/kafka-feature feature-dependencies --feature metadata.version=17
metadata.version=17 (3.7-IV2) has no dependencies
Reviewers: Justine Olshan <jolshan@confluent.io>, Artem Livshits <alivshits@confluent.io>
As a part of KIP-1022 the following has been implemented in this patch:
A version-mapping command to to look up the corresponding features for a given metadata version. Using the command with no --release-version argument will return the mapping for the latest stable metadata version.
This command has been added to the FeatureCommand Tool and the Storage Tool.
The storage tools parsing method has been made more modular similar to the feature command tool
Reviewers: Justine Olshan <jolshan@confluent.io>
This patch re-introduces the `group.version` feature flag and gates the new consumer rebalance protocol with it. The `group.version` feature flag is attached to the metadata version `4.0-IV0` and it is marked as production ready. This allows system tests to pick it up directly by default without requiring to set `unstable.feature.versions.enable` in all of them. This is fine because we don't plan to do any incompatible changes before 4.0.
Reviewers: Justine Olshan <jolshan@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
This patch makes the new group coordinator, introduced as part of KIP-848, the default. This means that any KRaft cluster created from trunk defaults to using the new group coordinator. This includes all the integration tests which do not specify it. This patch also changes the default in system tests.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
When you use kafka-share-groups.sh --describe for an empty group, it prints an empty table consisting of only the table header. kafka-consumer-groups.sh summarises the group status to make the output more informative and only prints the table if it contains more than zero rows.
This PR applies this principle across all of the variants of describing share groups which makes the output much nicer where the output would otherwise be strangely empty.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Currently, users need to set --transaction-duration-ms to enable transactions in kafka-producer-perf-test, which is not straightforward. A better approach is to enable transactions when a transaction ID is provided.
This PR allows enabling transaction in kafka-producer-perf-test by either
- set transaction.id=<id> via --producer-props or
- set transaction.id=<id> in config file via --producer.config or
- set --transaction-id <id> or
- set --transaction-duration-ms=<ms>
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Now that ConsumerRecord.deliveryCount() exists, enhance kafka-console-share-consumer.sh to exploit it. Added support to the DefaultMessageFormatter and the option print.delivery to the usage message for kafka-console-share-consumer.sh. Note that it was not added to kafka-console-consumer.sh even though the option would be recognised - this is because delivery with a consumer group does not count deliveries, and the result would include Delivery:NOT_PRESENT for all records if it was enabled - not really that useful with a consumer group.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Added kafka-console-share-consumer.sh which will start a share consumer on a share group.
This tool helps to read data from Kafka topics using share groups and outputs it to standard output.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
Introduces kafka-share-groups.sh for listing and describing share groups. The PR also contains the remaining options in the command parser and usage message in preparation of their implementation.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Apoorv Mittal <apoorvmittal10@gmail.com>
- Mark 3.9-IV0 as stable. Metadata version 3.9-IV0 should return Fetch version 17.
- Move ELR to 4.0-IV0. Remove 3.9-IV1 since it's no longer needed.
- Create a new 4.0-IV1 MV for KIP-848.
Reviewers: Jun Rao <junrao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Justine Olshan <jolshan@confluent.io>
Exit.scala is a wrapper around Exit.java, and its main benefit is that it allows methods to throw exceptions by returning Nothing. This benefit is minimal, and since we are planning to phase out Scala code, it would be better to use Exit.java directly and remove Exit.scala.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This PR adds support for add-controller and remove-controller in the kafka-metadata-quorum.sh
command-line tool. It also fixes some minor server-side bugs that blocked the tool from working.
In kafka-metadata-quorum.sh, the implementation of remove-controller is fairly straightforward. It
just takes some command-line flags and uses them to invoke AdminClient. The add-controller
implementation is a bit more complex because we have to look at the new controller's configuration
file. The parsing logic for the advertised.listeners and listeners server configurations that we
need was previously implemented in the :core module. However, the gradle module where
kafka-metadata-quorum.sh lives, :tools, cannot depend on :core. Therefore, I moved listener parsing
into SocketServerConfigs.listenerListToEndPoints. This will be a small step forward in our efforts
to move Kafka configuration out of :core.
I also made some minor changes in kafka-metadata-quorum.sh and Kafka-storage-tool.sh to handle
--help without displaying a backtrace on the screen, and give slightly better error messages on
stderr. Also, in DynamicVoter.toString, we now enclose the host in brackets if it contains a colon
(as IPV6 addresses can).
This PR fixes our handling of clusterId in addRaftVoter and removeRaftVoter, in two ways. Firstly,
it marks clusterId as nullable in the AddRaftVoterRequest.json and RemoveRaftVoterRequest.json
schemas, as it was always intended to be. Secondly, it allows AdminClient to optionally send
clusterId, by using AddRaftVoterOptions and RemoveRaftVoterOptions. We now also remember to
properly set timeoutMs in AddRaftVoterRequest. This PR adds unit tests for
KafkaAdminClient#addRaftVoter and KafkaAdminClient#removeRaftVoter, to make sure they are sending
the right things.
Finally, I fixed some minor server-side bugs that were blocking the handling of these RPCs.
Firstly, ApiKeys.ADD_RAFT_VOTER and ApiKeys.REMOVE_RAFT_VOTER are now marked as forwardable so that
forwarding from the broker to the active controller works correctly. Secondly,
org.apache.kafka.raft.KafkaNetworkChannel has now been updated to enable API_VERSIONS_REQUEST and
API_VERSIONS_RESPONSE.
Co-authored-by: Murali Basani muralidhar.basani@aiven.io
Reviewers: José Armando García Sancio <jsancio@apache.org>, Alyssa Huang <ahuang@confluent.io>
There is a lot of historical cruft in the usage message for kafka-topics.sh. For example, the --bootstrap-server option is required nowadays, but the usage message is written in some cases as if it was still optional.
This PR just tightens up the usage message, removing out-of-date text and improving the grammar.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
related to https://issues.apache.org/jira/browse/KAFKA-17235
The root cause of this issue is a change we introduced in KAFKA-16879, where we modified the PushHttpMetricsReporter constructor to use Time.System [1]. However, Time.System doesn't exist in Kafka versions 0.8.2 and 0.9.
In test_performance_services.py, we have system tests for Kafka versions 0.8.2 and 0.9 [2]. These tests always use the tools JAR from the trunk branch, regardless of the Kafka version being tested [3], while the client JAR aligns with the Kafka version specified in the test suite [4]. This discrepancy is what causes the issue to arise.
To resolve this issue, we have a few options:
1) Add Time.System to Kafka 0.8.2 and 0.9: This isn't practical, as we no longer maintain these versions.
2) Modify the PushHttpMetricsReporter constructor to use new SystemTime() instead of Time.System: This would contradict the intent of KAFKA-16879, which aims to make SystemTime a singleton.
3) Implement Time in PushHttpMetricsReporter use the time to get current time
4) Remove system tests for Kafka 0.8.2 and 0.9 from test_performance_services.py
Given that we no longer maintain Kafka 0.8.2 and 0.9, and altering the constructor goes against the design goals of KAFKA-16879, option 4 appears to be the most feasible solution. However, I'm not sure whether it's acceptable to remove these old version tests. Maybe someone else has a better solution
"We'll proceed with option 3 since support for versions 0.8 and 0.9 is still required, meaning we can't remove those Kafka versions from the system tests."
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This pr support EarliestLocalSpec LatestTierSpec in GetOffsetShell, and add integration tests.
Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, PoAn Yang <payang@apache.org>
we need to migate GroupMetadataMessageFormatter from scala code to java code,and make the message format is json pattern
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
describe --status now includes directory id and endpoint information for voter and observers.
describe --replication now includes directory id.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, José Armando García Sancio <jsancio@apache.org>
This change includes adding transaction.version (part of KIP-1022)
New transaction version 1 is introduced to support writing flexible fields in transaction state log messages.
Transaction version 2 is created in anticipation for further KIP-890 changes.
Neither are made production ready. Tests for the new transaction version and new MV are created.
Also include change to not report a feature as supported if the range is 0-0.
Reviewers: Jun Rao <junrao@apache.org>, David Jacot <djacot@confluent.io>, Artem Livshits <alivshits@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
As part of KIP-584, brokers expose a range of supported versions for each feature. For example, metadata.version might be supported from 1 to 21. (Note that feature level ranges are always inclusive, so this would include both level 1 and 21.)
These supported ranges are supposed to be able to include 0. For example, it should be possible for a broker to support a kraft.version between 0 and 1. However, in older software versions, there is an assertion in org.apache.kafka.common.feature.SupportedVersionRange that prevents this. This causes problems when the older software attempts to deserialize an ApiVersionsResponse containing such a range.
In order to resolve this dilemma, this PR bumps the version of ApiVersionsRequest from 3 to 4. Clients which send v4 promise to be able to handle ranges including 0. Clients which send v3 will not be exposed to these ranges. The feature will show up as having a minimum version of 1 instead. This work is part
of KIP-1022.
Similarly, this PR also introduces a new version of BrokerRegistrationRequest, and specifies that the
older versions of that RPC cannot handle supported version ranges including 0. Therefore, 0 is translated to 1 in the older requests.
Reviewers: Jun Rao <junrao@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>, Justine Olshan <jolshan@confluent.io>
This change implements response handling for the new version of Vote, Fetch, FetchSnapshot, BeginQuorumEpoch and EndQuorumEpoch. All of these responses were extended to include the leader's endpoint when the leader is known.
This change also includes sending the new version of the requests for Vote, Fetch, FetchSnapshot, BeginQuorumEpoch and EndQuorumEpoch. The two most notable changes are that:
1. The leader is going to include all of its endpoints in the BeginQuorumEpoch and EndQuorumEpoch requests.
2. The replica is going to include the destination replica key for the Vote and BeginQuorumEpoch request.
QuorumState was extended so that the replica transitions to UnattachedState instead of FollowerState during startup, if the leader is known but the leader's endpoint is not known. This can happen if the known leader is not part of the voter set replicated by the replica. The expectation is that the replica will rediscover the leader from Fetch responses from the bootstrap servers or from the BeginQuorumEpoch request from the leader.
To make sure that replicas never forget the leader of a given epoch the unattached state was extended to allow an optional leader id for when the leader is known but the leader's endpoint is not known.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
This patch partially reverts `group.version` in trunk. I kept the `GroupVersion` class but removed it from `Features` so it is not advertised. I also kept all the changes in the test framework. I removed the logic to require `group.version=1` to enable the new consumer rebalance protocol. The new protocol is enabled based on the static configuration.
For the context, I prefer to revert it in trunk now so we don't forget to revert it in the 3.9 release. I will bring it back for the 4.0 release.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Create 3 new metadata versions:
- 3.8-IV0, for the upcoming 3.8 release.
- 3.9-IV0, to add support for KIP-1005.
- 3.9-IV1, as the new release vehicle for KIP-966.
Create ListOffsetRequest v9, which will be used in 3.9-IV0 to support KIP-1005. v9 is currently an unstable API version.
Reviewers: Jun Rao <junrao@gmail.com>, Justine Olshan <jolshan@confluent.io>
This commit implements KIP-899: Allow producer and consumer clients to rebootstrap. It introduces the new setting `metadata.recovery.strategy`, applicable to all the types of clients.
Reviewers: Greg Harris <gharris1727@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
This patch introduces the `group.version` feature flag with one version:
1) Version 1 enables the new consumer group rebalance protocol (KIP-848).
Reviewers: Justine Olshan <jolshan@confluent.io>
This commit allows users to apply the scala version Formatters, but users will receive the warning messages about deprecation.
This compatibility support will be removed from 4.0.0
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This PR do the following cleanup for TestUtils.scala
1) remove unused methods
2) move methods used by single test class out of
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Some services are not closed, so our CI print following error.
org.opentest4j.AssertionFailedError: Found 16 unexpected threads during @BeforeAll: `kafka-admin-client-thread | adminclient-287,kafka-admin-client-thread | adminclient-276,kafka-admin-client-thread | adminclient-271,kafka-admin-client-thread | adminclient-293,kafka-admin-client-thread | adminclient-281,kafka-admin-client-thread | adminclient-302,kafka-admin-client-thread | adminclient-334,kafka-admin-client-thread | adminclient-323,kafka-admin-client-thread | adminclient-257,kafka-admin-client-thread | adminclient-336,kafka-admin-client-thread | adminclient-308,kafka-admin-client-thread | adminclient-263,kafka-admin-client-thread | adminclient-273,kafka-admin-client-thread | adminclient-278,kafka-admin-client-thread | adminclient-283,kafka-admin-client-thread | adminclient-317` ==> expected: <true> but was: <false>
#15679 use AfterEach to release service. However, the test cases having multi consumerConfigs will create a lot of services in testing. Hence, the intermediate servers are not closed.
Reviewers: Luke Chen <showuon@gmail.com>
The system test was failing because the VerifiableConsumer failed with a NullPointerException during startup. The reason for the NPE was an attempt to put a null as the value of --group-remote-assignor in the Consumer's configuration.
Reviewers: Lianet Magrans <lianetmr@gmail.com>, Lucas Brutschy <lbrutschy@confluent.io>
Add the support for DescribeTopicPartitions API to AdminClient. For this initial implementation, we are simply loading all of the results into memory on the client side.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Kirk True <ktrue@confluent.io>, David Jacot <djacot@confluent.io>, Artem Livshits <alivshits@confluent.io>, David Arthur <mumrah@gmail.com>
Currently, when executing kafka-reassign-partitions.sh with the --execute option, if a partition number specified in the JSON file does not exist, this check occurs only when submitting the reassignments to alterPartitionReassignments on the server-side.
We can perform this check in advance before submitting the reassignments to the server side.
Reviewers: Luke Chen <showuon@gmail.com>
Following test cases don't really run kraft case. The reason is that the test info doesn't contain parameter name, so it always returns false in TestInfoUtils#isKRaft.
1) TopicCommandIntegrationTest
2) DeleteConsumerGroupsTest
3) AuthorizerIntegrationTest
4) DeleteOffsetsConsumerGroupCommandIntegrationTest
We can fix it by adding options.compilerArgs << '-parameters' after
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Currently, when we using kafka-reassign-partitions to move the log directory, the output only indicates which replica's movement has successfully started.
This PR propose to show more detailed information, helping end users understand that the operation is proceeding as expected.
Reviewers: Luke Chen <showuon@gmail.com>, Andrew Schofield <aschofield@confluent.io>
1) This PR moves kafka.security classes from core to server module.
2) AclAuthorizer not moved, because it has heavy dependencies on core classes that not rewrited from scala at the moment.
3) AclAuthorizer will be deleted as part of ZK removal
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
There were different words for metadata.version like metadata version or metadataVersion. Unify format as metadata.version.
Reviewers: Luke Chen <showuon@gmail.com>
Fix getOffsetByMaxTimestamp for compressed records.
This PR adds:
1) For inPlaceAssignment case, compute the correct offset for maxTimestamp when traversing the batch records, and set to ValidationResult in the end, instead of setting to last offset always.
2) For not inPlaceAssignment, set the offsetOfMaxTimestamp for the log create time, like non-compressed, and inPlaceAssignment cases, instead of setting to last offset always.
3) Add tests to verify the fix.
Reviewers: Jun Rao <junrao@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
This PR is part of #14471
It contains some of ConsoleGroupCommand tests rewritten in java.
Intention of separate PR is to reduce changes and simplify review.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Is contains some of ConsoleGroupCommand tests rewritten in java.
Intention of separate PR is to reduce changes and simplify review.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
In KIP-848, we introduce the notion of Group Types based on the protocol type that the members in the consumer group use. As of now we support two types of groups:
* Classic : Members use the classic consumer group protocol ( existing one )
* Consumer : Members use the consumer group protocol introduced in KIP-848.
Currently List Groups allows users to list all the consumer groups available. KIP-518 introduced filtering the consumer groups by the state that they are in. We now want to allow users to filter consumer groups by type.
This patch includes the changes to the admin client and related files. It also includes changes to parameterize the tests to include permutations of the old GC and the new GC with the different protocol types.
Reviewers: David Jacot <djacot@confluent.io>
Introduces a new filter in ListTransactionsRequest API. This enables caller to filter on transactions that have been running for longer than a certain duration of time.
This PR includes the following changes:
bumps version for ListTransactionsRequest API to 1. Set the durationFilter to -1L when communicating with an older broker that does not support version 1.
bumps version for ListTransactionsResponse to 1 without changing the response structure.
adds durationFilter option to kafka-transactions.sh --list
Tests:
Client side test to build request with correct combination of duration filter and API version: testBuildRequestWithDurationFilter
Server side test to filter transactions based on duration: testListTransactionsFiltering
Added test case for kafka-transactions.sh change in TransactionsCommandTest
Reviewers: Justine Olshan <jolshan@confluent.io>, Raman Verma <rverma@confluent.io>
This PR improves TopicCommandIntegrationTest by :
- using TestUtils.createTopicWithAdmin
- replacing \n with lineSeperator
- using waitForAllReassignmentsToComplete
- adding more log when assertion fails
Reviewers: Luke Chen <showuon@gmail.com>, Justine Olshan <jolshan@confluent.io>
While reviewing https://github.com/apache/kafka/pull/15150, I found that our tests verifying the console output are really hard to read. Here is my proposal to make it better.
Reviewers: Ritika Reddy <rreddy@confluent.io>, Justine Olshan <jolshan@confluent.io>
The Python VerifiableConsumer now passes in the --group-protocol and --group-remote-assignor command line arguments to VerifiableConsumer if the node is running 3.7.0+ and using the new consumer group.protocol.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
This PR is part of #14471
Is contains some of ConsoleGroupCommand tests rewritten in java.
Intention of separate PR is to reduce changes and simplify review.
Reviewers: Luke Chen <showuon@gmail.com>
Add the optional --group-protocol command line option that can be set in the system tests
Reviewers: Andrew Schofield <aschofield@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>
Add a HeartbeatMetrics module to measure various metrics related to the heartbeat. Here is the highlight of the changes:
HeartMetrics encapsulate a heartbeat sensor
Add metrics suffix types to the AbstractConsumerMetrics so that it can be extended to different metric groups
Non-related refactor:
Rename the metric classes to "MetricManager" the reason being "Metrics" seems to relate to the Metric class however these managers are merely containers holding sensor references for recording purposes.
Created OffsetCommitMetricsManager so that we are more consistent with the metrics management
Extended KafkaConsumerMetrics to the AbstractConsumerMetrics so that the groupName is consistently created (without random concatenation)
Some follow ups:
Refactor commit sensor by introducing a CommitMetrics module so that we can keep most of the metrics in one place and let it be a bit more consistent stylistically
Possibly refactor fetch manager metrics
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
This PR is part of #14471
Is contains single test rewritten in java.
Intention of separate PR is to reduce changes and simplify review.
Reviewers: Justine Olshan <jolshan@confluent.io>
This PR is part of #14471
It contains ConsumerGroupCommandOptions and case classes used by ConsumerGroupCommand rewritten in java.
The goal of PR is to reduce main PR size.
Co-authored-by: Taras Ledkov <tledkov@apache.org>
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Taras Ledkov <tledkov@apache.org>, Justine Olshan <jolshan@confluent.io>
This PR creates MetadataVersion.latestTesting to represent the highest metadata version (which may be unstable) and MetadataVersion.latestProduction to represent the latest version that should be used in production. It fixes a few cases where the broker was advertising that it supported the testing versions even when unstable metadata versions had not been configured.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
This commit fixes a bug in LeaderElectionCommand due to which custom timeout configuration was not being respected.
Reviewers: Divij Vaidya <diviv@amazon.com>, Proven Provenzano <pprovenzano@confluent.io>
Moves ELR from MetadataVersion IBP_3_7_IV3 into the new IBP_3_8_IV0 because the ELR feature was not completed before 3.7 reached feature freeze. Leaves IBP_3_7_IV3 empty -- it is a no-op and is not reused for anything. Adds the new MetadataVersion IBP_3_7_IV4 for the FETCH request changes from KIP-951, which were mistakenly never associated with a MetadataVersion. Updates the LATEST_PRODUCTION MetadataVersion to IBP_3_7_IV4 to declare both KRaft JBOD and the KIP-951 changes ready for production use.
Reviewers: Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>, Ron Dagostino <rdagostino@confluent.io>, Ismael Juma <ismael@juma.me.uk>, José Armando García Sancio <jsancio@apache.org>, Justine Olshan <jolshan@confluent.io>
Initial implementation of kafka-client-metrics.sh tools for KIP-714 and KIP-1000.
Reviewers: Igor Soarez <soarez@apple.com>, Jun Rao <junrao@gmail.com>
This is a follow-up to https://github.com/apache/kafka/pull/14687 as we found out that some parameterized tests do not include the test method name in their name. For the context, the JUnit XML report does not include the name of the method by default but only rely on the display name provided.
Reviewers: David Arthur <mumrah@gmail.com>
Assign MetadataVersion.IBP_3_7_IV2 to JBOD.
Move KIP-966 support to MetadataVersion.IBP_3_7_IV3.
Create MetadataVersion.LATEST_PRODUCTION as the latest metadata version that can be used when formatting a
new cluster, or upgrading a cluster using kafka-features.sh. This will allow us to clearly distinguish between stable
and unstable metadata versions for the first time.
Reviewers: Igor Soarez <soarez@apple.com>, Ron Dagostino <rndgstn@gmail.com>, Calvin Liu <caliu@confluent.io>, Proven Provenzano <pprovenzano@confluent.io>
The topic name was displayed as `Optional<String>` when the topic was created.
```
% bin/kafka-topics.sh --create --topic test --bootstrap-server localhost:9092
Created topic Optional[test].
```
This PR fixed to print the topic name as `String` instead of `Optional<String>`.
```
% bin/kafka-topics.sh --create --topic test --bootstrap-server localhost:9092
Created topic test.
```
Reviewers: Ismael Juma <ismael@juma.me.uk>
This PR contains changes required to move PartitionReassignmentState class to java code.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Justine Olshan <jolshan@confluent.io>, Federico Valeri <fedevaleri@gmail.com>, Taras Ledkov Taras Ledkov <tledkov@apache.org>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>,
This patch introduces preliminary changes for Eligible Leader Replicas (KIP-966)
* New MetadataVersion 16 (3.7-IV1)
* New record versions for PartitionRecord and PartitionChangeRecord
* New tagged fields on PartitionRecord and PartitionChangeRecord
* New static config "eligible.leader.replicas.enable" to gate the whole feature
Reviewers: Artem Livshits <alivshits@confluent.io>, David Arthur <mumrah@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
In KIP-778 we introduced the "unsafe" (lossy) downgrade in case metadata has changes in one of the versions between target and current, as defined in MetadataVersion.
The documentation says it is possible:
"Note that the cluster metadata version cannot be downgraded to a pre-production 3.0.x, 3.1.x, or 3.2.x version once it has been upgraded. However, it is possible to downgrade to production versions such as 3.3-IV0, 3.3-IV1, etc."
The command line tool shows that this doesn't work:
bin/kafka-features.sh --bootstrap-server :9092 downgrade --metadata 3.4 --unsafe
Could not downgrade metadata.version to 8. Invalid metadata.version 8. Unsafe metadata downgrade is not supported in this version.
1 out of 1 operation(s) failed.
In addition to unsafe, also safe metadata downgrades are not supported in practice. For example, when you upgrade to 3.5, you land on 3.5-IV2 as metadata version, which has metadata changes and won't let you to downgrade. This is true for every other release at the moment.
This change fixes the documentation to reflect that, and improves the error messages.
Signed-off-by: Federico Valeri <fedevaleri@gmail.com>
Reviewers: Luke Chen <showuon@gmail.com>, Jakub Scholz <github@scholzj.com>
This PR is part of #13247
It contains ReassignPartitionsIntegrationTest rewritten in java.
Goal of PR is reduce changes size in main PR.
Reviewers: Taras Ledkov <tledkov@apache.org>, Justine Olshan <jolshan@confluent.io>
Add support for --bootstrap-controller in the following command-line tools:
- kafka-cluster.sh
- kafka-configs.sh
- kafka-features.sh
- kafka-metadata-quorum.sh
To implement this, the following AdminClient APIs now support the new bootstrap.controllers
configuration:
- Admin.alterConfigs
- Admin.describeCluster
- Admin.describeConfigs
- Admin.describeFeatures
- Admin.describeMetadataQuorum
- Admin.incrementalAlterConfigs
- Admin.updateFeatures
Command-line tool changes:
- Add CommandLineUtils.initializeBootstrapProperties to handle parsing --bootstrap-controller
in addition to --bootstrap-server.
- Add --bootstrap-controller to ConfigCommand.scala, ClusterTool.java, FeatureCommand.java, and
MetadataQuorumCommand.java.
KafkaAdminClient changes:
- Add the AdminBootstrapAddresses class to handle extracting bootstrap.servers or
bootstrap.controllers from the config map for KafkaAdminClient.
- In AdminMetadataManager, store the new usingBootstrapControllers boolean. Generalize
authException to encompass the concept of fatal exceptions in general. (For example, the
fatal exception where we talked to the wrong node type.) Treat
MismatchedEndpointTypeException and UnsupportedEndpointTypeException as fatal exceptions.
- Extend NodeProvider to include information about whether bootstrap.controllers is supported.
- Modify the APIs described above to support bootstrap.controllers.
Server-side changes:
- Support DescribeConfigsRequest on kcontrollers.
- Add KRaftMetadataCache to the kcontroller to simplify implemeting describeConfigs (and
probably more APIs in the future). It's mainly a wrapper around MetadataImage, so there is
essentially no extra resource consumption.
- Split RuntimeLoggerManager out of ConfigAdminManager to handle the incrementalAlterConfigs
support for BROKER_LOGGER. This is now supported on kcontrollers as well as brokers.
- Fix bug in AuthHelper.computeDescribeClusterResponse that resulted in us always sending back
BROKER as the endpoint type, even on the kcontroller.
Miscellaneous:
- Fix a few places in exceptions and log messages where we wrote "broker" instead of "node".
For example, an exception in NodeApiVersions.java, and a log message in NetworkClient.java.
- Fix the slf4j log prefix used by KafkaRequestHandler logging so that request handlers on a
controller don't look like they're on a broker.
- Make the FinalizedVersionRange constructor public for the sake of a junit test.
- Add unit and integration tests for the above.
Reviewers: David Arthur <mumrah@gmail.com>, Doguscan Namal <namal.doguscan@gmail.com>
This PR is part of #13247
It contains changes to rewrite single test in java.
Intention is reduce changes in parent PR.
Reviewers: Luke Chen <showuon@gmail.com>, Taras Ledkov <tledkov@apache.org>
Implement KIP-919: Allow AdminClient to Talk Directly with the KRaft Controller Quorum and add
Controller Registration. This KIP adds a new version of DescribeClusterRequest which is supported
by KRaft controllers. It also teaches AdminClient how to use this new DESCRIBE_CLUSTER request to
talk directly with the controller quorum. This is all gated behind a new MetadataVersion,
IBP_3_7_IV0.
In order to share the DESCRIBE_CLUSTER logic between broker and controller, this PR factors it out
into AuthHelper.computeDescribeClusterResponse.
The KIP adds three new errors codes: MISMATCHED_ENDPOINT_TYPE, UNSUPPORTED_ENDPOINT_TYPE, and
UNKNOWN_CONTROLLER_ID. The endpoint type errors can be returned from DescribeClusterRequest
On the controller side, the controllers now try to register themselves with the current active
controller, by sending a CONTROLLER_REGISTRATION request. This, in turn, is converted into a
RegisterControllerRecord by the active controller. ClusterImage, ClusterDelta, and all other
associated classes have been upgraded to propagate the new metadata. In the metadata shell, the
cluster directory now contains both broker and controller subdirectories.
QuorumFeatures previously had a reference to the ApiVersions structure used by the controller's
NetworkClient. Because this PR removes that reference, QuorumFeatures now contains only immutable
data. Specifically, it contains the current node ID, the locally supported features, and the list
of quorum node IDs in the cluster.
Reviewers: David Arthur <mumrah@gmail.com>, Ziming Deng <dengziming1993@gmail.com>, Luke Chen <showuon@gmail.com>
#14083 added support for delegation tokens in KRaft and attached that support to the existing
MetadataVersion 3.6-IV1. This patch moves that support into a separate MetadataVersion 3.6-IV2.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Implement the QuorumController side of KRaft metadata transactions.
As specified in KIP-868, this PR creates a new metadata version, IBP_3_6_IV1, which contains the
three new records: AbortTransactionRecord, BeginTransactionRecord, EndTransactionRecord.
In order to make offset management unit-testable, this PR moves it out of QuorumController.java and
into OffsetControlManager.java. The general approach here is to track the "last stable offset," which is
calculated by looking at the latest committed offset and the in-progress transaction (if any). When
a transaction is aborted, we revert back to this last stable offset. We also revert back to it when
the controller is transitioning from active to inactive.
In a follow-up PR, we will add support for the transaction records in MetadataLoader. We will also
add support for automatically aborting pending transactions after a controller failover.
Reviewers: David Arthur <mumrah@gmail.com>
This PR adds CommandDefaultOptions usage like in the other joptsimple based tools. It also moves the associated unit test class from streams to tools module as discussed in #13127 (comment)
Reviewers: Luke Chen <showuon@gmail.com>, Bruno Cadonna <cadonna@apache.org>, Sagar Rao <sagarmeansocean@gmail.com>
In JmxTool.scala, we will wait till all the object names are available from MBean server. But in the newer version, we only wait for subset of object names. Due to this, we may not enforce wait option and prematurely return the result if the objects are not yet registered in MBean sever.
Reviewers: Luke Chen <showuon@gmail.com>, Federico Valeri <fvaleri@redhat.com>
When the KRaft controller removes a replica from the ISR because of the controlled shutdown there is no need for the leader epoch to be increased by the KRaft controller. This is accurate as long as the topic partition leader doesn't add the removed replica back to the ISR.
This change also fixes a bug when computing the HWM. When computing the HWM, replicas that are not eligible to join the ISR but are caught up should not be included in the computation. Otherwise, the HWM will never increase for replica.lag.time.max.ms because the shutting down replica is not sending FETCH request. Without this additional fix PRODUCE requests would timeout if the request timeout is greater than replica.lag.time.max.ms.
Because of the bug above the KRaft controller needs to check the MV to guarantee that all brokers support this bug fix before skipping the leader epoch bump.
Reviewers: David Mao <47232755+splett2@users.noreply.github.com>, Divij Vaidya <diviv@amazon.com>, David Jacot <djacot@confluent.io>
When running kafka-metadata-quorum script to get the quorum replication status, the LastFetchTimestamp and LastCaughtUpTimestamp output is not human-readable.
I will be convenient to add an optional flag (-hr, --human-readable) to enable a human-readable format showing the delay in ms (i.e. 366 ms ago).
This dealy is computed as (now - timestamp), where they are both represented as Unix time (UTC based).
$ bin/kafka-metadata-quorum.sh --bootstrap-server :9092 describe --replication --human-readable
NodeId LogEndOffset Lag LastFetchTimestamp LastCaughtUpTimestamp Status
2 61 0 5 ms ago 5 ms ago Leader
3 61 0 56 ms ago 56 ms ago Follower
4 61 0 56 ms ago 56 ms ago Follower
Reviewers: Luke Chen <showuon@gmail.com>
This test was reported as flaky on CI server.
When connecting to a multi-homed machine using RMI, the wrong address may be returned by the RMI registry to the client, causing the connection to the RMI server to timeout.
This change explicitly set the hostname returned to the the clients in the remote stub object.
Reviewers: Luke Chen <showuon@gmail.com>, vamossagar12 <sagarmeansocean@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, hudeqi <16120374@bjtu.edu.cn>, Christo Lolov <christololov@gmail.com>
Why:
Using java.util.Random to generate every byte sent from the ProducerPerformance
appears to be a limiting factor. Throughput of the ProducerPerformance script is
higher with a file of records as compared to randomly generated records.
On my machine a single thread can generate ~100MB/second of uppercase letters using
java.util.Random and ~300MB/sec using java.util.SplittableRandom. This is a limit on
throughput.
Note: you can optimise further by expanding it from 26 letters to 32 letter generated
as it is more efficient to generate a nicely distributed int when the bound is a
power of two.
Reviewers: Luke Chen <showuon@gmail.com>
…g option
https://github.com/apache/kafka/pull/12951 accidentally changed the behavior of the `kafaka-metadata-quorum.sh` CLI by making it silently ignore a `--command-config <filename>` properties file that exists. This was an undetected regression in the 3.4.0 release. This patch fixes the issue such that any such specified file will be honored.
Reviewers: José Armando García Sancio <jsancio@apache.org>, Ismael Juma <ismael@juma.me.uk>
In MetadataQuorumCommandTest, we sometimes got the error:
java.util.concurrent.ExecutionException: java.lang.RuntimeException: Received a fatal error while waiting for the broker to catch up with the current cluster metadata.
Since we tried to bring up 3 broker + 3 controllers at the same time, and the config initial.broker.registration.timeout.ms (default 1 min) is sometimes not enough for them to start up. Checking the tests, it doesn't require so many nodes. Reducing the nodes number to make these tests reliable.
Reviewers: dengziming <dengziming1993@gmail.com>, Ismael Juma <ismael@juma.me.uk>
`core` should only be used for legacy cli tools and tools that require
access to `core` classes instead of communicating via the kafka protocol
(typically by using the client classes).
Summary of changes:
1. Convert the command implementation and tests to Java and move it to
the `tools` module.
2. Introduce mechanism to capture stdout and stderr from tests.
3. Change `kafka-metadata-quorum.sh` to point to the new command class.
4. Adjusted the test classpath of the `tools` module so that it supports tests
that rely on the `@ClusterTests` annotation.
5. Improved error handling when an exception different from `TerseFailure` is
thrown.
6. Changed `ToolsUtils` to avoid usage of arrays in favor of `List`.
Reviewers: dengziming <dengziming1993@gmail.com>
* KAFKA-13725: KIP-768 OAuth code mixes public and internal classes in same package
Move classes into a sub-package of "internal" named "secured" that
matches the layout more closely of the "unsecured" package.
Replaces the concrete implementations in the former packages with
sub-classes of the new package layout and marks them as deprecated. If
anyone is already using the newer OAuth code, this should still work.
* Fix checkstyle and spotbugs violations
Co-authored-by: Kirk True <kirk@mustardgrain.com>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Add `MetadataQuorumCommand` to describe quorum status, I'm trying to use arg4j style command format, currently, we only support one sub-command which is "describe" and we can specify 2 arguments which are --status and --replication.
```
# describe quorum status
kafka-metadata-quorum.sh --bootstrap-server localhost:9092 describe --replication
ReplicaId LogEndOffset Lag LastFetchTimeMs LastCaughtUpTimeMs Status
0 10 0 -1 -1 Leader
1 10 0 -1 -1 Follower
2 10 0 -1 -1 Follower
kafka-metadata-quorum.sh --bootstrap-server localhost:9092 describe --status
ClusterId: fMCL8kv1SWm87L_Md-I2hg
LeaderId: 3002
LeaderEpoch: 2
HighWatermark: 10
MaxFollowerLag: 0
MaxFollowerLagTimeMs: -1
CurrentVoters: [3000,3001,3002]
CurrentObservers: [0,1,2]
# specify AdminClient properties
kafka-metadata-quorum.sh --bootstrap-server localhost:9092 --command-config config.properties describe --status
```
Reviewers: Jason Gustafson <jason@confluent.io>
This patch fixes some strangeness and inconsistency in the messages written by `TransactionalMessageCopier` to stdout. Here is a sample of two messages.
Progress message:
```
{"consumed":33000,"stage":"ProcessLoop","totalProcessed":33000,"progress":"copier-0","time":"2022/04/24 05:40:31:649","remaining":333}
```
The `transactionalId` is set to the value of the `progress` key.
And a shutdown message:
```
{"consumed":33333,"shutdown_complete":"copier-0","totalProcessed":33333,"time":"2022/04/24 05:40:31:937","remaining":0}
```
The `transactionalId` this time is set to the `shutdown_complete` key and there is no `stage` key.
In this patch, we change the following:
1. Use a separate key for the `transactionalId`.
2. Drop the `progress` and `shutdown_complete` keys.
3. Use `stage=ShutdownComplete` in the shutdown message.
4. Modify `transactional_message_copier.py` system test service accordingly.
Reviewers: David Arthur <mumrah@gmail.com>
The class PushHttpMetricsReporter no longer pushes metrics when network failure is recovered.
I debugged the code and found the problem here: when we submit a task to the ScheduledThreadPoolExecutor that needs to be executed periodically, if the task throws an exception and is not swallowed, the task will no longer be scheduled to execute.
So when an IO exception occasionally occurs on the network, we should swallow it rather than throw it in task HttpReporter.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
This task is to provide a concrete implementation of the interfaces defined in KIP-255 to allow Kafka to connect to an OAuth/OIDC identity provider for authentication and token retrieval. While KIP-255 provides an unsecured JWT example for development, this will fill in the gap and provide a production-grade implementation.
The OAuth/OIDC work will allow out-of-the-box configuration by any Apache Kafka users to connect to an external identity provider service (e.g. Okta, Auth0, Azure, etc.). The code will implement the standard OAuth client credentials grant type.
The proposed change is largely composed of a pair of AuthenticateCallbackHandler implementations: one to login on the client and one to validate on the broker.
See the following for more detail:
KIP-768
KAFKA-13202
Reviewers: Yi Ding <dingyi.zj@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>