Commit Graph

8989 Commits

Author SHA1 Message Date
Lucas Bradstreet fe16912dfc
KAFKA-12736: KafkaProducer.flush holds onto completed ProducerBatch(s) until flush completes (#10620)
When flush is called a copy of incomplete batches is made. This
means that the full ProducerBatch(s) are held in memory until the flush
has completed. Note that the `Sender` removes producer batches
from the original incomplete collection when they're no longer
needed.

For batches where the existing memory pool is used this
is not as wasteful as the memory will be returned to the pool,
but for non pool memory it can only be GC'd after the flush has
completed. Rather than use copyAll we can make a new array with only the
produceFuture(s) and await on those.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
2021-05-17 06:34:32 -07:00
Gunnar Morling e35f5c88b1
MINOR: Fix typo in `ClusterTool` (#10706)
Reviewers: David Jacot <djacot@confluent.io>
2021-05-17 09:34:43 +02:00
Matthias J. Sax 3b6599c600
MINOR: remove unneccessary `public` keyword from `Partitioner` interface (#10708)
Reviewers: David Jacot <djacot@confluent.io>
2021-05-16 14:22:49 -07:00
Dejan Stojadinović ad91c5edf0
KAFKA-12728: Upgrade gradle to 7.0.2 and shadow to 7.0.0 (#10606)
Details:
* gradle upgrade: 6.8.3 -> 7.0.2
  https://github.com/gradle/gradle/releases/tag/v7.0.0
  https://github.com/gradle/gradle/releases/tag/v7.0.1
  https://github.com/gradle/gradle/releases/tag/v7.0.2
* 'distributionSha256Sum' gradle property is included into 'gradle-wrapper.properties' file
* gradle shadow plugin upgrade: 6.1.0 -> 7.0.0
  https://github.com/johnrengelman/shadow/releases/tag/7.0.0
* Remaining configurations obsoleted in Gradle 6 (and removed in Gradle 7) are replaced:
  `compile` -> `implementation`
  `testCompile` -> `testImplementation`

Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-05-15 06:10:04 -07:00
Colin Patrick McCabe f20fdbd839
KAFKA-12778: Fix QuorumController request timeouts and electLeaders (#10688)
The QuorumController should honor the timeout for RPC requests
which feature a timeout. For electLeaders, attempt to trigger a leader
election for all partitions when the request specifies null for the topics
argument.

Reviewers: David Arthur <mumrah@gmail.com>
2021-05-14 12:44:16 -07:00
Walker Carlson f2785f3c4f
KAFKA-12754: Improve endOffsets for TaskMetadata (#10634)
Improve endOffsets for TaskMetadata by updating immediately after polling a new batch

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2021-05-14 12:17:31 -07:00
Cong Ding db3e5e2c0d
Rework on KAFKA-3968: fsync the parent directory of a segment file when the file is created (#10680)
(reverted #10405). #10405 has several issues, for example:

It fails to create a topic with 9000 partitions.
It flushes in several unnecessary places.
If multiple segments of the same partition are flushed at roughly the same time, we may end up doing multiple unnecessary flushes: the logic of handling the flush in LogSegments.scala is weird.
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.

Did performance test which shows this PR has a minimal performance impact on Kafka clusters.

Reviewers: Jun Rao <junrao@gmail.com>
2021-05-14 09:25:24 -07:00
Chia-Ping Tsai 29c55fdbbc
MINOR: set replication.factor to 1 to make StreamsBrokerCompatibilityService work with old broker (#10673)
Reviewers: Matthias J. Sax <mjsax@conflunet.io>, A. Sophie Blee-Goldman <sophie@confluent.io>
2021-05-14 13:51:31 +08:00
A. Sophie Blee-Goldman 4153e754f1
MINOR: prevent cleanup() from being called while Streams is still shutting down (#10666)
Currently KafkaStreams#cleanUp only throw an IllegalStateException if the state is RUNNING or REBALANCING, however the application could be in the process of shutting down in which case StreamThreads may still be running. We should also throw if the state is PENDING_ERROR or PENDING_SHUTDOWN

Reviewers: Walker Carlson <wcarlson@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2021-05-13 16:16:35 -07:00
Daniyar Yeralin 6d1ae8bc00
KAFKA-8326: Introduce List Serde (#6592)
Introduce List serde for primitive types or custom serdes with a serializer and a deserializer according to KIP-466

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Matthias J. Sax <mjsax@conflunet.io>, John Roesler <roesler@confluent.io>, Michael Noll <michael@confluent.io>
2021-05-13 15:54:00 -07:00
Ryan Dielhenn e69571aecc
KAFKA-12697: Add Global Topic and Partition count metrics to the Quorum Controller (#10679)
Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-05-13 12:08:18 -07:00
A. Sophie Blee-Goldman 4b2736570c
KAFKA-12648: MINOR - Add TopologyMetadata.Subtopology class for subtopology metadata (#10676)
Introduce a Subtopology class to wrap the topicGroupId and namedTopology metadata.

Reviewers: Walker Carlson <wcarlson@confluent.io>
2021-05-13 11:25:18 -07:00
Ismael Juma 13ffebe2f1
MINOR: Update jacoco to 0.8.7 for JDK 16 support (#10654)
Details:
* https://github.com/jacoco/jacoco/releases/tag/v0.8.6
* https://github.com/jacoco/jacoco/releases/tag/v0.8.7

Ran `./gradlew clients:reportCoverage -PenableTestCoverage=true -Dorg.gradle.parallel=false`
successfully with Java 15 (see https://github.com/gradle/gradle/issues/15730 and
https://github.com/scoverage/gradle-scoverage/issues/150 for the reason why 
`-Dorg.gradle.parallel=false` is required).

Also updated `README.md` to include `-Dorg.gradle.parallel=false` alongside `reportCoverage`.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-05-12 08:13:03 -07:00
Chia-Ping Tsai 9c9ea3044e
MINOR: exclude all `src/generated` and `src/generated-test` (#10671)
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2021-05-12 15:27:49 +08:00
dengziming 2debbb98c8
KAFKA-12772: Move all transaction state transition rules into their states (#10667)
Co-authored-by: dengziming <dengziming@growingio.com>
2021-05-11 18:14:38 -07:00
Satish Duggana 7ef3879429
KAFKA-12758 Added `server-common` module to have server side common classes. (#10638)
Added server-common module to have server side common classes. Moved ApiMessageAndVersion, RecordSerde, AbstractApiMessageSerde, and BytesApiMessageSerde to server-common module.

Reivewers:  Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
2021-05-11 09:58:28 -07:00
Satish Duggana 80a468e2b3
MINOR Removed copying storage libraries specifically as they are already copied. (#10647)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
2021-05-11 09:04:40 -07:00
Vito Jeng fae0784ce3
KAFKA-5876: KIP-216 Part 4, Apply InvalidStateStorePartitionException for Interactive Queries (#10657)
KIP-216, part 4 - apply InvalidStateStorePartitionException

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2021-05-10 17:29:58 -07:00
Guozhang Wang 25f4ee879c
KAFKA-12747: Fix flakiness in shouldReturnUUIDsWithStringPrefix (#10643)
Consecutive UUID generation could result in same prefix.

Reviewers: Josep Prat <josep.prat@aiven.io>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2021-05-10 12:32:51 -07:00
Chia-Ping Tsai 8a574adcae
MINOR: remove unnecessary placeholder from WorkerSourceTask#recordSent (#10659)
Reviewers: Tom Bentley <tbentley@redhat.com>
2021-05-10 10:22:26 +01:00
Dejan Stojadinović 9bec36256e
MINOR: Remove unused `scalatest` definition from `dependencies.gradle` (#10655)
Related PR where the `scalatest` usage was removed: #9858

Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-05-09 19:22:29 -07:00
Dejan Stojadinović d934647484
MINOR: checkstyle version upgrade: 8.20 -> 8.36.2 (#10656)
Details:
* Release notes: https://checkstyle.org/releasenotes.html#Release_8.36.2
* Checkstyle version 8.42 should be skipped (lots of false positives, see here: https://github.com/checkstyle/checkstyle/issues/9957)
* More recent Checkstyle versions (i.e. 8.37 and above) are imposing more strict indentation
rules.
2021-05-09 19:03:14 -07:00
Luke Chen f1ef21f70a
KAFKA-12464: minor code cleanup and additional logging in constrained sticky assignment (#10645)
This is the follow up PR to address the remaining comments in #10509.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2021-05-08 18:11:40 -07:00
Jorge Esteban Quilcate Otoya 8f8f914efc
KAFKA-12536: Add Instant-based methods to ReadOnlySessionStore (#10390)
Implements: KIP-666 (https://cwiki.apache.org/confluence/display/KAFKA/KIP-666%3A+Add+Instant-based+methods+to+ReadOnlySessionStore)

Reviewers: John Roesler <vvcephei@apache.org>
2021-05-07 13:24:41 -05:00
Luke Chen 91b3be44a2
MINOR: replace deprecated Class.newInstance() to new one (#10610)
* replace deprecated Class.newInstance() to class.getDeclaredConstructor().newInstance()
* throw ReflectiveOperationException to cover all other exceptions

Reviewers: Tom Bentley <tbentley@redhat.com>
2021-05-07 14:16:58 +01:00
Chia-Ping Tsai f109240236
MINOR: remove storage/src/generated from tracked files (#10637)
Reviewers: Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>, Satish Duggana <satishd@apache.org>
2021-05-07 18:24:33 +08:00
Scott Hendricks 15d1cc8b54
MINOR: Improvements and fixes for Trogdor payload generators. (#10621)
* Changes the new Throughput Generators to track messages per window
instead of making per-second calculations which can have rounding errors.
Also, one of these had a calculation error which prompted this change in
the first place.

* Fixes a couple typos.

* Fixes an error where certain JSON fields were not exposed, causing the
workloads to not behave as intended.

* Fixes a bug where we use wait not in a loop, which exits too quickly.

* Adds additional constant payload generators.

* Fixes problems with an example spec.

* Fixes several off-by-one comparisons.

Reviewers: Colin P. McCabe <cmccabe@apache.org>
2021-05-06 17:46:01 -07:00
Sergio Peña 45d7440c15
KAFKA-10847: Set StreamsConfig on InternalTopologyDriver before writing topology (#10640)
Reviewers: Guozhang Wang <guozhang@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2021-05-06 17:27:23 -07:00
Bruno Cadonna 90fc875e24
KAFKA-8897: Upgrade RocksDB to 6.19.3 (#10568)
This PR upgrades RocksDB to 6.19.3. After the upgrade the Gradle build exited with code 134 due to SIGABRT signals ("Pure virtual function called!") coming from the C++ part of RocksDB. This error was caused by RocksDB state stores not properly closed in Streams' code. This PR adds the missing closings and updates the RocksDB option adapter.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Guozhang Wang <wangguoz@gmail.com>
2021-05-06 15:29:26 -07:00
Jorge Esteban Quilcate Otoya 12a1e68aeb
KAFKA-12451: Remove deprecation annotation on long-based read operations in WindowStore (#10296)
Complete https://cwiki.apache.org/confluence/display/KAFKA/KIP-667%3A+Remove+deprecated+methods+from+ReadOnlyWindowStore by removing deprecation annotation on long-based read operations in WindowStore.

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2021-05-06 11:25:23 -07:00
Bruno Cadonna 94be57d610
MINOR: Fix formatting in RelationalSmokeTest (#10639)
Fixes formatting in RelationalSmokeTest.

Reviewers: Leah Thomas <lthomas@confluent.io>
2021-05-06 17:25:02 +02:00
Shay Elkin b73d639adc
KAFKA-12752: Bump Jersey deps to 2.34 due to CVE-2021-28168 (#10636)
The version of the Eclipse Jersey library brought as dependences,
2.31, has a known vulnerability, CVE-2021-28168 (https://github.com/advisories/GHSA-c43q-5hpj-4crv).

This replaces it with 2.34, which is fully compatible with
2.31, except for bugs and vulnerabilities.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2021-05-06 20:14:03 +05:30
leah 03690d7a1f
MINOR: Stop using hamcrest in system tests (#10631)
We currently use hamcrest imports to check the outputs of the RelationalSmokeTest, but with the new gradle updates, the proper hamcrest imports are no longer included in the test jar.

This is a bit of a workaround to remove the hamcrest usage so we can get system tests up and running again. Potential follow-up could be to update the way we create the test-jar to pull in the proper dependencies.

Reviewers: Bruno Cadonna <cadonna@apache.org>
2021-05-06 12:05:36 +02:00
Chris Egerton ebd4748eba
MINOR: Stop logging raw record contents above TRACE level in WorkerSourceTask (#10630)
Reviewers: Tom Bentley <tbentley@redhat.com>
2021-05-06 09:57:45 +01:00
Luke Chen 80aea23beb
KAFKA-9295: increase startup timeout for flaky test in KTableKTableForeignKeyInnerJoinMultiIntegrationTest (#10635)
Try to address the extreme flakiness of shouldInnerJoinMultiPartitionQueryable since the recent test cleanup. Since we need to wait for 3 streams reach RUNNING state, it makes sense to increase the waiting time to make the test more reliable.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2021-05-05 21:58:27 -07:00
Luke Chen 79225504ed
KAFKA-12464: enhance constrained sticky Assign algorithm (#10509)
1. Make code simpler and cleaner
2. After the PR: the testLargeAssignmentAndGroupWithUniformSubscription (1 million partitions) will run from ~2600 ms down to ~1400 ms, improves 46% of performance, almost 2x faster!!

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Guozhang Wang <guozhang@confluent.io>
2021-05-05 18:44:59 -07:00
Matthias J. Sax 6a5992a814
KAFKA-8531: Change default replication factor config (#10532)
Implements KIP-733

Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>
2021-05-05 16:16:06 -07:00
Chris Egerton 9ba583f6d6
KAFKA-12252 and KAFKA-12262: Fix session key rotation when leadership changes (#10014)
Author: Chris Egerton <chrise@confluent.io>
Reviewers: Greg Harris <gregh@confluent.io>, Randall Hauch <rhauch@gmail.com>
2021-05-05 16:11:15 -05:00
Chia-Ping Tsai d881d11388
MINOR: fix streams_broker_compatibility_test.py (#10632)
The log message was changed and so the system test can't capture expected message.

Reviewers: Anna Sophie Blee-Goldman ableegoldman@apache.org>
2021-05-05 11:12:00 -07:00
Sergio Peña d915ce58d2
KAFKA-10847: Set shared outer store to an in-memory store when in-memory stores are supplied (#10613)
When users supply in-memory stores for left/outer joins, then the internal shared outer store must be switch to in-memory store too. This will allow users who want to keep all stores in memory to continue doing so.

Added unit tests to validate topology and left/outer joins work fine with an in-memory shared store.

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2021-05-05 10:21:43 -07:00
Satish Duggana a1367f57f5
KAFKA-12429: Added serdes for the default implementation of RLMM based on an internal topic as storage. (#10271)
KAFKA-12429: Added serdes for the default implementation of RLMM based on an internal topic as storage. This topic will receive events of RemoteLogSegmentMetadata, RemoteLogSegmentUpdate, and RemotePartitionDeleteMetadata. These events are serialized into Kafka protocol message format.
Added tests for all the event types for that topic.

This is part of the tiered storaqe implementation KIP-405.

Reivewers:  Kowshik Prakasam <kprakasam@confluent.io>, Jun Rao <junrao@gmail.com>
2021-05-05 07:48:52 -07:00
vamossagar12 9a71468cb0
KAFKA-10767: Adding test cases for all, reverseAll and reverseRange for ThreadCache (#9779)
The test cases for ThreaCache didn't have the corresponding unit tests for all, reverseAll and reverseRange methods. This PR aims to add the same.

Reviewers: Bruno Cadonna <cadonna@apache.org>
2021-05-05 12:26:51 +02:00
Jorge Esteban Quilcate Otoya 45f24c4195
KAFKA-12450: Remove deprecated methods from ReadOnlyWindowStore (#10294)
Implement first part of https://cwiki.apache.org/confluence/display/KAFKA/KIP-667%3A+Remove+deprecated+methods+from+ReadOnlyWindowStore.

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2021-05-04 09:23:30 -07:00
Sergio Peña 62221edaff
KAFKA-10847: Add internal flag to disable KAFKA-10847 fix (#10612)
Adds an internal flag that can be used to disable the fixes in KAFKA-10847. It defaults to true if the flag is not set or has an invalid boolean value.

The flag is named __enable.kstreams.outer.join.spurious.results.fix__. This flag is considered internal only. It is a temporary flag that will be used to help users to disable the join fixes while they do a transition from the previous semantics of left/outer joins. The flag may be removed in future releases.

Reviewers: Guozhang Wang <wangguoz@gmail.com>
2021-05-03 14:10:05 -07:00
Vito Jeng 816f5c3b86
KAFKA-5876: KIP-216 Part 3, Apply StreamsNotStartedException for Interactive Queries (#10597)
KIP-216 Part 3: Throw StreamsNotStartedException if KafkaStreams state is CREATED

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2021-05-03 13:53:35 -07:00
dengziming 3ddc377b05
MINOR: Remove duplicate method in test classes (#10535)
1. Remove duplicate serializing auto-generated data in RequestConvertToJsonTest, this is inspired by #9964
2. Remove RequestTestUtils.serializeRequestWithHeader since we added a AbstractRequest.serializeWithHeader in #10142

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-05-03 12:41:03 +08:00
Ismael Juma 711608b3b1
MINOR: Update test libraries and gradle plugins for better JDK 16/17 support (#10619)
Details:
* spotbugs gradle plugin from 4.6.0 to 4.7.1:
  https://github.com/spotbugs/spotbugs-gradle-plugin/releases/tag/4.6.1
  https://github.com/spotbugs/spotbugs-gradle-plugin/releases/tag/4.6.2
  https://github.com/spotbugs/spotbugs-gradle-plugin/releases/tag/4.7.0
  https://github.com/spotbugs/spotbugs-gradle-plugin/releases/tag/4.7.1
* spotless gradle plugin from 5.10.2 to 5.12.4:
  https://github.com/diffplug/spotless/blob/gradle/5.12.4/CHANGES.md
* test-retry gradle plugin from 1.2.0 to 1.2.1:
  https://github.com/gradle/test-retry-gradle-plugin/releases/tag/v1.2.1
* dependency check gradle plugin from 6.1.1 to 6.1.6:
  https://github.com/jeremylong/DependencyCheck/releases/tag/v6.1.2
  https://github.com/jeremylong/DependencyCheck/releases/tag/v6.1.3
  https://github.com/jeremylong/DependencyCheck/releases/tag/v6.1.4
  https://github.com/jeremylong/DependencyCheck/releases/tag/v6.1.5
  https://github.com/jeremylong/DependencyCheck/releases/tag/v6.1.6
* versions gradle plugin from 0.36.0 to 0.38.0:
https://github.com/ben-manes/gradle-versions-plugin/releases/tag/v0.37.0
https://github.com/ben-manes/gradle-versions-plugin/releases/tag/v0.38.0
* easymock from 4.2 to 4.3:
  https://github.com/easymock/easymock/releases/tag/easymock-4.3
* mockito from 3.6.0 to 3.9.0:
https://github.com/mockito/mockito/releases (too many releases to list
  them all individually)
* spotbugs from 4.1.4 to 4.2.2:
  https://github.com/spotbugs/spotbugs/blob/4.2.2/CHANGELOG.md
  4.2.3 has a regression that causes spurious errors related to `Random`
  usage.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-05-02 12:15:54 -07:00
wenbingshen 325cb8853b
MINOR: Clean up some redundant code from ReplicaManager (#10623)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2021-05-03 00:44:32 +08:00
Chia-Ping Tsai e73731d848
KAFKA-12661 ConfigEntry#equal does not compare other fields when value is NOT null (#10446)
Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-05-02 13:07:08 +08:00
José Armando García Sancio 6203bf8b94
KAFKA-12154; Raft Snapshot Loading API (#10085)
Implement Raft Snapshot loading API.

1. Adds a new method `handleSnapshot` to `raft.Listener` which is called whenever the `RaftClient` determines that the `Listener` needs to load a new snapshot before reading the log. This happens when the `Listener`'s next offset is less than the log start offset also known as the earliest snapshot.

2.  Adds a new type `SnapshotReader<T>` which provides a `Iterator<Batch<T>>` interface and de-serializes records in the `RawSnapshotReader` into `T`s

3.  Adds a new type `RecordsIterator<T>` that implements an `Iterator<Batch<T>>` by scanning a `Records` object and deserializes the batches and records into `Batch<T>`. This type is used by both `SnapshotReader<T>` and `RecordsBatchReader<T>` internally to implement the `Iterator` interface that they expose. 

4. Changes the `MockLog` implementation to read one or two batches at a time. The previous implementation always read from the given offset to the high-watermark. This made it impossible to test interesting snapshot loading scenarios.

5. Removed `throws IOException` from some methods. Some of types were inconsistently throwing `IOException` in some cases and throwing `RuntimeException(..., new IOException(...))` in others. This PR improves the consistent by wrapping `IOException` in `RuntimeException` in a few more places and replacing `Closeable` with `AutoCloseable`.

6. Updated the Kafka Raft simulation test to take into account snapshot. `ReplicatedCounter` was updated to generate snapshot after 10 records get committed. This means that the `ConsistentCommittedData` validation was extended to take snapshots into account. Also added a new invariant to ensure that the log start offset is consistently set with the earliest snapshot.

Reviewers: dengziming <swzmdeng@163.com>, David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>
2021-05-01 10:05:45 -07:00