Thanks for taking time to reply.
I thought about the confusion with overloads. creating another Partitioner 
seemed liked a good idea to start with, soon i realized the partitioner is 
interface written in a way which does not support anything other than 
key/value. It seems to me the idea of using RecordContext is a better option as 
suggested by Matthias. That would also open up the options for punctuate as 
well. I will look into 478 further and see if we can merge

On 2020/09/10 22:59:09, Sophie Blee-Goldman <sop...@confluent.io> wrote: 
> Hey Balan, thanks for the KIP!
> 
> The motivation here makes sense to me, but I have a few questions about the
> proposed API
> 
> I guess the main thing to point out is that if we just add new addSink()
> overloads to Topology,
> then only the lower level Processor API will benefit and users of the DSL
> won't be able to utilize
> this. This seems like a useful feature that we should make available to
> anyone.
> 
> We could follow a similar approach and add new toStream overloads to the
> KStream class, but
> that would expand the surface area of the API pretty significantly. The
> additional addSink()
> overloads alone would do this. The addSink() methods already have a pretty
> large number
> of optional parameters which means more and more overloads every time a new
> one is added.
> We should avoid making this problem worse wherever possible.
> 
> Existing StreamPartitioner  in SinkNode will be made null when context
> > partition is enabled
> 
> 
> This line from your KIP gave me some idea that it might be avoidable in
> this case. The implication
> of this quote is that the StreamPartitioner and useContextPartition
> parameter are inherently
> incompatible since they are two ways of specifying the same thing, the
> target partition. Well, if
> that's the case, then we should be able to just leverage the existing
> StreamPartitioner in some
> way to specify that we want records to end up in the source partition,
> without introducing a new
> parameter.
> 
> One option would be to just let users pass in a null StreamPartitioner to
> mean that it should
> use the source partition, but that seems a bit subtle. Maybe a better API
> would be to offer
> a new out-of-the-box StreamPartitioner called SourceContextPartitioner (or
> something),
> and then users just have to pass in an instance of this. WDYT?
> 
> On Thu, Sep 10, 2020 at 8:00 AM Balan k <ksbalan2...@gmail.com> wrote:
> 
> >
> > Forgot to add the link
> >
> > https://cwiki.apache.org/confluence/display/KAFKA/KIP-669%3A+Preserve+Source+Partition+in+Kafka+Streams+from+context
> >
> >
> >
> > On 2020/09/10 13:40:02, satyanarayan komandur <ksbalan2...@gmail.com>
> > wrote:
> > > Hi,
> > >
> > > I have submitted a new KIP for preserving processor record context
> > partition from source. I am looking for suggestions/comments.
> > >
> > > In most use cases where source message is getting transformed and sent
> > to a target topic, where
> > > 1. number of partitions on source and sink topic are same
> > > 2. there is no change to the key
> > > 3. more importantly if we would like to preserve the partition as is
> > without re-deriving using partition from context would help.
> > >
> > > I am aware of one caveat where record processor context partition is not
> > known in stream punctuation.
> > >
> > > Please look over the KIP and chime in more ideas
> > >
> > > Thanks
> > > Balan
> > >
> > >
> > >
> >
> 

Reply via email to