Commit Graph

112 Commits

Author SHA1 Message Date
Ken Huang d619cc5b88
KAFKA-19058 Running the streams/streams-scala module tests produces a streams-scala.log (#19324)
Remove streams-scala module log4j2 file appender.

Reviewers: PoAn Yang <payang@apache.org>, Matthias J. Sax <matthias@confluent.io>
2025-04-02 12:12:16 -07: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
Matthias J. Sax ab2facca58
KAFKA-12829: Remove deprecated KStream.process() for old Processor API (#18088)
Reviewers: Bill Bejeck <bill@confluent.io>
2024-12-11 14:28:47 -08:00
KApolinario1120 d83f09d014
KAFKA-18015: Add support for duration based offset reset strategy to Kafka Streams (#17973)
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>
2024-12-11 10:47:25 -08:00
Joao Pedro Fonseca Dantas d5c2029434
KAFKA-16339: [4/4 KStream#flatTransformValues] Remove Deprecated "transformer" methods and classes (#17882)
Reviewer: Matthias J. Sax <matthias@confluent.io>
2024-12-08 20:10:11 -08:00
Peter Lee c76fb5cb9b
KAFKA-17893: Support record keys in the foreignKeyExtractor argument of KTable foreign join (#17756)
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>
2024-12-03 17:34:13 +01:00
Joao Pedro Fonseca Dantas 866f0cc308
KAFKA-16339: [3/4 KStream#transformValues] Remove Deprecated "transformer" methods and classes (#17266)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-11-22 15:07:03 -08:00
Joao Pedro Fonseca Dantas 95650431df
KAFKA-16339: [2/4 KStream#flatTransform] Remove Deprecated "transformer" methods and classes (#17245)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-11-07 17:35:17 -08:00
Joao Pedro Fonseca Dantas 069667b6b2
KAFKA-16339: [1/4 KStream#transform] Remove Deprecated "transformer" methods and classes (#17198)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-11-05 22:04:41 -08:00
PoAn Yang 5a3b544d61
KAFKA-17880 Move integration test from streams module to streams/integration-tests module (#17615)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
2024-11-01 18:21:06 +08:00
Arnav Dadarya 05696037d3
KAFKA-12823 Remove Deprecated method KStream#through (#16761)
Implements KIP-1087

Reviewers: Matthias J. Sax <matthias@confluent.io>, Lucas Brutschy <lbrutschy@confluent.io>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2024-09-28 15:21:26 -07:00
JohnHuang c62c3899aa
KAFKA-12829: Remove deprecated StreamsBuilder#addGlobalStore of old Processor API (#17059)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, Matthias J. Sax <matthias@confluent.io>
2024-09-11 14:22:08 -07:00
Arnav Dadarya 4b5df1f8e9
KAFKA-12826: Remove Deprecated Class Serdes (#17023)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-08-31 19:42:46 -07:00
Caio Guedes e8b27b6a33
KAFKA-16335: Remove deprecated method of StreamPartitioner (#15482)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-08-26 13:16:43 -07:00
Murali Basani a2f89f5412
KAFKA-16333: remove deprecated join method with named param (#16764)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2024-08-22 13:20:56 -07:00
TengYao Chi c3d552d273
KAFKA-12824 Remove Deprecated method KStream#branch (#16803)
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>
2024-08-20 09:05:13 +08:00
PoAn Yang 242114eab9
KAFKA-12830: Remove Deprecated constructor in TimeWindowedDeserializer and TimeWindowedSerde (#16809)
Signed-off-by: PoAn Yang <payang@apache.org>

Reviewers: Josep Prat <josep.prat@aiven.io>
2024-08-09 11:57:44 +02:00
Matthias J. Sax 3922cadc5d
KAFKA-9738: Deprecate old Processor API (#16742)
Implements KIP-1070, which deprecates:
 - Transformer
 - TransformerSupplier
 - ValueTransformer
 - ValueTransformerSupplier
 - MockProcessorContext

Reviewers: Bill Bejeck <bill@confluent.io>
2024-08-02 12:55:53 -07:00
Almog Gavra 1c9f360f4a
KAFKA-15215: migrate StreamedJoinTest to Mockito (#15424)
Migrate StreamedJoinTest to Mockito

Reviewers: Anna Sophie Blee-Goldman <ableegoldman@apache.org>, 
Divij Vaidya <diviv@amazon.com>
2024-02-26 18:52:25 -08:00
Almog Gavra 9309653219
KAFKA-15215: [KIP-954] support custom DSL store providers (#14648)
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>
2023-11-21 13:51:39 -08:00
Lucas Brutschy eb7f490159
chore: Fix scaladoc warnings (#13164)
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>
2023-01-31 09:00:48 -08:00
Vladimir Korenev eeedde7ea9
MINOR: Add implicit for Serde[UUID] to Streams Scala API (#8335)
Reviewers: Matthias J. Sax <matthias@confluent.io>
2022-12-28 14:54:22 -08:00
Josep Prat 5f1810209f
MINOR: Fix small warning on javadoc and scaladoc (#11049)
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>
2022-12-28 13:41:45 -08:00
Matthew de Detrich e138772ba5
MINOR: Update Scalafmt to latest version (#12475)
Reviewers: Divij Vaidya <diviv@amazon.com>, Chris Egerton <fearthecellos@gmail.com>
2022-09-12 10:05:15 -04:00
A. Sophie Blee-Goldman 0928666987
MINOR: change Streams topic-level metrics tag from 'topic-name' to 'topic' (#12310)
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>
2022-06-21 13:10:36 +02:00
A. Sophie Blee-Goldman a6c5a74fdb
KAFKA-13945: add bytes/records consumed and produced metrics (#12235)
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>
2022-06-07 16:02:17 +02:00
Colin Patrick McCabe 4c9eeef5b2
MINOR: add timeouts to streams integration tests (#12216)
Reviewers: David Arthur <mumrah@gmail.com>
2022-05-31 14:22:13 -07:00
Jorge Esteban Quilcate Otoya fa0324485b
KAFKA-13654: Extend KStream process with new Processor API (#11993)
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>
2022-04-19 10:29:28 -05:00
Márton Sigmond e5eb180a6f
MINOR: Pass materialized to the inner KTable instance (#11888)
Reviewers: Luke Chen <showuon@gmail.com>
2022-03-21 17:03:04 +08:00
Ismael Juma 7c2d672413
MINOR: Update library dependencies (Q1 2022) (#11306)
- scala 2.13: 2.13.6 -> 2.13.8
  * Support Java 18 and improve Android compatibility
  * https://www.scala-lang.org/news/2.13.7
  * https://www.scala-lang.org/news/2.13.8
- scala 2.12: 2.12.14 -> 2.12.15. 
  * The `-release` flag now works with Scala 2.12, backend parallelism
    can be enabled via `-Ybackend-parallelism N` and string interpolation
    is more efficient.
  * https://www.scala-lang.org/news/2.12.5
- gradle versions plugin: 0.38.0 -> 0.42.0
  * Minor fixes
  * https://github.com/ben-manes/gradle-versions-plugin/releases/tag/v0.40.0
  * https://github.com/ben-manes/gradle-versions-plugin/releases/tag/v0.41.0
  * https://github.com/ben-manes/gradle-versions-plugin/releases/tag/v0.42.0
- gradle dependency check plugin: 6.1.6 -> 6.5.3
  * Minor fixes
- gradle spotbugs plugin: 4.7.1 -> 5.0.5
  * Fixes and minor improvements
  * There were too many releases to include all the links, include the major version bump
  * https://github.com/spotbugs/spotbugs-gradle-plugin/releases/tag/5.0.0
- gradle scoverage plugin: 5.0.0 -> 7.0.0
  * Support newer Gradle versions and other improvements
  * https://github.com/scoverage/gradle-scoverage/releases/tag/6.0.0
  * https://github.com/scoverage/gradle-scoverage/releases/tag/6.1.0
  * https://github.com/scoverage/gradle-scoverage/releases/tag/7.0.0
- gradle shadow plugin: 7.0.0 -> 7.1.2
  * Support gradle toolchains and security fixes
  * https://github.com/johnrengelman/shadow/releases/tag/7.1.0
  * https://github.com/johnrengelman/shadow/releases/tag/7.1.1
  * https://github.com/johnrengelman/shadow/releases/tag/7.1.2
- bcpkix: 1.66 -> 1.70
  * Several improvements and fixes
  * https://www.bouncycastle.org/releasenotes.html
- jline: 3.12.1 -> 3.21.0
  * Various fixes and improvements
- jmh: 1.32 -> 1.34
  * Compiler blackhole enabled by default when using Java 17 and improved
    gradle incremental compilation
  * https://mail.openjdk.java.net/pipermail/jmh-dev/2021-August/003355.html
  * https://mail.openjdk.java.net/pipermail/jmh-dev/2021-December/003406.html
- scalaLogging: 3.9.3 -> 3.9.4
  * Support for Scala 3.0
- jose4j: 0.7.8 -> 0.7.9
  * Minor fixes
- junit: 5.7.1 -> 5.8.2
  * Minor improvements and fixes
  * https://junit.org/junit5/docs/current/release-notes/index.html#release-notes-5.8.0
  * https://junit.org/junit5/docs/current/release-notes/index.html#release-notes-5.8.1
  * https://junit.org/junit5/docs/current/release-notes/index.html#release-notes-5.8.2
- jqwik: 1.5.0 -> 1.6.3
  * Numerous improvements
  * https://github.com/jlink/jqwik/releases/tag/1.6.0
- mavenArtifact: 3.8.1 -> 3.8.4
- mockito: 3.12.4 -> 4.3.1
  * Removed deprecated methods, `DoNotMock` annotation and
    minor fixes/improvements
  * https://github.com/mockito/mockito/releases/tag/v4.0.0
  * https://github.com/mockito/mockito/releases/tag/v4.1.0
  * https://github.com/mockito/mockito/releases/tag/v4.2.0
  * https://github.com/mockito/mockito/releases/tag/v4.3.0
- scalaCollectionCompat: 2.4.4 -> 2.6.0
  * Minor fixes
  * https://github.com/scala/scala-collection-compat/releases/tag/v2.5.0
  * https://github.com/scala/scala-collection-compat/releases/tag/v2.6.0
- scalaJava8Compat: 1.0.0 -> 1.0.2
  * Minor changes
- scoverage: 1.4.1 -> 1.4.11
  * Support for newer Scala versions
- slf4j: 1.7.30 -> 1.7.32
  * Minor fixes, 1.7.35 automatically uses reload4j and 1.7.33/1.7.34
    cause build failures, so we stick with 1.7.32 for now.
- zstd: 1.5.0-4 -> 1.5.2-1
  * zstd 1.5.2
  * Small refinements and performance improvements
  * https://github.com/facebook/zstd/releases/tag/v1.5.1
  * https://github.com/facebook/zstd/releases/tag/v1.5.2

Checkstyle, spotBugs and spotless will be upgraded separately as they
either require non trivial code changes or they have regressions
that affect us.

Reviewers: Manikumar Reddy <manikumar.reddy@gmail.com>
2022-02-07 15:24:50 -08:00
Victoria Xia 01e6a6ebf2
KAFKA-13261: Add support for custom partitioners in foreign key joins (#11368)
Implements KIP-775.

Co-authored-by: Tomas Forsman <tomas-forsman@users.noreply.github.com>
2021-11-03 10:55:24 -07:00
Josep Prat 83f0ae3821
KAFKA-12862: Update Scala fmt library and apply fixes (#10784)
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>
2021-08-09 12:05:31 +02:00
Israel Ekpo b3905d9f71
KAFKA-8613: New APIs for Controlling Grace Period for Windowed Operations (#10926)
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>
2021-06-30 17:09:19 -07:00
Geordie b0cfd1f4ca
KAFKA-12336 Custom stream naming does not work while calling stream[K… (#10190)
Custom stream naming does not work while calling stream[K, V](topicPattern: Pattern)

Reviewers: Bill Bejeck <bbejeck@apache.org>
2021-06-24 12:07:22 -04:00
John Roesler c3475081c5
KAFKA-10546: Deprecate old PAPI (#10869)
* 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.
2021-06-22 09:17:11 -05:00
Josep Prat a02b19cb77
KAFKA-12796: Removal of deprecated classes under streams-scala (#10710)
Removes previously deprecated methods in older KIPs

Reviewers: Bruno Cadonna <cadonna@apache.org>
2021-05-27 11:30:15 +02:00
ketulgupta1995 d949f1094e
KAFKA-12344 Support SlidingWindows in the Scala API (#10519)
Support SlidingWindows in the Scala API

Reviewers: Leah Thomas <lthomas@confluent.io>, Anna Sophie Blee-Goldman <ableegoldman@apache.org>
2021-04-26 11:38:48 -07:00
John Roesler 4ed7f2cd01
KAFKA-12593: Fix Apache License headers (#10452)
* 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
2021-04-01 10:38:37 -05:00
Chia-Ping Tsai 9af81955c4
KAFKA-12173 Migrate streams:streams-scala module to JUnit 5 (#9858)
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>
2021-03-25 01:04:39 +08:00
Ivan Ponomarev 5552da3a20
KAFKA-5488: Add type-safe split() operator (#9107)
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>
2021-02-04 16:23:35 -08:00
mathieu 550d8b8260
KAFKA-8744: Update Scala API to give names to processors (#9738)
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>
2021-01-28 09:57:18 -05:00
John Roesler 58bd0a6ee3
MINOR: TopologyTestDriver should not require dummy parameters (#9477)
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>
2020-10-22 08:19:01 -05:00
Xavier Léauté 7947c18b57 MINOR update comments and docs to be gender-neutral
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
2020-10-08 17:05:15 -07:00
Andy Coates dc81d442df
KAFKA-10077: Filter downstream of state-store results in spurious tombstones (#9156)
Reviewers: Guozhang Wang <guozhang@confluent.io>, Matthias J. Sax <matthias@confluent.io>
2020-09-25 13:58:04 -07:00
Yuriy Badalyantc 4662ed4aac
MINOR: Fix build scala 2.12 build after KAFKA-10020 (#9245)
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>
2020-09-02 12:49:47 -05:00
Yuriy Badalyantc d0111d3bcd
KAFKA-10020: Create a new version of a scala Serdes without name clash (KIP-616) (#8955)
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>
2020-09-01 16:48:40 -05:00
John Roesler 88d4bc4641
KAFKA-10379: Implement the KIP-478 StreamBuilder#addGlobalStore() (#9148)
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>
2020-08-20 14:06:16 -05:00
Matthias J. Sax 194c56fce2
KAFKA-9274: Mark `retries` config as deprecated and add new `task.timeout.ms` config (#8864)
- 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>
2020-07-21 12:19:13 -07:00
Ismael Juma 7f90a58b69
MINOR: Update Scala to 2.13.3 (#8931)
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>
2020-06-26 10:19:04 -07:00
Matthias J. Sax 712cc5d073
KAFKA-10168: fix StreamsConfig parameter name variable (#8865)
Implements KIP-626.

Reviewers: Boyang Chen <boyang@confluent.io>, John Roesler <john@confluent.io>
2020-06-19 17:41:06 -07:00