Re: [VOTE] KIP-244: Add Record Header support to Kafka Streams

2018-05-21 Thread Jorge Esteban Quilcate Otoya
ng >> > > > > `setRecordMetadata` API. Since this class is part of the public >> > > > test-utils >> > > > > package we should claim it in the wiki as well. >> > > > > >> > > > > >> > > >

Re: [VOTE] KIP-244: Add Record Header support to Kafka Streams

2018-05-21 Thread Jorge Esteban Quilcate Otoya
e are also adding one additional API plus modifying the existing > > > > > `setRecordMetadata` API. Since this class is part of the public > > > > test-utils > > > > > package we should claim it in the wiki as well. > > > > > > > > >

Re: [VOTE] KIP-244: Add Record Header support to Kafka Streams

2018-05-15 Thread Bill Bejeck
> > > > package we should claim it in the wiki as well. > > > > > > > > > > > > Guozhang > > > > > > > > On Mon, May 14, 2018 at 8:43 AM, Ted Yu <yuzhih...@gmail.com> wrote: > > > > > > > >>

Re: [VOTE] KIP-244: Add Record Header support to Kafka Streams

2018-05-15 Thread Damian Guy
the public > > test-utils > > > package we should claim it in the wiki as well. > > > > > > > > > Guozhang > > > > > > On Mon, May 14, 2018 at 8:43 AM, Ted Yu <yuzhih...@gmail.com> wrote: > > > > > >> +1 > > >

Re: [VOTE] KIP-244: Add Record Header support to Kafka Streams

2018-05-15 Thread Jorge Esteban Quilcate Otoya
gt; +1 > >> > >> On Mon, May 14, 2018 at 8:31 AM, Jorge Esteban Quilcate Otoya < > >> quilcate.jo...@gmail.com> wrote: > >> > >>> Hi everyone, > >>> > >>> I would like to start a vote on KIP-244: Add Record Header support

Re: [VOTE] KIP-244: Add Record Header support to Kafka Streams

2018-05-14 Thread Matthias J. Sax
t 8:43 AM, Ted Yu <yuzhih...@gmail.com> wrote: > >> +1 >> >> On Mon, May 14, 2018 at 8:31 AM, Jorge Esteban Quilcate Otoya < >> quilcate.jo...@gmail.com> wrote: >> >>> Hi everyone, >>> >>> I would like to start a vote on KIP-244: Add

Re: [VOTE] KIP-244: Add Record Header support to Kafka Streams

2018-05-14 Thread Guozhang Wang
m> wrote: > > > Hi everyone, > > > > I would like to start a vote on KIP-244: Add Record Header support to > Kafka > > Streams > > > > KIP wiki page: > > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > > 244%3A+Add+Record+Header+su

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-14 Thread Jorge Esteban Quilcate Otoya
t;>>> proposal. > > > > >>>>>>>>>>>> The current one pass it as part of `RecordContext`, so > if > > > it's > > > > >>>>>>> forward > > > > >>>>>>>>> it > &

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-14 Thread Guozhang Wang
ié., 2 may. 2018 a las 22:56, Guozhang Wang (< > > > >>>>> wangg...@gmail.com > > > >>>>>>>> ) > > > >>>>>>>>>>>> escribió: > > > >>>>>>>>>>&

Re: [VOTE] KIP-244: Add Record Header support to Kafka Streams

2018-05-14 Thread Ted Yu
+1 On Mon, May 14, 2018 at 8:31 AM, Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > Hi everyone, > > I would like to start a vote on KIP-244: Add Record Header support to Kafka > Streams > > KIP wiki page: > https://cwiki.apache.org/confluence/disp

[VOTE] KIP-244: Add Record Header support to Kafka Streams

