It is useful to allow ExponentialBackoff to be configured to work
without jitter, in order to make unit tests more repeatable.
Reviewers: David Arthur <mumrah@gmail.com>
* Remove `ExtendedSerializer` and `ExtendedDeserializer`, deprecated since 2.1.
The extra functionality was also made available in `Serializer` and `Deserializer`.
* Remove `close(long, TimeUnit)` from the producer, consumer and admin client,
deprecated since 2.0 for the consumer and 2.2 for the rest. The replacement is `close(Duration)`.
* Remove `ConsumerConfig.addDeserializerToConfig` and `ProducerConfig.addSerializerToConfig`,
deprecated since 2.7 with no replacement. These methods were not intended to be public API
and are likely not used much (if at all).
* Remove `NoOffsetForPartitionException.partition()`, deprecated since 0.11. `partitions()`
should be used instead.
* Remove `MessageFormatter.init(Properties)`, deprecated since 2.7. The `configure(Map)`
method should be used instead.
* Remove `kafka.common.MessageFormatter`, deprecated since 2.7.
`org.apache.kafka.common.MessageFormatter` should be used instead.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>
When auto-creating topics in KIP-500, the broker will send a `CreateTopics` request to the controller. It is useful in this case to preserve the original principal from the corresponding `Metadata` request by wrapping the `CreateTopics` request in an envelope so that the controller may repeat the authorization and to improve auditability. This follows a similar pattern to how standard `CreateTopics` requests are forwarded to the controller.
Reviewers: Jason Gustafson <jason@confluent.io>
KIP-467 added a field in the produce response to allow the broker to indicate which specific records failed validation. This patch adds the logic to propagate this message up to the application.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
The second `clearCompletedSends()` reference should be `clearCompletedReceives()`.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Co-authored-by: Zhao Haiyuan <zhaohaiyuan@mobike.com>
Kafka does not call fsync() on the directory when a new log segment is created and flushed to disk.
The problem is that following sequence of calls doesn't guarantee file durability:
fd = open("log", O_RDWR | O_CREATE); // suppose open creates "log"
write(fd);
fsync(fd);
If the system crashes after fsync() but before the parent directory has been flushed to disk, the log file can disappear.
This PR is to flush the directory when flush() is called for the first time.
Reviewers: Jun Rao <junrao@gmail.com>
Support sorting the elements in ImplicitLinkedHashCollection.
This is useful sometimes in unit tests for comparing collections.
Reviewers: Ismael Juma <ismael@juma.me.uk>
The config has been deprecated since Kafka 2.6 (released ~1 year before
3.0), but it was the default before it got deprecated. As such, it's
reasonably unlikely that people would have set it explicitly.
Given the confusing `default` name even though it's _not_ the default, I
think we should remove it in 3.0.
Also remove `ClientDnsLookup.DEFAULT` (not public API), which unlocks
a number of code simplications.
Reviewers: David Jacot <djacot@confluent.io>
Replaced File with Path in LogSegment Data.
This is a followup of #10173
Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
ConfigEntry's public constructor was deprecated in 1.1.0. This patch removes it in AK 3.0.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
`Admin.electLeaders` is the replacement since the deprecation in Apache Kafka 2.4.0.
The methods were originally introduced in Apache Kafka 2.2.0, so they were only
non deprecated for two releases.
Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
More specifically, remove deprecated:
- Constants in SslConfigs
- Constants in SaslConfigs
- AclBinding constructor
- AclBindingFilter constructor
- PrincipalBuilder and DefaultPrincipalBuilder classes
- ResourceFilter
Also simplify tests and code that no longer have to handle the removed `PrincipalBuilder`.
These removals seem non controversial. There is a straightforward alternative. The
deprecations happened in 1.0.0 and 2.0.0.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This patch fixes a race condition between the background thread calling `ready` and the call to `MockTime.sleep` in the test. If the call to `sleep` happens first, then the test hangs. I fixed it by giving `MockClient` a way to listen to `ready` calls. This combined with a latch fixes the race.
This patch also fixes a similar race condition in `testClientSideTimeoutAfterFailureToReceiveResponse`. After the disconnect, there is a race between the background thread sending the retry request and the foreground sleeping for the needed backoff delay.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, David Arthur <mumrah@gmail.com>
Fix a hanging test in KafkaAdminClientTest by forcing the admin client to shut down
whether or not there are pending requests once the test harness enters shutdown.
Reviewers: Ismael Juma <ijuma@apache.org>, Guozhang Wang <guozhang@apache.org>
This patch adds the new `Admin` API to describe producer state as described by KIP-664: https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions.
The three new APIs added by KIP-664 require different lookup and request patterns:
- DescribeProducers: send to partition leaders
- DescribeTransactions: send to coordinators
- ListTransactions: send to all brokers
Our method of handling complex workflows such as these in `KafkaAdminClient` by chaining together `Call` instances has been clumsy and error-prone at best. I have attempted to introduce a new pattern which separates the lookup stage (e.g. finding partition leaders) from the fulfillment stage (e.g. sending `DescribeProducers`). The lookup stage is implemented by `AdminApiLookupStrategy` and the fulfillment stage is implemented by `AdminApiHandler`. There is a new class `AdminApiDriver` which manages the bookkeeping for these two stages. See the corresponding javadocs for more detail.
This PR provides an example of usage through `DescribeProducersHandler`, which is an implementation of `AdminApiHandler`. It relies on `PartitionLeaderStrategy` which implements `AdminApiLookupStrategy`. In addition to allowing for easier reuse of lookup strategies, this approach provides a more convenient way for testing since all of the logic is not crammed into `KafkaAdminClient`. Follow-up PRs for the rest of KIP-664 will flesh out additional lookup strategies such as for coordinator APIs.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Jacot <djacot@confluent.io>
There were errors while generating javadoc for the streams:test-utils module
because the included TopologyTestDriver imported some excluded classes.
This fixes the errors by inlining the previously excluded packages.
Reviewers: Chia-Ping Tsai <chia7712@apache.org>, Ismael Juma <ijuma@apache.org>
Previously, if we assigned one or more calls to a remote node, but it
never became available, AdminClient would block until the calls hit
their the API timeout. This was particularly unfortunate in the case
where the calls could have been sent to a different node in the cluster.
This PR fixes this behavior by timing out pending connections to
remote nodes if they take longer than the request timeout.
There are a few other small cleanups in this PR: it removes the
unecessary Call#aborted, sets Call#curNode to null after the call has
failed to avoid confusion when debugging or logging, and adds a
"closing" boolean rather than setting newCalls to null when the client
is closed. Also, it increases the log level of the log message that
indicates that we timed out some calls because AdminClient closed,
and simplifies the type of callsInFlight.
Reviewers: Jason Gustafson <jason@confluent.io>
Return null for partitionLag if there is no current position.
This was the desired semantics, the lack of the check was an
oversight.
Patches: KIP-695
Patches: a92b986c85
Reviewers: Walker Carlson <wcarlson@confluent.io>, A. Sophie Blee-Goldman <ableegoldman@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>
Cluster ID has traditionally been treated as a string by the Kafka protocol (for example,
AdminClient returns it as a string). The new KIP-500 code should continue this practice. If
we don't do this, upgrading existing clusters may be harder to do.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Jason Gustafson <jason@confluent.io>
For KIP-698, we need a way to setup internal topics without validating them. This PR adds a setup method to the InternalTopicManager for that purpose.
Reviewers: Rohan Desai <rohan@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
1. Create a reason string to be used for INFO log entry whenever we request re-join or reset generation state.
2. Some minor cleanups.
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>
socket.connection.setup.timeout.max.ms should be 30 seconds.
The current value of 127 seconds is longer than the default API timeout
for AdminClient, and longer than the default request timeouts for the
producer and consumer. We should bring these configs into line with
each other.
Reviewers: Jason Gustafson <jason@confluent.io>
For KIP-698, we need a way to validate internal topics before we create them. This PR adds a validation method to the InternalTopicManager for that purpose.
Reviewers: Rohan Desai <rohan@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
1. rename INVALID_HIGHWATERMARK to INVALID_HIGH_WATERMARK
2. replace FetchResponse.AbortedTransaction by FetchResponseData.AbortedTransaction
3. remove redundant constructors from FetchResponse.PartitionData
4. rename recordSet to records
5. add helpers "recordsOrFail" and "recordsSize" to FetchResponse to process record casting
Reviewers: Ismael Juma <ismael@juma.me.uk>
This patch refactors the code, which constructs the `LeaderAndIsrResponse` in `ReplicaManager#becomeLeaderOrFollower`, to improve the readability and to remove unnecessary operations.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
KIP-405 introduces tiered storage feature in Kafka. With this feature, Kafka cluster is configured with two tiers of storage - local and remote. The local tier is the same as the current Kafka that uses the local disks on the Kafka brokers to store the log segments. The new remote tier uses systems, such as HDFS or S3 or other cloud storages to store the completed log segments. Consumers fetch the records stored in remote storage through the brokers with the existing protocol.
We introduced a few SPIs for plugging in log/index store and remote log metadata store.
This involves two parts
1. Storing the actual data in remote storage like HDFS, S3, or other cloud storages.
2. Storing the metadata about where the remote segments are stored. The default implementation uses an internal Kafka topic.
You can read KIP for more details at https://cwiki.apache.org/confluence/display/KAFKA/KIP-405%3A+Kafka+Tiered+Storage
Reviewers: Jun Rao <junrao@gmail.com>
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>
This patch updates request `listeners` tags to be in line with what the KIP-500 broker/controller support today. We will re-enable these APIs as needed once we have added the support.
I have also updated `ControllerApis` to use `ApiVersionManager` and simplified the envelope handling logic.
Reviewers: Ron Dagostino <rdagostino@confluent.io>, Colin P. McCabe <cmccabe@apache.org>
This patch moves the `peerHost` helper defined in `SslChannelBuilder` into `SslFactor`. `SaslChannelBuilder` is then updated to use a new `createSslEngine` overload which relies on `peerHost` when building its `SslEngine`. The purpose is to avoid the reverse DNS in `getHostName`.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Manikumar Reddy <manikumar.reddy@gmail.com>, Jason Gustafson <jason@confluent.io>