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>
Part of KIP-1106.
Adds the public APIs to Kafka Streams, to support the the newly added "by_duration" reset policy,
plus adds the missing "none" reset policy. Deprecates the enum `Topology.AutoOffsetReset` and
all related methods, and replaced them with new overload using the new `AutoOffsetReset` class.
Co-authored-by: Matthias J. Sax <matthias@confluent.io>
Reviewers: Matthias J. Sax <matthias@confluent.io>, Bill Bejeck <bill@confluent.io>, Manikumar Reddy <manikumar.reddy@gmail.com>
Currently, KTable foreign key joins only allow extracting the foreign key from the value of the source record. This forces users to duplicate data that might already exist in the key into the value when the foreign key needs to be derived from both the key and value. This leads to:
- Data duplication
- Additional storage overhead
- Potential data inconsistency if the duplicated data gets out of sync
- Less intuitive API when the foreign key is naturally derived from both key and value
This change allows user to extract the foreign key from the key and value of the source record.
Reviewers: Lucas Brutschy <lbrutschy@confluent.io>
Implements KIP-1087
Reviewers: Matthias J. Sax <matthias@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
The method branch in both Java and Scala KStream class was deprecated in version 2.8:
1) org.apache.kafka.streams.scala.kstream.KStream#branch
2) org.apache.kafka.streams.kstream.KStream#branch(org.apache.kafka.streams.kstream.Predicate<? super K,? super V>...)
3) org.apache.kafka.streams.kstream.KStream#branch(org.apache.kafka.streams.kstream.Named, org.apache.kafka.streams.kstream.Predicate<? super K,? super V>...)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Implementation for KIP-954: support custom DSL store providers
Testing Strategy:
- Updated the topology tests to ensure that the configuration is picked up in the topology builder
- Manually built a Kafka Streams application using a customer DslStoreSuppliers class and verified that it was used
Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, Guozhang Wang <guozhang.wang.us@gmail.com>
Make sure no scaladoc warnings are emitted from the streams-scala project build.
We cannot fully fix all scaladoc warnings due to limitations of the scaladoc tool,
so this is a best-effort attempt at fixing as many warnings as possible. We also
disable one problematic class of scaladoc wornings (link errors) in the gradle build.
The causes of existing warnings are that we link to java members from scaladoc, which
is not possible, or we fail to disambiguate some members.
The broad rule applied in the changes is
- For links to Java members such as [[StateStore]], we use the fully qualified name in a code tag
to make manual link resolution via a search engine easy.
- For some common terms that are also linked to Java members, like [[Serde]], we omit the link.
- We disambiguate where possible.
- In the special case of @throws declarations with Java Exceptions, we do not seem to be able
to avoid the warning altogther.
Reviewers: Matthias J. Sax <mjsax@apache.org>, Guozhang Wang <wangguoz@gmail.com>
Escape the `>` character in javadoc
Escape the `$` character when part of `${}` in scaladoc as this is the way to reference a variable
Reviewers: Matthias J. Sax <matthias@confluent.io>
Changes the tag name from topic-name to just topic to conform to the way this tag is named elsewhere (ie in the clients)
Also:
- fixes a comment about dynamic topic routing
- fixes some indentation in MockRecordCollector
- Undoes the changes to KStreamSplitTest.scala and TestTopicsTest which are no longer necessary after this hotfix
Reviewers: Bruno Cadonna <cadonna@apache.org>
Implementation of KIP-846: Source/sink node metrics for Consumed/Produced throughput in Streams
Adds the following INFO topic-level metrics for the total bytes/records consumed and produced:
bytes-consumed-total
records-consumed-total
bytes-produced-total
records-produced-total
Reviewers: Kvicii <Karonazaba@gmail.com>, Guozhang Wang <guozhang@apache.org>, Bruno Cadonna <cadonna@apache.org>
Updates the KStream process API to cover the use cases
of both process and transform, and deprecate the KStream transform API.
Implements KIP-820
Reviewer: John Roesler <vvcephei@apache.org>
Updates the scala fmt to the latest stable version.
Applies all the style fixes (all source code changes are done by scala
fmt).
Removes setting about dangling parentheses as `true` is already the
default.
Reviewer: John Roesler <john@confluent.io>
Implements KIP-633.
Grace-period is an important parameter and its best to make it the user's responsibility to set it expliclity. Thus, we move off to provide a default and make it a mandatory parameter when creating a window.
Reviewers: A. Sophie Blee-Goldman <sophie@confluent.io>, Luke Chen <showuon@gmail.com>, Matthias J. Sax <matthias@confluent.io>
* Deprecate the old Processor API
* Suppress warnings on all internal usages of the old API
(which will be migrated in other child tickets of KAFKA-8410)
* Add new KStream#process methods, since KAFKA-10603 has not seen any action.
* Standardize license headers in scala, python, and gradle files.
* Relocate copyright attribution to the NOTICE.
* Add a license header check to `spotless` for scala files.
Reviewers: Ewen Cheslack-Postava <ewencp@apache.org>, Matthias J. Sax <mjsax@apache.org>, A. Sophie Blee-Goldman <ableegoldman@apache.org
1. replace org.junit.Assert by org.junit.jupiter.api.Assertions
2. replace org.junit by org.junit.jupiter.api
3. replace Before by BeforeEach
4. replace After by AfterEach
5. remove ExternalResource from all scala modules
6. add explicit AfterClass/BeforeClass to stop/start EmbeddedKafkaCluster
Noted that this PR does not migrate stream module to junit 5 so it does not introduce callback of junit 5 to deal with beforeAll/afterAll. The next PR of migrating stream module can replace explicit beforeAll/afterAll by junit 5 extension. Or we can keep the beforeAll/afterAll if it make code more readable.
Reviewers: John Roesler <vvcephei@apache.org>
Implements KIP-418, that deprecated the `branch()` operator in favor of the newly added and type-safe `split()` operator.
Reviewers: Matthias J. Sax <matthias@confluent.io>, John Roesler <john@confluent.io>
As it's only API extension to match the java API with Named object with lots of duplication, I only tested the logic once.
Reviewers: Bill Bejeck <bbejeck@apache.org>
TopologyTestDriver comes with a paper cut that it passes through a
config requirement that application.id and bootstrap.servers must be
configured. But these configs are not required in the context of
TopologyTestDriver specifically. This change relaxes the requirement.
Reviewers: Boyang Chen <boyang@apache.org>, Matthias J. Sax <mjsax@apache.org>
While this is not technically part of KIP-629, I believe this makes our codebase more inclusive as well.
cc gwenshap
Author: Xavier Léauté <xvrl@apache.org>
Reviewers: Gwen Shapira
Closes#9398 from xvrl/neutral-term
Fixes a problem in which the Serdes class in the same package as
the tests (the old one) overshadows the one we explicitly imported
(the new one), but only in Scala 2.12. Since users (hopefully) don't
put their classes in our packages, they won't face the same problem.
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, David Arthur <mumrah@gmail.com>, John Roesler <vvcephei@apache.org>
Wildcard import of the old org.apache.kafka.streams.scala.Serdes leads
to a name clash because some of implicits has the same names as types
from the scala's std lib. The new oak.streams.scala.serialization.Serdes is
the same as the old Serdes, but without name clashes.
The old one is marked as deprecated.
Also, add missing serdes for UUID, ByteBuffer and Short types in
the new Serdes.
Implements: KIP-616
Reviewers: John Roesler <vvcephei@apache.org>
From KIP-478, implement the new StreamBuilder#addGlobalStore() overload
that takes a stateUpdateSupplier fully typed Processor<KIn, VIn, Void, Void>.
Where necessary, use the adapters to make the old APIs defer to the new ones,
as well as limiting the scope of this change set.
Reviewers: Boyang Chen <boyang@apache.org>
- 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>
I had to fix several compiler errors due to deprecation of auto application of `()`. A related
Xlint config (`-Xlint:nullary-override`) is no longer valid in 2.13, so we now only enable it
for 2.12. The compiler flagged two new inliner warnings that required suppression and
the semantics of `&` in `@nowarn` annotations changed, requiring a small change in
one of the warning suppressions.
I also removed the deprecation of a number of methods in `KafkaZkClient` as
they should not have been deprecated in the first place since `KafkaZkClient` is an
internal class and we still use these methods in the Controller and so on. This
became visible because the Scala compiler now respects Java's `@Deprecated`
annotation.
Finally, I included a few minor clean-ups (eg using `toBuffer` instead `toList`) when fixing
the compilation warnings.
Noteworthy bug fixes in Scala 2.13.3:
* Fix 2.13-only bug in Java collection converters that caused some operations to perform an extra pass
* Fix 2.13.2 performance regression in Vector: restore special cases for small operands in appendedAll and prependedAll
* Increase laziness of #:: for LazyList
* Fixes related to annotation parsing of @Deprecated from Java sources in mixed compilation
Full release notes:
https://github.com/scala/scala/releases/tag/v2.13.3
Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>