Commit Graph

10511 Commits

Author SHA1 Message Date
David Jacot dbe20a5994 KAFKA-14367; Add `JoinGroup` to the new `GroupCoordinator` interface
Add GroupCoordinator interface
2022-11-14 09:49:33 +01:00
David Jacot 7387a11625
KAFKA-14363; Add new `group-coordinator` module (KIP-848) (#12827)
Introduce new group-coordinator module that will host the future new group coordinator as part of KIP-848.

Reviewers: Ismael Juma <ismael@juma.me.uk>
2022-11-09 08:49:57 +01:00
Lucas Brutschy c034388a0a
KAFKA-14299: Avoid allocation & synchronization overhead in StreamThread loop (#12808)
The state updater code path introduced allocation and synchronization
overhead by performing relatively heavy operations in every iteration of
the StreamThread loop. This includes various allocations and acquiring
locks for handling `removedTasks` and `failedTasks`, even if the
corresponding queues are empty.

This change introduces `hasRemovedTasks` and
`hasExceptionsAndFailedTasks` in the `StateUpdater` interface that
can be used to skip over any allocation or synchronization. The new
methods do not require synchronization or memory allocation.

This change increases throughput by ~15% in one benchmark.

We extend existing unit tests to cover the slightly modified
behavior.

Reviewer: Bruno Cadonna <cadonna@apache.org>
2022-11-08 17:55:37 +01:00
Colin Patrick McCabe ac3a3687a0
MINOR: Avoid highestSupportedVersion outside tests (#12829)
We should avoid using Message.highestSupportedVersion to generate metadata records. Instead, we
need to pick the correct record version based on the current metadata version which is in effect.
In cases where there is only one record version that we know how to generate, we can hard code
that version, but it should just be a simple constant zero.

Reviewers: Jason Gustafson <jason@confluent.io>
2022-11-08 08:38:47 -08:00
Chris Egerton bb84476215
KAFKA-14098: Add meaningful client IDs for Connect workers (#12544)
Reviewers: Greg Harris <greg.harris@aiven.io>, Mickael Maison <mickael.maison@gmail.com>
2022-11-08 10:22:30 -05:00
Lucas Brutschy 4560978ed7
KAFKA-14309: FK join upgrades not tested with DEV_VERSION (#12760)
The streams upgrade system inserted FK join code for every version of the
the StreamsUpgradeTest except for the latest. Also, the original code
never switched on the `test.run_fk_join` flag for the target version of
the upgrade.

The effect was that FK join upgrades were not tested at all, since
no FK join code was executed after the bounce in the system test.

We introduce `extra_properties` in the system tests, that can be used
to pass any property to the upgrade driver, which is supposed to be
reused by system tests for switching on and off flags (e.g. for the
state restoration code).

Reviewers: Alex Sorokoumov <asorokoumov@confluent.io>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2022-11-07 15:46:51 -08:00
Jason Gustafson 150a0758cb
MINOR: Change system test console consumer default log level (#12819)
For tests which use the console consumer service, we are currently enabling TRACE logging by default. I have seen some system tests where this produces GBs of logging. A better default is probably DEBUG.

Reviewers: José Armando García Sancio <jsancio@apache.org>
2022-11-07 13:42:36 -08:00
Christo Lolov 4af5dae05c
KAFKA-14132: Replace PowerMock and EasyMock with Mockito in connect tests (#12823)
Reviewers: Chris Egerton <chrise@aiven.io>
2022-11-07 11:34:55 -05:00
Lucas Brutschy cd5f6c60b5
KAFKA-14299: Avoid busy polling in state updater (#12772)
The state updater can enter a busy polling loop if it
only updates standby tasks. We need to use the user-provided
poll-time to update always when using the state updater, since
the only other place where the state update blocks
(inside `waitIfAllChangelogsCompletelyRead`) is also
not blocking if there is at least one standby task.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Bruno Cadonna <cadonna@apache.org>
2022-11-07 16:46:25 +01:00
Greg Harris 98ed31c95b
KAFKA-14346: Remove difficult to mock Plugins.compareAndSwapLoader usages (#12817)
Reviewers: Chris Egerton <chrise@aiven.io>
2022-11-07 09:37:24 -05:00
Ashmeet Lamba a971448f3f
KAFKA-14254: Format timestamps as dates in logs (#12684)
Improves logs withing Streams by replacing timestamps to date instances to improve readability.

Approach - Adds a function within common.utils.Utils to convert a given long timestamp to a date-time string with the same format used by Kafka's logger.

Reviewers: Divij Vaidya <diviv@amazon.com>, Bruno Cadonna <cadonna@apache.org>
2022-11-07 13:42:39 +01:00
Lucas Brutschy 37a3645e7e
KAFKA-14299: Return emptied ChangelogReader to ACTIVE_RESTORING (#12773)
The ChangelogReader starts of in `ACTIVE_RESTORING` state, and
then goes to `STANDBY_RESTORING` when changelogs from standby
tasks are added. When the last standby changelogs are removed,
it remained in `STANDBY_RESTORING`, which means that an empty
ChangelogReader could be in either `ACTIVE_RESTORING` or
`STANDBY_RESTORING` depending on the exact sequence of
add/remove operations. This could lead the state updater into
an illegal state. Instead of changing the state updater,
I propose to stengthen the state invariant of the
`ChangelogReader` slightly: it should always be in
`ACTIVE_RESTORING` state, when empty.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Bruno Cadonna <cadonna@apache.org>
2022-11-07 13:35:18 +01:00
Omnia G H Ibrahim 9608f3a2e7
KAFKA-14344: Build EmbeddedKafkaCluster with common configs used for all clients (#12804)
Reviewers: Mickael Maison <mickael.maison@gmail.com>

, Tom Bentley <tbentley@redhat.com>
2022-11-07 11:11:16 +01:00
vamossagar12 7fd6a9b3e2
Kafka 12960: Follow up Commit to filter expired records from Windowed/Session Stores (#12756)
KAFKA-12960: Enforcing strict retention time for WindowStore and SessionStore

Reviewers: Luke Chen <showuon@gmail.com>, Vicky Papavasileiou
2022-11-07 11:53:34 +08:00
runom b8754c074a
KAFKA-14355: Fix integer overflow in ProducerPerformance (#12822)
Change types from int to long to avoid overflow

Reviewers: Luke Chen <showuon@gmail.com>,  Igor Soarez <soarez@apple.com>
2022-11-05 20:19:08 +08:00
Shawn fcab5fb888
Revert "KAFKA-13891: reset generation when syncgroup failed with REBALANCE_IN_PROGRESS (#12140)" (#12794)
This reverts commit c23d60d56c.

Reviewers: Luke Chen <showuon@gmail.com>
2022-11-05 14:05:48 +08:00
Justine Olshan 9467fc3305
MINOR: modified ProducerIdExpirationTest to hopefully be less flaky (#12824)
ProducerIdExpirationTest has failed on a few PR runs. Using a waitUntil instead.

Reviewers: Luke Chen <showuon@gmail.com>
2022-11-05 14:04:35 +08:00
David Jacot bc780c7c32
MINOR: Move timeline data structures from metadata to server-common (#12811)
This path moves the timeline data structures from metadata module to server-common module as those will be used in the new group coordinator.

Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Colin Patrick McCabe <cmccabe@apache.org>
2022-11-04 08:52:32 +01:00
Chris Egerton 14c36c7539
KAFKA-14058: Migrate ExactlyOnceWorkerSourceTaskTest from EasyMock and Powermock to Mockito (#12409)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2022-11-03 12:42:08 -04:00
Philip Nee dc18dd921c
MINOR: Remove outdated comment about metadata request (#12810)
This is an outdated comment about the metadata request from the old implementation. The name of the RPC was changed from GroupMetadata to FindCoordinator, but the comment was not updated.

Reviewers: Jason Gustafson <jason@confluent.io>
2022-11-01 17:01:19 -07:00
Lucas Brutschy e7c1e4a0a1
KAFKA-14299: Handle TaskCorruptedException during initialization (#12771)
State stores are initialized from the StreamThread even when
the state updater thread is enabled. However, we were missing
the corresponding handling of exceptions when thrown directly
during the initialization. In particular, TaskCorruptedException
would directly fall through to runLoop, and the task
would fall out of book-keeping, since the exception is thrown
when neither the StreamThread nor the StateUpdater is owning
the task.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Bruno Cadonna <cadonna@apache.org>
2022-11-01 18:25:19 +01:00
Matthew de Detrich 9ab140d541
KAFKA-14132; Replace EasyMock with Mockito ConnectorsResourceTest (#12725)
Reviewers: Chris Egerton <chrise@aiven.io>
2022-10-31 20:08:41 -04:00
Greg Harris 98bf375b6f
KAFKA-13989: Errors while evaluating connector type should return UNKNOWN (#12789)
Reviewers: Igor Soarez <i@soarez.me>, Chris Egerton <chrise@aiven.io>
2022-10-31 16:54:32 -04:00
Philip Nee fa10e213bf
KAFKA-14247: add handler impl to the prototype (#12775)
Minor revision for KAFKA-14247. Added how the handler is called and constructed to the prototype code path.

Reviewers: John Roesler <vvcephei@apache.org>, Kirk True <kirk@mustardgrain.com>
2022-10-31 14:01:22 -05:00
Jason Gustafson c710ecd071
MINOR: Reduce tries in RecordsIteratorTest to improve build time (#12798)
`RecordsIteratorTest` takes the longest times in recent builds (even including integration tests). The default of 1000 tries from jqwik is probably overkill and causes the test to take 10 minutes locally. Decreasing to 50 tries reduces that to less than 30s. 

Reviewers: David Jacot <djacot@confluent.io>
2022-10-31 09:29:19 -07:00
runom fdd60cc9a7
MINOR: Fix producer.id.expiration.check.interval.ms definition (#12799)
Reviewers: David Jacot <djacot@confluent.io>
2022-10-31 09:35:44 +01:00
dengziming 56d588d55a
MINOR: Fix SubscriptionInfoData name in exception message (#12076)
Reviewers: Andrew Choi <andrewchoi5@users.noreply.github.com>, Luke Chen <showuon@gmail.com>
2022-10-31 10:48:16 +08:00
Greg Harris f7304db6d2
KAFKA-14338: Use MockTime in RetryUtilTest to eliminate flakiness (#12791)
Reviewers: Chris Egerton <chrise@aiven.io>
2022-10-29 06:38:24 -04:00
Jason Gustafson cbe50d95a9
MINOR: Add test case for topic recreation with collision chars (#12796)
This patch adds a unit test for topic recreation with colliding characters (such as `.`). This was broken up until https://github.com/apache/kafka/pull/12790. 

Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>
2022-10-28 18:50:08 -07:00
Luke Chen c1bb307a36
KAFKA-14337; Correctly remove topicsWithCollisionChars after topic deletion (#12790)
In https://github.com/apache/kafka/pull/11910 , we added a feature to prevent topics with conflicting metrics names from being created. We added a map to store the normalized topic name to the topic names, but we didn't remove it correctly while deleting topics. This PR fixes this bug and add a test.

Reviewers: Igor Soarez <i@soarez.me>, dengziming <dengziming1993@gmail.com>, Jason Gustafson <jason@confluent.io>
2022-10-28 10:08:53 -07:00
Orsák Maroš a0e37b79aa
MINOR: Add test cases to the Raft module (#12692)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
, Divij Vaidya <diviv@amazon.com>, Ismael Juma <mlists@juma.me.uk>
2022-10-28 17:54:34 +02:00
(+ (* 1 2 3 4) 5 6 7) 61f48a9f63
KAFKA-14314: Add check for null upstreamTopic (#12769)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
Co-authored-by: John Krupka <john.krupka@united.com>
2022-10-28 17:28:26 +02:00
Divij Vaidya 5e399fe6f3
Move to mockito (#12465)
This PR build on top of #11017. I have added the previous author's comment in this PR for attribution. I have also addressed the pending comments from @chia7712 in this PR.

Notes to help the reviewer:

Mockito has mockStatic method which is equivalent to PowerMock's method.
When we run the tests using @RunWith(MockitoJUnitRunner.StrictStubs.class) Mockito performs a verify() for all stubs that are mentioned, hence, there is no need to explicitly verify the stubs (unless you want to verify the number of times etc.). Note that this does not work for static mocks.

Reviewers: Bruno Cadonna <cadonna@apache.org>, Walker Carlson <wcarlson@confluent.io>, Bill Bejeck <bbejeck@apache.org>
2022-10-27 14:08:44 -04:00
srishti-saraswat 57aefa9c82
MINOR: Migrate connect system tests to KRaft (#12621)
Adds the `metadata_quorum` parameter to the `@matrix(...)` annotation to many existing tests, so that they are run with both zookeeper and remote_kraft nodes.

Reviewers: Randall Hauch <rhauch@gmail.com>, Greg Harris <gharris1727@gmail.com>
2022-10-27 11:19:14 -05:00
Shenglong Zhang 47adb86636
KAFKA-10149: Allow auto preferred leader election when there are ongoing partition reassignments (#12543)
Reviewers: Justine Olshan <jolshan@confluent.io>, Chris Egerton <fearthecellos@gmail.com>
2022-10-26 09:21:55 -04:00
Manikumar Reddy 06dc67a480
MINOR: Fix allDepInsight gradle task with gradle 7.5 (#12779)
Reviewers: David Jacot <djacot@confluent.io>
2022-10-25 18:39:53 +05:30
Manikumar Reddy 0a111a0812
MINOR: Fix rat task verbose output config (#12786)
Reviewers: David Jacot <djacot@confluent.io>
2022-10-25 16:20:59 +05:30
Manikumar Reddy 09b8d511f8
MINOR: Upgrade spotbugs dependency (#12768)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2022-10-24 23:11:31 +05:30
Lucas Brutschy 7dcaec4a79
MINOR: Fix usage instruction of skipSigning build parameter (#12731)
skipSigning parameter must be set to a boolean value to work.

Reviewers: Bill Bejeck <bbejeck@apache.org>
2022-10-24 11:55:48 -04:00
Lucas Brutschy 732887b210
MINOR: Get console output in quickstart examples (#12719)
Quickstart examples didn't produce any console output, since it was missing a logger implementation in the classpath.

Also some minor cleanups.

Tested by creating a test project and running the code.
2022-10-24 11:44:57 -04:00
Chris Egerton 215d4f93bd
MINOR: Remove duplicate test dependency declarations for clients module (#12764)
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2022-10-21 12:18:00 -04:00
vamossagar12 9a793897ec
KAFKA-13152: KIP-770, cache size config deprecation (#12758)
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>
2022-10-20 17:03:50 -07:00
Jason Gustafson b525ddc9f1
MINOR: Fix PartitionRegistration.hashCode (#12774)
`PartitionRegistration.hashCode` passes raw arrays to `Objects.hash` in the `hashCode` implementation. This doesn't work since the `equals` implementation uses `Arrays.equals`. We should use `Arrays.hashCode` instead. 

Reviewers: David Arthur <mumrah@gmail.com>
2022-10-20 14:01:01 -07:00
Philip Nee 0a045d4ef7
KAFKA-14247: Consumer background thread base implementation (#12672)
Adds skeleton of the background thread.

1-pager: https://cwiki.apache.org/confluence/display/KAFKA/Proposal%3A+Consumer+Threading+Model+Refactor
Continuation of #12663

Reviewers: Guozhang Wang <guozhang@apache.org>, Kirk True <kirk@mustardgrain.com>, John Roesler <vvcephei@apache.org>
2022-10-19 21:02:55 -05:00
José Armando García Sancio d0ff869718
MINOR; Add accessor methods to OffsetAndEpoch (#12770)
Accessor are preferred over fields because they compose better with Java's
lambda syntax.

Reviewers: Jason Gustafson <jason@confluent.io>
2022-10-19 12:07:07 -07:00
Lucas Brutschy 2c8f14c57e
KAFKA-14299: Never transition to UpdateStandby twice (#12762)
In two situations, the current code could transition the ChangelogReader
to UpdateStandby when already in that state, causing an IllegalStateException. 
Namely these two cases are:

1. When only standby tasks are restoring and one of them crashes.
2. When only standby tasks are restoring and one of them is paused.

This change fixes both issues by only transitioning if the paused or
failed task is an active task.

Reviewer: Bruno Cadonna <cadonna@apache.org>
2022-10-19 09:29:19 +02:00
Jason Gustafson a692223a44
KAFKA-14316; Fix feature control iterator metadata version handling (#12765)
The iterator `FeatureControlIterator.hasNext()` checks two conditions: 1) whether we have already written the metadata version, and 2) whether the underlying iterator has additional records. However, in `next()`, we also check that the metadata version is at least high enough to include it in the log. When this fails, then we can see an unexpected `NoSuchElementException` if the underlying iterator is empty.

Reviewers: Colin Patrick McCabe <cmccabe@apache.org>
2022-10-18 15:30:45 -07:00
José Armando García Sancio 7dc17908de
KAFKA-14300; Generate snapshot after repeated controller resign (#12747)
Setting the `committedBytesSinceLastSnapshot` to 0 when resigning can cause the controller to not generate a snapshot after `snapshotMaxNewRecordBytes` committed bytes have been replayed.

This change fixes that by simply not resetting the counter during resignation. This is correct because the counter tracks the number of committed bytes replayed but not included in the latest snapshot. In other words, reverting the last committed state does not invalidate this value.

Reviewers: Colin Patrick McCabe <cmccabe@apache.org>
2022-10-18 15:09:20 -07:00
Lucas Brutschy 7a7ad9b422
KAFKA-14299: Fix busy polling with separate state restoration (#12749)
StreamThread in state PARTITIONS_ASSIGNED was running in
a busy loop until restoration is finished, stealing CPU
cycles from restoration.

Make sure the StreamThread uses poll_time when
state updater is enabled, and we are in state
PARTITIONS_ASSIGNED.

Reviewer: Bruno Cadonna <cadonna@apache.org>
2022-10-18 21:33:40 +02:00
Ismael Juma 2e0f9f42dd
MINOR: Inline "Running a Kafka broker in KRaft mode" (#12750)
Also moved KRaft mode above zk mode.

Reviewers: Mickael Maison <mimaison@users.noreply.github.com>
2022-10-18 06:12:25 -07:00