...
Here's a sample code fragment using the Scala wrapper library. Compare this example to the Scala code for the same example using the Java API directly in Confluent's repository.
Code Block | ||||
---|---|---|---|---|
| ||||
// Compute the total per region by summing the individual click counts per region. val clicksPerRegion: KTableS[String, Long] = userClicksStream // Join the stream against the table .leftJoin(userRegionsTable, (clicks: Long, region: String) => (if (region == null) "UNKNOWN" else region, clicks)) // Change the stream from <user> -> <region, clicks> to <region> -> <clicks> .map((_, regionWithClicks) => regionWithClicks) // Compute the total per region by summing the individual click counts per region. .groupByKey .reduce(_ + _) |
Implicit Serdes
One of the common complaints of Scala users with the Java API has been the repetitive usage of the serdes in API invocations. Many of the APIs need to take the serdes through abstractions like Serialized
, Consumed
, Produced
or Joined
. And the user has to supply them every time through the with
function of these classes.
The library uses the power of Scala implicits to alleviate this concern. As a user you can provide implicit serdes or implicit values of Serialized
, Joined
, Consumed
or Produced
once and make your code less verbose. In fact you can just have the implicit serdes in scope and the library will make the instances of Serialized
, Produced
, Consumed
or Joined
available in scope.
The library also bundles all implicit serdes of the commonly used primitive types in a Scala module - so just import the module vals and have all serdes in scope. Similar strategy of modular implicits can be sdopted for any user-defined serdes as well.
Here's an example:
Code Block | ||||
---|---|---|---|---|
| ||||
// DefaultSerdes brings into scope implicit serdes (mostly for primitives) // that will set up all Serialized, Produced, Consumed and Joined instances. // So all APIs below that accept Serialized, Produced, Consumed or Joined will // get these instances automatically import DefaultSerdes._ val builder = new StreamsBuilderS() val userClicksStream: KStreamS[String, Long] = builder.stream(userClicksTopic) val userRegionsTable: KTableS[String, String] = builder.table(userRegionsTopic) // The following code fragment does not have a single instance of Serialized, // Produced, Consumed or Joined supplied explicitly. // All of them are taken care of by the implicit serdes imported by DefaultSerdes val clicksPerRegion: KTableS[String, Long] = userClicksStream .leftJoin(userRegionsTable, (clicks: Long, region: String) => (if (region == null) "UNKNOWN" else region, clicks)) .map((_, regionWithClicks) => regionWithClicks) .groupByKey .reduce(_ + _) clicksPerRegion.toStream.to(outputTopic) |
Compare the code with the one that does not use implicit serdes and you will see the difference in verbosity. Also the library does not depend on serdes being supplied with the configuration that opens up a whole can of type unsafety in the mix. Instead the library expects implicit serdes to be available in scope for all the Scala APIs. For any missing serdes, it will emit compiler error which makes the library much more type-safe than the corresponding Java one.
Compatibility, Deprecation, and Migration Plan
- What impact (if any) will there be on existing users?
- If we are changing behavior how will we phase out the older behavior?
- If we need special migration tools, describe them here.
- When will we remove the existing behavior?
Rejected Alternatives
...
Quite a few things are going on in the above code snippet that may warrant a few lines of elaboration:
- The code snippet does not depend on any config defined serdes. In fact any serde defined as part of the config will be ignored
- All serdes are picked up from the implicits in scope. And import DefaultSerdes._ brings all necessary serdes in scope.
- This is an example of compile time type safety that we don't have in the Java APIs
- The code looks less verbose and more focused towards the actual transformation that it does on the data stream
When the default primitive serdes are not enough and we need to define custom serdes, the usage is exactly the same as above. Just define the implicit serdes and start building the stream transformation. Here's an example with AvroSerde
:
Code Block | ||||
---|---|---|---|---|
| ||||
// domain object as a case class
case class UserClicks(clicks: Long)
// An implicit Serde implementation for the values we want to
// serialize as avro
implicit val userClicksSerde: Serde[UserClicks] = new AvroSerde
// Primitive serdes
import DefaultSerdes._
// And then business as usual ..
val userClicksStream: KStreamS[String, UserClicks] = builder.stream(userClicksTopic)
val userRegionsTable: KTableS[String, String] = builder.table(userRegionsTopic)
// Compute the total per region by summing the individual click counts per region.
val clicksPerRegion: KTableS[String, Long] =
userClicksStream
// Join the stream against the table.
.leftJoin(userRegionsTable, (clicks: UserClicks, region: String) => (if (region == null) "UNKNOWN" else region, clicks.clicks))
// Change the stream from <user> -> <region, clicks> to <region> -> <clicks>
.map((_, regionWithClicks) => regionWithClicks)
// Compute the total per region by summing the individual click counts per region.
.groupByKey
.reduce(_ + _)
// Write the (continuously updating) results to the output topic.
clicksPerRegion.toStream.to(outputTopic) |
A complete example of user-defined serdes can be found in a test class within the library.
Scala Version Compatibility
Binary Compatibility
When two versions of Scala are binary compatible, it is safe to compile your project on one Scala version and link against another Scala version at run time.
http://docs.scala-lang.org/overviews/core/binary-compatibility-of-scala-releases.html
Binary compatibility is a common concern for Scala library authors. Scala releases are always backward and forward binary compatible between minor releases since Scala 2.10.x. This is automatically enforced by use of the Scala Binary Compatibility validation tool (MiMa). However binary compatibility is typically broken across major releases.
Scala major versions 2.11 and 2.12 are not binary compatible due to compiler changes that use several new language features made available in Java 8. Scala 2.13 has not been released yet, but it’s anticipated to be binary incompatible with 2.12. The Scala 2.13 release has a central theme of core library changes which will cause incompatibility across libraries compiled using earlier versions of Scala.
If there’s a desire MiMa could be used as part of the build and release process to manage binary compatibility for kafka-streams-scala releases inline with Apache Kafka’s version policy.
Source Compatibility
Two library versions are Source Compatible with each other if switching one for the other does not incur any compile errors or unintended behavioral changes (semantic errors).
To support multiple major versions of Scala it is necessary to cross build a source compatible project with two or more versions of Scala. This is commonly done between major versions of Scala such as 2.10/2.11 and 2.11/2.12.
Due to fundamental core library changes that will be released in 2.13 (such as the collections redesign effort), it’s anticipated source compatibility will be an issue due to the ubiquitous use of collections libraries. It’s anticipated that Lightbend will release a compatibility library that allows the library author to preserve source compatibility so that managing multiple code branches won’t be necessary. Guides from Lightbend will also be made available to make managing this transition as easy as possible for library authors.
Further Reading
Binary Compatibility for Library Authors
http://docs.scala-lang.org/overviews/core/binary-compatibility-for-library-authors.htmlScala Binary Compatibility validation tool (MiMa)
https://github.com/lightbend/migration-managerScala 2.13 Roadmap
https://www.scala-lang.org/news/roadmap-2.13.html
New or Changed Public Interfaces
Scala wrapped API’s include abstractions for the KGroupedStream
, KGroupedTable
, KStream
, KTable
, SessionWindowedKStream
, StreamBuilder
, and TimeWindowedKStream
types available in Kafka Streams. For details see the Scala API documentation.
The API docs for kafka-streams-scala is available here for Scala 2.12 and here for Scala 2.11.
The current kafka-streams-scala
project will be cannibalized and integrated into apache/kafka as a sub-project of the streams project called streams:scala
that depends on streams. A new package would be defined as org.apache.kafka.streams.scala
.
Fully qualified class names:
o.a.k.stream.scala
StreamsBuilderS
o.a.k.stream.scala.kstream
KGroupedStreamS
,KGroupedTableS
,KStreamS
,KTableS
,SessionWindowedKStreamS
,TimeWindowedKStreamS
The streams:scala
project will be integrated into the root build.gradle file. During release new build artifacts will be created that are cross-built with Scala 2.11 and 2.12.
Migration Plan and Compatibility
N/A
Current Status
Kafka Streams Scala library is available as an open source project from Lightbend on Github. The current available version is 0.2.0.
Rejected Alternatives
None.