Only initialize remote topic metrics when system-wise remote storage is enabled to avoid impacting performance for existing brokers. Also add tests.
Reviewers: Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
* KAFKA-15107: Support custom metadata for remote log segment
This commit does the changes discussed in the KIP-917. Mainly, changes the `RemoteStorageManager` interface in order to return `CustomMetadata` and then ensures these custom metadata are stored, propagated, (de-)serialized correctly along with the standard metadata throughout the whole lifecycle. It introduces the `remote.log.metadata.custom.metadata.max.size` to limit the custom metadata size acceptable by the broker and stop uploading in case a piece of metadata exceeds this limit.
On testing:
1. `RemoteLogManagerTest` checks the case when a piece of custom metadata is larger than the configured limit.
2. `RemoteLogSegmentMetadataTest` checks if `createWithUpdates` works correctly, including custom metadata.
3. `RemoteLogSegmentMetadataTransformTest`, `RemoteLogSegmentMetadataSnapshotTransformTest`, and `RemoteLogSegmentMetadataUpdateTransformTest` were added to test the corresponding class (de-)serialization, including custom metadata.
4. `FileBasedRemoteLogMetadataCacheTest` checks if custom metadata are being correctly saved and loaded to a file (indirectly, via `equals`).
5. `RemoteLogManagerConfigTest` checks if the configuration setting is handled correctly.
Reviewers: Luke Chen <showuon@gmail.com>, Satish Duggana <satishd@apache.org>, Divij Vaidya <diviv@amazon.com>
Implement some of the metrics from KIP-938: Add more metrics for
measuring KRaft performance.
Add these metrics to QuorumControllerMetrics:
kafka.controller:type=KafkaController,name=TimedOutBrokerHeartbeatCount
kafka.controller:type=KafkaController,name=EventQueueOperationsStartedCount
kafka.controller:type=KafkaController,name=EventQueueOperationsTimedOutCount
kafka.controller:type=KafkaController,name=NewActiveControllersCount
Create LoaderMetrics with these new metrics:
kafka.server:type=MetadataLoader,name=CurrentMetadataVersion
kafka.server:type=MetadataLoader,name=HandleLoadSnapshotCount
Create SnapshotEmitterMetrics with these new metrics:
kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedBytes
kafka.server:type=SnapshotEmitter,name=LatestSnapshotGeneratedAgeMs
Reviewers: Ron Dagostino <rndgstn@gmail.com>
This patch does a few things:
1) It introduces the `OffsetAndMetadata` class which hold the committed offsets in the group coordinator.
2) It adds methods to deal with OffsetCommit records to `RecordHelpers`.
3) It adds `MetadataVersion#offsetCommitValueVersion` to get the version of the OffsetCommit value record that should be used.
Reviewers: Jeff Kim <jeff.kim@confluent.io>, David Arthur <mumrah@gmail.com>, Justine Olshan <jolshan@confluent.io>
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>
This patch implements the existing JoinGroup protocol within the new group coordinator.
Some notable differences:
* Methods return a CoordinatorResult to the runtime framework, which includes records to append to the log as well as a future to complete after the append succeeds/fails.
* The coordinator runtime ensures that only a single thread will be processing a group at any given time, therefore there is no more locking on groups.
* Instead of using on purgatories, we rely on the Timer interface to schedule/cancel delayed operations.
Reviewers: David Jacot <djacot@confluent.io>
* KAFKA-14953: Adding RemoteLogManager metrics
In this PR, I have added the following metrics that are related to tiered storage mentioned in[ KIP-405](https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage).
|Metric|Description|
|-----------------------------------------|--------------------------------------------------------------|
| RemoteReadRequestsPerSec | Number of remote storage read requests per second |
| RemoteWriteRequestsPerSec | Number of remote storage write requests per second |
| RemoteBytesInPerSec | Number of bytes read from remote storage per second |
| RemoteReadErrorsPerSec | Number of remote storage read errors per second |
| RemoteBytesOutPerSec | Number of bytes copied to remote storage per second |
| RemoteWriteErrorsPerSec | Number of remote storage write errors per second |
| RemoteLogReaderTaskQueueSize | Number of remote storage read tasks pending for execution. |
| RemoteLogReaderAvgIdlePercent | Average idle percent of the remote storage reader thread pool|
| RemoteLogManagerTasksAvgIdlePercent | Average idle percent of RemoteLogManager thread pool |
Added unit tests for all the rate metrics.
Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>, Staniel Yao<yaolixinylx@gmail.com>, hudeqi<1217150961@qq.com>, Satish Duggana <satishd@apache.org>
This patch wires the new group coordinator in BrokerServer (KRaft only). With this, it is now possible to run a cluster with the new group coordinator and to use the ConsumerGroupHeartbeat API by specifying the following two properties:
- group.coordinator.new.enable = true (to enable the new group coordinator)
- unstable.api.versions.enable = true (to enable unreleased APIs)
Note that the new group coordinator does not support all the existing APIs yet.
Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
KAFKA-14522 Rewrite and Move of RemoteIndexCache to storage module.
Cleanedup index file suffix usages and other minor cleanups
Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Kamal Chandraprakash<kamal.chandraprakash@gmail.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>
This patch rewrites MockTimer in Java and moves it from core to server-common. This continues the work started in https://github.com/apache/kafka/pull/13820.
Reviewers: Divij Vaidya <diviv@amazon.com>
This patch adds (1) the logic to propagate a new MetadataImage to the running coordinators; and (2) the logic to ensure that all the consumer groups subscribed to topics with changes will refresh their subscriptions metadata on the next heartbeat. In the mean time, it ensures that freshly loaded consumer groups also refresh their subscriptions metadata on the next heartbeat.
Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
Poison the transaction manager if we detect an illegal transition in the Sender thread. A ThreadLocal in is stored in TransactionManager so that the Sender can inform TransactionManager which thread it's using.
Reviewers: Daniel Urban <durban@cloudera.com>, Justine Olshan <jolshan@confluent.io>, Jason Gustafson <jason@confluent.io>
This patch introduces the GroupCoordinatorService. This is the new (incomplete) implementation of the group coordinator based on the coordinator runtime introduced in https://github.com/apache/kafka/pull/13795.
Reviewers: Divij Vaidya <diviv@amazon.com>, Justine Olshan <jolshan@confluent.io>
The Java rewrite is kept relatively close to the Scala original
to minimize potential newly introduced bugs and to make reviewing
simpler. The following details might be of note:
- The `Logging` trait moved to InterBrokerSendThread with the
rewrite of ShutdownableThread has been similarly moved to any
subclasses that currently use it. InterBrokerSendThread's own
logging has been made to use ShutdownableThread's logger which
mimics the prefix/log identifier that the trait provided.
- The case RequestAndCompletionHandler class has been made a
separate POJO class and the internal-use UnsentRequests class
has been kept as a static nested class.
- The relatively commonly used but internal (not part of the
public API) clients classes that InterBrokerSendThread relies on
have been allowlisted in the server-common import control.
- The accompanying test class has also been moved and rewritten
with one new test added and most of the pre-existing tests made
stricter.
Reviewers: David Jacot <djacot@confluent.io>
This PR expands the scope of ApiVersionManager a bit to include returning the current
MetadataVersion and features that are in effect. This is useful in general because that information
needs to be returned in an ApiVersionsResponse. It also allows us to fix the ApiVersionManager
interface so that all subclasses implement all methods of the interface. Having subclasses that
don't implement some methods is dangerous because they could cause exceptions at runtime in
unexpected scenarios.
On the KRaft controller, we were previously performing a read operation in the QuorumController
thread to get the current metadata version and features. With this PR, we now read a volatile
variable maintained by a separate MetadataVersionContextPublisher object. This will improve
performance and simplify the code. It should not change the guarantees we are providing; in both
the old and new scenarios, we need to be robust against version skew scenarios during updates.
Add a Features class which just has a 3-tuple of metadata version, features, and feature epoch.
Remove MetadataCache.FinalizedFeaturesAndEpoch, since it just duplicates the Features class.
(There are some additional feature-related classes that can be consolidated in in a follow-on PR.)
Create a java class, EndpointReadyFutures, for managing the futures associated with individual
authorizer endpoints. This avoids code duplication between ControllerServer and BrokerServer and
makes this code unit-testable.
Reviewers: David Arthur <mumrah@gmail.com>, dengziming <dengziming1993@gmail.com>, Luke Chen <showuon@gmail.com>
This patch rewrite `MockTime` in Java and moves it to `server-common` module. This is a prerequisite to move `MockTimer` later on to `server-common` as well.
Reviewers: David Arthur <mumrah@gmail.com>
The OffsetFetcher is internally used by the KafkaConsumer to fetch offsets, validate and reset positions. For the new KafkaConsumer with a refactored threading model, similar functionality will be needed.
This is an initial refactoring for extracting logic from the OffsetFetcher, that will be reused by the new consumer implementation. No changes to the existing logic, just extracting classes, functions or pieces of logic.
All the functionality moved out of the OffsetFetcher is already covered by tests in OffsetFetcherTest and FetcherTest. There were no individual tests for the extracted functions, so no tests were migrated.
Reviewers: Jun Rao <junrao@gmail.com>
Adds CoordinatorEvent, CoordinatorEventProcessor, and MultiThreadedEventProcessor.
Reviewers: Kirk True <ktrue@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
This patch adds the GroupMetadataManager to the group-coordinator module. This manager is responsible for handling the groups management, the members management and the entire reconciliation process. At this point, only the new consumer group type/protocol is implemented.
The new manager is based on an architecture inspired from the quorum controller. A request can access/read the state but can't mutate it directly. Instead, a list of records is generated together with the response and those records are applied to the state by the runtime framework. We use timeline data structures. Note that the runtime framework is not part of this patch. It will come in a following one.
Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
Previously, if a user tried to perform an overly large batch operation on the KRaft controller
(such as creating a million topics), we would create a very large number of records in memory. Our
attempt to write these records to the Raft layer would fail, because there were too many to fit in
an atomic batch. This failure, in turn, would trigger a controller failover.
(Note: I am assuming here that no topic creation policy was in place that would prevent the
creation of a million topics. I am also assuming that the user operation must be done atomically,
which is true for all current user operations, since we have not implemented KIP-868 yet.)
With this PR, we fail immediately when the number of records we have generated exceeds the
threshold that we can apply. This failure does not generate a controller failover. We also now
fail with a PolicyViolationException rather than an UnknownServerException.
In order to implement this in a simple way, this PR adds the BoundedList class, which wraps any
list and adds a maximum length. Attempts to grow the list beyond this length cause an exception to
be thrown.
Reviewers: David Arthur <mumrah@gmail.com>, Ismael Juma <ijuma@apache.org>, Divij Vaidya <diviv@amazon.com>
Metadata image classes such as MetadataImage, ClusterImage, FeaturesImage, and so forth contain
numerous sub-images. This PR adds a structured way of traversing those sub-images. This is useful
for the metadata shell, and also for implementing toString functions.
In both cases, the previous solution was suboptimal. The metadata shell was previously implemented
in an ad-hoc way by mutating text-based tree nodes when records were replayed. This was difficult
to keep in sync with changes to the record types (for example, we forgot to do this for SCRAM). It
was also pretty low-level, being done at a level below that of the image classes. For toString, it
was difficult to keep the implementations consistent previously, and also support both redacted and
non-redacted output.
The metadata shell directory was getting crowded since we never had submodules for it. This PR
creates glob/, command/, node/, and state/ directories to keep things better organized.
Reviewers: David Arthur <mumrah@gmail.com>, Ron Dagostino <rdagostino@confluent.io>
This change includes
- Recognize the fetch requests with out of range local log offsets
- Add fetch implementation for the data lying in the range of [logStartOffset, localLogStartOffset]
- Add a new purgatory for async remote read requests which are served through a specific thread pool
We have an extended version of remote fetch that can fetch from multiple remote partitions in parallel, which we will raise as a followup PR.
A few tests for the newly introduced changes are added in this PR. There are some tests available for these scenarios in 2.8.x, refactoring with the trunk changes, will add them in followup PRs.
Other contributors:
Kamal Chandraprakash <kchandraprakash@uber.com> - Further improvements and adding a few tests
Luke Chen <showuon@gmail.com> - Added a few test cases for these changes.
PS: This functionality is pulled out from internal branches with other functionalities related to the feature in 2.8.x. The reason for not pulling all the changes as it makes the PR huge, and burdensome to review and it also needs other metrics, minor enhancements(including perf), and minor changes done for tests. So, we will try to have followup PRs to cover all those.
Reviewers: Jun Rao <junrao@gmail.com>, Alexandre Dupriez <alexandre.dupriez@gmail.com>, Divij Vaidya <diviv@amazon.com>, Jorge Esteban Quilcate Otoya <quilcate.jorge@gmail.com>
This patch introduces `GenericGroup` which rewrite the `GroupMetadata` in Java. The `GenericGroup` is basically a group using the current rebalance protocol in the new group coordinator.
Reviewers: Ritika Reddy <rreddy@confluent.io>, Christo Lolov <lolovc@amazon.com>, David Jacot <djacot@confluent.io>
This patch adds support for handling metadata snapshots while in dual-write mode. Prior to this change, if the active
controller loaded a snapshot, it would get out of sync with the ZK state.
In order to reconcile the snapshot state with ZK, several methods were added to scan through the metadata in ZK to
compute differences with the MetadataImage. Since this introduced a lot of code, I opted to split out a lot of methods
from ZkMigrationClient into their own client interfaces, such as TopicMigrationClient, ConfigMigrationClient, and
AclMigrationClient. Each of these has some iterator method that lets the caller examine the ZK state in a single pass
and without using too much memory.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Luke Chen <showuon@gmail.com>
Handle migrating SCRAM records in ZK when migrating from ZK to KRaft.
This includes handling writing back SCRAM records to ZK while in dual write mode where metadata updates are written to both the KRaft metadata log and to ZK. This allows for rollback of migration to include SCRAM metadata changes.
Reviewers: David Arthur <mumrah@gmail.com>
1. add ZkMigrationReady in apiVersionsResponse
2. check all nodes if ZkMigrationReady are ready before moving to next migration state
Reviewers: David Arthur <mumrah@gmail.com>, dengziming <dengziming1993@gmail.com>
This patch adds the concept of pre-migration mode to the KRaft controller. While in this mode,
the controller will only allow certain write operations. The purpose of this is to disallow metadata
changes when the controller is waiting for the ZK migration records to be committed.
The following ControllerWriteEvent operations are permitted in pre-migration mode
* completeActivation
* maybeFenceReplicas
* writeNoOpRecord
* processBrokerHeartbeat
* registerBroker (only for migrating ZK brokers)
* unregisterBroker
Raft events and other controller events do not follow the same code path as ControllerWriteEvent,
so they are not affected by this new behavior.
This patch also add a new metric as defined in KIP-868: kafka.controller:type=KafkaController,name=ZkMigrationState
In order to support upgrades from 3.4.0, this patch also redefines the enum value of value 1 to mean
MIGRATION rather than PRE_MIGRATION.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
This patch adds ConsumerGroupMember.
Reviewers: Christo Lolov <lolovc@amazon.com>, Jeff Kim <jeff.kim@confluent.io>, Jason Gustafson <jason@confluent.io>
This patch renames from `ControllerPurgatory` to `DeferredEventQueue` and moves it from the `metadata` module to `server-common` module.
Reviewers: Alexandre Dupriez <alexandre.dupriez@gmail.com>, Ziming Deng <dengziming1993@gmail.com>, José Armando García Sancio <jsancio@apache.org>
Rework UserScramCredentialRecord to store serverKey and StoredKey rather than saltedPassword. This
is necessary to support migration from ZK, since those are the fields we stored in ZK. Update
latest MetadataVersion to IBP_3_5_IV2 and make SCRAM support conditional on this version. Moved
ScramCredentialData.java from org.apache.kafka.image to org.apache.kafka.metadata, which seems more
appropriate.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
topic counts.
Introduces the use of persistent data structures in the KRaft metadata image to avoid copying the entire TopicsImage upon every change. Performance that was O(<number of topics in the cluster>) is now O(<number of topics changing>), which has dramatic time and GC improvements for the most common topic-related metadata events. We abstract away the chosen underlying persistent collection library via ImmutableMap<> and ImmutableSet<> interfaces and static factory methods.
Reviewers: Luke Chen <showuon@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>, Purshotam Chauhan <pchauhan@confluent.io>
This PR updates foreign-key table-table join processors to ignore out-of-order records from versioned tables, as specified in KIP-914.
Reviewers: Matthias J. Sax <matthias@confluent.io>
This PR updates primary-key table-table join processors to ignore out-of-order records from versioned tables, as specified in KIP-914.
Reviewers: Matthias J. Sax <matthias@confluent.io>
This patch adds Group, Record and Result.
Reviewers: Jason Gustafson <jason@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>
Added functionality to copy log segments, indexes to the target remote storage for each topic partition enabled with tiered storage. This involves creating scheduled tasks for all leader partition replicas to copy their log segments in sequence to tiered storage.
Reviewers: Jun Rao <junrao@gmail.com>, Luke Chen <showuon@gmail.com>
In preparation for updating DSL join processors to have updated semantics when versioned stores are used (cf KIP-914), this PR adds test coverage for out-of-order data in joins to the existing integration tests for stream-table joins and primary-key table-table joins. Follow-up PRs will build on top of this change by adding new tests for versioned stores, and the out-of-order data will produce different results in those settings.
Reviewers: Matthias J. Sax <matthias@confluent.io>
The SnapshotReader exposes the "last contained log time". This is mainly used during snapshot cleanup. The previous implementation used the append time of the snapshot record. This is not accurate as this is the time when the snapshot was created and not the log append time of the last record included in the snapshot.
The log append time of the last record included in the snapshot is store in the header control record of the snapshot. The header control record is the first record of the snapshot.
To be able to read this record, this change extends the RecordsIterator to decode and expose the control records in the Records type.
Reviewers: Colin Patrick McCabe <cmccabe@apache.org>
Until this PR, all the code added for KIP-889 for introducing versioned stores to Kafka Streams has been accessible from internal packages only. This PR exposes the stores via public Stores.java methods, and also updates the TopologyTestDriver.
Reviewers: Matthias J. Sax <matthias@confluent.io>
This patch refactors the loadCache method in AclAuthorizer to make it reusable by ZkMigrationClient.
The loaded ACLs are converted to AccessControlEntryRecord. I noticed we still have the defunct
AccessControlRecord, so I've deleted it.
Also included here are the methods to write ACL changes back to ZK while in dual-write mode.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Colin P. McCabe <cmccabe@apache.org>
* KAFKA-14365: Extract common logic from Fetcher
Extract logic from Fetcher into AbstractFetcher.
Also introduce FetchConfig as a more concise way to delineate state from
incoming configuration.
Formalized the defaults in CommonClientConfigs and ConsumerConfig to be
accessible elsewhere.
* Removed overridden methods in favor of synchronizing where needed
Reviewers: Guozhang Wang <wangguoz@gmail.com>
Separate out KRaft controller metrics into two groups: metrics directly managed by the
QuorumController, and metrics handled by an external publisher. This separation of concerns makes
the code easier to reason about, by clarifying what metrics can be changed where.
The external publisher, ControllerServerMetricsPublisher, handles all metrics which are related to
the content of metadata. For example, metrics about number of topics or number of partitions, etc.
etc. It fits into the MetadataLoader metadata publishing framework as another publisher. Since
ControllerServerMetricsPublisher operates off of a MetadataImage, we don't have to create
(essentially) another copy of the metadata in memory, as ControllerMetricsManager. This reduces
memory consumption. Another benefit of operating off of the MetadataImage is that we don't have to
have special handling for each record type, like we do now in ControllerMetricsManager.
Reviewers: David Arthur <mumrah@gmail.com>
The new group coordinator needs to access cluster metadata (e.g. topics, partitions, etc.) and it needs a mechanism to be notified when the metadata changes (e.g. to trigger a rebalance). In KRaft clusters, the easiest is to subscribe to metadata changes via the MetadataPublisher.
Reviewers: Justine Olshan <jolshan@confluent.io>
This commit adds support to store the SCRAM credentials in a cluster with KRaft quorum servers and
no ZK cluster backing the metadata. This includes creating ScramControlManager in the controller,
and adding support for SCRAM to MetadataImage and MetadataDelta.
Change UserScramCredentialRecord to contain only a single tuple (name, mechanism, salt, pw, iter)
rather than a mapping between name and a list. This will avoid creating an excessively large record
if a single user has many entries. Because record ID 11 (UserScramCredentialRecord) has not been
used before, this is a compatible change. SCRAM will be supported in 3.5-IV0 and later.
This commit does not include KIP-900 SCRAM bootstrapping support, or updating the credential cache
on the controller (as opposed to broker). We will implement these in follow-on commits.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
…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>
This patch adds the broker side `PartitionAssignor` interface as detailed in KIP-848. The interfaces differs a bit from the KIP in the following ways:
* The POJOs are not defined within the interface because the interface is to heavy like this.
* The interface is kept in the `group-coordinator` module for now. We don't want to have it out there until KIP-848 is ready to be released. We will move it to its final destination later.
Reviewers: Jeff Kim <jeff.kim@confluent.io>, Justine Olshan <jolshan@confluent.io>, Christo Lolov <lolovc@amazon.com>, Guozhang Wang <wangguoz@gmail.com>
Only send UMR to ZK brokers if the cluster metadata or topic metadata has changed.
Reviewers: Akhilesh C <akhileshchg@users.noreply.github.com>, Colin P. McCabe <cmccabe@apache.org>
Reviewers: Daniel Urban <durban@cloudera.com>, Greg Harris <greg.harris@aiven.io>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>, Mickael Maison <mickael.maison@gmail.com>
KAFKA-14551 Move/Rewrite LeaderEpochFileCache and its dependencies to the storage module.
For broader context on this change, you may want to look at KAFKA-14470: Move log layer to the storage module
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>, Alexandre Dupriez <alexandre.dupriez@gmail.com>
This patch moves the current `__consumer_offsets` records from the `core` module to the new `group-coordinator` module.
Reviewers: Christo Lolov <lolovc@amazon.com>, Mickael Maison <mickael.maison@gmail.com>
This patch migrates all the internal APIs of the current group coordinator to the new `GroupCoordinator` interface. It also makes the current implementation package private to ensure that it is not used anymore.
Reviewers: Justine Olshan <jolshan@confluent.io>
If KafkaEventQueue gets an InterruptedException while waiting for a condition variable, it
currently exits immediately. Instead, it should complete the remaining events exceptionally and
then execute the cleanup event. This will allow us to finish any necessary cleanup steps.
In order to do this, we require the cleanup event to be provided when the queue is contructed,
rather than when it's being shut down.
Also, handle cases where Event#handleException itself throws an exception.
Remove timed shutdown from the event queue code since nobody was using it, and it adds complexity.
Add server-common/src/test/resources/test/log4j.properties since this gradle module somehow avoided
having a test log4j.properties up to this point.
Reviewers: David Arthur <mumrah@gmail.com>
This patch introduces a preliminary state machine that can be used by KRaft
controller to drive online migration from Zk to KRaft.
MigrationState -- Defines the states we can have while migration from Zk to
KRaft.
KRaftMigrationDriver -- Defines the state transitions, and events to handle
actions like controller change, metadata change, broker change and have
interfaces through which it claims Zk controllership, performs zk writes and
sends RPCs to ZkBrokers.
MigrationClient -- Interface that defines the functions used to claim and
relinquish Zk controllership, read to and write from Zk.
Co-authored-by: David Arthur <mumrah@gmail.com>
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Additional notable changes to fix multiple dependency ordering issues:
* Moved `ConfigSynonym` to `server-common`
* Moved synonyms from `LogConfig` to `ServerTopicConfigSynonyms `
* Removed `LogConfigDef` `define` overrides and rely on
`ServerTopicConfigSynonyms` instead.
* Moved `LogConfig.extractLogConfigMap` to `KafkaConfig`
* Consolidated relevant defaults from `KafkaConfig`/`LogConfig` in the latter
* Consolidate relevant config name definitions in `TopicConfig`
* Move `ThrottledReplicaListValidator` to `storage`
Reviewers: Satish Duggana <satishd@apache.org>, Mickael Maison <mickael.maison@gmail.com>
Also improved `LogValidatorTest` to cover a bug that was originally
only caught by `LogAppendTimeTest`.
For broader context on this change, please check:
* KAFKA-14470: Move log layer to storage module
Reviewers: Jun Rao <junrao@gmail.com>
The controller metrics in the controllers has three problems. 1) the active controller exposes uncommitted data in the metrics. 2) the active controller doesn't update the metrics when the uncommitted data gets aborted. 3) the controller doesn't update the metrics when the entire state gets reset.
We fix these issues by only updating the metrics when processing committed metadata records and reset the metrics when the metadata state is reset.
This change adds a new type `ControllerMetricsManager` which processes committed metadata records and updates the metrics accordingly. This change also removes metrics updating responsibilities from the rest of the controller managers.
Reviewers: Ron Dagostino <rdagostino@confluent.io>
This PR introduces the new metadata loader and snapshot generator. For the time being, they are
only used by the controller, but a PR for the broker will come soon.
The new metadata loader supports adding and removing publishers dynamically. (In contrast, the old
loader only supported adding a single publisher.) It also passes along more information about each
new image that is published. This information can be found in the LogDeltaManifest and
SnapshotManifest classes.
The new snapshot generator replaces the previous logic for generating snapshots in
QuorumController.java and associated classes. The new generator is intended to be shared between
the broker and the controller, so it is decoupled from both.
There are a few small changes to the old snapshot generator in this PR. Specifically, we move the
batch processing time and batch size metrics out of BrokerMetadataListener.scala and into
BrokerServerMetrics.scala.
Finally, fix a case where we are using 'is' rather than '==' for a numeric comparison in
snapshot_test.py.
Reviewers: David Arthur <mumrah@gmail.com>
`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>
This patch adds support for reading and writing ZooKeeper metadata during a KIP-866 migration.
For reading metadata from ZK, methods from KafkaZkClient and ZkData are reused to ensure we are decoding the JSON consistently.
For writing metadata, we use a new multi-op transaction that ensures only a single controller is writing to ZK. This is similar to the existing multi-op transaction that KafkaController uses, but it also includes a check on the new "/migration" ZNode. The transaction consists of three operations:
* CheckOp on /controller_epoch
* SetDataOp on /migration with zkVersion
* CreateOp/SetDataOp/DeleteOp (the actual operation being applied)
In the case of a batch of operations (such as topic creation), only the final MultiOp has a SetDataOp on /migration while the other requests use a CheckOp (similar to /controller_epoch).
Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, dengziming <dengziming1993@gmail.com>
Split out the logic for applying dynamic configurations to a KafkaConfig object from
BrokerMetadataPublisher into a new class, DynamicConfigPublisher. This will allow the
ControllerServer to also run this code, in a follow-up change.
Create separate KafkaConfig objects in BrokerServer versus ControllerServer. This is necessary
because the controller will apply configuration changes as soon as its raft client catches up to
the high water mark, whereas the broker will wait for the active controller to acknowledge it has
caught up in a heartbeat response. So when running in combined mode, we want two separate
KafkaConfig objects that are changed at different times.
Minor changes: improve the error message when catching up broker metadata fails. Fix incorrect
indentation in checkstyle/import-control.xml. Invoke AppInfoParser.unregisterAppInfo from
SharedServer.stop so that it happens only when both the controller and broker have shut down.
Reviewers: David Arthur <mumrah@gmail.com>
This patch adds `joinGroup` to the new `GroupCoordinator` interface and updates `KafkaApis` to use it.
For the context, I will do the same for all the other interactions with the current group coordinator. In order to limit the changes, I have chosen to introduce the `GroupCoordinatorAdapter` that translates the new interface to the old one. It is basically a wrapper. This allows keeping the current group coordinator untouched for now and focus on the `KafkaApis` changes. Eventually, we can remove `GroupCoordinatorAdapter`.
Reviewers: Justine Olshan <jolshan@confluent.io>, Jeff Kim <jeff.kim@confluent.io>, Luke Chen <showuon@gmail.com>, Jason Gustafson <jason@confluent.io>
PR implementing KIP-770 (#11424) was reverted as it brought in a regression wrt pausing/resuming the consumer. That KIP also introduced a change to deprecate config CACHE_MAX_BYTES_BUFFERING_CONFIG and replace it with STATESTORE_CACHE_MAX_BYTES_CONFIG.
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
This PR adds a new ImageWriter interface which replaces the generic Consumer interface which
accepted lists of records. It is better to do batching in the ImageWriter than to try to deal with
that complexity in the MetadataImage#write functions, especially since batching is not semantically
meaningful in KRaft snapshots. The new ImageWriter interface also supports freeze and close, which
more closely matches the semantics of the underlying Raft classes.
The PR also adds an ImageWriterOptions class which we can use to pass parameters to control how the
new image is written. Right now, the parameters that we are interested in are the target metadata
version (which may be more or less than the original image's version) and a handler function which
is invoked whenever metadata is lost due to the target version.
Convert over the MetadataImage#write function (and associated functions) to use the new ImageWriter
and ImageWriterOptions. In particular, we now have a way to handle metadata losses by invoking
ImageWriterOptions#handleLoss. This allows us to handle writing an image at a lower version, for
the first time. This support is still not enabled externally by this PR, though. That will come in
a future PR.
Get rid of the use of SOME_RECORD_TYPE.highestSupportedVersion() in several places. In general, we
do not want to "silently" change the version of a record that we output, just because a new version
was added. We should be explicit about what record version numbers we are outputting.
Implement ProducerIdsDelta#toString, to make debug logs look better.
Move MockRandom to the server-common package so that other internal broker packages can use it.
Reviewers: José Armando García Sancio <jsancio@apache.org>
Also moves the Streams LogCaptureAppender class into the clients module so that it can be used by both Streams and Connect.
Reviewers: Nigel Liang <nigel@nigelliang.com>, Kalpesh Patel <kpatel@confluent.io>, John Roesler <vvcephei@apache.org>, Tom Bentley <tbentley@redhat.com>
In https://github.com/apache/kafka/pull/12695, we discovered a gap in our testing of `StandardAuthorizer`. We addressed the specific case that was failing, but I think we need to establish a better methodology for testing which incorporates randomized inputs. This patch is a start in that direction. We implement a few basic property tests using jqwik which focus on prefix searching. It catches the case from https://github.com/apache/kafka/pull/12695 prior to the fix. In the future, we can extend this to cover additional operation types, principal matching, etc.
Reviewers: David Arthur <mumrah@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>
This adds a new configuration `sasl.server.max.receive.size` that sets the maximum receive size for requests before and during authentication.
Reviewers: Tom Bentley <tbentley@redhat.com>, Mickael Maison <mickael.maison@gmail.com>
Co-authored-by: Manikumar Reddy <manikumar.reddy@gmail.com>
Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
When deserializing KRPC (which is used for RPCs sent to Kafka, Kafka Metadata records, and some
other things), check that we have at least N bytes remaining before allocating an array of size N.
Remove DataInputStreamReadable since it was hard to make this class aware of how many bytes were
remaining. Instead, when reading an individual record in the Raft layer, simply create a
ByteBufferAccessor with a ByteBuffer containing just the bytes we're interested in.
Add SimpleArraysMessageTest and ByteBufferAccessorTest. Also add some additional tests in
RequestResponseTest.
Reviewers: Tom Bentley <tbentley@redhat.com>, Mickael Maison <mickael.maison@gmail.com>, Colin McCabe <colin@cmccabe.xyz>
Co-authored-by: Colin McCabe <colin@cmccabe.xyz>
Co-authored-by: Manikumar Reddy <manikumar.reddy@gmail.com>
Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
The main changes here are ensuring that we always have a metadata.version record in the log, making
˘sure that the bootstrap file can be used for records other than the metadata.version record (for
example, we will want to put SCRAM initialization records there), and fixing some bugs.
If no feature level record is in the log and the IBP is less than 3.3IV0, then we assume the minimum KRaft
version for all records in the log.
Fix some issues related to initializing new clusters. If there are no records in the log at all,
then insert the bootstrap records in a single batch. If there are records, but no metadata version,
process the existing records as though they were metadata.version 3.3IV0 and then append a metadata
version record setting version 3.3IV0. Previously, we were not clearly distinguishing between the
case where the metadata log was empty, and the case where we just needed to add a metadata.version
record.
Refactor BootstrapMetadata into an immutable class which contains a 3-tuple of metadata version,
record list, and source. The source field is used to log where the bootstrap metadata was obtained
from. This could be a bootstrap file, the static configuration, or just the software defaults.
Move the logic for reading and writing bootstrap files into BootstrapDirectory.java.
Add LogReplayTracker, which tracks whether the log is empty.
Fix a bug in FeatureControlManager where it was possible to use a "downgrade" operation to
transition to a newer version. Do not store whether we have seen a metadata version or not in
FeatureControlManager, since that is now handled by LogReplayTracker.
Introduce BatchFileReader, which is a simple way of reading a file containing batches of snapshots
that does not require spawning a thread. Rename SnapshotFileWriter to BatchFileWriter to be
consistent, and to reflect the fact that bootstrap files aren't snapshots.
QuorumController#processBrokerHeartbeat: add an explanatory comment.
Reviewers: David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>
AccessControlEntryRecord and RemoveAccessControlEntryRecord are added in KIP-801, FeatureLevelRecord was added in KIP-778, and BrokerRegistrationChangeRecord was added in KIP-841, and NoOpRecord was added in KIP-835, I added these 5 record types in MetadataShell.
Reviewers: Luke Chen <showuon@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>
Before trying to commit a batch of records to the __cluster_metadata log, the active controller
should try to apply them to its current in-memory state. If this application process fails, the
active controller process should exit, allowing another node to take leadership. This will prevent
most bad metadata records from ending up in the log and help to surface errors during testing.
Similarly, if the active controller attempts to renounce leadership, and the renunciation process
itself fails, the process should exit. This will help avoid bugs where the active controller
continues in an undefined state.
In contrast, standby controllers that experience metadata application errors should continue on, in
order to avoid a scenario where a bad record brings down the whole controller cluster. The
intended effect of these changes is to make it harder to commit a bad record to the metadata log,
but to continue to ride out the bad record as well as possible if such a record does get committed.
This PR introduces the FaultHandler interface to implement these concepts. In junit tests, we use a
FaultHandler implementation which does not exit the process. This allows us to avoid terminating
the gradle test runner, which would be very disruptive. It also allows us to ensure that the test
surfaces these exceptions, which we previously were not doing (the mock fault handler stores the
exception).
In addition to the above, this PR fixes a bug where RaftClient#resign was not being called from the
renounce() function. This bug could have resulted in the raft layer not being informed of an active
controller resigning.
Reviewers: David Arthur <mumrah@gmail.com>
KIP-770 introduced a performance regression and needs some re-design.
Needed to resolve some conflict while reverting.
This reverts commits 1317f3f77a and 0924fd3f9f.
Reviewers: Sagar Rao <sagarmeansocean@gmail.com>, Guozhang Wang <guozhang@confluent.io>
This PR adds the ability to pause and resume KafkaStreams instances as well as named/modular topologies (KIP-834).
Co-authored-by: Bruno Cadonna <cadonna@apache.org>
Reviewers: Bonnie Varghese <bvarghese@confluent.io>, Walker Carlson <wcarlson@confluent.io>, Guozhang Wang <guozhang@apache.org>, Bruno Cadonna <cadonna@apache.org>
New gradle task `connect:runtime:genConnectOpenAPIDocs` that generates `connect_rest.yaml` under `docs/generated`.
This task is executed when `siteDocsTar` runs.
Implementation of KIP-846: Source/sink node metrics for Consumed/Produced throughput in Streams
Adds the following INFO topic-level metrics for the total bytes/records consumed and produced:
bytes-consumed-total
records-consumed-total
bytes-produced-total
records-produced-total
Reviewers: Kvicii <Karonazaba@gmail.com>, Guozhang Wang <guozhang@apache.org>, Bruno Cadonna <cadonna@apache.org>
Implements the behavior described in KIP-618: using a transactional producer for writes to the config topic that should only be performed by the leader of the cluster.
Reviewers: Luke Chen <showuon@gmail.com>, Tom Bentley <tbentley@redhat.com>
This PR refactors the leader API access in the follower fetch path.
Added a LeaderEndPoint interface which serves all access to the leader.
Added a LocalLeaderEndPoint and a RemoteLeaderEndPoint which implements the LeaderEndPoint interface to handle fetches from leader in local & remote storage respectively.
Reviewers: David Jacot <djacot@confluent.io>, Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
This patch builds on #12072 and adds controller support for metadata.version. The kafka-storage tool now allows a
user to specify a specific metadata.version to bootstrap into the cluster, otherwise the latest version is used.
Upon the first leader election of the KRaft quroum, this initial metadata.version is written into the metadata log. When
writing snapshots, a FeatureLevelRecord for metadata.version will be written out ahead of other records so we can
decode things at the correct version level.
This also includes additional validation in the controller when setting feature levels. It will now check that a given
metadata.version is supportable by the quroum, not just the brokers.
Reviewers: José Armando García Sancio <jsancio@gmail.com>, Colin P. McCabe <cmccabe@apache.org>, dengziming <dengziming1993@gmail.com>, Alyssa Huang <ahuang@confluent.io>
These tests belongs to ApiVersionsResponseTest, and accidentally copied them to MetadataVersionTest when working on #12072.
Reviewer: Luke Chen <showuon@gmail.com>
Refactoring ApiVersion to MetadataVersion to support both old IBP versioning and new KRaft versioning (feature flags)
for KIP-778.
IBP versions are now encoded as enum constants and explicitly prefixed w/ IBP_ instead of KAFKA_, and having a
LegacyApiVersion vs DefaultApiVersion was not necessary and replaced with appropriate parsing rules for extracting
the correct shortVersions/versions.
Co-authored-by: David Arthur <mumrah@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, David Arthur <mumrah@gmail.com>, dengziming <dengziming1993@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
Updates the KStream process API to cover the use cases
of both process and transform, and deprecate the KStream transform API.
Implements KIP-820
Reviewer: John Roesler <vvcephei@apache.org>
Ensure that we can set log.flush.interval.ms at the broker or cluster level via
IncrementalAlterConfigs. This was broken by KAFKA-13749, which added log.flush.interval.ms as the
second synonym rather than the first. Add a regression test to DynamicConfigChangeTest.
Create ControllerRequestContext and pass it to every controller API. This gives us a uniform way to
pass through information like the deadline (if there is one) and the Kafka principal which is
making the request (in the future we will want to log this information).
In ControllerApis, enforce a timeout for broker heartbeat requests which is equal to the heartbeat
request interval, to avoid heartbeats piling up on the controller queue. This should have been done
previously, but we overlooked it.
Add a builder for ClusterControlManager and ReplicationControlManager to avoid the need to deal
with a lot of churn (especially in test code) whenever a new constructor parameter gets added for
one of these.
In ControllerConfigurationValidator, create a separate function for when we just want to validate
that a ConfigResource is a valid target for DescribeConfigs. Previously we had been re-using the
validation code for IncrementalAlterConfigs, but this was messy.
Split out the replica placement code into a separate package and reorganize it a bit.
Reviewers: David Arthur <mumrah@gmail.com
Prior to this commit FK response sink routed FK results to
SubscriptionResolverJoinProcessorSupplier using the primary key.
There are cases, where this behavior is incorrect. For example,
if KTable key serde differs from the data source serde which might
happen without a key changing operation.
Instead of determining the resolver partition by serializing the PK
this patch includes target partition in SubscriptionWrapper payloads.
Default FK response-sink partitioner extracts the correct partition
from the value and routes the message accordingly.
Reviewers: Matthias J. Sax <matthias@confluent.io>
This PR includes the changes to feature flags that were outlined in KIP-778. Specifically, it
changes UpdateFeatures and FeatureLevelRecord to remove the maximum version level. It also adds
dry-run to the RPC so the controller can actually attempt the upgrade (rather than the client). It
introduces an upgrade type enum, which supersedes the allowDowngrade boolean. Because
FeatureLevelRecord was unused previously, we do not need to introduce a new version.
The kafka-features.sh tool was overhauled in KIP-778 and now includes the describe, upgrade,
downgrade, and disable sub-commands. Refer to
[KIP-778](https://cwiki.apache.org/confluence/display/KAFKA/KIP-778%3A+KRaft+Upgrades) for more
details on the new command structure.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, dengziming <dengziming1993@gmail.com>
Previously, when in KRaft mode, CreateTopics did not return the active configurations for the
topic(s) it had just created. This PR addresses that gap. We will now return these topic
configuration(s) when the user has DESCRIBE_CONFIGS permission. (In the case where the user does
not have this permission, we will omit the configurations and set TopicErrorCode. We will also omit
the number of partitions and replication factor data as well.)
For historical reasons, we use different names to refer to each topic configuration when it is set
in the broker context, as opposed to the topic context. For example, the topic configuration
"segment.ms" corresponds to the broker configuration "log.roll.ms". Additionally, some broker
configurations have synonyms. For example, the broker configuration "log.roll.hours" can be used to
set the log roll time instead of "log.roll.ms". In order to track all of this, this PR adds a
table in LogConfig.scala which maps each topic configuration to an ordered list of ConfigSynonym
classes. (This table is then passed to KafkaConfigSchema as a constructor argument.)
Some synonyms require transformations. For example, in order to convert from "log.roll.hours" to
"segment.ms", we must convert hours to milliseconds. (Note that our assumption right now is that
topic configurations do not have synonyms, only broker configurations. If this changes, we will
need to add some logic to handle it.)
This PR makes the 8-argument constructor for ConfigEntry public. We need this in order to make full
use of ConfigEntry outside of the admin namespace. This change is probably inevitable in general
since otherwise we cannot easily test the output from various admin APIs in junit tests outside the
admin package.
Testing:
This PR adds PlaintextAdminIntegrationTest#testCreateTopicsReturnsConfigs. This test validates
some of the configurations that it gets back from the call to CreateTopics, rather than just checking
if it got back a non-empty map like some of the existing tests. In order to test the
configuration override logic, testCreateDeleteTopics now sets up some custom static and dynamic
configurations.
In QuorumTestHarness, we now allow tests to configure what the ID of the controller should be. This
allows us to set dynamic configurations for the controller in testCreateDeleteTopics. We will have
a more complete fix for setting dynamic configuations on the controller later.
This PR changes ConfigurationControlManager so that it is created via a Builder. This will make it
easier to add more parameters to its constructor without having to update every piece of test code
that uses it. It will also make the test code easier to read.
Reviewers: David Arthur <mumrah@gmail.com>
Since the topology-level cache size config only controls whether we disable the caching layer entirely for that topology, setting it to anything other than 0 has no effect. The actual cache memory is still just split evenly between the threads, and shared by all topologies.
It's possible we'll want to change this in the future, but for now we should make sure to log a warning so that users who do try to set this override to some nonzero value are made aware that it doesn't work like this.
Also includes some minor refactoring plus a fix for an off-by-one error in #11796
Reviewers: Luke Chen <showuon@gmail.com>, Walker Carlson <wcarlson@confluent.io>, Sagar Rao <sagarmeansocean@gmail.com>
With major server components like the new quorum controller being moved outside of the `core` module, it is useful to have shared dependencies moved into `server-common`. An example of this is Yammer metrics which server components still rely heavily upon. All server components should have access to the default registry used by the broker so that new metrics can be registered and metric naming conventions should be standardized. This is particularly important in KRaft where we are attempting to recreate identically named metrics in the controller context. This patch takes a step in this direction. It moves `KafkaYammerMetrics` into `server-common` and it implements
standard metric naming utilities there.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
This patch fixes a bug in the `AlterConfigPolicy.RequestMetadata.equals` method where we were not comparing the class correctly.
Co-authored-by: David Jacot <djacot@confluent.io>
Reviewers: David Jacot <djacot@confluent.io>
Create KafkaConfigSchema to encapsulate the concept of determining the types of configuration keys.
This is useful in the controller because we can't import KafkaConfig, which is part of core. Also
introduce the TimelineObject class, which is a more generic version of TimelineInteger /
TimelineLong.
Reviewers: David Arthur <mumrah@gmail.com>
Currently, when using KRaft mode, users still have to have an Apache ZooKeeper instance if they want to use AclAuthorizer. We should have a built-in Authorizer for KRaft mode that does not depend on ZooKeeper. This PR introduces such an authorizer, called StandardAuthorizer. See KIP-801 for a full description of the new Authorizer design.
Authorizer.java: add aclCount API as described in KIP-801. StandardAuthorizer is currently the only authorizer that implements it, but eventually we may implement it for AclAuthorizer and others as well.
ControllerApis.scala: fix a bug where createPartitions was authorized using CREATE on the topic resource rather than ALTER on the topic resource as it should have been.
QuorumTestHarness: rename the controller endpoint to CONTROLLER for consistency (the brokers already called it that). This is relevant in AuthorizerIntegrationTest where we are examining endpoint names. Also add the controllerServers call.
TestUtils.scala: adapt the ACL functions to be usable from KRaft, by ensuring that they use the Authorizer from the current active controller.
BrokerMetadataPublisher.scala: add broker-side ACL application logic.
Controller.java: add ACL APIs. Also add a findAllTopicIds API in order to make junit tests that use KafkaServerTestHarness#getTopicNames and KafkaServerTestHarness#getTopicIds work smoothly.
AuthorizerIntegrationTest.scala: convert over testAuthorizationWithTopicExisting (more to come soon)
QuorumController.java: add logic for replaying ACL-based records. This means storing them in the new AclControlManager object, and integrating them into controller snapshots. It also means applying the changes in the Authorizer, if one is configured. In renounce, when reverting to a snapshot, also set newBytesSinceLastSnapshot to 0.
Reviewers: YeonCheol Jang <YeonCheolGit@users.noreply.github.com>, Jason Gustafson <jason@confluent.io>
During some recent reviews, @mjsax pointed out that StateStore layers
are constructed differently the stores are added via the PAPI vs. the DSL.
This PR adds KeyValueStore PAPI construction to the
IQv2StoreIntegrationTest so that we can ensure IQv2 works on every
possible state store.
Reviewers: Patrick Stuedi <pstuedi@apache.org>, Guozhang Wang <guozhang@apache.org>
Require that topics exist before topic configurations can be created for them.
Merge the code from ConfigurationControlManager#checkConfigResource into
ControllerConfigurationValidator to avoid duplication.
Add KRaft support to DynamicConfigChangeTest.
Split out tests in DynamicConfigChangeTest that don't require a cluster into
DynamicConfigChangeUnitTest to save test time.
Reviewers: David Arthur <mumrah@gmail.com>
Implements the major part of the IQv2 framework as proposed in KIP-796.
Reviewers: Patrick Stuedi <pstuedi@apache.org>, Vicky Papavasileiou <vpapavasileiou@confluent.io>, Bruno Cadonnna <cadonna@apache.org>
With the changes for topic IDs, we have a different flow. When a broker receives a request, it uses a map to convert the topic ID to topic names. If the topic ID is not found in the map, we return a top level error and close the session. This decision was motivated by the difficulty to store “unresolved” partitions in the session. In earlier iterations we stored an “unresolved” partition object in the cache, but it was somewhat hard to reason about and required extra logic to try to resolve the topic ID on each incremental request and add to the session. It also required extra logic to forget the topic (either by topic ID if the topic name was never known or by topic name if it was finally resolved when we wanted to remove from the session.)
One helpful simplifying factor is that we only allow one type of request (uses topic ID or does not use topic ID) in the session. That means we can rely on a session continuing to have the same information. We don’t have to worry about converting topics only known by name to topic ID for a response and we won’t need to convert topics only known by ID to name for a response.
This PR introduces a change to store the "unresolved partitions" in the cached partition object. If a version 13+ request is sent with a topic ID that is unknown, a cached partition will be created with that fetch request data and a null topic name. On subsequent incremental requests, unresolved partitions may be resolved with the new IDs found in the metadata cache. When handling the request, getting all partitions will return a TopicIdPartition object that will be used to handle the request and build the response. Since we can rely on only one type of request (with IDs or without), the cached partitions map will have different keys depending on what fetch request version is being used.
This PR involves changes both in FetchSessionHandler and FetchSession. Some major changes are outlined below.
1. FetchSessionHandler: Forgetting a topic and adding a new topic with the same name - We may have a case where there is a topic foo with ID 1 in the session. Upon a subsequent metadata update, we may have topic foo with ID 2. This means that topic foo has been deleted and recreated. When sending fetch requests version 13+ we will send a request to add foo ID 2 to the session and remove foo ID 1. Otherwise, we will fall back to the same behavior for versions 12 and below
2. FetchSession: Resolving in Incremental Sessions - Incremental sessions contain two distinct sets of partitions. Partitions that are sent in the latest request that are new/updates/forgotten partitions and the partitions already in the session. If we want to resolve unknown topic IDs we will need to handle both cases.
* Partitions in the request - These partitions are either new or updating/forgetting previous partitions in the session. The new partitions are trivial. We either have a resolved partition or create a partition that is unresolved. For the other cases, we need to be a bit more careful.
* For updated partitions we have a few cases – keep in mind, we may not programmatically know if a partition is an update:
1. partition in session is resolved, update is resolved: trivial
2. partition in session is unresolved, update is unresolved: in code, this is equivalent to the case above, so trivial as well
3. partition in session is unresolved, update is resolved: this means the partition in the session does not have a name, but the metadata cache now contains the name – to fix this we can check if there exists a cached partition with the given ID and update it both with the partition update and with the topic name.
4. partition in session is resolved, update is unresolved: this means the partition in the session has a name, but the update was unable to be resolved (ie, the topic is deleted) – this is the odd case. We will look up the partition using the ID. We will find the old version with a name but will not replace the name. This will lead to an UNKNOWN_TOPIC_OR_PARTITION or INCONSISTENT_TOPIC_ID error which will be handled with a metadata update. Likely a future request will forget the partition, and we will be able to do so by ID.
5. Two partitions in the session have IDs, but they are different: only one topic ID should exist in the metadata at a time, so likely only one topic ID is in the fetch set. The other one should be in the toForget. We will be able to remove this partition from the session. If for some reason, we don't try to forget this partition — one of the partitions in the session will cause an inconsistent topic ID error and the metadata for this partition will be refreshed — this should result in the old ID being removed from the session. This should not happen if the FetchSessionHandler is correctly in sync.
* For the forgotten partitions we have the same cases:
1. partition in session is resolved, forgotten is resolved: trivial
2. partition in session is unresolved, forgotten is unresolved: in code, this is equivalent to the case above, so trivial as well
3. partition in session is unresolved, forgotten is resolved: this means the partition in the session does not have a name, but the metadata cache now contains the name – to fix this we can check if there exists a cached partition with the given ID and try to forget it before we check the resolved name case.
4. partition in session is resolved, update is unresolved: this means the partition in the session has a name, but the update was unable to be resolved (ie, the topic is deleted) We will look up the partition using the ID. We will find the old version with a name and be able to delete it.
5. both partitions in the session have IDs, but they are different: This should be the same case as described above. If we somehow do not have the ID in the session, no partition will be removed. This should not happen unless the Fetch Session Handler is out of sync.
* Partitions in the session - there may be some partitions in the session already that are unresolved. We can resolve them in forEachPartition using a method that checks if the partition is unresolved and tries to resolve it using a topicName map from the request. The partition will be resolved before the function using the cached partition is applied.
Reviewers: David Jacot <djacot@confluent.io>
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>
When loading a snapshot the broker BrokerMetadataListener was using the batch's append time, offset
and epoch. These are not the same as the append time, offset and epoch from the log. This PR fixes
it to instead use the lastContainedLogTimeStamp, lastContainedLogOffset and lastContainedLogEpoch
from the SnapshotReader.
This PR refactors the MetadataImage and MetadataDelta to include an offset and epoch. It also swaps
the order of the arguments for ReplicaManager.applyDelta, in order to be more consistent with
MetadataPublisher.publish.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
Added snapshots for consumed remote log metadata for each partition to avoid consuming again in case of broker restarts. These snapshots are stored in the respective topic partition log directories.
Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Cong Ding <cong@ccding.com>, Jun Rao <junrao@gmail.com>
Avoid O(N) behavior in KRaftMetadataCache#topicNamesToIds and
KRaftMetadataCache#topicIdsToNames by returning a map subclass that
exposes the TopicsImage data structures without copying them.
Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>
Add support for CreateTopicsPolicy and AlterConfigsPolicy when running in KRaft mode.
Reviewers: David Arthur <mumrah@gmail.com>, Niket Goel <ngoel@confluent.io>
The ReplicaManager, LogManager, and KafkaApis class all have many
constructor parameters. It is often difficult to add or remove a
parameter, since there are so many locations that need to be updated. In
order to address this problem, we should use named parameters when
constructing these objects from Scala code. This will make it easy to
add new optional parameters without modifying many test cases. It will
also make it easier to read git diffs and PRs, since the parameters will
have names next to them. Since Java does not support named paramters,
this PR adds several Builder classes which can be used to achieve the
same effect.
ReplicaManager also had a secondary constructor, which this PR removes.
The function of the secondary constructor was just to provide some
default parameters for the main constructor. However, it is simpler just
to actually use default parameters.
Reviewers: David Arthur <mumrah@gmail.com>
This PR aims to remove tombstones that persist indefinitely due to low throughput. Previously, deleteHorizon was calculated from the segment's last modified time.
In this PR, the deleteHorizon will now be tracked in the baseTimestamp of RecordBatches. After the first cleaning pass that finds a record batch with tombstones, the record batch is recopied with deleteHorizon flag and a new baseTimestamp that is the deleteHorizonMs. The records in the batch are rebuilt with relative timestamps based on the deleteHorizonMs that is recorded. Later cleaning passes will be able to remove tombstones more accurately on their deleteHorizon due to the individual time tracking on record batches.
KIP 534: https://cwiki.apache.org/confluence/display/KAFKA/KIP-534%3A+Retain+tombstones+and+transaction+markers+for+approximately+delete.retention.ms+milliseconds
Co-authored-by: Ted Yu <yuzhihong@gmail.com>
Co-authored-by: Richard Yu <yohan.richard.yu@gmail.com>
* Add the following producer metrics:
flush-time-total: cumulative sum of time elapsed during in flush.
txn-init-time-total: cumulative sum of time elapsed during in initTransactions.
txn-begin-time-total: cumulative sum of time elapsed during in beginTransaction.
txn-send-offsets-time-total: cumulative sum of time elapsed during in sendOffsetsToTransaction.
txn-commit-time-total: cumulative sum of time elapsed during in commitTransaction.
txn-abort-time-total: cumulative sum of time elapsed during in abortTransaction.
* Add the following consumer metrics:
commited-time-total: cumulative sum of time elapsed during in committed.
commit-sync-time-total: cumulative sum of time elapsed during in commitSync.
* Add a total-blocked-time metric to streams that is the sum of:
consumer’s io-waittime-total
consumer’s iotime-total
consumer’s committed-time-total
consumer’s commit-sync-time-total
restore consumer’s io-waittime-total
restore consumer’s iotime-total
admin client’s io-waittime-total
admin client’s iotime-total
producer’s bufferpool-wait-time-total
producer's flush-time-total
producer's txn-init-time-total
producer's txn-begin-time-total
producer's txn-send-offsets-time-total
producer's txn-commit-time-total
producer's txn-abort-time-total
Reviewers: Bruno Cadonna <cadonna@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Updates the scala fmt to the latest stable version.
Applies all the style fixes (all source code changes are done by scala
fmt).
Removes setting about dangling parentheses as `true` is already the
default.
Reviewer: John Roesler <john@confluent.io>
Pt. 1: #10609
Pt. 2: #10683
Pt. 3: #10788
In Pt. 3 we implement the addNamedTopology API. This can be used to update the processing topology of a running Kafka Streams application without resetting the app, or even pausing/restarting the process. It's up to the user to ensure that this API is called on every instance of an application to ensure all clients are able to run the newly added NamedTopology.
Reviewers: Guozhang Wang <guozhang@confluent.io>, Walker Carlson <wcarlson@confluent.io>
Pt. 1: #10609
Pt. 2: #10683
Pt. 3: #10788
The TopologyMetadata is next up after Pt. 1 #10609. This PR sets up the basic architecture for running an app with multiple NamedTopologies, though the APIs to add/remove them dynamically are not implemented until Pt. 3
Reviewers: Guozhang Wang <guozhang@confluent.io>, Walker Carlson <wcarlson@confluent.io>
This patch adds support for unregistering listeners to `RaftClient`.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Jason Gustafson <jason@confluent.io>
KAFKA-9555 Added default RLMM implementation based on internal topic storage.
This is the initial version of the default RLMM implementation.
This includes changes containing default RLMM configs, RLMM implementation, producer/consumer managers.
Introduced TopicBasedRemoteLogMetadataManagerHarness which takes care of bringing up a Kafka cluster and create remote log metadata topic and initializes TopicBasedRemoteLogMetadataManager.
Refactored existing RemoteLogMetadataCacheTest to RemoteLogSegmentLifecycleTest to have parameterized tests to run both RemoteLogMetadataCache and also TopicBasedRemoteLogMetadataManager.
Refactored existing InmemoryRemoteLogMetadataManagerTest, RemoteLogMetadataManagerTest to have parameterized tests to run both InmemoryRemoteLogMetadataManager and also TopicBasedRemoteLogMetadataManager.
This is part of tiered storage KIP-405 efforts.
Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Cong Ding <cong@ccding.com>, Jun Rao <junrao@gmail.com>
Support the KIP-455 reassignment API when in KRaft mode. Reassignments
which merely rearrange partitions complete immediately. Those that only
remove a partition complete immediately if the ISR would be non-empty
after the specified removals. Reassignments that add one or more
partitions follow the KIP-455 pattern of adding all the adding replicas
to the replica set, and then waiting for the ISR to include all the new
partitions before completing. Changes to the partition sets are
accomplished via PartitionChangeRecord.
Reviewers: Jun Rao <junrao@gmail.com>
1) Bring the generation field back to the CooperativeStickyAssignor so we don't need to rely so heavily on the ConsumerCoordinator properly updating its SubscriptionState after eg falling out of the group. The plain StickyAssignor always used the generation since it had to, so we just make sure the CooperativeStickyAssignor has this tool as well
2) In case of unforeseen problems or further bugs that slip past the generation field safety net, the assignor will now explicitly look out for partitions that are being claimed by multiple consumers as owned in the same generation. Such a case should never occur, but if it does, we have to invalidate this partition from the ownedPartitions of both consumers, since we can't tell who, if anyone, has the valid claim to this partition.
3) Fix a subtle bug that I discovered while writing tests for the above two fixes: in the constrained algorithm, we compute the exact number of partitions each consumer should end up with, and keep track of the "unfilled" members who must -- or might -- require more partitions to hit their quota. The problem was that members at the minQuota were being considered as "unfilled" even after we had already hit the maximum number of consumers allowed to go up to the maxQuota, meaning those minQuota members could/should not accept any more partitions beyond that. I believe this was introduced in #10509, so it shouldn't be in any released versions and does not need to be backported.
Reviewers: Guozhang Wang <guozhang@apache.org>, Luke Chen <showuon@gmail.com>
Updated FetchRequest and FetchResponse to use topic IDs rather than topic names.
Some of the complicated code is found in FetchSession and FetchSessionHandler.
We need to be able to store topic IDs and maintain a cache on the broker for IDs that may not have been resolved. On incremental fetch requests, we will try to resolve them or remove them if in toForget.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
This implements the request and response portion of KIP-709. It updates the OffsetFetch request and response to support fetching offsets for multiple consumer groups at a time. If the broker does not support the new OffsetFetch version, clients can revert to the previous behaviour and use a request for each coordinator.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Konstantine Karantasis <konstantine@confluent.io>
Create the image/ module for storing, reading, and writing broker metadata images.
Metadata images are immutable. New images are produced from existing images
using delta classes. Delta classes are mutable, and represent changes to a base
image.
MetadataImage objects can be converted to lists of KRaft metadata records. This
is essentially writing a KRaft snapshot. The resulting snapshot can be read
back into a MetadataDelta object. In practice, we will typically read the
snapshot, and then read a few more records to get fully up to date. After that,
the MetadataDelta can be converted to a MetadataImage as usual.
Sometimes, we have to load a snapshot even though we already have an existing
non-empty MetadataImage. We would do this if the broker fell too far behind and
needed to receive a snapshot to catch up. This is handled just like the normal
snapshot loading process. Anything that is not in the snapshot will be marked
as deleted in the MetadataDelta once finishSnapshot() is called.
In addition to being used for reading and writing snapshots, MetadataImage also
serves as a cache for broker information in memory. A follow-up PR will replace
MetadataCache, CachedConfigRepository, and the client quotas cache with the
corresponding Image classes. TopicsDelta also replaces the "deferred
partition" state that the RaftReplicaManager currently implements. (That change
is also in a follow-up PR.)
Reviewers: Jason Gustafson <jason@confluent.io>, David Arthur <mumrah@gmail.com>
Implements KIP-745 https://cwiki.apache.org/confluence/display/KAFKA/KIP-745%3A+Connect+API+to+restart+connector+and+tasks to change connector REST API to restart a connector and its tasks as a whole.
Testing strategy
- [x] Unit tests added for all possible combinations of onlyFailed and includeTasks
- [x] Integration tests added for all possible combinations of onlyFailed and includeTasks
- [x] System tests for happy path
Reviewers: Randall Hauch <rhauch@gmail.com>, Diego Erdody <erdody@gmail.com>, Konstantine Karantasis <k.karantasis@gmail.com>
Add header and footer records for raft snapshots. This helps identify when the snapshot
starts and ends. The header also contains a time. The time field is currently set to 0.
KAFKA-12997 will add in the necessary wiring to use the correct timestamp.
Reviewers: Jose Sancio <jsancio@gmail.com>, Colin P. McCabe <cmccabe@apache.org>
We had been using `RecordAccumulator.beginFlush` in order to force the `RecordAccumulator` to flush pending batches when a transaction was being completed. Internally, `RecordAccumulator` has a simple counter for the number of flushes in progress. The count gets incremented in `beginFlush` and it is expected to be decremented by `awaitFlushCompletion`. The second call to decrement the counter never happened in the transactional path, so the counter could get stuck at a positive value, which means that the linger time would effectively be ignored.
This patch fixes the problem by removing the use of `beginFlush` in `Sender`. Instead, we now add an additional condition in `RecordAccumulator` to explicitly check when a transaction is being completed.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
Added tiered storage related configs including remote log manager configs.
Added local log retention configs to LogConfig.
Added tests for the added configs.
Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
They have been deprecated since 0.10.0. Full list of removes configs:
* port
* host.name
* advertised.port
* advertised.host.name
Also adjust tests to take the removals into account. Some tests were
no longer relevant and have been removed.
Finally, took the chance to:
* Clean up unnecessary usage of `KafkaConfig$.MODULE$` in
related files.
* Add missing `Test` annotations to `AdvertiseBrokerTest` and
make necessary changes for the tests to pass.
Reviewers: David Jacot <djacot@confluent.io>, Luke Chen <showuon@gmail.com>
Directly use `RaftClient.Listener`, `SnapshotWriter` and `SnapshotReader` in the quorum controller.
1. Allow `RaftClient` users to create snapshots by specifying the last committed offset and last committed epoch. These values are validated against the log and leader epoch cache.
2. Remove duplicate classes in the metadata module for writing and reading snapshots.
3. Changed the logic for comparing snapshots. The old logic was assuming a certain batch grouping. This didn't match the implementation of the snapshot writer. The snapshot writer is free to merge batches before writing them.
4. Improve `LocalLogManager` to keep track of multiple snapshots.
5. Improve the documentation and API for the snapshot classes to highlight the distinction between the offset of batches in the snapshot vs the offset of batches in the log. These two offsets are independent of one another. `SnapshotWriter` and `SnapshotReader` expose a method called `lastOffsetFromLog` which represents the last inclusive offset from the log that is represented in the snapshot.
Reviewers: dengziming <swzmdeng@163.com>, Jason Gustafson <jason@confluent.io>
This PR includes adding the NamedTopology to the Subscription/AssignmentInfo, and to the StateDirectory so it can place NamedTopology tasks within the hierarchical structure with task directories under the NamedTopology parent dir.
Reviewers: Walker Carlson <wcarlson@confluent.io>, Guozhang Wang <guozhang@confluent.io>
This patch removes the temporary shim layer we added to bridge the interface
differences between MetaLogManager and RaftClient. Instead, we now use the
RaftClient directly from the metadata module. This also means that the
metadata gradle module now depends on raft, rather than the other way around.
Finally, this PR also consolidates the handleResign and handleNewLeader APIs
into a single handleLeaderChange API.
Co-authored-by: Jason Gustafson <jason@confluent.io>
Implement a striped replica placement algorithm for KRaft. This also
means implementing rack awareness. Previously, KRraft just chose
replicas randomly in a non-rack-aware fashion. Also, allow replicas to
be placed on fenced brokers if there are no other choices. This was
specified in KIP-631 but previously not implemented.
Reviewers: Jun Rao <junrao@gmail.com>
The QuorumController should honor the timeout for RPC requests
which feature a timeout. For electLeaders, attempt to trigger a leader
election for all partitions when the request specifies null for the topics
argument.
Reviewers: David Arthur <mumrah@gmail.com>
Introduce List serde for primitive types or custom serdes with a serializer and a deserializer according to KIP-466
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias J. Sax <mjsax@conflunet.io>, John Roesler <roesler@confluent.io>, Michael Noll <michael@confluent.io>
Added server-common module to have server side common classes. Moved ApiMessageAndVersion, RecordSerde, AbstractApiMessageSerde, and BytesApiMessageSerde to server-common module.
Reivewers: Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
KAFKA-12429: Added serdes for the default implementation of RLMM based on an internal topic as storage. This topic will receive events of RemoteLogSegmentMetadata, RemoteLogSegmentUpdate, and RemotePartitionDeleteMetadata. These events are serialized into Kafka protocol message format.
Added tests for all the event types for that topic.
This is part of the tiered storaqe implementation KIP-405.
Reivewers: Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
Implement Raft Snapshot loading API.
1. Adds a new method `handleSnapshot` to `raft.Listener` which is called whenever the `RaftClient` determines that the `Listener` needs to load a new snapshot before reading the log. This happens when the `Listener`'s next offset is less than the log start offset also known as the earliest snapshot.
2. Adds a new type `SnapshotReader<T>` which provides a `Iterator<Batch<T>>` interface and de-serializes records in the `RawSnapshotReader` into `T`s
3. Adds a new type `RecordsIterator<T>` that implements an `Iterator<Batch<T>>` by scanning a `Records` object and deserializes the batches and records into `Batch<T>`. This type is used by both `SnapshotReader<T>` and `RecordsBatchReader<T>` internally to implement the `Iterator` interface that they expose.
4. Changes the `MockLog` implementation to read one or two batches at a time. The previous implementation always read from the given offset to the high-watermark. This made it impossible to test interesting snapshot loading scenarios.
5. Removed `throws IOException` from some methods. Some of types were inconsistently throwing `IOException` in some cases and throwing `RuntimeException(..., new IOException(...))` in others. This PR improves the consistent by wrapping `IOException` in `RuntimeException` in a few more places and replacing `Closeable` with `AutoCloseable`.
6. Updated the Kafka Raft simulation test to take into account snapshot. `ReplicatedCounter` was updated to generate snapshot after 10 records get committed. This means that the `ConsistentCommittedData` validation was extended to take snapshots into account. Also added a new invariant to ensure that the log start offset is consistently set with the earliest snapshot.
Reviewers: dengziming <swzmdeng@163.com>, David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>
Fixes the issue with https://issues.apache.org/jira/browse/KAFKA-10847.
To fix the above problem, the left/outer stream-stream join processor uses a buffer to hold non-joined records for some time until the window closes, so they are not processed if a join is found during the join window time. If the window of a record closes and a join was not found, then this should be emitted and processed by the consequent topology processor.
A new time-ordered window store is used to temporary hold records that do not have a join and keep the records keys ordered by time. The KStreamStreamJoin has a reference to this new store . For every non-joined record seen, the processor writes it to this new state store without processing it. When a joined record is seen, the processor deletes the joined record from the new state store to prevent further processing.
Records that were never joined at the end of the window + grace period are emitted to the next topology processor. I use the stream time to check for the expiry time for determinism results . The KStreamStreamJoin checks for expired records and emit them every time a new record is processed in the join processor.
The new state store is shared with the left and right join nodes. The new store needs to serialize the record keys using a combined key of <joinSide-recordKey>. This key combination helps to delete the records from the other join if a joined record is found. Two new serdes are created for this, KeyAndJoinSideSerde which serializes a boolean value that specifies the side where the key is found, and ValueOrOtherValueSerde that serializes either V1 or V2 based on where the key was found.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Deprecates the following
1. StreamsConfig.EXACTLY_ONCE
2. StreamsConfig.EXACTLY_ONCE_BETA
3. Producer#sendOffsetsToTransaction(Map offsets, String consumerGroupId)
And introduces a new StreamsConfig.EXACTLY_ONCE_V2 config. Additionally, this PR replaces usages of the term "eos-beta" throughout the code with the term "eos-v2"
Reviewers: Matthias J. Sax <mjsax@confluent.io>
Implement the createPartitions RPC which adds more partitions to a topic
in the KIP-500 controller. Factor out some of the logic for validating
manual partition assignments, so that it can be shared between
createTopics and createPartitions. Add a startPartition argument to the
replica placer.
Reviewers: Jason Gustafson <jason@confluent.io>
KAFKA-12368: Added inmemory implementations for RemoteStorageManager and RemoteLogMetadataManager.
Added inmemory implementation for RemoteStorageManager and RemoteLogMetadataManager. A major part of inmemory RLMM will be used in the default RLMM implementation which will be based on topic storage. These will be used in unit tests for tiered storage.
Added tests for both the implementations and their supported classes.
This is part of tiered storage implementation, KIP-405.
Reivewers: Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
Implement controller-side snapshot generation.Implement QuorumController snapshot
generation. Note that this PR does not handle KRaft integration, just the internal
snapshot record generation and consumption logic.
Reading a snapshot is relatively straightforward. When the QuorumController
starts up, it loads the most recent snapshot. This is just a series of records
that we replay, plus a log offset ("snapshot epoch") that we advance to.
Writing a snapshot is more complex. There are several components:
the SnapshotWriter which persists the snapshot, the SnapshotGenerator
which manages writing each batch of records, and the SnapshotGeneratorManager
which interfaces the preceding two classes with the event queue.
Controller snapshots are done incrementally. In order to avoid blocking the
controller thread for a long time, we pull a few record batches at a time from
our record batch iterators. These iterators are implemented by controller
manager classes such as ReplicationControlManager, ClusterControlManager, etc.
Finally, this PR adds ControllerTestUtils#deepSortRecords and
ControllerTestUtils#assertBatchIteratorContains, which make it easier to write
unit tests. Since records are often constructed from unsorted data structures,
it is often useful to sort them before comparing them.
Reviewers: David Arthur <mumrah@gmail.com>
1. Add `canGrantVote` to `EpochState`
2. Move the if-else in `KafkaRaftCllient.handleVoteRequest` to `EpochState`
3. Add unit tests for `canGrantVote`
Reviewers: Jason Gustafson <jason@confluent.io>
Introduce "testkit" package which includes KafkaClusterTestKit class for enabling integration tests of self-managed clusters. Also make use of this new integration test harness in the ClusterTestExtentions JUnit extension.
Adds RaftClusterTest for basic self-managed integration test.
Reviewers: Jason Gustafson <jason@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
Co-authored-by: Colin P. McCabe <cmccabe@apache.org>
Previously we implemented ClusterId validation for the Fetch API in the Raft implementation. This patch adds ClusterId validation to the remaining Raft RPCs.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Jason Gustafson <jason@confluent.io>
This patch changes the raft simulation tests to use jqwik, which is a property testing library. This provides two main benefits:
- It simplifies the randomization of test parameters. Currently the tests use a fixed set of `Random` seeds, which means that most builds are doing redundant work. We get a bigger benefit from allowing each build to test different parameterizations.
- It makes it easier to reproduce failures. Whenever a test fails, jqwik will report the random seed that failed. A developer can then modify the `@Property` annotation to use that specific seed in order to reproduce the failure.
This patch also includes an optimization for `MockLog.earliestSnapshotId` which reduces the time to run the simulation tests dramatically.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chia-Ping Tsai <chia7712@gmail.com>, José Armando García Sancio <jsancio@gmail.com>, David Jacot <djacot@confluent.io>
1. Remove org.apache.log4j from allowed import list of shell, trogdor subpackage; they uses slf4j, not log4.
2. Remove org.slf4j from allowed import list of clients, server subpackage: org.slf4j is allowed globally.
3. Remove org.apache.log4j from streams subpackage's allowed import list
Reviewers: David Jacot <david.jacot@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
This patch enables delete topic support for the new KIP-500 controller. Also fixes the following:
- Fix a bug where feature level records were not correctly replayed.
- Fix a bug in TimelineHashMap#remove where the wrong type was being returned.
Reviewers: Jason Gustafson <jason@confluent.io>, Justine Olshan <jolshan@confluent.io>, Ron Dagostino <rdagostino@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Jun Rao <junrao@gmail.com>
Co-authored-by: Jason Gustafson <jason@confluent.io>
Always grab a new thread.id and verify that a thread has fully shut down to DEAD before removing from the `threads` list and making that id available again
Reviewers: Walker Carlson <wcarlson@confluent.io>, Bruno Cadonna <cadonna@confluent.io>
Implements KIP-695
Reverts a previous behavior change to Consumer.poll and replaces
it with a new Consumer.currentLag API, which returns the client's
currently cached lag.
Uses this new API to implement the desired task idling semantics
improvement from KIP-695.
Reverts fdcf8fbf72 / KAFKA-10866: Add metadata to ConsumerRecords (#9836)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Guozhang Wang <guozhang@apache.org>
The quorum controller stores metadata in the KIP-500 metadata log, not in Apache
ZooKeeper. Each controller node is a voter in the metadata quorum. The leader of the
quorum is the active controller, which processes write requests. The followers are standby
controllers, which replay the operations written to the log. If the active controller goes away,
a standby controller can take its place.
Like the ZooKeeper-based controller, the quorum controller is based on an event queue
backed by a single-threaded executor. However, unlike the ZK-based controller, the quorum
controller can have multiple operations in flight-- it does not need to wait for one operation
to be finished before starting another. Therefore, calls into the QuorumController return
CompleteableFuture objects which are completed with either a result or an error when the
operation is done. The QuorumController will also time out operations that have been
sitting on the queue too long without being processed. In this case, the future is completed
with a TimeoutException.
The controller uses timeline data structures to store multiple "versions" of its in-memory
state simultaneously. "Read operations" read only committed state, which is slightly older
than the most up-to-date in-memory state. "Write operations" read and write the latest
in-memory state. However, we can not return a successful result for a write operation until
its state has been committed to the log. Therefore, if a client receives an RPC response, it
knows that the requested operation has been performed, and can not be undone by a
controller failover.
Reviewers: Jun Rao <junrao@gmail.com>, Ron Dagostino <rdagostino@confluent.io>
The Kafka Metadata shell is a new command which allows users to
interactively examine the metadata stored in a KIP-500 cluster.
It can examine snapshot files that are specified via --snapshot.
The metadata tool works by replaying the log and storing the state into
in-memory nodes. These nodes are presented in a fashion similar to
filesystem directories.
Reviewers: Jason Gustafson <jason@confluent.io>, David Arthur <mumrah@gmail.com>, Igor Soarez <soarez@apple.com>
Previously all APIs were accessible on every listener exposed by the broker, but
with KIP-500, that is no longer true. We now have more complex requirements for
API accessibility.
For example, the KIP-500 controller exposes some APIs which are not exposed by
brokers, such as BrokerHeartbeatRequest, and does not expose most client APIs,
such as JoinGroupRequest, etc. Similarly, the KIP-500 broker does not implement
some APIs that the ZK-based broker does, such as LeaderAndIsrRequest and
UpdateFeaturesRequest.
All of this means that we need more sophistication in how we expose APIs and
keep them consistent with the ApiVersions API. Up until now, we have been
working around this using the controllerOnly flag inside ApiKeys, but this is
not rich enough to support all of the cases listed above. This PR introduces a
new "listeners" field to the request schema definitions. This field is an array
of strings which indicate the listener types in which the API should be exposed.
We currently support "zkBroker", "broker", and "controller". ("broker"
indicates the KIP-500 broker, whereas zkBroker indicates the old broker).
This PR also creates ApiVersionManager to encapsulate the creation of the
ApiVersionsResponse based on the listener type. Additionally, it modifies
SocketServer to check the listener type of received requests before forwarding
them to the request handler.
Finally, this PR also fixes a bug in the handling of the ApiVersionsResponse
prior to authentication. Previously a static response was sent, which means that
changes to features would not get reflected. This also meant that the logic to
ensure that only the intersection of version ranges supported by the controller
would get exposed did not work. I think this is important because some clients
rely on the initial pre-authenticated ApiVersions response rather than doing a
second round after authentication as the Java client does.
One final cleanup note: I have removed the expectation that envelope requests
are only allowed on "privileged" listeners. This made sense initially because
we expected to use forwarding before the KIP-500 controller was available. That
is not the case anymore and we expect the Envelope API to only be exposed on the
controller listener. I have nevertheless preserved the existing workarounds to
allow verification of the forwarding behavior in integration testing.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
This PR adds the KIP-500 BrokerServer and ControllerServer classes and
makes some related changes to get them working. Note that the ControllerServer
does not instantiate a QuorumController object yet, since that will be added in
PR #10070.
* Add BrokerServer and ControllerServer
* Change ApiVersions#computeMaxUsableProduceMagic so that it can handle
endpoints which do not support PRODUCE (such as KIP-500 controller nodes)
* KafkaAdminClientTest: fix some lingering references to decommissionBroker
that should be references to unregisterBroker.
* Make some changes to allow SocketServer to be used by ControllerServer as
we as by the broker.
* We now return a random active Broker ID as the Controller ID in
MetadataResponse for the Raft-based case as per KIP-590.
* Add the RaftControllerNodeProvider
* Add EnvelopeUtils
* Add MetaLogRaftShim
* In ducktape, in config_property.py: use a KIP-500 compatible cluster ID.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, David Arthur <mumrah@gmail.com>
We don't really need it and it causes problems in older Android versions
and GraalVM native image usage (there are workarounds for the latter).
Move the logic to separate classes that are only invoked when the
relevant compression library is actually used. Place such classes
in their own package and enforce via checkstyle that only these
classes refer to compression library packages.
To avoid cyclic dependencies, moved `BufferSupplier` to the `utils`
package.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Add MetaLogListener, LocalLogManager, and related classes. These
classes are used by the KIP-500 controller and broker to interface with the
Raft log.
Also add the Controller interface. The implementation will be added in a separate PR.
Reviewers: Ron Dagostino <rdagostino@confluent.io>, David Arthur <mumrah@gmail.com>
Add KafkaEventQueue, which is used by the KIP-500 controller to manage its event queue.
Compared to using an Executor, KafkaEventQueue has the following advantages:
* Events can be given "deadlines." If an event lingers in the queue beyond the deadline, it
will be completed with a timeout exception. This is useful for implementing timeouts for
controller RPCs.
* Events can be prepended to the queue as well as appended.
* Events can be given tags to make them easier to manage. This is especially useful for
rescheduling or cancelling events which were previously scheduled to execute in the future.
Reviewers: Jun Rao <junrao@gmail.com>, José Armando García Sancio <jsancio@gmail.com>
This patch adds a `RecordSerde` implementation for the metadata record format expected by KIP-631.
Reviewers: Colin McCabe <cmccabe@apache.org>, Ismael Juma <mlists@juma.me.uk>
Replace BrokerStates.scala with BrokerState.java, to make it easier to use from Java code if needed. This also makes it easier to go from a numeric type to an enum.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Add the metadata gradle module, which will contain the metadata record
definitions, and other metadata-related broker-side code.
Add MetadataParser, MetadataParseException, etc.
Reviewers: José Armando García Sancio <jsancio@gmail.com>, Ismael Juma <ismael@juma.me.uk>, David Arthur <mumrah@gmail.com>
* Use the packages/classes from JUnit 5
* Move description in `assert` methods to last parameter
* Convert parameterized tests so that they work with JUnit 5
* Remove `hamcrest`, it didn't seem to add much value
* Fix `Utils.mkEntry` to have correct `equals` implementation
* Add a missing `@Test` annotation in `SslSelectorTest` override
* Adjust regex in `SaslAuthenticatorTest` due to small change in the
assert failure string in JUnit 5
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This patch contains the following improvements:
- Separate inbound/outbound request flows so that we can open the door for concurrent inbound request handling
- Rewrite `KafkaNetworkChannel` to use `InterBrokerSendThread` which fixes a number of bugs/shortcomings
- Get rid of a lot of boilerplate conversions in `KafkaNetworkChannel`
- Improve validation of inbound responses in `KafkaRaftClient` by checking correlationId. This fixes a bug which could cause an out of order Fetch to be applied incorrectly.
Reviewers: David Arthur <mumrah@gmail.com>
This change propagates topic ids to brokers in LeaderAndIsr Request. It also removes the topic name from the LeaderAndIsr Response, reorganizes the response to be sorted by topic, and includes the topic ID.
In addition, the topic ID is persisted to each replica in Log as well as in a file on disk. This file is read on startup and if the topic ID exists, it will be reloaded.
Reviewers: David Jacot <djacot@confluent.io>, dengziming <dengziming1993@gmail.com>, Nikhil Bhatia <rite2nikhil@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
Includes:
- New API to authorize by resource type
- Default implementation for the method that supports super users and ACLs
- Optimized implementation in AclAuthorizer that supports ACLs, super users and allow.everyone.if.no.acl.found
- Benchmarks and tests
- InitProducerIdRequest authorized for Cluster:IdempotentWrite or WRITE to any topic, ProduceRequest authorized only for topic even if idempotent
Reviewers: Lucas Bradstreet <lucas@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
This PR adds support for generating snapshot for KIP-630.
1. Adds the interfaces `RawSnapshotWriter` and `RawSnapshotReader` and the implementations `FileRawSnapshotWriter` and `FileRawSnapshotReader` respectively. These interfaces and implementations are low level API for writing and reading snapshots. They are internal to the Raft implementation and are not exposed to the users of `RaftClient`. They operation at the `Record` level. These types are exposed to the `RaftClient` through the `ReplicatedLog` interface.
2. Adds a buffered snapshot writer: `SnapshotWriter<T>`. This type is a higher-level type and it is exposed through the `RaftClient` interface. A future PR will add the related `SnapshotReader<T>`, which will be used by the state machine to load a snapshot.
Reviewers: Jason Gustafson <jason@confluent.io>
Fix infinite loop in assignor when trying to resolve the number of partitions in a topology with a windowed FKJ. Also adds a check to this loop to break out and fail the application if we detect that we are/will be stuck in an infinite loop
Reviewers: Matthias Sax <matthias@confluent.io>
This PR adds support for forwarding of the following RPCs:
AlterConfigs
IncrementalAlterConfigs
AlterClientQuotas
CreateTopics
Co-authored-by: Jason Gustafson <jason@confluent.io>
Reviewers: Jason Gustafson <jason@confluent.io>
Currently, we pass multiple object reference (AdminClient,TaskManager, and a few more) into StreamsPartitionAssignor. Furthermore, we (miss)use TaskManager#mainConsumer() to get access to the main consumer (we need to do this, to avoid a cyclic dependency).
This PR unifies how object references are passed into a single ReferenceContainer class to
- not "miss use" the TaskManager as reference container
- unify how object references are passes
Note: we need to use a reference container to avoid cyclic dependencies, instead of using a config for each passed reference individually.
Reviewers: John Roesler <john@confluent.io>
Implements KIP-478 for the test-utils module:
* adds mocks of the new ProcessorContext and StateStoreContext
* adds tests that all stores and store builders are usable with the new mock
* adds tests that the new Processor api is usable with the new mock
* updates the demonstration Processor to the new api
Reviewers: Guozhang Wang <guozhang@apache.org>
Adds support for SSL key and trust stores to be specified in PEM format either as files or directly as configuration values.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
* Extract the mock RocksDBConfigSetter into a separate class.
* De-dup unit tests covering RocksDBConfigSetter.
Reviewers: Boyang Chen <boyang@confluent.io>
Migrate different components of the old ProcessorContext interface
into separate interfaces that are more appropriate for their usages.
See KIP-478 for the details.
Reviewers: Guozhang Wang <guozhang@apache.org>, Paul Whalen <pgwhalen@gmail.com>
This is the core Raft implementation specified by KIP-595: https://cwiki.apache.org/confluence/display/KAFKA/KIP-595%3A+A+Raft+Protocol+for+the+Metadata+Quorum. We have created a separate "raft" module where most of the logic resides. The new APIs introduced in this patch in order to support Raft election and such are disabled in the server until the integration with the controller is complete. Until then, there is a standalone server which can be used for testing the performance of the Raft implementation. See `raft/README.md` for details.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Boyang Chen <boyang@confluent.io>
Co-authored-by: Boyang Chen <boyang@confluent.io>
Co-authored-by: Guozhang Wang <wangguoz@gmail.com>
Add a backwardFetch call to the window store for sliding window
processing. While the implementation works with the forward call
to the window store, using backwardFetch allows for the iterator
to be closed earlier, making implementation more efficient.
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, John Roesler <vvcephei@apache.org>
For the generated message code, put the JSON conversion functionality
in a separate JsonConverter class.
Make MessageDataGenerator simply another generator class, alongside the
new JsonConverterGenerator class. Move some of the utility functions
from MessageDataGenerator into FieldSpec and other places, so that they
can be used by other generator classes.
Use argparse4j to support a better command-line for the generator.
Reviewers: David Arthur <mumrah@gmail.com>
Only check if positions need validation if there is new metadata.
Also fix some inefficient java.util.stream code in the hot path of SubscriptionState.
This patch fixes the generated serde logic for the 'records' type so that it uses the compact byte array representation consistently when flexible versions are enabled.
Reviewers: David Arthur <mumrah@gmail.com>
- part of KIP-572
- removed the usage of `retries` in `GlobalStateManger`
- instead of retries the new `task.timeout.ms` config is used
Reviewers: John Roesler <john@confluent.io>, Boyang Chen <boyang@confluent.io>, Guozhang Wang <guozhang@confluent.io>
- replace System.exit with Exit.exit in all relevant classes
- forbid use of System.exit in all relevant classes and add exceptions for others
Co-authored-by: John Roesler <vvcephei@apache.org>
Co-authored-by: Matthias J. Sax <matthias@confluent.io>
Reviewers: Lucas Bradstreet <lucas@confluent.io>, Ismael Juma <ismael@confluent.io>
Refactored FetchRequest and FetchResponse to use the generated message classes for serialization and deserialization. This allows us to bypass unnecessary Struct conversion in a few places. A new "records" type was added to the message protocol which uses BaseRecords as the field type. When sending, we can set a FileRecords instance on the message, and when receiving the message class will use MemoryRecords.
Also included a few JMH benchmarks which indicate a small performance improvement for requests with high partition counts or small record sizes.
Reviewers: Jason Gustafson <jason@confluent.io>, Boyang Chen <boyang@confluent.io>, David Jacot <djacot@confluent.io>, Lucas Bradstreet <lucas@confluent.io>, Ismael Juma <ismael@juma.me.uk>, Colin P. McCabe <cmccabe@apache.org>
This PR includes 3 MessageFormatters for MirrorMaker2 internal topics:
- HeartbeatFormatter
- CheckpointFormatter
- OffsetSyncFormatter
This also introduces a new public interface org.apache.kafka.common.MessageFormatter that users can implement to build custom formatters.
Reviewers: Konstantine Karantasis <k.karantasis@gmail.com>, Ryanne Dolan <ryannedolan@gmail.com>, David Jacot <djacot@confluent.io>
Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
Co-authored-by: Edoardo Comar <ecomar@uk.ibm.com>
We need to make sure that corrupted standby tasks are actually cleaned up upon a TaskCorruptedException. However due to the commit prior to invoking handleCorruption, it's possible to throw a TaskMigratedException before actually cleaning up any of the corrupted tasks.
This is fine for active tasks since handleLostAll will finish up the job, but it does nothing with standby tasks. We should make sure that standby tasks are handled before attempting to commit (which we can do, since we don't need to commit anything for the corrupted standbys)
Reviewers: Guozhang Wang <wangguoz@gmail.com>
Bug Details:
Mistakenly setting the value serde to the key serde for an internal wrapped serde in the FKJ workflow.
Testing:
Modified the existing test to reproduce the issue, then verified that the test passes.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, John Roesler <vvcephei@apache.org>
In this PR, I have implemented various classes and integration for the read path of the feature versioning system (KIP-584). The ultimate plan is that the cluster-wide finalized features information is going to be stored in ZK under the node /feature. The read path implemented in this PR is centered around reading this finalized features information from ZK, and, processing it inside the Broker.
Here is a summary of what's in this PR (a lot of it is new classes):
A facility is provided in the broker to declare its supported features, and advertise its supported features via its own BrokerIdZNode under a features key.
A facility is provided in the broker to listen to and propagate cluster-wide finalized feature changes from ZK.
When new finalized features are read from ZK, feature incompatibilities are detected by comparing against the broker's own supported features.
ApiVersionsResponse is now served containing supported and finalized feature information (using the newly added tagged fields).
Reviewers: Boyang Chen <boyang@confluent.io>, Jun Rao <junrao@gmail.com>
In the first version of the incremental cooperative protocol, in the presence of a failed sync request by the leader, the assignor was designed to treat the unapplied assignments as lost and trigger a rebalance delay.
This commit applies optimizations in these cases to avoid the unnecessary activation of the rebalancing delay. First, if the worker that loses the sync group request or response is the leader, then it detects this failure by checking the what is the expected generation when it performs task assignments. If it's not the expected one, it resets its view of the previous assignment because it wasn't successfully applied and it doesn't represent a correct state. Furthermore, if the worker that has missed the assignment sync is an ordinary worker, then the leader is able to detect that there are lost assignments and instead of triggering a rebalance delay among the same members of the group, it treats the lost tasks as new tasks and reassigns them immediately. If the lost assignment included revocations that were not applied, the leader reapplies these revocations again.
Existing unit tests and integration tests are adapted to test the proposed optimizations.
Reviewers: Randall Hauch <rhauch@gmail.com>
Motivation and pseudo code algorithm in the ticket.
Added a scale test with large number of topic partitions and consumers and 30s timeout.
With these changes, assignment with 2,000 consumers and 200 topics with 2,000 each completes within a few seconds.
Porting the same test to trunk, it took 2 minutes even with a 100x reduction in the number of topics (ie, 2 minutes for 2,000 consumers and 2 topics with 2,000 partitions)
Should be cherry-picked to 2.6, 2.5, and 2.4
Reviewers: Guozhang Wang <wangguoz@gmail.com>
This PR has gone through several significant transitions of its own, but here's the latest:
* TaskManager just collects the tasks to transition and refers to the active/standby task creator to handle closing & recycling the old task and creating the new one. If we ever hit an exception during the close, we bail and close all the remaining tasks as dirty.
* The task creators tell the task to "close but recycle state". If this is successful, it tells the recycled processor context and state manager that they should transition to the new type.
* During "close and recycle" the task just does a normal clean close, but instead of closing the state manager it informs it to recycle itself: maintain all of its store information (most importantly the current store offsets) but unregister the changelogs from the changelog reader
* The new task will (re-)register its changelogs during initialization, but skip re-registering any stores. It will still read the checkpoint file, but only use the written offsets if the store offsets are not already initialized from pre-transition
* To ensure we don't end up with manual compaction disabled for standbys, we have to call the state restore listener's onRestoreEnd for any active restoring stores that are switching to standbys
Reviewers: John Roesler <vvcephei@apache.org>, Guozhang Wang <wangguoz@gmail.com>
Implementation for KIP-610: https://cwiki.apache.org/confluence/display/KAFKA/KIP-610%3A+Error+Reporting+in+Sink+Connectors based on which sink connectors can now report errors at the final stages of the stream that exports records to the sink system.
This PR adds the `ErrantRecordReporter` interface as well as its implementation - `WorkerErrantRecordReporter`. The `WorkerErrantRecordReporter` is created in `Worker` and brought up through `WorkerSinkTask` to `WorkerSinkTaskContext`.
An integration test and unit tests have been added.
Reviewers: Lev Zemlyanov <lev@confluent.io>, Greg Harris <gregh@confluent.io>, Chris Egerton <chrise@confluent.io>, Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>
Implemented KIP-606 to add metadata context to MetricsReporter.
Author: Xiaodong Du <xdu@confluent.io>
Reviewers: David Arthur <mumrah@gmail.com>, Randall Hauch <rhauch@gmail.com>, Xavier Léauté <xavier@confluent.io>, Ryan Pridgeon <ryan.n.pridgeon@gmail.com>
Add e2e latency metrics at the beginning and end of task topologies
as INFO-level processor-node-level metrics.
Implements: KIP-613
Reviewers: John Roesler <vvcephei@apache.org>, Andrew Choi <a24choi@edu.uwaterloo.ca>, Bruno Cadonna <cadonna@confluent.io>, Matthias J. Sax <matthias@confluent.io>
Kafka Connect workers have been able to create Connect's internal topics using the new admin client for some time now (see KAFKA-4667 for details). However, tasks of source connectors are still relying upon the broker to auto-create topics with default config settings if they don't exist, or expect these topics to exist before the connector is deployed, if their configuration needs to be specialized.
With the implementation of KIP-158 here, if `topic.creation.enable=true`, Kafka Connect will supply the source tasks of connectors that are configured to create topics with an admin client that will allow them to create new topics on-the-fly before writing the first source records to a new topic. Additionally, each source connector has the opportunity to customize the topic-specific settings of these new topics by defining groups of topic configurations.
This feature is tested here via unit tests (old tests that have been adjusted and new ones) as well as integration tests.
Reviewers: Randall Hauch <rhauch@gmail.com>
Added support for customizing the HTTP response headers for Kafka Connect as described in KIP-577.
Author: Jeff Huang <jeff.huang@confluent.io>
Reviewer: Randall Hauch <rhauch@gmail.com>
Added support for -1 replication factor and partitions for distributed worker internal topics by expanding the allowed values for the internal topics’ replication factor and partitions from positive values to also include -1 to signify that the broker defaults should be used.
The Kafka storage classes were already constructing a `NewTopic` object (always with a replication factor and partitions) and sending it to Kafka when required. This change will avoid setting the replication factor and/or number of partitions on this `NewTopic` if the worker configuration uses -1 for the corresponding configuration value.
Also added support for extra settings for internal topics on distributed config, status, and offset internal topics.
Quite a few new tests were added to verify that the `TopicAdmin` utility class is correctly using the AdminClient, and that the `DistributedConfig` validators for these configurations are correct. Also added integration tests for internal topic creation, covering preexisting functionality plus the new functionality.
Author: Randall Hauch <rhauch@gmail.com>
Reviewer: Konstantine Karantasis <konstantine@confluent.io>
First set of cleanup pushed to followup PR after KIP-441 Pt. 5. Main changes are:
1. Moved `RankedClient` and the static `buildClientRankingsByTask` to a new file
2. Moved `Movement` and the static `getMovements` to a new file (also renamed to `TaskMovement`)
3. Consolidated the many common variables throughout the assignment tests to the new `AssignmentTestUtils`
4. New utility to generate comparable/predictable UUIDs for tests, and removed the generic from `TaskAssignor` and all related classes
Reviewers: John Roesler <vvcephei@apache.org>, Andrew Choi <a24choi@edu.uwaterloo.ca>
Adds a new TaskAssignor implementation, currently hidden behind an internal feature flag, that implements the high availability algorithm of KIP-441.
Reviewers: Bruno Cadonna <bruno@confluent.io>, John Roesler <vvcephei@apache.org>
This PR works to improve high watermark checkpointing performance.
`ReplicaManager.checkpointHighWatermarks()` was found to be a major contributor to GC pressure, especially on Kafka clusters with high partition counts and low throughput.
Added a JMH benchmark for `checkpointHighWatermarks` which establishes a
performance baseline. The parameterized benchmark was run with 100, 1000 and
2000 topics.
Modified `ReplicaManager.checkpointHighWatermarks()` to avoid extra copies and cached
the Log parent directory Sting to avoid frequent allocations when calculating
`File.getParent()`.
A few clean-ups:
* Changed all usages of Log.dir.getParent to Log.parentDir and Log.dir.getParentFile to
Log.parentDirFile.
* Only expose public accessor for `Log.dir` (consistent with `Log.parentDir`)
* Removed unused parameters in `Partition.makeLeader`, `Partition.makeFollower` and `Partition.createLogIfNotExists`.
Benchmark results:
| Topic Count | Ops/ms | MB/sec allocated |
|-------------|---------|------------------|
| 100 | + 51% | - 91% |
| 1000 | + 143% | - 49% |
| 2000 | + 149% | - 50% |
Reviewers: Lucas Bradstreet <lucas@confluent.io>. Ismael Juma <ismael@juma.me.uk>
Co-authored-by: Gardner Vickers <gardner@vickers.me>
Co-authored-by: Ismael Juma <ismael@juma.me.uk>
Once we have encoded the offset sums per task for each client, we can compute the overall lag during assign by fetching the end offsets for all changelog and subtracting.
If the listOffsets request fails, we simply return a "completely sticky" assignment, ie all active tasks are given to previous owners regardless of balance.
Builds (but does not yet use) the statefulTasksToRankedCandidates map with the ranking:
Rank -1: active running task
Rank 0: standby or restoring task whose overall lag is within acceptableRecoveryLag
Rank 1: tasks whose lag is unknown (eg during version probing)
Rank 1+: all other tasks are ranked according to their actual total lag
Implements: KIP-441
Reviewers: Bruno Cadonna <bruno@confluent.io>, John Roesler <vvcephei@apache.org>
Rewrite ReassignPartitionsCommand to use the KIP-455 API when possible, rather
than direct communication with ZooKeeper. Direct ZK access is still supported,
but deprecated, as described in KIP-455.
As specified in KIP-455, the tool has several new flags. --cancel stops
an assignment which is in progress. --preserve-throttle causes the
--verify and --cancel commands to leave the throttles alone.
--additional allows users to execute another partition assignment even
if there is already one in progress. Finally, --show displays all of
the current partition reassignments.
Reorganize the reassignment code and tests somewhat to rely more on unit
testing using the MockAdminClient and less on integration testing. Each
integration test where we bring up a cluster seems to take about 5 seconds, so
it's good when we can get similar coverage from unit tests. To enable this,
MockAdminClient now supports incrementalAlterConfigs, alterReplicaLogDirs,
describeReplicaLogDirs, and some other APIs. MockAdminClient is also now
thread-safe, to match the real AdminClient implementation.
In DeleteTopicTest, use the KIP-455 API rather than invoking the reassignment
command.
- part of KIP-447
- commit all tasks at once using non-eos (and eos-beta in follow up work)
- unified commit logic into TaskManager
- split existing methods of Task interface in pre/post parts
Reviewers: Boyang Chen <boyang@confluent.io>, Guozhang Wang <guozhang@confluent.io>
This PR removes intermediate conversions between `MetadataResponse.TopicMetadata` => `MetadataResponseTopic` and `MetadataResponse.PartitionMetadata` => `MetadataResponsePartition` objects.
There is 15-20% reduction in object allocations and 5-10% improvement in metadata request performance.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson<jason@confluent.io>
To dismiss the usage of operation ++ against Set which is slow when Set has many entries. This pr introduces a new class 'AclSets' which takes multiple Sets as parameters and do 'find' against them one by one. For more details about perf and benchmark, refer to [KAFKA-9685](https://issues.apache.org/jira/browse/KAFKA-9685)
Author: jiao <jiao.zhang@linecorp.com>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#8261 from jiao-zhangS/jira-9685
* Consolidate task/producer management. Now, exactly one component manages
the creation and destruction of Producers, whether they are per-thread or per-task.
* Add missing test coverage on TaskManagerTest
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Boyang Chen <boyang@confluent.io>
This change is the client-side part of KIP-360. It identifies cases where it is safe to abort a transaction, bump the producer epoch, and allow the application to continue without closing the producer. In these cases, when KafkaProducer.abortTransaction() is called, the producer sends an InitProducerId following the transaction abort, which causes the producer epoch to be bumped. The application can then start a new transaction and continue processing.
For recoverable errors in the idempotent producer, the epoch is bumped locally. In-flight requests for partitions with an error are rewritten to reflect the new epoch, and in-flights of all other partitions are allowed to complete using the old epoch.
Reviewers: Boyang Chen <boyang@confluent.io>, Jason Gustafson <jason@confluent.io>
Correct the Connect worker logic to properly disable the new topic status (KIP-558) feature when `topic.tracking.enable=false`, and fix automatic topic status reset after a connector is deleted.
Also adds new `ConnectorTopicsIntegrationTest` and expanded unit tests.
Reviewers: Randall Hauch <rhauch@gmail.com>
This change mainly have 2 components:
1. extend the existing transactions_test.py to also try out new sendTxnOffsets(groupMetadata) API to make sure we are not introducing any regression or compatibility issue
a. We shrink the time window to 10 seconds for the txn timeout scheduler on broker so that we could trigger expiration earlier than later
2. create a completely new system test class called group_mode_transactions_test which is more complicated than the existing system test, as we are taking rebalance into consideration and using multiple partitions instead of one. For further breakdown:
a. The message count was done on partition level, instead of global as we need to visualize
the per partition order throughout the test. For this sake, we extend ConsoleConsumer to print out the data partition as well to help message copier interpret the per partition data.
b. The progress count includes the time for completing the pending txn offset expiration
c. More visibility and feature improvements on TransactionMessageCopier to better work under either standalone or group mode.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Corrects a flaw leading to an exception while building topologies that include both:
* A foreign-key join with the result not explicitly materialized
* An operation after the join that requires source materialization
Also corrects a flaw in TopologyTestDriver leading to output records being enqueued in the wrong order under some (presumably rare) circumstances.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
This PR is collaborated by Guozhang Wang and John Roesler. It is a significant tech debt cleanup on task management and state management, and is broken down by several sub-tasks listed below:
Extract embedded clients (producer and consumer) into RecordCollector from StreamTask.
guozhangwang#2
guozhangwang#5
Consolidate the standby updating and active restoring logic into ChangelogReader and extract out of StreamThread.
guozhangwang#3
guozhangwang#4
Introduce Task state life cycle (created, restoring, running, suspended, closing), and refactor the task operations based on the current state.
guozhangwang#6
guozhangwang#7
Consolidate AssignedTasks into TaskManager and simplify the logic of changelog management and task management (since they are already moved in step 2) and 3)).
guozhangwang#8
guozhangwang#9
Also simplified the StreamThread logic a bit as the embedded clients / changelog restoration logic has been moved into step 1) and 2).
guozhangwang#10
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Bruno Cadonna <bruno@confluent.io>, Boyang Chen <boyang@confluent.io>
This patch adds a new API to the producer to implement transactional offset commit fencing through the group coordinator as proposed in KIP-447. This PR mainly changes on the Producer end for compatible paths to old `sendOffsetsToTxn(offsets, groupId)` vs new `sendOffsetsToTxn(offsets, groupMetadata)`.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>, Jason Gustafson <jason@confluent.io>
Allow null as a valid default for tagged fields. Fix a bunch of cases where this would previously result in null pointer dereferences.
Also allow inferring FieldSpec#versions based on FieldSpec#taggedVersions. Prefix 'key' with an underscore when it is used in the generated code, to avoid potential name collisions if someone names an RPC field "key".
Allow setting setting hexadecimal constants and 64-bit contstants.
Add a lot more test cases to SimpleExampleMessage.json.
Reviewers: Jason Gustafson <jason@confluent.io>
Rather than maintain hand coded protocol serialization code, Streams could use the same code-generation framework as Clients/Core.
There isn't a perfect match, since the code generation framework includes an assumption that you're generating "protocol messages", rather than just arbitrary blobs, but I think it's close enough to justify using it, and improving it over time.
Using the code generation allows us to drop a lot of detail-oriented, brittle, and hard-to-maintain serialization logic in favor of a schema spec.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Boyang Chen <boyang@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
The RPC code generator should support using the ByteBuffer class in addition to byte arrays. By using the ByteBuffer class, we can avoid performing a copy in many situations. Also modify TestByteBufferDataTest to test the new feature.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Guozhang Wang <wangguoz@gmail.com>
Migrate this integration test to use TopologyTestDriver instead of running 3 Streams instances.
Dropped one test that was attempting to produce specific interleavings. If anything, these should be verified deterministically by unit testing.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Added plugin isolation unit tests for various scenarios, with a `TestPlugins` class that compiles and builds multiple test plugins without them being on the classpath and verifies that the Plugins and DelegatingClassLoader behave properly. These initially failed for several cases, but now pass since the issues have been fixed.
KAFKA-8340 and KAFKA-8819 are closely related, and this fix corrects the problems reported in both issues.
Author: Greg Harris <gregh@confluent.io>
Reviewers: Chris Egerton <chrise@confluent.io>, Magesh Nandakumar <mageshn@confluent.io>, Konstantine Karantasis <konstantine@confluent.io>, Randall Hauch <rhauch@gmail.com>
Key improvements with this PR:
* tasks will remain available for IQ during a rebalance (but not during restore)
* continue restoring and processing standby tasks during a rebalance
* continue processing active tasks during rebalance until the RecordQueue is empty*
* only revoked tasks must suspended/closed
* StreamsPartitionAssignor tries to return tasks to their previous consumers within a client
* but do not try to commit, for now (pending KAFKA-7312)
Reviewers: John Roesler <john@confluent.io>, Boyang Chen <boyang@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Implementation of [KIP-382 "MirrorMaker 2.0"](https://cwiki.apache.org/confluence/display/KAFKA/KIP-382%3A+MirrorMaker+2.0)
Author: Ryanne Dolan <ryannedolan@gmail.com>
Author: Arun Mathew <arunmathew88@gmail.com>
Author: In Park <inpark@cloudera.com>
Author: Andre Price <obsoleted@users.noreply.github.com>
Author: christian.hagel@rio.cloud <christian.hagel@rio.cloud>
Reviewers: Eno Thereska <eno.thereska@gmail.com>, William Hammond <william.t.hammond@gmail.com>, Viktor Somogyi <viktorsomogyi@gmail.com>, Jakub Korzeniowski, Tim Carey-Smith, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>, Arun Mathew, Jeremy-l-ford, vpernin, Oleg Kasian <oleg.kasian@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Qihong Chen, Sriharsha Chintalapani <sriharsha@apache.org>, Jun Rao <junrao@gmail.com>, Randall Hauch <rhauch@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#6295 from ryannedolan/KIP-382
https://issues.apache.org/jira/browse/KAFKA-3705
Allows for a KTable to map its value to a given foreign key and join on another KTable keyed on that foreign key. Applies the joiner, then returns the tuples keyed on the original key. This supports updates from both sides of the join.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Matthias J. Sax <mjsax@apache.org>, John Roesler <john@confluent.io>, Boyang Chen <boyang@confluent.io>, Christopher Pettitt <cpettitt@confluent.io>, Bill Bejeck <bbejeck@gmail.com>, Jan Filipiak <Jan.Filipiak@trivago.com>, pgwhalen, Alexei Daniline
Implemented KIP-507 to secure the internal Connect REST endpoints that are only for intra-cluster communication. A new V2 of the Connect subprotocol enables this feature, where the leader generates a new session key, shares it with the other workers via the configuration topic, and workers send and validate requests to these internal endpoints using the shared key.
Currently the internal `POST /connectors/<connector>/tasks` endpoint is the only one that is secured.
This change adds unit tests and makes some small alterations to system tests to target the new `sessioned` Connect subprotocol. A new integration test ensures that the endpoint is actually secured (i.e., requests with missing/invalid signatures are rejected with a 400 BAD RESPONSE status).
Author: Chris Egerton <chrise@confluent.io>
Reviewed: Konstantine Karantasis <konstantine@confluent.io>, Randall Hauch <rhauch@gmail.com>
Implemented KIP-495 to expose a new `admin/loggers` endpoint for the Connect REST API that lists the current log levels and allows the caller to change log levels.
Author: Arjun Satish <arjun@confluent.io>
Reviewer: Randall Hauch <rhauch@gmail.com>
Implemented KIP-440 to allow Connect converters to use record headers when serializing or deserializing keys and values. This change is backward compatible in that the new methods default to calling the older existing methods, so existing Converter implementations need not be changed. This changes the WorkerSinkTask and WorkerSourceTask to use the new converter methods, but Connect's existing Converter implementations and the use of converters for internal topics are intentionally not modified. Added unit tests.
Author: Yaroslav Tkachenko <sapiensy@gmail.com>
Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Ewen Cheslack-Postava <me@ewencp.org>, Randall Hauch <rhauch@gmail.com>
Move the generator checkstyle suppressions to a special section, rather
than mixing them in with the other sections. For generated code, do not
complain about variable names or cyclic complexity.
FieldType.java: remove isInteger since it isn't used anywhere. This way, we
don't have to decide whether a UUID is an integer or not (there are arguments
for both choices). Add FieldType#serializationIsDifferentInFlexibleVersions
and FieldType#isVariableLength.
HeaderGenerator: add the ability to generate static imports. Add
IsNullConditional, VersionConditional, and ClauseGenerator as easier ways of
generating "if" statements.
Part of supporting KIP-213 ( https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+Support+non-key+joining+in+KTable ). Murmur3 hash is used as a hashing mechanism in KIP-213 for the large range of uniqueness. The Murmur3 class and tests are ported directly from Apache Hive, with no alterations to the code or dependencies.
Author: Adam Bellemare <adam.bellemare@wishabi.com>
Reviewers: John Roesler <vvcephei@users.noreply.github.com>, Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <wangguoz@gmail.com>
Closes#7271 from bellemare/murmur3hash
This PR makes two changes to code in the ReplicaManager.updateFollowerFetchState path, which is in the hot path for follower fetches. Although calling ReplicaManager.updateFollowerFetch state is inexpensive on its own, it is called once for each partition every time a follower fetch occurs.
1. updateFollowerFetchState no longer calls maybeExpandIsr when the follower is already in the ISR. This avoid repeated expansion checks.
2. Partition.maybeIncrementLeaderHW is also in the hot path for ReplicaManager.updateFollowerFetchState. Partition.maybeIncrementLeaderHW calls Partition.remoteReplicas four times each iteration, and it performs a toSet conversion. maybeIncrementLeaderHW now avoids generating any intermediate collections when updating the HWM.
**Benchmark results for Partition.updateFollowerFetchState on a r5.xlarge:**
Old:
```
1288.633 ±(99.9%) 1.170 ns/op [Average]
(min, avg, max) = (1287.343, 1288.633, 1290.398), stdev = 1.037
CI (99.9%): [1287.463, 1289.802] (assumes normal distribution)
```
New (when follower fetch offset is updated):
```
261.727 ±(99.9%) 0.122 ns/op [Average]
(min, avg, max) = (261.565, 261.727, 261.937), stdev = 0.114
CI (99.9%): [261.605, 261.848] (assumes normal distribution)
```
New (when follower fetch offset is the same):
```
68.484 ±(99.9%) 0.025 ns/op [Average]
(min, avg, max) = (68.446, 68.484, 68.520), stdev = 0.023
CI (99.9%): [68.460, 68.509] (assumes normal distribution)
```
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
This creates a test that generates sustained connections against Kafka. There
are three different components we can stress with this, KafkaConsumer,
KafkaProducer, and AdminClient. This test tries use minimal bandwidth per
connection to reduce overhead impacts.
This test works by creating a threadpool that creates connections and then
maintains a central pool of connections at a specified keepalive rate. The
keepalive action varies by which component is being stressed:
* KafkaProducer: Sends one single produce record. The configuration for
the produce request uses the same key/value generator as the ProduceBench
test.
* KafkaConsumer: Subscribes to a single partition, seeks to the end, and
then polls a minimal number of records. Each consumer connection is its
own consumer group, and defaults to 1024 bytes as FETCH_MAX_BYTES to keep
traffic to a minimum.
* AdminClient: Makes an API call to get the nodes in the cluster.
NOTE: This test is designed to be run alongside a ProduceBench test for a
specific topic, due to the way the Consumer test polls a single partition.
There may be no data returned by the consumer test if this is run on its own.
The connection should still be kept alive, but with no data returned.
Author: Scott Hendricks <scott.hendricks@confluent.io>
Reviewers: Stanislav Kozlovski, Gwen Shapira
Closes#7289 from scott-hendricks/trunk
This PR changes the TxnOffsetCommit protocol to auto-generated types, and add more unit test coverage to the plain OffsetCommit protocol.
Reviewers: Jason Gustafson <jason@confluent.io>
New Java Authorizer API and a new out-of-the-box authorizer (AclAuthorizer) that implements the new interface.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Prior to this change an NPE is raised when calling AssignedTasks.close
under the following conditions:
1. EOS is enabled
2. The task was in a suspended state
The cause for the NPE is that when a clean close is requested for a
StreamTask the StreamTask tries to commit. However, in the suspended
state there is no producer so ultimately an NPE is thrown for the
contained RecordCollector in flush.
The fix put forth in this commit is to have AssignedTasks call
closeSuspended when it knows the underlying StreamTask is suspended.
Note also that this test is quite involved. I could have just tested
that AssignedTasks calls closeSuspended when appropriate, but that is
testing, IMO, a detail of the implementation and doesn't actually verify
we reproduced the original problem as it was described. I feel much more
confident that we are reproducing the behavior - and we can test exactly
the conditions that lead to it - when testing across AssignedTasks and
StreamTask. I believe this is an additional support for the argument of
eventually consolidating the state split across classes.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Add the AlterPartitionReassignments and ListPartitionReassignments APIs. Also remove an unused methodlength suppression for KafkaAdminClient.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Viktor Somogyi <viktorsomogyi@gmail.com>
Implement KIP-480, which specifies that the default partitioner should use a "sticky" partitioning strategy for records that have a null key.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Lucas Bradstreet <lucasbradstreet@gmail.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Jun Rao <junrao@gmail.com>, Kamal Chandraprakash <kamal.chandraprakash@gmail.com>
This patch is part of KIP-345. We are aiming to support batch leave group request issued from admin client. This diff is the first effort to bump leave group request version.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Jason Gustafson <jason@confluent.io>
ZkUtils was removed so we don't need this anymore.
Also:
* Fix ZkSecurityMigrator and ReplicaManagerTest not to
reference ZkClient classes.
* Remove references to zkclient in various `log4j.properties`
and `import-control.xml`.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
This upgrade exposes a number of new options, including the WriteBufferManager which -- along with existing TableConfig options -- allows users to limit the total memory used by RocksDB across instances. This can alleviate some cascading OOM potential when, for example, a large number of stateful tasks are suddenly migrated to the same host.
The RocksDB docs guarantee backwards format compatibility across versions
Reviewers: Matthias J. Sax <mjsax@apache.org>, Bill Bejeck <bbejeck@gmail.com>,
Implementation to enable policy for Connector Client config overrides. This is
implemented per the KIP-458.
Reviewers: Randall Hauch <rhauch@gmail.com>
Fix registration of Connect REST extensions to prevent deadlocks when extensions get the list of connectors before the herder is available. Added integration test to check the behavior.
Author: Chris Egerton <cegerton@oberlin.edu>
Reviewers: Arjun Satish <arjun@confluent.io>, Randall Hauch <rhauch@gmail.com>
This is the first diff for the implementation of JoinGroup logic for static membership. The goal of this diff contains:
* Add group.instance.id to be unique identifier for consumer instances, provided by end user;
Modify group coordinator to accept JoinGroupRequest with/without static membership, refactor the logic for readability and code reusability.
* Add client side support for incorporating static membership changes, including new config for group.instance.id, apply stream thread client id by default, and new join group exception handling.
* Increase max session timeout to 30 min for more user flexibility if they are inclined to tolerate partial unavailability than burdening rebalance.
* Unit tests for each module changes, especially on the group coordinator logic. Crossing the possibilities like:
6.1 Dynamic/Static member
6.2 Known/Unknown member id
6.3 Group stable/unstable
6.4 Leader/Follower
The rest of the 345 change will be broken down to 4 separate diffs:
* Avoid kicking out members through rebalance.timeout, only do the kick out through session timeout.
* Changes around LeaveGroup logic, including version bumping, broker logic, client logic, etc.
* Admin client changes to add ability to batch remove static members
* Deprecate group.initial.rebalance.delay
Reviewers: Liquan Pei <liquanpei@gmail.com>, Stanislav Kozlovski <familyguyuser192@windowslive.com>, Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>