Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-6670: Implement a Scala wrapper library for Kafka Streams #4756

Closed
wants to merge 23 commits into from

Conversation

@debasishg
Copy link
Contributor

commented Mar 22, 2018

This PR implements a Scala wrapper library for Kafka Streams. The library is implemented as a project under streams, namely :streams:streams-scala. The PR contains the following:

  • the library implementation of the wrapper abstractions
  • the test suite
  • the changes in build.gradle to build the library jar

The library has been tested running the tests as follows:

$ ./gradlew -Dtest.single=StreamToTableJoinScalaIntegrationTestImplicitSerdes streams:streams-scala:test
$ ./gradlew -Dtest.single=StreamToTableJoinScalaIntegrationTestImplicitSerdesWithAvro streams:streams-scala:test
$ ./gradlew -Dtest.single=WordCountTest streams:streams-scala:test

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

@mjsax mjsax requested review from ijuma, dguy and guozhangwang Mar 22, 2018

@mjsax mjsax added the streams label Mar 22, 2018

@mjsax mjsax self-requested a review Mar 22, 2018

@mjsax

This comment has been minimized.

Copy link
Member

commented Mar 22, 2018

@guozhangwang

This comment has been minimized.

Copy link
Contributor

commented Mar 23, 2018

retest this please

testCompile project(':core')
testCompile libs.avro4s
testCompile libs.scalaLogging
testCompile libs.logback

This comment has been minimized.

Copy link
@ijuma

ijuma Mar 23, 2018

Contributor

We don't use logback for anything else. I'd suggest keeping it consistent with the project.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 23, 2018

Author Contributor

done ..

mavenArtifact: "3.5.2"
mavenArtifact: "3.5.2",
avro4sVersion: "1.8.3",
scalaLoggingVersion: "3.8.0",

This comment has been minimized.

Copy link
@ijuma

ijuma Mar 23, 2018

Contributor

scalaLogging is already defined in this file.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 23, 2018

Author Contributor

removed.

mavenArtifact: "org.apache.maven:maven-artifact:$versions.mavenArtifact"
mavenArtifact: "org.apache.maven:maven-artifact:$versions.mavenArtifact",
avro4s: "com.sksamuel.avro4s:avro4s-core_$versions.baseScala:$versions.avro4sVersion",
scalaLogging: "com.typesafe.scala-logging:scala-logging_$versions.baseScala:$versions.scalaLoggingVersion",

This comment has been minimized.

Copy link
@ijuma

ijuma Mar 23, 2018

Contributor

This is already defined in this file.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 23, 2018

Author Contributor

removed.

@guozhangwang
Copy link
Contributor

left a comment

Thanks for the PR. Could you rebase?

Also a meta question about the avro-dependent test StreamToTableJoinScalaIntegrationTestImplicitSerdesWithAvro.scala, do people think it's worth adding Avro dependency for demonstrating this test?


testCompile libs.junit
testCompile libs.scalatest
testCompile libs.scalatestEmbeddedKafka

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 23, 2018

Contributor

Do we need to import these two dependencies? Could we use Kafka's own EmbeddedKafkaCluster?

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 24, 2018

Author Contributor

This library scalatestEmbeddedKafka has a nice integration with scalatest. Hence it makes writing tests easier and we don't have to bother starting / managing the embedded kafka instance. The test code becomes very concise.

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 25, 2018

Contributor

Generally speaking AK repo tend to avoid dependencies unless it is necessary. I'm wondering if we can improve on Kafka's own EmbeddedKafkaCluster to have the same functionalities as the net.manub:scalatest-embedded-kafka-streams.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 25, 2018

Author Contributor

We can definitely use Kafka's own EmbeddedKafkaCluster to integrate with ScalaTest. In net.manub:scalatest-embedded-kafka-streams, the main value add is integration with ScalaTest and hence you don't have to explicitly start / stop server as part of the test.

Also with Kafka Streams it has very nice constructs like the one we use here .. https://github.com/lightbend/kafka/blob/scala-streams/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/StreamToTableJoinScalaIntegrationTestImplicitSerdes.scala#L77-L99 .. Note you just have to define the transformations and do the publish and consume as part of a closure. No need to start / stop the topology. Hence the test code becomes very concise.

Of course it depends on the opinion of the committee but I think this would be a great addition to the dependency.

Here's a suggestion ..

We use net.manub:scalatest-embedded-kafka-streams for now. After all it's a test dependency. And work on a separate PR to make the integration between EmbeddedKafkaCluster and Scalatest better and in line with the functionalities offered by the library.

WDYAT ?

This comment has been minimized.

Copy link
@seglo

seglo Mar 26, 2018

Contributor

I'm a fan of net.manub:scalatest-embedded-kafka-streams, but I understand the concern about bringing in more deps. I only mentioned it on the dev-kafka list because I didn't think there was much value in improving the embedded Kafka implementation in kafka-stream-scala and making it a public interface because scalatest-embedded-kafka already existed. I wasn't aware of EmbeddedKafkaCluster.

If scalatest-embedded-kafka were brought into the project then there will be drift between the version of Kafka broker in code and whatever this test lib references.

@debasishg I like your suggestion:

work on a separate PR to make the integration between EmbeddedKafkaCluster and Scalatest better and in line with the functionalities offered by the library.

Perhaps we can do this now for this PR, but keep it simple. I could work on it if you're busy.

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 26, 2018

Contributor

I have used net.manub:scalatest-embedded-kafka-streams before, and it is very nice.

But I also worry about adding dependencies to the core project, even test deps. If our tests become a bit uglier, or if we have some test-util class that duplicates some of the functionality you're using, I would consider that to be a worthy tradeoff for dropping the dependency.

I would absolutely support planning to come back in a follow-up PR to build out support for testing scala code and then terraforming these tests to use the new support. Or even delaying this PR until a test-support one is available.

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 26, 2018

Contributor

@seglo @vvcephei @debasishg thanks for your thoughts. We do have plans to publish testing-util artifacts inside AK in the future. And in fact we have been doing so for kafka-streams module as a first step and going to do that for kafka-core and kafka-clients soon. In kafka-clients testing utils we are going to include some improved version of embeddedkafkacluster for users to easily write their integration tests that involve interacting with a mock a kafka cluster.

