This patch is the first of a series of patches to introduce support for server side regular expression. It introduces the re2j dependency.
Co-authored-by: Lianet Magrans <lmagrans@confluent.io>
Reviewers: Lianet Magrans <lmagrans@confluent.io>
Adds the DefaultStatePersister and other supporting classes for managing share state.
* Added DefaultStatePersister implementation. This is the entry point for callers who wish to invoke the share state RPC API.
* Added PersisterStateManager which is used by DefaultStatePersister to manage and send the RPCs over the network.
* Added code to BrokerServer and BrokerMetadataPublisher to instantiate the appropriate persister based on the config value for group.share.persister.class.name. If this is not specified, the DefaultStatePersister will be used. To force use of NoOpStatePersister, set the config to empty. This is an internal config, not to be exposed to the end user. This will be used to factory plug the appropriate persister.
* Using this persister, the internal __share_group_state topic will come to life and will be used for persistence of share group info.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Jun Rao <junrao@gmail.com>, David Arthur <mumrah@gmail.com>
As part of KIP-932, a new internal topic __share_group_state was introduced. There are 2 types of records which are currently being added in this topic - ShareSnapshotKey/Value and ShareUpdateKey/Value
In light of this, we must make the existing tooling like kafka-console-consumer and kafka-dump-log aware of these records for debugging and introspection purposes.
This PR introduces ShareGroupStateMessageFormatter to be used used with kafka-console-consumer and adds an internal class ShareGroupStateMessageParser in DumpLogSegments.scala.
Unit tests have been added to DumpLogSegmentsTest.scala
Reviewers: Andrew Schofield <aschofield@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
Introduce new quarantinedTest that excludes tests tagged with "flaky". Also introduce two new build parameters "maxQuarantineTestRetries" and "maxQuarantineTestRetryFailures".
Reviewers: Chia-Ping Tsai <chia7712@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>
Introduces the share coordinator. This coordinator is built on the new coordinator runtime framework. It
is responsible for persistence of share-group state in a new internal topic named "__share_group_state".
The responsibility for being a share coordinator is distributed across the brokers in a cluster.
Reviewers: David Arthur <mumrah@gmail.com>, Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>
The ignoreFailures property was removed in #17066 to prevent test failures from being cached. However, this breaks the JUnit report and makes the github workflow less user friendly.
The problem is that we are copying the junit test report files into a new directory (added in #17098) in a Gradle doLast closure. If we don't run with ignoreFailures=true, then this closure will not run and the test failures won't be processed by junit.py.
This patch adds logic to ensure the doLast closure of :test is always run. The user provided -PignoreFailures is still honored for the test tasks so local developer workflows should not be disturbed.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Recently, we fixed caching for ":jar" and ":test" tasks. A side effect of this is that the test results will be restored as part of the Gradle cache resolution. This means test tasks which are skipped (as a result of FROM-CACHE) will still have test results in their build directory. To avoid incorrectly reporting these results in the job summary, this patch uses a doLast task handler to relocate JUnit XML files into a new directory.
This patch also removes the "continue-on-error" from the JUnit test step which caused timed-out builds to appear successful.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
For several modules, we include a kafka-version.properties in the Jar file. This file includes the Git SHA of the project at the time of the build. This means that even if no source files change, the :jar task will never be UP-TO-DATE between two git commits. Ultimately, this breaks Gradle caching.
This patch marks all of the createVersionFile tasks as cacheable and also changes our Gradle invocation to override the commit ID to a dummy static value. This will allow the :jar task to be cacheable and reusable between builds.
This patch also configures the trunk build to only write to the build cache and not read from it. This will prevent any cache pollution/corruption from propagating from build to build.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This patch adds a "deflake" github action which can be used to run a single JUnit test or suites. It works by parameterizing the --tests Gradle option. If the test extends ClusterTest, the "deflake" workflow can repeat number of times by setting the kafka.cluster.test.repeat system property.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Introduce ShareCoordinator interface and related classes.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
This patch introduces a wrapper around [HdrHistogram](https://github.com/HdrHistogram/HdrHistogram) to use for group coordinator histograms, event queue time, event processing time, flush time, and purgatory time.
Reviewers: David Jacot <djacot@confluent.io>
Establishes the new `:share` Gradle module. This module is intended to be used for server-side KIP-932 classes that are not part of the new share group coordinator.
This patch relocates and renames some existing classes. A small amount of compatibility changes were also made, but do not affect any logic.
Reviewers: Andrew Schofield <aschofield@confluent.io>, David Arthur <mumrah@gmail.com>
What
Introduced a new module share-coordinator to house relevant implementation code and resources.
Modified settings.gradle and build.gradle to accommodate the module.
Added ShareSnapshot[Key, Value], ShareUpdate[Key, Value] message record schemas.
Introduced a trivial impl of ShareCoordinatorShard class to establish dependencies with other modules (:coordinator-common, :metadata). The actual impl for this class will be done in future PRs.
Why
The share coordinator component has been introduced as part of KIP-932 (QFK). This component is will be responsible for managing persistence for various data related to share partitions into a dedicated internal topic.
To keep all this functionality contained, we want to create a separate module in line with group and transaction coordinators.
Reviewers: Andrew Schofield <aschofield@confluent.io>, Apoorv Mittal <apoorvmittal10@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
There is a lot of code in group-coordinator which is not share/consumer/classic group specific.
Since we are introducing a share-coordinator as part of KIP-932 (in a new module), it would make sense to get the common coordinator functionality into a separate common coordinator module so that share-coordinator need not depend on group-coordinator.
Reviewers: Apoorv Mittal <apoorvmittal10@gmail.com>, David Jacot <djacot@confluent.io>, Andrew Schofield <aschofield@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
This change fixes the Kafka configuration validation to take into account the reconfiguration changes to configuration and allows KRaft observers to start with an unknown set of voters.
For the Kafka configuration validation the high-level change is that now the user only needs to specify either the controller.quorum.bootstrap.servers property or the controller.quorum.voters property. The other notable change in the configuration is that controller listeners can now be (and should be) specified in advertise.listeners property.
Because Kafka can now be configured without any voters and just the bootstrap servers. The KRaft client needs to allow for an unknown set of voters during the initial startup. This is done by adding the VoterSet#empty set of voters to the KRaftControlRecordStateMachine.
Lastly the RaftClientTestContext type is updated to support this new configuration for KRaft and a test is added to verify that observers can start and send Fetch requests when the voters are unknown.
Reviewers: David Arthur <mumrah@gmail.com>
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>
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>
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>
The generated code path added to test scope of sourceSets will make `Gradle` assume there are tests in module, and then it results in failed test task since the module does not define test engine.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This version of the ShadowJarPlugin uses the incorrect classifier for the published archive. This is a temporary measure to fix publishing prior to the upcoming release.
Signed-off-by: Greg Harris <greg.harris@aiven.io>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
In #12148 , we removed log4jAppender dependency, and add testImplementation dependency for slf4jlog4j lib. However, we need this runtime dependency in tools module to output logs. (ref) Adding this dependency back.
Note: The slf4jlog4j lib was added in log4j-appender dependency. Since it's removed, we need to explicitly declare it.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Edoardo Comar <ecomar@uk.ibm.com>
This patch moves the `PartitionAssignor` interface and all the related classes to a newly created `group-coordinator/api` module, following the pattern used by the storage and tools modules.
Reviewers: Ritika Reddy <rreddy@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
This PR aims to add JVM based Docker Official Image for Apache Kafka as per the following KIP - https://cwiki.apache.org/confluence/display/KAFKA/KIP-1028%3A+Docker+Official+Image+for+Apache+Kafka
This PR adds the following functionalities:
Introduces support for Apache Kafka Docker Official Images via:
GitHub Workflows:
- Workflow to prepare static source files for Docker images
- Workflow to build and test Docker official images
- Scripts to prepare source files and perform Docker image builds and tests
A new directory for Docker official images, named docker/docker_official_images. This is the new directory to house all Docker Official Image assets.
Co-authored-by: Vedarth Sharma <vesharma@confluent.io>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Vedarth Sharma <vesharma@confluent.io>
When the consumer group protocol is used in a cluster, it is, at the moment, impossible to see all records stored in the __consumer_offsets topic with kafka-dump-log --offsets-decoder. It does not know how to handle all the new records.
This patch refactors the OffsetsMessageParser used internally by kafka-dump-log to use the RecordSerde used by the new group coordinator. It ensures that the tool is always in sync with the coordinator implementation. The patch also changes the format to using the toString'ed representations of the records instead of having custom logic to dump them. It ensures that all the information is always dumped. The downside of the latest is that inner byte arrays (e.g. assignment in the classic protocol) are no longer deserialized. Personally, I feel like that it is acceptable and it is actually better to stay as close as possible to the actual records in this tool. It also avoids issues like https://issues.apache.org/jira/browse/KAFKA-15603.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>