Metadata may be updated from the background thread, so we need to protect access to SubscriptionState. This patch restructures the metadata handling so that we only check pattern subscriptions in the foreground. Additionally, it improves the following:
1. SubscriptionState is now the source of truth for the topics that will be fetched. We had a lot of messy logic previously to try and keep the the topic set in Metadata consistent with the subscription, so this simplifies the logic.
2. The metadata needs for the producer and consumer are quite different, so it made sense to separate the custom logic into separate extensions of Metadata. For example, only the producer requires topic expiration.
3. We've always had an edge case in which a metadata change with an inflight request may cause us to effectively miss an expected update. This patch implements a separate version inside Metadata which is bumped when the needed topics changes.
4. This patch removes the MetadataListener, which was the cause of https://issues.apache.org/jira/browse/KAFKA-7764.
Reviewers: David Arthur <mumrah@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
KAFKA-7880: Name worker thread to include task id
Change Connect's `WorkerTask` to name the thread using the `task-thread-<connectorTaskId>` pattern.
Reviewers: Randall Hauch <rhauch@gmail.com>
MINOR: Increase produce timeout for EmbeddedKafkaCluster to 120 seconds
Previous value was 500ms. This change gives more room to pass tests on systems with low resources running many parallel tests.
Reviewers: Randall Hauch <randall@confluent.io>
The test `org.apache.kafka.connect.runtime.rest.RestServerTest#testCORSEnabled` assumes Jersey client can send restricted HTTP headers(`Origin`).
Jersey client uses `sun.net.www.protocol.http.HttpURLConnection`.
`sun.net.www.protocol.http.HttpURLConnection` drops restricted headers(`Host`, `Keep-Alive`, `Origin`, etc) based on static property `allowRestrictedHeaders`.
This property is initialized in a static block by reading Java system property `sun.net.http.allowRestrictedHeaders`.
So, if classloader loads `HttpURLConnection` before we set `sun.net.http.allowRestrictedHeaders=true`, then all subsequent changes of this system property won't take any effect(which happens if `org.apache.kafka.connect.integration.ExampleConnectIntegrationTest` is executed before `RestServerTest`).
To prevent this, we have to either make sure we set `sun.net.http.allowRestrictedHeaders=true` as early as possible or do not rely on this system property at all.
This PR adds test dependency on `httpcomponents-client` which doesn't depend on `sun.net.http.allowRestrictedHeaders` system property. Thus none of existing tests should interfere with `RestServerTest`.
Author: Alex Diachenko <sansanichfb@gmail.com>
Reviewers: Randall Hauch, Konstantine Karantasis, Gwen Shapira
Closes#6236 from avocader/KAFKA-7799
JUnit 4.13 fixes the issue where `Category` and `Parameterized` annotations
could not be used together. It also deprecates `ExpectedException` and
`assertThat`. Given this, we:
- Replace `ExpectedException` with the newly introduced `assertThrows`.
- Replace `Assert.assertThat` with `MatcherAssert.assertThat`.
- Annotate `AbstractLogCleanerIntegrationTest` with `IntegrationTest` category.
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>, David Arthur <mumrah@gmail.com>
Explicitly seek KafkaBasedLog’s consumer to the beginning of the topic partitions, rather than potentially use committed offsets (which would be unexpected) if group.id is set or rely upon `auto.offset.reset=earliest` if the group.id is null.
This should not change existing behavior but should remove some potential issues introduced with KIP-287 if `group.id` is not set in the consumer configurations. Note that even if `group.id` is set, we still always want to consume from the beginning.
Reviewers: Jason Gustafson <jason@confluent.io>
[KIP-297](https://cwiki.apache.org/confluence/display/KAFKA/KIP-297%3A+Externalizing+Secrets+for+Connect+Configurations#KIP-297:ExternalizingSecretsforConnectConfigurations-PublicInterfaces) introduced the `ConfigProvider` mechanism, which was primarily intended for externalizing secrets provided in connector configurations. However, when querying the Connect REST API for the configuration of a connector or its tasks, those secrets are still exposed. The changes here prevent the Connect REST API from ever exposing resolved configurations in order to address that. rhauch has given a more thorough writeup of the thinking behind this in [KAFKA-5117](https://issues.apache.org/jira/browse/KAFKA-5117)
Tested and verified manually. If these changes are approved unit tests can be added to prevent a regression.
Author: Chris Egerton <chrise@confluent.io>
Reviewers: Robert Yokota <rayokota@gmail.com>, Randall Hauch <rhauch@gmail.com, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#6129 from C0urante/hide-provided-connect-configs
Start the Rest server in the standalone mode similar to how it's done for distributed mode.
Author: Magesh Nandakumar <magesh.n.kumar@gmail.com>
Reviewers: Arjun Satish <arjun@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#6148 from mageshn/KAFKA-7826
Added testing of logical types for Kafka Connect in support of KIP-145 features.
Added tests for Boolean, Time, Date and Timestamp, including the valid conversions.
The area of ISO8601 strings is a bit of a mess because the tokenizer is not compatible with
that format, and a subsequent JIRA will be needed to fix that.
A few small fixes as well as creating test cases, but they're clearly just corrections such as
using 0 to mean January (java.util.Calendar uses zero-based month numbers).
Author: Andrew Schofield <andrew_schofield@uk.ibm.com>
Reviewers: Mickael Maison <mimaison@users.noreply.github.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#6077 from AndrewJSchofield/KAFKA-7461-ConverterValuesLogicalTypesTest
Expose a programmatic way to bring up a Kafka and Zk cluster through Java API to facilitate integration tests for framework level changes in Kafka Connect. The Kafka classes would be similar to KafkaEmbedded in streams. The new classes would reuse the kafka.server.KafkaServer classes from :core, and provide a simple interface to bring up brokers in integration tests.
Signed-off-by: Arjun Satish <arjunconfluent.io>
Author: Arjun Satish <arjun@confluent.io>
Author: Arjun Satish <wicknicks@users.noreply.github.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5516 from wicknicks/connect-integration-test
The test always fails if testOptionsDoesNotIncludeWadlOutput is executed before testCORSEnabled. It seems the problem is the use of the system property. Perhaps there is some static caching somewhere.
Reviewers: Randall Hauch <rhauch@gmail.com>, Guozhang Wang <wangguoz@gmail.com>
The null map returned from the current snapshot causes the null type in response. The connector class name can be taken from the config of request instead since we require the config should contain the connector class name.
Reviewers: Jason Gustafson <jason@confluent.io>
When using the Connect `JsonConverter`, it's impossible to produce tombstone messages, thus impacting the compaction of the topic. This patch allows the converter with and without schemas to output a NULL byte value in order to have a proper tombstone message. When it's regarding to get this data into a connect record, the approach is the same as when the payload looks like `"{ "schema": null, "payload": null }"`, this way the sink connectors can maintain their functionality and reduces the BCC.
Reviewers: Gunnar Morling <gunnar.morling@googlemail.com>, Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
This patch disables support for WADL output in the Connect REST API since it was never intended to be exposed.
Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
Should be ported back to 2.0
Author: Cyrus Vafadari <cyrus@confluent.io>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#6004 from cyrusv/cyrus-npe
This is minor refactoring that brings in the creation of producer and consumer to the Worker. Currently, the consumer is created in the WorkerSinkTask. This should not affect any functionality and it just makes the code structure easier to understand.
Author: Magesh Nandakumar <magesh.n.kumar@gmail.com>
Reviewers: Ryanne Dolan <ryannedolan@gmail.com>, Randall Hauch <rhauch@gmail.com>, Robert Yokota <rayokota@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5842 from mageshn/KAFKA-7551
Includes Update to ConnectRecord string representation to give
visibility into schemas, useful in SMT tracing
Author: Cyrus Vafadari <cyrus@confluent.io>
Reviewers: Randall Hauch <rhauch@gmail.com>, Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5860 from cyrusv/cyrus-logging
The restart logic for TTLs in `WorkerConfigTransformer` was broken when trying to make it toggle-able. Accessing the toggle through the `Herder` causes the same code to be called recursively. This fix just accesses the toggle by simply looking in the properties map that is passed to `WorkerConfigTransformer`.
Author: Robert Yokota <rayokota@gmail.com>
Reviewers: Magesh Nandakumar <magesh.n.kumar@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5914 from rayokota/KAFKA-7620
Changes made as part of this [KIP-374](https://cwiki.apache.org/confluence/x/FgSQBQ) and [KAFKA-7418](https://issues.apache.org/jira/browse/KAFKA-7418)
- Checking for empty args or help option in command file to print Usage
- Added new class to enforce help option to all commands
- Refactored few lines (ex `PreferredReplicaLeaderElectionCommand`) to
make use of `CommandDefaultOptions` attributes.
- Made the changes in help text wordings
Run the unit tests in local(Windows) few Linux friendly tests are failing but
not any functionality, verified `--help` and no option response by running
Scala classes, since those all are having `main` method.
Author: Srinivas Reddy <srinivas96alluri@gmail.com>
Author: Srinivas Reddy <mrsrinivas@users.noreply.github.com>
Author: Srinivas <srinivas96alluri@gmail.com>
Reviewers: Colin Patrick McCabe <colin@cmccabe.xyz>, Manikumar Reddy <manikumar.reddy@gmail.com>, Guozhang Wang <wangguoz@gmail.com>, Mickael Maison <mickael.maison@gmail.com>
Closes#5910 from mrsrinivas/KIP-374
While metrics like Min, Avg and Max make sense to respective use Double.MAX_VALUE, 0.0 and Double.MIN_VALUE as default values to ease computation logic, exposing those values makes reading them a bit misleading. For instance, how would you differentiate whether your -avg metric has a value of 0 because it was given samples of 0 or no samples were fed to it?
It makes sense to standardize on the output of these metrics with something that clearly denotes that no values have been recorded.
Reviewers: Bill Bejeck <bill@confluent.io>, John Roesler <john@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
Some connector configs may be sensitive, so we should avoid logging them.
Reviewers: Alex Diachenko, Dustin Cote <dustin@confluent.io>, Jason Gustafson <jason@confluent.io>
- Use Xlint:all with 3 exclusions (filed KAFKA-7613 to remove the exclusions)
- Use the same javac options when compiling tests (seems accidental that
we didn't do this before)
- Replaced several deprecated method calls with non-deprecated ones:
- `KafkaConsumer.poll(long)` and `KafkaConsumer.close(long)`
- `Class.newInstance` and `new Integer/Long` (deprecated since Java 9)
- `scala.Console` (deprecated in Scala 2.11)
- `PartitionData` taking a timestamp (one of them seemingly a bug)
- `JsonMappingException` single parameter constructor
- Fix unnecessary usage of raw types in several places.
- Add @SuppressWarnings for deprecations, unchecked and switch fallthrough in
several places.
- Scala clean-ups (var -> val, ETA expansion warnings, avoid reflective calls)
- Use lambdas to simplify code in a few places
- Add @SafeVarargs, fix varargs usage and remove unnecessary `Utils.mkList` method
Reviewers: Matthias J. Sax <mjsax@apache.org>, Manikumar Reddy <manikumar.reddy@gmail.com>, Randall Hauch <rhauch@gmail.com>, Bill Bejeck <bill@confluent.io>, Stanislav Kozlovski <stanislav_kozlovski@outlook.com>
[KAFKA-7431](https://issues.apache.org/jira/browse/KAFKA-7431)
Changes made to improve the code readability:
- Removed `throws Exception` from the place where there won't be an
exception
- Removed type arguments where those can be inferred explicitly by compiler
- Rewritten Anonymous classes to Java 8 with lambdas
Author: Srinivas Reddy <mrsrinivas@users.noreply.github.com>
Author: Srinivas Reddy <srinivas96alluri@gmail.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Ismael Juma <ismael@juma.me.uk>, Ryanne Dolan <ryannedolan@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5681 from mrsrinivas/cleanup-connect-uts
This patch makes two improvements to internal metadata handling logic and testing:
1. It reduce dependence on the public object `Cluster` for internal metadata propagation since it is not easy to evolve. As an example, we need to propagate leader epochs from the metadata response to `Metadata`, but it is not straightforward to do this without exposing it in `PartitionInfo` since that is what `Cluster` uses internally. By doing this change, we are able to remove some redundant `Cluster` building logic.
2. We want to make the metadata handling in `MockClient` simpler and more consistent. Currently we have mix of metadata update mechanisms which are internally inconsistent with each other and do not match the implementation in `NetworkClient`.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
KIP-368 implementation to enable periodic re-authentication of SASL clients. Also adds a broker configuration option to terminate client connections that do not re-authenticate within the configured interval.
Adds `client.dns.lookup=resolve_canonical_bootstrap_servers_only` option to perform full dns resolution of bootstrap addresses
Reviewers: Colin Patrick McCabe <colin@cmccabe.xyz>, Sriharsha Chintalapani <sriharsha@apache.org>, Edoardo Comar <ecomar@uk.ibm.com>, Mickael Maison <mickael.maison@gmail.com>, Manikumar Reddy <manikumar.reddy@gmail.com>, Rajini Sivaram <rajinisivaram@googlemail.com>
Implementation of KIP-302: Based on the new client configuration `client.dns.lookup`, a NetworkClient can use InetAddress.getAllByName to find all IPs and iterate over them when they fail to connect. Only uses either IPv4 or IPv6 addresses similar to the default mode.
Co-authored-by: Edoardo Comar <ecomar@uk.ibm.com>
Co-authored-by: Mickael Maison <mickael.maison@gmail.com>
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
- SslFactoryTest should use SslFactory to create SSLEngine
- Use Mockito instead of EasyMock in `ConsoleConsumerTest` as one of
the tests mocks a standard library class and the latest released EasyMock
version can't do that when Java 11 is used.
- Avoid mocking `ConcurrentMap` in `SourceTaskOffsetCommitterTest`
for similar reasons. As it happens, mocking is not actually needed here.
Reviewers: Rajini Sivaram <rajinisivaram@googlemail.com>
The parent classloader of the DelegatingClassLoader and therefore the classloading scheme used by Connect does not have to be fixed to the System classloader.
Setting it the same as the one that was used to load the DelegatingClassLoader class itself is more flexible and, while in most cases will result in the System classloader to be used, it will also work in othr managed environments that control classloading differently (OSGi, and others).
The fix is minimal and the mainstream use is tested via system tests.
Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>
Various converters (AvroConverter and JsonConverter) produce a
SchemaAndValue consisting of a logical schema type and a java.util.Date.
This is a fix for SchemaProjector to properly handle the Date.
Author: Robert Yokota <rayokota@gmail.com>
Reviewers: Konstantine Karantasis <konstantine@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5736 from rayokota/KAFKA-7476
Allow to cast LogicalType to string by calling the serialized (Java) object's toString().
Added tests for `BigDecimal` and `Date` as whole record and as fields.
Author: Amit Sela <amitsela33@gmail.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Robert Yokota <rayokota@gmail.com>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#4820 from amitsela/cast-transform-bytes
Switches to normal year format instead of week date years and day of month instead of day of year.
This is directly from #4820, but separated into a different JIRA/PR to keep the fixes independent. Original authorship should be maintained in the commit.
Author: Amit Sela <amitsela33@gmail.com>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5718 from ewencp/fix-header-converter-date-format
*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: Michał Borowiecki <mbor81@gmail.com>
Reviewers: Arjun Satish <arjun@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5700 from mihbor/KAFKA-7434
Modified several classes' `equals` methods and simplified a complex method to
reduce the NPath complexity so they could be removed from the checkstyle
suppressions that were required with the recent move to Java 8 and upgrade
of Checkstyle: https://github.com/apache/kafka/pull/5046.
Reviewers: Robert Yokota <rayokota@gmail.com>, Arjun Satish <arjun@confluent.io>, Ismael Juma <ismael@juma.me.uk>
findBugs is abandoned, it doesn't work with Java 9 and the Gradle plugin will be deprecated in
Gradle 5.0: https://github.com/gradle/gradle/pull/6664
spotBugs is actively maintained and it supports Java 8, 9 and 10. Java 11 is not supported yet,
but it's likely to happen soon.
Also fixed a file leak in Connect identified by spotbugs.
Manually tested spotBugsMain, jarAll and importing kafka in IntelliJ and running
a build in the IDE.
Author: Ismael Juma <ismael@juma.me.uk>
Reviewers: Colin Patrick McCabe <colin@cmccabe.xyz>, Dong Lin <lindong28@gmail.com>
Closes#5625 from ijuma/kafka-5887-spotbugs
Replace 'this' reference in anonymous inner class logs to out class's 'this'
Author: Kevin Lafferty <kevin.lafferty@gmail.com>
Reviewers: Randall Hauch <rhauch@gmail.com>, Arjun Satish <arjun@confluent.io>, Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5583 from kevin-laff/connect_logging
During actions such as a reconfiguration, the task configs are obtained
via `Worker.connectorTaskConfigs` and then subsequently saved into an
instance of `ClusterConfigState`. The values of the properties that are saved
are post-transformation (of variable references) when they should be
pre-transformation. This is to avoid secrets appearing in plaintext in
the `connect-configs` topic, for example.
The fix is to change the 2 clients of `Worker.connectorTaskConfigs` to
perform a reverse transformation (values converted back into variable
references) before saving them into an instance of `ClusterConfigState`.
The 2 places where the save is performed are
`DistributedHerder.reconfigureConnector` and
`StandaloneHerder.updateConnectorTasks`.
The way that the reverse transformation works is by using the
"raw" connector config (with variable references still intact) from
`ClusterConfigState` to convert config values back into variable
references for those keys that are common between the task config
and the connector config.
There are 2 additional small changes that only affect `StandaloneHerder`:
1) `ClusterConfigState.allTasksConfigs` has been changed to perform a
transformation (resolution) on all variable references. This is
necessary because the result of this method is compared directly to
`Worker.connectorTaskConfigs`, which also has variable references
resolved.
2) `StandaloneHerder.startConnector` has been changed to match
`DistributedHerder.startConnector`. This is to fix an issue where
during `StandaloneHerder.restartConnector`, the post-transformed
connector config would be saved back into `ClusterConfigState`.
I also performed an analysis of all other code paths where configs are
saved back into `ClusterConfigState` and did not find any other
issues.
Author: Robert Yokota <rayokota@gmail.com>
Reviewers: Ewen Cheslack-Postava <ewen@confluent.io>
Closes#5475 from rayokota/KAFKA-7242-reverse-xform-props
Currently logical types are dropped during Cast Transformation.
This patch fixes this behaviour.
Reviewers: Randall Hauch <rhauch@gmail.com>, Jason Gustafson <jason@confluent.io>