It's good for us to add support for Java 20 in preparation for Java 21 - the next LTS.
Given that Scala 2.12 support has been deprecated, a Scala 2.12 variant is not included.
Also remove some branch builds that add load to the CI, but have
low value: JDK 8 & Scala 2.13 (JDK 8 support has been deprecated),
JDK 11 & Scala 2.12 (Scala 2.12 support has been deprecated) and
JDK 17 & Scala 2.12 (Scala 2.12 support has been deprecated).
A newer version of Mockito (4.9.0 -> 4.11.0) is required for Java 20 support, but we
only use it with Scala 2.13+ since it causes compilation errors with Scala 2.12. Similarly,
we upgrade easymock when the Java version is 16 or newer as it's incompatible
with powermock (which doesn't support Java 16 or newer).
Filed KAFKA-15117 for a test that fails with Java 20 (SslTransportLayerTest.testValidEndpointIdentificationCN).
Finally, fixed some lossy conversions that were added after #13582 was submitted.
Reviewers: Ismael Juma <ismael@juma.me.uk>
Fixed a regression described in KAFKA-15053 that security.protocol only allows uppercase values like PLAINTEXT, SSL, SASL_PLAINTEXT, SASL_SSL. With this fix, both lower case and upper case values will be supported (e.g. PLAINTEXT, SSL, SASL_PLAINTEXT, SASL_SSL, plaintext, ssl, sasl_plaintext, sasl_ssl)
Reviewers: Chris Egerton <chrise@aiven.io>, Divij Vaidya <diviv@amazon.com>
Discovered while researching KAFKA-14718
Currently, we perform a check during zombie fencing that causes the round of zombie fencing to fail when a rebalance is pending (i.e., when we've detected from a background poll of the config topic that a new connector has been created, that an existing connector has been deleted, or that a new set of connector tasks has been generated).
It's possible but not especially likely that this check causes issues when running vanilla Kafka Connect. Even when it does, it's easy enough to restart failed tasks via the REST API.
However, when running MirrorMaker 2 in dedicated mode, this check is more likely to cause issues as we write three connector configs to the config topic in rapid succession on startup. And in that mode, there is no API to restart failed tasks aside from restarting the worker that they are hosted on.
In either case, this check can lead to test flakiness in integration tests for MirrorMaker 2 both in dedicated mode and when deployed onto a vanilla Kafka Connect cluster.
This check is not actually necessary, and we can safely remove it. Copied from Jira:
>If the worker that we forward the zombie fencing request to is a zombie leader (i.e., a worker that believes it is the leader but in reality is not), it will fail to finish the round of zombie fencing because it won't be able to write to the config topic with a transactional producer.
>If the connector has just been deleted, we'll still fail the request since we force a read-to-end of the config topic and refresh our snapshot of its contents before checking to see if the connector exists.
>And regardless, the worker that owns the task will still do a read-to-end of the config topic and verify that (1) no new task configs have been generated for the connector and (2) the worker is still assigned the connector, before allowing the task to process any data.
In addition, while waiting on a fix for KAFKA-14718 that adds more granularity for diagnosing failures in the DedicatedMirrorIntegrationTest suite (#13284), some of the timeouts in that test are bumped to work better on our CI infrastructure.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Yash Mayya <yash.mayya@gmail.com>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
Most of the contents in the README.md was already covered in the docs therefore only had to add the section for Exactly Once support.
Reviewers: Luke Chen <showuon@gmail.com>
In case the Kafka Broker cluster and the Kafka Connect cluster is started together and Connect would want to create its topics, there's a high chance to fail the creation with InvalidReplicationFactorException.
---------
Co-authored-by: Daniel Urban <durban@cloudera.com>
Reviewers: Daniel Urban <durban@cloudera.com>, Mickael Maison <mickael.maison@gmail.com>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>, Chris Egerton <chrise@aiven.io>, Laszlo Hunyadi <laszlo.istvan.hunyady@gmail.com>
`KafkaBasedLog` is a widely used utility class that provides a generic implementation of a shared, compacted log of records in a Kafka topic. It isn't in Connect's public API, but has been used outside of Connect and we try to preserve backward compatibility whenever possible. KAFKA-14455 modified the two overloaded void `KafkaBasedLog::send` methods to return a `Future`. While this change is source compatible, it isn't binary compatible. We can restore backward compatibility simply by renaming the new Future returning send methods, and reinstating the older send methods to delegate to the newer methods.
This refactoring changes no functionality other than restoring the older methods.
Reviewers: Randall Hauch <rhauch@gmail.com>
#13557 introduced a utils method to close executors silently. This PR leverages that method to close executors in connect runtime. There was duplicate code while closing the executors which isn't the case with this PR.
Note that there are a few more executors used in Connect runtime but their close methods don't follow this pattern of shutdown, await and shutdown. Some of them have some logic like executor like Worker, so not changing at such places.
---------
Co-authored-by: Sagar Rao <sagarrao@Sagars-MacBook-Pro.local>
Reviewers: Daniel Urban <durban@cloudera.com>, Yash Mayya <yash.mayya@gmail.com>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
We are handling complex workflows ListOffsets by chaining together MetadataCall instances and ListOffsetsCall instances, there are many complex and error-prone logic. In this PR we rewrote it with the `AdminApiDriver` infra, notable changes better than old logic:
1. Retry lookup stage on receiving `NOT_LEADER_OR_FOLLOWER` and `LEADER_NOT_AVAILABLE`, whereas in the past we failed the partition directly without retry.
2. Removing class field `supportsMaxTimestamp` and calculating it on the fly to avoid the mutable state, this won't change any behavior of the client.
3. Retry fulfillment stage on `RetriableException`, whereas in the past we just retry fulfillment stage on `InvalidMetadataException`, this means we will retry on `TimeoutException` and other `RetriableException`.
We also `handleUnsupportedVersionException` to `AdminApiHandler` and `AdminApiLookupStrategy`, they are used to keep consistency with old logic, and we can continue improvise them.
Reviewers: Ziming Deng <dengziming1993@gmail.com>, David Jacot <djacot@confluent.io>
KafkaStatusBackingStore uses an infinite retry logic on producer send, which can lead to a stack overflow.
To avoid the problem, a background thread was added, and the callback submits the retry onto the background thread.
New add.source.alias.to.metrics setting to add the source cluster alias to the MirrorSourceConnector metrics
Reviewers: Chris Egerton <fearthecellos@gmail.com>
This fixes a regression introduced in #12828, which caused workers to start unconditionally loading (and therefore validating) SSL-related properties when issuing REST requests to other workers. That was fine for the most part, but caused unnecessary failures when workers were configured with invalid SSL-related properties and their REST API used HTTP instead of HTTPS.
Reviewers: Ian McDonald <imcdonald@confluent.io>, Greg Harris <greg.harris@aiven.io>, Yash Mayya <yash.mayya@gmail.com>, Justine Olshan <jolshan@confluent.io>
KAFKA-12468: Fix negative lag on down consumer groups synced by MirrorMaker 2
KAFKA-13659: Stop syncing consumer groups with stale offsets in MirrorMaker 2
KAFKA-12566: Fix flaky MirrorMaker 2 integration tests
Reviewers: Chris Egerton <chrise@aiven.io>
* assertEquals called on array
* Method is identical to its super method
* Simplifiable assertions
* Unused imports
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Divij Vaidya <diviv@amazon.com>
Reviewers: Daniel Urban <durban@cloudera.com>, Greg Harris <greg.harris@aiven.io>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>, Mickael Maison <mickael.maison@gmail.com>
This is a small refactor extracted from https://github.com/apache/kafka/pull/12845. It basically moves the logic to handle the backward compatibility of `JoinGroupResponseData.protocolName` from `KafkaApis` to `JoinGroupResponse`.
The patch adds a new unit test for `JoinGroupResponse` and relies on existing tests as well.
Reviewers: Justine Olshan <jolshan@confluent.io>, Jason Gustafson <jason@confluent.io>
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Tom Bentley <tombentley@users.noreply.github.com>
Co-authored-by: Tom Bentley <tombentley@users.noreply.github.com>
Co-authored-by: oibrahim3 <omnia@apple.com>
Also moves the Streams LogCaptureAppender class into the clients module so that it can be used by both Streams and Connect.
Reviewers: Nigel Liang <nigel@nigelliang.com>, Kalpesh Patel <kpatel@confluent.io>, John Roesler <vvcephei@apache.org>, Tom Bentley <tbentley@redhat.com>
The motivation for this change is to guard against timing attacks when using InternalRequestSignature.equals()
Pros of this PR
if the InternalRequestSignature.equal() method could be used for a timing attack, then this PR fixes a security vulnerability
Cons of this PR
MessageDigest.isEquals() is slower than Arrays.equal since the former is time constant i.e. it runs for a fixed time irrespective of the length of original signature. The execution time of MessageDigest.isEquals() is a function of length of the byte array that it is being tested against.
Even if InternalRequestSignature.equals() is not being used anywhere in code today where it may cause a timing attack, we should still guard against the possibility where a future change might start using it (especially in an open source project where changes might be contributed & reviewed by multiple group of people). The downside of slower equality comparison over a signature is risk worth accepting given the upside we get to safeguard future use cases.
Co-authored-by: Nandini Krishna Anagondi <nandini@mac.local>
Reviewers: Luke Chen <showuon@gmail.com>, Divij Vaidya <diviv@amazon.com>, Karan Kumar <karankumar1100@gmail.com>, Andrew Eugene Choi <andrew.choi@uwaterloo.ca>
Why
Current set of integration tests leak files in the /tmp directory which makes it cumbersome if you don't restart the machine often.
Fix
Replace the usage of File.createTempFile with existing TestUtils.tempFile method across the test files. TestUtils.tempFile automatically performs a clean up of the temp files generated in /tmp/ folder.
Reviewers: Luke Chen <showuon@gmail.com>, Ismael Juma <mlists@juma.me.uk>
Also includes a minor quality-of-life improvement to clarify why some internal REST requests to workers may fail while that worker is still starting up.
Reviewers: Tom Bentley <tbentley@redhat.com>, Luke Chen <showuon@gmail.com>, José Armando García Sancio <jsancio@gmail.com>, Mickael Maison <mickael.maison@gmail.com>
This implements KIP-830: https://cwiki.apache.org/confluence/display/KAFKA/KIP-830%3A+Allow+disabling+JMX+Reporter
It adds a new configuration `auto.include.jmx.reporter` that can be set to false to disable the JMX Reporter. This configuration is deprecated and will be removed in the next major version.
Reviewers: Tom Bentley <tbentley@redhat.com>, Christo Lolov <christo_lolov@yahoo.com>
Cache the Kafka cluster Id once it has been retrieved to avoid creating many Admin clients at startup.
Reviewers: Chris Egerton <fearthecellos@gmail.com>
This PR is created on top of #10904 and includes commits from original author for attribution.
## Testing
1. `./gradlew connect:runtime:unitTest --tests WorkerGroupMemberTest` is successful.
2. Verified that test is run as part of `./gradlew connect:runtime:unitTest` (see report in the PR)
Reviewers: Ismael Juma <ismael@juma.me.uk>
Co-authored-by: Chun-Hao Tang <tang7526@gmail.com>
Catches valid 404 exceptions, triggered by any HTTP request to a nonexistent path on the Connect REST API, higher in the code to not to log an ERROR log which can be seen as a false alarm
Reviewers: Chris Egerton <fearthecellos@gmail.com>
We should not treat UNKNOWN_MEMBER_ID as an unexpected error in the Admin client. In MirrorMaker, check the result of committing offsets and log an useful error message in case that failed with UNKNOWN_MEMBER_ID.
Reviewers: Chris Egerton <fearthecellos@gmail.com>
The HeaderConverter interface extends Closeable, but we weren't closing them anywhere before. This change causes header converters to be closed as part of task shutdown.
Reviewers: Kvicii <42023367+Kvicii@users.noreply.github.com>, Chris Egerton <fearthecellos@gmail.com>
In KAFKA-13310, we tried to fix a issue that consumer#poll(duration) will be returned after the provided duration. It's because if rebalance needed, we'll try to commit current offset first before rebalance synchronously. And if the offset committing takes too long, the consumer#poll will spend more time than provided duration. To fix that, we change commit sync with commit async before rebalance (i.e. onPrepareJoin).
However, in this ticket, we found the async commit will keep sending a new commit request during each Consumer#poll, because the offset commit never completes in time. The impact is that the existing consumer will be kicked out of the group after rebalance timeout without joining the group. That is, suppose we have consumer A in group G, and now consumer B joined the group, after the rebalance, only consumer B in the group.
Besides, there's also another bug found during fixing this bug. Before KAFKA-13310, we commitOffset sync with rebalanceTimeout, which will retry when retriable error until timeout. After KAFKA-13310, we thought we have retry, but we'll retry after partitions revoking. That is, even though the retried offset commit successfully, it still causes some partitions offsets un-committed, and after rebalance, other consumers will consume overlapping records.
Reviewers: RivenSun <riven.sun@zoom.us>, Luke Chen <showuon@gmail.com>
Make sure to ack all records where produce failed, when a connector's `errors.tolerance` config property is set to `all`. Acking is essential so that the task will continue to commit future record offsets properly and remove the records from internal tracking, preventing a memory leak.
(cherry picked and slightly modified from commit 63e06aafd0)
Reviewers: Chris Egerton <fearthecellos@gmail.com>, Randall Hauch <rhauch@gmail.com>
Implements embedded end-to-end integration tests for KIP-618, and brings together previously-decoupled logic from upstream PRs.
Reviewers: Luke Chen <showuon@gmail.com>, Tom Bentley <tbentley@redhat.com>, Mickael Maison <mickael.maison@gmail.com>
This causes the artificial reductions in the Connect REST request timeout to be more isolated. Specifically, they now only take place in the tests that need them (instead of any tests that happen to be running after the reduction has taken place and before it has been reset), and they are only performed for the requests that are expected to time out, before being immediately reset. This should help reduce spurious test failures (especially in slow environments like Jenkins) for all Connect integration tests that interact with the REST API, not just the BlockingConnectorTest test suite.
Reviewers: Bruno Cadonna <cadonna@apache.org>
Implements support for per-connector offsets topics as described in KIP-618.
Reviewers: Luke Chen <showuon@gmail.com>, Tom Bentley <tbentley@redhat.com>
Use the newly added function to replace the old addMetric function that may throw illegal argument exceptions.
Although in some cases concurrency should not be possible they do not necessarily remain always true in the future, so it's better to use the new API just to be less error-prone.
Reviewers: Bruno Cadonna <cadonna@apache.org>
New gradle task `connect:runtime:genConnectOpenAPIDocs` that generates `connect_rest.yaml` under `docs/generated`.
This task is executed when `siteDocsTar` runs.
Implements the behavior described in KIP-618: using a transactional producer for writes to the config topic that should only be performed by the leader of the cluster.
Reviewers: Luke Chen <showuon@gmail.com>, Tom Bentley <tbentley@redhat.com>
As an initial step to address the notoriously flaky BlockingConnectorTest test suite, we can try increasing test timeouts.
This approach may not be sufficient, and even if it is, it's still suboptimal. Although it may address flakiness on Jenkins, it will make genuine failures harder to detect when testing local changes. Additionally, if the workload on Jenkins continues to increase, we'll probably have to bump these timeouts in the future again at some point.
Potential next steps, for this PR and beyond:
Stop leaking threads that block during test runs
Instead of artificially reducing the REST request timeout at the beginning of every test, reduce it selectively right before issuing a REST request that is expected to time out, and then immediately reset it.
Eliminate artificial reduction of the REST request timeout entirely, as it may be negatively impacting other Connect integration tests that are being run concurrently.
Test repeatedly on Jenkins, ideally at least 50 times
Gather information on the number of CPU cores available to each Jenkins node and the distribution of how many threads are allocated over a given time period (maybe a day?); this is especially relevant since local testing indicates that these tests all do much better when parallelism is reduced, which shouldn't be too surprising considering that each Connect integration test spins up separate threads for at least one Zookeeper node, one Kafka broker, one Connect worker, and usually at least one connector and one task.
I'd like to test these changes as a first step before investigating any of the above (except maybe items 1 and 2, which should be fairly straightforward). To trigger new runs I plan on pushing empty commits or, if those do not trigger new Jenkins runs, dummy commits. If this is objectionable let me know and hopefully we can find a suitable alternative.
Reviewers: Kvicii <Karonazaba@gmail.com>, Bruno Cadonna <cadonna@apache.org>
Minor change to use ' and not LEFT SINGLE QUOTATION MARK in this log message, as it's the only place we are using such a quote and it can break ingestion pipelines
Reviewers: Kvicii <Karonazaba@gmail.com>, Divij Vaidya <diviv@amazon.com>, Konstantine Karantasis <k.karantasis@gmail.com>
The goals here include:
1. Create an overloaded variant of the IncrementalCooperativeAssignor::performTaskAssignment method that is more testing friendly
2. Simplify the parameter list for the IncrementalCooperativeAssignor::handleLostAssignments method, which in turn simplifies the logic for testing this class
3. Capture repeated Java 8 streams logic in simple, reusable, easily-verifiable utility methods added to the ConnectUtils class
Reviewers: Luke Chen <showuon@gmail.com>
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Reviewers: Tom Bentley <tbentley@redhat.com>, Hector Geraldino <hgeraldino@bloomberg.net>, Andrew Eugene Choi <andrew.choi@uwaterloo.ca>
The `DESTROYED` state is represented internally as a tombstone record when running in distributed mode and by the removal of the connector/task from the in-memory status map when running in standalone mode. As a result, it will never appear to users of the REST API, and we should remove mention of it from our docs so that developers creating tooling against the REST API don't write unnecessary logic to account for that state.
Reviewers: Mickael Maison <mickael.maison@gmail.com>
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Luke Chen <showuon@gmail.com>, Chris Egerton <fearthecellos@gmail.com>
Co-authored-by: Chris Egerton <fearthecellos@gmail.com>
The `retryEndOffsets(…)` method in `TopicAdmin` recently added (KAFKA-12879, #11797) to allow the `KafkaBasedLog.start()` method to retry any failures reading the last offsets for a topic. However, this introduce a regression when talking to older brokers (0.10.x or earlier).
The `KafkaBasedLog` already had logic that expected an `UnsupportedVersionException` thrown by the admin client when a Kafka API is not available on an older broker, but the new retry logic in `TopicAdmin` did not account for this and wrapped the exception, thereby breaking the `KafkaBasedLog` logic and preventing startup.
The fix is to propagate this `UnsupportedVersionException` from the `TopicAdmin.retryEndOffsets(…)` method. Added a new unit test that first replicated the problem before the fix, and verified the fix corrects the problem.
With AK 3.0, idempotence was enabled by default in Kafka producers. However, if idempotence is enabled, Connect won't be able to communicate via its producers with Kafka brokers older than version 0.11. Perhaps more importantly, for brokers older than version 2.8 the IDEMPOTENT_WRITE ACL is required to be granted to the principal of the Connect worker.
Therefore this commit disables producer idempotence by default to all the producers instantiated by Connect. Users can still choose to enable producer idempotence by explicitly setting the right worker and/or connector properties.
The changes were tested via existing unit, integration and system tests.
Reviewers: Randall Hauch <rhauch@gmail.com>
This reverts commit 76cf7a5793.
Connect already allows users to enable idempotent producers for connectors and the Connect workers. Although Kafka producers enabled idempotency by default in 3.0, due to compatibility requirements and the fact that [KIP-318](https://cwiki.apache.org/confluence/display/KAFKA/KIP-318%3A+Make+Kafka+Connect+Source+idempotent) hasn't been explicitly approved, the changes here are reverted. A separate commit will explicitly disable idempotency in producers instantiated by Connect by default until KIP-318 is approved and scheduled for release.
I collected a list of the most flaky tests observed lately, checked / created their corresponding tickets, and mark them as ignored for now. Many of these failures are:
0. Failing very frequently in the past (at least in my observations).
1. not investigated for some time.
2. have a PR for review (mostly thanks to @showuon !), but not reviewed for some time.
Since 0), these tests failures are hindering our development; and from 1/2) above, people are either too busy to look after them, or honestly the tests are not considered as providing values since otherwise people should care enough to panic and try to resolve. So I think it's reasonable to disable all these tests for now. If we later learned our lesson a hard way, it would motivate us to tackle flaky tests more diligently as well.
I'm only disabling those tests that have been failed for a while, and if for such time no one have been looking into them, I'm concerned that just gossiping around about those flakiness would not bring people's attention to them either. So my psychological motivation is that "if people do not care about those failed tests for weeks (which, is not a good thing! :P), let's teach ourselves the lesson a hard way when it indeed buries a bug that bites us, or not learn the lesson at all --- that indicates those tests are indeed not valuable". For tests that I only very recently saw I did not disable them.
Reviewers: John Roesler <vvcephei@apache.org>, Matthias J. Sax <mjsax@apache.org>, Luke Chen <showuon@gmail.com>, Randall Hauch <rhauch@gmail.com>
This is an addendum to the KAFKA-12879 (#11797) to fix some tests that are somewhat flaky when a build machine is heavily loaded (when the timeouts are too small).
- Add an if check to void sleep(0)
- Increase timeout in the tests
Fixes the compatibility issue regarding KAFKA-12879 by reverting the changes to the admin client from KAFKA-12339 (#10152) that retry admin client operations, and instead perform the retries within Connect's `KafkaBasedLog` during startup via a new `TopicAdmin.retryEndOffsets(..)` method. This method delegates to the existing `TopicAdmin.endOffsets(...)` method, but will retry on `RetriableException` until the retry timeout elapses.
This change should be backward compatible to the KAFKA-12339 so that when Connect's `KafkaBasedLog` starts up it will retry attempts to read the end offsets for the log's topic. The `KafkaBasedLog` existing thread already has its own retry logic, and this is not changed.
Added more unit tests, and thoroughly tested the new `RetryUtil` used to encapsulate the parameterized retry logic around any supplied function.
Make SetSchemaMetadata SMT ignore records with null value and valueSchema or key and keySchema.
The transform has been unit tested for handling null values gracefully while still providing the necessary validation for non-null values.
Reviewers: Konstantine Karantasis<konstantine@confluent.io>, Bill Bejeck <bbejeck@apache.org>
Lower the log level of a message in `WorkerSourceTask` which indicates that no messages have been produced by the task since it is spammy and causing users confusion.
Reviewers: Jason Gustafson <jason@confluent.io>
This patch adds null value check to the connector config validation, and extends unit tests to cover this functionality.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chris Egerton <chrise@confluent.io>, Boyang Chen <bchen11@outlook.com>, Andras Katona <akatona@cloudera.com>
Title: KafkaConsumer cannot jump out of the poll method, and cpu and traffic on the broker side increase sharply
description: The local test has been passed, the problem described by jira can be solved
JIRA link : https://issues.apache.org/jira/browse/KAFKA-13310
Reviewers: Luke Chen <showuon@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
In v3.0, we changed the default value for `enable.idempotence` to true, but we didn't adjust the validator and the `idempotence` enabled check method. So if a user didn't explicitly enable idempotence, this feature won't be turned on. This patch addresses the problem, cleans up associated logic, and fixes tests that broke as a result of properly applying the new default. Specifically it does the following:
1. fix the `ProducerConfig#idempotenceEnabled` method, to make it correctly detect if `idempotence` is enabled or not
2. remove some unnecessary config overridden and checks due to we already default `acks`, `retries` and `enable.idempotence` configs.
3. move the config validator for the idempotent producer from `KafkaProducer` into `ProducerConfig`. The config validation should be the responsibility of `ProducerConfig` class.
4. add an `AbstractConfig#hasKeyInOriginals` method, to avoid `originals` configs get copied and only want to check the existence of the key.
5. fix many broken tests. As mentioned, we didn't actually enable idempotence in v3.0. After this PR, there are some tests broken due to some different behavior between idempotent and non-idempotent producer.
6. add additional tests to validate configuration behavior
Reviewers: Kirk True <kirk@mustardgrain.com>, Ismael Juma <ismael@juma.me.uk>, Mickael Maison <mimaison@users.noreply.github.com>, Jason Gustafson <jason@confluent.io>
EmbeddedKafkaCluster in other projects use 2MB for their offset map to reduce
memory consumption in test runs. Generally we allocate multiple of these offset maps,
one for each broker.
Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Ismael Juma <ismael@juma.me.uk>
I've skipped the following classes as they use powermock to stub/access private and static fields/methods:
- KafkaConfigBackingStoreTest
- KafkaOffsetBackingStoreTest
Those will require some refactoring and will be updated in a separate PR.
Reviewers: Tom Bentley <tbentley@redhat.com>, dengziming <dengziming1993@gmail.com>
Although committing source task offsets without blocking on the delivery of all in-flight records is beneficial most of the time, it can lead to duplicate record delivery if there are in-flight records at the time of the task's end-of-life offset commit.
A best-effort attempt is made here to wait for any such in-flight records to be delivered before proceeding with the end-of-life offset commit for source tasks. Connect will block for up to offset.flush.timeout.ms milliseconds before calculating the latest committable offsets for the task and flushing those to the persistent offset store.
Author: Chris Egerton <chrise@confluent.io>
Reviewer: Randall Hauch <rhauch@gmail.com>
The `WorkerSinkTask.lastCommittedOffsets` field is now added to (via `Map::putAll`) after a successful offset commit, instead of being completely overwritten. In order to prevent this collection from growing indefinitely, elements are removed from it after topic partitions are revoked from the task's consumer.
Two test cases are added to `WorkerSinkTaskTest`:
- A basic test to verify the "rewind for redelivery" behavior when a task throws an exception from `SinkTask::preCommit`; surprisingly, no existing test cases appear to cover this scenario
- A more sophisticated test to verify this same behavior, but with a few rounds of cooperative consumer rebalancing beforehand that expose a bug in the current logic for the `WorkerSinkTask` class
The `VerifiableSinkTask` class is also updated to only flush the requested topic partitions in its `flush` method. This is technically unrelated to the issue addressed by this PR and can be moved to a separate PR if necessary; including it here as the original context for identifying this bug was debugging failed system tests and the logic in this part of the tests was originally suspected as a cause of the test failure.
Reviewers: Konstantine Karantasis <k.karantasis@gmail.com>
When running in dedicated mode, Connect runtimes are configured to use the `.internal` suffix for their topics.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Omnia G H Ibrahim <o.g.h.ibrahim@gmail.com>
Currently, the `WorkerSinkTask`'s consumer rebalance listener (and related logic) is hardcoded to assume eager rebalancing, which means that all partitions are revoked any time a rebalance occurs and then the set of partitions included in `onPartitionsAssigned` is assumed to be the complete assignment for the task. Not only does this cause failures when the cooperative consumer protocol is used, it fails to take advantage of the benefits provided by that protocol.
These changes alter framework logic to not only not break when the cooperative consumer protocol is used for a sink connector, but to reap the benefits of it as well, by not revoking partitions unnecessarily from tasks just to reopen them immediately after the rebalance has completed.
This change will be necessary in order to support [KIP-726](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=177048248), which currently proposes that the default consumer partition assignor be changed to the `CooperativeStickyAssignor`.
Two integration tests are added to verify sink task behavior with both eager and cooperative consumer protocols, and new and existing unit tests are adopted as well.
Reviewers: Nigel Liang <nigel@nigelliang.com>, Konstantine Karantasis <k.karantasis@gmail.com>
Replaces the current logic for committing source offsets, which is batch-based and blocks until the entirety of the current batch is fully written to and acknowledged by the broker, with a new non-blocking approach that commits source offsets for source records that have been "fully written" by the producer. The new logic consider a record fully written only if that source record and all records before it with the same source partition have all been written to Kafka and acknowledged.
This new logic uses a deque for every source partition that a source task produces records for. Each element in that deque is a SubmittedRecord with a flag to track whether the producer has ack'd the delivery of that source record to Kafka. Periodically, the worker (on the same thread that polls the source task for records and transforms, converts, and dispatches them to the producer) polls acknowledged elements from the beginning of each of these deques and collects the latest offsets from these elements, storing them in a snapshot that is then committed on the separate source task offset thread.
The behavior of the `offset.flush.timeout.ms property` is retained, but essentially now only applies to the actual writing of offset data to the internal offsets topic (if running in distributed mode) or the offsets file (if running in standalone mode). No time is spent during `WorkerSourceTask::commitOffsets` waiting on the acknowledgment of records by the producer.
This behavior also does not change how the records are dispatched to the producer nor how the producer sends or batches those records.
It's possible that memory exhaustion may occur if, for example, a single Kafka partition is offline for an extended period. In cases like this, the collection of deques in the SubmittedRecords class may continue to grow indefinitely until the partition comes back online and the SubmittedRecords in those deques that targeted the formerly-offline Kafka partition are acknowledged and can be removed. Although this may be suboptimal, it is no worse than the existing behavior of the framework in these cases.
Author: Chris Egerton <chrise@confluent.io>
Reviewed: Randall Hauch <rhauch@gmail.com>
* URL wasn't urlencoded when forwarded reconfiguration to leader connect worker
* handling previously swallowed errors in connect RestClient
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
Co-authored-by: Andras Katona <akatona@cloudera.com>
Co-authored-by: Daniel Urban <durban@cloudera.com>
Even after the implementation of KIP-745 it makes sense to return a response code of 204 NO CONTENT when the request is to restart the connector but not the tasks.
This maintains the current behavior for this existing REST call and is also aligned with the description in the RFC:
https://www.w3.org/Protocols/rfc2616/rfc2616-sec10.html#sec10.2.5
Reviewers: Kalpesh Patel <kpatel@confluent.io>, Randall Hauch <rhauch@gmail.com>
Setting the max.poll.interval.ms to MAX_VALUE causes overflow when computing the joinGroupTimeoutMs and results in the JoinGroup timeout being set to the request.timeout.ms instead, which is much lower.
This can easily make consumers drop out of the group, since they must rejoin now within 30s (by default) yet have no obligation to almost ever call poll() given the high max.poll.interval.ms, especially when each record takes a long time to process or the `max.poll.records` is also very large. We just need to check for overflow and fix it to Integer.MAX_VALUE when it occurs.
Reviewers: Luke Chen <showuon@gmail.com>, John Roesler <vvcephei@apache.org>
Moved the responsibility for recording task and connector startup and failure metrics from the invocation code
into the status listener. The reason behind this is that the WorkerTasks (and subclasses) were either not propagating exceptions upwards, or were unable to do so easily because they were running on completely different threads.
Also split out WorkerMetricsGroup from being an inner class into being a standard class. This was to make sure
the Data Abstraction Count checkStyle rule was not violated.
Author: Michael Carter <michael.carter@instaclustr.com>
Reviewers: Chris Egerton <chrise@confluent.io>, Randall Hauch <rhauch@gmail.com>
[Jira](https://issues.apache.org/jira/browse/KAFKA-13017)
Reverts https://github.com/apache/kafka/pull/7496, which added `ERROR`-level logging for deserialization errors in sink tasks even when connectors had logging for these errors disabled.
No information is lost by this change that cannot be retained in an opt-in fashion by setting `errors.log.enable` and `errors.log.include.messages` to `true` in a connector config.
Reviewers: Arjun Satish <arjun@confluent.io>, Tom Bentley <tbentley@redhat.com>
* Adds SynchronizationTest class for concurrency testing of the classloading isolation mechanism
* Adds a test which deterministically reproduced a deadlock between simultaneous upward (Plugin -> Delegating) & downward (Delegating -> Plugin) class loading operations.
* Makes PluginClassLoader parallel capable, resolving the above deadlock by allowing multiple threads to concurrently use the PluginClassLoader.
* Makes DelegatingClassLoader parallel capable to allow parallel loading of classes from the parent loader (usually the system class loader)
Reviewers: Konstantine Karantasis <k.karantasis@gmail.com>, Tom Bentley <tbentley@redhat.com>, Randall Hauch <rhauch@gmail.com>
The following error message
`org.apache.kafka.connect.errors.DataException: Invalid Java object for schema type INT64: class java.lang.Long for field: "moderate_time"`
can be confusing because java.lang.Long is acceptable type for schema INT64.
In fact, in this case `org.apache.kafka.connect.data.Timestamp` is used but this info is not logged.
Reviewers: Randall Hauch <rhauch@gmail.com>, Chris Egerton <chrise@confluent.io>, Konstantine Karantasis <k.karantasis@gmail.com>