Re: kstream transform forward to different topics

2019-02-13 Thread Matthias J. Sax
The goal of KIP-307 is a different one. It's about providing names to make debugging easier. Thus, I don't think "307 is doing it the wrong way" -- the question is, what problem is addressed, and KIP-307 addresses a different one as discussed on this question. -Matthias On 2/13/19 1:53 AM, Jan

Re: kstream transform forward to different topics

2019-02-13 Thread Jan Filipiak
For now, just use the name it gets automatically, or crack the AbstractStream open with reflection ;) 307 is doing it the wrong way again, just make name accessible instead of make the users put them :face_with_rolling_eyes: On 08.02.2019 02:36, Guozhang Wang wrote: > Hi Nan, > > Glad it helps

Re: kstream transform forward to different topics

2019-02-07 Thread Nan Xu
that will be really helpful, thanks for the heads up. On Thu, Feb 7, 2019 at 7:36 PM Guozhang Wang wrote: > Hi Nan, > > Glad it helps with your case. Just another note that in the next release > when KIP-307 is in place [1], you can actually combine the DSL with PAPI by > naming the last

Re: kstream transform forward to different topics

2019-02-07 Thread Guozhang Wang
Hi Nan, Glad it helps with your case. Just another note that in the next release when KIP-307 is in place [1], you can actually combine the DSL with PAPI by naming the last operator that creates your transformed KStream, and then manually add the sink nodes like: stream2 =

Re: kstream transform forward to different topics

2019-02-07 Thread Nan Xu
awesome, this solution is great, thanks a lot. Nan On Thu, Feb 7, 2019 at 2:28 PM Bill Bejeck wrote: > Hi Nan, > > l see what you are saying about reproducing a join in the PAPI. > > I have another thought. > >1. Have your Transform return a List [r1, r2, r3] >2. Then after your

Re: kstream transform forward to different topics

2019-02-07 Thread Bill Bejeck
Hi Nan, l see what you are saying about reproducing a join in the PAPI. I have another thought. 1. Have your Transform return a List [r1, r2, r3] 2. Then after your transform operation use a flatMapValues operator as this will forward KV pairs of (k, r1), (k, r2), and (k, r3). >From

Re: kstream transform forward to different topics

2019-02-07 Thread Nan Xu
hmm, but my DSL logic at beginning involve some join between different streams, so I feel that will be quit complex to write everything in PAPI. what if I do this. in the transform, I return all 3 classes as a tuple. then to map 3 times on the same stream like this transformer { return

Re: kstream transform forward to different topics

2019-02-07 Thread Bill Bejeck
Hi Nan, I wanted to follow up some more. Since you need your Transformer forward to 3 output topics or more generally any time you want a processor to forward to multiple child nodes or specific nodes in the topology, you can best achieve this kind of control and flexibility using the PAPI.

Re: kstream transform forward to different topics

2019-02-07 Thread Nan Xu
thanks, just to make sure I understand this correctly,. I have some processing logic using DSL, after those processing, I have a kstream, from this kstream, I need to do a transform and put result to different topics. To use processor api, I need to put this kstream to a topic, then use

Re: kstream transform forward to different topics

2019-02-07 Thread Bill Bejeck
Hi Nan, To forward to the 3 different topics it will probably be easier to do this in the Processor API. Based off what your stated in your question, the topology will look something like this: final Topology topology = new Topology(); topology.addSource("source-node", "input-topic");

kstream transform forward to different topics

2019-02-06 Thread Nan Xu
when I do the transform, for a single input record, I need to output 3 different records, those 3 records are in different classes. I want to send the each type of records to a separate topic, my understanding is I should use context.forward inside the transformer like Transformer{..