Skip the partition incurring ZooKeeper exception when becoming leader or follower.
Reviewers: Joel Koshy <jjkoshy@gmail.com>, Jun Rao <junrao@gmail.com>
Users often get confused after an unclean shutdown when log recovery takes a long time. This patch attempts to make the logging clearer and provide a simple indication of loading progress.
Reviewers: Ismael Juma <ismael@juma.me.uk>
The documentation for max.block.ms said it affected only send()
and partitionsFor(), but it actually also affects initTransactions(),
abortTransaction() and commitTransaction(). So rework the
documentation to cover these methods too.
Reviewers: Boyang Chen <boyang@confluent.io>
There are two new configs introduced by 371f14c3c1 and 1c4eb1a575 so we have to update the expected configs in the connect_rest_test.py system test too.
Reviewer: Konstantine Karantasis <konstantine@confluent.io>
This PR includes 3 MessageFormatters for MirrorMaker2 internal topics:
- HeartbeatFormatter
- CheckpointFormatter
- OffsetSyncFormatter
This also introduces a new public interface org.apache.kafka.common.MessageFormatter that users can implement to build custom formatters.
Reviewers: Konstantine Karantasis <k.karantasis@gmail.com>, Ryanne Dolan <ryannedolan@gmail.com>, David Jacot <djacot@confluent.io>
Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
Co-authored-by: Edoardo Comar <ecomar@uk.ibm.com>
Replaces the previous upgrade test's trivial Streams app
with the commonly used SmokeTest, exercising many more
features. Also adjust the test matrix to test upgrading
from each released version since 2.2 to the current branch.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
1. return the topicsNotReady to makeReady including tempUnknownTopics, and not create topic to wait for next retry
2. tempUnknownTopics will be created each retry since we count the tempUnknownTopics as part of topicsNotReady
3. add 2 more tests to total test 3 cases:
3.1 shouldCreateTopicWhenTopicLeaderNotAvailableAndThenTopicNotFound
3.2 shouldCompleteValidateWhenTopicLeaderNotAvailableAndThenDescribeSuccess
3.3 shouldThrowExceptionWhenKeepsTopicLeaderNotAvailable
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Boyang Chen <boyang@confluent.io>
This includes important fixes. Netty is required by ZooKeeper if TLS is
enabled.
I verified that the netty jars were changed from 4.1.48 to 4.1.50 with
this PR, `find . -name '*netty*'`:
```text
./core/build/dependant-libs-2.13.3/netty-handler-4.1.50.Final.jar
./core/build/dependant-libs-2.13.3/netty-transport-native-epoll-4.1.50.Final.jar
./core/build/dependant-libs-2.13.3/netty-codec-4.1.50.Final.jar
./core/build/dependant-libs-2.13.3/netty-transport-native-unix-common-4.1.50.Final.jar
./core/build/dependant-libs-2.13.3/netty-transport-4.1.50.Final.jar
./core/build/dependant-libs-2.13.3/netty-resolver-4.1.50.Final.jar
./core/build/dependant-libs-2.13.3/netty-buffer-4.1.50.Final.jar
./core/build/dependant-libs-2.13.3/netty-common-4.1.50.Final.jar
```
Note that the previous netty exclude no longer worked since we upgraded
to ZooKeeper 3.5.x as it switched to Netty 4 which has different module names.
Also, the Netty dependency is needed by ZooKeeper for TLS support so we
cannot exclude it.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Improved the exception messages that are thrown to indicate whether it was a key or value conversion problem.
Author: Mario Molina <mmolimar@gmail.com>
Reviewer: Randall Hauch <rhauch@gmail.com>
Added a section about error reporting in Connect documentation, and another about how to safely use the new errant record reporter in SinkTask implementations.
Author: Aakash Shah <ashah@confluent.io>
Reviewer: Randall Hauch <rhauch@gmail.com>
After 3661f981fff2653aaf1d5ee0b6dde3410b5498db security_config is cached. Hence, the later changes to security flag can't impact the security_config used by later tests.
issue: https://issues.apache.org/jira/browse/KAFKA-10214
Author: Chia-Ping Tsai <chia7712@gmail.com>
Reviewers: Ron Dagostino <rdagostino@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#8949 from chia7712/KAFKA-10214
During Streams' system tests the PIDs of the Streams
clients are collected. The method the collects the PIDs
swallows any exception that might be thrown by the
ssh_capture() function. Swallowing any exceptions
might make the investigation of failures harder,
because no information about what happened are recorded.
Reviewers: John Roesler <vvcephei@apache.org>
Fix findbugs multithreaded correctness warnings for streams, updated variables to be threadsafe
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Boyang Chen <boyang@confluent.io>, John Roesler <vvcephei@apache.org>
It's currently not possible to unit-test custom processors that use windowed stores,
because the provided windowed store implementations cast the context to
InternalProcessorContext.
This change adds a public API example using windowed stores, and fixes the
casts internally that would make that example fail previously.
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Bruno Cadonna <bruno@confluent.io>
Call Producer#flush to make sure all records are indeed sent "synchronously" when EOS is not enabled in the OptimizedKTableIntegrationTest#shouldApplyUpdatesToStandbyStore.
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Boyang Chen <boyang@confluent.io>
Since https://issues.apache.org/jira/browse/KAFKA-8834, describing topics with the TopicCommand requires privileges to use ListPartitionReassignments or fails to describe the topics with the following error:
> Error while executing topic command : Cluster authorization failed.
This is a quite hard restriction has most of the secure clusters do not authorize non admin members to access ListPartitionReassignments.
This patch catches the `ClusterAuthorizationException` exception and gracefully fails back. We already do this when the API is not available so it remains consistent.
Author: David Jacot <djacot@confluent.io>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#8947 from dajac/KAFKA-10212
A simple increase in the timeout of the consumer that verifies that records have been replicated seems to fix the integration tests in `MirrorConnectorsIntegrationTest` that have been failing more often recently.
Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Sanjana Kaundinya <skaundinya@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>
Recently, commit 492306a updated both jetty to version 9.4.27.v20200227 and jersey to version 2.31
However in the latest versions of jetty, the renaming of the method `Response#closeOutput` to `Response#completeOutput` has been reverted, with the latest version using again `Response#closeOutput`.
Jersey has not released a recent version in which `Response#closeOutput` is called directly. In its currently latest version (2.31) `Response#closeOutput` will be called if `Response#completeOutput` throws a `NoSuchMethodError` exception. Given that, this version combination is compatible. Jersey should be upgraded once a new version that uses `Response#closeOutput` directly is out.
Reviewers: Ismael Juma <ismael@juma.me.uk>
We inadvertently changed the binary schema of the suppress buffer changelog
in 2.4.0 without bumping the schema version number. As a result, it is impossible
to upgrade from 2.3.x to 2.4+ if you are using suppression.
* Refactor the schema compatibility test to use serialized data from older versions
as a more foolproof compatibility test.
* Refactor the upgrade system test to use the smoke test application so that we
actually exercise a significant portion of the Streams API during upgrade testing
* Add more recent versions to the upgrade system test matrix
* Fix the compatibility bug by bumping the schema version to 3
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
This should address at least some of the excessive TaskCorruptedExceptions we've been seeing lately. Basically, at the moment we only commit tasks if commitNeeded is true -- this seems obvious by definition. But the problem is we do some essential cleanup in postCommit that should always be done before a task is closed:
* clear the PartitionGroup
* write the checkpoint
The second is actually fine to skip when commitNeeded = false with ALOS, as we will have already written a checkpoint during the last commit. But for EOS, we only write the checkpoint before a close -- so even if there is no new pending data since the last commit, we have to write the current offsets. If we don't, the task will be assumed dirty and we will run into our friend the TaskCorruptedException during (re)initialization.
To fix this, we should just always call prepareCommit and postCommit at the TaskManager level. Within the task, it can decide whether or not to actually do something in those methods based on commitNeeded.
One subtle issue is that we still need to avoid checkpointing a task that was still in CREATED, to avoid potentially overwriting an existing checkpoint with uninitialized empty offsets. Unfortunately we always suspend a task before closing and committing, so we lose the information about whether the task as in CREATED or RUNNING/RESTORING by the time we get to the checkpoint. For this we introduce a special flag to keep track of whether a suspended task should actually be checkpointed or not
Reviewers: Guozhang Wang <wangguoz@gmail.com>
The enum ```State``` is private so it is fine to fix typo without breaking compatibility.
Author: Chia-Ping Tsai <chia7712@gmail.com>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#8932 from chia7712/MINOR-8932
I had to fix several compiler errors due to deprecation of auto application of `()`. A related
Xlint config (`-Xlint:nullary-override`) is no longer valid in 2.13, so we now only enable it
for 2.12. The compiler flagged two new inliner warnings that required suppression and
the semantics of `&` in `@nowarn` annotations changed, requiring a small change in
one of the warning suppressions.
I also removed the deprecation of a number of methods in `KafkaZkClient` as
they should not have been deprecated in the first place since `KafkaZkClient` is an
internal class and we still use these methods in the Controller and so on. This
became visible because the Scala compiler now respects Java's `@Deprecated`
annotation.
Finally, I included a few minor clean-ups (eg using `toBuffer` instead `toList`) when fixing
the compilation warnings.
Noteworthy bug fixes in Scala 2.13.3:
* Fix 2.13-only bug in Java collection converters that caused some operations to perform an extra pass
* Fix 2.13.2 performance regression in Vector: restore special cases for small operands in appendedAll and prependedAll
* Increase laziness of #:: for LazyList
* Fixes related to annotation parsing of @Deprecated from Java sources in mixed compilation
Full release notes:
https://github.com/scala/scala/releases/tag/v2.13.3
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
In order to make the Kafka consumer and stream application migrate from source to target cluster
transparently and conveniently, e.g. in event of source cluster failure, a background job is proposed
to periodically sync the consumer offsets from the source to target cluster, so that when the
consumer and stream applications switche to the target cluster, they will resume to consume from
where they left off at source cluster.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ryanne Dolan <ryannedolan@gmail.com>, Thiago Pinto, Srinivas Boga
Minor cleanup on streams internal classes, with diamond class removal and long function signature breakdown.
Reviewers: Boyang Chen <boyang@confluent.io>, John Roesler <john@confluent.io>
We just needed to add the check in StreamTask#closeClean to closeAndRecycleState as well. I also renamed closeAndRecycleState to closeCleanAndRecycleState to drive this point home: it needs to be clean.
This should be cherry-picked back to the 2.6 branch
Reviewers: Matthias J. Sax <matthias@confluent.io>, John Roesler <john@confluent.io>, Guozhang Wang <wangguoz@gmail.com>,
If there's any pending data and we haven't flushed the producer when we abort a transaction, a KafkaException is returned for the previous send. This is a bit misleading, since the situation is not an unrecoverable error and so the Kafka Exception is really non-fatal. For now, we should just catch and swallow this in the RecordCollector (see also: KAFKA-10169)
The reason we ended up aborting an un-flushed transaction was due to the combination of
a. always aborting the ongoing transaction when any task is closed/revoked
b. only committing (and flushing) if at least one of the revoked tasks needs to be committed (regardless of whether any non-revoked tasks have data/transaction in flight)
Given the above, we can end up with an ongoing transaction that isn't committed since none of the revoked tasks have any data in the transaction. We then abort the transaction anyway, when those tasks are closed. So in addition to the above (swallowing this exception), we should avoid unnecessarily aborting data for tasks that haven't been revoked.
We can handle this by splitting the RecordCollector's close into a dirty and clean flavor: if dirty, we need to abort the transaction since it may be dirty due to the commit attempt failing. But if clean, we can skip aborting the transaction since we know that either we just committed and thus there is no ongoing transaction to abort, or else the transaction in flight contains no data from the tasks being closed
Note that this means we still abort the transaction any time a task is closed dirty, so we must close/reinitialize any active task with pending data (that was aborted).
In sum:
* hackily check the KafkaException message and swallow
* only abort the transaction during a dirty close
* refactor shutdown to make sure we don't closeClean a task whose data was actually aborted
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Boyang Chen <boyang@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
ConsumerPerformance has not implemented options numThreadsOpt and numFetchersOpt as so far.
This patch adds a warning message when used these options according to comments from
https://issues.apache.org/jira/browse/KAFKA-10126 . Once these two options are implemented,
this warning message should be removed.
Reviewers: Boyang Chen <boyang@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
Add unit tests for KafkaProducer.close(), KafkaProducer.abortTransaction(), and KafkaProducer.flush() in the KafkaProducerTest.
Increase KafkaProducer test code coverage from 82% methods, 82% lines to 86% methods, 87% lines when being merged.
Reviewers: Boyang Chen <boyang@confluent.io>
Looks it is a typo, the actual key supposed to be this #replicaFetchWaitMaxTimeMs(replica.fetch.wait.max.ms) instead of that the docs have this #replicaMaxWaitTimeMs
Author: sbellapu <sbellapu@visa.com>
Author: sbellapu <satishbabumsc@gmail.com>
Reviewers: Matthias J. Sax <matthias@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#8877 from satishbellapu/trunk
Ports the test from #8886 to trunk -- this should be merged to 2.6 branch.
One open question. In 2.6 and trunk we rely on the active tasks to wipe out the store if it crashes. However, assume there is a hard JVM crash and we don't call closeDirty() the store would not be wiped out. Thus, I am wondering, if we would need to fix this (for both active and standby tasks) and do a check on startup if a local store must be wiped out?
The current test passes, as we do a proper cleanup after the exception is thrown.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
Most builds don't require test coverage output, so it's wasteful
to spend cycles tracking coverage information for each method
invoked.
I ran a quick test in a fast desktop machine, the absolute
difference will be larger in a slower machine. The tests were
executed after `./gradlew clean` and with a gradle daemon
that was started just before the test (and mildly warmed up
by running `./gradlew clean` again).
`./gradlew unitTest --continue --profile`:
* With coverage enabled: 6m32s
* With coverage disabled: 5m47s
I ran the same test twice and the results were within 2s of
each other, so reasonably consistent.
16% reduction in the time taken to run the unit tests is a
reasonable gain with little downside, so I think this is a
good change.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
It is helpful to include as much information as possible when deleting log segments. This patch introduces log messages that give more specific details as to why the log segment was deleted and the specific metadata regarding that log segment.
Reviewers: Jason Gustafson <jason@confluent.io>
This patch fixes a bug in the constructor of `LogTruncationException`. We were passing the divergent offsets to the super constructor as the fetch offsets. There is no way to fix this without breaking compatibility, but the harm is probably minimal since this exception was not getting raised properly until KAFKA-9840 anyway.
Note that I have also moved the check for unknown offset and epoch into `SubscriptionState`, which ensures that the partition is still awaiting validation and that the fetch offset hasn't changed. Finally, I made some minor improvements to the logging and exception messages to ensure that we always have the fetch offset and epoch as well as the divergent offset and epoch included.
Reviewers: Boyang Chen <boyang@confluent.io>, David Arthur <mumrah@gmail.com>