Type names: I vote for option 2.  The user must explicitly add a dependency
to this library and the wrapper types are in a different package.  It seems
reasonable to expect them to do an import rename if there's a need to drop
down to the Java API.

Test Utils: The test utils in kafka-streams-scala are nice and lean, but
I'm not sure if it provides much more value than other options that exist
in the community.  There's an embedded Kafka/ZK project implementation for
ScalaTest that's popular and active: manub/scalatest-embedded-kakfa.  It
implies you must also use ScalaTest, which I acknowledge isn't everyone's
first choice for Scala test framework, but it probably is one of, if not
the most, popular library.  It includes a DSL for Kafka Streams too.  If
this KIP is accepted then perhaps a PR to that project could be made to
support the new wrapper implementations.

https://github.com/manub/scalatest-embedded-kafka#scalatest-embedded-kafka-streams

Sean

On Sun, Mar 18, 2018 at 4:05 AM, Debasish Ghosh <
debasish.gh...@lightbend.com> wrote:

> >
> > > Should this be 1.2  (maybe it's even better to not put any version at
> > all)
>
>
> Actually wanted to emphasize that the support is from 1.0.0 onwards ..
> Should we make that explicit ? Like ..
>
> kafka-streams-scala only depends on the Scala standard library and Kafka
> > Streams 1.0.0+.
>
>
>  In 1.1 release, we add a new module `kafka-streams-test-utils` to simplify
> > testing for Kafka Streams applications. Are those test utils suitable for
> > Scala users or should we add Scala wrappers for those, too?
>
>
> I will check up and let you know ..
>
> Also I am not clear about the decision on renaming of Scala abstractions.
> Can we have a consensus on this ? Here's the summary ..
>
> *Option 1:* Keep names separate (KStream for Java class, KStreamS for
> Scala). No renaming of imports required.
> *Option 2:* Unify names (KStream for Java and Scala class names). No
> conflict since they will reside in different packages. But if we need to
> use both abstractions, renaming of imports are required. But again, this
> may not be a too frequent use case.
>
> Suggestions ?
>
> regards.
>
> On Sat, Mar 17, 2018 at 3:07 AM, Matthias J. Sax <matth...@confluent.io>
> wrote:
>
> > Thanks a lot for the KIP! Two questions:
> >
> > 1) the KIP states:
> >
> > > kafka-streams-scala only depends on the Scala standard library and
> Kafka
> > Streams 1.0.0.
> >
> > Should this be 1.2  (maybe it's even better to not put any version at
> all)
> >
> >
> > 2) In 1.1 release, we add a new module `kafka-streams-test-utils` to
> > simplify testing for Kafka Streams applications. Are those test utils
> > suitable for Scala users or should we add Scala wrappers for those, too?
> >
> >
> > -Matthias
> >
> >
> > On 3/16/18 11:10 AM, Ted Yu wrote:
> > > Import renames seem to be fine.
> > >
> > > The class names with trailing 'S' look clean.
> > >
> > > Cheers
> > >
> > > On Fri, Mar 16, 2018 at 11:04 AM, Ismael Juma <ism...@juma.me.uk>
> wrote:
> > >
> > >> If this is rare (as it sounds), relying on import renames seems fine
> to
> > me.
> > >> Let's see what others think.
> > >>
> > >> Ismael
> > >>
> > >> On Fri, Mar 16, 2018 at 10:51 AM, Debasish Ghosh <
> > >> debasish.gh...@lightbend.com> wrote:
> > >>
> > >>> I am not sure if this is practical or not. But theoretically a user
> may
> > >>> want to extract the unsafe Java abstraction from the Scala ones and
> use
> > >>> Java APIs on them .. e.g.
> > >>>
> > >>> val userClicksStream: KStreamS[String, Long] =
> > >>> builder.stream(userClicksTopic) // Scala abstraction
> > >>>
> > >>> val jStream: KStream[String, Long] = userClicksStream.inner //
> > publishes
> > >>> the underlying Java abstraction
> > >>>
> > >>> //.. work with Java, may be pass to some function written in Java
> > >>>
> > >>> I do realize this is somewhat of a convoluted use case and may not be
> > >>> practically useful ..
> > >>>
> > >>> Otherwise we can very well work on the suggested approach of unifying
> > the
> > >>> names ..
> > >>>
> > >>> regards.
> > >>>
> > >>>
> > >>>
> > >>> On Fri, Mar 16, 2018 at 10:28 PM, Ismael Juma <ism...@juma.me.uk>
> > wrote:
> > >>>
> > >>>> What does "mixed mode application" mean? What are the cases where a
> > >> user
> > >>>> would want to use both APIs? I think that would help understand the
> > >>>> reasoning.
> > >>>>
> > >>>> Thanks,
> > >>>> Ismael
> > >>>>
> > >>>> On Fri, Mar 16, 2018 at 8:48 AM, Debasish Ghosh <
> > >>>> debasish.gh...@lightbend.com> wrote:
> > >>>>
> > >>>>> Hi Damian -
> > >>>>>
> > >>>>> We could. But in case the user wants to use both Scala and Java
> APIs
> > >>> (may
> > >>>>> be for some mixed mode application), won't that be confusing ? She
> > >> will
> > >>>>> have to do something like ..
> > >>>>>
> > >>>>> import o.a.k.s.scala.{KStream => KStreamS}
> > >>>>>
> > >>>>> to rename Scala imports or the other way round for imported Java
> > >>> classes.
> > >>>>>
> > >>>>> regards.
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> On Fri, Mar 16, 2018 at 9:07 PM, Damian Guy <damian....@gmail.com>
> > >>>> wrote:
> > >>>>>
> > >>>>>> Hi Debasish,
> > >>>>>>
> > >>>>>> Thanks for the KIP - will be a great addition to streams. I've
> only
> > >>>> had a
> > >>>>>> quick scan, but seeing as the Scala classes are going to be in
> > >> their
> > >>>> own
> > >>>>>> package could we drop the S at the end of the class names?
> > >>>>>>
> > >>>>>> Thanks,
> > >>>>>> Damian
> > >>>>>>
> > >>>>>>
> > >>>>>> On Fri, 16 Mar 2018 at 15:25 Debasish Ghosh <
> > >>>>> debasish.gh...@lightbend.com>
> > >>>>>> wrote:
> > >>>>>>
> > >>>>>>> Hi -
> > >>>>>>>
> > >>>>>>> A new KIP, KIP-270 is up for discussion:
> > >>>>>>>
> > >>>>>>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >>>>>> 270+-+A+Scala+Wrapper+Library+for+Kafka+Streams
> > >>>>>>>
> > >>>>>>> The relevant JIRA issue: https://issues.apache.org/
> > >>>>>> jira/browse/KAFKA-6670
> > >>>>>>>
> > >>>>>>> The library as proposed in the KIP has been implemented at
> > >>>>>>> https://github.com/lightbend/kafka-streams-scala and the current
> > >>>>> release
> > >>>>>>> is
> > >>>>>>> 0.2.0 (
> > >>>>>>> https://github.com/lightbend/kafka-streams-scala/releases/
> > >>> tag/v0.2.0
> > >>>> ).
> > >>>>>>> We at Lightbend has been using it since quite some time now.
> > >>>>>>>
> > >>>>>>> regards.
> > >>>>>>>
> > >>>>>>> --
> > >>>>>>> Debasish Ghosh
> > >>>>>>> Principal Engineer
> > >>>>>>>
> > >>>>>>> Twitter: @debasishg
> > >>>>>>> Blog: http://debasishg.blogspot.com
> > >>>>>>> Code: https://github.com/debasishg
> > >>>>>>>
> > >>>>>>
> > >>>>>
> > >>>>>
> > >>>>>
> > >>>>> --
> > >>>>> Debasish Ghosh
> > >>>>> Principal Engineer
> > >>>>>
> > >>>>> Twitter: @debasishg
> > >>>>> Blog: http://debasishg.blogspot.com
> > >>>>> Code: https://github.com/debasishg
> > >>>>>
> > >>>>
> > >>>
> > >>>
> > >>>
> > >>> --
> > >>> Debasish Ghosh
> > >>> Principal Engineer
> > >>>
> > >>> Twitter: @debasishg
> > >>> Blog: http://debasishg.blogspot.com
> > >>> Code: https://github.com/debasishg
> > >>>
> > >>
> > >
> >
> >
>
>
> --
> Debasish Ghosh
> Principal Engineer
>
> Twitter: @debasishg
> Blog: http://debasishg.blogspot.com
> Code: https://github.com/debasishg
>



-- 
Senior Software Engineer, Lightbend, Inc.

<http://lightbend.com>

@seg1o <https://twitter.com/seg1o>, in/seanaglover
<https://www.linkedin.com/in/seanaglover/>

Reply via email to