Update:
I want to give a quick update on what I found porting the 0.10 version
towards 1.0.
1. It is difficult to provide a stock CombinedKey Serde.
We effectively wrap 2 serdes for the key. We do not have good topic
names to feed into the Avro Serde for K1 and K2 for the same topic.
Hi Guozhang,
thanks for the update.
On 15.12.2017 22:54, Guozhang Wang wrote:
Jan,
Thanks for the updated KIP, and the raised questions. Here are my thoughts
around the "back and forth mapper" approach on your wiki:
1) regarding the key-value types of KTableValueGetter, we do not
necessarily
Jan,
Thanks for the updated KIP, and the raised questions. Here are my thoughts
around the "back and forth mapper" approach on your wiki:
1) regarding the key-value types of KTableValueGetter, we do not
necessarily enforce its template K, V to be the same as its calling
Processor, although today
Hi, I updated the KIP
I would be open for this:
We mark the "less intrusive" and the "back and forth mapper" approach as
rejected alternatives.
and implement the two remaining methods.
any thoughts?
Best jan
On 07.12.2017 12:58, Jan Filipiak wrote:
On 05.12.2017 00:42, Matthias J. Sax wr
On 05.12.2017 00:42, Matthias J. Sax wrote:
Jan,
The KTableValueGetter thing is a valid point. I think we would need a
backwards mapper (or merge both into one and sacrifices lambdas?).
Another alternative would be, to drop the optimization and materialize
the KTable.operator() result... (not a
Jan,
The KTableValueGetter thing is a valid point. I think we would need a
backwards mapper (or merge both into one and sacrifices lambdas?).
Another alternative would be, to drop the optimization and materialize
the KTable.operator() result... (not a great solution either). I am
personally fine w
Hi Matthias,
2 things that pop into my mind sunday morning. Can we provide an
KTableValueGetter when key in the store is different from the key
forwarded?
1. we would need a backwards mapper
2. I am not sure if we can pull it of w/o said forth generic type in
KTable (that I am in favour of bt
Just to clarify, though "CombinedKey" will still be exposed as public APIs
it would not be used in any of the returned key types, so users do not need
to worry providing a serde for it at all; it will only be used in the
Mapper parameter, and internally Streams library would know how to serde it
if
Jan,
Thanks for explaining the Serde issue! This makes a lot of sense.
I discussed with Guozhang about this issue and came up with the
following idea that bridges both APIs:
We still introduce CombinedKey as a public interface and exploit it to
manage the key in the store and the changelog topic
Just list what each thing is:
K0: key type of first/this table
K1: key type of second/other table
KO: key type of result table (concatenation of both input keys )
something like this (not sure it the example above is correct---it's
just for illustration)
-Matthias
On 11/18/17 2:30 PM, Jan Fi
Hello Jan,
I think I get your point about the cumbersome that CombinedKey would
introduce for serialization and tooling based on serdes. What I'm still
wondering is the underlying of joinPrefixFakers mapper: from your latest
comment it seems this mapper will be a one-time mapper: we use this to ma
-> it think the relationships between the different used types, K0,K1,KO
should be explains explicitly (all information is there implicitly, but
one need to think hard to figure it out)
I'm probably blind for this. can you help me here? how would you
formulate this?
Thanks,
Jan
On 16.11.20
On 17.11.2017 06:59, Guozhang Wang wrote:
Thanks for the explanation Jan. On top of my head I'm leaning towards the
"more intrusive" approach to resolve the race condition issue we discussed
above. Matthias has some arguments for this approach already, so I would
not re-iterate them here. To me
Hi Matthias
answers to the questions inline.
On 16.11.2017 23:18, Matthias J. Sax wrote:
Hi,
I am just catching up on this discussion and did re-read the KIP and
discussion thread.
In contrast to you, I prefer the second approach with CombinedKey as
return type for the following reasons:
1
Thanks for the explanation Jan. On top of my head I'm leaning towards the
"more intrusive" approach to resolve the race condition issue we discussed
above. Matthias has some arguments for this approach already, so I would
not re-iterate them here. To me I find the "ValueMapper
joinPrefixFaker" is a
Hi,
I am just catching up on this discussion and did re-read the KIP and
discussion thread.
In contrast to you, I prefer the second approach with CombinedKey as
return type for the following reasons:
1) the oneToManyJoin() method had less parameter
2) those parameters are easy to understand
3
1. Going over KIP-213, I am leaning toward the "less intrusive" approach.
In my use case, I am planning on performing a sequence of several oneToMany
joins, From my understanding, the more intrusive approach would result in
several nested levels of CombinedKey's. For example, consider Tables A, B,
We are running this perfectly fine. for us the smaller table changes
rather infrequent say. only a few times per day. The performance of the
flush is way lower than the computing power you need to bring to the
table to account for all the records beeing emmited after the one single
update.
On
Hi Trevor,
thank you very much for your interested. Too keep discussion mailing
list focused and not Jira or Confluence I decided to reply here.
1. its tricky activity is indeed very low. In the KIP-213 there are 2
proposals about the return type of the join. I would like to settle on one.
Un
;>>> wrote:
>>>>>>>
>>>>>>> Hi thanks for the feedback
>>>>>>>> On 01.11.2017 12:58, Damian Guy wrote:
>>>>>>>>
>>>>>>>> Hi Jan, Thanks for the KIP!
>>>>>>>>
u, Oct 26, 2017 at 11:04 PM, Jan Filipiak
Not even sure if it makes sense to introduce A and B or just stick
with
"this ktable", "other ktable"
Thank you
Jan
On 27.10.2017 06:58, Ted Yu wrote:
Do you mind addressing my previous comments ?
http://search-hadoop.com/m/Kafka/uyzND1hzF
>>>>>> argument
>>>>>> with this IQ overloads things.
>>>>>> Didn't expect something to have happend already so I just copied from
>>>>>> the
>>>>>> PR. Will update.
>>>>>> Will al
k
Thanks for the remarks. hope I didn't miss any.
Not even sure if it makes sense to introduce A and B or just stick
with
"this ktable", "other ktable"
Thank you
Jan
On 27.10.2017 06:58, Ted Yu wrote:
Do you mind addressing my previous comments ?
http://search-hadoop.com
ck
with
"this ktable", "other ktable"
Thank you
Jan
On 27.10.2017 06:58, Ted Yu wrote:
Do you mind addressing my previous comments ?
http://search-hadoop.com/m/Kafka/uyzND1hzF8SRzUqb?subj=Re+
DISCUSS+KIP+213+Support+non+key+joining+in+KTable
On Thu, Oct 26, 2017 at 9:38 PM,
>> This is easy todo for Avro + Protobuf + custom Serdes and Hadoop
>>> Writables. Its a nightmare for JSON serdes.
>>>
>>>
>>>
>>>> Thanks,
>>>> Damian
>>>>
>>>>
>>>> On Fri, 27 Oct 2017 at 10:27 Ted Yu wr
are in the beginning, it makes sense
to
use them since readers would know who they reference.
Cheers
On Thu, Oct 26, 2017 at 11:04 PM, Jan Filipiak
Thanks for the remarks. hope I didn't miss any.
Not even sure if it makes sense to introduce A and B or just stick with
"this ktable",
reference.
Cheers
On Thu, Oct 26, 2017 at 11:04 PM, Jan Filipiak
Thanks for the remarks. hope I didn't miss any.
Not even sure if it makes sense to introduce A and B or just stick with
"this ktable", "other ktable"
Thank you
Jan
On 27.10.2017 06:58, Ted Yu wrote:
D
wrote:
>>
>> I think if you explain what A and B are in the beginning, it makes sense
>>> to
>>> use them since readers would know who they reference.
>>>
>>> Cheers
>>>
>>> On Thu, Oct 26, 2017 at 11:04 PM, Jan Filipiak >> >
Thanks for the remarks. hope I didn't miss any.
Not even sure if it makes sense to introduce A and B or just stick with
"this ktable", "other ktable"
Thank you
Jan
On 27.10.2017 06:58, Ted Yu wrote:
Do you mind addressing my previous comments ?
http://search-hadoop.
if it makes sense to introduce A and B or just stick with
> > "this ktable", "other ktable"
> >
> > Thank you
> > Jan
> >
> >
> > On 27.10.2017 06:58, Ted Yu wrote:
> >
> >> Do you mind addressing my previous comments ?
>
se to introduce A and B or just stick with
> "this ktable", "other ktable"
>
> Thank you
> Jan
>
>
> On 27.10.2017 06:58, Ted Yu wrote:
>
>> Do you mind addressing my previous comments ?
>>
>> http://search-hadoop.com/m/Kafka/uyzND1hzF8SR
sing my previous comments ?
http://search-hadoop.com/m/Kafka/uyzND1hzF8SRzUqb?subj=Re+DISCUSS+KIP+213+Support+non+key+joining+in+KTable
On Thu, Oct 26, 2017 at 9:38 PM, Jan Filipiak
wrote:
Hello everyone,
this is the new discussion thread after the ID-clash.
Best
Jan
__
Hello Kafka-users,
Do you mind addressing my previous comments ?
http://search-hadoop.com/m/Kafka/uyzND1hzF8SRzUqb?subj=Re+DISCUSS+KIP+213+Support+non+key+joining+in+KTable
On Thu, Oct 26, 2017 at 9:38 PM, Jan Filipiak
wrote:
> Hello everyone,
>
> this is the new discussion thread after the ID-clash.
Done. I changed my KIP to KIP-214. So this KIP doesn't need to change.
On Wed, Oct 25, 2017 at 10:33 PM, Onur Karaman wrote:
> Looks like Jan technically made his KIP wiki page first so I'll just
> change my KIP number.
>
> On Wed, Oct 25, 2017 at 4:59 PM, Matthias J. Sax
> wrote:
>
>> Thanks a
Looks like Jan technically made his KIP wiki page first so I'll just change
my KIP number.
On Wed, Oct 25, 2017 at 4:59 PM, Matthias J. Sax
wrote:
> Thanks a lot for the KIP. Can we please move the discussion to the dev
> list?
>
> Thus, after fixing the KIP collision, just start a new DISCUSS t
Thanks a lot for the KIP. Can we please move the discussion to the dev list?
Thus, after fixing the KIP collision, just start a new DISCUSS thread.
Thx.
-Matthias
On 10/25/17 4:20 PM, Ted Yu wrote:
> Have you seen the email a moment ago from Onur which uses the same KIP
> number ?
>
> Looks l
36 matches
Mail list logo