Commit Graph

9934 Commits

Author SHA1 Message Date
Bruno Cadonna 2d2926cf81
MINOR: Make link in quickstart dynamic (#12057)
Reviewer: Matthias J. Sax <mjsax@apache.org>
2022-04-15 16:22:16 +02:00
Xiaobing Fang f4e3ccd81a
MINOR: update comment in LocalLog.replaceSegments() (#12054)
Reviewers: Luke Chen <showuon@gmail.com>
2022-04-15 21:00:11 +08:00
RivenSun 19b5853565
MINOR: Improve the description of principal under different mechanisms of sasl (#11947)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2022-04-15 11:09:20 +02:00
dengziming 77cd827104
MINOR: Move some TopicCommand and ConfigCommand integration tests to unit tests (#12024)
Move some TopicCommand and ConfigCommand integration tests to unit tests to speed up the tests

Reviewers: Luke Chen <showuon@gmail.com>
2022-04-15 16:35:52 +08:00
Bruno Cadonna b7f2d979d6
[MINOR] Update upgrade documentation for 3.2 (#12055)
Reviewer: Bruno Cadonna <cadonna@apache.org>
2022-04-15 10:21:05 +02:00
RivenSun cf5e714a8b
MINOR: ignore unused configuration when ConsumerCoordinator is not constructed (#12041)
Following PR #11940, ignore unused config when ConsumerCoordinator is not constructed.

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2022-04-14 17:30:43 -07:00
RivenSun f49cff412d
MINOR: Remove redundant conditional judgments in Selector.clear() (#12048)
Condition 'sendFailed' is always 'false' when reached.

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2022-04-14 17:25:37 -07:00
Bruno Cadonna a3a4323a5a
MINOR: Update LICENSE-binary (#12051)
Updates the license file.

Reviewer: Bill Bejeck <bbejeck@apache.org>
2022-04-14 22:23:52 +02:00
David Arthur 55ff5d3603
KAFKA-13823 Feature flag changes from KIP-778 (#12036)
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>
2022-04-14 10:04:32 -07:00
Lee Dongjin 01e4ceba52
KAFKA-12613: Fix inconsistent validation logic between KafkaConfig and LogConfig (#10472)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2022-04-14 11:58:57 +02:00
dengziming 87aa8259dd KAFKA-13743: Prevent topics with conflicting metrics names from being created in KRaft mode #11910
In ZK mode, the topic "foo_bar" will conflict with "foo.bar" because of limitations in metric
names. We should implement this in KRaft mode.  This PR also changes TopicCommandIntegrationTest to
support KRaft mode.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2022-04-13 11:59:29 -07:00
Jason Gustafson f97646488c
KAFKA-13651; Add audit logging to `StandardAuthorizer` (#12031)
This patch adds audit support through the kafka.authorizer.logger logger to StandardAuthorizer. It
follows the same conventions as AclAuthorizer with a similarly formatted log message. When
logIfAllowed is set in the Action, then the log message is at DEBUG level; otherwise, we log at
trace. When logIfDenied is set, then the log message is at INFO level; otherwise, we again log at
TRACE.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2022-04-13 10:33:15 -07:00
José Armando García Sancio a6d86b9998
MINOR: Verify stopReplica if broker epoch not stale (#12040)
Verify that ReplicaManager.stopReplica is called if the stop replica
request doesn't result in a stale broker epoch error.

Reviewers: Mickael Maison <mimaison@users.noreply.github.com>
2022-04-13 09:05:27 -07:00
Hao Li c93b717836
KAFKA-13542: Add rebalance reason in Kafka Streams (#12018)
Reviewers: Bruno Cadonna <bruno@confluent.io>, David Jacot <djacot@confluent.io>
2022-04-13 13:49:31 +02:00
David Jacot 4eeb707107
KAFKA-13828; Ensure reasons sent by the consumer are small (#12043)
This PR reworks the reasons used in the ConsumerCoordinator to ensure that they remain reasonably short.

Reviewers: Bruno Cadonna <bruno@confluent.io>
2022-04-13 13:42:27 +02:00
Chris Egerton b9fc893546
MINOR: Correct Connect docs on connector/task states (#11914)
The `DESTROYED` state is represented internally as a tombstone record when running in distributed mode and by the removal of the connector/task from the in-memory status map when running in standalone mode. As a result, it will never appear to users of the REST API, and we should remove mention of it from our docs so that developers creating tooling against the REST API don't write unnecessary logic to account for that state.

Reviewers: Mickael Maison <mickael.maison@gmail.com>
2022-04-13 12:51:34 +02:00
RivenSun 1df232c839
MINOR: Supplement the description of `Valid Values` in the documentation of `compression.type` (#11985)
Because a validator is added to ProducerConfig.COMPRESSION_TYPE_CONFIG and KafkaConfig.CompressionTypeProp, the corresponding testCase is improved to verify whether the wrong value of compression.type will throw a ConfigException.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
2022-04-12 21:24:57 -07:00
Jorge Esteban Quilcate Otoya 0d518aaed1
MINOR: Fix SessionStore#fetchSession parameter names (#11999)
Fixes a small copy/paste error from #10390 that changed the parameter names
for fetchSession from the singular session form (eg `startTime`) to the range
form (eg `earliestSessionStartTime`).

Reviewers: John Roesler <vvcephei@apache.org>
2022-04-11 16:17:01 -05:00
José Armando García Sancio 317fff9bb0
MINOR: Re-use counter in mocking of LogSegment.size (#12021)
When migrating from Easymock to Mockito, the mockito implemetnation
didn't have the same semantic as the Easymock implementation.

Without this fix the mocking of LogSegment.size() always returns 0 because
a new AtomicInteger was getting created for each invocation of
LogSegment.size()

Reviewers: Mickael Maison <mimaison@users.noreply.github.com>
2022-04-11 13:37:47 -07:00
RivenSun 4ad439c56d
MINOR: Change the log output information in the KafkaConsumer assign method (#12026)
Reviewers: Luke Chen <showuon@gmail.com>
2022-04-11 10:06:56 +08:00
David Jacot 6e9cd0c7f5
MINOR: A few code cleanups in DynamicBrokerConfig (#12015)
Reviewers: Luke Chen <showuon@gmail.com>
2022-04-09 11:42:42 +02:00
Xiaoyue Xue 9596c7b9cf
KAFKA-13794: Follow up to fix producer batch comparator (#12006)
In comparator, objects that are not equal need to have a stable order otherwise, binary search may not find the objects. Improve the producer batch comparator
2022-04-09 10:58:16 +08:00
Alok Nikhil 7a5f0cfaef
MINOR: Fix DescribeLogDirs API error handling for older API versions (#12017)
With KAFKA-13527 / KIP-784 we introduced a new top-level error code for
the DescribeLogDirs API for versions 3 and above. However, the change
regressed the error handling for versions less than 3 since the response
converter fails to write the non-zero error code out (rightly) for
versions lower than 3 and drops the response to the client which
eventually times out instead of receiving an empty log dirs response and
processing that as a Cluster Auth failure.

With this change, the API conditionally propagates the error code out to
the client if the request API version is 3 and above. This keeps the
semantics of the error handling the same for all versions and restores
the behavior for older versions.

See current behavior in the broker log:
```bash
ERROR] 2022-04-08 01:22:56,406 [data-plane-kafka-request-handler-10] kafka.server.KafkaApis - [KafkaApi-0] Unexpected error handling request RequestHeader(apiKey=DESCRIBE_LOG_DIRS, apiVersion=0, clientId=sarama, correlationId=1) -- DescribeLogDirsRequestData(topics=null)
org.apache.kafka.common.errors.UnsupportedVersionException: Attempted to write a non-default errorCode at version 0
[ERROR] 2022-04-08 01:22:56,407 [data-plane-kafka-request-handler-10] kafka.server.KafkaRequestHandler - [Kafka Request Handler 10 on Broker 0], Exception when handling request
org.apache.kafka.common.errors.UnsupportedVersionException: Attempted to write a non-default errorCode at version 0
```

Reviewers: Ismael Juma <ismael@juma.me.uk>
2022-04-08 12:54:09 -07:00
Ismael Juma ce4f2ad606
MINOR: Fix support for custom commit ids in the build (#12014)
This regressed in ca375d8004 due to a typo. We need tests
for our builds. :)

I verified that passing the commitId via `-PcommitId=123`
works correctly.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2022-04-08 08:55:52 -07:00
bozhao12 a2149c4178
MINOR: Fix method javadoc and typo in comments (#12007)
Reviewers: Luke Chen <showuon@gmail.com>
2022-04-07 18:16:12 +08:00
Xavier Léauté 60c0916bfd
KAFKA-13801: Kafka server does not respect MetricsReporter contract for dynamically configured reporters (#11998)
MetricsReporter.contextChange contract states the method should always be called first before MetricsReporter.init is called. This is done correctly for reporters enabled by default (e.g. JmxReporter) but not for metrics reporters configured dynamically.

This fixes the call ordering for dynamically configured metrics reporter and updates tests to enforce ordering.

Reviewers: David Jacot <djacot@confluent.io>
2022-04-07 10:13:15 +02:00
Xiaoyue Xue e7cfbad04f
MINOR: Clean up for TransactionManager and RecordAccumulator (#11979)
Reviewers: Luke Chen <showuon@gmail.com>
2022-04-07 14:24:23 +08:00
sciclon2 92305c2cf2
KAFKA-13687: Limiting the amount of bytes to be read in a segment logs (#11842)
This PR allows to limit the output batches while they are inspected via the kafka-dump-log.sh script.

The idea is to take samples from the logsegments without affecting a production cluster as the current script will read the whole files, this could create issues related to performance.

Please see the KIP-824

Reviewers: Jun Rao <junrao@gmail.com>
2022-04-06 08:48:43 -07:00
Anastasia Vela 206ad4d2b5
MINOR: Fix flaky testIdleConnection() test (#11996)
The test expects that the connection becomes idle before the mock time is moved forward, but the processor thread runs concurrently and may run some activity on the connection after the mock time is moved forward, thus the connection never expires.

The solution is to wait until the message is received on the socket, and only then wait until the connection is unmuted (it's not enough to wait for unmuted without waiting for message being received on the socket, because the channel might have not been muted yet).

Reviewers: David Jacot <djacot@confluent.io>
2022-04-06 16:35:25 +02:00
Ismael Juma b964c07a68
MINOR: Upgrade build and test dependencies (#11984)
* gradle: 7.3.3 -> 7.4.2
  Configuration cache improvements and several other improvements.
  https://docs.gradle.org/7.4.2/release-notes.html
* dependencycheck gradle plugin: 6.5.3 -> 7.0.3
  Minor fixes.
* spotbugs gradle plugin: 5.0.5 -> 5.0.6
  Minor fixes.
  https://github.com/spotbugs/spotbugs-gradle-plugin/releases/tag/5.0.6
* jmh: 1.34 -> 1.35
  Fixes and profiler improvements.
  https://mail.openjdk.java.net/pipermail/jmh-dev/2022-March/003422.html
* jqwik: 1.6.3 -> 1.6.5
  Various tweaks and some breaking changes that don't seem to affect us.
  https://github.com/jlink/jqwik/releases/tag/1.6.4
  https://github.com/jlink/jqwik/releases/tag/1.6.5
* mockito: 4.3.1 -> 4.4.0
  Add feature to verify static methods calls in order and minor fixes/improvements.
  https://github.com/mockito/mockito/releases/tag/v4.4.0

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2022-04-06 07:27:03 -07:00
Tom Bentley 9b8f6289be
MINOR: Mention KAFKA-13748 in release notes (#11994)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Bruno Cadonna <bruno@confluent.io>
2022-04-06 10:29:44 +01:00
Xavier Léauté 29a6979c54
KAFKA-6204 KAFKA-7402 ProducerInterceptor should implement AutoCloseable (#11997)
As part of KIP-376 we had ConsumerInterceptor implement AutoCloseable
but forgot to do the same for ProducerInterceptor. This fixes the
inconsistency and also addresses KAFKA-6204 at the same time.

Reviewers: John Roesler <vvcephei@apache.org>
2022-04-05 21:23:31 -05:00
Chris Egerton 88e5f133b5
KAFKA-13763: Improve unit testing coverage and flexibility for IncrementalCooperativeAssignor (#11974)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2022-04-05 21:42:08 +02:00
Xiaoyue Xue f0a2b62b0e
KAFKA-13794; Fix comparator of `inflightBatchesBySequence` in `TransactionManager` (#11991)
Fixes a bug in the comparator used to sort producer inflight batches for a topic partition. This can cause batches in the map `inflightBatchesBySequence` to be removed incorrectly: i.e. one batch may be removed by another batch with the same sequence number. This leads to an `IllegalStateException` when the inflight request finally returns. This patch fixes the comparator to check equality of the `ProducerBatch` instances if the base sequences match.

Reviewers: Jason Gustafson <jason@confluent.io>
2022-04-05 10:03:33 -07:00
bozhao12 4218fc61fe
KAFKA-13778: Fetch from follower should never run the preferred read replica selection (#11965)
The current preferred read replica selection logic relies on `partition.leaderReplicaIdOpt` to determine if the selection must be run. The issue is that `partition.leaderReplicaIdOpt` is defined for both the leader and the followers thus the logic is ran all the time. The impact is not too bad as the leader is selected most of the time when the logic is ran by the follower and the leader is filtered out. However there are cases where the selection on a follower could redirect the consumer to another follower under certain rare conditions. For instance with the `RackAwareReplicaSelector `, the follower must have stale replica states from a previous leadership and must have other followers in the same rack for instance. Other implementation of the selection logic could be more impacted.

This patch ensures that the preferred read replica selection is only ran by the leader.

Reviewers: David Jacot <djacot@confluent.io>
2022-04-05 18:56:23 +02:00
Jason Gustafson 3ceedac79e
KAFKA-13782; Ensure correct partition added to txn after abort on full batch (#11995)
Fixes a regression introduced in https://github.com/apache/kafka/pull/11452. Following [KIP-480](https://cwiki.apache.org/confluence/display/KAFKA/KIP-480%3A+Sticky+Partitioner), the `Partitioner` will receive a callback when a batch has been completed so that it can choose another partition. Because of this, we have to wait until the batch has been successfully appended to the accumulator before adding the partition in `TransactionManager.maybeAddPartition`. This is still safe because the `Sender` cannot dequeue a batch from the accumulator until it has been added to the transaction successfully.

Reviewers: Artem Livshits <84364232+artemlivshits@users.noreply.github.com>, David Jacot <djacot@confluent.io>,  Tom Bentley <tbentley@redhat.com>
2022-04-05 09:48:21 -07:00
yun-yun 481cc13a13
KAFKA-13791: Fix potential race condition in FetchResponse#`fetchData` and `forgottenTopics` (#11981)
Fix FetchResponse#`fetchData` and `forgottenTopics`: Assignment of lazy-initialized members should be the last step with double-checked locking

Reviewers: Luke Chen <showuon@gmail.com>
2022-04-05 15:27:32 +08:00
RivenSun 74909e000a
MINOR: Fix wrong configuration in Adding and Removing Listeners docs (#11992)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2022-04-04 17:27:34 +02:00
Mickael Maison dd4afdeb6d
MINOR: Doc updates for Kafka 3.0.1 (#11906)
Reviewers: David Jacot <djacot@confluent.io>
2022-04-04 14:43:31 +02:00
Anastasia Vela ae45c59e61
MINOR: Fix flaky testClientDisconnectionUpdatesRequestMetrics() (#11987)
Reviewers: David Jacot <djacot@confluent.io>
2022-04-04 09:10:33 +02:00
bozhao12 02a465b090
MINOR: fix typo in FetchRequest.json (#11988)
Reviewers: David Jacot <djacot@confluent.io>
2022-04-04 09:07:58 +02:00
Colin Patrick McCabe 62ea4c46a9
KAFKA-13749: CreateTopics in KRaft must return configs (#11941)
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>
2022-04-01 10:50:25 -07:00
RivenSun 1bdd35d8d8
KAFKA-13786: Add a note in`control.plane.listener.name` doc (#11978)
Add a note in `control.plane.listener.name` doc to mention the value can't be identical with `inter.broker.listener.name`.

Reviewers: Luke Chen <showuon@gmail.com>
2022-04-01 16:23:29 +08:00
dengziming 502f2caca4
MINOR: Remove some unused codes (#11935)
`validateChars` and `BaseEnum` are used in old version of clients. Remove them.

Reviewers: Luke Chen <showuon@gmail.com>
2022-04-01 11:39:50 +08:00
Yang Yu eefdf9d6a7
KAFKA-12875: Change Log layer segment map mutations to avoid absence of active segment (#11950)
Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
2022-03-31 10:56:07 -07:00
Hao Li 6b2a0bcf8c
KAFKA-13785: add processor metadata to be committed with offset (#11829)
Part of KIP-825

Reviewers: Matthias J. Sax <matthias@confluent.io>
2022-03-31 09:48:21 -07:00
Bounkong Khamphousone 3c279b63fa
fix: make sliding window works without grace period (#kafka-13739) (#11928)
Fix upperbound for sliding window, making it compatible with no grace period (kafka-13739)

Added unit test for early sliding window and "normal" sliding window for both events within one time difference (small input) and above window time difference (large input).

Fixing this window interval may slightly change stream behavior but probability to happen is extremely slow and may not have a huge impact on the result given.

Reviewers Leah Thomas <lthomas@confluent.io>, Bill Bejeck <bbejeck@apache.org>
2022-03-31 10:05:53 -04:00
Yu 430f9c9901
KAFKA-13772: Partitions are not correctly re-partitioned when the fetcher thread pool is resized (#11953)
Partitions are assigned to fetcher threads based on their hash modulo the number of fetcher threads. When we resize the fetcher thread pool, we basically re-distribute all the partitions based on the new fetcher thread pool size. The issue is that the logic that resizes the fetcher thread pool updates the `fetcherThreadMap` while iterating over it. The `Map` does not give any guarantee in this case - especially when the underlying map is re-hashed - and that led to not iterating over all the fetcher threads during the process and thus in leaving some partitions in the wrong fetcher threads.

Reviewers: Luke Chen <showuon@gmail.com>, David Jacot <djacot@confluent.io>
2022-03-31 14:45:59 +02:00
David Jacot ce7788aada
KAFKA-13783; Remove reason prefixing in JoinGroupRequest and LeaveGroupRequest (#11971)
KIP-800 introduced a mechanism to pass a reason in the join group request and in the leave group request. A default reason is used unless one is provided by the user. In this case, the custom reason is prefixed by the default one.

When we tried to used this in Kafka Streams, we noted a significant degradation of the performances, see https://github.com/apache/kafka/pull/11873. It is not clear wether the prefixing is the root cause of the issue or not. To be on the safe side, I think that we should remove the prefixing. It does not bring much anyway as we are still able to distinguish a custom reason from the default one on the broker side.

This patch removes prefixing the user provided reasons. So if a the user provides a reason, the reason is used directly. If the reason is empty or null, the default reason is used.

Reviewers: Luke Chen <showuon@gmail.com>, <jeff.kim@confluent.io>, Hao Li <hli@confluent.io>
2022-03-31 14:31:31 +02:00
dengziming 669a49063d
MINOR: Fix an uncompatible bug in GetOffsetShell (#11936)
In KIP-815 we replaced KafkaConsumer with AdminClient in GetOffsetShell. In the previous implementation, partitions were just ignored if there is no offset for them, however, we will print -1 instead now, This PR fix this inconsistency.

Reviewers: David Jacot <djacot@confluent.io>, Luke Chen <showuon@gmail.com>
2022-03-31 10:34:39 +08:00