Commit Graph

2263 Commits

Author SHA1 Message Date
Colin Patrick McCabe 137491c724
MINOR: Support ExponentialBackoff without jitter (#10455)
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>
2021-04-06 09:49:14 -07:00
Ismael Juma 2f36001987
KAFKA-12579: Remove various deprecated clients classes/methods for 3.0 (#10438)
* 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>
2021-04-06 08:54:53 -07:00
Boyang Chen cad514bff9
KAFKA-12294; forward auto topic request within envelope on behalf of clients (#10142)
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>
2021-04-05 15:54:57 -07:00
Jason Gustafson 66ba91733c
KAFKA-12548; Propagate record error messages to application (#10445)
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>
2021-04-05 11:50:54 -07:00
zhaohaidao 136f584c05
KAFKA-12615: Fix `Selector.clear()` javadoc typo (#10477)
The second `clearCompletedSends()` reference should be `clearCompletedReceives()`.

Reviewers: Ismael Juma <ismael@juma.me.uk>

Co-authored-by: Zhao Haiyuan <zhaohaiyuan@mobike.com>
2021-04-05 08:11:58 -07:00
Cong Ding 66b0c5c64f
KAFKA-3968: fsync the parent directory of a segment file when the file is created (#10405)
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>
2021-04-02 17:31:56 -07:00
Colin Patrick McCabe ec38dcb72f
MINOR: support ImplicitLinkedHashCollection#sort (#10456)
Support sorting the elements in ImplicitLinkedHashCollection.
This is useful sometimes in unit tests for comparing collections.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-04-01 18:30:53 -07:00
Chia-Ping Tsai 1104848697
KAFKA-12587 Remove KafkaPrincipal#fromString for 3.0 (#10447)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-04-02 09:04:21 +08:00
Ismael Juma 2342ec1d1c
KAFKA-12600: Remove deprecated config value `default` for client config `client.dns.lookup` (#10458)
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>
2021-04-01 07:59:59 -07:00
Satish Duggana 3829df103e
MINOR Replaced File with Path in LogSegmentData. (#10424)
Replaced File with Path in LogSegment Data.

This is a followup of #10173

Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
2021-03-31 10:43:10 -07:00
David Jacot 667578860a
KAFKA-12577; Remove deprecated `ConfigEntry` constructor for 3.0 (#10436)
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>
2021-03-31 11:28:29 +02:00
Ismael Juma a7251ce174
KAFKA-12581: Remove deprecated `Admin.electPreferredLeaders` (#10440)
`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>
2021-03-30 22:04:13 -07:00
Ismael Juma e8754ba7a0
KAFKA-12578: Remove deprecated security classes/methods for 3.0 (#10435)
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>
2021-03-30 22:02:16 -07:00
dengziming 482b79dc25
MINOR: Remove unnecessary suppress (#10434)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-03-31 12:02:46 +08:00
Jason Gustafson d22c49217e
MINOR: Fix newly added client side timeout tests in `KafkaAdminClientTest` (#10398)
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>
2021-03-30 15:45:08 -07:00
John Roesler d02ad2c0e1
KAFKA-12557: Fix hanging KafkaAdminClientTest (#10404)
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>
2021-03-30 11:36:13 -05:00
David Jacot 0018c3c089
KAFKA-12573; Remove deprecated `Metric#value` (#10425)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-03-30 09:28:48 +02:00
Jason Gustafson 3ab9ab6b6f
KAFKA-12434; Admin support for `DescribeProducers` API (#10275)
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>
2021-03-24 20:15:58 -07:00
John Roesler 9bf5c57997
KAFKA-12435: Fix javadoc errors (#10392)
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>
2021-03-24 13:55:27 -05:00
Colin Patrick McCabe 11f0ea3a5e
KAFKA-12432: AdminClient should time out nodes that are never ready (#10281)
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>
2021-03-24 11:10:19 -07:00
Rajini Sivaram e3f60c254c
KAFKA-12479: Batch partition offset requests in ConsumerGroupCommand (#10371)
Reviewers: David Jacot <djacot@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2021-03-23 09:56:56 +00:00
Justine Olshan 3d0b4d910b
MINOR: use new method to get number of topics in DeleteTopicsRequest (#10351)
Reviewers: David Jacot <djacot@confluent.io>
2021-03-23 08:29:17 +01:00
wenbingshen e0cbd0fa66
MINOR: Remove duplicate definition about 'the' from kafka project (#10370)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-03-23 10:44:55 +08:00
John Roesler 2bfe7a9d1e
KAFKA-12514: Fix NPE in SubscriptionState (#10369)
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>
2021-03-21 20:56:16 -05:00
CHUN-HAO TANG 2e2fad747d
MINOR: Remove redundant semicolon (#10358) 2021-03-21 17:43:03 +01:00
dengziming 69eebbf968
KAFKA-12440; ClusterId validation for Vote, BeginQuorum, EndQuorum and FetchSnapshot (#10289)
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>
2021-03-19 10:27:47 -07:00
Ron Dagostino d9bb2ef596
MINOR: The new KIP-500 code should treat cluster ID as a string (#10357)
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>
2021-03-19 09:56:04 -07:00
CHUN-HAO TANG 4abdb59d71
KAFKA-10697: Remove ProduceResponse.responses (#10332)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-03-19 10:04:56 +08:00
Bruno Cadonna 4be0033e62
KAFKA-10357: Add setup method to internal topics (#10317)
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>
2021-03-18 09:52:08 -07:00
Chia-Ping Tsai b5a80e5be6
MINOR: remove redundant null check when testing specified type (#10314)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-03-18 23:39:26 +08:00
Ismael Juma 36c5ad4c6a
MINOR: Add toString to various Kafka Metrics classes (#10330)
This was useful while debugging a JDK 16 test failure, I noticed these
were missing.

Reviewers: David Jacot <david.jacot@gmail.com>
2021-03-16 13:51:44 -07:00
David Mao bf639905f6
KAFKA-12427: Don't update connection idle time for muted connections (#10267)
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
2021-03-16 19:18:28 +00:00
Guozhang Wang 2387d191fc
KAFKA-12352: Make sure all rejoin group and reset state has a reason (#10232)
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>
2021-03-15 09:24:38 -07:00
Colin Patrick McCabe 15620ae54f
MINOR: socket setup max should be 30 seconds #10306 (#10306)
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>
2021-03-11 14:50:43 -08:00
Bruno Cadonna 800d9b5abc
KAFKA-10357: Add validation method for internal topics (#10266)
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>
2021-03-11 09:55:30 -08:00
dengziming 8c9bc9c640
MINOR: Add entityType for metadata record definitions (#10116)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-03-10 13:27:06 +08:00
Lee Dongjin e6f8f5d0ae
MINOR: Remove unused variables, methods, parameters, unthrown exceptions, and fix typos (#9457)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com
2021-03-10 13:21:30 +08:00
Manikumar Reddy 728be21bde
MINOR: Update log level in SaslServerAuthenticator (#10270)
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
2021-03-05 18:00:50 +05:30
Chia-Ping Tsai be1476869f
MINOR: make sure all generated data tests cover all versions (#10078)
Reviewers: David Jacot <djacot@confluent.io>
2021-03-05 00:22:57 +08:00
Chia-Ping Tsai 8205051e90
MINOR: remove FetchResponse.AbortedTransaction and redundant construc… (#9758)
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>
2021-03-04 18:06:50 +08:00
David Jacot 3ef39e1365
MINOR; Clean up LeaderAndIsrResponse construction in `ReplicaManager#becomeLeaderOrFollower` (#10234)
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>
2021-03-04 10:31:35 +01:00
Satish Duggana 0d9a95a7d0
KAFKA-9548 Added SPIs and public classes/interfaces introduced in KIP-405 for tiered storage feature in Kafka. (#10173)
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>
2021-03-03 08:55:13 -08:00
dengziming cfb60064ec
MINOR: Fix null exception in coordinator log (#10250)
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
2021-03-03 12:38:06 +08:00
Jason Gustafson 3708a7c6c1
KAFKA-12369; Implement `ListTransactions` API (#10206)
This patch implements the `ListTransactions` API as documented in KIP-664: https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions.

Reviewers: Tom Bentley <tbentley@redhat.com>, Chia-Ping Tsai <chia7712@gmail.com>
2021-03-02 13:01:35 -08:00
John Roesler a92b986c85
KAFKA-12268: Implement task idling semantics via currentLag API (#10137)
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>
2021-03-02 08:20:47 -06:00
Chia-Ping Tsai 652cca377a
MINOR: correct the error message of validating uint32 (#10193)
Reviewers: Tom Bentley <tbentley@redhat.com>, David Jacot <djacot@confluent.io>
2021-03-02 17:51:47 +08:00
dengziming c9832aabae
DOCS: Update protocol doc for missing data type (#10162)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-02-26 15:04:25 +08:00
Jason Gustafson 74dfe80bb8
KAFKA-12365; Disable APIs not supported by KIP-500 broker/controller (#10194)
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>
2021-02-25 19:38:21 -08:00
Davor Poldrugo 7f90eda047
KAFKA-8562; SaslChannelBuilder - Avoid (reverse) DNS lookup while building SslTransportLayer (#10059)
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>
2021-02-25 17:02:40 -08:00
Jason Gustafson 3f09fb97b6
KAFKA-12267; Implement `DescribeTransactions` API (#10183)
This patch implements the `DescribeTransactions` API as documented in KIP-664: https://cwiki.apache.org/confluence/display/KAFKA/KIP-664%3A+Provide+tooling+to+detect+and+abort+hanging+transactions. This is only the server-side implementation and does not contain the `Admin` API.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-02-24 12:50:18 -08:00