So I'd suggest we stay with the uglier implementation with the existing embedded kafka cluster and not bring in the dependency.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 27, 2018

Author Contributor

Thanks for your thoughts @guozhangwang .. we will remove the dependency on net.manub:scalatest-embedded-kafka-streams and use EmbeddedKafkaCluster instead.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 28, 2018

Author Contributor

@guozhangwang - Removed all dependencies on net.manub:scalatest-embedded-kafka and net.manub:scalatest-embedded-kafka-streams. Now using EmbeddedKafkaCluster instead for tests. Also removed the test that used avro - hence dependency on avros eliminated.

This comment has been minimized.

Copy link
@guozhangwang
compile libs.scalaLibrary

testCompile project(':core')
testCompile libs.avro4s

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 23, 2018

Contributor

Does it worth to include this dependency at test runtime? cc @ewencp @ijuma .

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 24, 2018

Author Contributor

It's only to demonstrate custom Serdes. We picked up Avro since (AFAIR) @guozhangwang suggested this example in one of the earlier PR discussions. This example goes to show that custom serdes can be handled as seamlessly as primitive ones.

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 26, 2018

Contributor

Cool. In that case, maybe we should also add 'streams:streams-scala:examples' and put it there?

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 26, 2018

Contributor

I'm a bit on the fence for introducing Avro as "the one" serde in our demonstration examples rather than keeping Kafka and Avro separate, since there are many protobufs / etc fans in the community.

How about adding avro examples in eco-system repos, e.g. in Lightbend / Confluent / etc's own examples repo They can add their own example? cc @ijuma @gwenshap

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 26, 2018

Member

+1 Sounds reasonable to me.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 27, 2018

Author Contributor

@mjsax - Cool .. then we can remove the test StreamToTableJoinScalaIntegrationTestImplicitSerdesWithAvro and the dependencies from build.gradle .. ok ?

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 27, 2018

Contributor

Sounds good.

implicit val stringSerde: Serde[String] = Serdes.String()
implicit val longSerde: Serde[Long] = Serdes.Long().asInstanceOf[Serde[Long]]
implicit val byteArraySerde: Serde[Array[Byte]] = Serdes.ByteArray()
implicit val bytesSerde: Serde[org.apache.kafka.common.utils.Bytes] = Serdes.Bytes()

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 23, 2018

Contributor

Could we add the default for Short and ByteBuffer as well?

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 25, 2018

Author Contributor

+1

/**
* Implicit values for default serdes
*/
object DefaultSerdes {

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 23, 2018

Contributor

I'm wondering if we could provide default serdes for windowed key as well? See o.a.k.streams.kstream.WindowedSerdes for Java code.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 26, 2018

Author Contributor

+1 ..

def asValueMapper: ValueMapper[V, VR] = v => f(v)
}