2018-05-14 Thread Jorge Esteban Quilcate Otoya
Hi everyone, I would like to start a vote on KIP-244: Add Record Header support to Kafka Streams KIP wiki page: https://cwiki.apache.org/confluence/display/KAFKA/KIP-244%3A+Add+Record+Header+support+to+Kafka+Streams+Processor+API The discussion thread is here: http://mail-archives.apache.org

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-14 Thread Jorge Esteban Quilcate Otoya
h John's): > > >>>>>>>>>>>>> > > >>>>>>>>>>>>> 1. We need to have a clear inheritance protocol of headers > in > > >>> our > > >>>>>>>>>>>> topology: > > >>>>>>

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-11 Thread Guozhang Wang
e headers then in order to be sent downstreams? > >>>>>>>>>>>>> > >>>>>>>>>>>>> 3. In future work "Adding DSL Processors to use Headers to > >>>>>>>>>>>>> filter/map/branc

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-11 Thread Matthias J. Sax
ders headers)", >> this >>>>>>> should >>>>>>>>>>> be >>>>>>>>>>>>> removed? >>>>>>>>>>>>> >>>>>>>>>>>>> 5. MINOR: it seems to be the

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-11 Thread Guozhang Wang
gt; I agree that de-scoping the work to just the Processor API > will > > >>>> help > > >>>>>>>>>>> contain the design and implementation complexity. > > >>>>>>>>>>> > > >>>>>>>>>>> In

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-11 Thread Jorge Esteban Quilcate Otoya
quot;process", since it wouldn't be necessary in > >> conjunction > >>>>>>>> with > >>>>>>>>>> the > >>>>>>>>>>> ProcessorContext change, and it's not represented in your PR.

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-10 Thread Matthias J. Sax
> make >>>>>>>> the >>>>>>>>>> API >>>>>>>>>>> changes, while promising to drop all headers from input records. >>>>>>>>>>> >>>>>>>>>>>

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-10 Thread Guozhang Wang
n having a complex list of operators that do or don't > >>>>>> forward > >>>>>>>>> them. Plus, I think it might be tricky to define this behavior > >> while > >>>>>>> not > >>>>&

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-07 Thread Matthias J. Sax
;>>>>>>> >>>>>>>>>> Hi Matthias, >>>>>>>>>> >>>>>>>>>> I've created a new JIRA to track this, updated the KIP and create >> a >>>>>>> PR. >>>>>>>

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-07 Thread Guozhang Wang
>>>> question of this work, seems to be how to expose headers at DSL > >>>>>> level. > >>>>>>>>> This related to KIP-149 and KIP-159. However, for Processor API, > >>>> it > >>>>>>>>> seems to be

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-06 Thread Matthias J. Sax
and KIP accordingly. Note, >>>>> that >>>>>> we >>>>>>>>> have two JIRA that are duplicates atm. We can scope them >>>>> accordingly: >>>>>>>>> one for PAPI only,

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-06 Thread Guozhang Wang
t;> key/value > >>>>>> and > >>>>>>>> then group using current KeyValue structure. > >>>>>>>> > >>>>>>>> 2. Yes. IMO key/value stores, like RocksDB, rely on KV as > >&g

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-06 Thread Matthias J. Sax
e >>> that >>>>>> need >>>>>>>> this). >>>>>>>> >>>>>>>> 3. and 4. Changes on 1. will solve this issue. >>>>>>>> >>>>>>>> Probably I rush a bit proposing this change

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-04 Thread Guozhang Wang
> > > >> As Bill pointed out already, we should be careful with adding > > new > > > > > > >> overloads as this contradicts the work done via KIP-182. > > > > > > >> > > > > > > >> This KIP also s

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-04 Thread Florian Garcia
; > > > >> > > > > > >> - why do you want to add the headers to `KeyValue`? I am not > sure > > > if > > > > we > > > > > >> should consider headers as optional metadata and add it to > > > > > >> `

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-03 Thread Jorge Esteban Quilcate Otoya
l. Do you suggest that all other operator just drop headers on > > the > > > > >> floor? > > > > >> > > > > >> - Why do you only want to put headers into in-memory and cache > but > > > not > > > >

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-02 Thread Guozhang Wang
ed to store record header in state in the first > place, > > if > > > >> we exclude stateful operator at DSL level? > > > >> > > > >> > > > >> What is the motivation for the "border lines" you choose? > > &

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-02 Thread John Roesler
tthias > > >> > > >> > > >> On 12/21/17 8:18 AM, Bill Bejeck wrote: > > >>> Jorge, > > >>> > > >>> Thanks for the KIP, I know this is a feature others in the community > > have > > >>> been interes

Re: KIP-244: Add Record Header support to Kafka Streams

2018-05-02 Thread Jorge Esteban Quilcate Otoya
gt;> I took a quick pass over it, and I have one initial question. > >>> > >>> We recently reduced overloads with KIP-182, and in this KIP we are > >>> increasing them again. > >>> > >>> I can see from the KIP why they are necessa

[jira] [Created] (KAFKA-6850) KIP-244: Add Record Header support to Kafka Streams Processor API

2018-05-02 Thread Jorge Quilcate (JIRA)
Jorge Quilcate created KAFKA-6850: - Summary: KIP-244: Add Record Header support to Kafka Streams Processor API Key: KAFKA-6850 URL: https://issues.apache.org/jira/browse/KAFKA-6850 Project: Kafka

Re: KIP-244: Add Record Header support to Kafka Streams

2018-02-13 Thread Matthias J. Sax
t;>> don't have any sound suggestions ATM, so I'll have to think about it some >>> more, but I wanted to put the thought out there. >>> >>> Thanks, >>> Bill >>> >>> On Thu, Dec 21, 2017 at 9:06 AM, Jorge Esteban Quilcate Otoya < >&

Re: KIP-244: Add Record Header support to Kafka Streams

2017-12-30 Thread Jorge Esteban Quilcate Otoya
, > > Bill > > > > On Thu, Dec 21, 2017 at 9:06 AM, Jorge Esteban Quilcate Otoya < > > quilcate.jo...@gmail.com> wrote: > > > >> Hi all, > >> > >> I have created a KIP to add Record Headers support to Kafka Streams API: > >> https://cwik

Re: KIP-244: Add Record Header support to Kafka Streams

2017-12-26 Thread Matthias J. Sax
ght out there. > > Thanks, > Bill > > On Thu, Dec 21, 2017 at 9:06 AM, Jorge Esteban Quilcate Otoya < > quilcate.jo...@gmail.com> wrote: > >> Hi all, >> >> I have created a KIP to add Record Headers support to Kafka Streams API: >> https://cwiki.

Re: KIP-244: Add Record Header support to Kafka Streams

2017-12-21 Thread Bill Bejeck
AM, Jorge Esteban Quilcate Otoya < quilcate.jo...@gmail.com> wrote: > Hi all, > > I have created a KIP to add Record Headers support to Kafka Streams API: > https://cwiki.apache.org/confluence/display/KAFKA/KIP- > 244%3A+Add+Record+Header+support+to+Kafka+Streams > > &g

KIP-244: Add Record Header support to Kafka Streams

2017-12-21 Thread Jorge Esteban Quilcate Otoya
Hi all, I have created a KIP to add Record Headers support to Kafka Streams API: https://cwiki.apache.org/confluence/display/KAFKA/KIP-244%3A+Add+Record+Header+support+to+Kafka+Streams The main goal is to be able to use headers to filter, map and process records as streams. Stateful processing