Commit Graph

9010 Commits

Author SHA1 Message Date
Josep Prat e23ede1ece
KAFKA-12809: Remove deprecated methods of Stores factory (#10729)
Removes methods deprecated via KIP-319 and KIP-358.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2021-05-19 16:07:35 -07:00
Josep Prat 0af37730fc
KAFKA-12813: Remove deprecated schedule method in ProcessorContext (#10730)
Removes methods deprecated via KIP-358.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2021-05-19 16:03:01 -07:00
Matthias J. Sax 476eccb968
KAFKA-12815: Preserve context for KTable.transformValues when getting value from upstream state store (#10720)
Reviewers: Victoria Xia <victoria.xia@confluent.io>, John Roesler <john@confluent.io>
2021-05-19 14:58:46 -07:00
Josep Prat b58da356be
KAFKA-12810: Remove deprecated TopologyDescription.Source#topics (#10727)
Removes methods that were deprecated via KIP-321.

Reviewers: Matthias J. Sax <matthias@confluent.io>
2021-05-19 11:49:26 -07:00
Colin Patrick McCabe bb48cf33b3
MINOR: add ConfigUtils method for printing configurations (#10714)
Reviewers: Luke Chen <showuon@gmail.com>, David Arthur <mumrah@gmail.com>
2021-05-19 11:03:49 -07:00
Ron Dagostino 5b0c58ed53
MINOR: Support using the ZK authorizer with KRaft (#10550)
This patch adds support for running the ZooKeeper-based
kafka.security.authorizer.AclAuthorizer with KRaft clusters. Set the
authorizer.class.name config as well as the zookeeper.connect config while also
setting the typical KRaft configs (node.id, process.roles, etc.), and the
cluster will use KRaft for metadata and ZooKeeper for ACL storage. A system
test that exercises the authorizer is included.

This patch also changes "Raft" to "KRaft" in several system test files. It also
fixes a bug where system test admin clients were unable to connect to a cluster
with broker credentials via the SSL security protocol when the broker was using
that for inter-broker communication and SASL for client communication.

Reviewers: Colin P. McCabe <cmccabe@apache.org>, Ismael Juma <ismael@juma.me.uk>
2021-05-19 10:32:56 -07:00
zhaohaidao 7beec14e5c
KAFKA-12789: Remove Stale comments for meta response handling logic (#10700)
Correct empty meta response comment, since it is no longer related only to brokers associating with the query topic.

Reviewers: Boyang Chen <boyang@confluent.io>
2021-05-19 09:38:41 -07:00
Josep Prat a8bd649dbf
MINOR: Update Scala to 2.13.6 (#10711)
This includes TASTy Reader support for Scala 3.0.0. This makes it easier
for Kafka libraries to be used in Scala 3.0 projects

Release notes: https://github.com/scala/scala/releases/tag/v2.13.6

Reviewers: Ismael Juma <ismael@juma.me.uk>
2021-05-19 05:26:08 -07:00
Luke Chen e11f249327
KAFKA-9295: increase session timeout to fix flaky KTableKTableForeignKeyInnerJoinMultiIntegrationTest (#10715)
Increase session timeout to fix flaky KTableKTableForeignKeyInnerJoinMultiIntegrationTest.shouldInnerJoinMultiPartitionQueryable

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2021-05-18 23:00:43 -07:00
David Arthur 937e28db5d
Fix compile errors from KAFKA-12543 (#10719)
Reviewers: Colin P. Mccabe <cmccabe@confluent.io>, Jun Rao <junrao@gmail.com>, José Armando García Sancio <jsancio@users.noreply.github.com>
2021-05-18 17:09:51 -04:00
A. Sophie Blee-Goldman 3a42baa260
HOTFIX: undo renaming of public part of Subtopology API (#10713)
In #10676 we renamed the internal Subtopology class that implemented the TopologyDescription.Subtopology interface. By mistake, we also renamed the interface itself, which is a public API. This wasn't really the intended point of that PR, so rather than do a retroactive KIP, let's just reverse the renaming.

Reviewers: Walker Carlson <wcarlson@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2021-05-18 12:59:20 -07:00
José Armando García Sancio 924c870fb1
KAFKA-12543: Change RawSnapshotReader ownership model (#10431)
Kafka networking layer doesn't close FileRecords and assumes that they are already open when sending them over a channel. To support this pattern this commit changes the ownership model for FileRawSnapshotReader so that they are owned by KafkaMetadataLog.

Reviewers: dengziming <swzmdeng@163.com>, David Arthur <mumrah@gmail.com>, Jun Rao <junrao@gmail.com>
2021-05-18 14:14:17 -04:00
David Arthur aefe0e4d8c
KAFKA-12686 AlterIsr and LeaderAndIsr race condition (#10561)
Remove the clearPending method from AlterIsrManager

Reviewers: Colin P. Mccabe <cmccabe@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
2021-05-18 09:56:37 -04:00
Colin Patrick McCabe 9e5b77fb96
KAFKA-12788: improve KRaft replica placement (#10494)
Implement a striped replica placement algorithm for KRaft. This also
means implementing rack awareness.  Previously, KRraft just chose
replicas randomly in a non-rack-aware fashion.  Also, allow replicas to
be placed on fenced brokers if there are no other choices.  This was
specified in KIP-631 but previously not implemented.

Reviewers: Jun Rao <junrao@gmail.com>
2021-05-17 16:49:47 -07:00
Ron Dagostino 12377bd3c6
MINOR: Add missing @cluster annotation to StreamsNamedRepartitionTopicTest (#10697)
The StreamsNamedRepartitionTopicTest system tests did not have the @cluster annotation and was therefore taking up the entire cluster. For example, we see this in the log output:

kafkatest.tests.streams.streams_named_repartition_topic_test.StreamsNamedRepartitionTopicTest.test_upgrade_topology_with_named_repartition_topic is using entire cluster. It's possible this test has no associated cluster metadata.

This PR adds the missing annotation.

Reviewers: Bill Bejeck <bbejeck@apache.org>
2021-05-17 17:33:43 -04:00
Rajini Sivaram 9eb9b16d57
KAFKA-12751: Reset AlterIsr in-flight state for duplicate update requests (#10633)
Reviewers: David Arthur <david.arthur@confluent.io>
2021-05-17 19:31:39 +01:00
Ron Dagostino 55b24ce9d6
MINOR: fix system test TestSecurityRollingUpgrade (#10694)
Ensure security protocol and sasl mechanism are updated in the cached SecurityConfig during rolling system tests. Also explicitly indicate which SASL mechanisms we wish to expose during the tests.

Reviewers: David Arthur <mumrah@gmail.com>
2021-05-17 13:46:44 -04:00
A. Sophie Blee-Goldman cc6f4c49a9
KAFKA-12574: remove internal Producer config and auto downgrade logic (#10675)
Minor followup to #10573. Removes this internal Producer config which was only ever used to avoid a very minor amount of work to downgrade the consumer group metadata in the txn commit request in Kafka Streams

Reviewers: Ismael Juma <ismael@juma.me.uk>, Matthias J. Sax <mjsax@confluent.io>, Guozhang Wang <guozhang@confluent.io>
2021-05-17 10:25:35 -07:00
Colin Patrick McCabe 0b9a7bd75e
KAFKA-12792: Fix metrics bug and introduce TimelineInteger (#10707)
Introduce a TimelineInteger class which represents a single integer
value which can be changed while maintaining snapshot consistency. Fix a
case where a metric value would be corrupted after a snapshot restore.

Reviewers: David Arthur <mumrah@gmail.com>
2021-05-17 10:21:26 -07:00
vamossagar12 b9acc492a5
KAFKA-12313: KIP-725: Streamlining configs for Windowed Deserialisers (#10542)
This PR aims to streamline the configurations for WindowedDeserialisers as described in KIP-725. It deprecates default.windowed.key.serde.inner and default.windowed.value.serde.inner configs in StreamConfig and adds windowed.inner.class.serde. 

Reviewers: Anna Sophie Blee-Goldman<ableegoldman@apache.org>
2021-05-17 10:17:31 -07:00
Nathan Lincoln 0188fbea09
KAFKA-12794: Fix trailing json tokens in `DescribeProducersRequest.json` (#10709)
Reviewers: David Jacot <djacot@confluent.io>
2021-05-17 17:44:10 +02:00
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