KIP-368 implementation to enable periodic re-authentication of SASL clients. Also adds a broker configuration option to terminate client connections that do not re-authenticate within the configured interval.
Just a doc change
Author: John Eismeier <john.eismeier@gmail.com>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4573 from jeis2497052/trunk
Decrease the lower bound for expected available memory, as thread
scheduling entails that a variable amount of deallocation happens by
the point of assertion.
Also make minor clarifications to test logic and comments.
The passing rate improved from 98% to 100% locally after these
changes (100+ runs).
Reviewers: Ismael Juma <ismael@juma.me.uk>
### Committer Checklist (excluded from commit message)
- [ ] Verify design and implementation
- [ ] Verify test coverage and CI build status
- [ ] Verify documentation (including upgrade notes)
Reduce tick interval of the mock timer and avoid large timer increments to avoid hitting idle expiry on the client-side before delayed close is processed by the server. Also reduce poll interval in the server to make the test complete faster (since delayed close is only processed when poll returns).
Reviewers: Ismael Juma <ismael@juma.me.uk>
Add CreatePartitionsRequest.PartitionDetails similar to CreateTopicsRequest.TopicDetails to avoid references from `common.requests` package to `clients`.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Update `ClientUtilsTest.testParseAndValidateAddressesWithReverseLookup` test to work in environments where ipv6 is not enabled and `InetAddress.getAllByName` doesn't return ipv6 addresses.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Adds `client.dns.lookup=resolve_canonical_bootstrap_servers_only` option to perform full dns resolution of bootstrap addresses
Reviewers: Colin Patrick McCabe <colin@cmccabe.xyz>, Sriharsha Chintalapani <sriharsha@apache.org>, Edoardo Comar <ecomar@uk.ibm.com>, Mickael Maison <mickael.maison@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
Implementation of KIP-302: Based on the new client configuration `client.dns.lookup`, a NetworkClient can use InetAddress.getAllByName to find all IPs and iterate over them when they fail to connect. Only uses either IPv4 or IPv6 addresses similar to the default mode.
Co-authored-by: Edoardo Comar <ecomar@uk.ibm.com>
Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
- SslFactoryTest should use SslFactory to create SSLEngine
- Use Mockito instead of EasyMock in `ConsoleConsumerTest` as one of
the tests mocks a standard library class and the latest released EasyMock
version can't do that when Java 11 is used.
- Avoid mocking `ConcurrentMap` in `SourceTaskOffsetCommitterTest`
for similar reasons. As it happens, mocking is not actually needed here.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Development of EasyMock and PowerMock has stagnated while Mockito
continues to be actively developed. With the new Java release cadence,
it's a problem to depend on libraries that do bytecode manipulation
and are not actively maintained. In addition, Mockito is also
easier to use.
While updating the tests, I attempted to go from failing test to
passing test. In cases where the updated test passed on the first
attempt, I artificially broke it to ensure the test was still doing its
job.
I included a few improvements that were helpful while making these
changes:
1. Better exception if there are no nodes in `leastLoadedNodes`
2. Always close the producer in `KafkaProducerTest`
3. requestsInFlight producer metric should not hold a reference to
`Sender`
Finally, `Metadata` is no longer final so that we don't need
`PowerMock` to mock it. It's an internal class, so it's OK.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Viktor Somogyi <viktorsomogyi@gmail.com>, Dong Lin <lindong28@gmail.com>
Closes#5691 from ijuma/kafka-7438-mockito
OAuthBearerLoginModule is used both on the server-side and client-side (similar to login modules for other mechanisms). OAUTHBEARER tokens are client credentials used only on the client-side to authenticate with servers, but the current implementation requires tokens to be provided on the server-side even if OAUTHBEARER is not used for inter-broker communication. This commit makes tokens optional for server-side login context to allow brokers to be configured without a token when OAUTHBEARER is not used for inter-broker communication.
Reviewers: Ron Dagostino <rndgstn@gmail.com>, Jun Rao <junrao@gmail.com>
This patch contains the broker-side support for the fencing improvements from KIP-320. This includes the leader epoch validation in the ListOffsets, OffsetsForLeaderEpoch, and Fetch APIs as well as the changes needed in the fetcher threads to maintain and use the current leader epoch. The client changes from KIP-320 will be left for a follow-up.
One notable change worth mentioning is that we now require the read lock in `Partition` in order to read from the log or to query offsets. This is necessary to ensure the safety of the leader epoch validation. Additionally, we forward all leader epoch changes to the replica fetcher thread and go through the truncation phase. This is needed to ensure the fetcher always has the latest epoch and to guarantee that we cannot miss needed truncation if we missed an epoch change.
Reviewers: Jun Rao <junrao@gmail.com>
This patch adds checks before reading the first record of a control batch. If the batch is empty, it is treated as having already been cleaned. In the case of LogCleaner this means it is safe to discard. In the case of ProducerStateManager it means it shouldn't cause state to be stored because the relevant transaction has already been cleaned. In the case of Fetcher, it just preempts the check for an abort. In the case of GroupMetadataManager, it doesn't process the offset as a commit. The patch also adds isControl to the output of DumpLogSegments. Changes were tested with unit tests, except the DumpLogSegments change which was tested manually.
`ConsumerCoordinator.onJoinPrepare()` currently makes multiple copies of the set of assigned partitions. We can let `subscriptions.assignedPartitions()` return a view of the underlying partition set, copy it only once and re-use the copied value.
Author: radai-rosenblatt <radai.rosenblatt@gmail.com>
Reviewers: Ismael Juma <ismael@juma.me.uk>, Dong Lin <lindong28@gmail.com>
Closes#5124 from radai-rosenblatt/copy-all-the-things
Lazy allocation helps when there are a large number of connections
that have been accepted, but where no data has been received from
the clients. Each buffer is often around 16k (max TLS record size).
Nulling the buffers should not make a difference in the current
implementation since we release the reference to the channel
and transport layer after we close them, but it's a good practice
to release medium/large buffers after `close` is called.
Reviewers: Jun Rao <junrao@gmail.com>, Ismael Juma <ismael@juma.me.uk>
This patch implements KIP-336. It adds a default implementation to the Serializer/Deserializer interface to support the use of headers and it deprecates the ExtendedSerializer and ExtendedDeserializer interfaces for later removal.
Reviewers: Satish Duggana <sduggana@hortonworks.com>, John Roesler <john@confluent.io>, Jason Gustafson <jason@confluent.io>
What changes were proposed in this pull request?
atLeast(0) in StreamsConfig, ProducerConfig and ConsumerConfig were replaced by SEND_BUFFER_LOWER_BOUND and RECEIVE_BUFFER_LOWER_BOUND from CommonClientConfigs.
How was this patch tested?
Three unit tests were added to KafkaStreamsTest
Reviewers: Guozhang Wang <guozhang@confluent.io>, John Roesler <john@confluent.io>, Matthias J. Sax <mjsax@apache.org>
This patch fixes unsafe concurrent access in the consumer by the heartbeat thread and the thread calling `poll()` to the fetch session state in `FetchSessionHandler`.
Reviewers: Viktor Somogyi <viktorsomogyi@gmail.com>, Jason Gustafson <jason@confluent.io>
* Refactor the StreamThread main loop, in the following:
1. Fetch from consumer and enqueue data to tasks.
2. Check if any tasks should be enforced process.
3/ Loop over processable tasks and process them for N iterations, and then check for 1) commit, 2) punctuate, 3) need to call consumer.poll
4. Even if there is not data to process in this iteration, still need to check if commit / punctuate is needed
5. Finally, try update standby tasks.
*Add an optimization to only commit when it is needed (i.e. at least some process() or punctuate() was triggered since last commit).
*Found and fixed a ProducerFencedException scenario: producer.send() call would never throw a ProducerFencedException directly, but it may throw a KafkaException whose "cause" is a ProducerFencedException.
Reviewers: Matthias J. Sax <matthias@confluent.io>, John Roesler <john@confluent.io>, Bill Bejeck <bill@confluent.io>
A call to `kafka-consumer-groups --describe --group ...` can result in NullPointerException for two reasons:
1) `Fetcher.fetchOffsetsByTimes()` may return too early, without sending list offsets request for topic partitions that are not in cached metadata.
2) `ConsumerGroupCommand.getLogEndOffsets()` and `getLogStartOffsets()` assumed that endOffsets()/beginningOffsets() which eventually call Fetcher.fetchOffsetsByTimes(), would return a map with all the topic partitions passed to endOffsets()/beginningOffsets() and that values are not null. Because of (1), null values were possible if some of the topic partitions were already known (in metadata cache) and some not (metadata cache did not have entries for some of the topic partitions). However, even with fixing (1), endOffsets()/beginningOffsets() may return a map with some topic partitions missing, when list offset request returns a non-retriable error. This happens in corner cases such as message format on broker is before 0.10, or maybe in cases of some other errors.
Testing:
-- added unit test to verify fix in Fetcher.fetchOffsetsByTimes()
-- did some manual testing with `kafka-consumer-groups --describe`, causing NPE. Was not able to reproduce any NPE cases with DescribeConsumerGroupTest.scala,
Reviewers: Jason Gustafson <jason@confluent.io>
With KIP-320, the OffsetsForLeaderEpoch API is intended to be used by consumers to detect log truncation. Therefore the new response schema should expose a field for the throttle time like all the other APIs.
Reviewers: Dong Lin <lindong28@gmail.com>
findBugs is abandoned, it doesn't work with Java 9 and the Gradle plugin will be deprecated in
Gradle 5.0: https://github.com/gradle/gradle/pull/6664
spotBugs is actively maintained and it supports Java 8, 9 and 10. Java 11 is not supported yet,
but it's likely to happen soon.
Also fixed a file leak in Connect identified by spotbugs.
Manually tested spotBugsMain, jarAll and importing kafka in IntelliJ and running
a build in the IDE.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Colin Patrick McCabe <colin@cmccabe.xyz>, Dong Lin <lindong28@gmail.com>
Closes#5625 from ijuma/kafka-5887-spotbugs
[KAFKA-4932](https://issues.apache.org/jira/browse/KAFKA-4932)
Added a UUID Serializer / Deserializer.
Added the UUID type to the SerializationTest
Author: Brandon Kirchner <brandon.kirchner@civitaslearning.com>
Reviewers: Jeff Klukas <jeff@klukas.net>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4438 from brandonkirchner/KAFKA-4932.uuid-serde
This patch contains the protocol updates needed for KIP-320 as well as some of the basic consumer APIs (e.g. `OffsetAndMetadata` and `ConsumerRecord`). The inter-broker format version has not been changed and the brokers will continue to use the current API versions.
Author: Jason Gustafson <jason@confluent.io>
Reviewers: Dong Lin <lindong28@gmail.com>
Closes#5564 from hachikuji/KAFKA-7333
With idempotent/transactional producers, we may leave empty batches in the log during log compaction. When filtering the data, we keep track of state like `maxOffset` and `maxTimestamp` of filtered data. This patch ensures we maintain this state correctly for the case when only empty batches are left in `MemoryRecords#filterTo`. Without this patch, we did not initialize `maxOffset` in this edge case which led us to append data to the log with `maxOffset` = -1L, causing the append to fail and log cleaner to crash.
Reviewers: Jason Gustafson <jason@confluent.io>
This patch removes the duplication of the out of range handling between `ReplicaFetcherThread` and `ReplicaAlterLogDirsThread` and attempts to expose a cleaner API for extension. It also adds a mock implementation to facilitate testing and several new test cases.
Reviewers: Jun Rao <junrao@gmail.com>
We should retry when possible if ListGroups fails due to a retriable error (e.g. coordinator loading).
Reviewers: Colin Patrick McCabe <colin@cmccabe.xyz>, Guozhang Wang <wangguoz@gmail.com>
Pull the epoch request build logic up to `AbstractFetcherThread`. Also get rid of the `FetchRequest` indirection.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Rajini Sivaram <rajinisivaram@googlemail.com>
Java 11 supports TLS 1.3 which has different cipher names than
previous TLS versions so the simplistic way of choosing ciphers
is not guaranteed to work. Fix it by configuring the context
to use TLS 1.2.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
Set empty extensions if null is passed in.
Reviewers: Satish Duggana <sduggana@hortonworks.com>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Rajini Sivaram <rajinisivaram@googlemail.com>