The class claims to be immutable, but there are some mutable features of this class.
Increase the immutability of it and add a little cleanup:
* Pre-initialize size of ArrayList
* Remove superfluous syntax
* Use ArrayList instead of LinkedList since the list is created once
Reviewers: Ron Dagostino <rdagostino@confluent.io>, Konstantine Karantasis <konstantine@confluent.io>
* 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>
Added check if the transformation is abstract. If so throw an error message with guidance for the user. Ensure that the child classes are also not abstract.
Author: Jeremy Custenborder <jcustenborder@gmail.com>
Reviewer: Randall Hauch <rhauch@gmail.com>
Add actual values instead of just passing null in unit tests that check the behavior of the InsertField SMT when trying to insert a field that takes its value from the Kafka record timestamp.
Reviewers: Chris Egerton <chrise@confluent.io>, Konstantine Karantasis <konstantine@confluent.io>
This change turns on exact decimal processing in JSON Converter for deserializing decimals, meaning trailing zeros are maintained. Serialization was already using the decimal scale to output the right value, so this change means a value of `1.2300` can now be serialized to JSON and deserialized back to Connect without any loss of information.
Author: Andy Coates <big-andy-coates@users.noreply.github.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Almog Gavra <almog@confluent.io>
The rest.advertised.listener config is currently broken as setting it to http when listeners are configured for both https and http will cause the framework to choose whichever of the two listeners is listed first. The changes here attempt to fix this by checking not only that ServerConnector::getName begins with the specified protocol, but also that that protocol is immediately followed by an underscore, which the framework uses as a delimiter between the protocol and the remainder of the connector name.
An existing unit test for the RestServer::advertisedUrl method has been expanded to include a case that fails with the framework in its current state and passes with the changes in this commit.
* KAFKA-9768: Fix handling of rest.advertised.listener config
* KAFKA-9768: Add comments on server connector names
* KAFKA-9768: Update RestServerTest comment
Co-authored-by: Randall Hauch <rhauch@gmail.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Andrew Choi <andchoi@linkedin.com>
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>
Simple logging additions at TRACE level that should help when the worker can't get caught up to the end of an internal topic.
Reviewers: Gwen Shapira <cshapi@gmail.com>, Aakash Shah <ashah@confluent.io>, Konstantine Karantasis <konstantine@confluent.io>
* The DeadLetterQueueReporter has a KafkaProducer that it must close to clean up resources
* Currently, the producer and its threads are leaked every time a task is stopped
* Responsibility for cleaning up ErrorReporters is transitively assigned to the
ProcessingContext, RetryWithToleranceOperator, and WorkerSinkTask/WorkerSinkTask classes
* One new unit test in ErrorReporterTest asserts that the producer is closed by the dlq reporter
Reviewers: Arjun Satish <arjun@confluent.io>, Chris Egerton <chrise@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>
Fixed typos in two MM2 configs that define the replication factor for internal Connect topics.
Only a single test was affected.
Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>
When the Connect worker forwards a REST API request to the leader, it might get back a `RequestTargetException` that suggests the worker should forward the request to a different worker. This can happen when the leader changes, and the worker that receives the original request forwards the request to the worker that it thinks is the current leader, but that worker is not the current leader. In this case. In most cases, the worker that received the forwarded request includes the URL of the current leader, but it is possible (albeit rare) that the worker doesn’t know the current leader and will include a null leader URL in the resulting `RequestTargetException`.
When this rare case happens, the user gets a null pointer exception in their response and the NPE is logged. Instead, the worker should catch this condition and provide a more useful error message that is similar to other existing error messages that might occur.
Added a unit test that verifies this corner case is caught and this particular NPE does not occur.
Author: Randall Hauch <rhauch@gmail.com>
Reviewer: Konstantine Karantasis <konstantine@confluent.io>
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>
* Fixed DataException thrown when handling tombstone events with null value
* Passes through original record when finding a null key when it's configured for keys or a null value when it's configured for values.
* Added unit tests for schema and schemaless data
In case of an error while flattening a record with schema, the Flatten transformation was reporting an error about a record without schema, as follows:
```
org.apache.kafka.connect.errors.DataException: Flatten transformation does not support ARRAY for record without schemas (for field ...)
```
The expected behaviour would be an error message specifying "with schemas".
This looks like a simple copy/paste typo from the schemaless equivalent methods, in the same file
Reviewers: Ewen Cheslack-Postava <me@ewencp.org>, Konstantine Karantasis <konstantine@confluent.io>
For reasons outlined in https://issues.apache.org/jira/browse/KAFKA-9771
we can't upgrade to a version of Jetty with the bug fixed, or downgrade to one prior to the introduction of the bug. Luckily, the actual fix is pretty straightforward and can be ported over to Connect for use until it's possible to upgrade to a version of Jetty with that bug fixed: https://github.com/eclipse/jetty.project/pull/4404/files#diff-58640db0f8f2cd84b7e653d1c1540913R2188-R2193
The changes here have been verified locally; a test with multiple certificates/multiple hostnames will be submitted in a follow up.
Reviewers: Jeff Huang <47870461+jeffhuang26@users.noreply.github.com>, Konstantine Karantasis <konstantine@confluent.io>
* KAFKA-9707: Fix InsertField.Key not applying to tombstone events
* Fix typo that hardcoded .value() instead of abstract operatingValue
* Add test for Key transform that was previously not tested
Signed-off-by: Greg Harris <gregh@confluent.io>
* Add null value assertion to tombstone test
* Remove mis-named function and add test for passing-through a null-keyed record.
Signed-off-by: Greg Harris <gregh@confluent.io>
* Simplify unchanged record assertion
Signed-off-by: Greg Harris <gregh@confluent.io>
* Replace assertEquals with assertSame
Signed-off-by: Greg Harris <gregh@confluent.io>
* Fix checkstyleTest indent issue
Signed-off-by: Greg Harris <gregh@confluent.io>
* The Herder interface is extended with a default method that allows choosing whether to log all the connector configurations during connector validation or not.
* The `PUT /connector-plugins/{connector-type}/config/validate` is modified to stop logging the connector's configurations when a validation request is hitting this endpoint. Validations during creation or reconfiguration of a connector are still logging all the connector configurations at the INFO level, which is useful in general and during troubleshooting in particular.
Co-authored-by: Konstantine Karantasis <konstantine@confluent.io>
Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Ismael Juma <github@juma.me.uk>
Simple fix to return in all cases an immutable list from poll in SchemaSourceTask.
Co-authored-by: David Mollitor <dmollitor@apache.org>
Reviewers: Ron Dagostino <rdagostino@confluent.io>, Ismael Juma <github@juma.me.uk>, Konstantine Karantasis <konstantine@confluent.io>
When incorrect connector configuration is detected, the returned exception message suggests to check the connector's configuration against the `{connectorType}/config/validate` endpoint.
Changing the error message to refer to the exact REST endpoint which is `/connector-plugins/{connectorType}/config/validate`
This aligns the exception message with the documentation at: https://kafka.apache.org/documentation/#connect_rest
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>
Exit.exit needs to be used in code instead of System.exit.
Particularly in integration tests using System.exit is disrupting because it exits the jvm process and does not just fail the test correctly. Integration tests override procedures in Exit to protect against such cases.
Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Randall Hauch <rhauch@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.
This commit works around a bug in version v0.9.12 of the upstream `reflections` library by catching and handling the exception thrown.
The reflections issue is tracked by:
https://github.com/ronmamo/reflections/issues/273
New unit tests were introduced to test the behavior.
* KAFKA-9712: Catch and handle exception thrown by reflections scanner
* Update connect/runtime/src/main/java/org/apache/kafka/connect/runtime/isolation/DelegatingClassLoader.java
Co-Authored-By: Konstantine Karantasis <konstantine@confluent.io>
* Move result initialization back to right before it is used
* Use `java.io.File` in tests
* Fix checkstyle
Co-authored-by: Konstantine Karantasis <konstantine@confluent.io>
Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Chia-Ping Tsai <chia7712@gmail.com>
Correct the Connect worker logic to properly disable the new topic status (KIP-558) feature when `topic.tracking.enable=false`, and fix automatic topic status reset after a connector is deleted.
Also adds new `ConnectorTopicsIntegrationTest` and expanded unit tests.
Reviewers: Randall Hauch <rhauch@gmail.com>
Changed `EmbeddedConnectCluster` to add utility methods that return `Response`, throw `ConnectException` instead of `IOException` for failures, and deprecate the old methods that returned primitive types rather than `Response`.
Also introduce common assertions for embedded clusters under `EmbeddedConnectClusterAssertions`.
Author: Konstantine Karantasis <konstantine@confluent.io>
Reviewer: Randall Hauch <rhauch@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
The test case `org.apache.kafka.connect.mirror.MirrorConnectorsIntegrationTest.testReplication` has shown to be increasingly flaky recently. This PR aims to make this test more deterministic. Specifically, the flakiness was due to a timing issue between the tasks not starting up in time for the test to start running. This PR remediates that by introducing a status check after every connector is started up. These status checks include that the connector is found on the connect cluster as well as there are tasks created and up and running for that connector. These checks are introduced before the test starts running so that there is a confidence that the connectors and tasks are started up correctly before the test runs.
Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Jason Gustafson <jason@confluent.io>
* KAFKA-9074: Correct Connect’s `Values.parseString` to properly parse a time and timestamp literal
Time and timestamp literal strings contain a `:` character, but the internal parser used in the `Values.parseString(String)` method tokenizes on the colon character to tokenize and parse map entries. The colon could be escaped, but then the backslash character used to escape the colon is not removed and the parser fails to match the literal as a time or timestamp value.
This fix corrects the parsing logic to properly parse timestamp and time literal strings whose colon characters are either escaped or unescaped. Additional unit tests were added to first verify the incorrect behavior and then to validate the correction.
Author: Randall Hauch <rhauch@gmail.com>
Reviewers: Chris Egerton <chrise@confluent.io>, Nigel Liang <nigel@nigelliang.com>, Jason Gustafson <jason@confluent.io>
`emit.heartbeats.enabled` and `emit.checkpoints.enabled` are supposed to
be the knobs to control if the heartbeat message or checkpoint message
will be sent or not to the topics respectively. In our experiments,
setting them to false will not suspend the activity in their SourceTasks,
e.g. MirrorHeartbeatTask, MirrorCheckpointTask.
The observations are, when setting those knobs to false, huge volume of
`SourceRecord` are being sent without interval, causing significantly high
CPU usage and GC time of MirrorMaker 2 instance and congesting the single
partition of the heartbeat topic and checkpoint topic.
The proposed fix in the following PR is to (1) explicitly check if `interval`
is set to negative (e.g. -1), when the `emit.heartbeats.enabled` or
`emit.checkpoints.enabled` is off. (2) if `interval` is indeed set to negative,
no task is created.
Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ryanne Dolan <ryannedolan@gmail.com>
This feature corresponds to KIP-558 and extends how the internal status topic (set via `status.storage.topic` distributed worker config) is used to include information that allows Kafka Connect to keep track which topics a connector is using.
The set of topics a connector is actively using, is exposed via a new endpoint that is added to the REST API of Connect workers.
* A `GET /connectors/{name}/topics` request will return the set of topics that have been recorded as active since a connector started or since the set of topics was reset for this connector.
An additional endpoints allows users to reset the set of active topics for a connector via the second endpoint that this feature is adding:
* A `PUT /connectors/{name}/topics/reset` request clears the set of active topics. An operator may enable or disable this feature by setting `topic.tracking.enable` (true by default).
The `topic.tracking.enable` worker config property (true by default) allows an operator to enable/disable the entire feature. Or if the feature is enabled, the `topic.tracking.allow.reset` worker config property (true by default) allows an operator to control whether reset requests submitted to the Connect REST API are allowed.
Author: Konstantine Karantasis <konstantine@confluent.io>
Reviewer: Randall Hauch <rhauch@gmail.com>
This commit makes `DistributedHerder` log that some error has happened during task reconfiguration only when it actually has happened.
Author: Ivan Yurchenko <ivan0yurchenko@gmail.com>
Reviewer: Randall Hauch <rhauch@gmail.com>