Commit Graph

10811 Commits

Author SHA1 Message Date
Lucas Brutschy cc582897bf
KAFKA-14299: Fix incorrect pauses in separate state restoration (#12743)
The original code path paused the main consumer for
all tasks before entering the restoration section
of the code, and then resumed all after restoration
has finished.

In the new state updater part of the code, tasks that
do not require restoration skip the restoration completely.
They remain with the TaskManger and are never transferred
to the StateUpdater, and thus are never resumed.

This change makes sure that tasks that remain with the
TaskManager are not paused.

Reviewers: Guozhang Wang <wangguoz@gmail.com>, Bruno Cadonna <cadonna@apache.org>
2022-10-18 12:16:44 +02:00
liuzhuang2017 109b74c590
MINOR: Update the README file in examples. (#12272)
Modify Intellij to IntelliJ IDEA, and mention KRaft in example docs.

Reviewers: Divij Vaidya <divijvaidya13@gmail.com>, Kvicii <kvicii.yu@gmail.com>, dengziming <dengziming1993@gmail.com>
2022-10-18 13:03:52 +08:00
Niket 3e7eddecd6
MINOR: Address flakiness in `KRaftClusterTest::testDescribeQuorumRequestToBrokers` (#12738)
We have seen some errors such as the following:
```
org.opentest4j.AssertionFailedError: expected: not equal but was: <OptionalLong.empty>
Stacktrace
org.opentest4j.AssertionFailedError: expected: not equal but was: <OptionalLong.empty>
	at org.junit.jupiter.api.AssertionUtils.fail(AssertionUtils.java:39)
	at org.junit.jupiter.api.AssertNotEquals.failEqual(AssertNotEquals.java:276)
	at org.junit.jupiter.api.AssertNotEquals.assertNotEquals(AssertNotEquals.java:265)
	at org.junit.jupiter.api.AssertNotEquals.assertNotEquals(AssertNotEquals.java:260)
	at org.junit.jupiter.api.Assertions.assertNotEquals(Assertions.java:2815)
	at kafka.server.KRaftClusterTest.$anonfun$testDescribeQuorumRequestToBrokers$5(KRaftClusterTest.scala:818)
	at java.util.ArrayList.forEach(ArrayList.java:1259)
	at kafka.server.KRaftClusterTest.testDescribeQuorumRequestToBrokers(KRaftClusterTest.scala:814)
```
The patch changes some of the assertions to wait longer for the condition to be satisfied.

Reviewers: Jason Gustafson <jason@confluent.io>
2022-10-17 16:16:26 -07:00
José Armando García Sancio e8e815b1b6
MINOR; Update documentation for printing dependencies (#12610)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2022-10-17 11:27:49 -07:00
Ismael Juma de8c9ea04c
MINOR: Include TLS version in transport layer debug log (#12751)
This was helpful when debugging an issue recently.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Divij Vaidya <divijvaidya13@gmail.com>
2022-10-17 08:26:23 -07:00
Philip Nee 7d67ddce22
MINOR: typo in KafkaChannel (#12757)
Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>
2022-10-16 20:08:53 +08:00
Guozhang Wang 55a3a95b7a
Kafka Streams Threading P3: TaskManager Impl (#12754)
0. Add name to task executors.
1. DefaultTaskManager implementation, for interacting with the TaskExecutors and support add/remove/lock APIs.
2. Related unit tests.
2022-10-14 16:10:57 -07:00
Guozhang Wang dfb5929665
Kafka Streams Threading P2: Skeleton TaskExecutor Impl (#12744)
0. Address comments from P1.
1. Add the DefaultTaskExecutor implementation class.
2. Related DefaultTaskExecutorTest.

Pending in future PRs: a) exception handling, primarily to send them to polling thread, b) light-weight task flushing procedure.
2022-10-14 15:32:48 -07:00
Luke Chen 78b4ba7d85
MINOR; Add kraft controller log level in log4j prop (#12707)
Add KRaft controller log level entry in log4j.properties, otherwise, the controller.log is empty in KRaft mode.

Reviewers: José Armando García Sancio <jsancio@apache.org>, Ismael Juma <ismael@juma.me.uk>
2022-10-14 11:27:31 -07:00
Ismael Juma f1e0fa91ab
MINOR; Delete config/kraft/README.me (#12688)
We have since updated the main documentation to cover KRaft and the
duplication is not helpful. Furthermore, this file ships with the distribution,
which means we cannot update it without a release.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2022-10-13 19:33:40 -07:00
David Jacot 5cff8f67b0
KAFKA-14296; Partition leaders are not demoted during kraft controlled shutdown (#12741)
When the `BrokerServer` starts its shutting down process, it transitions to `SHUTTING_DOWN` and sets `isShuttingDown` to `true`. With this state change, the follower state changes are short-cutted. This means that a broker which was serving as leader would remain acting as a leader until controlled shutdown completes. Instead, we want the leader and ISR state to be updated so that requests will return NOT_LEADER and the client can find the new leader.

We missed this case while implementing https://github.com/apache/kafka/pull/12187.

This patch fixes the issue and updates an existing test to ensure that `isShuttingDown` has not effect. We should consider adding integration tests for this as well. We can do this separately.

Reviewers: Ismael Juma <ismael@juma.me.uk>, José Armando García Sancio <jsancio@users.noreply.github.com>, Jason Gustafson <jason@confluent.io>
2022-10-13 16:59:19 -07:00
Alyssa Huang 0cb1d61413
KAFKA-14292; Fix KRaft controlled shutdown delay (#12736)
The `controlledShutDownOffset` is defined as the "offset at which the broker should complete its controlled shutdown, or -1 if the broker is not performing a controlled shutdown". The controller sets this offset to a non-negative integer on receiving a heartbeat from a broker that's in controlled shutdown state. Currently, this offset is being updated and bumped every single time a broker in controlled shutdown mode send a heartbeat, delaying when controlled shutdown can actually complete for the broker. We should only update the offset when it was previously set to -1 to allow controlled shutdown to complete.

Reviewers: Luke Chen <showuon@gmail.com>, Jason Gustafson <jason@confluent.io>
2022-10-13 13:29:45 -07:00
Bruno Cadonna 484f85ff53
HOTFIX: Revert "KAFKA-12960: Enforcing strict retention time for WindowStore and Sess… (#11211)" (#12745)
This reverts commit 07c1002489 which broke trunk.

Reviewers: David Jacot <djacot@confluent.io>, Bill Bejeck <bbejeck@apache.org>
2022-10-13 13:27:19 -07:00
Colin Patrick McCabe dac81161db
MINOR; Introduce ImageWriter and ImageWriterOptions (#12715)
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>
2022-10-13 09:56:19 -07:00
Chris Egerton 18e60cb000
KAFKA-12497: Skip periodic offset commits for failed source tasks (#10528)
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>
2022-10-13 10:15:42 -04:00
vamossagar12 07c1002489
KAFKA-12960: Enforcing strict retention time for WindowStore and Sess… (#11211)
WindowedStore and SessionStore do not implement a strict retention time in general. We should consider to make retention time strict: even if we still have some record in the store (due to the segmented implementation), we might want to filter expired records on-read. This might benefit PAPI users.

This PR, adds the filtering behaviour in the Metered store so that, it gets automatically applied for cases when a custom state store is implemented

Reviewer: Luke Chen <showuon@gmail.com>, A. Sophie Blee-Goldman <sophie@confluent.io>, Matthias J. Sax <mjsax@apache.org>
2022-10-13 09:39:58 +08:00
Guozhang Wang 69059b5f28
Kafka Streams Threading P1: Add Interface for new TaskManager and TaskExecutor (#12737)
The interfaces (and their future impls) are added under the processor/internals/tasks package, to distinguish with the existing old classes:

1. TaskExecutor is the interface for a processor thread. It takes at most one task to process at a given time from the task manager. When being asked from the task manager to un-assign the current processing task, it will stop processing and give the task back to task manager.
2. TaskManager schedules all the active tasks to assign to TaskExecutors. Specifically: 1) when a task executor ask it for an unassigned task to process (assignNextTask), it will return the available task based on its scheduling algorithm. 2) when the task manager decides to commit (all) tasks, or when a rebalance event requires it to modify the maintained active tasks (via onAssignment), it will lock all the tasks that are going to be closed / committed, asking the TaskExecutor to give them back if they were being processed at the moment.

Reviewers: John Roesler <vvcephei@apache.org>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2022-10-12 16:33:13 -07:00
Jason Gustafson 4b30ed8e47
MINOR: Fix incorrect example in feature command help (#12732)
When using `kafka-features.sh` with the `--feature` parameter, we expect a numeric feature level (e.g. `metadata.version=5`). The help example suggests that we can also use a descriptive version string for `metadata.version` such as `3.3-IV3`, which doesn't work.

Reviewers: David Jacot <djacot@confluent.io>
2022-10-12 11:04:55 -07:00
Alexandre Garnier 62914129c7
KAFKA-14099 - Fix request logging in connect (#12434)
Reviewers: Chris Egerton <chrise@aiven.io>
2022-10-12 10:28:55 -04:00
Ramesh a6b60e7cf3
KAFKA-12965 - Graceful clean up of task error metrics (#10910)
Reviewers: Chris Egerton <chrise@aiven.io>
2022-10-12 09:59:52 -04:00
Huilin Shi d834947ae7
KAFKA-14225; Fix deadlock caused by lazy val exemptSensor (#12634)
There is a chance to cause deadlock when multiple threads access ClientRequestQuotaManager. In the version Scala 2.12, the lazy val initialization is under the object lock. The deadlock could happen in the following condition:

In thread a, when ClientRequestQuotaManager.exemptSensor is being initialized, it has acquired the object lock and enters the the actual initialization block. The initialization of 'exemptSensor' requires another lock private val lock = new ReentrantReadWriteLock() and it is waiting for this lock.

In thread b, at the same time, ClientQuotaManager.updateQuota() is called and it has already acquired ReentrantReadWriteLock lock by calling lock.writeLock().lock(). And then it executes info(). If this is the first time accessing Logging.logger, which is also a lazy val, it need to wait for the object lock.

The deadlock happens.

Since the lazy val initialization is under the object lock, we should avoid using lazy val if the initialization function holds another lock to prevent holding two locks at the same time which is prone for deadlock. Change to create exemptSensor during ClientRequestQuotaManager initialization with an expiration time of Long.MaxValue to prevent expiration if request quota is not enabled at that time.

Reviewers: Jason Gustafson <jason@confluent.io>
2022-10-11 10:30:07 -07:00
Niket 98a3dcb477
KAFKA-14275; KRaft Controllers should crash after failing to apply any metadata record (#12709)
Make all faults in metadata processing on standby controllers be fatal. This is the same behavior-wise as the active controller. This prevents a standby controller from eventually becoming active with incomplete state.

Reviewers: Colin Patrick McCabe <cmccabe@apache.org>, Jason Gustafson <jason@confluent.io>
2022-10-11 09:46:42 -07:00
José Armando García Sancio 24cb2d2f95
MINOR; Some sections are missing in the TOC (#12726)
Reviewers: Luke Chen <showuon@gmail.com>
2022-10-11 09:44:39 -07:00
Bruno Cadonna daae2a189d
HOTFIX: Only update input partitions of standby tasks if they really changed (#12730)
Updating the input partitions of tasks also updates the mapping from
source nodes to input topics in the processor topology within the task.
The mapping is updated with the topics from the topology metadata.
The topology metadata does not prefix intermediate internal topics with
the application ID. Thus, if a standby task has input partitions from an
intermediate internal topic the update of the mapping in the processor
topology leads to an invalid topology exception during recycling of a
standby task to an active task when the input queues are created. This
is because the input topics in the processor topology and the input
partitions of the task do not match because the former miss the
application ID prefix.

The added verification to only update input partitions of standby tasks
if they really changed avoids the invalid topology exception if the
standby task only has input partitions from intermediate internal
topics since they should never change. If the standby task has input
partitions from intermediate internal topics and external topics
subscribed to via a regex pattern, the invalid topology exception
might still be triggered.

Reviewers: Guozhang Wang <guozhang@apache.org>, John Roesler <vvcephei@apache.org>
2022-10-11 18:13:58 +02:00
Colin Patrick McCabe 1c07095cbd
MINOR: fix indentation and add builders in some KRaft tests (#12720)
Add builders for LocalLogManagerTestEnv and QuorumControllerTestEnv, since the constructor
overloads were starting to get unwieldy.

Make indentation more consistent in QuorumControllerTest. Take advantage of the fact that you can
initialize multiple resources in a Java try-with-resources block to avoid excessive indentation in a few
cases.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2022-10-07 13:53:41 -07:00
José Armando García Sancio e8d32563f3
MINOR; Fix error message when validating KRaft config (#12717)
The error message reported when advertised.listeners is used in
controller only is confusing. When the KRaft server is configured to
controller only the following must be true:

1. `advertised.listeners` is not set
2. `listeners` contains a listener for every name in `controller.listener.names`
3. `controller.listener.names` contains a name for every listener in `listeners`

Reviewers: Jason Gustafson <jason@confluent.io>, Igor Soarez <i@soarez.me>
2022-10-07 10:40:55 -07:00
Matthew de Detrich ba0f105e2f
MINOR: Use LEADER_URL variable in ConnectorsResourceTest (#12722)
Reviewers: Chris Egerton <chrise@aiven.io>
2022-10-07 10:18:31 -04:00
Matthew de Detrich de05665876
KAFKA-14283: Fix connector creation Auth tests (#12721)
Reviewers: Sagar Rao <sagarmeansocean@gmail.com>, Chris Egerton <chrise@aiven.io>
2022-10-07 10:14:15 -04:00
Luke Chen 42b311ed44
MINIOR: some typos in javadoc (#12723)
Reviewer: David Jacot <djacot@confluent.io>
2022-10-07 19:59:18 +08:00
Walker Carlson cbdcd20ac1
MINOR: Include all hosts in metadata for topology (#12594)
When building streams metadata we want to build even if the host is empty as it is a common way to find the other host addresses

Reviewers: John Roesler <vvcephei@apache.org>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2022-10-06 21:33:00 -07:00
Vicky Papavasileiou 1cb7736de1
KAFKA-14209 : Integration tests 3/3 (#12676)
Tests for 21a15c6b1f
Implements KIP-862: https://cwiki.apache.org/confluence/x/WSf1D

Reviewer: John Roesler <vvcephei@apache.org>
2022-10-06 19:07:34 -05:00
Andras Katona 2762ce4694
MINOR: Add shebang to gradlewAll (#11373)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
2022-10-06 16:18:12 +02:00
Mickael Maison a7a026cabb
MINOR: Fix closing code tag in producer config docs (#12718)
Reviewers: David Jacot <djacot@confluent.io>, Luke Chen <showuon@gmail.com>
2022-10-06 14:33:28 +02:00
Mickael Maison 335d70ede7
MINOR: Fix link to Connect OpenAPI spec (#12713)
Reviewers: José Armando García Sancio <jsancio@confluent.io>
2022-10-06 10:41:17 +02:00
Vicky Papavasileiou 21a15c6b1f
KAFKA-14209 : Rewrite self joins to use single state store 2/3 (#12644)
Implements KIP-862: https://cwiki.apache.org/confluence/x/WSf1D

Reviewers: Guozhang Wang <guozhang@apache.org>,  Austin Heyne <aheyne>, John Roesler <vvcephei@apache.org>
2022-10-05 07:36:04 -05:00
Jason Gustafson c5745d2845
MINOR: Add initial property tests for StandardAuthorizer (#12703)
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>
2022-10-04 16:31:43 -07:00
José Armando García Sancio 5c5dcb7a96
MINOR; Use 3.3.1 release for system test (#12714)
The following files are available in https://s3-us-west-2.amazonaws.com/kafka-packages/:

kafka-streams-3.3.1-test.jar
kafka_2.12-3.3.1.tgz
kafka_2.13-3.3.1.tgz

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2022-10-04 16:19:24 -07:00
Philip Nee 997dfa950e
MINOR: Fix typo in selector documentation (#12710)
Reviewers: David Jacot <djacot@confluent.io>
2022-10-04 13:05:26 +02:00
Oliver Eikemeier fb0ae71f33
KAFKA-14270: Fix generated Kafka Streams version file name (#12700)
Kafka Streams expects a version resource at /kafka/kafka-streams-version.properties which is read by ClientMetrics. Unfortunately, the name of the generated file was wrong and thus was not copied as a resource.

Reviewer: Bruno Cadonna <cadonna@apache.org>
2022-10-04 11:58:07 +02:00
Philip Nee c3690a3b4a
KAFKA-14247; Define event handler interface and events (#12663)
Add some initial interfaces to kick off https://cwiki.apache.org/confluence/display/KAFKA/Proposal%3A+Consumer+Threading+Model+Refactor. We introduce an `EventHandler` interface and a new consumer implementation which demonstrates how it will be used. Subsequent PRs will continue to flesh out the implementation.

Reviewers: Guozhang Wang wangguoz@gmail.com, Jason Gustafson <jason@confluent.io>
2022-10-03 09:36:40 -07:00
David Arthur c1f23b6c9a
MINOR: Fix delegation token system test (#12693)
KIP-373 added a "token requester" field to the output of kafka-delegation-tokens.sh. The system test was failing since it was not expecting this new field. This patch adds support for this field and improves the error output if we can't parse.

Reviewers: José Armando García Sancio <jsancio@apache.org>, Manikumar Reddy <manikumar.reddy@gmail.com>
2022-10-01 19:22:46 -07:00
LinShunKang 496ae054c2
Fix ByteBufferSerializer#serialize(String, ByteBuffer) not roundtrip input with ByteBufferDeserializer#deserialize(String, byte[]) (#12704)
Reviewers: Guozhang Wang <wangguoz@gmail.com>
2022-09-30 06:45:18 -07:00
Christo Lolov 3a9efc77b2
KAFKA-14133: Replace EasyMock with Mockito in streams tests (#12527)
Batch 4 of the tests detailed in https://issues.apache.org/jira/browse/KAFKA-14133 which use EasyMock and need to be moved to Mockito.

Reviewers: Dalibor Plavcic <dalibor.os@proton.me>, Bruno Cadonna <cadonna@apache.org>
2022-09-30 11:20:51 +02:00
Divij Vaidya bc95aa2116
KAFKA-14248; Fix flaky test PlaintextAdminIntegrationTest.testCreateTopicsReturnsConfigs (#12669)
The test is failing intermittently because we do not wait for propagation of the altered config (LogRetentionTimeMillisProp) across all brokers before proceeding ahead with the test.

This PR makes the following changes:
1. Wait for propagation of altered configuration to propagate to all brokers.
2. Use the existing `killBroker` utility method which waits for shutdown using `awaitshutdown`.
3. Improve code readability by using `TestUtils.incrementalAlterConfigs` to send alter config requests.

Reviewers: Jason Gustafson <jason@confluent.io>
2022-09-29 16:24:03 -07:00
José Armando García Sancio 281e178352
MINOR; Add missing code end tag (#12702)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2022-09-29 15:09:00 -07:00
José Armando García Sancio bee7ec6f26
MINOR; Update upgrade documentation for 3.3.1 (#12701)
Reviewers: David Arthur <mumrah@gmail.com>
2022-09-29 14:56:32 -07:00
LinShunKang 51dbd175b0
KAFKA-4852: Fix ByteBufferSerializer#serialize(String, ByteBuffer) not compatible with offsets (#12683)
Reviewers: Guozhang Wang <wangguoz@gmail.com>
2022-09-29 10:59:47 -07:00
Colin P. McCabe b9da249bdf fix test 2022-09-29 09:17:03 -07:00
Colin P. McCabe fc786c335c add unit and integration tests 2022-09-28 22:58:39 -07:00
Colin P. McCabe ba89eaf0d7 KAFKA-14265: Prefix ACLs may shadow other prefix ACLs 2022-09-28 17:03:41 -07:00