Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-09-19 Thread Matthias J. Sax
Sorry that I never replied. I am fine with the current proposal :) -Matthias On 7/12/19 5:09 AM, Adam Bellemare wrote: > @Matthias J. Sax - Thoughts on the > semantics of simply leaving it as-is, with the extra tombstones? As John > put it: "It may be unnecessary to

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-07-12 Thread Adam Bellemare
@Matthias J. Sax - Thoughts on the semantics of simply leaving it as-is, with the extra tombstones? As John put it: "It may be unnecessary to "delete" a non-existant record from a view, but it's never incorrect." It may not be ideal, but the complexity of eliminating it seems to be high and frank

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-07-11 Thread Jan Filipiak
On 10.07.2019 06:25, Adam Bellemare wrote: > In my experience (obviously empirical) it seems that many people just want > the ability to join on foreign keys for the sake of handling all the > relational data in their event streams and extra tombstones don't matter at > all. This has been my own

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-07-10 Thread Adam Bellemare
Hi John Excellent post! I greatly appreciate your insight (and XKCD reference). I think you've hit it on the head. I was mostly concerned with the semantics of having a true INNER join where no unnecessary tombstones are emitted. As you point out though, the semantics are not the issue, it is the

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-07-10 Thread John Roesler
Hey Matthias and Adam, I've been mulling over your recent conversation. I'll share my two cents... - First (because it's always best to get the semantics clear before the details): Extra tombstones do NOT harm the semantics of an inner join. The difference between a left join and an inner joi

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-07-09 Thread Adam Bellemare
I know what I posted was a bit of a wall of text, but three follow up thoughts to this: 1) Is it possible to enforce exactly-once for a portion of the topology? I was trying to think about how to process my proposal with at-least-once processing (or at-most-once processing) and I came up empty-han

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-07-04 Thread Adam Bellemare
Hi Matthias A thought about a variation of S1 that may work - it has a few moving parts, so I hope I explained it clearly enough. When we change keys on the LHS: (k,a) -> (k,b) (k,a,hashOf(b),PROPAGATE_OLD_VALUE) goes to RHS-0 (k,b,PROPAGATE_NEW_VALUE) goes to RHS-1 A) When the (k,a,hash

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-07-03 Thread Matthias J. Sax
It was KIP-77: https://cwiki.apache.org/confluence/display/KAFKA/KIP-77%3A+Improve+Kafka+Streams+Join+Semantics It did all kind of improvements, including removing unnecessary tombstones. About S-1: I agree that it would be complex and impact performance. I did not think about all details yet, bu

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-07-03 Thread Adam Bellemare
Hi Matthias Do you happen to recall what the impact was of having unnecessary tombstones? I am wondering if the negative impact is still relevant today, and if so, if you can recall the PRs or KIPs related to it. That being said, I think that S-1 is too complex in terms of synchronization. It see

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-07-02 Thread Matthias J. Sax
Thanks for the example. I was thinking about the problem a little bit, and I believe we should look at it in some more details. Basically, there are 3 cases: a) insert new record LHS b) delete record LHS c) update exiting record LHS For those cases we want different things to happen: a-1) sent

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-28 Thread John Roesler
Woah. It's your turn to make my head hurt! I think we can make one simplifying assumption: we will probably never need a second version. We're just hedging in case we do. Recursively, if we need a second one, then we'll probably never need a third one, etc. In other words, I wouldn't worry too muc

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-28 Thread Adam Bellemare
Just some thoughts around the versioning. I'm trying to work out a more elegant way to handle it than what I've come up with so far below. *1) I'm planning to use an enum for the versions, but I am not sure how to tie the versions to any particular release. For instance, something like this is do

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-28 Thread Adam Bellemare
Hi Matthias Yes, thanks for the questions - I know it's hard to keep up with all of the various KIPs and everything. The instructions are not stored anywhere, but are simply a way of letting the RHS know how to handle the subscription and reply accordingly. The only case where we send an unneces

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-27 Thread Matthias J. Sax
Thanks for bringing this issue to our attention. Great find @Joe! Adding the instruction field to the `subscription` sounds like a good solution. What I don't understand atm: for which case would we need to send unnecessary tombstone? I thought that the `instruction` field helps to avoid any unnec

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-27 Thread John Roesler
Hi Adam, Hah! Yeah, I felt a headache coming on myself when I realized this would be a concern. For what it's worth, I'd also lean toward versioning. It seems more explicit and more likely to keep us all sane in the long run. Since we don't _think_ our wire protocol will be subject to a lot of re

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-27 Thread Adam Bellemare
You're stretching my brain, John! I prefer STRATEGY 1 because it solves the problem in a simple way, and allows us to deprecate support for older message types as we go (ie, we only support the previous 3 versions, so V5,V4,V3, but not v2 or V1). STRATEGY 2 is akin to Avro schemas between two mic

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-27 Thread John Roesler
I think I agree with you, right joins (and therefore full outer joins) don't make sense here, because the result is a keyed table, where the key is the PK of the left-hand side. So, when you have a right-hand-side record with no incoming FK references, you would want to produce a join result like `

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-26 Thread Adam Bellemare
Hi John Good thinking with regards to upgrade path between versions regarding over-the-wire instructions in SubscriptionWrapper. At this point in time I can't think of any new wire message instructions, but I would appreciate as many eyes on it as possible. I have just included the LEFT join in th

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-26 Thread John Roesler
Thanks, Adam! One unrelated thought that has just now occurred to me is that (unlike the equi-joins we currently have), this join logic is potentially spread over multiple Streams instances, which in general means that the instances may be running different versions of Kafka Streams. This means t

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-26 Thread Adam Bellemare
Sigh... Forgot the link: https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=74684836&selectedPageVersions=78&selectedPageVersions=74 I'll update it when I validate that there are no issues with removing the SubscriptionResponseWrapper boolean. On Wed, Jun 26, 2019 at 3:37

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-26 Thread Adam Bellemare
>Maybe just call it as (k, leftval, null) or (k, null, rightval)? Done. > if you update the KIP, you might want to send a new "diff link" to this thread Here it is: > Looking closely at the proposal, can you explain more about the propagateIfNull field in SubscriptionResponseWrapper? It sort of l

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-26 Thread John Roesler
I think the "scenario trace" is very nice, but has one point that I found confusing: You indicate a retraction in the join output as (k,null) and a join result as (k, leftval, rightval), but confusingly, you also write a join result as (k, JoinResult) when one side is null. Maybe just call it as (

Re: [DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-26 Thread Adam Bellemare
Thanks John. I'm looking forward to any feedback on this. In the meantime I will work on the unit tests to ensure that we have well-defined and readable coverage. At the moment I cannot see a way around emitting (k,null) whenever we emit an event that lacks a matching foreign key on the RHS, exce

[DISCUSS] KIP-213: Second follow-up on Foreign Key Joins

2019-06-26 Thread John Roesler
Hi Adam, Thanks for the proposed revision to your KIP (https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=74684836&selectedPageVersions=77&selectedPageVersions=74) in response to the concern pointed out during code review (https://github.com/apache/kafka/pull/5527#issuecom