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>
When an older message format is in use, we should disable the leader epoch cache so that we resort to truncation by high watermark. Previously we updated the cache for all versions when a broker became leader for a partition. This can cause large and unnecessary truncations after leader changes because we relied on the presence of _any_ cached epoch in order to tell whether to use the improved truncation logic possible with the OffsetsForLeaderEpoch API.
Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>, Jun Rao <junrao@gmail.com>
Replaces original loginContext if login fails in the refresh thread to ensure that the refresh thread is left in a clean state when there are exceptions while connecting to an OAuth server. Also makes client callback handler more robust by using the token with the longest remaining time for expiry instead of throwing an exception if multiple tokens are found.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Add logic in ConsumerBounceTest to check the error code in FindCoordinator responses and retry if needed. This should help with transient failures or at least get us closer to the actual problem.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Allow the Trogdor agent to execute external commands. The agent communicates with the external commands via stdin, stdout, and stderr.
Based on a patch by Xi Yang <xi@confluent.io>
Reviewers: David Arthur <mumrah@gmail.com>
* Enable heap dumps on OOM with -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=<file.bin> in the major services in system tests
* Collect the heap dump from the predefined location as part of the result logs for each service
* Change Connect service to delete the whole root directory instead of individual expected files
* Tested by running the full suite of system tests
Author: Konstantine Karantasis <konstantine@confluent.io>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#6158 from kkonstantine/KAFKA-7834
Rather than using sed, use built-in Python regular expressions to strip
the SNAPSHOT expression from the pom.xml files. Sed has different flags
on different platforms, such as Linux. Using Python directly here is
more compatible, as well as being more efficient, and not requiring an
rm command afterwards.
When running release_notes.py, use the current Python interpreter.
This is needed to prevent attempting to run release_notes.py with
Python 3 on some systems. release_notes.py will not (yet) work with
Python 3.
Author: Colin P. Mccabe <cmccabe@confluent.io>
Reviewers: Magnus Edenhill <magnus@edenhill.se>, David Arthur <mumrah@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#6198 from cmccabe/release_py
This PR adds a upgrade notes and changes examples to use the bootstrap-server.
Author: Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
Reviewers: Srinivas <srinivas96alluri@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#6118 from viktorsomogyi/topiccommand-adminclient-doc
The default backoff of 1000ms when there are no partitions to fetch can cause `shouldExecuteThrottledReassignment` to fail due to it taking too long. So we reduce
it to 100ms.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>, Ismael Juma <ismael@juma.me.uk
Split the Gradle invocation in the jenkins.sh script into two commands so
we can fail fast for validation checks such as compile errors and checkstyle
errors.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Reviewers: Damian Guy <damian@confluent.io>, John Roesler <john@confluent.io>, Boyang Chen <bchen11@outlook.com>, Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>
This patch introduces a new config - "group.max.size", which caps the maximum size any group can reach. It has a default value of Int.MAX_VALUE. Once a group is of the maximum size, subsequent JoinGroup requests receive a MAX_SIZE_REACHED error.
In the case where the config is changed and a Coordinator broker with the new config loads an old group that is over the threshold, members are kicked out of the group and a rebalance is forced.
Reviewers: Vahid Hashemian <vahid.hashemian@gmail.com>, Boyang Chen <bchen11@outlook.com>, Gwen Shapira <cshapi@gmail.com>, Jason Gustafson <jason@confluent.io>
The PR adds --bootstrap-server and --admin.config options to TopicCommand and implements an alternative, AdminClient based way of topic management.
As testing I've duplicated the existing tests and made them working with the AdminClient options.
Author: Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
Reviewers: Andras Katona <41361962+akatona84@users.noreply.github.com>, Sandor Murakozi <smurakozi@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Jason Gustafson <jason@confluent.io>
Closes#5683 from viktorsomogyi/topiccommand-adminclient
Explicitly seek KafkaBasedLog’s consumer to the beginning of the topic partitions, rather than potentially use committed offsets (which would be unexpected) if group.id is set or rely upon `auto.offset.reset=earliest` if the group.id is null.
This should not change existing behavior but should remove some potential issues introduced with KIP-287 if `group.id` is not set in the consumer configurations. Note that even if `group.id` is set, we still always want to consume from the beginning.
Reviewers: Jason Gustafson <jason@confluent.io>
Limit the number of new connections processed in each iteration of each
Processor. Block Acceptor if the connection queue is full on all Processors.
Added a metric to track accept blocked time percent. See KIP-402 for details.
Reviewers: Ismael Juma <ismael@juma.me.uk>