Method split takes up too many resources and might
cause outOfMemory error when the bigBatch is huge.
Call closeForRecordAppends() to free up resources
like compression buffers.
Change-Id: Iac6519fcc2e432330b8af2d9f68a8d4d4a07646b
Signed-off-by: Jiamei Xie <jiamei.xiearm.com>
*More detailed description of your change,
if necessary. The PR title and PR message become
the squashed commit message, so use a separate
comment to ping reviewers.*
*Summary of testing strategy (including rationale)
for the feature or bug fix. Unit and/or integration
tests are expected for any behaviour change and
system tests should be considered for larger changes.*
Author: Jiamei Xie <jiamei.xie@arm.com>
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Jiangjie (Becket) Qin <becket.qin@gmail.com>
Closes#8286 from jiameixie/outOfMemory
QuotaViolationException generates an exception message via String.format in the constructor
even though the message is often not used, e.g. https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/ClientQuotaManager.scala#L258. We now override `toString` instead.
It also generates an unnecessary stack trace, which is now avoided using the same pattern as in ApiException.
I have also avoided use of QuotaViolationException for control flow in
ReplicationQuotaManager which is another hotspot that we have seen in practice.
Reviewers: Gwen Shapira <gwen@confluent.io>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Ismael Juma <ismael@juma.me.uk>
FetchRequest.PartitionData.equals unnecessarily uses Object.equals generating a lot of allocations due to boxing, even though primitives are being compared. This is shown in the allocation profile below. Note that the CPU overhead is negligble.


