Newer clients were getting stuck entering the validation phase even when a broker didn't support it. This commit will bypass the AWAITING_VALIDATION state when the broker is on an older version of the OffsetsForLeaderEpoch RPC.
Also fixes a system test by configuring the HATA to perform a one-shot balanced assignment
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Bruno Cadonna <bruno@confluent.io>
Added support for -1 replication factor and partitions for distributed worker internal topics by expanding the allowed values for the internal topics’ replication factor and partitions from positive values to also include -1 to signify that the broker defaults should be used.
The Kafka storage classes were already constructing a `NewTopic` object (always with a replication factor and partitions) and sending it to Kafka when required. This change will avoid setting the replication factor and/or number of partitions on this `NewTopic` if the worker configuration uses -1 for the corresponding configuration value.
Also added support for extra settings for internal topics on distributed config, status, and offset internal topics.
Quite a few new tests were added to verify that the `TopicAdmin` utility class is correctly using the AdminClient, and that the `DistributedConfig` validators for these configurations are correct. Also added integration tests for internal topic creation, covering preexisting functionality plus the new functionality.
Author: Randall Hauch <rhauch@gmail.com>
Reviewer: Konstantine Karantasis <konstantine@confluent.io>
If the request timeout is larger than the rebalance timeout, we should use the former as the JoinGroup request timeout. This patch also includes some minor improvements to request/response logging in `NetworkClient` including adding the request timeout to the log message.
Reviewers: Boyang Chen <boyang@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
At the time of this writing there are 6 schemas in kafka APIs with no fields - 3
versions each of LIST_GROUPS and API_VERSIONS.
When reading instances of these schemas off the wire there's little point in
returning a unique Struct object (or a unique values array inside that Struct)
since there is no payload.
Reviewers: Ismael Juma <ismael@juma.me.uk>
* MINOR: Fix typo in RecordAccumulator
* MINOR: Fix typo in several files
Reviewers: Ron Dagostino <rdagostino@confluent.io>, Konstantine Karantasis <konstantine@confluent.io>
* If two exceptions are thrown the `closePartitions` exception is suppressed
* Add unit tests that throw exceptions in put and close to verify that
the exceptions are propagated and suppressed appropriately out of WorkerSinkTask::execute
Reviewers: Chris Egerton <chrise@confluent.io>, Nigel Liang <nigel@nigelliang.com>, Konstantine Karantasis <konstantine@confluent.io>
Validate that the assignment is always balanced wrt:
* active assignment balance
* stateful assignment balance
* task-parallel balance
Reviewers: Bruno Cadonna <bruno@confluent.io>, A. Sophie Blee-Goldman <sophie@confluent.io>
Prior to KAFKA-8106, we allowed the v0 and v1 message formats to contain non-consecutive inner offsets. Inside `LogValidator`, we would detect this case and rewrite the batch. After KAFKA-8106, we changed the logic to raise an error in the case of the v1 message format (v0 was still expected to be rewritten). This caused an incompatibility for older clients which were depending on the looser validation. This patch reverts the old logic of rewriting the batch to fix the invalid inner offsets.
Note that the v2 message format has always had stricter validation. This patch also adds a test case for this.
Reviewers: José Armando García Sancio <jsancio@users.noreply.github.com>, Ismael Juma <ismael@juma.me.uk>
* Use `forEach` instead of `asScala.foreach` for Java Iterables.
* Use `ifPresent` instead of `asScala.foreach` for Java Optionals.
* Use `forEach` instead of `entrySet.forEach` for Java maps.
* Keep `asScala.foreach` for `Properties` as the Scala implementation
has a better interface (keys and values are of type `String`).
* A few clean-ups: unnecessary `()`, `{}`, `new`, etc.
Reviewers: Manikumar Reddy <manikumar@confluent.io>
The CircularIterator class uses a wrapping index-based approach to iterate over a list. This can be a performance problem O(n^2) for a LinkedList. Also, the index counter itself is never reset, a modulo is applied to it for every list access. At some point, it may be possible that the index counter overflows to a negative value and therefore may cause a negative index read and an ArrayIndexOutOfBoundsException.
This fix changes the implementation to avoid these two scenarios. Uses the Collection Iterator classes to avoid using an index counter and it avoids having to seek to the correct index every time, this avoiding the LinkedList performance issue.
I have added unit tests to validate the new implementation.
* KAFKA-9419: Integer Overflow Possible with CircularIterator
* Added JavaDoc. Support null values in the underlying collection
* Always return true for hasNext(). Add more JavaDoc
* Use an advance method to load next value and always return true in hasNext()
* Simplify test suite
* Use assertThrows in tests and remove redundant 'this' identifier
Co-authored-by: David Mollitor <dmollitor@apache.org>
Co-authored-by: Konstantine Karantasis <konstantine@confluent.io>
Reviewers: Ron Dagostino <rdagostino@confluent.io>, Konstantine Karantasis <konstantine@confluent.io>
* `MockAdminClient` should behave the same way as `Admin` for `createTopics()`
* Changed from throwing an `IllegalArgumentException` to `InvalidReplicationFactorException` when `brokers.size() < replicationFactor`
Author: jeff kim <jeff.kim@confluent.io>
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#8617 from jeffkbkim/MockAdminClient-InvalidReplicationFactorException
In the case described in the JIRA, there was a 50%+ increase in the total fetch request rate in
2.4.0 due to this change.
I included a few additional clean-ups:
* Simplify `findPreferredReadReplica` and avoid unnecessary collection copies.
* Use `LongSupplier` instead of `Supplier<Long>` in `SubscriptionState` to avoid unnecessary boxing.
Added a unit test to ReplicaManagerTest and cleaned up the test class a bit including
consistent usage of Time in MockTimer and other components.
Reviewers: Gwen Shapira <gwen@confluent.io>, David Arthur <mumrah@gmail.com>, Jason Gustafson <jason@confluent.io>
This patch ensures that `SslEngineFactory` is closed. The default implementation (**DefaultSslEngineFactory**) does not have any releasable object so we didn't notice this issue. However, it would be better to fix this issue for custom engine factories.
Reviewers: Jason Gustafson <jason@confluent.io>
KafkaAdminClientTest.testAlterClientQuotas() is uncalled. It is clearly intended to be a test method, but lacks `Test`.
Author: Tom Bentley <tbentley@redhat.com>
Reviewers: Gwen Shapira, Brian Byrne
Closes#8456 from tombentley/MINOR-annotate-test-method
1. Added a recordInternal function to let all other public functions trigger, so that shouldRecord would only be checked once.
2. In Streams, pass along the current wall-clock time inside InternalProcessorContext when process / punctuate which can be passed in to the record function to reduce the calling frequency of SystemTime.milliseconds().
Reviewers: John Roesler <vvcephei@apache.org>
ConfigProvider extends Closeable, but were not closed in the following contexts:
* AbstractConfig
* WorkerConfigTransformer
* Worker
This commit ensures that ConfigProviders are close in the above contexts.
It also adds MockFileConfigProvider.assertClosed()
Gradle executes test classes concurrently, so MockFileConfigProvider
can't simply use a static field to hold its closure state.
Instead use a protocol whereby the MockFileConfigProvider is configured
with some unique ket identifying the test which also used when calling
assertClosed().
Reviewers: Konstantine Karantasis <konstantine@confluent.io>
After KIP-219, responses are sent immediately and we rely on a combination
of clients and muting of the channel to throttle. The result of this is that
we need to track `apiThrottleTimeMs` as an explicit value instead of
inferring it. On the other hand, we no longer need
`apiRemoteCompleteTimeNanos`.
Extend `BaseQuotaTest` to verify that throttle time in the request channel
metrics are being set. Given the nature of the throttling numbers, the test
is not particularly precise.
I included a few clean-ups:
* Pass KafkaMetric to QuotaViolationException so that the caller doesn't
have to retrieve it from the metrics registry.
* Inline Supplier in SocketServer (use SAM).
* Reduce redundant `time.milliseconds` and `time.nanoseconds`calls.
* Use monotonic clock in ThrottledChannel and simplify `compareTo` method.
* Simplify `TimerTaskList.compareTo`.
* Consolidate the number of places where we update `apiLocalCompleteTimeNanos`
and `responseCompleteTimeNanos`.
* Added `toString` to ByteBufferSend` and `MultiRecordsSend`.
* Restrict access to methods in `QuotaTestClients` to expose only what we need
to.
Reviewers: Jun Rao <junrao@gmail.com>
* add a config to set the TaskAssignor
* set the default assignor to HighAvailabilityTaskAssignor
* fix broken tests (with some TODOs in the system tests)
Implements: KIP-441
Reviewers: Bruno Cadonna <bruno@confluent.io>, A. Sophie Blee-Goldman <sophie@confluent.io>
Makes the main thread wait for workers to be ready to test the
desired functionality before proceeding.
Reviewers: Ted Yu <yuzhihong@gmail.com>, Ismael Juma <ismael@juma.me.uk>
Add an option to kafka-configs.sh `--add-config-file` that adds the configs from a properties file.
Testing: Added new tests to ConfigCommandTest.scala
Author: Aneel Nazareth <aneel@confluent.io>
Reviewers: David Jacot <djacot@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
Closes#8184 from WanderingStar/KAFKA-9612
For join / sync / commit / heartbeat request, we would remember the sent generation in the created handler object, and then upon getting the error code, we could check whether the sent generation still matches the current generation. If not, it means that the member has already reset its generation or has participated in a new rebalance already. This means:
1. For join / sync-group request, we do not need to call reset-generation any more for illegal-generation / unknown-member. But we would still set the error since at a given time only one join/sync round-trip would be in flight, and hence we should not be participating in a new rebalance. Also for fenced instance error we still treat it as fatal since we should not be participating in a new rebalance, so this is still not expected.
2. For commit request, we do not set the corresponding error for illegal-generation / unknown-member / fenced-instance but raise rebalance-in-progress. For commit-sync it would be still thrown to user, while for commit-async it would be logged and swallowed.
3. For heartbeat request, we do not treat illegal-generation / unknown-member / fenced-instance errors and just consider it as succeeded since this should be a stale heartbeat which can be ignored.
Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Boyang Chen <boyang@confluent.io>, Jason Gustafson <jason@confluent.io>
Adjusted one assert condition on a testcase as the success was dependant
on thread runtimes and the much lower tolerances due to the reduced time
broke this test.
Reviewers: Ismael Juma <ismael@juma.me.uk>
* Add AbstractResponse#errorCounts(Stream) to avoid having to call
AbstractResponse#errorCounts(Collection) with a computed collection.
* A microbenchmark showed that using errorCounts(Stream) was
around 7.5 times faster than errorCounts(Collection). Using forEach()
loops with updateErrorCounts() is slightly faster, but is usually more
code.
* Use updateErrorMap() consistently.
* Replace for statements with forEach() for consistency.
* Use singleton errorMap() consistently.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Ismael Juma <ismael@juma.me.uk>
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.

![image](https://user-images.githubusercontent.com/252189/79079019-46686300-7cc1-11ea-9bc9-44fd17bae888.png)
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>