This patch adds a basic downgrade system test. It verifies that producing and consuming continues to work before and after the downgrade.
Reviewers: Ismael Juma <ismael@juma.me.uk>, David Arthur <mumrah@gmail.com>
Fixes#7717, which did not actually achieve its intended effect. The event manager failed to process the new event because we disabled the rate metric, which it expected to be present.
Reviewers: Ismael Juma <ismael@juma.me.uk
For some reason, PR builds are failing due to the `rat` license
check even though it should ignore files included in `.gitignore`.
Reviewers: Jason Gustafson <jason@confluent.io>
Given we need to follow the Apache rule of not checking
any binaries into the source code, Kafka has always had
a bit of a tricky Gradle bootstrap.
Using ./gradlew as users expect doesn’t work and a
local and compatible version of Gradle was required to
generate the wrapper first.
This patch changes the behavior of the wrapper task to
instead generate a gradlew script that can bootstrap the
jar itself. Additionally it adds a license, removes the bat
script, and handles retries.
The documentation in the readme was also updated.
Going forward patches that upgrade gradle should run
`gradle wrapper` before checking in the change.
With this change users using ./gradlew can be sure they
are always building with the correct version of Gradle.
Reviewers: Viktor Somogyi <viktorsomogyi@gmail.com>, Ismael Juma <ismael@juma.me.uk
Create a controller event for handling UpdateMetadata responses and log a message when a response contains an error.
Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Ismael Juma <ismael@juma.me.uk>
Recently, system tests test_rebalance_[simple|complex] failed
repeatedly with a verfication error. The cause was most probably
the missing clean-up of a state directory of one of the processors.
A node is cleaned up when a service on that node is started and when
a test is torn down.
If the clean-up flag clean_node_enabled of a EOS Streams service is
unset, the clean-up of the node is skipped.
The clean-up flag of processor1 in the EOS tests should stay set before
its first start, so that the node is cleaned before the service is started.
Afterwards for the multiple restarts of processor1 the cleans-up flag should
be unset to re-use the local state.
After the multiple restarts are done, the clean-up flag of processor1 should
again be set to trigger node clean-up during the test teardown.
A dirty node can lead to test failures when tests from Streams EOS tests are
scheduled on the same node, because the state store would not start empty
since it reads the local state that was not cleaned up.
Reviewers: Matthias J. Sax <mjsax@apache.org>, Andrew Choi <andchoi@linkedin.com>, Bill Bejeck <bbejeck@gmail.com>
Allow null as a valid default for tagged fields. Fix a bunch of cases where this would previously result in null pointer dereferences.
Also allow inferring FieldSpec#versions based on FieldSpec#taggedVersions. Prefix 'key' with an underscore when it is used in the generated code, to avoid potential name collisions if someone names an RPC field "key".
Allow setting setting hexadecimal constants and 64-bit contstants.
Add a lot more test cases to SimpleExampleMessage.json.
Reviewers: Jason Gustafson <jason@confluent.io>
The latter was previously hardcoding logDirCount instead of using the
method defined in the superclass since it was unnecessarily duplicating
logic.
Also tweak IntegrationTestHarness and remove unnecessary method
override from SaslPlainPlaintextConsumerTest.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
After a number of last minute bugs were found stemming from the incremental closing of lost tasks in StreamsRebalanceListener#onPartitionsLost, a safer approach to this edge case seems warranted. We initially wanted to be as "future-proof" as possible, and avoid baking further protocol assumptions into the code that may be broken as the protocol evolves. This meant that rather than simply closing all active tasks and clearing all associated state in #onPartitionsLost(lostPartitions) we would loop through the lostPartitions/lost tasks and remove them one by one from the various data structures/assignments, then verify that everything was empty in the end. This verification in particular has caused us significant trouble, as it turns out to be nontrivial to determine what should in fact be empty, and if so whether it is also being correctly updated.
Therefore, before worrying about it being "future-proof" it seems we should make sure it is "present-day-proof" and implement this callback in the safest possible way, by blindly clearing and closing all active task state. We log all the relevant state (at debug level) before clearing it, so we can at least tell from the logs whether/which emptiness checks were being violated.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Bill Bejeck <bbejeck@gmail.com>, Andrew Choi <andchoi@linkedin.com>
Refactors metrics according to KIP-444
Introduces ProcessorNodeMetrics as a central provider for processor node metrics
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Bill Bejeck <bbejeck@gmail.com>
* Add rate limiting to tc
* Feedback from PR
* Add a sanity test for tc
* Add iperf to vagrant scripts
* Dynamically determine the network interface
* Add some temp code for testing on AWS
* Temp: use hostname instead of external IP
* Temp: more AWS debugging
* More AWS WIP
* More AWS temp
* Lower latency some
* AWS wip
* Trying this again now that ping should work
* Add cluster decorator to tests
* Fix broken import
* Fix device name
* Fix decorator arg
* Remove errant import
* Increase timeouts
* Fix tbf command, relax assertion on latency test
* Fix log line
* Final bit of cleanup
* Newline
* Revert Trogdor retry count
* PR feedback
* More PR feedback
* Feedback from PR
* Remove unused argument
This patch removes the explicit version check pattern we used in `getErrorResponse`, which is a pain to maintain (as seen by KAFKA-9200). We already check that requests have a valid version range in the `AbstractRequest` constructor.
Reviewers: Andrew Choi <andrewchoi5@users.noreply.github.com>, Ismael Juma <ismael@juma.me.uk>
Force completion of delayed operations when receiving a StopReplica request. In the case of a partition reassignment, we cannot rely on receiving a LeaderAndIsr request in order to complete these operations because the leader may no longer be a replica. Previously when this happened, the delayed operations were left to eventually timeout.
Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Ismael Juma <ismael@juma.me.uk>
Co-Authored-By: Kun Du <kidkun@users.noreply.github.com>
While investigating KAFKA-9180, I noticed that we had no
unit test coverage. It turns out that the behavior was
correct, so we just fix the test coverage issue.
Also updated .gitignore with jmh-benchmarks/generated.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
ListOffsetResponse getErrorResponse is missing a a case for version 5, introduced
by 152292994e and released in 2.3.0.
```
java.lang.IllegalArgumentException: Version 5 is not valid. Valid versions for ListOffsetRequest are 0 to 5
at org.apache.kafka.common.requests.ListOffsetRequest.getErrorResponse(ListOffsetRequest.java:282)
at kafka.server.KafkaApis.sendErrorOrCloseConnection(KafkaApis.scala:3062)
at kafka.server.KafkaApis.sendErrorResponseMaybeThrottle(KafkaApis.scala:3045)
at kafka.server.KafkaApis.handleError(KafkaApis.scala:3027)
at kafka.server.KafkaApis.handle(KafkaApis.scala:209)
at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:78)
at java.lang.Thread.run(Thread.java:748)
```
Reviewers: Ismael Juma <ismael@juma.me.uk>
When we roll a new segment, the log offset metadata tied to the high watermark may
need to be updated. This is needed when the high watermark is equal to the log end
offset at the time of the roll. Otherwise, we risk exposing uncommitted data early.
Reviewers: Dhruvil Shah <dhruvil@confluent.io>, Ismael Juma <ismael@juma.me.uk>
Author: Arvind Thirunarayanan <athirunar@confluent.io>
Reviewers: Colin Patrick McCabe <colin@cmccabe.xyz>
Closes#7694 from arvindth/ath-debugdeps-task
This patch creates a `BaseAdminIntegrationTest` to be the root for all integration test extensions. Most of the existing tests will only be tested in `PlaintextAdminIntegrationTest`, which extends from `BaseAdminIntegrationTest`. This should cut off about 30 minutes from the overall build time.
Reviewers: David Arthur <mumrah@gmail.com>, Ismael Juma <ismael@juma.me.uk>
When Trogdor wants to clear all the faults injected to Kibosh, it sends the empty JSON object {}. However, Kibosh expects {"faults":[]} instead. Kibosh should handle the empty JSON object, since that's consistent with how Trogdor handles empty JSON fields in general (if they're empty, they can be omitted). We should also have a test for this.
Reviewers: David Arthur <mumrah@gmail.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
The current Utils::readFileAsString method creates a FileChannel and
memory maps file and copies its content to a String and returns it. But
that means that we need to know the size of the file in advance. This
precludes us from reading files whose size is not known in advance, i.e.
any file opened with flag S_IFIFO.
This change updates the method to use stream to read the content of the file.
It has couple of practical advantages:
Allows bash process substitution to pass in strings as file. So we can
say `./bin/kafka-reassign-partitions.sh --reassignment-json-file <(echo
"reassignment json")
When adding systest for commands that take file, we don't have to
create real physical file. Instead we can just dump the content of the
file on the command line.
Reviewers: Ismael Juma <ismael@juma.me.uk>
This is a followup PR for #7520 to address issue of multiple calls to get() as it was pointed out by @bbejeck in #7520 (comment)
Reviewers: Bill Bejeck <bbejeck@gmail.com>
Explicitly mention that max.request.size validates uncompressed record sizes and max.message.bytes validates compressed record sizes.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Sometimes to be backwards compatible regarding metrics the simplest
solution is to create an empty sensor. Recording an empty sensor on
the hot path may negatively impact performance. With hasMetrics()
recordings of empty sensors on the hot path can be avoided without
being to invasive.
Reviewers: Bill Bejeck <bbejeck@gmail.com>
Fixed a small typo on the Processor API page of the Kafka Streams developer guide docs. ("buildeer" changed to "builder")
Reviewers: Bill Bejeck <bbejeck@gmail.com>
[Jira](https://issues.apache.org/jira/browse/KAFKA-9046)
The changes here are meant to find a healthy compromise between the pre- and post-KIP-458 functionality of Connect workers when configuring admin clients for use with DLQs. Before KIP-458, admin clients were configured using the top-level worker configs; after KIP-458, they are configured using worker configs with a prefix of `admin.` and then optionally overridden by connector configs with a prefix of `admin.override.`. The behavior proposed here is to use, in ascending order of precedence, the top-level worker configs, worker configs prefixed with `admin.`, and connector configs prefixed with `admin.override.`; essentially, use the pre-KIP-458 behavior by default but allow it to be overridden by the post-KIP-458 behavior.
Author: Chris Egerton <chrise@confluent.io>
Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Randall Hauch <rhauch@gmail.com>, Nigel Liang <nigel@nigelliang.com>
Closes#7525 from C0urante/kafka-9046
Fixes `java.util.concurrent.ExecutionException: java.lang.AssertionError: Close finished too quickly 5999`.
The close test sets a close duration in milliseconds, but measures the time taken in nanoseconds. This leads to small error due to the resolution in each, where the close is deemed to have taken too little time.
When I measured the start and end with nanoTime, I found the time taken to close was `5999641566 ns (5999.6ms)` which seems close enough to be a resolution error. I've run the test 50 times and have not hit the "Close finished too quickly" issue again, whereas previously I hit a failure pretty quickly.
Author: Lucas Bradstreet <lucas@confluent.io>
Reviewers: Ismael Juma <ismael@juma.me.uk>
Closes#7683 from lbradstreet/flaky-consumer-bounce-test
Instead of caching the checkpoint map during StandbyTask
initialization, use the latest checkpoints (which would have
been updated during suspend).
Reviewers: Bill Bejeck <bill@confluent.io>
This patch implements the broker-side changes for KIP-360. It adds two new fields to InitProducerId: lastEpoch and producerId. Passing these values allows the TransactionCoordinator to safely bump a producer's epoch after some failures (such as UNKNOWN_PRODUCER_ID and INVALID_PRODUCER_ID_MAPPING). When a producer calls InitProducerId after a failure, the coordinator first checks the producer ID from the request to make sure no other producer has been started using the same transactional ID. If it is safe to continue, the coordinator checks the epoch from the request; if it matches the existing epoch, the epoch is bumped and the producer can safely continue. If it matches the previous epoch, the the current epoch is returned without bumping. Otherwise, the producer is fenced.
Reviewers: Boyang Chen <boyang@confluent.io>, Jason Gustafson <jason@confluent.io>
This change fixes a performance regression due to follower last seen highwatermark
handling introduced in 23beeea. maybeUpdateHwAndSendResponse is expensive for
brokers with high partition counts, as it requires a partition and a replica lookup for every
partition being fetched. This refactor moves the last seen watermark update into the follower
fetch state update where we have already looked up the partition and replica.
I've seen cases where maybeUpdateHwAndSendResponse is responsible 8% of CPU usage, not including the responseCallback call that is part of it.
I have benchmarked this change with `UpdateFollowerFetchStateBenchmark` and it adds 5ns
of overhead to Partition.updateFollowerFetchState, which is a rounding error compared to the
current overhead of maybeUpdateHwAndSendResponse.
Reviewers: David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>
7.3.1.1 JAAS configuration for Kafka brokers was followed by
7.3.1.4 JAAS configuration for Kafka clients instead of 7.3.1.2.
Reviewers: Mickael Maison <mickael.maison@gmail.com>