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 that if we discover a bug that requires us to again change
the wire message (as you did in this proposal update), we need to
consider what should happen if the PK instance is newer than the FK
instance, or vice-versa, during a rolling upgrade. We should think
ahead to this condition and make sure the logic is forward compatible.

Related: what about the initial case, when we release this feature
(let's say in 2.4)? What will happen if I decide to adopt 2.4 and add
a FK join together in one upgrade. Thus, the 2.4 member of the cluster
is producing the SubscriptionWrapper messages, and some 2.3 members
get the subscription topic assigned to them, but they have no idea
what to do with it? I'm not sure this is a problem; hopefully they
just do nothing. If it is a problem, it would be fine to say you have
to upgrade completely to 2.4 before deploying a FK join.

Just want to make sure we anticipate these issues in case it affects
the design at all.

Thanks,
-John

On Wed, Jun 26, 2019 at 2:38 PM Adam Bellemare <adam.bellem...@gmail.com> wrote:
>
> 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 PM Adam Bellemare <adam.bellem...@gmail.com>
> wrote:
>
> > >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 looks like
> > it's always going to be equal to (RHS-result != null).
> > I believe you are correct, and I missed the forest for the trees. They are
> > effectively the same thing, and I can simply remove the flag. I will code
> > it up and try it out locally just to be sure.
> >
> > Thanks again for your help, it is greatly appreciated!
> >
> > On Wed, Jun 26, 2019 at 2:54 PM John Roesler <j...@confluent.io> wrote:
> >
> >> 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 (k, leftval, null) or (k, null, rightval)? That way the readers
> >> can more easily determine if the results meet their expectations for
> >> each join type.
> >>
> >> (procedural note: if you update the KIP, you might want to send a new
> >> "diff link" to this thread, since the one I posted at the beginning
> >> would not automatically show your latest changes)
> >>
> >> I was initially concerned that the proposed algorithm would wind up
> >> propagating something that looks like a left join (k, leftval, null)
> >> under the case that Joe pointed out, but after reviewing your
> >> scenario, I see that it will emit a tombstone (k, null) instead. This
> >> is appropriate, and unavoidable, since we have to retract the join
> >> result from the logical view (the join result is a logical Table).
> >>
> >> Looking closely at the proposal, can you explain more about the
> >> propagateIfNull field in SubscriptionResponseWrapper?
> >> It sort of looks like it's always going to be equal to (RHS-result !=
> >> null).
> >>
> >> In other words, can we drop that field and just send back RHS-result
> >> or null, and then handle it on the left-hand side like:
> >> if (rhsOriginalValueHash doesn't match) {
> >>     emit nothing, just drop the update
> >> } else if (joinType==inner && rhsValue == null) {
> >>     emit tombstone
> >> } else {
> >>     emit joiner(lhsValue, rhsValue)
> >> }
> >>
> >> To your concern about emitting extra tombstones, personally, I think
> >> it's fine. Clearly, we should try to avoid unnecessary tombstones, but
> >> all things considered, it's not harmful to emit some unnecessary
> >> tombstones: their payload is small, and they are trivial to handle
> >> downstream. If users want to, they can materialize the join result to
> >> suppress any extra tombstones, so there's a way out.
> >>
> >> Thanks for the awesome idea. It's better than what I was thinking.
> >> -john
> >>
> >> On Wed, Jun 26, 2019 at 11:37 AM Adam Bellemare
> >> <adam.bellem...@gmail.com> wrote:
> >> >
> >> > 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, except in the
> >> > (k,null) -> (k,fk) case.
> >> > If this LHS oldValue=null, we know we would have emitted a deletion and
> >> so
> >> > (k,null) would be emitted out of the join. In this case we don't need to
> >> > send another null.
> >> >
> >> > Adam
> >> >
> >> > On Wed, Jun 26, 2019 at 11:53 AM John Roesler <j...@confluent.io>
> >> wrote:
> >> >
> >> > > 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#issuecomment-505137962)
> >> > >
> >> > > We should have a brief discussion thread (here) in the mailing list to
> >> > > make sure everyone who wants to gets a chance to consider the
> >> > > modification to the design.
> >> > >
> >> > > Thanks,
> >> > > -John
> >> > >
> >>
> >

Reply via email to