Going to put all possible szenarios. Just wanted to layout the table and ask if its usefull and understandable first

On 06.11.2017 22:33, Ted Yu wrote:
bq. Update in A delete in A update in B delete in B

Are you going to fill in the above scenario (currently blank) ?

On Mon, Nov 6, 2017 at 12:31 PM, Jan Filipiak <jan.filip...@trivago.com>
wrote:

I created an example Table in the WIKI page
Can you quickly check if that would be a good format?
I tried todo it ~like the unit tests but with the information of what
state is there _AFTER_
processing happend.
I make the first 2 columns exclusive even though the in fact run in
parallel but the joining
task serializes the effects.

Best Jan

On 06.11.2017 21:20, Jan Filipiak wrote:

Will do! Need to do it carefully. One mistake in this detailed approach
and confusion is perfect ;)
Hope I can deliver this week.

Best Jan


On 06.11.2017 17:21, Matthias J. Sax wrote:

Jan,

thanks a lot for this KIP. I did an initial pass over it, but feel a
little lost. Maybe I need to read it more carefully, but atm it's not
clear to me at all what algorithm you propose.

I think it would be super helpful, to do an example with concrete data
that show how records are stored, what the different value mappers
extract, and what is written into repartitioning topics.



-Matthias


On 11/5/17 2:09 AM, Jan Filipiak wrote:

Hi Gouzhang

I hope the wikipage looks better now. made a little more effort into the
diagram. Still not ideal but I think it serves its purpose.



On 02.11.2017 01:17, Guozhang Wang wrote:

Thanks for the KIP writeup Jan. I made a first pass and here are some
quick
comments:


1. Could we use K0 / V0 and K1 / V1 etc since K0 and KO are a bit
harder to
differentiate when reading.

2. I think you missed the key type in the intrusive approach example
code
snippet regarding "KTable <V0> oneToManyJoin"? Should that be

KTable<CombinedKey<K,KO>, V0> oneToManyJoin

3. Some of the arrows in your algorithm section's diagrams seems
reversed.

4. In the first step of the algorithm, "Materialize B first", that
happens
in the "Repartition by A's key" block right? If yes, could you clarify
it
in the block?

5. "skip old if A's key didn't change": hmm, not sure if we can skip
it.
What if other fields (neither A's key or B's key) changes? Suppose you
have
an aggregation after the join, we still need to subtract the old value
from
the aggregation right?

6. In the block of "Materialize B", I think from your description we
are
actually materializing both A and B right? If yes could you update the
diagram?

7. This is a meta question: "in the sink, only use A's key to determine
partition" I think we had the discussion long time ago, that if we are
sending the old and new entries of the pair to different partitions,
their
ordering may get reversed later when reading from the join operator
(i.e.
the "Materialize B" block in your diagram). How did you address that
with
this proposal?

8. "B records with a 'null' A-key value would be silently dropped"
Where
are we dropping it, do we drop it at the first sub-topology (i.e the
"Repartition by A's key" block)?

Guozhang





On Wed, Nov 1, 2017 at 12:18 PM, Jan Filipiak <
jan.filip...@trivago.com>
wrote:

Hi thanks for the feedback
On 01.11.2017 12:58, Damian Guy wrote:

Hi Jan, Thanks for the KIP!
In both alternatives the API will need to use the `Joined` class
rather
than than passing in `Serde`s. Also, as with all other joins etc,
there
probably should be an overload that doesn't require any `Serdes`.

Will check again how current API looks. I remember loosing the
argument
with this IQ overloads things.
Didn't expect something to have happend already so I just copied from
the
PR. Will update.
Will also add the overload.

It isn't clear to me what `joinPrefixFaker` is doing? In the comment
it
says "returning an outputKey that when serialized only produces a
prefix
of
the output key which is the same serializing K" So why not just use
"K" ?

The faker in fact returns K wich can be serialized by the Key Serde
in the
rocks. But it needs to only contain A's key and it needs to be a
strict
prefix
byte[] of all K with this A's key. We gonna seek there with an
RocksIterator and continue to read as long as the "faked key"
serialized
form is a prefix
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 <yuzhih...@gmail.com> 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
<jan.filip...@trivago.com
wrote:


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/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 <
jan.filip...@trivago.com
wrote:

Hello everyone,

this is the new discussion thread after the ID-clash.
Best
Jan

______


Hello Kafka-users,

I want to continue with the development of KAFKA-3705, which
allows
the
Streams DSL to perform KTableKTable-Joins when the KTables have a
one-to-many relationship.
To make sure we cover the requirements of as many users as
possible
and
have a good solution afterwards I invite everyone to read
through the
KIP I
put together and discuss it here in this Thread.

https://cwiki.apache.org/confluence/display/KAFKA/KIP-213+
Support+non-key+joining+in+KTable
https://issues.apache.org/jira/browse/KAFKA-3705
https://github.com/apache/kafka/pull/3720

I think a public discussion and vote on a solution is exactly
what is
needed to bring this feauture into kafka-streams. I am looking
forward

to
everyones opinion!

Please keep the discussion on the mailing list rather than
commenting

on
the wiki (wiki discussions get unwieldy fast).

Best
Jan







Reply via email to