Partition epochs are tracked for consumer groups where epoch is the current assigned member epoch. As share groups have partitions shared hence maintaing the partition epochs is not required.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
In old code leaderId and oldLeaderId all use randomReplicaId(), thus it will be flaky when leaderId is equals oldLeaderId. When KafkaRaftClient initialize it will find leaderId and oldLeaderId, so it will tansfer to candidate, and the leaderEpoch will be + 1
Reviewers: TengYao Chi <kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
Only voters should be able to transition to Candidate state. This removes VotedState as one of the EpochStates and moves voted information into UnattachedState.
Reviewers: José Armando García Sancio <jsancio@apache.org>
In MetadataVersion 3.7-IV2 and above, the broker's AssignmentsManager sends an RPC to the
controller informing it about which directory we have chosen to place each new replica on.
Unfortunately, the code does not check to see if the topic still exists in the MetadataImage before
sending the RPC. It will also retry infinitely. Therefore, after a topic is created and deleted in
rapid succession, we can get stuck including the now-defunct replica in our subsequent
AssignReplicasToDirsRequests forever.
In order to prevent this problem, the AssignmentsManager should check if a topic still exists (and
is still present on the broker in question) before sending the RPC. In order to prevent log spam,
we should not log any error messages until several minutes have gone past without success.
Finally, rather than creating a new EventQueue event for each assignment request, we should simply
modify a shared data structure and schedule a deferred event to send the accumulated RPCs. This
will improve efficiency.
Reviewers: Igor Soarez <i@soarez.me>, Ron Dagostino <rndgstn@gmail.com>
Make DelayedOperations#checkAndCompleteAll has chance to complete delayed ops even though there is a exception caused by one of delayed op.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
- StreamsConfig#RETRIES_CONFIG was deprecated in AK 2.7 and is no longer in use.
- StreamsConfig#DEFAULT_WINDOWED_KEY_SERDE_INNER_CLASS and
- StreamsConfig#DEFAULT_WINDOWED_VALUE_SERDE_INNER_CLASS were deprecated in AK 3.0.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
The initial drop of ShareMembershipManager contained a lot of code duplicated from MembershipManagerImpl. The plan was always to share as much code as possible between the membership managers for consumer groups and share groups. This issue refactors the membership managers so that almost all of the code is in common.
Reviewers: Lianet Magrans <lianetmr@gmail.com>, Chia Chuan Yu <yujuan476@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
This patch enables the static member using the old protocol to be replaced in ConsumerGroupHeartbeat even if it hasn't sent a request to leave the group.
This helps in cases where existing static member rejoins and triggers a group upgrade, because the classic static consumer doesn't send leave group request before shutting down.
Reviewers: TengYao Chi <kitingiao@gmail.com>, David Jacot <djacot@confluent.io>
When handling an OFFSET_FETCH request requiring stable offsets, the new
group coordinator may encounter a timeout under some circumstances, such
as a zombie coordinator or a lagging __consumer_offsets replica that has
not yet dropped out of the ISR. Existing and older clients do not expect
the REQUEST_TIMED_OUT error code won't retry, so remap it to
NOT_COORDINATOR to trigger a coordinator lookup and retry.
Reviewers: David Jacot <djacot@confluent.io>
This change implements the KRaft voter sending UpdateVoter request. The
UpdateVoter RPC is used to update a voter's listeners and supported
kraft versions. The UpdateVoter RPC is sent if the replicated voter set
(VotersRecord in the log) doesn't match the local voter's supported
kraft versions and controller listeners.
To not starve the Fetch request, the UpdateVoter request is sent at most
every 3 fetch timeouts. This is required to make sure that replication
is making progress and eventually the voter set in the replicated log
matches the local voter configuration.
This change also modifies the semantic for UpdateVoter. Now the
UpdateVoter response is sent right after the leader has created the new
voter set. This is required so that updating voter can transition from
sending UpdateVoter request to sending Fetch request. If the leader
waits for the VotersRecord control record to commit before sending the
UpdateVoter response, it may never send the UpdateVoter response. This
can happen if the leader needs that voter's Fetch request to commit the
control record.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
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>
* KAFKA-15875: Stops leak Snapshot in public methods
The Snapshot class is package protected but it's returned in
several public methods in SnapshotRegistry.
To prevent this accidental leakage, these methods are made
package protected as well. For getOrCreateSnapshot a new
method called IdempotentCreateSnapshot is created that returns void.
* Make builer package protected, replace <br> with <p>
Reviewers: Greg Harris <greg.harris@aiven.io>
In ShareConsumeRequestManager, currently every time we perform a commitSync/commitAsync/acknowledgeOnClose we create one AcknowledgeRequestState for each call. But this can be optimised further as we can batch up the acknowledgements to be sent to the same node before the next poll() is invoked.
This will ensure that between 2 polls, the acknowledgements are accumulated in one request per node and then sent during poll, resulting in lesser RPC calls.
To achieve this, we are storing a pair of acknowledge request states for every node, the first value denotes the requestState for commitAsync() and the second value denotes the requestState for commitSync() and acknowledgeOnClose(). All the acknowledgements to be sent to a particular node are stored in the corresponding acknowledgeRequestState based on whether it was synchronous or asynchronous.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
Handle local log deletion when remote.log.copy.disabled=true based on the KIP-950.
When tiered storage is disabled or becomes read-only on a topic, the local retention configuration becomes irrelevant, and all data expiration follows the topic-wide retention configuration exclusively.
- added remoteLogEnabledAndRemoteCopyEnabled method to check if this topic enables tiered storage and remote log copy is enabled. We should adopt local.retention.ms/bytes when remote.storage.enable=true,remote.log.copy.disable=false.
- Changed to use retention.bytes/retention.ms when remote copy disabled.
- Added validation to ask users to set local.retention.ms == retention.ms and local.retention.bytes == retention.bytes
- Added tests
Reviewers: Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Satish Duggana <satishd@apache.org>, Christo Lolov <lolovc@amazon.com>
The replicaDirectoryId field for FetchRequest and FetchSnapshotRequest should be ignorable. This allows data objects with the directory id to be serialized to any version of the requests.
Reviewers: José Armando García Sancio <jsancio@apache.org>, Chia-Ping Tsai <chia7712@apache.org>
Adds the methods to the admin client for listing and describing share groups.
There are some unit tests, but integration tests will be in a follow-on PR.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
The actual type name in the JSON descriptor files is "bool" not "boolean"
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Andrew Schofield <andrew_schofield@live.com>
The method overrides of metricName in KafkaMetricsGroup are no longer required since there's a new constructor that implement this with the same behavior.
Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
Implemented handleShareFetch request RPC in KafkaApis.scala. This method is called whenever the client sends a Share Fetch request to the broker. Although Share Fetch request support acknowledgements, since the logic for acknowledging records is not completely implemented in SharePartitionManager.java class, this method currently includes placeholder code for acknowledging, which will be replaced by the actual functionality in the upcoming PRs.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, Abhinav Dixit <adixit@confluent.io>, Jun Rao <junrao@gmail.com>
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>
Add support for handling the update voter RPC. The update voter RPC is used to automatically update
the voters supported kraft versions and available endpoints as the operator upgrades and
reconfigures the KRaft controllers.
The add voter RPC is handled as follow:
1. Check that the leader has fenced the previous leader(s) by checking that the HWM is known;
otherwise, return the REQUEST_TIMED_OUT error.
2. Check that the cluster supports kraft.version 1; otherwise, return the UNSUPPORTED_VERSION error.
3. Check that there are no uncommitted voter changes, otherwise return the REQUEST_TIMED_OUT error.
4. Check that the updated voter still supports the currently finalized kraft.version; otherwise
return the INVALID_REQUEST error.
5. Check that the updated voter is still listening on the default listener.
6. Append the updated VotersRecord to the log. The KRaft internal listener will read this
uncommitted record from the log and update the voter in the set of voters.
7. Wait for the VotersRecord to commit using the majority of the voters. Return a REQUEST_TIMED_OUT
error if it doesn't commit in time.
8. Send the UpdateVoter successful response to the voter.
This change also implements the ability for the leader to update its own entry in the voter
set when it becomes leader for an epoch. This is done by updating the voter set and writing a
control batch as the first batch in a new leader epoch.
Finally, fix a bug in KafkaAdminClient's handling of removeRaftVoterResponse where we tried to cast
the response to the wrong type.
Reviewers: Alyssa Huang <ahuang@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
* KAFKA-17227: Update zstd-jni lib
* Add note in upgrade docs
* Change zstd-jni version in docker native file and add warning in dependencies.gradle file
* Add reference to snappy in upgrade
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Mickael Maison <mickael.maison@gmail.com>
The purpose of this PR is to remove ConsumerTestBuilder.java since it is no longer needed. The following PRs have eliminated the use of ConsumerTestBuilder:
#14930#16140#16200#16312
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>
When reverting the ZK migration, we must also remove the /migration ZNode in order to allow the migration to be re-attempted in the future.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>