Previously, we had some code hard-coded to generate message type classes
for RPCs. We might want to generate message type classes for other
things as well, so make it more generic.
Reviewers: Boyang Chen <boyang@confluent.io>
If we call org.apache.kafka.common.utils.Exit#exit(int code) with code=0, the current implementation will go into an infinite recursion and kill the VM with a stack overflow error. This happens only in integration tests because of the overrides of shutdown procedures and this commit addresses this issue by removing the redundant call to Exit#exit.
Reviewers: Lucas Bradstreet <lucas@confluent.io>, Konstantine Karantasis <k.karantasis@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
The `testBlockInConnectorStop` test is failing semi-frequently on Jenkins. It's difficult to verify the cause without complete logs and I'm unable to reproduce locally, but I suspect the cause may be that the Connect worker hasn't completed startup yet by the time the test begins and so the initial REST request to create a connector times out with a 500 error. This isn't an issue for normal tests but we artificially reduce the REST request timeout for these tests as some requests are meant to exhaust that timeout.
The changes here use a small hack to verify that the worker has started and is ready to handle all types of REST requests before tests start by querying the REST API for a non-existent connector.
Reviewers: Boyang Chan <boyang@confluent.io>, Konstantine Karantasis <k.karantasis@gmail.com>
A benign `WakeupException` can be thrown by a sink task's consumer if the task is scheduled for shutdown by the worker. This is caught and handled gracefully if the exception is thrown when calling `poll` on the consumer, but not if calling `commitSync`, which is invoked by a task during shutdown and also when its partition assignment is updated.
If thrown during a partition assignment update, the `WakeupException` is caught and handled gracefully as part of the task's `iteration` loop. If thrown during shutdown, however, it is not caught and instead leads to the misleading log message "Task threw an uncaught and unrecoverable exception. Task is being killed and will not recover until manually restarted.".
These changes catch the `WakeupException` during shutdown and handle it gracefully with a `TRACE`-level log message.
A unit test is added to verify this behavior by simulating a thrown `WakeupException` during `Consumer::commitSync`, running through the `WorkerSinkTask::execute` method, and confirming that it does not throw a `WakeupException` itself.
Reviewers: Greg Harris <gregh@confluent.io>, Nigel Liang <nigel@nigelliang.com>, Konstantine Karantasis <k.karantasis@gmail.com>
Update checkpoint files once for all deleted partitions instead of updating them for each deleted partitions. With this, a stop replica requests with 2000 partitions to be deleted takes ~2 secs instead of ~40 secs previously.
Refactor the checkpointing methods to not compute the logsByDir all the time. It is now reused as much as possible.
Refactor the exception handling. Some checkpointing methods were handling IOException but the underlying write process already catches them and throws KafkaStorageException instead.
Reduce the logging in the log cleaner manager. It does not log anymore when a partition is deleted as it is not a useful information.
Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
Also piggy-back a small fix to use TreeMap other than HashMap to preserve iteration ordering.
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, John Roesler <vvcephei@apache.org>
Fixes an asymmetry in which we avoid writing checkpoints for non-persistent stores, but still expected to read them, resulting in a spurious TaskCorruptedException.
Reviewers: Matthias J. Sax <mjsax@apache.org>, John Roesler <vvcephei@apache.org>
- Currently we create single channel builder and reuse it in ConnectStressor workload. This will fail when testing with secure connections, as we close channel builder after first connection. This PR creates ChannelBuilder for each test connection.
- Also increase to connect ready wait timeout to 500ms.
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#8937 from omkreddy/Connect
Until now we always passed the default changelog topic name
to the state serdes. However, for optimized source tables
and global tables the changelog topic is the source topic.
Most serdes do not use the topic name passed to them.
However, if the serdes actually use the topic name for
(de)serialization a
org.apache.kafka.common.errors.SerializationException is thrown.
This commits passed the correct changelog topic to the state
serdes of the metered state stores.
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Matthias J. Sax <matthias@confluent.io>, John Roesler <vvcephei@apache.org>
Reducing timeout of transaction to clean up the unstable offsets quicker. IN hard_bounce mode, transactional client is killed ungracefully. Hence, it produces unstable offsets which obstructs TransactionalMessageCopier from receiving position of group.
Reviewers: Jun Rao <junrao@gmail.com>
Author: Tom Bentley <tbentley@redhat.com>
Reviewers: David Jacot <djacot@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#8808 from tombentley/KAFKA-10109-AclComment-multiple-AdminClients
Add null check for configurationKey to avoid NPE, and add test for it.
Author: Luke Chen <showuon@gmail.com>
Reviewers: Tom Bentley <tbentley@redhat.com>, huxi <huxi_2b@hotmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#8966 from showuon/KAFKA-10220
The intention of using poll(0) is to not block on rebalance but still return some data; however, `updateAssignmentMetadataIfNeeded` have three different logic: 1) discover coordinator if necessary, 2) join-group if necessary, 3) refresh metadata and fetch position if necessary. We only want to make 2) to be non-blocking but not others, since e.g. when the coordinator is down, then heartbeat would expire and cause the consumer to fetch with timeout 0 as well, causing unnecessarily high CPU.
Since splitting this function is a rather big change to make as a last minute blocker fix for 2.6, so I made a smaller change to make updateAssignmentMetadataIfNeeded has an optional boolean flag to indicate if 2) above should wait until either expired or complete, otherwise do not wait on the join-group future and just poll with zero timer.
Reviewers: Jason Gustafson <jason@confluent.io>
Call KafkaStreams#cleanUp to reset local state before starting application up the second run.
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Boyang Chen <boyang@confluent.io>, John Roesler <john@confluent.io>
This PR fixes a bug introduced in #8683.
While processing connection set up timeouts, we are iterating through the connecting nodes to process timeouts and we disconnect within the loop, removing the entry from the set in the loop that it iterating over the set. That raises a ConcurrentModificationException exception. The current unit test did not catch this because it was using only one node.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Log it as a warning and without a stacktrace (instead of error with stacktrace). This error can be seen in the
following cases:
* Topic creation, a follower broker of a new partition starts replica fetcher before the prospective leader broker
of the new partition receives the leadership information from the controller (see KAFKA-6221).
* Topic deletion, a follower broker of a to-be-deleted partition starts replica fetcher after the leader broker of the
to-be-deleted partition processes the deletion information from the controller.
As expected, clusters with frequent topic creation and deletion report UnknownTopicOrPartitionException with
relatively higher frequency.
Despite typically being a transient issue, UnknownTopicOrPartitionException may also indicate real issues if it
doesn't fix itself after a short period of time. To ensure detection of such scenarios, we set the log level to warn
instead of info.
Reviewers: Jun Rao <junrao@gmail.com>, Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>
Two more edge cases I found producing extra TaskcorruptedException while playing around with the failing eos-beta upgrade test (sadly these are unrelated problems, as the test still fails with these fixes in place).
* Need to write the checkpoint when recycling a standby: although we do preserve the changelog offsets when recycling a task, and should therefore write the offsets when the new task is itself closed, we do NOT write the checkpoint for uninitialized tasks. So if the new task is ultimately closed before it gets out of the CREATED state, the offsets will not be written and we can get a TaskCorruptedException
* We do not write the checkpoint file if the current offset map is empty; however for eos the checkpoint file is not only used for restoration but also for clean shutdown. Although skipping a dummy checkpoint file does not actually violate any correctness since we are going to re-bootstrap from the log-start-offset anyways, it throws unnecessary TaskCorruptedException which has an overhead itself.
Reviewers: John Roesler <vvcephei@apache.org>, Guozhang Wang <wangguoz@gmail.com>
Most of the values in the metadata upgrade test matrix are just testing
the upgrade/downgrade path between two previous releases. This is
unnecessary. We run the tests for all supported branches, so what we
should test is the up-/down-gradability of released versions with respect
to the current branch.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
The current failures we're seeing with this test are due to faulty assumptions that it makes and not any real bug in eos-beta (at least, from what I've seen so far).
The test relies on tightly controlling the commits, which it does by setting the commit interval to MAX_VALUE and manually requesting commits on the context. In two phases, the test assumes that any pending data will be committed after a rebalance. But we actually take care to avoid unnecessary commits -- with eos-alpha, we only commit tasks that are revoked while in eos-beta we must commit all tasks if any are revoked, but only if the revoked tasks themselves need a commit.
The failure we see occurs when we try to verify the committed data after a second client is started and the group rebalances. The already-running client has to give up two tasks to the newly started client, but those tasks may not need to be committed in which case none of the tasks would be. So we still have an open transaction on the partitions where we try to read committed data.
Reviewers: John Roesler <john@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Without this change, we would catch the NPE and log it.
This was misleading and could cause excessive log
volume.
The NPE can happen after `AlterReplicaLogDirs` completes
successfully and when unmapping older regions. Example
stacktrace:
```text
[2019-05-20 14:08:13,999] ERROR Error unmapping index /tmp/kafka-logs/test-0.567a0d8ff88b45ab95794020d0b2e66f-delete/00000000000000000000.index (kafka.log.OffsetIndex)
java.lang.NullPointerException
at org.apache.kafka.common.utils.MappedByteBuffers.unmap(MappedByteBuffers.java:73)
at kafka.log.AbstractIndex.forceUnmap(AbstractIndex.scala:318)
at kafka.log.AbstractIndex.safeForceUnmap(AbstractIndex.scala:308)
at kafka.log.AbstractIndex.$anonfun$closeHandler$1(AbstractIndex.scala:257)
at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251)
at kafka.log.AbstractIndex.closeHandler(AbstractIndex.scala:257)
at kafka.log.AbstractIndex.deleteIfExists(AbstractIndex.scala:226)
at kafka.log.LogSegment.$anonfun$deleteIfExists$6(LogSegment.scala:597)
at kafka.log.LogSegment.delete$1(LogSegment.scala:585)
at kafka.log.LogSegment.$anonfun$deleteIfExists$5(LogSegment.scala:597)
at kafka.utils.CoreUtils$.$anonfun$tryAll$1(CoreUtils.scala:115)
at kafka.utils.CoreUtils$.$anonfun$tryAll$1$adapted(CoreUtils.scala:114)
at scala.collection.immutable.List.foreach(List.scala:392)
at kafka.utils.CoreUtils$.tryAll(CoreUtils.scala:114)
at kafka.log.LogSegment.deleteIfExists(LogSegment.scala:599)
at kafka.log.Log.$anonfun$delete$3(Log.scala:1762)
at kafka.log.Log.$anonfun$delete$3$adapted(Log.scala:1762)
at scala.collection.Iterator.foreach(Iterator.scala:941)
at scala.collection.Iterator.foreach$(Iterator.scala:941)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
at scala.collection.IterableLike.foreach(IterableLike.scala:74)
at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
at kafka.log.Log.$anonfun$delete$2(Log.scala:1762)
at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
at kafka.log.Log.maybeHandleIOException(Log.scala:2013)
at kafka.log.Log.delete(Log.scala:1759)
at kafka.log.LogManager.deleteLogs(LogManager.scala:761)
at kafka.log.LogManager.$anonfun$deleteLogs$6(LogManager.scala:775)
at kafka.utils.KafkaScheduler.$anonfun$schedule$2(KafkaScheduler.scala:114)
at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:63)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
```
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Co-authored-by: Jaikiran Pai <jaikiran.pai@gmail.com>
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 371f14c3c12d2e341ac96bd52393b43a10acfa84 and 1c4eb1a5757df611735cfac9b709e0d80d0da4b3 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>