implicit class ValueMapperFromFunctionX[V, VR](val f: V => Iterable[VR]) extends AnyVal {

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 23, 2018

Contributor

nit: how about rename to FlatValueMapperFromFunction for better understanding? Ditto below.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 25, 2018

Author Contributor

+1


// applies the predicate to know what messages should go to the left stream (predicate == true)
// or to the right stream (predicate == false)
def split(predicate: (K, V) => Boolean): (KStream[K, V], KStream[K, V]) = {

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 23, 2018

Contributor

Is it a syntax sugar as branch? I'd prefer to keep Java and Scala interfaces consistent, so that if we think it is worthwhile we'd better add it in Java APIs as well, otherwise we should remove it from Scala APIs.

WDYT? @mjsax @bbejeck @vvcephei

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 26, 2018

Member

I agree that both APIs should be consistent. Does a split add much value compare to branch? Btw, this might be related to https://issues.apache.org/jira/browse/KAFKA-5488

I am also open to add a split() if we think it's useful.

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 26, 2018

Contributor

I agree.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 27, 2018

Author Contributor

For the time being, we can remove split from the Scala API and rethink if / when it's implemented as a Java API

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 27, 2018

Contributor

Sounds good.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 28, 2018

Author Contributor

ok, will remove split from KStream for now.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 28, 2018

Author Contributor

Removed!

c.mapValues[Long](Long2long(_))
}

def count(store: String, keySerde: Option[Serde[K]] = None): KTable[Windowed[K], Long] = {

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 23, 2018

Contributor

This is a deprecated API in java, we should replace it with Materialized.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 24, 2018

Author Contributor

We can add the following:

def count(store: String,
    materialized: Materialized[K, Long, WindowStore[Bytes, Array[Byte]]]): KTable[Windowed[K], Long] = { //..

But this one also may be useful when the user just needs to pass in the keySerde. She need not construct any Materialized which is abstracted within the implementation of the API.

Suggestions ?

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 25, 2018

Contributor

I'd vote for keeping java / scala API consistent, and we are going to remove deprecated APIs in future releases anyway.

In current API we'd only have one additional overload:

def count(materialized: Materialized[K, Long, WindowStore[Bytes, Array[Byte]]]): KTable[Windowed[K], Long] = { //..

I think for users who do not want to specify the store name at all, they can rely on

static Materialized<K, V, S> with(final Serde<K> keySerde, final Serde<V> valueSerde)

to still hide the materialized parameter with implicit conversion.

For users who do want to specify the store name, but want to rely on type conversion, we could call withKeySerde and withValueSerde internally in the implicit conversion so that user only need to give Materialized.as(storeName)

Does that work?

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 25, 2018

Author Contributor

+1 .. will remove this overload for count.

## * limitations under the License.
## */
# Set root logger level to DEBUG and its only appender to A1.
log4j.rootLogger=ERROR, R

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 23, 2018

Contributor

nit: default to INFO?

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 25, 2018

Author Contributor

+1

log4j.appender.A1=org.apache.log4j.ConsoleAppender

log4j.appender.R=org.apache.log4j.RollingFileAppender
log4j.appender.R.File=logs/kafka-server.log

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 23, 2018

Contributor

The file logs/kafka-server.log seems not appropriate as it is not for kafka broker logs right?

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 25, 2018

Author Contributor

Will change the name to kafka-streams-scala.log

* limitations under the License.
*/
!-->
<configuration>

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 23, 2018

Contributor

This is a meta comment: I'd suggest we consider adding logback generally for Kafka, instead of sneaking in for Streams Scala wrapper. We can still use log4j for now. See https://issues.apache.org/jira/browse/KAFKA-2717. cc @ijuma

This comment has been minimized.

Copy link
@ijuma

ijuma Mar 24, 2018

Contributor

Yes, let's stick to log4j in this PR.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 24, 2018

Author Contributor

should we remove logback.xml ?

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 25, 2018

Contributor

Yup, please remove that file as well.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 25, 2018

Author Contributor

+1

@debasishg

This comment has been minimized.

Copy link
Contributor Author

commented Mar 25, 2018

Added commit with changes for code review feedback. Waiting for more feedback and some of the still unresolved questions. Should I rebase now or wait till all questions resolved ?

@mjsax
Copy link
Member

left a comment

I am not a Scala person -- so take my review with a grain of salt...

Did not look at the tests yet.

mavenArtifact: "3.5.2"
mavenArtifact: "3.5.2",
avro4sVersion: "1.8.3",
scalatestEmbeddedKafkaVersion: "1.1.0"

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 26, 2018

Member

nit: those are actually sorted alphabetically -- can we clean this up? Thx.

mavenArtifact: "org.apache.maven:maven-artifact:$versions.mavenArtifact",
avro4s: "com.sksamuel.avro4s:avro4s-core_$versions.baseScala:$versions.avro4sVersion",
scalatestEmbeddedKafka: "net.manub:scalatest-embedded-kafka_$versions.baseScala:$versions.scalatestEmbeddedKafkaVersion",
scalatestEmbeddedKafkaStreams: "net.manub:scalatest-embedded-kafka-streams_$versions.baseScala:$versions.scalatestEmbeddedKafkaVersion"

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 26, 2018

Member

as above.

<Match>
<Package name="org.apache.kafka.streams.scala"/>
<Source name="ScalaSerde.scala"/>
<Bug pattern="NM_SAME_SIMPLE_NAME_AS_INTERFACE"/>

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 26, 2018

Member

Should we add those exclusions? I know that we put exclusions when introducing findbugs because it is not possible to introduce it and rewrite all the code -- but for new code, we should consider changing the code. I am not a Scale person though -- can you elaborate on this?

This comment has been minimized.

Copy link
@seglo

seglo Mar 26, 2018

Contributor

This seems to be a false positive. FindBugs is reporting that Serializer and Deserializer should be defined as a different type name than what it's inheriting. IIRC the consensus earlier is that we want type names the same as the base types they're wrapping (which includes traits and interfaces IMO). I've updated the FindBugs rule exclusion to be specific to the types generating the violation, rather than the entire ScalaSerde file.

import org.apache.kafka.common.utils.Bytes

package object scala {
type ByteArrayKVStore = KeyValueStore[Bytes, Array[Byte]]

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 26, 2018

Member

Should this be ByteArrayKeyValueStore? -- we don't use abbreviations in the Java code base.

*/
object DefaultSerdes {
implicit val stringSerde: Serde[String] = Serdes.String()
implicit val longSerde: Serde[Long] = Serdes.Long().asInstanceOf[Serde[Long]]

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 26, 2018

Member

Why do we need an asInstanceOf here? (same below)

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 26, 2018

Contributor

I think it's because we're presenting the Serde[java.lang.Long] as a Serde[scala.lang.Long], but casting the Serde won't automatically cast the parameters and returns of its methods. I'm surprised you don't get cast class exceptions trying to use the Java long serde as a Scala long serde. Unless I'm wrong about what this is for...

This comment has been minimized.

Copy link
@ijuma

ijuma Mar 26, 2018

Contributor

There is no such thing as a scala.Long at runtime, Scala changed to use the same classes as Java for boxing around the 2.8 timeframe if I remember correctly. Previously there was a RichLong, RichInt, etc.

In any case, this seems like a variance issue, but I didn't look into it.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 27, 2018

Author Contributor

In Scala there's an implicit conversion between scala.Long and java.lang.Long but not between Serde[scala.Long] and Serde[java.lang.Long] - hence the cast. In fact picked this trick from https://github.com/confluentinc/kafka-streams-examples/blob/4.0.0-post/src/test/scala/io/confluent/examples/streams/StreamToTableJoinScalaIntegrationTest.scala#L106

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 27, 2018

Contributor

Huh! Well, that explains it. I have seen the conversions of the raw types, and also suffered from type errors that Serde[scala.Long] != Serde[java.lang.Long], so I just assumed that scala.Long was a different class than java.Long. Thanks for the explanation, @ijuma .


override def init(context: ProcessorContext): Unit = transformerS.init(context)

@deprecated ("Please use Punctuator functional interface at https://kafka.apache.org/10/javadoc/org/apache/kafka/streams/processor/Punctuator.html instead", "0.1.3") // scalastyle:ignore

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 26, 2018

Member

Nit: I am not a fan of adding links in JavaDocs, because links might break; better reference to the corresponding class as JavaDoc cross reference?

Also: I am wondering if we should remove this method from the wrapper in the first place? IMHO, it's not a good idea to add deprecated API in new code?

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 26, 2018

Contributor

+1 on not including this method in the wrapper. The code that would use this library is not written yet, so it's better if deprecated methods are simply not available.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 27, 2018

Author Contributor

Here, the deprecation is on punctuate, which is part of the contract of Transformer. How do we remove this ? We can only remove this when punctuate is removed from Transformer ? Or am I missing something ?

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 27, 2018

Contributor

Oh, right, I thought this was one of your scala replacement classes.

What I have been doing for cases like this is throwing an UnsupportedOperationException in the body. It's not as good as not having the method, but it def. ensures it can't be used. And you don't have to maintain the code that's in the body.

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 27, 2018

Member

I understand that you cannot change the Java Transformer interface and must implement the deprecated method when calling new Transformer -- what I was wondering is about scala.Transformer interface -- should we add one and remove punctuate from it?

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 27, 2018

Contributor

FWIW, I think adding a new scala interface just to remove a method that we plan to remove from the java interface is not necessary. Better just to implement it and move on.

Also, it would be a bit trickier to swap in a scala replacement for Transformer than for the top-level DSL classes, since implementations of the java Transformer won't implement the scala Transfomer, so you wouldn't be able to plug them in via the scala DSL wrapper. But there's otherwise no reason this shouldn't work.

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 27, 2018

Member

Ack. Was just an idea. I don't really speak Scala (yet) -- this is an exercise to learn something about it...

If we need to have it, I vote to throw an exception to forces users to use the new API.

def transformValues[VR](valueTransformerSupplier: () => ValueTransformer[V, VR],
stateStoreNames: String*): KStream[K, VR] = {

val valueTransformerSupplierJ: ValueTransformerSupplier[V, VR] = () => valueTransformerSupplier()

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 26, 2018

Member

A ValueTransformer also have init(), punctuate() and close() method. Why is this code much simpler than the wrapper for transform() above?

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 27, 2018

Author Contributor

We don't need to provide implementation of ValueTransformer here since the passed in () => ValueTransformer[V, VR] gets converted to ValueTransformerSupplier[V, VR] in the implementation through SAM type conversion. We could not do that for transform as the SAM type conversion will not be handled automatically in that case. Please have a look here for SAM type conversions in Scala.

def process(processorSupplier: () => Processor[K, V],
stateStoreNames: String*): Unit = {

val processorSupplierJ: ProcessorSupplier[K, V] = () => processorSupplier()

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 26, 2018

Member

As above. What about init(), punctuate(), and close() ?

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 27, 2018

Author Contributor

Same logic as ValueTransformer above.


// applies the predicate to know what messages should go to the left stream (predicate == true)
// or to the right stream (predicate == false)
def split(predicate: (K, V) => Boolean): (KStream[K, V], KStream[K, V]) = {

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 26, 2018

Member

I agree that both APIs should be consistent. Does a split add much value compare to branch? Btw, this might be related to https://issues.apache.org/jira/browse/KAFKA-5488

I am also open to add a split() if we think it's useful.

c.mapValues[Long](Long2long _)
}

def count(store: String, keySerde: Option[Serde[K]] = None): KTable[K, Long] = {

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 26, 2018

Member

Why do we only allow to specify a keySerde but not replace the store with a different one?

Scala noob question: would it be possible to have a single count / reduce etc instead of overloads and use Option and implicits to infer optional arguments?

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 27, 2018

Author Contributor

@mjsax -

I think I may be missing something here. We are allowing the user to pass in the store and keySerde. And we create the Materialized out of the 2 with Long as the value serde.

However we were allowing the user to pass in the keySerde optionally and in case the user does not supply one we assumed it will be taken from the config. This is actually a remnant from the earlier thoughts where we thought passing serdes through config may be a good idea. However in the current context, we should not make keySerde optional. Here's the suggestion for the changed API ..

def count(store: String, keySerde: Serde[K]): KTable[K, Long] = { 
  val materialized = Materialized.as[K, java.lang.Long, ByteArrayKeyValueStore](store).withKeySerde(keySerde)
  val c: KTable[K, java.lang.Long] = inner.count(materialized)
  c.mapValues[Long](Long2long _)
}

An alternative option could have been to allow the user to pass in the Materialized instance itself (like we do in the reduce function). The problem with that alternative is that the Java API expects java.lang.Long as the value serde, while the Scala API needs to take a scala.Long. And there is no way we can convert a Materialized.as[K, scala.Long, ByteArrayKeyValueStore] to Materialized.as[K, java.lang.Long, ByteArrayKeyValueStore] without a cast. Hence the divergence in the API signature between count and reduce.

Please share if u have any other thoughts.

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 27, 2018

Contributor

That makes sense.

My 2 cents:
We're presenting the scala KGroupedStream basically as the java one, but not implementing the interface so that we can smooth over a couple of specific gaps. I think this is a good call, but it's also a huge risk for cognitive dissonance and developer confusion, since they will read the java version of the docs and try to use that knowledge in scala. Therefore, it's important to be super disciplined about making sure the methods available are as close to the java interface as possible.

Clearly, moving serdes, etc., to implicit params is the kind of thing we do want to do. But I think that presenting count(String,Serde[K]) instead of count(Materialized[K, Long, KeyValueStore[Bytes, Array[Byte]]] is too far off.

I do agree that the method should take a scala type. Apparently, it's perfectly fine to cast Serde[scala.Long] to Serde[java.Long]. Does that same logic apply here? Alternatively, we can actually convert the Materialized[java] to a Materialized[scala].

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 27, 2018

Author Contributor

@vvcephei .. Sure we can do the following instead ..

def count(materialized: Materialized[K, Long, ByteArrayKeyValueStore]): KTable[K, Long] = { 
  val c: KTable[K, java.lang.Long] = inner.count(materialized.asInstanceOf[Materialized[K, java.lang.Long, ByteArrayKeyValueStore]])
  c.mapValues[Long](Long2long _)
}

WDYT ?

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 27, 2018

Contributor

Ok, I'm already a little nervous about the cast in one direction, so this feels super gross, but would this work?

def count(materialized: Materialized[K, Long, ByteArrayKeyValueStore]): KTable[K, Long] = { 
  inner.count(materialized.asInstanceOf[Materialized[K, java.lang.Long, ByteArrayKeyValueStore]]).asInstanceOf[KTable[K, scala.Long]]
}

Please understand I'm wincing as I type this.

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 27, 2018

Member

Well, we allowing to pass in a store "name" (String) but not a store. Note, that Materialized allows to replace default RocksDB with an in-memory story, disable change-capture-logging or even use a custom store implementation.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 27, 2018

Author Contributor

How about the API that I suggested above ? It takes Materialized much like the Java API though we need a cast.

@vvcephei - in my implementation we have 1 cast and the other map for the long conversion in KTable.

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 27, 2018

Contributor

@debasishg Sorry, I should have acked your implementation. I was actually proposing an evolution of it.

It just seems a bit unfortunate to have to add a real function invocation to the topology in order to do the cast back to scala.Long. The version I proposed just does a cast back out without adding anything new to the topology. Does that make sense?

At the risk of sounding like an idiot, if it's fine to do the cast on the way in, then it should be fine again on the way out, right?

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 28, 2018

Author Contributor

@vvcephei - cast is a runtime operation and my philosophy is to minimize its use. And scala.Predef indeed uses Long2long to do such conversions. Hence I would like to prefer using proper functions when available instead of the cast.

@vvcephei
Copy link
Contributor

left a comment

hey @debasishg , thanks so much for this PR. I've been super frustrated in the past with how awkward it is to use KStreams in Scala vs. how awkward it should be.

My favorite features here are:

  • serdes for Scala types
  • DSL methods taking implicits for things that are easy to infer (like the serde implementation).

My personal preference would be to avoid renaming imports and just using fully-qualified package names inline. It makes our code longer, but easier to understand. For a "base-level" library, I'd prefer to have less pretty but more comprehensible code in general.

I'm also wondering if we really want to use the same class names as the Java DSL, this seems like it would be confusing to use in an IDE with code-completion. What do you think about using ScalaKStream, etc. instead? This would also get around the desire to rename imports.

Thanks again,
-John

// use experimental flag to enable SAM type conversions in compiler
if (versions.baseScala == '2.11') {
tasks.withType(ScalaCompile) {
scalaCompileOptions.additionalParameters += ["-Xexperimental"]

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 26, 2018

Contributor

I'd like to confirm that this option is actually safe. Is this a best practice at this point for targeting 2.11? Also, how can we know we're not dragging in other (potentially unwanted) experimental compiler features with this?

This comment has been minimized.

Copy link
@seglo

seglo Mar 26, 2018

Contributor

I agree it looks concerning, I'll need to check what other potential features this brings in, unfortunately there's no way to be more specific about just enabling SAM type conversion AFAIK.

We could remove this flag, but we would need to desugar all the places where conversions occur.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 27, 2018

Author Contributor

I'm also wondering if we really want to use the same class names as the Java DSL, this seems like it would be confusing to use in an IDE with code-completion. What do you think about using ScalaKStream, etc. instead? This would also get around the desire to rename imports.

Hi @vvcephei - regarding the above, we had a discussion on the 2 approaches on this thread only and the universal suggestion was to use the same name across Scala and Java APIs. In fact in the initial version that we posted, we had different names (KStream / KStreamS). The reasoning of using the same names is that the renaming of imports in the user code needs to be done only very occasionally when we mix usage of Scala and Java APIs.

This comment has been minimized.

Copy link
@seglo

seglo Mar 27, 2018

Contributor

@vvcephei There's no list of what's brought in with the experimental flag, unless you grep the compiler code. The purpose of the SAM type conversion feature in 2.11 was only to get early feedback and only properly finished in 2.12. As its name suggests it's not meant to be used in production code. Since Kafka is still targeting 2.11 it makes sense to not include this flag to build a releasable artifact. I'll remove the flag and desugar the conversions.

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 27, 2018

Contributor

@debasishg Thanks for the explanation, I missed that discussion. Sorry to bring it back up!

@seglo Thanks! Am I correct in thinking that only affects our code, and not our users' code? I.e., if they are using 2.12, they can pass in lambdas as arguments, right?

This comment has been minimized.

Copy link
@seglo

seglo Mar 27, 2018

Contributor

@vvcephei Yes, this change won't affect end users at all. If Kafka drops Scala 2.11 support then we can bring back the SAM conversions as they're available without any ominous compiler flags.

compile libs.scalaLibrary

testCompile project(':core')
testCompile libs.avro4s

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 26, 2018

Contributor

Cool. In that case, maybe we should also add 'streams:streams-scala:examples' and put it there?

project(':streams:streams-scala') {
println "Building project 'streams-scala' with Scala version ${versions.scala}"
apply plugin: 'scala'
archivesBaseName = "kafka-streams-streams-scala"

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 26, 2018

Contributor

nit: this is a bit cumbersome. Can we do project(':streams:scala-wrapper') and archive: 'kafka-streams-scala-wrapper' or some such instead?


testCompile libs.junit
testCompile libs.scalatest
testCompile libs.scalatestEmbeddedKafka

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 26, 2018

Contributor

I have used net.manub:scalatest-embedded-kafka-streams before, and it is very nice.

But I also worry about adding dependencies to the core project, even test deps. If our tests become a bit uglier, or if we have some test-util class that duplicates some of the functionality you're using, I would consider that to be a worthy tradeoff for dropping the dependency.

I would absolutely support planning to come back in a follow-up PR to build out support for testing scala code and then terraforming these tests to use the new support. Or even delaying this PR until a test-support one is available.

*/
object DefaultSerdes {
implicit val stringSerde: Serde[String] = Serdes.String()
implicit val longSerde: Serde[Long] = Serdes.Long().asInstanceOf[Serde[Long]]

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 26, 2018

Contributor

I think it's because we're presenting the Serde[java.lang.Long] as a Serde[scala.lang.Long], but casting the Serde won't automatically cast the parameters and returns of its methods. I'm surprised you don't get cast class exceptions trying to use the Java long serde as a Scala long serde. Unless I'm wrong about what this is for...


override def init(context: ProcessorContext): Unit = transformerS.init(context)

@deprecated ("Please use Punctuator functional interface at https://kafka.apache.org/10/javadoc/org/apache/kafka/streams/processor/Punctuator.html instead", "0.1.3") // scalastyle:ignore

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 26, 2018

Contributor

+1 on not including this method in the wrapper. The code that would use this library is not written yet, so it's better if deprecated methods are simply not available.


// applies the predicate to know what messages should go to the left stream (predicate == true)
// or to the right stream (predicate == false)
def split(predicate: (K, V) => Boolean): (KStream[K, V], KStream[K, V]) = {

This comment has been minimized.

Copy link
@vvcephei

vvcephei Mar 26, 2018

Contributor

I agree.

/**
* Wraps the Java class StreamsBuilder and delegates method calls to the underlying Java object.
*/
class StreamsBuilder(inner: StreamsBuilderJ = new StreamsBuilderJ) {

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 27, 2018

Member

This and all other classes are public API. Thus, we should improve the JavaDocs for those classes and also add JavaDocs for all methods. I guess we can c&p from existing Java classes.

This comment has been minimized.

Copy link
@debasishg

debasishg Mar 28, 2018

Author Contributor

@mjsax - Looking for suggestions. Should we copy/paste Javadoc from Java classes or use @see annotation ? The problem with copy is maintenance - when one changes someone needs to be careful enough to change the other.

This comment has been minimized.

Copy link
@mjsax

mjsax Mar 29, 2018

Member

Good question. Not sure. I agree that maintaining JavaDocs twice is a hassle and error prone. But might be annoying for user if it's only linked on the other hand. Would be good to hear what others thing. \cc @guozhangwang @deanwampler @ijuma @vvcephei @bbejeck @seglo

This comment has been minimized.

Copy link
@seglo

seglo Mar 29, 2018

Contributor

Adding JavaDocs to all the public API methods in the PR is the same amount of work any way we do it. From an end user perspective I agree it would be nice to have the same (or slightly tweaked, as necessary) JavaDocs for all public API methods, plus a @see or @link tag to the corresponding Java API method. It will be a small burden to maintain it going forward so I defer to the AK committers to make the call on the format.

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Mar 29, 2018

Contributor

I'd vote for using @see and @link to avoid maintaining two copies, because we have some public classes following this pattern in the repo (like https://github.com/apache/kafka/blob/23d01c805bef7504abfa83ecac7e384d121a583a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java), and from the past I find most people would not remember or bother to update two places than one.

Additionally, this PR should include updates to the web docs in docs/streams/... and in "notable changes" in docs/upgrade.html ?

Meta Comment - Besides updating documentation and Javadoc, is there any outstanding item in this PR that needs to be addressed ?

I think the web docs (in docs/streams) needs to be updated as well, especially in the upgrade-guide.html page, as well as the streams-api page.

This comment has been minimized.

This comment has been minimized.

Copy link
@ijuma

ijuma Mar 29, 2018

Contributor

Some thoughts:

  • I think the Consumer and KafkaConsumer pattern is bad. The documentation should have been on Consumer instead. The AdminClient follows the latter pattern.
  • I think it's a poor user experience to ask users to read the docs in the Java class. My recommendation would be to at least include a short description in the Scala docs along with a link to the relevant Java documentation. The short description is less likely to change and it helps users make progress without having to jump to the Java code all the time. However, for more detailed information (which is more likely to change), they can check the Java code.

This comment has been minimized.

Copy link
@mjsax

mjsax Apr 1, 2018

Member

I tend to agree with @ijuma comment about KafkaConsumer/Consumer pattern -- it's quite annoying to not get the JavaDocs directly. Thus, even if it's a burden it seems to be worth to maintain two copies.

This comment has been minimized.

Copy link
@debasishg

debasishg Apr 1, 2018

Author Contributor

@mjsax - I have started writing the Scaladocs in the commit 631ab9a .. Pls review if it's following the correct pattern

@mjsax

This comment has been minimized.

Copy link
Member

commented Mar 27, 2018

Meta comment: For JavaDocs, so we need to set up some pipeline to get JavaDocs published? Or will this happen automatically?

Additionally, this PR should include updates to the web docs in docs/streams/... and in "notable changes" in docs/upgrade.html ?

@debasishg

This comment has been minimized.

Copy link
Contributor Author

commented Mar 29, 2018

Meta Comment - Besides updating documentation and Javadoc, is there any outstanding item in this PR that needs to be addressed ?

@seglo

This comment has been minimized.

Copy link
Contributor

commented Mar 31, 2018

For user documentation my plan is to update the following:

  • Streams homepage - Update the Scala example to use Scala DSL
  • Developer Guide
    • Writing a Streams Application - Add a reference to the streams-scala artifact and add Scala examples
    • Configuring a Streams Application - Note that default key/value serdes not required for Scala DSL.
    • Streams DSL - Add a new section for the Scala DSL. Packages, examples, implicit serdes and other differences from Java DSL. The bulk of the content would go here.
    • Data Types & Serialization - Reference implicit serdes from Streams DSL page which includes implicit serdes explanation and usage, or vice versa.

I could use some advice on how to actually go about updating the docs efficiently. The Contributing Website Documentation Changes doesn't go into detail about a workflow I can use while developing docs (make a change, preview, etc). I noticed the code samples for the streams docs are formatted as HTML, but I couldn't find the source for these examples in the project, are they generated somehow?

EDIT: I also added to the https://kafka.apache.org/documentation/#streamsapi section, to explain how to include the dependency in maven build file, e.g. to use the scala wrapper.

EDIT 2: I also updated the section 2.3 (streams api) to reference the new client library.

EDIT 3: I also updated the streams upgrade guide.

@mjsax

This comment has been minimized.

Copy link
Member

commented Apr 1, 2018

@seglo Sounds great.

You are right that it is HTML -- just edit it directly. I have no concrete workflow suggestion. What code examples do you mean? It should all be in the HTML files. There is nothing in streams docs thats generated.

@seglo

This comment has been minimized.

Copy link
Contributor

commented Apr 1, 2018

@mjsax I'm referring to the code snippets found in pages like the DSL API. They look like they've been generated with syntax highlightling.

                        <div class="highlight-java"><div class="highlight"><pre><span></span><span class="kn">import</span> <span class="nn">org.apache.kafka.common.serialization.Serdes</span><span class="o">;</span>
<span class="kn">import</span> <span class="nn">org.apache.kafka.streams.StreamsBuilder</span><span class="o">;</span>
<span class="kn">import</span> <span class="nn">org.apache.kafka.streams.kstream.GlobalKTable</span><span class="o">;</span>

<span class="n">StreamsBuilder</span> <span class="n">builder</span> <span class="o">=</span> <span class="k">new</span> <span class="n">StreamsBuilder</span><span class="o">();</span>

<span class="n">GlobalKTable</span><span class="o">&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">&gt;</span> <span class="n">wordCounts</span> <span class="o">=</span> <span class="n">builder</span><span class="o">.</span><span class="na">globalTable</span><span class="o">(</span>
    <span class="s">&quot;word-counts-input-topic&quot;</span><span class="o">,</span>
    <span class="n">Materialized</span><span class="o">.&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">,</span> <span class="n">KeyValueStore</span><span class="o">&lt;</span><span class="n">Bytes</span><span class="o">,</span> <span class="kt">byte</span><span class="o">[]&gt;&gt;</span><span class="n">as</span><span class="o">(</span>
      <span class="s">&quot;word-counts-global-store&quot;</span> <span class="cm">/* table/store name */</span><span class="o">)</span>
      <span class="o">.</span><span class="na">withKeySerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">String</span><span class="o">())</span> <span class="cm">/* key serde */</span>
      <span class="o">.</span><span class="na">withValueSerde</span><span class="o">(</span><span class="n">Serdes</span><span class="o">.</span><span class="na">Long</span><span class="o">())</span> <span class="cm">/* value serde */</span>
    <span class="o">);</span>
</pre></div>
                        </div>

https://github.com/apache/kafka/blob/trunk/docs/streams/developer-guide/dsl-api.html#L189

@guozhangwang

This comment has been minimized.

Copy link
Contributor

commented Apr 2, 2018

@seglo For the doc changes, as I mentioned before we should also add a new section in the https://kafka.apache.org/documentation/#streamsapi section, to explain how to include the dependency in maven build file, e.g. to use the scala wrapper.

@mjsax

This comment has been minimized.

Copy link
Member

commented Apr 2, 2018

@seglo I see -- I guess, somebody used some special HTML editor... \cc @joel-hamill might be able to shed some light... Most people just use plain text editors. You can just add plain HTML without any syntax highlighting. Cf https://github.com/apache/kafka/blob/trunk/docs/streams/developer-guide/testing.html#L76-L93

(I was just not sure what you mean by "generated" because the configs HTML is generated from the source code directly -- cf. https://kafka.apache.org/documentation/#configuration and https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java#L1095)

@seglo

This comment has been minimized.

Copy link
Contributor

commented Apr 3, 2018

I've implemented the user docs. However I don't know how to view the documentation in my browser. None of the HTML files in docs subdir render when I open them locally with Chrome (even before I made any edits). There are likely some formatting mistakes since they're difficult to spot in the markup alone. Can someone familiar with editing these docs please describe a workflow I can use to view rendered documentation locally?

In the meantime I encourage people to review the content.

/cc @mjsax @guozhangwang @joel-hamill

@guozhangwang

This comment has been minimized.

Copy link
Contributor

commented Apr 3, 2018

@seglo you can read this wiki page for render the pages locally: https://cwiki.apache.org/confluence/display/KAFKA/Contributing+Website+Documentation+Changes

To do that you'd need to first copy-paste the docs to kafka-site repo, and start the apache server locally in order to see the difference.

@guozhangwang
Copy link
Contributor

left a comment

@seglo please update the streams/upgrade-guide.html, under the

<h3><a id="streams_api_changes_120" href="#streams_api_changes_120">Streams API changes in 1.2.0</a></h3>

section, to talk about this addition to the public APIs.

</p>
<p>The library wraps Java Stream DSL APIs in Scala thereby providing:</p>
<ol class="last arabic simple">
<li>Better type inference in Scala</li>

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Apr 3, 2018

Contributor

nit: period at the end of the sentence.

This comment has been minimized.

Copy link
@seglo

seglo Apr 4, 2018

Contributor

👍

<li>Implicit serializers and de-serializers leading to better abstraction and less verbosity</li>
<li>Better type safety during compile time<li>
</ol>
<p>All functionality provided by Kafka Streams DSL for Scala are under the root package name of org.apache.kafka.streams.scala.</p>

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Apr 3, 2018

Contributor

Maybe mention again which artifact to include in order to import this package.

This comment has been minimized.

Copy link
@seglo

seglo Apr 4, 2018

Contributor

👍

<p>Here's an example of the classic Word Count program that uses the Scala builder StreamBuilder and then builds an instance of KStream using the wrapped API builder.stream. Then we reify to a table and get a KTable, which, again is a wrapper around Java KTable.</p>
<p>The net result is that the following code is structured just like using the Java API, but from Scala and with far fewer type annotations compared to using the Java API directly from Scala. The difference in type annotation usage is more obvious when given an example. Below is an example WordCount implementation that will be used to demonstrate the differences between the Scala and Java API.</p>
<pre class="brush: scala;">
import org.apache.kafka.common.serialization.Serdes;

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Apr 3, 2018

Contributor

We do not need indentation in the code block; the following code blocks are formatted correctly.

This comment has been minimized.

Copy link
@seglo

seglo Apr 4, 2018

Contributor

I copied the formatting from the streams main page which indented the WordCount examples: https://github.com/apache/kafka/blob/trunk/docs/streams/index.html#L155

This comment has been minimized.

Copy link
@seglo

seglo Apr 4, 2018

Contributor

I removed the initial indentation for this example on this page to make it consistent with the others.

The computational logic of a Kafka Streams application is defined as a <a class="reference internal" href="../concepts.html#streams-concepts"><span class="std std-ref">processor topology</span></a>,
which is a graph of stream processors (nodes) and streams (edges).</p>
<p>You can define the processor topology with the Kafka Streams APIs:</p>
<dl class="docutils">
<dt><a class="reference internal" href="dsl-api.html#streams-developer-guide-dsl"><span class="std std-ref">Kafka Streams DSL</span></a></dt>
<dd>A high-level API that provides provides the most common data transformation operations such as <code class="docutils literal"><span class="pre">map</span></code>, <code class="docutils literal"><span class="pre">filter</span></code>, <code class="docutils literal"><span class="pre">join</span></code>, and <code class="docutils literal"><span class="pre">aggregations</span></code> out of the box. The DSL is the recommended starting point for developers new to Kafka Streams, and should cover many use cases and stream processing needs.</dd>
<dd>A high-level API that provides provides the most common data transformation operations such as <code class="docutils literal"><span class="pre">map</span></code>, <code class="docutils literal"><span class="pre">filter</span></code>, <code class="docutils literal"><span class="pre">join</span></code>, and <code class="docutils literal"><span class="pre">aggregations</span></code> out of the box. The DSL is the recommended starting point for developers new to Kafka Streams, and should cover many use cases and stream processing needs. If you're writing a Scala application you can use the <a href="dsl-api.html#streams-developer-guide-scala"><span class="std std-ref">Scala Kafka Streams DSL</span> library which removes the Java/Scala integraiton boilerplate when working with the Java DSL.</a></dd>

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Apr 3, 2018

Contributor

This is not introduced in this PR: duplicated provides.

This comment has been minimized.

Copy link
@seglo

seglo Apr 4, 2018

Contributor

I fixed the typos in this line, but I'm not sure what you mean by it not being introduced in this PR. This line is to indicate the presence of the Kafka Streams DSL for Scala library.

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Apr 4, 2018

Contributor

I meant the duplicated provides exist before this PR, so it is not a regression introduced from this PR.

@seglo

This comment has been minimized.

Copy link
Contributor

commented Apr 4, 2018

@guozhangwang I was able to setup a local apache2 webserver (wow, I haven't done that in 10 years!) I fixed formatting issues, typos, added the blurb to the streams upgrade guide, and other misc. feedback you provided.

@guozhangwang

This comment has been minimized.

Copy link
Contributor

commented Apr 4, 2018

@guozhangwang I was able to setup a local apache2 webserver (wow, I haven't done that in 10 years!) I fixed formatting issues, typos, added the blurb to the streams upgrade guide, and other misc. feedback you provided.

Thanks!

Could you also rebase your PR against trunk to resolve the conflicts as well?

@seglo seglo force-pushed the lightbend:scala-streams branch from 88185aa to ee7b8f1 Apr 5, 2018

@seglo

This comment has been minimized.

Copy link
Contributor

commented Apr 5, 2018

@guozhangwang

This comment has been minimized.

Copy link
Contributor

commented Apr 5, 2018

retest this please

@dguy
Copy link
Contributor

left a comment

Thanks for the PR - do we need the integration tests? i.e., the scala api is just translating to the underlying java api, so we could write tests that ensure that the correct topology is built?

@debasishg

This comment has been minimized.

Copy link
Contributor Author

commented Apr 6, 2018

@dguy - One of the reasons we wrote the tests was to demonstrate the idiomatic usage of the Scala APIs. If you think we should write additional tests to verify if the correct topology is built, what kind of tests can do this verification ? Is there any example test in the Java APIs that does this verification of building the correct topology ?

@seglo seglo force-pushed the lightbend:scala-streams branch from 40c7691 to 453d4ab Apr 12, 2018

@mjsax
Copy link
Member

left a comment

Couple of nits.

@@ -78,6 +78,19 @@ <h3><a id="streamsapi" href="#streamsapi">2.3 Streams API</a></h3>
&lt;/dependency&gt;
</pre>

<p>
When using Scala you may optionally include the <code>kafka-streams-scala</code> library. Additional documentation on using the Kafka Streams DSL for Scala is available <a href="/{{version}}/documentation/streams/developer-guide/dsl-api.html#scala-dsl">here</a>.

This comment has been minimized.

Copy link
@mjsax

mjsax Apr 16, 2018

Member

"is available here" is bad phrasing.

here -> in the developer guide

This comment has been minimized.

Copy link
@seglo

seglo Apr 18, 2018

Contributor

👍

<li><code class="docutils literal"><span class="pre">org.apache.kafka.streams.scala.DefaultSerdes</span></code>: Module that brings into scope the implicit values of all primitive SerDes.</li>
<li><code class="docutils literal"><span class="pre">org.apache.kafka.streams.scala.ScalaSerde</span></code>: Base abstraction that can be used to implement custom SerDes in a type safe way.</li>
</ul>
<p>The library is cross-built with Scala 2.11 and 2.12. To include it your maven <code>pom.xml</code> add the following:</p>

This comment has been minimized.

Copy link
@mjsax

mjsax Apr 16, 2018

Member

To include it your maven -- sounds weird

This comment has been minimized.

Copy link
@seglo

seglo Apr 18, 2018

Contributor

👍

&lt;groupId&gt;org.apache.kafka&lt;/groupId&gt;
&lt;artifactId&gt;kafka-streams-scala&lt;/artifactId&gt;
&lt;version&gt;{{fullDotVersion}}&lt;/version&gt;
&lt;/dependency&gt;

This comment has been minimized.

Copy link
@mjsax

mjsax Apr 16, 2018

Member

I am wondering if this is correct? Should the Scala version not be included here?

This comment has been minimized.

Copy link
@guozhangwang

guozhangwang Apr 16, 2018

Contributor

For different scala version compiled packages, their project name is actually the same. And here people only need to specify the version of the artifact itself, which will be the Kafka version.

Users can, indeed, build kafka-streams-scala with different scala versions other than the default one, but that is to be done before they include it in the dependency. For maven, it will always be whatever is uploaded to maven central.

This comment has been minimized.

Copy link
@seglo

seglo Apr 18, 2018

Contributor

I'm not very familiar with gradle, but it appears to not support cross building jars in the same manner as SBT. The build needs to be run for each Scala version you want a jar for, but the output won't encode the version into the filename. I think what we need to do is add a task to the gradle file, or some other build related packaging script, to pluck the generated kafka-streams-scala file, rename it to include the Scala version, and then publish it to maven central.

Ex) The built outputs this when specifying a 2.12 SCALA_VERSION (./gradlew -PscalaVersion=2.12 jar)

-rw-rw-r--  1 seglo seglo 105423 Apr 18 14:17 kafka-streams-scala-1.2.0-SNAPSHOT.jar

When a release artifact is published we'll publish a file: kafka-streams-scala_2.12-1.2.0.jar with the Scala major version encoded into the artifact name. A maven user would reference the artifact with:

<dependency>
    <groupId>org.apache.kafka</groupId>
    <artifactId>kafka-streams-scala_2.12</artifactId>
    <version>1.1.0</version>
</dependency>

Please let me know if I'm missing something here about the Kafka build system.

On a related note I found a gradle build plugin that handles cross building projects and referencing Scala dependencies in a SBT style here: https://github.com/ADTRAN/gradle-scala-multiversion-plugin

This comment has been minimized.

Copy link
@ijuma

ijuma Apr 18, 2018

Contributor

We already so the right thing for core jars. We just need to follow the same approach. And yes, the Scala version needs to be encoded in the artifact id. Not sure what @guozhangwang was trying to say, but doesn't seem correct to me.

This comment has been minimized.