Addition of configs for custom topic creation with KIP-158 created a regression when transformation configs are also included in the configuration of a source connector.
To experience the issue, just enabling topic creation at the worker is not sufficient. A user needs to supply a source connector configuration that contains both transformations and custom topic creation properties.
The issue is that the enrichment of configs in `SourceConnectorConfig` happens on top of an `AbstractConfig` rather than a `ConnectorConfig`. Inheriting from the latter allows enrichment to be composable for both topic creation and transformations.
Unit tests and integration tests are written to test these combinations.
Reviewers: Randall Hauch <rhauch@gmail.com>
- part of KIP-572
- deprecates producer config `retries` (still in use)
- deprecates admin config `retries` (still in use)
- deprecates Kafka Streams config `retries` (will be ignored)
- adds new Kafka Streams config `task.timeout.ms` (follow up PRs will leverage this new config)
Reviewers: John Roesler <john@confluent.io>, Jason Gustafson <jason@confluent.io>, Randall Hauch <randall@confluent.io>
If we call org.apache.kafka.common.utils.Exit#exit(int code) with code=0, the current implementation will go into an infinite recursion and kill the VM with a stack overflow error. This happens only in integration tests because of the overrides of shutdown procedures and this commit addresses this issue by removing the redundant call to Exit#exit.
Reviewers: Lucas Bradstreet <lucas@confluent.io>, Konstantine Karantasis <k.karantasis@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
The `testBlockInConnectorStop` test is failing semi-frequently on Jenkins. It's difficult to verify the cause without complete logs and I'm unable to reproduce locally, but I suspect the cause may be that the Connect worker hasn't completed startup yet by the time the test begins and so the initial REST request to create a connector times out with a 500 error. This isn't an issue for normal tests but we artificially reduce the REST request timeout for these tests as some requests are meant to exhaust that timeout.
The changes here use a small hack to verify that the worker has started and is ready to handle all types of REST requests before tests start by querying the REST API for a non-existent connector.
Reviewers: Boyang Chan <boyang@confluent.io>, Konstantine Karantasis <k.karantasis@gmail.com>
A benign `WakeupException` can be thrown by a sink task's consumer if the task is scheduled for shutdown by the worker. This is caught and handled gracefully if the exception is thrown when calling `poll` on the consumer, but not if calling `commitSync`, which is invoked by a task during shutdown and also when its partition assignment is updated.
If thrown during a partition assignment update, the `WakeupException` is caught and handled gracefully as part of the task's `iteration` loop. If thrown during shutdown, however, it is not caught and instead leads to the misleading log message "Task threw an uncaught and unrecoverable exception. Task is being killed and will not recover until manually restarted.".
These changes catch the `WakeupException` during shutdown and handle it gracefully with a `TRACE`-level log message.
A unit test is added to verify this behavior by simulating a thrown `WakeupException` during `Consumer::commitSync`, running through the `WorkerSinkTask::execute` method, and confirming that it does not throw a `WakeupException` itself.
Reviewers: Greg Harris <gregh@confluent.io>, Nigel Liang <nigel@nigelliang.com>, Konstantine Karantasis <k.karantasis@gmail.com>
This PR includes 3 MessageFormatters for MirrorMaker2 internal topics:
- HeartbeatFormatter
- CheckpointFormatter
- OffsetSyncFormatter
This also introduces a new public interface org.apache.kafka.common.MessageFormatter that users can implement to build custom formatters.
Reviewers: Konstantine Karantasis <k.karantasis@gmail.com>, Ryanne Dolan <ryannedolan@gmail.com>, David Jacot <djacot@confluent.io>
Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
Co-authored-by: Edoardo Comar <ecomar@uk.ibm.com>
Improved the exception messages that are thrown to indicate whether it was a key or value conversion problem.
Author: Mario Molina <mmolimar@gmail.com>
Reviewer: Randall Hauch <rhauch@gmail.com>
A simple increase in the timeout of the consumer that verifies that records have been replicated seems to fix the integration tests in `MirrorConnectorsIntegrationTest` that have been failing more often recently.
Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Sanjana Kaundinya <skaundinya@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>
In order to make the Kafka consumer and stream application migrate from source to target cluster
transparently and conveniently, e.g. in event of source cluster failure, a background job is proposed
to periodically sync the consumer offsets from the source to target cluster, so that when the
consumer and stream applications switche to the target cluster, they will resume to consume from
where they left off at source cluster.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ryanne Dolan <ryannedolan@gmail.com>, Thiago Pinto, Srinivas Boga
Upgrade jetty to 9.4.27.v20200227 and jersey to 2.31
Also remove the workaround used on previous versions from Connect's SSLUtils.
(Reverts KAFKA-9771 - commit ee832d7d)
Reviewers: Ismael Juma <ismael@juma.me.uk>, Chris Egerton <chrise@confluent.io>, Konstantine Karantasis <konstantine@confluent.io>
* Add documentation for using transformation predicates.
* Add `PredicateDoc` for generating predicate config docs, following the style of `TransformationDoc`.
* Fix the header depth mismatch.
* Avoid generating HTML ids based purely on the config name since there
are very likely to conflict (e.g. #name). Instead allow passing a function
which can be used to generate an id from a config key.
The docs have been generated and tested locally.
Reviewer: Konstantine Karantasis <konstantine@confluent.io>
Commit 9a4f00f78b changed the constructor of DescribeConfigsResponse.
The build failed with the following error:
```
/home/chia7712/kafka/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java:582: error: incompatible types: int cannot be converted to Struct
return new DescribeConfigsResponse(1000, configs);
^
Note: Some messages have been simplified; recompile with -Xdiags:verbose to get full output
1 error
```
Reviewers: Boyang Chan <boyang@confluent.io>, Tom Bentley <tbentley@redhat.com>, Konstantine Karantasis <konstantine@confluent.io>
These changes allow herders to continue to function even when a connector they are running hangs in its start, stop, initialize, validate, and/or config methods.
The main idea is to make these connector interactions asynchronous and accept a callback that can be invoked upon the completion (successful or otherwise) of these interactions. The distributed herder handles any follow-up logic by adding a new herder request to its queue in that callback, which helps preserve some synchronization and ordering guarantees provided by the current tick model.
If any connector refuses to shut down within a graceful timeout period, the framework will abandon it and potentially start a new connector in its place (in cases such as connector restart or reconfiguration).
Existing unit tests for the distributed herder and worker have been modified to reflect these changes, and a new integration test named `BlockingConnectorTest` has been added to ensure that they work in practice.
Reviewers: Greg Harris <gregh@confluent.io>, Nigel Liang <nigel@nigelliang.com>, Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>
Make sure that the Errant Record Reporter recently added in KIP-610 adheres to the `errors.tolerance` policy.
Author: Aakash Shah <ashah@confluent.io>
Reviewers: Arjun Satish <arjunconfluent.io>, Randall Hauch <rhauch@gmail.com>
This change adds a check to the KafkaConfigBackingStore, KafkaOffsetBackingStore, and KafkaStatusBackingStore to use the admin client to verify that the internal topics are compacted and do not use the `delete` cleanup policy.
Connect already will create the internal topics with `cleanup.policy=compact` if the topics do not yet exist when the Connect workers are started; the new topics are created always as compacted, overwriting any user-specified `cleanup.policy`. However, if the topics already exist the worker did not previously verify the internal topics were compacted, such as when a user manually creates the internal topics before starting Connect or manually changes the topic settings after the fact.
The current change helps guard against users running Connect with topics that have delete cleanup policy enabled, which will remove all connector configurations, source offsets, and connector & task statuses that are older than the retention time. This means that, for example, the configuration for a long-running connector could be deleted by the broker, and this will cause restart issues upon a subsequent rebalance or restarting of Connect worker(s).
Connect behavior requires that its internal topics are compacted and not deleted after some retention time. Therefore, this additional check is simply enforcing the existing expectations, and therefore does not need a KIP.
Author: Randall Hauch <rhauch@gmail.com>
Reviewer: Konstantine Karantasis <konstantine@confluent.io>, Chris Egerton <chrise@confluent.io>
With Incremental Cooperative Rebalancing, if a worker returns after it's been out of the group for sometime (essentially as a zombie worker) and hasn't voluntarily revoked its own connectors and tasks in the meantime, there's the possibility that these assignments have been distributed to other workers and redundant connectors and tasks might be running now in the Connect cluster.
This PR complements previous fixes such as KAFKA-9184, KAFKA-9849 and KAFKA-9851 providing a last line of defense against zombie tasks: if at any rebalance round the leader worker detects that there are duplicate assignments in the group, it revokes them completely and resolves duplication with a correct assignment in the rebalancing round that will follow task revocation.
Author: Wang <ywang50@ebay.com>
Reviewer: Konstantine Karantasis <konstantine@confluent.io>
This fix excludes `ConnectorClientConfigRequest` and its inner class from class loading isolation in a similar way that KAFKA-8415 excluded `ConnectorClientConfigOverridePolicy`.
Reviewer: Konstantine Karantasis <konstantine@confluent.io>
Struct value validation in Kafka Connect can be optimized
to avoid creating an Iterator when the expectedClasses list is of
size 1. This is a meaningful enhancement for high throughput
connectors.
Reviewers: Konstantine Karantasis <konstantine@confluent.io>
When Incremental Cooperative Rebalancing is enabled and a worker fails to read to the end of the config topic, it needs to voluntarily revoke its locally running tasks on time, before these tasks get assigned to another worker, creating a situation where redundant tasks are running in the Connect cluster.
Additionally, instead of using the delay `worker.unsync.backoff.ms` that was defined for the eager rebalancing protocol and has a long default value (which coincidentally is equal to the default value of the rebalance delay of the incremental cooperative protocol), the worker should quickly attempt to re-read the config topic and backoff for a fraction of the rebalance delay. After this fix, the worker will retry for a maximum time of 5 times before it revokes its running assignment and for a cumulative delay less than the configured `scheduled.rebalance.max.delay.ms`.
Unit tests are added to cover the backoff logic with incremental cooperative rebalancing.
Reviewers: Randall Hauch <rhauch@gmail.com>
In the first version of the incremental cooperative protocol, in the presence of a failed sync request by the leader, the assignor was designed to treat the unapplied assignments as lost and trigger a rebalance delay.
This commit applies optimizations in these cases to avoid the unnecessary activation of the rebalancing delay. First, if the worker that loses the sync group request or response is the leader, then it detects this failure by checking the what is the expected generation when it performs task assignments. If it's not the expected one, it resets its view of the previous assignment because it wasn't successfully applied and it doesn't represent a correct state. Furthermore, if the worker that has missed the assignment sync is an ordinary worker, then the leader is able to detect that there are lost assignments and instead of triggering a rebalance delay among the same members of the group, it treats the lost tasks as new tasks and reassigns them immediately. If the lost assignment included revocations that were not applied, the leader reapplies these revocations again.
Existing unit tests and integration tests are adapted to test the proposed optimizations.
Reviewers: Randall Hauch <rhauch@gmail.com>
Currently, Kafka Connect creates its config backing topic with a fire and forget approach.
This is fine unless someone has manually created that topic already with the wrong partition count.
In such a case Kafka Connect may run for some time. Especially if it's in standalone mode and once switched to distributed mode it will almost certainly fail.
This commits adds a check when the KafkaConfigBackingStore is starting.
This check will throw a ConfigException if there is more than one partition in the backing store.
This exception is then caught upstream and logged by either:
- DistributedHerder#run
- ConnectStandalone#main
A unit tests was added in KafkaConfigBackingStoreTest to verify the behaviour.
Author: Evelyn Bayes <evelyn@confluent.io>
Co-authored-by: Randall Hauch <rhauch@gmail.com>
Reviewer: Konstantine Karantasis <konstantine@confluent.io>
Until recently revocation of connectors and tasks was the result of a rebalance that contained a new assignment. Therefore the view of the running assignment was kept consistent outside the call to `RebalanceListener#onRevoke`. However, after KAFKA-9184 the need appeared for the worker to revoke tasks voluntarily and proactively without having received a new assignment.
This commit will allow the worker to restart tasks that have been stopped as a result of voluntary revocation after a rebalance reassigns these tasks to the work.
The fix is tested by extending an existing integration test.
Reviewers: Randall Hauch <rhauch@gmail.com>
Define SSL configs in all worker config classes, not just distributed
Author: Chris Egerton <chrise@confluent.io>
Reviewers: Nigel Liang <nigel@nigelliang.com>, Randall Hauch <rhauch@gmail.com>
Connector projects may have their own mock or testing implementations of the `SinkTaskContext`, and this newly-added method should be a default method to prevent breaking those projects. Changing this to a default method that returns null also makes sense w/r/t the method semantics, since the method is already defined to return null if the reporter has not been configured.
Author: Randall Hauch <rhauch@gmail.com>
Reviewer: Konstantine Karantasis <konstantine@confluent.io>
This applies to the producer, consumer, admin client, connect worker
and inter broker communication.
`ClientDnsLookup.DEFAULT` has been deprecated and a warning
will be logged if it's explicitly set in a client config.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk>
With the recent introduction of predicated SMTs, properties named "predicate" and "negate" should be ignored and removed in case they are present in transformation configs.
This commit fixes the equality check to be with the key of the config to apply proper removal.
Reviewers: Tom Bentley <tbentley@redhat.com>, Konstantine Karantasis <konstantine@confluent.io>
This class has been unused since KIP-146 was merged, which introduced another way to load Connect plugins. Removing from the code base.
Reviewers: Boyang Chan <boyang@confluent.io>, Konstantine Karantasis <konstantine@confluent.io>
1. Remove redundant connect#stop from test in InternalTopicsIntegrationTest since we'll do it after each test case in the @After method
2. Refine the error message in topic assertions to make it better explain the errors
Reviewers: Konstantine Karantasis <konstantine@confluent.io>
Implemented KIP-585 to support Filter and Conditional SMTs. Added unit tests and integration tests.
Author: Tom Bentley <tbentley@redhat.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>
Implementation for KIP-610: https://cwiki.apache.org/confluence/display/KAFKA/KIP-610%3A+Error+Reporting+in+Sink+Connectors based on which sink connectors can now report errors at the final stages of the stream that exports records to the sink system.
This PR adds the `ErrantRecordReporter` interface as well as its implementation - `WorkerErrantRecordReporter`. The `WorkerErrantRecordReporter` is created in `Worker` and brought up through `WorkerSinkTask` to `WorkerSinkTaskContext`.
An integration test and unit tests have been added.
Reviewers: Lev Zemlyanov <lev@confluent.io>, Greg Harris <gregh@confluent.io>, Chris Egerton <chrise@confluent.io>, Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>
A recently added assertion in Connect integration tests uses `KafkaConsumer#partitionsFor` to verify that a topic was created with the expected number of partitions and replicas. However, probably because of metadata propagation delays, this call doesn't always return a valid `PartitionInfo` for the topic that has just been created and the test is terminated with a NPE.
This commit changes the assertion to perform retries in order to verify the topic settings and uses the admin client to describe the topics instead.
Tests have been adjusted to use the new assertion.
Reviewers: Randall Hauch <rhauch@gmail.com>
Implemented KIP-606 to add metadata context to MetricsReporter.
Author: Xiaodong Du <xdu@confluent.io>
Reviewers: David Arthur <mumrah@gmail.com>, Randall Hauch <rhauch@gmail.com>, Xavier Léauté <xavier@confluent.io>, Ryan Pridgeon <ryan.n.pridgeon@gmail.com>
Kafka Connect workers have been able to create Connect's internal topics using the new admin client for some time now (see KAFKA-4667 for details). However, tasks of source connectors are still relying upon the broker to auto-create topics with default config settings if they don't exist, or expect these topics to exist before the connector is deployed, if their configuration needs to be specialized.
With the implementation of KIP-158 here, if `topic.creation.enable=true`, Kafka Connect will supply the source tasks of connectors that are configured to create topics with an admin client that will allow them to create new topics on-the-fly before writing the first source records to a new topic. Additionally, each source connector has the opportunity to customize the topic-specific settings of these new topics by defining groups of topic configurations.
This feature is tested here via unit tests (old tests that have been adjusted and new ones) as well as integration tests.
Reviewers: Randall Hauch <rhauch@gmail.com>
Although the statuses for tasks are removed from the status store when their connector is deleted, their statuses are not removed when only the task is deleted, which happens in the case that the number of tasks for a connector is reduced.
This commit adds logic for deleting the statuses for those tasks from the status store whenever a rebalance has completed and the leader of a distributed cluster has detected that there are recently-deleted tasks. Standalone is also updated to accomplish this.
Unit tests for the `DistributedHerder` and `StandaloneHerder` classes are updated and an integration test has been added.
Reviewers: Nigel Liang <nigel@nigelliang.com>, Konstantine Karantasis <konstantine@confluent.io>
Added access to OffsetStorageReader from SourceConnector per KIP-131.
Added two interfaces SinkConnectorContext/SourceConnectContext that extend ConnectorContext in order to expose an OffsetStorageReader instance.
Added unit tests for Connector, SinkConnector and SourceConnector default methods
Author: Florian Hussonnois <florian.hussonnois@gmail.com>, Randall Hauch <rhauch@gmail.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>
Implemented KIP-437 by adding a new optional configuration property for the `MaskField` transformation that allows users to define a replacement literal for specific fields in matching records.
Author: Valeria Vasylieva <valeria.vasylieva@gmail.com>
Reviewer: Randall Hauch <rhauch@gmail.com>
Added support for customizing the HTTP response headers for Kafka Connect as described in KIP-577.
Author: Jeff Huang <jeff.huang@confluent.io>
Reviewer: Randall Hauch <rhauch@gmail.com>
The changes made in KIP-454 involved adding a `connectorConfig` method to the ConnectClusterState interface that REST extensions could use to query the worker for the configuration of a given connector. The implementation for this method returns the Java `Map` that's stored in the worker's view of the config topic (when running in distributed mode). No copying is performed, which causes mutations of that `Map` to persist across invocations of `connectorConfig` and, even worse, propagate to the worker when, e.g., starting a connector.
In this commit the map is copied before it's returned to REST extensions.
An existing unit test is modified to ensure that REST extensions receive a copy of the connector config, not the original.
Reviewers: Nigel Liang <nigel@nigelliang.com>, Konstantine Karantasis <konstantine@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>