Currently, commitTransaction and abortTransaction wait indefinitely for the respective operation to be completed. This patch uses the producer's max block time to limit the time that we will wait. If the timeout elapses, we raise a TimeoutException, which allows the user to either close the producer or retry the operation.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Jason Gustafson <jason@confluent.io>
Use of `MetadataRequest.isAllTopics` is not consistently defined for all versions of the api. For v0, it evaluates to false. This patch makes the behavior consistent for all versions.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Since we are logging offset resets and such at info level, it makes sense to use the same level for subscriptions and assignments.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Removed quotes from LogDir variable generation as there are additional quotes in Line 127.
This caused problems when those batch files are invoked from a path that contains space characters.
Per the KIP-263 discussion, we think we can improve broker restart time by avoiding performing costly disk operations when sanity checking index files for segments below recovery point on broker startup.
This PR includes the following changes:
1. Mmap the index file and populate fields of the index file on-demand rather than performing costly disk operations when creating the index object on broker startup.
2. Skip sanity checks on the time index and offset index of segments.
1. For segment with offset below the flushed point (recovery point), these segments are safely flushed so we don't need to sanity check the index files. if there are indeed data corruption on disk, given that we don't sanity check the segment file, sanity checking only the indexes adds little benefit.
2. For segment with offset above the flushed point (recovery point), we will recover these segments in `recoveryLog()` (Log.scala) in any case so sanity checking the index files for these segments is redundant.
We did experiments on a cluster with 15 brokers, each of which has ~3k segments (and there are 31.8k partitions with RF=3 which are evenly distributed across brokers; total bytes-in-rate is around 400 MBps). The results show that rolling bounce time reduces from 135 minutes to 55 minutes.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
Implemented an in-memory window store allowing for range queries. A finite retention period defines how long records will be kept, ie the window of time for fetching, and the grace period defines the window within which late-arriving data may still be written to the store.
Unit tests were written to test the functionality of the window store, including its insert/update/delete and fetch operations. Single-record, all records, and range fetch were tested, for both time ranges and key ranges. The logging and metrics for late-arriving (dropped)records were tested as well as the ability to restore from a changelog.
Reviewers: John Roesler <john@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
This PR fixes the issue found in the soak testing cluster regarding using RocksDBTimestampedStore when a regular RocksDB store should have been used.
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Matthias J. Sax <mjsax@apache.org>, John Roesler <john@confluent.io>
The StreamsBrokerBounceTest.test_broker_type_bounce experienced what looked like a transient failure. After looking over this test and failure, it seems like it is vulnerable to timing error that streams will start before the kafka service creates all topics.
Reviewers: Matthias J. Sax <mjsax@apache.org>, John Roesler <john@confluent.io>
Fail produce requests using zstd until the inter.broker.protocol.version is large enough that replicas are ensured to support it. Otherwise, followers receive the `UNSUPPORTED_COMPRESSION_TYPE` when fetching zstd data and ISRs shrink.
Reviewers: Jason Gustafson <jason@confluent.io>
Fix the following situations, where pending members (one that has a member-id, but hasn't joined the group) can cause rebalance operations to fail:
- In AbstractCoordinator, a pending consumer should be allowed to leave.
- A rebalance operation must successfully complete if a pending member either joins or times out.
- During a rebalance operation, a pending member must be able to leave a group.
Reviewers: Boyang Chen <bchen11@outlook.com>, Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
- Compare last offset of first batch (instead of first offset) with index offset
- Early exit from loop due to zero entries must happen before checking for mismatch
- {TimeIndex,OffsetIndex}.entry should return absolute offset like other methods.
These methods are only used by DumpLogSegments.
- DumpLogSegments now calls `closeHandlers` on OffsetIndex, TimeIndex
and FileRecords.
- Add OffsetIndex, TimeIndex and DumpLogSegments tests
- Remove unnecessary casts by using covariant returns in OffsetIndex and TimeIndex
- Minor clean-ups
- Fix `checkArgs` so that it does what it says only.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Sriharsha Chintalapani <sriharsha@apache.org>
Replaced `forall` with `exists`. Added a unit test to `KafkaApisTest` that failed before the change.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
We identified that we spend a lot of time in the creation of Logger instances
when creating OffsetIndex/TimeIndex due to the Logging mixin.
When the broker is bootstrapping it's just doing this in a tight loop, so the
time adds up.
This patch moves the logger to the companion objects of OffsetIndex,
TimeIndex and AbstractIndex resolving this issue.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Co-authored-by: Kyle Ambroff <kyle@ambroff.com>
Co-authored-by: Ismael Juma <ismael@juma.me.uk>
When debugging KafkaConsumer production issues, it's pretty
useful to have log entries related to seeking and committed
offset retrieval enabled by default. These are currently present,
but only when debug logging is enabled. Change them to `info`.
Also included a minor code simplication and a slight improvement
to an exception message.
Reviewers: Jason Gustafson <jason@confluent.io>
The config docs for message.format.version and log.message.format.version show invalid (corrupt?) "valid values". The problem is that`ApiVersionValidator#toString` is missing. In contrast, all other Validators like `ThrottledReplicaListValidator` or `Range`, have its own `toString` method. This patch solves this problem by adding `ApiVersionValidator#toString`. It also provides a unit test for it.
Reviewers: Jason Gustafson <jason@confluent.io>
* MINOR: add test for StreamsSmokeTestDriver
Hi @bbejeck@mjsax@ableegoldman@guozhangwang ,
Please take a look at this when you get the chance.
The primary concern is adding the test. It will help us verify changes to the smoke test (such as adding suppression).
I've also added some extra output to the smoke test stdout, which will hopefully aid us in diagnosing the flaky tests.
Finally, I bundled in some cleanup. It was my intention to do that in a separate PR, but it wound up getting smashed together during refactoring.
Please let me know if you'd prefer for me to pull any of these out into a separate request.
Thanks,
-John
Also, add more output for debuggability
* cleanup
* cleanup
* refactor
* refactor
* remove redundant printlns
* Update EmbeddedKafkaCluster.java
* move to integration package
* replace early-exit on pass
* use classrule for embedded kafka
* pull in smoke test improvements from side branch
* try-with-resources
* format events instead of printing long lines
* minor formatting fix
Reviewers: Matthias J. Sax <mjsax@apache.org>, Bill Bejeck <bbejeck@gmail.com>
- Add missing string interpolation
- Fix and simplify testElectPreferredLeaders
- Remove unused code
- Replace deprecated usage of JUnit `assertThat`
- Change var to val and fix non-exhaustive pattern match
- Fix eta warning
- Simplify code
- Remove commented out code
Reviewers: Jun Rao <junrao@gmail.com>
The change-logging stores should not bypass methods in underlying stores.
If some of you have a minute, can you take a quick look at this? I happened to notice during some other refactoring that the change-logging store layer sometimes bypasses the underlying store and instead calls across to a different layer.
It seems unexpected that it should do so, and it might actually cause problems. There was one spot where it's impossible to avoid it (in the windowed store), but I added a note justifying why we bypass the underlying store.
Thanks,
-John
* MINOR: fix bypasses in ChangeLogging stores
* fix test
Reviewers: Guozhang Wang <wangguoz@gmail.com>, Matthias J. Sax <mjsax@apache.org>, Bill Bejeck <bbejeck@gmail.com>
It used to preallocate an array of responses and then complete each response from the original collection sequentially. The problem was that the original collection could have been modified (another thread completing the response) while this was hapenning
Avoid unnecessary lock acquire when KafkaConsumer commits offsets.
Reviewers: Viktor Somogyi-Vass <viktorsomogyi@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
Refactor internal store wrapping for improved maintainability.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <guozhang@confluent.io>
- Update the AclCommandTest
Author: Manikumar Reddy <manikumar.reddy@gmail.com>
Reviewers: Jun Rao <junrao@gmail.com>
Closes#6263 from omkreddy/aclcommand
This condition is a fatal error, so error level is warranted, to provide more context on why Streams shuts down.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <guozhang@confluent.io>
As part of KIP-320, we should add the expected leader epoch to Fetch and ListOffsets requests. This will allow us ultimately to detect log truncation.
Reviewers: Jason Gustafson <jason@confluent.io>
1. In the caching layer's flush listener call, we should always write to the underlying store, before flushing (see #4331 's point 4) for detailed explanation). When fixing 4331, it only touches on KV stores, but it turns out that we should fix for window and session store as well.
2. Also apply the optimization that was in session-store already: when the new value bytes and old value bytes are all null (this is possible e.g. if there is a put(K, V) followed by a remove(K) or put(K, null) and these two operations only hit the cache), upon flushing this mean the underlying store does not have this value at all and also no intermediate value has been sent to downstream as well. We can skip both putting a null to the underlying store as well as calling the flush listener sending `null -> null` in this case.
Modifies corresponding unit tests.
Reviewers: John Roesler <john@confluent.io>, Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>
The test `org.apache.kafka.connect.runtime.rest.RestServerTest#testCORSEnabled` assumes Jersey client can send restricted HTTP headers(`Origin`).
Jersey client uses `sun.net.www.protocol.http.HttpURLConnection`.
`sun.net.www.protocol.http.HttpURLConnection` drops restricted headers(`Host`, `Keep-Alive`, `Origin`, etc) based on static property `allowRestrictedHeaders`.
This property is initialized in a static block by reading Java system property `sun.net.http.allowRestrictedHeaders`.
So, if classloader loads `HttpURLConnection` before we set `sun.net.http.allowRestrictedHeaders=true`, then all subsequent changes of this system property won't take any effect(which happens if `org.apache.kafka.connect.integration.ExampleConnectIntegrationTest` is executed before `RestServerTest`).
To prevent this, we have to either make sure we set `sun.net.http.allowRestrictedHeaders=true` as early as possible or do not rely on this system property at all.
This PR adds test dependency on `httpcomponents-client` which doesn't depend on `sun.net.http.allowRestrictedHeaders` system property. Thus none of existing tests should interfere with `RestServerTest`.
Author: Alex Diachenko <sansanichfb@gmail.com>
Reviewers: Randall Hauch, Konstantine Karantasis, Gwen Shapira
Closes#6236 from avocader/KAFKA-7799
- Include more detail in the client log message if the disconnection happens
during authentication.
- Include exception message in the Selector info entry when authentication
fails and unwrap `DelayedResponseAuthenticationException`.
- Remove duplicate debug log on authentication failure in the Selector.
Empty username or password would result in the "expected 3 tokens"
error instead of "username not specified" or "password not specified".
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Replace `Assert.assertThat` with `MatcherAssert.assertThat`.
Two commits (08036fa4b1 and c7f99bc2b) were merged at a similar time and
they passed the build in isolation, but not together.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Ismael Juma <ismael@juma.me.uk>
JUnit 4.13 fixes the issue where `Category` and `Parameterized` annotations
could not be used together. It also deprecates `ExpectedException` and
`assertThat`. Given this, we:
- Replace `ExpectedException` with the newly introduced `assertThrows`.
- Replace `Assert.assertThat` with `MatcherAssert.assertThat`.
- Annotate `AbstractLogCleanerIntegrationTest` with `IntegrationTest` category.
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>, David Arthur <mumrah@gmail.com>
This patch adds 2 props to the log4j kafka appender that get put directly
into the sasl properties passed to the producer:
- ClientJaasConf: This property sets sasl.jaas.config
- SaslMechanim: This property sets sasl.mechanism
Author: Rohan Desai <desai.p.rohan@gmail.com>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#6216 from rodesai/add-kafka-appender-security-props
* KAFKA-7236: Add --under-min-isr option to describe topics command (KIP-351)
* Minor changes to description and make test consistent with others
* Fix option, and add additional test with mixed partition status
* Add fully-replicated-topic to test case
* Address review nits
Don't return error messages from `SaslException` to clients. Error messages to be returned to clients to aid debugging must be thrown as AuthenticationExceptions. This is a fix for a regression from KAFKA-7352.
Reviewers: Ron Dagostino <rndgstn@gmail.com>, Ismael Juma <ismael@juma.me.uk
This helps narrow down the specific broker they came from when debugging
ACL propagation issues.
Reviewers: Vahid Hashemian <vahid.hashemian@gmail.com>, Ismael Juma <ismael@juma.me.uk>
This PR addressed the following test failure:
```
java.lang.AssertionError:
Expected: a string starting with "process-state-manager-test Failed to write offset checkpoint file to ["
but: was "[AdminClient clientId=adminclient-874] Connection to node -1 (localhost/127.0.0.1:8080) could not be established. Broker may not be available."
```
Reviewers: Bill Bejeck <bill@confluent.io>, John Roesler <john@confluent.io>, Guozhang Wang <guozhang@confluent.io>