MINOR: Build and code sample updates for Kafka Streams DSL for Scala#4949
MINOR: Build and code sample updates for Kafka Streams DSL for Scala#4949guozhangwang merged 4 commits intoapache:trunkfrom
Conversation
|
FYI @miguno, @guozhangwang, @ijuma, @debasishg |
| println "Building project 'streams-scala' with Scala version ${versions.scala}" | ||
| apply plugin: 'scala' | ||
| archivesBaseName = "kafka-streams-scala" | ||
| archivesBaseName = "kafka-streams-scala_${versions.baseScala}" |
There was a problem hiding this comment.
Unrelated question, why is scalaLogging just a test compile dependency?
There was a problem hiding this comment.
It's required by the LazyLogging trait referenced by several tests, but it doesn't look like any logging is actually done. It's probably safe to remove this. @debasishg can you provide some insight here?
There was a problem hiding this comment.
+1 .. it's there, along with log4j.properties as part of logging infrastructure from the tests. Currently not used though. We can remove the imports and do away with the dependency. Do we keep log4j.properties ?
There was a problem hiding this comment.
Let's remove the dependency. I think it makes sense to keep log4j.properties so users can see what's happening with the embedded Kafka instance and Kafka streams when the tests are run.
|
@seglo Thanks a lot for the PR. It lgtm overall. Just a side note: could you double check if the release script https://github.com/apache/kafka/blob/trunk/release.py for publishing the built artifacts to maven central as part of the release process (https://cwiki.apache.org/confluence/display/KAFKA/Release+Process) needs to be updated; I made a quick look around https://github.com/apache/kafka/blob/trunk/release.py#L451-L456 and looks like we are well covered, but better have another pair of eyes. |
|
Hey @guozhangwang, I took a peek at The first line builds and publishes all artifacts compiled with the 1.7 JDK, whereas the second line I think just publishes Kafka core for Scala 2.12 compiled with 1.8 JDK. I tried executing I think we could add that command to |
|
I think you'd want the equivalent of |
ijuma
left a comment
There was a problem hiding this comment.
LGTM for the build and test changes. Someone from Streams needs to review the documentation changes.
| .flatMapValues(textLine => textLine.toLowerCase.split("\\W+")) | ||
| .groupBy((_, word) => word) | ||
| .count(Materialized.as("counts-store")) | ||
| .count(Materialized.as("counts-store").withKeySerde(DefaultSerdes.stringSerde)) |
There was a problem hiding this comment.
@seglo @debasishg do we still need to enforce withKeySerde after https://github.com/apache/kafka/pull/4919/files#diff-09491c933b9023904f73054c5246c893R128 has been merged?
guozhangwang
left a comment
There was a problem hiding this comment.
Doc changes LGTM except a minor question.
|
@guozhangwang I've confirmed that with #4919 the Key serdes is no longer required. Good job! I updated the documentation code samples accordingly. |
|
@seglo Thanks! I've just merged it to trunk. |
…pache#4949) Several build and documentation updates were required after the merge of KAFKA-6670: Implement a Scala wrapper library for Kafka Streams. Encode Scala major version into streams-scala artifacts. To differentiate versions of the kafka-streams-scala artifact across Scala major versions it's required to encode the version into the artifact name before its published to a maven repository. This is accomplished by following a similar release process as kafka core, which encodes the Scala major version and then runs the build for each major version of Scala supported. This is considered standard practice when releasing Scala libraries, but is not handled for us automatically with the basic Scala for Gradle support. After this change you can generate and install the kafka-streams-scala artifact into the local maven repository: $ ./gradlew -PscalaVersion=2.11 install $ ./gradlew -PscalaVersion=2.12 install Reviewers: Ismael Juma <ismael@juma.me.uk>, Guozhang Wang <wangguoz@gmail.com>
Several build and documentation updates were required after the merge of KAFKA-6670: Implement a Scala wrapper library for Kafka Streams.
Encode Scala major version into
streams-scalaartifacts.To differentiate versions of the
kafka-streams-scalaartifact across Scala major versions it's required to encode the version into the artifact name before its published to a maven repository. This is accomplished by following a similar release process as kafka core, which encodes the Scala major version and then runs the build for each major version of Scala supported. This is considered standard practice when releasing Scala libraries, but is not handled for us automatically with the basic Scala for Gradle support.After this change you can generate and install the
kafka-streams-scalaartifact into the local maven repository:Which results in the following files generated:
Review Code Samples
I reviewed all the code samples introduced by the main PR. I created a sample project which validates the main WordCount example. The other examples are code snippets instead of whole programs, but are equivalent to existing tests found in the
apache/kafkastreams-scalaproject, or the originallightbend/kafka-streams-scalaproject.Code sample / documentation references
WordCountApplication example
Docs location:
/documentation/streams/(Scala Example)/documentation/streams/developer-guide/dsl-api.html#scala-dsl-sample-usageSee
WordCountApplicationin sample project.Implicit SerDes Example
Docs location:
/documentation/streams/developer-guide/dsl-api.html#scala-dsl-implicit-serdesSee
StreamToTableJoinScalaIntegrationTestImplicitSerdestest inapache/kafkahttps://github.com/apache/kafka/blob/trunk/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/StreamToTableJoinScalaIntegrationTestImplicitSerdes.scala#L77..L102
User-defined SerDes Example
Docs location:
/documentation/streams/developer-guide/dsl-api.html#scala-dsl-user-defined-serdesSee
StreamToTableJoinScalaIntegrationTestImplicitSerdesWithAvrointegration test inlightbend/kafka-streams-scala. This test doesn't exist inapache/kafkabecause we didn't want to add the Avro dep.https://github.com/lightbend/kafka-streams-scala/blob/v0.2.1/src/test/scala/com/lightbend/kafka/scala/streams/StreamToTableJoinScalaIntegrationTestImplicitSerdesWithAvro.scala#L61..L142
Committer Checklist (excluded from commit message)