* 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>
During actions such as a reconfiguration, the task configs are obtained
via `Worker.connectorTaskConfigs` and then subsequently saved into an
instance of `ClusterConfigState`. The values of the properties that are saved
are post-transformation (of variable references) when they should be
pre-transformation. This is to avoid secrets appearing in plaintext in
the `connect-configs` topic, for example.
The fix is to change the 2 clients of `Worker.connectorTaskConfigs` to
perform a reverse transformation (values converted back into variable
references) before saving them into an instance of `ClusterConfigState`.
The 2 places where the save is performed are
`DistributedHerder.reconfigureConnector` and
`StandaloneHerder.updateConnectorTasks`.
The way that the reverse transformation works is by using the
"raw" connector config (with variable references still intact) from
`ClusterConfigState` to convert config values back into variable
references for those keys that are common between the task config
and the connector config.
There are 2 additional small changes that only affect `StandaloneHerder`:
1) `ClusterConfigState.allTasksConfigs` has been changed to perform a
transformation (resolution) on all variable references. This is
necessary because the result of this method is compared directly to
`Worker.connectorTaskConfigs`, which also has variable references
resolved.
2) `StandaloneHerder.startConnector` has been changed to match
`DistributedHerder.startConnector`. This is to fix an issue where
during `StandaloneHerder.restartConnector`, the post-transformed
connector config would be saved back into `ClusterConfigState`.
I also performed an analysis of all other code paths where configs are
saved back into `ClusterConfigState` and did not find any other
issues.
Author: Robert Yokota <rayokota@gmail.com>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5475 from rayokota/KAFKA-7242-reverse-xform-props
Sender/RecordAccumulator never resets the next batch expiry time. Its always computed as the min of the current value and the expiry time for all batches being processed. This means that its always set to the expiry time of the first batch, and once that time has passed Sender starts spinning on epoll with a timeout of 0, which consumes a lot of CPU. This patch updates Sender to reset the next batch expiry time on each poll loop so that a new value reflecting the expiry time for the current set of batches is computed.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
1) As titled, add a rewriteTopology that 1) sets application id, 2) maybe disable caching, 3) adjust for source KTable. This optimization can hence be applied for both DSL or PAPI generated Topology.
2) Defer the building of globalStateStores in rewriteTopology so that we can also disable caching. But we still need to build the state stores before InternalTopologyBuilder.build() since we should only build global stores once for all threads.
3) Added withCachingDisabled to StoreBuilder, it is a public API change.
4) [Optional] Fixed unit test config setting functionalities, and set the necessary config to shorten the unit test latency (now it reduces from 5min to 3.5min on my laptop).
Reviewers: Matthias J. Sax <matthias@confluent.io>, John Roesler <john@confluent.io>, Bill Bejeck <bill@confluent.io>, Ted Yu <yuzhihong@gmail.com>
We should check TxnOffsetCommit responses for the COORDINATOR_LOADING_IN_PROGRESS error code and retry if we see it. Additionally, if we encounter an abortable error, we need to ensure that pending transaction offset commits are cleared.
Reviewers: Viktor Somogyi <viktorsomogyi@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
Ensure that sends are completed before waiting for channel to be closed based on idle expiry, since channel will not be expired if added to ready keys in the next poll as a result of pending sends.
Reviewers: Jun Rao <junrao@gmail.com>
Use a volatile field to track the size of the set of assigned partitions to avoid the concurrent access to the underlying linked hash map.
Reviewers: Vahid Hashemian <vahidhashemian@us.ibm.com>, Rajini Sivaram <rajinisivaram@googlemail.com>, Ismael Juma <ismael@juma.me.uk>, Jason Gustafson <jason@confluent.io>
In AK's documentation, the config props for connectors are not listed (https://kafka.apache.org/documentation/#connectconfigs). This PR adds these sink and source connector configs to the html site-docs.
Signed-off-by: Arjun Satish <arjunconfluent.io>
Author: Arjun Satish <arjun@confluent.io>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5469 from wicknicks/add-connector-configs-to-docs
We currently do a lot of bookkeeping for timeouts which is both error-prone and distracting. This patch adds a new `Timer` class to simplify this logic and control unnecessary calls to system time. In particular, this helps with nested timeout operations. The consumer has been updated to use the new class.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <wangguoz@gmail.com>
Currently Fetcher.getTopicMetadata() will not include offline partitions. Thus
KafkaConsumer.partitionsFor(topic) will not return all partitions of a topic if
there if any partition of the topic is offline. This causes problem if user
tries to query the total number of partitions of the given topic.
Author: radai-rosenblatt <radai.rosenblatt@gmail.com>
Reviewers: Jason Gustafson <jason@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
Closes#4679 from radai-rosenblatt/partition_shenanigans
Fixed incorrect use of default timeout instead of the argument explicitly passed to `newClientRequest`.
Reviewers: Ron Dagostino <rndgstn@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Use delivery timeout instead of retries when possible and remove various TODOs associated with completion of KIP-91.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <wangguoz@gmail.com>
This patch forces metadata update for consumers with pattern subscription at the beginning of rebalance (retry.backoff.ms is respected). This is to prevent such consumers from detecting subscription changes (e.g., new topic creation) independently and triggering multiple unnecessary rebalances. KAFKA-7126 contains detailed scenarios and rationale.
Author: Jon Lee <jonlee@linkedin.com>
Reviewers: Jason Gustafson <jason@confluent.io>, Ted Yu <yuzhihong@gmail.com>, Dong Lin <lindong28@gmail.com>
Closes#5408 from jonlee2/KAFKA-7126
An untimely wakeup can cause ConsumerCoordinator.onJoinComplete to throw a WakeupException before completion. On the next poll(), it will be retried, but this leads to an underflow error because the buffer containing the assignment data will already have been advanced. The solution is to duplicate the buffer passed to onJoinComplete.
Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Rajini Sivaram <rajinisivaram@googlemail.com>
After successful completion of KafkaProducer#close, it is possible that an application calls KafkaProducer#send. If the send is invoked for a topic for which we do not have any metadata, the producer will block until `max.block.ms` elapses - we do not expect to receive any metadata update in this case because Sender (and NetworkClient) has already exited. It is only when RecordAccumulator#append is invoked that we notice that the producer has already been closed and throw an exception. If `max.block.ms` is set to Long.MaxValue (or a sufficiently high value in general), the producer could block awaiting metadata indefinitely.
This patch makes sure `Metadata#awaitUpdate` periodically checks if the network client has been closed, and if so bails out as soon as possible.