Commit Graph

1188 Commits

Author SHA1 Message Date
Sanskar Jhajharia 27383970b6
MINOR: Cleanup Connect Module (1/n) (#19869)
CI / build (push) Waiting to run Details
Now that Kafka support Java 17, this PR makes some changes in connect
module. The changes in this PR are limited to only some files. A future
PR(s) shall follow.
The changes mostly include:
- Collections.emptyList(), Collections.singletonList() and
Arrays.asList() are replaced with List.of()
- Collections.emptyMap() and Collections.singletonMap() are replaced
with Map.of()
- Collections.singleton() is replaced with Set.of()

Sub modules targeted: api, basic-auth-extensions, file, json, mirror,
mirror-client

Reviewers: Ken Huang <s7133700@gmail.com>, TengYao Chi
<kitingiao@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-07-11 00:35:41 +08:00
Ken Huang d31885d33c
MINOR: Use <code> block instead of backtick (#20107)
CI / build (push) Waiting to run Details
When writing HTML, it's recommended to use the <code> element instead of
backticks for inline code formatting.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, TengYao Chi
<frankvicky@apache.org>
2025-07-06 14:49:51 +08:00
Ken Huang b919836551
KAFKA-17662: config.providers configuration missing from the docs (#18930)
Ensure the config.providers configuration is documented for all
components supporting it

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Greg Harris
<gharris1727@gmail.com>, Matthias J. Sax <mjsax@apache.org>
2025-06-27 14:13:55 +02:00
Hong-Yi Chen dc82c766fa
KAFKA-18834 Fix LoggingResourceTest#testSetLevelDefaultScope (#19920)
- Flaky behavior
`LoggingResourceTest#testSetLevelDefaultScope` sometimes fails by not
capturing its expected WARN log.

- Root cause
Both `LoggersTest#testSetLevelWithValidRootLoggerNames` and
`LoggingResourceTest#testSetLevelDefaultScope` may share the same
`LoggerContext` when executed in the same JVM.
`LoggersTest#testSetLevelWithValidRootLoggerNames` calls
`loggers.setLevel("", ERROR)`, which mutates the global root logger
level to ERROR and suppresses WARN logs, which causes subsequent tests
to fail to emit WARN-level output.

- Fix in this PR
Resets the Log4j configuration after each test in `LoggersTest`,
ensuring that any global changes are reverted.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-26 13:39:08 +08:00
Ken Huang 15ad3016b7
KAFKA-19140 ConnectAssignor#performAssignment parameter can be replace to ConnectProtocolCompatibility (#19476)
CI / build (push) Waiting to run Details
The protocol type; for Connect assignors this is "eager", "compatible",
or "sessioned"

Since `ConnectAssignor` is an interface and the protocol parameter is
restricted to "eager", "compatible", or "sessioned", it aligns with the
existing ConnectProtocolCompatibility enum. Therefore, we can update the
code to use `ConnectProtocolCompatibility` directly.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-06-20 03:07:57 +08:00
Jhen-Yung Hsu 2e968560e0
MINOR: Cleanup simplify set initialization with Set.of (#19925)
Simplify Set initialization and reduce the overhead of creating extra
collections.

The changes mostly include:
- new HashSet<>(List.of(...))
- new HashSet<>(Arrays.asList(...)) / new HashSet<>(asList(...))
- new HashSet<>(Collections.singletonList()) / new
HashSet<>(singletonList())
- new HashSet<>(Collections.emptyList())
- new HashSet<>(Set.of())

This change takes the following into account, and we will not change to
Set.of in these scenarios:
- Require `mutability` (UnsupportedOperationException).
- Allow `duplicate` elements (IllegalArgumentException).
- Allow `null` elements (NullPointerException).
- Depend on `Ordering`. `Set.of` does not guarantee order, so it could
make tests flaky or break public interfaces.

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
 <payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-06-11 18:36:14 +08:00
snehashisp 2694d7aad9
KAFKA-19248: Multiversioning in Kafka Connect - Plugin Loading Isolation Tests (#18325)
This adds tests for [KIP-891](https://cwiki.apache.org/confluence/display/KAFKA/KIP-891%3A+Running+multiple+versions+of+Connector+plugins).
It primarily focuses on tests for the new additions in plugin loading
isolation. It has dependency on the actual KIP implementation PRs and
should be merged post https://github.com/apache/kafka/pull/17742

Reviewers: Greg Harris <greg.harris@aiven.io>
2025-06-04 18:01:18 -07:00
PoAn Yang 949617b0b2
KAFKA-17747: [7/N] Add consumer group integration test for rack aware assignment (#19856)
* Add `RackAwareAssignor`. It uses `racksForPartition` to check the rack
id of a partition and assign it to a member which has the same rack id.
* Add `ConsumerIntegrationTest#testRackAwareAssignment` to check
`racksForPartition` works correctly.

Reviewers: David Jacot <djacot@confluent.io>

---------

Signed-off-by: PoAn Yang <payang@apache.org>
2025-06-04 19:32:17 +02:00
S.Y. Wang 543fb6c848
KAFKA-19336 Upgrade Jackson to 2.19.0 (#19835)
`JsonNode.fields()` method has been deprecated by 
- https://github.com/FasterXML/jackson-databind/issues/4863
- https://github.com/FasterXML/jackson-databind/pull/4871

So modified accordingly.

Reviewers: Luke Chen <showuon@gmail.com>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-05-28 20:53:43 +08:00
yunchi bff5ba4ad9
MINOR: replace .stream().forEach() with .forEach() (#19626)
CI / build (push) Waiting to run Details
replace all applicable `.stream().forEach()` in codebase with just
`.forEach()`.

Reviewers: TengYao Chi <kitingiao@gmail.com>, Ken Huang
<s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-05-04 20:39:55 +08:00
Ken Huang 676e0f2ad6
KAFKA-19139 Plugin#wrapInstance should use LinkedHashMap instead of Map (#19519)
CI / build (push) Waiting to run Details
There will be an update to the PluginMetrics#metricName method: the type
of the tags parameter will be changed
from Map to LinkedHashMap.
This change is necessary because the order of metric tags is important
1. If the tag order is inconsistent, identical metrics may be treated as
distinct ones by the metrics backend
2. KAFKA-18390 is updating metric naming to use LinkedHashMap. For
consistency, we should follow the same approach here.

Reviewers: TengYao Chi <frankvicky@apache.org>, Jhen-Yung Hsu
 <jhenyunghsu@gmail.com>, lllilllilllilili
2025-04-30 10:43:01 +08:00
Nick Guo 51ef2903f7
KAFKA-19178 Replace Vector by ArrayList for PluginClassLoader#getResources (#19529)
The vector is a synchronized collection, and in the case we don't need
to sync. Also, we can use `Collections.enumeration` to convert
collection to enumeration easily.

Reviewers: PoAn Yang <payang@apache.org>, Ken Huang
 <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-04-26 23:39:02 +08:00
snehashisp d6133f6997
KAFKA-18988: Connect Multiversion Support (Updates to status and metrics) (#17988)
Reviewers: Greg Harris <greg.harris@aiven.io>
2025-04-24 10:23:31 -07:00
Andrew Schofield 8d66481a83
KAFKA-17897 Deprecate Admin.listConsumerGroups (#19477)
The final part of KIP-1043 is to deprecate Admin.listConsumerGroups() in
favour of Admin.listGroups() which works for all group types.

Reviewers: PoAn Yang <payang@apache.org>, Chia-Ping Tsai
<chia7712@gmail.com>
2025-04-17 23:00:57 +08:00
xijiu 8bdd73c803
KAFKA-19137 Use `StandardCharsets.UTF_8` instead of `StandardCharsets.UTF_8.name()` (#19464)
Replace `StandardCharsets.UTF_8.name()` with `StandardCharsets.UTF_8` to
avoid UnsupportedEncodingException and optimize the related code at the
same time.

Reviewers: Ken Huang <s7133700@gmail.com>, PoAn Yang
<payang@apache.org>, Chia-Ping Tsai <chia7712@gmail.com>
2025-04-15 23:44:00 +08:00
Mickael Maison b3bc674d32
MINOR: Cleanups in connect:runtime (#19452)
Various cleanups in Connect:
- use enhanced switch
- remove dead code
- convert classes to records

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, TengYao Chi
 <kitingiao@gmail.com>, Ken Huang <s7133700@gmail.com>
2025-04-14 20:04:44 +08:00
Thomas Gebert a65626b6a8
MINOR: Add functionalinterface to the producer callback (#19366)
The Callback interface is a perfect example of a place that can use the
functionalinterface in Java. Strictly for Java, this isn't "required"
since Java will automatically coerce, but for Clojure (and other JVM
languages I belive) to interop with Java lambdas it needs the
FunctionalInterface annotation.

Since FunctionalInterface doesn't add any overhead and provides
compiler-enforced documentation, I don't see any reason *not* to have
this. This has already been added into Kafka Streams here:
https://github.com/apache/kafka/pull/19234#pullrequestreview-2740742487

I am happy to add it to any other spots in that might be useful too.

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-04-06 22:21:09 +08:00
TengYao Chi 74acbd200d
KAFKA-16758: Extend Consumer#close with an option to leave the group or not (#17614)
JIRA: [KAFKA-16758](https://issues.apache.org/jira/browse/KAFKA-16758)
This PR is aim to deliver

[KIP-1092](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=321719077),
please refer to KIP-1092 and KAFKA-16758 for further details.

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Chia-Ping
Tsai <chia7712@gmail.com>, Kirk True <kirk@kirktrue.pro>
2025-04-05 22:02:45 -07:00
Kondrat Bertalan e3800e02f3
KAFKA-18843: MirrorMaker2 unique workerId (#18994)
Change the worker ID to the combination of the host name, flow and a random UUID to make it unique.

Reviewers: Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
2025-03-14 09:49:57 +01:00
Jhen-Yung Hsu a383a954c8
KAFKA-17808: Fix id typo for connector-dlq-adminclient (#19171)
Reviewers: Greg Harris <greg.harris@aiven.io>, Chia-Ping Tsai <chia7712@gmail.com>
2025-03-11 13:37:11 -07:00
Lucas Brutschy fc2e3dfce9
MINOR: Disallow unused local variables (#18963)
Recently, we found a regression that could have been detected by static
analysis, since a local variable wasn't being passed to a method during
a refactoring, and was left unused. It was fixed in
[7a749b5](7a749b589f),
but almost slipped into 4.0. Unused variables are typically detected by
IDEs, but this is insufficient to prevent these kinds of bugs. This
change enables unused local variable detection in checkstyle for Kafka.

A few notes on the usage:
- There are two situations in which people actually want to have a local
variable but not use it. First, there are `for (Type ignored:
collection)` loops which have to loop `collection.length` number of
times, but that do not use `ignored` in the loop body. These are
typically still easier to read than a classical `for` loop. Second, some
IDEs detect it if a return value of a function such as `File.delete` is
not being used. In this case, people sometimes store the result in an
unused local variable to make ignoring the return value explicit and to
avoid the squiggly lines.
- In Java 22, unsued local variables can be omitted by using a single
underscore `_`. This is supported by checkstyle. In pre-22 versions,
IntelliJ allows such variables to be named `ignored` to suppress the
unused local variable warning. This pattern is often (but not
consistently) used in the Kafka codebase. This is, however, not
supported by checkstyle.

Since we cannot switch to Java 22, yet, and we want to use automated
detection using checkstyle, we have to resort to prefixing the unused
local variables with `@SuppressWarnings("UnusedLocalVariable")`. We have
to apply this in 11 cases across the Kafka codebase. While not being
pretty, I'd argue it's worth it to prevent bugs like the one fixed in
[7a749b5](7a749b589f).

Reviewers: Andrew Schofield <aschofield@confluent.io>, David Arthur
<mumrah@gmail.com>, Matthias J. Sax <matthias@confluent.io>, Bruno
Cadonna <cadonna@apache.org>, Kirk True <ktrue@confluent.io>
2025-03-10 09:37:35 +01:00
co63oc 3d7ac0c3d1
MINOR: Fix typos in multiple files (#19102)
Fix typos in multiple files

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-05 14:27:32 +00:00
co63oc e4ece37dbf
Fix typos in multiple files (#19086)
Fix typos in multiple files

Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-03-04 16:05:51 +00:00
Xuan-Zhang Gong 45f932819e
KAFKA-18864:remove the Evolving tag from stable public interfaces (#19036)
The purpose of this PR is to remove the `@InterfaceStability.Evolving` from classes that were created over a year ago.

Reviewers: Jun Rao <junrao@gmail.com>
2025-02-28 13:24:24 -08:00
snehashisp 9dc9973c1c
KAFKA-18863: Connect Multiversion Support (Versioned Connector Creation and related changes) (#17743)
Reviewers: Greg Harris <greg.harris@aiven.io>
2025-02-26 15:12:34 -08:00
Mickael Maison c13324fc16
KAFKA-15995: Adding KIP-877 support to Connect (#17804)
Reviewers: Greg Harris <gharris1727@gmail.com>
2025-02-11 18:27:15 +01:00
Peter Lee 0621c0b4de
KAFKA-18275 Restarting broker in testing should use the same port (#18381)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-02-07 03:02:36 +08:00
Piotr P. Karwasz 666571216b
KAFKA-18483 Disable `Log4jController` and `Loggers` if Log4j Core absent (#18496)
If Log4j Core is absent, most calls to Log4jController and Loggers will end up with a NoClassDefFoundError.

This changeset:

- Profits from the major version bump to rename k.util.Log4jController to LoggingController.
- Removes o.a.l.l.Level from the signature of public methods of o.a.k.connect.runtime.Loggers and replaces it with String.
- Provides an additional no-op implementation of k.util.LoggingController and o.a.k.connect.runtime.Loggers: if Log4j Core is not present on the runtime classpath the no-op implementation will be used.

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2025-02-07 00:04:33 +08:00
Kuan-Po Tseng b99be961b8
KAFKA-18206: EmbeddedKafkaCluster must set features (#18189)
related to KAFKA-18206, set features in EmbeddedKafkaCluster in both streams and connect module, note that this PR also fix potential transaction with empty records in sendPrivileged method as transaction version 2 doesn't allow this kind of scenario.

Reviewers: Justine Olshan <jolshan@confluent.io>
2025-02-05 09:14:36 -08:00
Martin Sillence d001b47093
KAFKA-17792: Efficiently parse decimals with large exponents in Connect Values (#17510)
Reviewers: Greg Harris <greg.harris@aiven.io>, Mickael Maison <mickael.maison@gmail.com>
2025-01-27 09:04:29 -08:00
Ken Huang b4674c002f
KAFKA-18585 Fix fail test ValuesTest#shouldConvertDateValues (#18611)
Reviewers: Divij Vaidya <diviv@amazon.com>, Andrew Schofield <aschofield@confluent.io>
2025-01-21 13:51:39 +00:00
Andrew Schofield 42fd4d1d11
KAFKA-7776: Tests for ISO8601 in Connect value parsing (#18374)
Reviewers: Andrew Schofield <aschofield@confluent.io>
2025-01-16 14:26:03 +00:00
snehashisp 00b22b001f
KAFKA-18211: Override class loaders for class graph scanning in connect. (#18403)
Reviewers: Greg Harris <greg.harris@aiven.io>
2025-01-14 09:45:53 -08:00
Thomas Thornton b35c29401a
KAFKA-18073: Prevent dropped records from failed retriable exceptions (#18146)
Reviewers: Greg Harris <greg.harris@aiven.io>
2025-01-09 10:13:11 -08:00
PoAn Yang a97fb662fd
MINOR: add testRestoreCompactedDeletedConnector back to KafkaConfigBackingStoreTest (#18392)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2025-01-08 17:55:48 +08:00
snehashisp ad3369859c
KAFKA-18419: KIP-891 Connect Multiversion Support (Transformation and Predicate Changes) (#17742)
Reviewers: Greg Harris <greg.harris@aiven.io>
2025-01-06 12:18:45 -08:00
Ismael Juma 409a43eff7
MINOR: Collection/Option usage simplification via methods introduced in Java 9 & 11 (#18305)
Relevant methods:
1. `List.of`, `Set.of`, `Map.of` and similar (introduced in Java 9)
2. Optional: `isEmpty` (introduced in Java 11), `stream` (introduced in Java 9).

Reviewers: Mickael Maison <mimaison@users.noreply.github.com>
2025-01-03 16:13:39 -08:00
Ismael Juma d6f24d3665
Use `instanceof` pattern to avoid explicit cast (#18373)
This feature was introduced in Java 16.

Reviewers: David Arthur <mumrah@gmail.com>, Apoorv Mittal <apoorvmittal10@gmail.com>
2025-01-02 09:32:51 -08:00
YunKui Lu af7f403ae3
MINOR: Optimize log printing format for AbstractHerder#validateConverterConfig (#18199)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-30 23:29:34 +08:00
TengYao Chi 96527be90d
KAFKA-18243 Fix compatibility of Loggers class between log4j and log4j2 (#18185)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-30 01:50:52 +08:00
Kuan-Po Tseng e9d4aa4cfe
KAFKA-18296 Remove deprecated KafkaBasedLog constructor (#18257)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-20 15:14:45 +08:00
Nick Guo 0163fa2d06
KAFKA-18294 Remove deprecated SourceTask#commitRecord (#18260)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-19 01:06:33 +08:00
TengYao Chi b37b89c668
KAFKA-9366 Upgrade log4j to log4j2 (#17373)
This pull request replaces Log4j with Log4j2 across the entire project, including dependencies, configurations, and code. The notable changes are listed below:

1. Introduce Log4j2 Instead of Log4j
2. Change Configuration File Format from Properties to YAML
3. Adds warnings to notify users if they are still using Log4j properties, encouraging them to transition to Log4j2 configurations

Co-authored-by: Lee Dongjin <dongjin@apache.org>

Reviewers: Luke Chen <showuon@gmail.com>, Mickael Maison <mickael.maison@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-14 01:14:31 +08:00
Logan Zhu 92352a96e8
MINOR: ensure SuppressWarnings annotation is effective for mockValidationIsolation (#18158)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-12-13 07:57:29 +08:00
snehashisp f4fe6064cc
KAFKA-18215: KIP-891 Connect Multiversioning Support (Configs and Validation changes for Connectors and Converters) (#17741)
Reviewers: Greg Harris <greg.harris@aiven.io>
2024-12-11 15:34:21 -08:00
TengYao Chi de2ccb5789
KAFKA-18021: Disabled MirrorCheckpointConnector throws RetriableException on task config generation (#18098)
Reviewers: Greg Harris <greg.harris@aiven.io>
2024-12-11 13:56:38 -08:00
Christopher L. Shannon bd6d0fbf3d
KAFKA-16437 Upgrade to Jakarta and Jetty 12 (KIP-1032) (#16754)
This commit implements the changes for KIP-1032. This updates Kafka to Jakarta specs, JavaEE 10 and Jetty 12. The changes here primarily effect Kafka Connect and MM2.

Todo/Notes:

1) I bumped the connect modules to JDK 17 but I also had to bump a couple other things that had a dependency on conect. The tools project depends on connect so that had to be bumped, and streams depends on tools so that needed to be bumped. This means we may need to separate some things if we don't want to enforce JDK 17 on streams.

2) There is an issue with a test in DedicatedMirrorIntegrationTest that I had to change for now that involves escaping characters and not quite sure what to do about it yet. The cause is the Servlet 6 spec changing what is allowed in the path. See: Jetty 12: 400: Ambiguous URI path encoding for path <%=FOO%>~1 (encoded: %3C%25%3DFOO%25%3E%7E1) jetty/jetty.project#11890

3) I had to configure the idle timeout in Jetty requests to match our request timeout so tests didn't fail. This was needed to fix the ConnectWorkerIntegrationTest#testPollTimeoutExpiry() test

Testing is being done by just using the existing tests for Connect and MM2 which should be sufficient.

Reviewers: Greg Harris <greg.harris@aiven.io>, David Arthur <mumrah@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
2024-12-11 23:24:14 +08:00
Dmitry Werner 45835a0e45
MINOR: Cleanup connect runtime module (#18074)
Reviewers: Mickael Maison <mickael.maison@gmail.com>
2024-12-10 18:25:38 +01:00
snehashisp af0054b502
KAFKA-18182: KIP-891 Add VersionRange to Plugins and DelegatingClassLoader APIs (#16984)
Reviewers: Greg Harris <greg.harris@aiven.io>
2024-12-07 09:20:02 -08:00
Patrik Marton 0bbed823e8
KAFKA-17200: Allow the replication of user internal topics (#17815)
Reviewers: Viktor Somogyi-Vass <viktorsomogyi@gmail.com>
2024-12-06 15:23:58 +01:00