Yeah.
Cleaner in this formation.

On Thu, Feb 1, 2018 at 3:59 PM, Bill Bejeck <bbej...@gmail.com> wrote:

> `To` works for me.
>
> Thanks,
> Bill
>
> On Thu, Feb 1, 2018 at 6:47 PM, Matthias J. Sax <matth...@confluent.io>
> wrote:
>
> > @Paolo:
> >
> > The timestamp will be used to set the message/record metadata timestamp
> > on `Producer.send(new ProducerRecord(...,timestamp,...))`.
> >
> > @Bill,Ted:
> >
> > Might be a good idea. I was thinking about the name, and came up with
> `To`:
> >
> > > context.forward(key, value, To.child("processorX").withTimestamp(5));
> > > context.forward(key, value, To.child(1).withTimestamp(10));
> >
> > Without specifying the downstream child processor:
> >
> > > context.forward(key, value, To.all().withTimestamp(5));
> >
> > WDYT?
> >
> >
> > -Matthias
> >
> > On 2/1/18 8:45 AM, Ted Yu wrote:
> > > I like Bill's idea (pending a better name for the Forwarded).
> > >
> > > Cheers
> > >
> > > On Thu, Feb 1, 2018 at 7:47 AM, Bill Bejeck <bbej...@gmail.com> wrote:
> > >
> > >> Hi Matthias,
> > >>
> > >> Thanks for the KIP!
> > >>
> > >> Could we consider taking an approach similar to what was done in
> KIP-182
> > >> with regards to overloading?
> > >>
> > >> Meaning we could add a "Forwarded" object (horrible name I know) with
> > >> methods withTimestamp, withChildName, and withChildIndex. To handle
> the
> > >> case when both a child-name and child-index is provided we could throw
> > an
> > >> exception.
> > >>
> > >> Then we could reduce the overloaded {{forward}} methods from 6 to 2.
> > >>
> > >> Thanks,
> > >> Bill
> > >>
> > >>
> > >> On Thu, Feb 1, 2018 at 3:49 AM, Paolo Patierno <ppatie...@live.com>
> > wrote:
> > >>
> > >>> Hi Matthias,
> > >>>
> > >>> just a question : what will be the timestamp "type" in the new
> message
> > on
> > >>> the wire ?
> > >>>
> > >>> Thanks,
> > >>> Paolo.
> > >>> ________________________________
> > >>> From: Matthias J. Sax <matth...@confluent.io>
> > >>> Sent: Wednesday, January 31, 2018 2:06 AM
> > >>> To: dev@kafka.apache.org
> > >>> Subject: [DISCUSS] KIP-251: Allow timestamp manipulation in Processor
> > API
> > >>>
> > >>> Hi,
> > >>>
> > >>> I want to propose a new KIP for Kafka Streams that allows timestamp
> > >>> manipulation at Processor API level.
> > >>>
> > >>> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > >>> 251%3A+Allow+timestamp+manipulation+in+Processor+API
> > >>>
> > >>> Looking forward to your feedback.
> > >>>
> > >>>
> > >>> -Matthias
> > >>>
> > >>>
> > >>
> > >
> >
> >
>

Reply via email to