First set of cleanup pushed to followup PR after KIP-441 Pt. 5. Main changes are:
1. Moved `RankedClient` and the static `buildClientRankingsByTask` to a new file
2. Moved `Movement` and the static `getMovements` to a new file (also renamed to `TaskMovement`)
3. Consolidated the many common variables throughout the assignment tests to the new `AssignmentTestUtils`
4. New utility to generate comparable/predictable UUIDs for tests, and removed the generic from `TaskAssignor` and all related classes
Reviewers: John Roesler <vvcephei@apache.org>, Andrew Choi <a24choi@edu.uwaterloo.ca>
There is a race on receiving a LeaderAndIsr request for a replica with an active log dir reassignment. If the reassignment completes just before the LeaderAndIsr handler updates epoch information, it can lead to an illegal state error since no future log dir exists. This patch fixes the problem by ensuring that the future log dir exists when the fetcher is started. Removal cannot happen concurrently because it requires access the same partition state lock.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>
Co-authored-by: Chia-Ping Tsai <chia7712@gmail.com>
The runtime type of Metric.metricValue() needn't always be a Double,
for example, if it's a gauge from IntGaugeSuite.
Since it's impossible to format non-double values with 3 point precision
IllegalFormatConversionException resulted.
Author: Tom Bentley <tbentley@redhat.com>
Author: Tom Bentley <tombentley@users.noreply.github.com>
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Closes#8373 from tombentley/KAFKA-9775-IllegalFormatConversionException
For some context, when building a streams application, the optimizer keeps track of the key-changing operations and any repartition nodes that are descendants of the key-changer. During the optimization phase (if enabled), any repartition nodes are logically collapsed into one. The optimizer updates the graph by inserting the single repartition node between the key-changing node and its first child node. This graph update process is done by searching for a node that has the key-changing node as one of its direct parents, and the search starts from the repartition node, going up in the parent hierarchy.
The one exception to this rule is if there is a merge node that is a descendant of the key-changing node, then during the optimization phase, the map tracking key-changers to repartition nodes is updated to have the merge node as the key. Then the optimization process updates the graph to place the single repartition node between the merge node and its first child node.
The error in KAFKA-9739 occurred because there was an assumption that the repartition nodes are children of the merge node. But in the topology from KAFKA-9739, the repartition node was a parent of the merge node. So when attempting to find the first child of the merge node, nothing was found (obviously) resulting in StreamException(Found a null keyChangingChild node for..)
This PR fixes this bug by first checking that all repartition nodes for optimization are children of the merge node.
This PR includes a test with the topology from KAFKA-9739.
Reviewers: John Roesler <john@confluent.io>
Revert the decision for the sendOffsetsToTransaction(groupMetadata) API to fail with old version of brokers for the sake of making the application easier to adapt between versions. This PR silently downgrade the TxnOffsetCommit API when the build version is small than 3.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
As documented in the KIP:
We shall set `transaction.timout.ms` default to 10000 ms (10 seconds) on Kafka Streams.
Reviewer: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Adds a new TaskAssignor implementation, currently hidden behind an internal feature flag, that implements the high availability algorithm of KIP-441.
Reviewers: Bruno Cadonna <bruno@confluent.io>, John Roesler <vvcephei@apache.org>
Rename the test suite to later add unit tests that don't depend on
ZK or the AdminClient TopiCommand types.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Co-authored-by: Andrew Olson <aolson1@cerner.com>
Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <wangguoz@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
Once Scala 2.13.2 is officially released, I will submit a follow up PR
that enables `-Xfatal-warnings` with the necessary warning
exclusions. Compiler warning exclusions were only introduced in 2.13.2
and hence why we have to wait for that. I used a snapshot build to
test it in the meantime.
Changes:
* Remove Deprecated annotation from internal request classes
* Class.newInstance is deprecated in favor of
Class.getConstructor().newInstance
* Replace deprecated JavaConversions with CollectionConverters
* Remove unused kafka.cluster.Cluster
* Don't use Map and Set methods deprecated in 2.13:
- collection.Map +, ++, -, --, mapValues, filterKeys, retain
- collection.Set +, ++, -, --
* Add scala-collection-compat dependency to streams-scala and
update version to 2.1.4.
* Replace usages of deprecated Either.get and Either.right
* Replace usage of deprecated Integer(String) constructor
* `import scala.language.implicitConversions` is not needed in Scala 2.13
* Replace usage of deprecated `toIterator`, `Traversable`, `seq`,
`reverseMap`, `hasDefiniteSize`
* Replace usage of deprecated alterConfigs with incrementalAlterConfigs
where possible
* Fix implicit widening conversions from Long/Int to Double/Float
* Avoid implicit conversions to String
* Eliminate usage of deprecated procedure syntax
* Remove `println`in `LogValidatorTest` instead of fixing the compiler
warning since tests should not `println`.
* Eliminate implicit conversion from Array to Seq
* Remove unnecessary usage of 3 argument assertEquals
* Replace `toStream` with `iterator`
* Do not use deprecated SaslConfigs.DEFAULT_SASL_ENABLED_MECHANISMS
* Replace StringBuilder.newBuilder with new StringBuilder
* Rename AclBuffers to AclSeqs and remove usage of `filterKeys`
* More consistent usage of Set/Map in Controller classes: this also fixes
deprecated warnings with Scala 2.13
* Add spotBugs exclusion for inliner artifact in KafkaApis with Scala 2.12.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
We find that brokers may send empty assignment for some members unexpectedly, and would need more logs investigating this issue.
Reviewers: John Roesler <vvcephei@apache.org>
Measure the percentage ratio the stream thread spent on processing each task among all assigned active tasks (KIP-444). Also add unit tests to cover the added metrics in this PR and the previous #8358. Also trying to fix the flaky test reported in KAFKA-5842
Co-authored-by: John Roesler <vvcephei@apache.org>
Reviewers: Bruno Cadonna <bruno@confluent.io>, John Roesler <vvcephei@apache.org>
This patch addresses a locking issue with DelayTxnMarker completion. Because of the reliance on the shared read lock in TransactionStateManager and the deadlock avoidance algorithm in `DelayedOperation`, we cannot guarantee that a call to checkAndComplete will offer an opportunity to complete the job. This patch removes the reliance on this lock in two ways:
1. We replace the transaction marker purgatory with a map of transaction with pending markers. We were not using purgatory expiration anyway, so this avoids the locking issue and simplifies usage.
2. We were also relying on the read lock for the `DelayedProduce` completion when calling `ReplicaManager.appendRecords`. As far as I can tell, this was not necessary. The lock order is always 1) state read/write lock, 2) txn metadata locks. Since we only call `appendRecords` while holding the read lock, a deadlock does not seem possible.
Reviewers: Jun Rao <junrao@gmail.com>
* Fixed DataException thrown when handling tombstone events with null value
* Passes through original record when finding a null key when it's configured for keys or a null value when it's configured for values.
* Added unit tests for schema and schemaless data
The tasks `unitTest` and `integrationTest` used to run tests don't exclude
the ```**/*Suite``` so the tests included by Suite class are executed twice.
For example:
```
11:42:25 org.apache.kafka.streams.integration.StoreQuerySuite > org.apache.kafka.streams.integration.QueryableStateIntegrationTest.shouldBeAbleToQueryMapValuesState STARTED
11:42:26
11:42:26 org.apache.kafka.streams.integration.GlobalKTableIntegrationTest > shouldKStreamGlobalKTableJoin PASSED
11:42:30
11:42:30 org.apache.kafka.streams.integration.StoreQuerySuite > org.apache.kafka.streams.integration.QueryableStateIntegrationTest.shouldBeAbleToQueryMapValuesState PASSED
...
11:48:42 org.apache.kafka.streams.integration.QueryableStateIntegrationTest > shouldBeAbleToQueryMapValuesState STARTED
11:48:46
11:48:46 org.apache.kafka.streams.integration.QueryableStateIntegrationTest > shouldBeAbleToQueryMapValuesState PASSED
```
For consistency, move the existing exclusion for the `test` task from the
`streams` project to `subprojects`.
Reviewers: Ismael Juma <ismael@juma.me.uk>
In case of an error while flattening a record with schema, the Flatten transformation was reporting an error about a record without schema, as follows:
```
org.apache.kafka.connect.errors.DataException: Flatten transformation does not support ARRAY for record without schemas (for field ...)
```
The expected behaviour would be an error message specifying "with schemas".
This looks like a simple copy/paste typo from the schemaless equivalent methods, in the same file
Reviewers: Ewen Cheslack-Postava <me@ewencp.org>, Konstantine Karantasis <konstantine@confluent.io>
Simple doc fix in a code snippet in connect.html
Co-authored-by: Scott Ferguson <smferguson@gmail.com>
Reviewers: Ewen Cheslack-Postava <me@ewencp.org>, Konstantine Karantasis <konstantine@confluent.io>
Add missing @Override annotations and use lambdas when declaring threads to suppress warnings in IDEs and improve readability.
Reviewers: Ron Dagostino <rdagostino@confluent.io>, Konstantine Karantasis <konstantine@confluent.io>
This PR enhances the epoch checking logic for endTransaction call in TransactionCoordinator. Previously it relaxes the checking by allowing a producer epoch bump, which is error-prone since there is no reason to see a producer epoch bump from client.
Reviewers: Jason Gustafson <jason@confluent.io>
When a caching state store is closed it calls its flush() method.
If flush() throws an exception the underlying state store is not closed.
This commit ensures that state stores underlying a wrapped state stores
are closed even when preceding operations in the close method throw.
Co-authored-by: John Roesler <vvcephei@apache.org>
Reviewers: John Roesler <vvcephei@apache.org>, Guozhang Wang <wangguoz@gmail.com>, Matthias J. Sax <matthias@confluent.io>
For reasons outlined in https://issues.apache.org/jira/browse/KAFKA-9771
we can't upgrade to a version of Jetty with the bug fixed, or downgrade to one prior to the introduction of the bug. Luckily, the actual fix is pretty straightforward and can be ported over to Connect for use until it's possible to upgrade to a version of Jetty with that bug fixed: https://github.com/eclipse/jetty.project/pull/4404/files#diff-58640db0f8f2cd84b7e653d1c1540913R2188-R2193
The changes here have been verified locally; a test with multiple certificates/multiple hostnames will be submitted in a follow up.
Reviewers: Jeff Huang <47870461+jeffhuang26@users.noreply.github.com>, Konstantine Karantasis <konstantine@confluent.io>
* delete topics before tearing down multi-node clusters to avoid leader elections during shutdown
* tear down all nodes concurrently instead of sequentially
Reviewers: Matthias J. Sax <matthias@confluent.io>
1. Within a single while loop, process the tasks in AAABBBCCC instead of ABCABCABC. This also helps the follow-up PR to time the per-task processing ratio to record less time, hence less overhead.
2. Add thread-level process / punctuate / poll / commit ratio metrics.
3. Fixed a few issues discovered (inline commented).
Reviewers: John Roesler <vvcephei@apache.org>
1. Defaults state-change log level to INFO.
2. INFO level state-change log includes (a) request level logging with just partition counts; (b) the leader/isr changes per partition in the controller and in the broker (reduced to mostly just 1 logging per partition).
Reviewers: Jun Rao <junrao@gmail.com>
* KAFKA-9707: Fix InsertField.Key not applying to tombstone events
* Fix typo that hardcoded .value() instead of abstract operatingValue
* Add test for Key transform that was previously not tested
Signed-off-by: Greg Harris <gregh@confluent.io>
* Add null value assertion to tombstone test
* Remove mis-named function and add test for passing-through a null-keyed record.
Signed-off-by: Greg Harris <gregh@confluent.io>
* Simplify unchanged record assertion
Signed-off-by: Greg Harris <gregh@confluent.io>
* Replace assertEquals with assertSame
Signed-off-by: Greg Harris <gregh@confluent.io>
* Fix checkstyleTest indent issue
Signed-off-by: Greg Harris <gregh@confluent.io>
A write lock is currently taken out whenever an ACL update is triggered. This update requires a round trip to ZK to get the ACLs for the resource (https://github.com/apache/kafka/pull/7882/files#diff-852b9cb2ceb2b85ec25b422f72c42620R489). This round trip to ZK can block any ACL lookups, which will block any requests and that require authorization, and their corresponding handler threads. This PR attempts to avoid these read locks by snapshotting the aclCache which is a threadsafe scala immutable.TreeMap.
Author: Lucas Bradstreet <lucas@confluent.io>
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk
Closes#7882 from lbradstreet/acl-read-lock
KAFKA-7283 enabled lazy mmap on index files by initializing indices
on-demand rather than performing costly disk/memory operations when
creating all indices on broker startup. This helped reducing the startup
time of brokers. However, segment indices are still created on closing
segments, regardless of whether they need to be closed or not.
This is a cleaned up version of #7900, which was submitted by @efeg. It
eliminates unnecessary disk accesses and memory map operations while
deleting, renaming or closing offset and time indexes.
In a cluster with 31 brokers, where each broker has 13K to 20K segments,
@efeg and team observed up to 2 orders of magnitude faster LogManager
shutdown times - i.e. dropping the LogManager shutdown time of each
broker from 10s of seconds to 100s of milliseconds.
To avoid confusion between `renameTo` and `setFile`, I replaced the
latter with the more restricted updateParentDir` (it turns out that's
all we need).
Reviewers: Jun Rao <junrao@gmail.com>, Andrew Choi <a24choi@edu.uwaterloo.ca>
Co-authored-by: Adem Efe Gencer <agencer@linkedin.com>
Co-authored-by: Ismael Juma <ismael@juma.me.uk>
This PR works to improve high watermark checkpointing performance.
`ReplicaManager.checkpointHighWatermarks()` was found to be a major contributor to GC pressure, especially on Kafka clusters with high partition counts and low throughput.
Added a JMH benchmark for `checkpointHighWatermarks` which establishes a
performance baseline. The parameterized benchmark was run with 100, 1000 and
2000 topics.
Modified `ReplicaManager.checkpointHighWatermarks()` to avoid extra copies and cached
the Log parent directory Sting to avoid frequent allocations when calculating
`File.getParent()`.
A few clean-ups:
* Changed all usages of Log.dir.getParent to Log.parentDir and Log.dir.getParentFile to
Log.parentDirFile.
* Only expose public accessor for `Log.dir` (consistent with `Log.parentDir`)
* Removed unused parameters in `Partition.makeLeader`, `Partition.makeFollower` and `Partition.createLogIfNotExists`.
Benchmark results:
| Topic Count | Ops/ms | MB/sec allocated |
|-------------|---------|------------------|
| 100 | + 51% | - 91% |
| 1000 | + 143% | - 49% |
| 2000 | + 149% | - 50% |
Reviewers: Lucas Bradstreet <lucas@confluent.io>. Ismael Juma <ismael@juma.me.uk>
Co-authored-by: Gardner Vickers <gardner@vickers.me>
Co-authored-by: Ismael Juma <ismael@juma.me.uk>