Hi John,

Thanks for KIP! I've a few comments below:

1. So far the "Motivation" section is very general, and the only concrete
example that I have in mind is `TransformValues#punctuate`. Do we have any
other concrete issues that drive this KIP? If not then I feel better to
narrow the scope of this KIP to:

1.a) modifying ProcessorContext only with the output types on forward.
1.b) modifying Transformer signature to have generics of ProcessorContext,
and then lift the restricting of not using punctuate: if user did not
follow the enforced typing and just code without generics, they will get
warning at compile time and get run-time error if they forward wrong-typed
records, which I think would be acceptable.

I feel this would be a good solution for this specific issue; again, feel
free to update the wiki page with other known issues that cannot be
resolved.

2. If, we want to go with the current scope then my next question would be,
how much breakage we would introducing if we just modify the Processor
signature directly? My feeling is that DSL users would be most likely not
affected and PAPI users only need to modify a few lines on class
declaration. I feel it worth doing some research on this part and then
decide if we really want to bite the bullet of duplicated Processor /
ProcessorSupplier classes for maintaining compatibility.


Guozhang



On Wed, Jun 19, 2019 at 12:21 PM John Roesler <j...@confluent.io> wrote:

> Hi all,
>
> In response to the feedback so far, I changed the package name from
> `processor2` to `processor.generic`.
>
> Thanks,
> -John
>
> On Mon, Jun 17, 2019 at 4:49 PM John Roesler <j...@confluent.io> wrote:
> >
> > Thanks for the feedback, Sophie!
> >
> > I actually felt a little uneasy when I wrote that remark, because it's
> > not restricted at all in the API, it's just available to you if you
> > choose to give your stores and context the same parameters. So, I
> > think your use case is valid, and also perfectly permissable under the
> > current KIP. Sorry for sowing confusion on my own discussion thread!
> >
> > I'm not crazy about the package name, either. I went with it only
> > because there's seemingly nothing special about the new package except
> > that it can't have the same name as the old one. Otherwise, the
> > existing "processor" and "Processor" names for the package and class
> > are perfectly satisfying. Rather than pile on additional semantics, it
> > seemed cleaner to just add a number to the package name.
> >
> > This wouldn't be the first project to do something like this... Apache
> > Commons, for example, has added a "2" to the end of some of their
> > packages for exactly the same reason.
> >
> > I'm open to any suggestions. For example, we could do something like
> > org.apache.kafka.streams.typedprocessor.Processor or
> > org.apache.kafka.streams.processor.typed.Processor , which would have
> > just about the same effect. One microscopic thought is that, if
> > there's another interface in the "processor" package that we wish to
> > do the same thing to, would _could_ pile it in to "processor2", but we
> > couldn't do the same if we use a package that has "typed" in the name,
> > unless that change is _also_ related to types in some way. But this
> > seems like a very minor concern.
> >
> > What's your preference?
> > -John
> >
> > On Mon, Jun 17, 2019 at 3:56 PM Sophie Blee-Goldman <sop...@confluent.io>
> wrote:
> > >
> > > Hey John, thanks for writing this up! I like the proposal but there's
> one
> > > point that I think may be too restrictive:
> > >
> > > "A processor that happens to use a typed store is actually emitting the
> > > same types that it is storing."
> > >
> > > I can imagine someone could want to leverage this new type safety
> without
> > > also limiting how they can interact with/use their store. As an
> (admittedly
> > > contrived) example, say you have an input stream of purchases of a
> certain
> > > type (entertainment, food, etc), and on seeing a new record you want to
> > > output how many types of purchase a shopper has made more than 5
> purchases
> > > of in the last month. Your state store will probably be holding some
> more
> > > complicated PurchaseHistory object (keyed by user), but your output is
> just
> > > a <User, Long>
> > >
> > > I'm also not crazy about "processor2" as the package name ... not sure
> what
> > > a better one would be though (something with "typed"?)
> > >
> > > On Mon, Jun 17, 2019 at 12:47 PM John Roesler <j...@confluent.io>
> wrote:
> > >
> > > > Hi all,
> > > >
> > > > I'd like to propose KIP-478 (
> https://cwiki.apache.org/confluence/x/2SkLBw
> > > > ).
> > > >
> > > > This proposal would add output type bounds to the Processor interface
> > > > in Kafka Streams, which enables static checking of a number of useful
> > > > properties:
> > > > * A processor B that consumes the output of processor A is actually
> > > > expecting the same types that processor A produces.
> > > > * A processor that happens to use a typed store is actually emitting
> > > > the same types that it is storing.
> > > > * A processor is simply forwarding the expected types in all code
> paths.
> > > > * Processors added via the Streams DSL, which are not permitted to
> > > > forward results at all are statically prevented from doing so by the
> > > > compiler
> > > >
> > > > Internally, we can use the above properties to achieve a much higher
> > > > level of confidence in the Streams DSL implementation's correctness.
> > > > Actually, while doing the POC, I found a few bugs and mistakes, which
> > > > become structurally impossible with KIP-478.
> > > >
> > > > Additionally, the stronger types dramatically improve the
> > > > self-documentation of our Streams internal implementations, which
> > > > makes it much easier for new contributors to ramp up with confidence.
> > > >
> > > > Thanks so much for your consideration!
> > > > -John
> > > >
>


-- 
-- Guozhang

Reply via email to