- After #8312, older brokers are returning empty configs, with latest `adminClient.describeConfigs`. Old brokers are receiving empty configNames in `AdminManageer.describeConfigs()` method. Older brokers does not handle empty configKeys. Due to this old brokers are filtering all the configs.
- Update ClientCompatibilityTest to verify describe configs
- Add test case to test describe configs with empty configuration Keys
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#9046 from omkreddy/KAFKA-9432
add a timeout for event queue time histogram;
reset eventQueueTimeHist when the controller event queue is empty;
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Igor Soarez <i@soarez.me>, Jun Rao <junrao@gmail.com>
Add docs for KIP-441 and KIP-613.
Fixed some miscellaneous unrelated issues in the docs:
* Adds some missing configs to the Streams config docs: max.task.idle.ms,topology.optimization, default.windowed.key.serde.inner.class, and default.windowed.value.serde.inner.class
* Defines the previously-undefined default windowed serde class configs, including choosing a default (null) and giving them a doc string, so the yshould nwo show up in the auto-generated general Kafka config docs
* Adds a note to warn users about the rocksDB bug that prevents setting a strict capacity limit and counting write buffer memory against the block cache
Reviewers: Bruno Cadonna <bruno@confluent.io>, John Roesler <vvcephei@apache.org>
Currently, the system tests `connect_distributed_test` and `connect_rest_test` only wait for the REST api to come up.
The startup of the worker includes an asynchronous process for joining the worker group and syncing with other workers.
There are some situations in which this sync takes an unusually long time, and the test continues without all workers up.
This leads to flakey test failures, as worker joins are not given sufficient time to timeout and retry without waiting explicitly.
This changes the `ConnectDistributedTest` to wait for the Joined group message to be printed to the logs before continuing with tests. I've activated this behavior by default, as it's a superset of the checks that were performed by default before.
This log message is present in every version of DistributedHerder that I could find, in slightly different forms, but always with `Joined group` at the beginning of the log message. This change should be safe to backport to any branch.
Signed-off-by: Greg Harris <gregh@confluent.io>
Author: Greg Harris <gregh@confluent.io>
Reviewer: Randall Hauch <rhauch@gmail.com>
Brokers currently return NOT_LEADER_FOR_PARTITION to producers and REPLICA_NOT_AVAILABLE to consumers if a replica is not available on the broker during reassignments. Non-Java clients treat REPLICA_NOT_AVAILABLE as a non-retriable exception, Java consumers handle this error by explicitly matching the error code even though it is not an InvalidMetadataException. This PR renames NOT_LEADER_FOR_PARTITION to NOT_LEADER_OR_FOLLOWER and uses the same error for producers and consumers. This is compatible with both Java and non-Java clients since all clients handle this error code (6) as retriable exception. The PR also makes ReplicaNotAvailableException a subclass of InvalidMetadataException.
- ALTER_REPLICA_LOG_DIRS continues to return REPLICA_NOT_AVAILABLE. Retained this for compatibility since this request never returned NOT_LEADER_FOR_PARTITION earlier.
- MetadataRequest version 0 also returns REPLICA_NOT_AVAILABLE as topic-level error code for compatibility. Newer versions filter these out and return Errors.NONE, so didn't change this.
- Partition responses in MetadataRequest return REPLICA_NOT_AVAILABLE to indicate that one of the replicas is not available. Did not change this since NOT_LEADER_FOR_PARTITION is not suitable in this case.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>, Bob Barrett <bob.barrett@confluent.io>
Add log4j entry summarizing the assignment (previous owned and assigned) at the consumer level.
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Boyang Chen <boyang@confluent.io>
The test case `OffsetValidationTest.test_fencing_static_consumer` fails periodically due to this error:
```
Traceback (most recent call last):
File "/home/jenkins/workspace/system-test-kafka_2.6/kafka/venv/lib/python2.7/site-packages/ducktape-0.7.8-py2.7.egg/ducktape/tests/runner_client.py", line 134, in run
data = self.run_test()
File "/home/jenkins/workspace/system-test-kafka_2.6/kafka/venv/lib/python2.7/site-packages/ducktape-0.7.8-py2.7.egg/ducktape/tests/runner_client.py", line 192, in run_test
return self.test_context.function(self.test)
File "/home/jenkins/workspace/system-test-kafka_2.6/kafka/venv/lib/python2.7/site-packages/ducktape-0.7.8-py2.7.egg/ducktape/mark/_mark.py", line 429, in wrapper
return functools.partial(f, *args, **kwargs)(*w_args, **w_kwargs)
File "/home/jenkins/workspace/system-test-kafka_2.6/kafka/tests/kafkatest/tests/client/consumer_test.py", line 257, in test_fencing_static_consumer
assert len(consumer.dead_nodes()) == num_conflict_consumers
AssertionError
```
When a consumer stops, there is some latency between when the shutdown is observed by the service and when the node is added to the dead nodes. This patch fixes the problem by giving some time for the assertion to be satisfied.
Reviewers: Boyang Chen <boyang@confluent.io>
When fencing producers, we currently blindly bump the epoch by 1 and write an abort marker to the transaction log. If the log is unavailable (for example, because the number of in-sync replicas is less than min.in.sync.replicas), we will roll back the attempted write of the abort marker, but still increment the epoch in the transaction metadata cache. During periods of prolonged log unavailability, producer retires of InitProducerId calls can cause the epoch to be increased to the point of exhaustion, at which point further InitProducerId calls fail because the producer can no longer be fenced. With this patch, we track whenever we have failed to write the bumped epoch, and when that has happened, we don't bump the epoch any further when attempting to fence. This is safe because the in-memory epoch is still causes old producers to be fenced.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Boyang Chen <boyang@confluent.io>, Jason Gustafson <jason@confluent.io>
security_rolling_upgrade_test may change the security listener and then restart Kafka servers. has_sasl and has_ssl get out-of-date due to cached _security_config. This PR offers a simple fix that we always check the changes of port mapping and then update the sasl/ssl flag.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
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>