Author: Lucas Bradstreet <lucasbradstreet@gmail.com>
Reviewers: Chia-Ping Tsai, Gwen Shapira
Closes#8473 from lbradstreet/avoid-boxing-partition-data-equals
This is a follow-up to #8077. The bug exposed a testing gap in how we group partitions. This patch adds a test case which reproduces the reported problem.
Reviewers: David Arthur <mumrah@gmail.com>
Change TimeoutException to BufferExhaustedException when no memory can be allocated for a record within max.block.ms
Refactored BufferExhaustedException to be a subclass of TimeoutException so existing code that catches TimeoutExceptions keeps working.
Added handling to count these Exceptions in the metric "buffer-exhausted-records".
Test Strategy
There were existing test cases to check this behavior, which I refactored.
I then added an extra case to check whether the expected Exception is actually thrown, which was not covered by current tests.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>
Remove the restriction in the protocol generation code that a structure
field needs to be part of an array.
Reviewers: Colin P. McCabe <cmccabe@apache.org>
In `validateOffsetsAsync` in t he consumer, we group the requests by leader node for efficiency. The list of topic-partitions are grouped from `partitionsToValidate` (all partitions) to `node` => `fetchPostitions` (partitions by node). However, when actually sending the request with `OffsetsForLeaderEpochClient`, we use `partitionsToValidate`, which is the list of all topic-partitions passed into `validateOffsetsAsync`. This results in extra partitions being included in the request sent to brokers that are potentially not the leader for those partitions.
This PR fixes the issue by using `fetchPositions`, which is the proper list of partitions that we should send in the request. Additionally, a small typo of API name in `OffsetsForLeaderEpochClient` is corrected (it originally referenced `LisfOffsets` as the API name).
Reviewers: David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>
A read from the end of the log interleaved with a concurrent write can result in reading data above the expected read limit. In particular, this would allow a read above the high watermark. The root of the problem is consecutive calls to `sizeInBytes` in `FileRecords.slice` which do not account for an increase in size due to a concurrent write. This patch fixes the problem by using a single call to `sizeInBytes` and caching the result.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Fix the direct cause of the observed issue on the client side: when heartbeat getting errors and resetting generation, we only need to set it to UNJOINED when it was not already in REBALANCING; otherwise, the join-group handler would throw the retriable UnjoinedGroupException to force the consumer to re-send join group unnecessarily.
Fix the root cause of the issue on the broker side: we should still trigger rebalance when static member joins in CompletingRebalance phase; otherwise the member.ids would be changed when the assignment is received from the leader, hence causing the new member.id's assignment to be empty.
Reviewers: Boyang Chen <boyang@confluent.io>, Jason Gustafson <jason@confluent.io>
On metadata change for assigned topics, we trigger rebalance, revoke partitions and send JoinGroup. If metadata reverts to the original value and JoinGroup fails, we don't resend JoinGroup because we don't set `rejoinNeeded`. This PR sets `rejoinNeeded=true` when rebalance is triggered due to metadata change to ensure that we retry on failure.
Reviewers: Boyang Chen <boyang@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Jason Gustafson <jason@confluent.io>
https://issues.apache.org/jira/browse/KAFKA-8889 attempted to fill in the missing stacktrace in the log message when handling errors in FetchSessionHandler#handleError
But the fix is not effective without KAFKA-7016
The current fix removes the redundant pair of braces {} at the end of the log message. If and when the Throwable that is passed as argument to this method has a stacktrace, the log message will include it. Currently it doesn't because the Throwable argument does not have a stacktrace.
Reviewers: Colin P. McCabe <cmccabe@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
Revert the decision for the sendOffsetsToTransaction(groupMetadata) API to fail with old version of brokers for the sake of making the application easier to adapt between versions. This PR silently downgrade the TxnOffsetCommit API when the build version is small than 3.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Co-authored-by: Andrew Olson <aolson1@cerner.com>
Reviewers: Jason Gustafson <jason@confluent.io>, Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <wangguoz@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
Once Scala 2.13.2 is officially released, I will submit a follow up PR
that enables `-Xfatal-warnings` with the necessary warning
exclusions. Compiler warning exclusions were only introduced in 2.13.2
and hence why we have to wait for that. I used a snapshot build to
test it in the meantime.
Changes:
* Remove Deprecated annotation from internal request classes
* Class.newInstance is deprecated in favor of
Class.getConstructor().newInstance
* Replace deprecated JavaConversions with CollectionConverters
* Remove unused kafka.cluster.Cluster
* Don't use Map and Set methods deprecated in 2.13:
- collection.Map +, ++, -, --, mapValues, filterKeys, retain
- collection.Set +, ++, -, --
* Add scala-collection-compat dependency to streams-scala and
update version to 2.1.4.
* Replace usages of deprecated Either.get and Either.right
* Replace usage of deprecated Integer(String) constructor
* `import scala.language.implicitConversions` is not needed in Scala 2.13
* Replace usage of deprecated `toIterator`, `Traversable`, `seq`,
`reverseMap`, `hasDefiniteSize`
* Replace usage of deprecated alterConfigs with incrementalAlterConfigs
where possible
* Fix implicit widening conversions from Long/Int to Double/Float
* Avoid implicit conversions to String
* Eliminate usage of deprecated procedure syntax
* Remove `println`in `LogValidatorTest` instead of fixing the compiler
warning since tests should not `println`.
* Eliminate implicit conversion from Array to Seq
* Remove unnecessary usage of 3 argument assertEquals
* Replace `toStream` with `iterator`
* Do not use deprecated SaslConfigs.DEFAULT_SASL_ENABLED_MECHANISMS
* Replace StringBuilder.newBuilder with new StringBuilder
* Rename AclBuffers to AclSeqs and remove usage of `filterKeys`
* More consistent usage of Set/Map in Controller classes: this also fixes
deprecated warnings with Scala 2.13
* Add spotBugs exclusion for inliner artifact in KafkaApis with Scala 2.12.
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
We find that brokers may send empty assignment for some members unexpectedly, and would need more logs investigating this issue.
Reviewers: John Roesler <vvcephei@apache.org>
Add missing @Override annotations and use lambdas when declaring threads to suppress warnings in IDEs and improve readability.
Reviewers: Ron Dagostino <rdagostino@confluent.io>, Konstantine Karantasis <konstantine@confluent.io>
KAFKA-7283 enabled lazy mmap on index files by initializing indices
on-demand rather than performing costly disk/memory operations when
creating all indices on broker startup. This helped reducing the startup
time of brokers. However, segment indices are still created on closing
segments, regardless of whether they need to be closed or not.
This is a cleaned up version of #7900, which was submitted by @efeg. It
eliminates unnecessary disk accesses and memory map operations while
deleting, renaming or closing offset and time indexes.
In a cluster with 31 brokers, where each broker has 13K to 20K segments,
@efeg and team observed up to 2 orders of magnitude faster LogManager
shutdown times - i.e. dropping the LogManager shutdown time of each
broker from 10s of seconds to 100s of milliseconds.
To avoid confusion between `renameTo` and `setFile`, I replaced the
latter with the more restricted updateParentDir` (it turns out that's
all we need).
Reviewers: Jun Rao <junrao@gmail.com>, Andrew Choi <a24choi@edu.uwaterloo.ca>
Co-authored-by: Adem Efe Gencer <agencer@linkedin.com>
Co-authored-by: Ismael Juma <ismael@juma.me.uk>
There are cases where `currentEstimation` is less than
`COMPRESSION_RATIO_IMPROVING_STEP` causing
`estimatedCompressionRatio` to be negative. This, in turn,
may result in `MESSAGE_TOO_LARGE`.
Reviewers: Ismael Juma <ismael@juma.me.uk>
SSLEngine#beginHandshake is possible to throw authentication failures (for example, no suitable cipher suites) so we ought to catch SSLException and then convert it to SslAuthenticationException so as to process authentication failures correctly.
Reviewers: Jun Rao <junrao@gmail.com>
In Streams `StreamsMetadataState.getMetadataWithKey`, we should use the inferred max topic partitions passed in directly from the caller than relying on cluster to contain its topic-partition information.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
In Kafka Connect, a ConfigProvider instance can be used concurrently (e.g. via a PUT request to the `/connector-plugins/{connectorType}/config/validate` REST endpoint), but there is no mention of concurrent usage in the Javadocs of the ConfigProvider interface.
It's worth calling out that implementations need to be thread safe.
Reviewers: Konstantine Karantasis <konstantine@confluent.io>
This improvement fixes several linking errors to classes and methods from within javadocs.
Related to #8291
Reviewers: Konstantine Karantasis <konstantine@confluent.io>
Once we have encoded the offset sums per task for each client, we can compute the overall lag during assign by fetching the end offsets for all changelog and subtracting.
If the listOffsets request fails, we simply return a "completely sticky" assignment, ie all active tasks are given to previous owners regardless of balance.
Builds (but does not yet use) the statefulTasksToRankedCandidates map with the ranking:
Rank -1: active running task
Rank 0: standby or restoring task whose overall lag is within acceptableRecoveryLag
Rank 1: tasks whose lag is unknown (eg during version probing)
Rank 1+: all other tasks are ranked according to their actual total lag
Implements: KIP-441
Reviewers: Bruno Cadonna <bruno@confluent.io>, John Roesler <vvcephei@apache.org>
If partitions are revoked, an application may want to commit the current offsets.
Using transactions, committing offsets would be done via the producer passing in the current ConsumerGroupMetadata. If the metadata is not updates before the callback, the call to commitTransaction(...) fails as and old generationId would be used.
Reviewers: Guozhang Wang <wangguoz@gmail.com>
This bug was incurred by #7994 with a too-strong consistency check. It is because a reset generation operation could be called in between the joinGroupRequest -> joinGroupResponse -> SyncGroupRequest -> SyncGroupResponse sequence of events, if user calls unsubscribe in the middle of consumer#poll().
Proper fix is to avoid the protocol name check when the generation is invalid.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Rewrite ReassignPartitionsCommand to use the KIP-455 API when possible, rather
than direct communication with ZooKeeper. Direct ZK access is still supported,
but deprecated, as described in KIP-455.
As specified in KIP-455, the tool has several new flags. --cancel stops
an assignment which is in progress. --preserve-throttle causes the
--verify and --cancel commands to leave the throttles alone.
--additional allows users to execute another partition assignment even
if there is already one in progress. Finally, --show displays all of
the current partition reassignments.
Reorganize the reassignment code and tests somewhat to rely more on unit
testing using the MockAdminClient and less on integration testing. Each
integration test where we bring up a cluster seems to take about 5 seconds, so
it's good when we can get similar coverage from unit tests. To enable this,
MockAdminClient now supports incrementalAlterConfigs, alterReplicaLogDirs,
describeReplicaLogDirs, and some other APIs. MockAdminClient is also now
thread-safe, to match the real AdminClient implementation.
In DeleteTopicTest, use the KIP-455 API rather than invoking the reassignment
command.
Previously, `AdminClient` group operations did not respect a `Call`'s number of configured tries and retry backoff. This could lead to tight retry loops that put a lot of pressure on the broker. This PR introduces fixes that ensures for all group operations the `AdminClient` respects the number of tries and the backoff a given `Call` has.
Reviewers: Vikas Singh <vikas@confluent.io>, Jason Gustafson <jason@confluent.io>
This PR removes intermediate conversions between `MetadataResponse.TopicMetadata` => `MetadataResponseTopic` and `MetadataResponse.PartitionMetadata` => `MetadataResponsePartition` objects.
There is 15-20% reduction in object allocations and 5-10% improvement in metadata request performance.
Reviewers: Ismael Juma <ismael@juma.me.uk>, Jason Gustafson<jason@confluent.io>
1. Inside StateDirectory#cleanRemovedTasks, skip deleting the lock file (and hence the parent directory) until releasing the lock. And after the lock is released only go ahead and delete the parent directory if manualUserCall == true. That is, this is triggered from KafkaStreams#cleanUp and users are responsible to make sure that Streams instance is not started and hence there are no other threads trying to grab that lock.
2. As a result, during scheduled cleanup the corresponding task.dir would not be empty but be left with only the lock file, so effectively we still achieve the goal of releasing disk spaces. For callers of listTaskDirectories like KIP-441 (cc @ableegoldman to take a look) I've introduced a new listNonEmptyTaskDirectories which excludes such dummy task.dirs with only the lock file left.
3. Also fixed KAFKA-8999 along the way to expose the exception while traversing the directory.
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, John Roesler <vvcephei@apache.org>
This PR tries to add an internal flag to throw if we hit an unexpected protocol version for offset fetch. It could be used together with EOS_BETA flag so that if server side downgrades unexpectedly, we shall fail the application ASAP.
Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>