You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by John Roesler <jo...@confluent.io> on 2019/06/26 15:53:04 UTC

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

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

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

Posted by John Roesler <jo...@confluent.io>.
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 much about the version
management problem. It's just important that we have the _ability_ to
update the protocol later on.

Second: there doesn't seem to be any utility in mapping between
protocol versions and release versions. As long as you're getting a
message with a version you understand, it's all good.

So to the main question, I don't think we need to worry about how
future versions may or may not use the instructions enum. In the
initial version, it's enough just to add the version number to the
serial format, and then do something reasonable if we get a version we
don't understand, like log an error and shut down.

To design beyond this, we'd have to speculate about what kind of
modifications we might have to make later, and we just don't know how
accurate those speculations would be until it happens.

For the actual version, I think a numberical Byte is fine, otherwise,
we'll have a proliferation of Version enums.

How does that sound?
-John

On Fri, Jun 28, 2019 at 9:59 AM Adam Bellemare <ad...@gmail.com> wrote:
>
> 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 doable, but may become more complex as versions go on. See 2 & 3
> below.*
>
> public enum Version {
>   V0((byte) 0x00),  //2.4.0
>   V1((byte) 0x01);  //2.5.0
> }
>
> *2) Keeping track of compatibility is a bit tricky. For instance, how do we
> know which messages are compatible and which are breaking? Which upgrade
> paths do we handle and which ones do we not?  How long do we handle support
> for old message versions? 2 minor releases? 1 major release?*
>
> For example:
> Version 2.4.0:   V0
> Version 2.5.0:   V0, V1  (V1 processor must also handle V0).
> Version 2.6.0:   V0?, V1, V2  (Can we now drop support for V0? What happens
> if someone upgrades from 2.4.0 to 2.6.0 directly and it's not supported?)
>
> *3) Does the RHS 1:1 processor, which I currently call
> `ForeignKeySingleLookupProcessorSupplier`, basically end up looking like
> this?*
> if (value.version == Version.V0)
>   //do V0 processing
> else if (value.version == Version.V1)
>   //do V1 processing
> else if (value.version == Version.V2)
>   //do V2 processing
> ....
>
> The tricky part becomes keeping all the Instructions straight for each
> Version. For instance, one option (*OPTION-A*) is:
> //Version = 2.4.0
> enum Instruction {  A, B, C; }
>
> //Version = 2.5.0
> enum Instruction {  A, B, C, //Added in 2.4.0, Value.Version = V0
>                                D; //Added in 2.5.0, Value.Version = V1.
> Also uses Value.Version = V0 instructions.
> }
>
> //Version = 2.6.0
> enum Instruction{  A, B, C, //Added in 2.4.0, Value.Version = V0. *Don't
> use for V2*
>                                D, //Added in 2.5.0, Value.Version = V1.
> Also uses Value.Version = V0 instructions. *Don't use for V2*
>                                E,F,G,H,I,J; //Added in 2.6.0, Value.Version
> = V2.
> }
>
> Alternatively, something like this (*OPTION-B*), where the Version and the
> Instruction are tied together in the Instruction itself.
>
> enum Instruction{  V0_A, V0_B, V0_C, //Added in 2.4.0
>                               V1_A, V1_B, V1_C, V1_D, //Added in 2.5.0
>                               V2_E, V2_F, V2_G, V2_H, V2_I;  //Added in
> 2.6.0
> }
> At this point our logic in the `ForeignKeySingleLookupProcessorSupplier`
> looks something like this:
> if (value.version == Version.V0) {
>     if (value.instruction == Instruction.V0_A) ...
>     else if (value.instruction == Instruction.V0_B) ...
>     else if (value.instruction == Instruction.V0_C) ...
>     else ...
> } else if (value.version == Version.V1) {
>     if (value.instruction == Instruction.V1_A) ...
>     else if (value.instruction == Instruction.V1_B) ...
>     else if (value.instruction == Instruction.V1_C) ...
>     else if (value.instruction == Instruction.V1_D) ...
>     else ...
> } else if ...
>
> I prefer option B because Instruction meaning can change between versions,
> especially in scenarios where we may be reworking, say, 2 instructions into
> 4 instructions that aren't neat subsets of the original 2.
>
>
> *4) If we hard-stop on incompatible events (say we don't support that
> version), how does the user go about handling the upgrade? *
> We can't ignore the events as it would ruin the delivery guarantees. At
> this point it seems to me that they would have to do a full streams reset
> for that applicationId. Am I incorrect in this?
>
>
> Adam
>
>
> On Fri, Jun 28, 2019 at 9:19 AM Adam Bellemare <ad...@gmail.com>
> wrote:
>
> > 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 unnecessary tombstone is (that I can
> > tell...) when we do the following:
> > RHS:
> > (1, bar)
> >
> > LHS
> > (K,1)  -> Results in (K, 1, bar) being output
> > (K,1) -> (K,2) ->  Results in (K, null) being output for INNER (no
> > matching element on LHS)
> > (K,2) -> (K,3) ->  Results in (K, null) being output for INNER (because we
> > don't maintain state to know we already output the tombstone on the
> > previous transition).
> > (K,2) -> (K,9000) ->  Results in (K, null)... etc.
> >
> > Byte versioning is going in today, then I hope to get back to addressing a
> > number of John's previous questions in the PR.
> >
> > Adam
> >
> >
> > On Thu, Jun 27, 2019 at 5:47 PM Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> >> 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 unnecessary tombstone? Seems I a missing case?
> >>
> >> Also for my own understanding: the `instruction` is only part of the
> >> message? It is no necessary to store it in the RHS auxiliary store, right?
> >>
> >> About right/full-outer joins. Agreed. Getting left-joins would be awesome!
> >>
> >> About upgrading: Good call John! Adding a version byte for subscription
> >> and response is good forward thinking. I personally prefer version
> >> numbers, too, as they carry more information.
> >>
> >> Thanks for all the hard to everybody involved!
> >>
> >>
> >> -Matthias
> >>
> >> On 6/27/19 1:44 PM, John Roesler wrote:
> >> > 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 revisions,
> >> > we can just use one byte. The worst case is that we run out of numbers
> >> > and reserve the last one to mean, "consult another field for the
> >> > actual version number". It seems like a single byte on each message
> >> > isn't too much to pay.
> >> >
> >> > Since you point it out, we might as well put a version number on the
> >> > SubscriptionResponseWrapper as well. It may not be needed, but if we
> >> > ever need it, even just once, we'll be glad we have it.
> >> >
> >> > Regarding the instructions field, we can also serialize the enum very
> >> > compactly as a single byte (which is the same size a boolean takes
> >> > anyway), so it seems like an Enum in Java-land and a byte on the wire
> >> > is a good choice.
> >> >
> >> > Agreed on the right and full outer joins, it doesn't seem necessary
> >> > right now, although I am happy to see the left join "join" the party,
> >> > since as you said, we were so close to it anyway. Can you also add it
> >> > to the KIP?
> >> >
> >> > Thanks as always for your awesome efforts on this,
> >> > -John
> >> >
> >> > On Thu, Jun 27, 2019 at 3:04 PM Adam Bellemare <
> >> adam.bellemare@gmail.com> wrote:
> >> >>
> >> >> 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 microservices - there
> >> are
> >> >> indeed cases where a breaking change must be made, and forward
> >> >> compatibility will provide us with no out other than requiring a full
> >> stop
> >> >> and full upgrade for all nodes, shifting us back towards STRATEGY 1.
> >> >>
> >> >> My preference is STRATEGY 1 with instructions as an ENUM, and we can
> >> >> certainly include a version. Would it make sense to include a version
> >> >> number in  SubscriptionResponseWrapper as well? Currently we don't
> >> have any
> >> >> instructions in there, as I removed the boolean, but it is certainly
> >> >> plausible that it could happen in the future. I don't *think* we'll
> >> need
> >> >> it, but I also didn't think we'd need it for SubscriptionWrapper and
> >> here
> >> >> we are.
> >> >>
> >> >> Thanks for the thoughts, and the info on the right-key. That was
> >> >> enlightening, though I can't think of a use-case for it *at this point
> >> in
> >> >> time*. :)
> >> >>
> >> >> Adam
> >> >>
> >> >>
> >> >>
> >> >> On Thu, Jun 27, 2019 at 12:29 PM John Roesler <jo...@confluent.io>
> >> wrote:
> >> >>
> >> >>> 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 `nullKey: (null, rhsValue)`, but we
> >> >>> don't currently allow null keys in Streams. It actually is possible to
> >> >>> define them, and therefore to add right- and full-outer foreign-key
> >> >>> joins later, but it's non-trivial in a streaming context with
> >> >>> continuously updated results. (See the PS if you're curious what I'm
> >> >>> thinking). You're correct, right- and full-outer joins are trivial on
> >> >>> our current 1:1 table joins because they are equi-joins.
> >> >>>
> >> >>> Regarding the transition, it sounds like what you're proposing is that
> >> >>> we would say, "adding a foreign-key join to your topology requires a
> >> >>> full application reset (or a new application id)". This is also an
> >> >>> acceptable constraint to place on the feature, but not strictly
> >> >>> necessary. Since 2.3, it's now possible to give all the state in your
> >> >>> application stable names. This means that it's no longer true that
> >> >>> adding a node to your topology graph would break its structure, and it
> >> >>> does become possible to add new operators and simply restart the app.
> >> >>> Revisiting my prior thought, though, I think the problem is not
> >> >>> specific to your feature. For example, adding a new grouped
> >> >>> aggregation would produce a new repartition topic, but the repartition
> >> >>> topic partitions might get assigned to old nodes in the middle of a
> >> >>> rolling bounce, and they would need to just ignore them. This
> >> >>> requirement is the same for the repartition topics in the FK join, so
> >> >>> it's orthogonal to your design.
> >> >>>
> >> >>> Back to the first concern, though, I'm not sure I followed the
> >> >>> explanation. As a thought experiment, let's imagine that Joe hadn't
> >> >>> taken the time to experiment with your feature branch. We wouldn't
> >> >>> have noticed the problem until the feature was already released in
> >> >>> 2.4. So the wire protocol on that PK->FK subscription topic would have
> >> >>> been V1: "FK,PK,HASH,BOOLEAN". Then, Joe would have let us know the
> >> >>> problem once they picked up the feature, so we would want to implement
> >> >>> your proposed fix and change the wire protocol to V2:
> >> >>> "FK,PK,HASH,INSTRUCTIONS" in 2.5. Upon rolling out the update, we
> >> >>> would see both 2.4 nodes encountering V2 messages and 2.5 nodes
> >> >>> encountering V1 messages. How can they both detect that they are
> >> >>> attempting to process a newer or older protocol? If they can detect
> >> >>> it, then what should they do?
> >> >>>
> >> >>> From experience, there are two basic solutions to this problem:
> >> >>>
> >> >>> STRATEGY1. Add a protocol version to the message (could be a number at
> >> >>> the start of the message payload, or it could be a number in the
> >> >>> message headers, not sure if it matters much. Payload is probably more
> >> >>> compact, since the header would need a name.) In this case, the 2.4
> >> >>> worker would know that it's max protocol version is V1, and when it
> >> >>> sees the V2 message, it knows that it can't handle it properly. Rather
> >> >>> than doing something wrong, it would just not do anything. This means
> >> >>> it would stop the task, if not shut down the whole instance. On the
> >> >>> other hand, a 2.5 worker would have some defined logic for how to
> >> >>> handle all versions (V1 and V2), so once the upgrade is complete, all
> >> >>> messages can be processed.
> >> >>>
> >> >>> STRATEGY2. Make the schema forward-compatible. Basically, we ensure
> >> >>> that new fields can only be appended to the message schema, and that
> >> >>> older workers using only a prefix of the full message would still
> >> >>> behave correctly. Using the example above, we'd instead evolve the
> >> >>> schema to V2': "FK,PK,HASH,BOOLEAN,INSTRUCTIONS", and continue to set
> >> >>> the boolean field to true for the "new" foreign key. Then, 2.4 workers
> >> >>> encountering the a "new FK" message would just see the prefix of the
> >> >>> payload that makes sense to them, and they would still continue
> >> >>> processing the messages as they always have. Only after the 2.5 code
> >> >>> is fully rolled out to the cluster would we be sure to see the desired
> >> >>> behavior. Note: in the reverse case, a 2.5 worker knows how to fully
> >> >>> parse the new message format, even if it plans to ignore the BOOLEAN
> >> >>> field.
> >> >>>
> >> >>> There are some tradeoffs between these strategies: STRATEGY1 ensures
> >> >>> that all messages are only handled by workers that can properly handle
> >> >>> them, although it results in processing stalls while there are still
> >> >>> old nodes in the cluster. STRATEGY2 ensures that all messages can be
> >> >>> processed by all nodes, so there are no stalls, but we can never
> >> >>> remove fields from the message, so if there are a lot of revisions in
> >> >>> the future, the payloads will become bloated. Also, it's not clear
> >> >>> that you can actually pull off STRATEGY2 in all cases. If there's some
> >> >>> new kind of message you want to send that has no way to be correctly
> >> >>> processed at all under the 2.4 code paths, the prefix thing simply
> >> >>> doesn't work. Etc.
> >> >>>
> >> >>> Also, note that you can modify the above strategies by instead
> >> >>> designing the message fields for extensibility. E.g., if you make the
> >> >>> instructions field an enum, then you can make sure that the default
> >> >>> case is handled sensibly (probably similarly to STRATEGY1, just choke
> >> >>> on unknown instructions) and that you never remove an instruction type
> >> >>> from the enum in future versions.
> >> >>>
> >> >>> Does this make sense?
> >> >>> -John
> >> >>>
> >> >>>
> >> >>>
> >> >>>
> >> >>> PS:
> >> >>> We can define null keys for streaming tables, but it's tricky.
> >> >>>
> >> >>> Specifically, you'd want to define some concept of null keys that
> >> >>> allows all null keys to be unique, but _also_ to have a fixed
> >> >>> identity, so that a particular null-key can be updated later. One
> >> >>> example could be to union the existing keyspace with a new
> >> >>> null-keyspace, where normal keys are like "key" and null-keys are like
> >> >>> "null(identity)". Then given a query like
> >> >>> "KTable<String,Integer>.rightJoin(KTable<Integer,Boolean>)", and
> >> >>> inputs like:
> >> >>> LHS:
> >> >>> "a": 1
> >> >>> "b": 2
> >> >>>
> >> >>> RHS:
> >> >>> 1: true
> >> >>> 3: false
> >> >>>
> >> >>> a full outer join would produce:
> >> >>> "a": (1, true)
> >> >>> "b": (2, null)
> >> >>> null(3): (null, false)
> >> >>>
> >> >>> which can be correctly updated later if we get an update on the LHS:
> >> >>> PUT("c": 3)
> >> >>>
> >> >>> We'd emit for the results:
> >> >>> DELETE(null(e))
> >> >>> EMIT("c": (3, false))
> >> >>>
> >> >>> Resulting in the correct result table of:
> >> >>> "a": (1, true)
> >> >>> "b": (2, null)
> >> >>> "c": (3, false)
> >> >>>
> >> >>> As mentioned, this is tricky, and I would avoid it until we have
> >> >>> evidence that it's actually useful to cover this part of the design
> >> >>> space. Certainly, it would be a separate KIP if it came to that.
> >> >>>
> >> >>> On Wed, Jun 26, 2019 at 8:57 PM Adam Bellemare <
> >> adam.bellemare@gmail.com>
> >> >>> wrote:
> >> >>>>
> >> >>>> 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
> >> the
> >> >>> last
> >> >>>> commit (about 10 min ago) along with INNER join. I do not think that
> >> >>> RIGHT
> >> >>>> join and OUTER are possible given that there is no LHS key
> >> available, so
> >> >>>> LHSTable.outerJoinOnForeignKey(RHSTable) wouldn't even make sense.
> >> This
> >> >>> is
> >> >>>> in contrast to the current LHSTable.outerJoin(RHSTable), as they are
> >> both
> >> >>>> keyed on the same key. I have buffed up the Integration tests and
> >> have
> >> >>>> tried to make them more readable to ensure that we're covering all
> >> the
> >> >>>> scenarios. I think that if we can get more eyes on the workflow
> >> showing
> >> >>> the
> >> >>>> various LHS and RHS events and outputs then that may help us validate
> >> >>> that
> >> >>>> we have all the scenarios covered.
> >> >>>>
> >> >>>> With regards to the 2.3->2.4 scenario you described, I'm not entirely
> >> >>> sure
> >> >>>> I follow. If they want to add a FK-join, they will need to rework
> >> their
> >> >>>> code in the KStreams app and make a new release, since the underlying
> >> >>>> topology would be different and new internal topics would need to be
> >> >>>> created. In other words, I don't think a rolling upgrade where the
> >> user
> >> >>>> introduces a FK join would be possible since their topology would
> >> >>>> necessitate a full KStreams reset. Is this what you meant?
> >> >>>>
> >> >>>>
> >> >>>>
> >> >>>> On Wed, Jun 26, 2019 at 4:10 PM John Roesler <jo...@confluent.io>
> >> wrote:
> >> >>>>
> >> >>>>> 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.bellemare@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.bellemare@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 <jo...@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
> >> >>>>>>>> <ad...@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 <
> >> >>> john@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
> >> >>>>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>
> >> >>>>>
> >> >>>
> >>
> >>

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

Posted by Adam Bellemare <ad...@gmail.com>.
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 doable, but may become more complex as versions go on. See 2 & 3
below.*

public enum Version {
  V0((byte) 0x00),  //2.4.0
  V1((byte) 0x01);  //2.5.0
}

*2) Keeping track of compatibility is a bit tricky. For instance, how do we
know which messages are compatible and which are breaking? Which upgrade
paths do we handle and which ones do we not?  How long do we handle support
for old message versions? 2 minor releases? 1 major release?*

For example:
Version 2.4.0:   V0
Version 2.5.0:   V0, V1  (V1 processor must also handle V0).
Version 2.6.0:   V0?, V1, V2  (Can we now drop support for V0? What happens
if someone upgrades from 2.4.0 to 2.6.0 directly and it's not supported?)

*3) Does the RHS 1:1 processor, which I currently call
`ForeignKeySingleLookupProcessorSupplier`, basically end up looking like
this?*
if (value.version == Version.V0)
  //do V0 processing
else if (value.version == Version.V1)
  //do V1 processing
else if (value.version == Version.V2)
  //do V2 processing
....

The tricky part becomes keeping all the Instructions straight for each
Version. For instance, one option (*OPTION-A*) is:
//Version = 2.4.0
enum Instruction {  A, B, C; }

//Version = 2.5.0
enum Instruction {  A, B, C, //Added in 2.4.0, Value.Version = V0
                               D; //Added in 2.5.0, Value.Version = V1.
Also uses Value.Version = V0 instructions.
}

//Version = 2.6.0
enum Instruction{  A, B, C, //Added in 2.4.0, Value.Version = V0. *Don't
use for V2*
                               D, //Added in 2.5.0, Value.Version = V1.
Also uses Value.Version = V0 instructions. *Don't use for V2*
                               E,F,G,H,I,J; //Added in 2.6.0, Value.Version
= V2.
}

Alternatively, something like this (*OPTION-B*), where the Version and the
Instruction are tied together in the Instruction itself.

enum Instruction{  V0_A, V0_B, V0_C, //Added in 2.4.0
                              V1_A, V1_B, V1_C, V1_D, //Added in 2.5.0
                              V2_E, V2_F, V2_G, V2_H, V2_I;  //Added in
2.6.0
}
At this point our logic in the `ForeignKeySingleLookupProcessorSupplier`
looks something like this:
if (value.version == Version.V0) {
    if (value.instruction == Instruction.V0_A) ...
    else if (value.instruction == Instruction.V0_B) ...
    else if (value.instruction == Instruction.V0_C) ...
    else ...
} else if (value.version == Version.V1) {
    if (value.instruction == Instruction.V1_A) ...
    else if (value.instruction == Instruction.V1_B) ...
    else if (value.instruction == Instruction.V1_C) ...
    else if (value.instruction == Instruction.V1_D) ...
    else ...
} else if ...

I prefer option B because Instruction meaning can change between versions,
especially in scenarios where we may be reworking, say, 2 instructions into
4 instructions that aren't neat subsets of the original 2.


*4) If we hard-stop on incompatible events (say we don't support that
version), how does the user go about handling the upgrade? *
We can't ignore the events as it would ruin the delivery guarantees. At
this point it seems to me that they would have to do a full streams reset
for that applicationId. Am I incorrect in this?


Adam


On Fri, Jun 28, 2019 at 9:19 AM Adam Bellemare <ad...@gmail.com>
wrote:

> 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 unnecessary tombstone is (that I can
> tell...) when we do the following:
> RHS:
> (1, bar)
>
> LHS
> (K,1)  -> Results in (K, 1, bar) being output
> (K,1) -> (K,2) ->  Results in (K, null) being output for INNER (no
> matching element on LHS)
> (K,2) -> (K,3) ->  Results in (K, null) being output for INNER (because we
> don't maintain state to know we already output the tombstone on the
> previous transition).
> (K,2) -> (K,9000) ->  Results in (K, null)... etc.
>
> Byte versioning is going in today, then I hope to get back to addressing a
> number of John's previous questions in the PR.
>
> Adam
>
>
> On Thu, Jun 27, 2019 at 5:47 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
>
>> 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 unnecessary tombstone? Seems I a missing case?
>>
>> Also for my own understanding: the `instruction` is only part of the
>> message? It is no necessary to store it in the RHS auxiliary store, right?
>>
>> About right/full-outer joins. Agreed. Getting left-joins would be awesome!
>>
>> About upgrading: Good call John! Adding a version byte for subscription
>> and response is good forward thinking. I personally prefer version
>> numbers, too, as they carry more information.
>>
>> Thanks for all the hard to everybody involved!
>>
>>
>> -Matthias
>>
>> On 6/27/19 1:44 PM, John Roesler wrote:
>> > 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 revisions,
>> > we can just use one byte. The worst case is that we run out of numbers
>> > and reserve the last one to mean, "consult another field for the
>> > actual version number". It seems like a single byte on each message
>> > isn't too much to pay.
>> >
>> > Since you point it out, we might as well put a version number on the
>> > SubscriptionResponseWrapper as well. It may not be needed, but if we
>> > ever need it, even just once, we'll be glad we have it.
>> >
>> > Regarding the instructions field, we can also serialize the enum very
>> > compactly as a single byte (which is the same size a boolean takes
>> > anyway), so it seems like an Enum in Java-land and a byte on the wire
>> > is a good choice.
>> >
>> > Agreed on the right and full outer joins, it doesn't seem necessary
>> > right now, although I am happy to see the left join "join" the party,
>> > since as you said, we were so close to it anyway. Can you also add it
>> > to the KIP?
>> >
>> > Thanks as always for your awesome efforts on this,
>> > -John
>> >
>> > On Thu, Jun 27, 2019 at 3:04 PM Adam Bellemare <
>> adam.bellemare@gmail.com> wrote:
>> >>
>> >> 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 microservices - there
>> are
>> >> indeed cases where a breaking change must be made, and forward
>> >> compatibility will provide us with no out other than requiring a full
>> stop
>> >> and full upgrade for all nodes, shifting us back towards STRATEGY 1.
>> >>
>> >> My preference is STRATEGY 1 with instructions as an ENUM, and we can
>> >> certainly include a version. Would it make sense to include a version
>> >> number in  SubscriptionResponseWrapper as well? Currently we don't
>> have any
>> >> instructions in there, as I removed the boolean, but it is certainly
>> >> plausible that it could happen in the future. I don't *think* we'll
>> need
>> >> it, but I also didn't think we'd need it for SubscriptionWrapper and
>> here
>> >> we are.
>> >>
>> >> Thanks for the thoughts, and the info on the right-key. That was
>> >> enlightening, though I can't think of a use-case for it *at this point
>> in
>> >> time*. :)
>> >>
>> >> Adam
>> >>
>> >>
>> >>
>> >> On Thu, Jun 27, 2019 at 12:29 PM John Roesler <jo...@confluent.io>
>> wrote:
>> >>
>> >>> 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 `nullKey: (null, rhsValue)`, but we
>> >>> don't currently allow null keys in Streams. It actually is possible to
>> >>> define them, and therefore to add right- and full-outer foreign-key
>> >>> joins later, but it's non-trivial in a streaming context with
>> >>> continuously updated results. (See the PS if you're curious what I'm
>> >>> thinking). You're correct, right- and full-outer joins are trivial on
>> >>> our current 1:1 table joins because they are equi-joins.
>> >>>
>> >>> Regarding the transition, it sounds like what you're proposing is that
>> >>> we would say, "adding a foreign-key join to your topology requires a
>> >>> full application reset (or a new application id)". This is also an
>> >>> acceptable constraint to place on the feature, but not strictly
>> >>> necessary. Since 2.3, it's now possible to give all the state in your
>> >>> application stable names. This means that it's no longer true that
>> >>> adding a node to your topology graph would break its structure, and it
>> >>> does become possible to add new operators and simply restart the app.
>> >>> Revisiting my prior thought, though, I think the problem is not
>> >>> specific to your feature. For example, adding a new grouped
>> >>> aggregation would produce a new repartition topic, but the repartition
>> >>> topic partitions might get assigned to old nodes in the middle of a
>> >>> rolling bounce, and they would need to just ignore them. This
>> >>> requirement is the same for the repartition topics in the FK join, so
>> >>> it's orthogonal to your design.
>> >>>
>> >>> Back to the first concern, though, I'm not sure I followed the
>> >>> explanation. As a thought experiment, let's imagine that Joe hadn't
>> >>> taken the time to experiment with your feature branch. We wouldn't
>> >>> have noticed the problem until the feature was already released in
>> >>> 2.4. So the wire protocol on that PK->FK subscription topic would have
>> >>> been V1: "FK,PK,HASH,BOOLEAN". Then, Joe would have let us know the
>> >>> problem once they picked up the feature, so we would want to implement
>> >>> your proposed fix and change the wire protocol to V2:
>> >>> "FK,PK,HASH,INSTRUCTIONS" in 2.5. Upon rolling out the update, we
>> >>> would see both 2.4 nodes encountering V2 messages and 2.5 nodes
>> >>> encountering V1 messages. How can they both detect that they are
>> >>> attempting to process a newer or older protocol? If they can detect
>> >>> it, then what should they do?
>> >>>
>> >>> From experience, there are two basic solutions to this problem:
>> >>>
>> >>> STRATEGY1. Add a protocol version to the message (could be a number at
>> >>> the start of the message payload, or it could be a number in the
>> >>> message headers, not sure if it matters much. Payload is probably more
>> >>> compact, since the header would need a name.) In this case, the 2.4
>> >>> worker would know that it's max protocol version is V1, and when it
>> >>> sees the V2 message, it knows that it can't handle it properly. Rather
>> >>> than doing something wrong, it would just not do anything. This means
>> >>> it would stop the task, if not shut down the whole instance. On the
>> >>> other hand, a 2.5 worker would have some defined logic for how to
>> >>> handle all versions (V1 and V2), so once the upgrade is complete, all
>> >>> messages can be processed.
>> >>>
>> >>> STRATEGY2. Make the schema forward-compatible. Basically, we ensure
>> >>> that new fields can only be appended to the message schema, and that
>> >>> older workers using only a prefix of the full message would still
>> >>> behave correctly. Using the example above, we'd instead evolve the
>> >>> schema to V2': "FK,PK,HASH,BOOLEAN,INSTRUCTIONS", and continue to set
>> >>> the boolean field to true for the "new" foreign key. Then, 2.4 workers
>> >>> encountering the a "new FK" message would just see the prefix of the
>> >>> payload that makes sense to them, and they would still continue
>> >>> processing the messages as they always have. Only after the 2.5 code
>> >>> is fully rolled out to the cluster would we be sure to see the desired
>> >>> behavior. Note: in the reverse case, a 2.5 worker knows how to fully
>> >>> parse the new message format, even if it plans to ignore the BOOLEAN
>> >>> field.
>> >>>
>> >>> There are some tradeoffs between these strategies: STRATEGY1 ensures
>> >>> that all messages are only handled by workers that can properly handle
>> >>> them, although it results in processing stalls while there are still
>> >>> old nodes in the cluster. STRATEGY2 ensures that all messages can be
>> >>> processed by all nodes, so there are no stalls, but we can never
>> >>> remove fields from the message, so if there are a lot of revisions in
>> >>> the future, the payloads will become bloated. Also, it's not clear
>> >>> that you can actually pull off STRATEGY2 in all cases. If there's some
>> >>> new kind of message you want to send that has no way to be correctly
>> >>> processed at all under the 2.4 code paths, the prefix thing simply
>> >>> doesn't work. Etc.
>> >>>
>> >>> Also, note that you can modify the above strategies by instead
>> >>> designing the message fields for extensibility. E.g., if you make the
>> >>> instructions field an enum, then you can make sure that the default
>> >>> case is handled sensibly (probably similarly to STRATEGY1, just choke
>> >>> on unknown instructions) and that you never remove an instruction type
>> >>> from the enum in future versions.
>> >>>
>> >>> Does this make sense?
>> >>> -John
>> >>>
>> >>>
>> >>>
>> >>>
>> >>> PS:
>> >>> We can define null keys for streaming tables, but it's tricky.
>> >>>
>> >>> Specifically, you'd want to define some concept of null keys that
>> >>> allows all null keys to be unique, but _also_ to have a fixed
>> >>> identity, so that a particular null-key can be updated later. One
>> >>> example could be to union the existing keyspace with a new
>> >>> null-keyspace, where normal keys are like "key" and null-keys are like
>> >>> "null(identity)". Then given a query like
>> >>> "KTable<String,Integer>.rightJoin(KTable<Integer,Boolean>)", and
>> >>> inputs like:
>> >>> LHS:
>> >>> "a": 1
>> >>> "b": 2
>> >>>
>> >>> RHS:
>> >>> 1: true
>> >>> 3: false
>> >>>
>> >>> a full outer join would produce:
>> >>> "a": (1, true)
>> >>> "b": (2, null)
>> >>> null(3): (null, false)
>> >>>
>> >>> which can be correctly updated later if we get an update on the LHS:
>> >>> PUT("c": 3)
>> >>>
>> >>> We'd emit for the results:
>> >>> DELETE(null(e))
>> >>> EMIT("c": (3, false))
>> >>>
>> >>> Resulting in the correct result table of:
>> >>> "a": (1, true)
>> >>> "b": (2, null)
>> >>> "c": (3, false)
>> >>>
>> >>> As mentioned, this is tricky, and I would avoid it until we have
>> >>> evidence that it's actually useful to cover this part of the design
>> >>> space. Certainly, it would be a separate KIP if it came to that.
>> >>>
>> >>> On Wed, Jun 26, 2019 at 8:57 PM Adam Bellemare <
>> adam.bellemare@gmail.com>
>> >>> wrote:
>> >>>>
>> >>>> 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
>> the
>> >>> last
>> >>>> commit (about 10 min ago) along with INNER join. I do not think that
>> >>> RIGHT
>> >>>> join and OUTER are possible given that there is no LHS key
>> available, so
>> >>>> LHSTable.outerJoinOnForeignKey(RHSTable) wouldn't even make sense.
>> This
>> >>> is
>> >>>> in contrast to the current LHSTable.outerJoin(RHSTable), as they are
>> both
>> >>>> keyed on the same key. I have buffed up the Integration tests and
>> have
>> >>>> tried to make them more readable to ensure that we're covering all
>> the
>> >>>> scenarios. I think that if we can get more eyes on the workflow
>> showing
>> >>> the
>> >>>> various LHS and RHS events and outputs then that may help us validate
>> >>> that
>> >>>> we have all the scenarios covered.
>> >>>>
>> >>>> With regards to the 2.3->2.4 scenario you described, I'm not entirely
>> >>> sure
>> >>>> I follow. If they want to add a FK-join, they will need to rework
>> their
>> >>>> code in the KStreams app and make a new release, since the underlying
>> >>>> topology would be different and new internal topics would need to be
>> >>>> created. In other words, I don't think a rolling upgrade where the
>> user
>> >>>> introduces a FK join would be possible since their topology would
>> >>>> necessitate a full KStreams reset. Is this what you meant?
>> >>>>
>> >>>>
>> >>>>
>> >>>> On Wed, Jun 26, 2019 at 4:10 PM John Roesler <jo...@confluent.io>
>> wrote:
>> >>>>
>> >>>>> 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.bellemare@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.bellemare@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 <jo...@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
>> >>>>>>>> <ad...@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 <
>> >>> john@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
>> >>>>>>>>>>
>> >>>>>>>>
>> >>>>>>>
>> >>>>>
>> >>>
>>
>>

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
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 <ma...@confluent.io> - 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 frankly I don't have any better ideas at the moment.
> 
> Unless you strongly object, I think we'll have to move forward with it
> as-is. There is still time to come up with another solution before I
> *hopefully* get this into 2.4, but in the meantime I'll look to continue
> on otherwise.
> 
> Adam
> 
> 
> On Thu, Jul 11, 2019 at 9:57 AM Jan Filipiak <Jan.Filipiak@trivago.com
> <ma...@trivago.com>> wrote:
> 
> 
> 
>     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 experience from our usage of our internal
>     > implementation at my company, and that of many others who have
>     reached out
>     > to me.
> 
>     backing this.
> 


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

Posted by Adam Bellemare <ad...@gmail.com>.
@Matthias J. Sax <ma...@confluent.io> - 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 frankly I don't have any better ideas at the moment.

Unless you strongly object, I think we'll have to move forward with it
as-is. There is still time to come up with another solution before I
*hopefully* get this into 2.4, but in the meantime I'll look to continue on
otherwise.

Adam


On Thu, Jul 11, 2019 at 9:57 AM Jan Filipiak <Ja...@trivago.com>
wrote:

>
>
> 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 experience from our usage of our internal
> > implementation at my company, and that of many others who have reached
> out
> > to me.
>
> backing this.
>
>

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

Posted by Jan Filipiak <Ja...@trivago.com>.

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 experience from our usage of our internal
> implementation at my company, and that of many others who have reached out
> to me.

backing this.


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

Posted by Adam Bellemare <ad...@gmail.com>.
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 potential performance impact that is
an issue, and all the solutions proposed so far will have non-trivial
performance impacts for the sake of trying to reduce performance impacts...

I think that at this point, unless someone has an example of where it would
be semantically incorrect that we have overlooked, or a use case where the
extra tombstones would be extremely problematic, it's probably best to
simply accept the additional *broken* FK
reference to another *broken* FK reference tombstones.

Adam

On Wed, Jul 10, 2019 at 11:59 AM John Roesler <jo...@confluent.io> wrote:

> 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 join result would be:
> LEFT: (k -> (left-val, null))
> INNER: (k -> null)
>
> So, the tombstone result doesn't somehow make the inner join result
> "left-join-like". Instead we're saying, "delete the join result for k
> from the post-join view". It may be unnecessary to "delete" a
> non-existant record from a view, but it's never incorrect. The issue
> we're discussing is an operational one (whether the unnecessary
> tombstones are a performance problem), not a semantic one.
>
> Second, just to be clear, we have no obligation under current Streams
> semantics to eliminate all unnecessary updates. We make no promise
> whatsoever to "emit on change", so anyone consuming the output of a
> Streams table operation should _expect_ to see unnecessary (aka
> "duplicate") updates. We _can_ drop unnecessary updates (and do, when
> the opportunity presents itself) to save on performance, though.
>
> -----
> Since it's clear that we're _only_ discussing a performance
> optimization, we should try to characterize the actual performance
> problem so that we can weigh the solution against the benefit it
> yields.
>
> Unless I'm mistaken, the exact problem we're discussing with the most
> recent version of the proposal is that we'll emit an unnecessary
> tombstone for inner joins or an unnecessary (left,null) result for
> left joins specifically when a record transitions from one *broken* FK
> reference to another *broken* FK reference. Any other transition
> (valid FK -> valid FK, valid FK -> broken FK, or broken FK -> valid
> FK) would yield the correct result (a necessary join result or a
> necessary tombstone).
>
> How likely are these transitions to occur in practice? We should
> consider both the likelihood of FK changes at all and the likelihood
> of broken FK references.
>
> Probability of FK change:
> It seems generally rare for records to change FK references at all.
> How often to employees change departments, or comments change
> articles, or people change addresses, etc, etc.? If someone has
> anecdotal experience with a domain involving multiple, rapid FK
> updates, please share them, but in all the domains I have experience
> with, FK reference changes are _much_ less common than data updates. I
> do happen to know that FK updates are moderately common in
> Bazaarvoice's data model, but that they're orders of magnitude less
> frequent than other kinds of updates.
>
> Probability of broken reference:
> Tables represented by Kafka topics (obviously) can't enforce
> referential integrity, but (spitballing here) upstream databases can
> and probably do. How often are we dealing with employees who work for
> a manager who isn't in the HR database? Or a comment on a non-existent
> article? Or someone who lives at a non-existent address? Again, this
> is anecdotal, but that's the whole point of a likelihood estimation.
> Please share your experiences. Again, I happen to know that at
> Bazaarvoice, the incoming data actually doesn't guarantee referential
> integrity, but that broken references are still relatively rare (and
> actually, as it happens, it's not possible to change the FK without
> first repairing a broken reference, so the broken -> broken transition
> would be exceedingly rare).
>
> If we can assume that the upstream data _mostly_ maintains referential
> integrity, then we can make a further assumption that when we observe
> a broken reference, it is probably transient. I.e., we can expect the
> reference to be repaired "pretty soon". If this is the case, then the
> probability we get a FK change from one broken reference to another
> broken reference before the first reference gets repaired should be
> quite rare for "normal" data sets.
>
> Caveat: https://www.xkcd.com/2167/
>
> -----
>
> So, under the assumption that these unnecessary tombstones are rare,
> and with the understanding that they're semantically ok to emit, it
> really doesn't seem worthwhile to take on all the extra complexity
> proposed earlier in this conversation.
>
> A much simpler solution, if someone is really struggling with extra
> tombstones, would be just be to slap an LRU cache on the result table
> and drop unnecessary tombstones that way.
>
> Again, just my personal thoughts, FWIW...
> -John
>
> On Tue, Jul 9, 2019 at 11:25 PM Adam Bellemare <ad...@gmail.com>
> wrote:
> >
> > 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-handed.
> >
> > 2) A very deft solution is to also just support left-joins but not
> > inner-joins. Practically speaking, either INNER or LEFT join as it
> > currently is would support all of my use-cases.
> >
> > 3) Accept that there may be some null tombstones (though this makes me
> want
> > to just go with LEFT only instead of LEFT and PSEUDO-INNER).
> >
> > 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 experience from our usage of our internal
> > implementation at my company, and that of many others who have reached
> out
> > to me.
> >
> > What would help most at this point is if someone can come up with a
> > scenario where sending unnecessary tombstones actually poses a downstream
> > problem beyond that of confusing behaviour, as I cannot think of one
> > myself.  With that being said, I am far more inclined to actually then
> > support just option #2 above and only have LEFT joins, forgoing INNER
> > completely since it would not be a true inner join.
> >
> > Adam
> >
> >
> >
> >
> >
> >
> >
> >
> > On Thu, Jul 4, 2019 at 8:50 AM Adam Bellemare <ad...@gmail.com>
> > wrote:
> >
> > > 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,hashOf(b),PROPAGATE_OLD_VALUE) hits RHS-0, the
> following
> > > occurs:
> > >   1) Store the current (CombinedKey<FK,K>, Value=(Hash, ForeignValue))
> in
> > > a variable
> > >   2) Delete the key from the store
> > >   3) Publish the event from step A-1 downstream with an instruction:
> > > (eventType = COMPARE_TO_OTHER) (or whatever)
> > > *      (key, (hashOf(b),wasForeignValueNull, eventType))*
> > >     //Don't need the old hashOf(b) as it is guaranteed to be out of
> date
> > >     //We do need the hashOf(b) that came with the event to be passed
> > > along. Will be used in resolution.
> > >     //Don't need the actual value as we aren't joining or comparing the
> > > values, just using it to determine nulls. Reduces payload size.
> > >
> > > B) When (k,b,PROPAGATE_NEW_VALUE) hits RHS-1, the following occurs:
> > >   1) Store it in the prefix state store (as we currently do)
> > >   2) Get the FK-value (as we currently do)
> > >   3) Return the normal SubscriptionResponse payload (eventType =
> UPDATE)
> > > (or whatever)
> > > *     (key, (hashOf(b), foreignValue, eventType))*
> > >
> > >
> > > C) The Resolver Table is keyed on (as per our example):
> > > key = CombinedKey<k-hash(b)>, value =
> > > NullValueResolution<wasForeignValueNull (set by RHS-0), foreignValue
> (set
> > > by RHS-1)>
> > >
> > > Resolution Steps per event:
> > >
> > > When one of either the output events from A (eventType ==
> > > COMPARE_TO_OTHER) or B (eventType == UPDATE) is received
> > > 1) Check if this event matches the current hashOf(b). If not, discard
> it,
> > > for it is stale and no longer matters.  Additionally, delete entry
> > > CombinedKey<k-hash(b)> from the Resolver Table.
> > >
> > > 2) Lookup event in table on its CombinedKey:
> > >   - If it's not in the table, create the  NullValueResolution value,
> > > populate the field related to the eventType, and add it to the table.
> > >   - If it already IS in the table, get the existing NullValueResolution
> > > object and finish populating it:
> > >
> > > 3) If the NullValueResolution is fully populated, move on to the
> > > resolution logic below.
> > >
> > > Format:
> > > (wasForeignValueNull, foreignValue) -> Result
> > > If:
> > > ( false  , Null ) -> Send tombstone. Old value was not null, new one
> is,
> > > send tombstone.
> > > (  true  , Null ) -> Do nothing.  See * below for more details.
> > > (  true  , NewValue ) -> Send the new result
> > > (  true  , NewValue ) -> Send the new result
> > >
> > > * wasForeignValueNull may have been false at some very recent point,
> but
> > > only just translated to true (race condition). In this case, the RHS
> table
> > > was updated and the value was set to null due to a an RHS update of (a,
> > > oldVal) -> (a, null). This event on its own will propagate a delete
> event
> > > through to the resolver (of a different eventType), so we don't need to
> > > handle this case from the LHS and doing nothing is OK.
> > >
> > > In the case that it's truly (true, Null), we also don't need to send a
> > > tombstone because wasForeignKeyNull == true means that a tombstone was
> > > previously sent.
> > >
> > > 4) Check the hashOf(b) one last time before sending the resolved
> message
> > > out. If the hash is old, discard it.
> > >
> > > 5) Delete the row from the Resolver Table.
> > >
> > >
> > > Takeaways:
> > > 1) I believe this is only necessary for INNER joins when we transition
> > > from (k, non-Null-val) -> (k, non-Null-new-val)
> > > 2) We can maintain state until we get both events back from RHS-0 and
> > > RHS-1, at which point we delete it and clean up.
> > >   NOTE: I think this still works with at-least-once processing, but I
> am
> > > not 100% on this. The concern is that we receive events from RHS-0,
> RHS-1
> > > (causing a row deletion), then RHS-1 again from at-least once
> (creating an
> > > entry that never gets deleted).
> > > 3) My naive implementation requires SubscriptionResponseWrapper event
> > > types.
> > > 4) This shouldn't affect performance much, as it should only be
> > > maxOf(RHS-0, RHS-1) event propagation and processing.
> > >
> > >
> > > Adam
> > >
> > > On Wed, Jul 3, 2019 at 2:18 PM Matthias J. Sax <ma...@confluent.io>
> > > wrote:
> > >
> > >> 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, but want to throw out the idea
> first.
> > >>
> > >> > S-2 could probably be simplified to "for a given key, was the
> previous
> > >> > propagated result a null/tombstone or not?"
> > >>
> > >> Yes, that's the idea.
> > >>
> > >> > However,
> > >> > the table will grow indefinitely large as we can never remove keys
> from
> > >> it.
> > >>
> > >> That a good point. I missed that... Seems that this rules out S-2
> > >> because if we cannot provide _strict_ guarantees, it does not seem to
> be
> > >> worth to even try?
> > >>
> > >>
> > >> About the impact of unnecessary tombstones: There is a performance
> > >> impact, as we increase the output data rate, potentially broker load
> if
> > >> the result is written into a topic, and also increase downstream
> > >> processing load. It's hard to judge, how much the overhead will be, as
> > >> it will depend on the selectivity of the join. But it might be
> > >> significant?
> > >>
> > >> Also, users have certain expectations on the result and it's
> unintuitive
> > >> (even if semantically correct) to send those tombstones. From
> > >> experience, we often have a hard time to explain semantics to people
> and
> > >> I was hoping we could avoid introducing unintuitive behavior.
> > >>
> > >>
> > >> Would be good to get input from others and how they judge the impact.
> I
> > >> think it might be still worth to explore how complex S-1 would be. If
> we
> > >> think it's too complex it might be a good argument to just accept the
> > >> unnecessary tombstones?
> > >>
> > >>
> > >> -Matthias
> > >>
> > >>
> > >> On 7/3/19 8:03 AM, Adam Bellemare wrote:
> > >> > 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 seems to me that the processor would need to
> block
> > >> > while it waits for the unsubscribe to propagate and return, which
> would
> > >> > cause throughput to drop significantly. Alternately, we would need
> to
> > >> > maintain state anyways about which events were sent and which
> responses
> > >> > returned, while being sure to respect the offset order in which
> they're
> > >> > emitted. I think this would only reduce blocking slightly while
> > >> increasing
> > >> > complexity. If I am wrong in understanding this, please let me know
> > >> where
> > >> > my thinking is erroneous.
> > >> >
> > >> > S-2 could probably be simplified to "for a given key, was the
> previous
> > >> > propagated result a null/tombstone or not?"
> > >> > It would act very similarly to the hash value mechanism, where we
> > >> discard
> > >> > any events that are not of the correct hash. In this case, we simply
> > >> store
> > >> > (key, wasLastOutputATombstone) right before the event is output
> > >> downstream
> > >> > of the Join + Resolver. This ignores all the complexities of which
> > >> event is
> > >> > propagating over which wire and simply squelches any extra
> tombstones
> > >> from
> > >> > being sent.
> > >> >
> > >> > For storage, we need to use the full primary key and a boolean.
> However,
> > >> > the table will grow indefinitely large as we can never remove keys
> from
> > >> it.
> > >> > If we delete key=k from the table and propagate a tombstone, but
> later
> > >> (say
> > >> > 3 weeks, 3 months, etc) we publish (k, baz), but baz does not exist
> on
> > >> the
> > >> > RHS, we will end up publishing an extra tombstone because we have no
> > >> idea
> > >> > what the previously sent record was for k. For this reason I think
> it's
> > >> > worth asking if we really can maintain state, and if it's even
> necessary
> > >> > (again, a full understanding of the impact of extra tombstones may
> help
> > >> us
> > >> > figure out a better solution).
> > >> >
> > >> > As it stands, I don't think either of these will work well. That
> being
> > >> > said, I myself do not have any better ideas at the moment, but I
> would
> > >> > still like to better understand circumstances where it has a
> negative
> > >> > impact downstream as that may provide some insights.
> > >> >
> > >> >
> > >> > Thanks
> > >> >
> > >> > Adam
> > >> >
> > >> >
> > >> >
> > >> >
> > >> >
> > >> > On Tue, Jul 2, 2019 at 11:18 PM Matthias J. Sax <
> matthias@confluent.io>
> > >> > wrote:
> > >> >
> > >> >> 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 subscribe message to RHS
> > >> >> a-2) RHS lookup and send result back if there is one
> > >> >> a-3) emit result on LHS if any is returned
> > >> >>
> > >> >> b-1) delete subscription from RHS
> > >> >> b-2) if there was a previous result (can easily be decided by
> looking
> > >> up
> > >> >> RHS table for an existing key), send tombstone back
> > >> >> b-3) emit tombstone on LHS if any is returned
> > >> >>
> > >> >> c-1) delete old subscription from RHS
> > >> >> c-2) send new subscription to RHS
> > >> >> c-3) if there was no previous result and there is no new result
> emit
> > >> >> nothing
> > >> >> c-4) if there was a previous result and there is no new result
> emit a
> > >> >> tombstone LHS
> > >> >> c-5) if there is a new result (old result may or may not exist),
> emit
> > >> >> only new result LHS (don't emit a tombstone)
> > >> >>
> > >> >>
> > >> >> Case (a) and (b) are simple and could be implemented with a "fire
> and
> > >> >> forget" strategy. The LHS just "blindly" updates the subscription,
> the
> > >> >> RHS can process the subscription with local knowledge and may send
> > >> >> something back. If something is sent back, the LHS blindly emits
> it.
> > >> >>
> > >> >> We know that for both cases, we never miss anything and we never
> emit
> > >> >> anything unnecessary.
> > >> >>
> > >> >> However, using this pattern for (c), we don't get our expected
> result:
> > >> >>
> > >> >> Issues: LHS sends both subscription updates in parallel. It does
> not
> > >> >> know if zero, one, or two result records will be produced by RHS.
> If
> > >> RHS
> > >> >> produces two results, their order is not known (however, LHS would
> need
> > >> >> to emit them in the right order; also forcing RHS to always send a
> > >> >> result back is not a sufficient solution). If only one result comes
> > >> >> back, it's unclear if a second result may arrive later and thus the
> > >> >> result may need to be buffered... Overall, local knowledge does not
> > >> seem
> > >> >> to be sufficient to tackle the case.
> > >> >>
> > >> >> The current proposal tries to address the issue with encoding
> > >> additional
> > >> >> information, to tell the RHS to send a tombstone back or not. But
> it
> > >> >> does not seem to be perfect, and it might result in unnecessary
> > >> >> tombstones as it still uses local knowledge only and thus misses
> some
> > >> >> information.
> > >> >>
> > >> >> I think the main problem is, that the knowledge about the a
> potential
> > >> >> previous result and a potential new result is sharded on the RHS.
> > >> Hence,
> > >> >> the "unsubscribe" does not know if it needs to send a tombstone
> back
> > >> for
> > >> >> the case that there was an old result but there is no new result.
> > >> >> Similarly, the "new subscribe" cannot know if it needs to send a
> > >> >> tombstone or not (as it does not know if there was a previous
> result)
> > >> if
> > >> >> it does not match.
> > >> >>
> > >> >> To really solve the issue, I see two possible solutions (both are
> not
> > >> >> great, but I wanted to discuss them anyway):
> > >> >>
> > >> >> S-1: First unsubscribe, and send new subscription after result
> comes
> > >> >> back. For this case, the RHS must always send something back to
> the LHS
> > >> >> on unsubscribe. The answer if "previous result exists/not-exist"
> can be
> > >> >> added to the new-subscription and hence RHS can either return
> nothing,
> > >> a
> > >> >> tombstone, or a new result. The LHS can blindly emit whatever RHS
> > >> >> returns. This would also cover (a) and (b) cases. However, the
> overall
> > >> >> time to emit the join result is doubled for the (common) update
> case...
> > >> >> (we need two consecutive round-trips to the RHS).
> > >> >>
> > >> >> S-2: Remember/store if a previous result exists on LHS: for this
> case,
> > >> >> (a) is handled straightforward, (b) is handled by telling RHS to
> send
> > >> >> tombstone if previous result exits, and (c) can send both request
> in
> > >> >> parallel letting the unsubscribe never return anything, and
> subscribe
> > >> is
> > >> >> handled as in (b). However, we need a second store on the LHS to
> > >> >> remember if there was a previous result. (Also not sure how
> > >> >> interleaving/inflight computation might affect the algorithm...)
> > >> >>
> > >> >> I think, sending unnecessary tombstones is quite bad (in very old
> > >> >> releases we had a similar issue and fixed it). However, I am also
> not
> > >> >> 100% sure if the solutions I came up with are good enough to
> justify
> > >> >> them. (Personally, I slightly tend to prefer S-2 because I think
> that
> > >> >> the additional store is less of an issue than the increase
> processing
> > >> >> time).
> > >> >>
> > >> >> Would love to hear your thoughts.
> > >> >>
> > >> >>
> > >> >> -Matthias
> > >> >>
> > >> >>
> > >> >> On 6/28/19 6:19 AM, Adam Bellemare wrote:
> > >> >>> 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 unnecessary tombstone is (that I
> can
> > >> >>> tell...) when we do the following:
> > >> >>> RHS:
> > >> >>> (1, bar)
> > >> >>>
> > >> >>> LHS
> > >> >>> (K,1)  -> Results in (K, 1, bar) being output
> > >> >>> (K,1) -> (K,2) ->  Results in (K, null) being output for INNER (no
> > >> >> matching
> > >> >>> element on LHS)
> > >> >>> (K,2) -> (K,3) ->  Results in (K, null) being output for INNER
> > >> (because
> > >> >> we
> > >> >>> don't maintain state to know we already output the tombstone on
> the
> > >> >>> previous transition).
> > >> >>> (K,2) -> (K,9000) ->  Results in (K, null)... etc.
> > >> >>>
> > >> >>> Byte versioning is going in today, then I hope to get back to
> > >> addressing
> > >> >> a
> > >> >>> number of John's previous questions in the PR.
> > >> >>>
> > >> >>> Adam
> > >> >>>
> > >> >>>
> > >> >>> On Thu, Jun 27, 2019 at 5:47 PM Matthias J. Sax <
> > >> matthias@confluent.io>
> > >> >>> wrote:
> > >> >>>
> > >> >>>> 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 unnecessary tombstone? Seems I a missing case?
> > >> >>>>
> > >> >>>> Also for my own understanding: the `instruction` is only part of
> the
> > >> >>>> message? It is no necessary to store it in the RHS auxiliary
> store,
> > >> >> right?
> > >> >>>>
> > >> >>>> About right/full-outer joins. Agreed. Getting left-joins would be
> > >> >> awesome!
> > >> >>>>
> > >> >>>> About upgrading: Good call John! Adding a version byte for
> > >> subscription
> > >> >>>> and response is good forward thinking. I personally prefer
> version
> > >> >>>> numbers, too, as they carry more information.
> > >> >>>>
> > >> >>>> Thanks for all the hard to everybody involved!
> > >> >>>>
> > >> >>>>
> > >> >>>> -Matthias
> > >> >>>>
> > >> >>>> On 6/27/19 1:44 PM, John Roesler wrote:
> > >> >>>>> 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
> > >> revisions,
> > >> >>>>> we can just use one byte. The worst case is that we run out of
> > >> numbers
> > >> >>>>> and reserve the last one to mean, "consult another field for the
> > >> >>>>> actual version number". It seems like a single byte on each
> message
> > >> >>>>> isn't too much to pay.
> > >> >>>>>
> > >> >>>>> Since you point it out, we might as well put a version number
> on the
> > >> >>>>> SubscriptionResponseWrapper as well. It may not be needed, but
> if we
> > >> >>>>> ever need it, even just once, we'll be glad we have it.
> > >> >>>>>
> > >> >>>>> Regarding the instructions field, we can also serialize the enum
> > >> very
> > >> >>>>> compactly as a single byte (which is the same size a boolean
> takes
> > >> >>>>> anyway), so it seems like an Enum in Java-land and a byte on the
> > >> wire
> > >> >>>>> is a good choice.
> > >> >>>>>
> > >> >>>>> Agreed on the right and full outer joins, it doesn't seem
> necessary
> > >> >>>>> right now, although I am happy to see the left join "join" the
> > >> party,
> > >> >>>>> since as you said, we were so close to it anyway. Can you also
> add
> > >> it
> > >> >>>>> to the KIP?
> > >> >>>>>
> > >> >>>>> Thanks as always for your awesome efforts on this,
> > >> >>>>> -John
> > >> >>>>>
> > >> >>>>> On Thu, Jun 27, 2019 at 3:04 PM Adam Bellemare <
> > >> >> adam.bellemare@gmail.com>
> > >> >>>> wrote:
> > >> >>>>>>
> > >> >>>>>> 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 microservices -
> > >> there
> > >> >> are
> > >> >>>>>> indeed cases where a breaking change must be made, and forward
> > >> >>>>>> compatibility will provide us with no out other than requiring
> a
> > >> full
> > >> >>>> stop
> > >> >>>>>> and full upgrade for all nodes, shifting us back towards
> STRATEGY
> > >> 1.
> > >> >>>>>>
> > >> >>>>>> My preference is STRATEGY 1 with instructions as an ENUM, and
> we
> > >> can
> > >> >>>>>> certainly include a version. Would it make sense to include a
> > >> version
> > >> >>>>>> number in  SubscriptionResponseWrapper as well? Currently we
> don't
> > >> >> have
> > >> >>>> any
> > >> >>>>>> instructions in there, as I removed the boolean, but it is
> > >> certainly
> > >> >>>>>> plausible that it could happen in the future. I don't *think*
> we'll
> > >> >> need
> > >> >>>>>> it, but I also didn't think we'd need it for
> SubscriptionWrapper
> > >> and
> > >> >>>> here
> > >> >>>>>> we are.
> > >> >>>>>>
> > >> >>>>>> Thanks for the thoughts, and the info on the right-key. That
> was
> > >> >>>>>> enlightening, though I can't think of a use-case for it *at
> this
> > >> point
> > >> >>>> in
> > >> >>>>>> time*. :)
> > >> >>>>>>
> > >> >>>>>> Adam
> > >> >>>>>>
> > >> >>>>>>
> > >> >>>>>>
> > >> >>>>>> On Thu, Jun 27, 2019 at 12:29 PM John Roesler <
> john@confluent.io>
> > >> >>>> wrote:
> > >> >>>>>>
> > >> >>>>>>> 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 `nullKey: (null, rhsValue)`,
> but we
> > >> >>>>>>> don't currently allow null keys in Streams. It actually is
> > >> possible
> > >> >> to
> > >> >>>>>>> define them, and therefore to add right- and full-outer
> > >> foreign-key
> > >> >>>>>>> joins later, but it's non-trivial in a streaming context with
> > >> >>>>>>> continuously updated results. (See the PS if you're curious
> what
> > >> I'm
> > >> >>>>>>> thinking). You're correct, right- and full-outer joins are
> > >> trivial on
> > >> >>>>>>> our current 1:1 table joins because they are equi-joins.
> > >> >>>>>>>
> > >> >>>>>>> Regarding the transition, it sounds like what you're
> proposing is
> > >> >> that
> > >> >>>>>>> we would say, "adding a foreign-key join to your topology
> > >> requires a
> > >> >>>>>>> full application reset (or a new application id)". This is
> also an
> > >> >>>>>>> acceptable constraint to place on the feature, but not
> strictly
> > >> >>>>>>> necessary. Since 2.3, it's now possible to give all the state
> in
> > >> your
> > >> >>>>>>> application stable names. This means that it's no longer true
> that
> > >> >>>>>>> adding a node to your topology graph would break its
> structure,
> > >> and
> > >> >> it
> > >> >>>>>>> does become possible to add new operators and simply restart
> the
> > >> app.
> > >> >>>>>>> Revisiting my prior thought, though, I think the problem is
> not
> > >> >>>>>>> specific to your feature. For example, adding a new grouped
> > >> >>>>>>> aggregation would produce a new repartition topic, but the
> > >> >> repartition
> > >> >>>>>>> topic partitions might get assigned to old nodes in the
> middle of
> > >> a
> > >> >>>>>>> rolling bounce, and they would need to just ignore them. This
> > >> >>>>>>> requirement is the same for the repartition topics in the FK
> > >> join, so
> > >> >>>>>>> it's orthogonal to your design.
> > >> >>>>>>>
> > >> >>>>>>> Back to the first concern, though, I'm not sure I followed the
> > >> >>>>>>> explanation. As a thought experiment, let's imagine that Joe
> > >> hadn't
> > >> >>>>>>> taken the time to experiment with your feature branch. We
> wouldn't
> > >> >>>>>>> have noticed the problem until the feature was already
> released in
> > >> >>>>>>> 2.4. So the wire protocol on that PK->FK subscription topic
> would
> > >> >> have
> > >> >>>>>>> been V1: "FK,PK,HASH,BOOLEAN". Then, Joe would have let us
> know
> > >> the
> > >> >>>>>>> problem once they picked up the feature, so we would want to
> > >> >> implement
> > >> >>>>>>> your proposed fix and change the wire protocol to V2:
> > >> >>>>>>> "FK,PK,HASH,INSTRUCTIONS" in 2.5. Upon rolling out the
> update, we
> > >> >>>>>>> would see both 2.4 nodes encountering V2 messages and 2.5
> nodes
> > >> >>>>>>> encountering V1 messages. How can they both detect that they
> are
> > >> >>>>>>> attempting to process a newer or older protocol? If they can
> > >> detect
> > >> >>>>>>> it, then what should they do?
> > >> >>>>>>>
> > >> >>>>>>> From experience, there are two basic solutions to this
> problem:
> > >> >>>>>>>
> > >> >>>>>>> STRATEGY1. Add a protocol version to the message (could be a
> > >> number
> > >> >> at
> > >> >>>>>>> the start of the message payload, or it could be a number in
> the
> > >> >>>>>>> message headers, not sure if it matters much. Payload is
> probably
> > >> >> more
> > >> >>>>>>> compact, since the header would need a name.) In this case,
> the
> > >> 2.4
> > >> >>>>>>> worker would know that it's max protocol version is V1, and
> when
> > >> it
> > >> >>>>>>> sees the V2 message, it knows that it can't handle it
> properly.
> > >> >> Rather
> > >> >>>>>>> than doing something wrong, it would just not do anything.
> This
> > >> means
> > >> >>>>>>> it would stop the task, if not shut down the whole instance.
> On
> > >> the
> > >> >>>>>>> other hand, a 2.5 worker would have some defined logic for
> how to
> > >> >>>>>>> handle all versions (V1 and V2), so once the upgrade is
> complete,
> > >> all
> > >> >>>>>>> messages can be processed.
> > >> >>>>>>>
> > >> >>>>>>> STRATEGY2. Make the schema forward-compatible. Basically, we
> > >> ensure
> > >> >>>>>>> that new fields can only be appended to the message schema,
> and
> > >> that
> > >> >>>>>>> older workers using only a prefix of the full message would
> still
> > >> >>>>>>> behave correctly. Using the example above, we'd instead
> evolve the
> > >> >>>>>>> schema to V2': "FK,PK,HASH,BOOLEAN,INSTRUCTIONS", and
> continue to
> > >> set
> > >> >>>>>>> the boolean field to true for the "new" foreign key. Then, 2.4
> > >> >> workers
> > >> >>>>>>> encountering the a "new FK" message would just see the prefix
> of
> > >> the
> > >> >>>>>>> payload that makes sense to them, and they would still
> continue
> > >> >>>>>>> processing the messages as they always have. Only after the
> 2.5
> > >> code
> > >> >>>>>>> is fully rolled out to the cluster would we be sure to see the
> > >> >> desired
> > >> >>>>>>> behavior. Note: in the reverse case, a 2.5 worker knows how to
> > >> fully
> > >> >>>>>>> parse the new message format, even if it plans to ignore the
> > >> BOOLEAN
> > >> >>>>>>> field.
> > >> >>>>>>>
> > >> >>>>>>> There are some tradeoffs between these strategies: STRATEGY1
> > >> ensures
> > >> >>>>>>> that all messages are only handled by workers that can
> properly
> > >> >> handle
> > >> >>>>>>> them, although it results in processing stalls while there are
> > >> still
> > >> >>>>>>> old nodes in the cluster. STRATEGY2 ensures that all messages
> can
> > >> be
> > >> >>>>>>> processed by all nodes, so there are no stalls, but we can
> never
> > >> >>>>>>> remove fields from the message, so if there are a lot of
> > >> revisions in
> > >> >>>>>>> the future, the payloads will become bloated. Also, it's not
> clear
> > >> >>>>>>> that you can actually pull off STRATEGY2 in all cases. If
> there's
> > >> >> some
> > >> >>>>>>> new kind of message you want to send that has no way to be
> > >> correctly
> > >> >>>>>>> processed at all under the 2.4 code paths, the prefix thing
> simply
> > >> >>>>>>> doesn't work. Etc.
> > >> >>>>>>>
> > >> >>>>>>> Also, note that you can modify the above strategies by instead
> > >> >>>>>>> designing the message fields for extensibility. E.g., if you
> make
> > >> the
> > >> >>>>>>> instructions field an enum, then you can make sure that the
> > >> default
> > >> >>>>>>> case is handled sensibly (probably similarly to STRATEGY1,
> just
> > >> choke
> > >> >>>>>>> on unknown instructions) and that you never remove an
> instruction
> > >> >> type
> > >> >>>>>>> from the enum in future versions.
> > >> >>>>>>>
> > >> >>>>>>> Does this make sense?
> > >> >>>>>>> -John
> > >> >>>>>>>
> > >> >>>>>>>
> > >> >>>>>>>
> > >> >>>>>>>
> > >> >>>>>>> PS:
> > >> >>>>>>> We can define null keys for streaming tables, but it's tricky.
> > >> >>>>>>>
> > >> >>>>>>> Specifically, you'd want to define some concept of null keys
> that
> > >> >>>>>>> allows all null keys to be unique, but _also_ to have a fixed
> > >> >>>>>>> identity, so that a particular null-key can be updated later.
> One
> > >> >>>>>>> example could be to union the existing keyspace with a new
> > >> >>>>>>> null-keyspace, where normal keys are like "key" and null-keys
> are
> > >> >> like
> > >> >>>>>>> "null(identity)". Then given a query like
> > >> >>>>>>> "KTable<String,Integer>.rightJoin(KTable<Integer,Boolean>)",
> and
> > >> >>>>>>> inputs like:
> > >> >>>>>>> LHS:
> > >> >>>>>>> "a": 1
> > >> >>>>>>> "b": 2
> > >> >>>>>>>
> > >> >>>>>>> RHS:
> > >> >>>>>>> 1: true
> > >> >>>>>>> 3: false
> > >> >>>>>>>
> > >> >>>>>>> a full outer join would produce:
> > >> >>>>>>> "a": (1, true)
> > >> >>>>>>> "b": (2, null)
> > >> >>>>>>> null(3): (null, false)
> > >> >>>>>>>
> > >> >>>>>>> which can be correctly updated later if we get an update on
> the
> > >> LHS:
> > >> >>>>>>> PUT("c": 3)
> > >> >>>>>>>
> > >> >>>>>>> We'd emit for the results:
> > >> >>>>>>> DELETE(null(e))
> > >> >>>>>>> EMIT("c": (3, false))
> > >> >>>>>>>
> > >> >>>>>>> Resulting in the correct result table of:
> > >> >>>>>>> "a": (1, true)
> > >> >>>>>>> "b": (2, null)
> > >> >>>>>>> "c": (3, false)
> > >> >>>>>>>
> > >> >>>>>>> As mentioned, this is tricky, and I would avoid it until we
> have
> > >> >>>>>>> evidence that it's actually useful to cover this part of the
> > >> design
> > >> >>>>>>> space. Certainly, it would be a separate KIP if it came to
> that.
> > >> >>>>>>>
> > >> >>>>>>> On Wed, Jun 26, 2019 at 8:57 PM Adam Bellemare <
> > >> >>>> adam.bellemare@gmail.com>
> > >> >>>>>>> wrote:
> > >> >>>>>>>>
> > >> >>>>>>>> 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
> > >> >> the
> > >> >>>>>>> last
> > >> >>>>>>>> commit (about 10 min ago) along with INNER join. I do not
> think
> > >> that
> > >> >>>>>>> RIGHT
> > >> >>>>>>>> join and OUTER are possible given that there is no LHS key
> > >> >> available,
> > >> >>>> so
> > >> >>>>>>>> LHSTable.outerJoinOnForeignKey(RHSTable) wouldn't even make
> > >> sense.
> > >> >>>> This
> > >> >>>>>>> is
> > >> >>>>>>>> in contrast to the current LHSTable.outerJoin(RHSTable), as
> they
> > >> are
> > >> >>>> both
> > >> >>>>>>>> keyed on the same key. I have buffed up the Integration
> tests and
> > >> >> have
> > >> >>>>>>>> tried to make them more readable to ensure that we're
> covering
> > >> all
> > >> >> the
> > >> >>>>>>>> scenarios. I think that if we can get more eyes on the
> workflow
> > >> >>>> showing
> > >> >>>>>>> the
> > >> >>>>>>>> various LHS and RHS events and outputs then that may help us
> > >> >> validate
> > >> >>>>>>> that
> > >> >>>>>>>> we have all the scenarios covered.
> > >> >>>>>>>>
> > >> >>>>>>>> With regards to the 2.3->2.4 scenario you described, I'm not
> > >> >> entirely
> > >> >>>>>>> sure
> > >> >>>>>>>> I follow. If they want to add a FK-join, they will need to
> rework
> > >> >>>> their
> > >> >>>>>>>> code in the KStreams app and make a new release, since the
> > >> >> underlying
> > >> >>>>>>>> topology would be different and new internal topics would
> need
> > >> to be
> > >> >>>>>>>> created. In other words, I don't think a rolling upgrade
> where
> > >> the
> > >> >>>> user
> > >> >>>>>>>> introduces a FK join would be possible since their topology
> would
> > >> >>>>>>>> necessitate a full KStreams reset. Is this what you meant?
> > >> >>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>>
> > >> >>>>>>>> On Wed, Jun 26, 2019 at 4:10 PM John Roesler <
> john@confluent.io>
> > >> >>>> wrote:
> > >> >>>>>>>>
> > >> >>>>>>>>> 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.bellemare@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.bellemare@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 <
> > >> john@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
> > >> >>>>>>>>>>>> <ad...@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 <
> > >> >>>>>>> john@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
> > >> >>>>>>>>>>>>>>
> > >> >>>>>>>>>>>>
> > >> >>>>>>>>>>>
> > >> >>>>>>>>>
> > >> >>>>>>>
> > >> >>>>
> > >> >>>>
> > >> >>>
> > >> >>
> > >> >>
> > >> >
> > >>
> > >>
>
>

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

Posted by John Roesler <jo...@confluent.io>.
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 join result would be:
LEFT: (k -> (left-val, null))
INNER: (k -> null)

So, the tombstone result doesn't somehow make the inner join result
"left-join-like". Instead we're saying, "delete the join result for k
from the post-join view". It may be unnecessary to "delete" a
non-existant record from a view, but it's never incorrect. The issue
we're discussing is an operational one (whether the unnecessary
tombstones are a performance problem), not a semantic one.

Second, just to be clear, we have no obligation under current Streams
semantics to eliminate all unnecessary updates. We make no promise
whatsoever to "emit on change", so anyone consuming the output of a
Streams table operation should _expect_ to see unnecessary (aka
"duplicate") updates. We _can_ drop unnecessary updates (and do, when
the opportunity presents itself) to save on performance, though.

-----
Since it's clear that we're _only_ discussing a performance
optimization, we should try to characterize the actual performance
problem so that we can weigh the solution against the benefit it
yields.

Unless I'm mistaken, the exact problem we're discussing with the most
recent version of the proposal is that we'll emit an unnecessary
tombstone for inner joins or an unnecessary (left,null) result for
left joins specifically when a record transitions from one *broken* FK
reference to another *broken* FK reference. Any other transition
(valid FK -> valid FK, valid FK -> broken FK, or broken FK -> valid
FK) would yield the correct result (a necessary join result or a
necessary tombstone).

How likely are these transitions to occur in practice? We should
consider both the likelihood of FK changes at all and the likelihood
of broken FK references.

Probability of FK change:
It seems generally rare for records to change FK references at all.
How often to employees change departments, or comments change
articles, or people change addresses, etc, etc.? If someone has
anecdotal experience with a domain involving multiple, rapid FK
updates, please share them, but in all the domains I have experience
with, FK reference changes are _much_ less common than data updates. I
do happen to know that FK updates are moderately common in
Bazaarvoice's data model, but that they're orders of magnitude less
frequent than other kinds of updates.

Probability of broken reference:
Tables represented by Kafka topics (obviously) can't enforce
referential integrity, but (spitballing here) upstream databases can
and probably do. How often are we dealing with employees who work for
a manager who isn't in the HR database? Or a comment on a non-existent
article? Or someone who lives at a non-existent address? Again, this
is anecdotal, but that's the whole point of a likelihood estimation.
Please share your experiences. Again, I happen to know that at
Bazaarvoice, the incoming data actually doesn't guarantee referential
integrity, but that broken references are still relatively rare (and
actually, as it happens, it's not possible to change the FK without
first repairing a broken reference, so the broken -> broken transition
would be exceedingly rare).

If we can assume that the upstream data _mostly_ maintains referential
integrity, then we can make a further assumption that when we observe
a broken reference, it is probably transient. I.e., we can expect the
reference to be repaired "pretty soon". If this is the case, then the
probability we get a FK change from one broken reference to another
broken reference before the first reference gets repaired should be
quite rare for "normal" data sets.

Caveat: https://www.xkcd.com/2167/

-----

So, under the assumption that these unnecessary tombstones are rare,
and with the understanding that they're semantically ok to emit, it
really doesn't seem worthwhile to take on all the extra complexity
proposed earlier in this conversation.

A much simpler solution, if someone is really struggling with extra
tombstones, would be just be to slap an LRU cache on the result table
and drop unnecessary tombstones that way.

Again, just my personal thoughts, FWIW...
-John

On Tue, Jul 9, 2019 at 11:25 PM Adam Bellemare <ad...@gmail.com> wrote:
>
> 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-handed.
>
> 2) A very deft solution is to also just support left-joins but not
> inner-joins. Practically speaking, either INNER or LEFT join as it
> currently is would support all of my use-cases.
>
> 3) Accept that there may be some null tombstones (though this makes me want
> to just go with LEFT only instead of LEFT and PSEUDO-INNER).
>
> 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 experience from our usage of our internal
> implementation at my company, and that of many others who have reached out
> to me.
>
> What would help most at this point is if someone can come up with a
> scenario where sending unnecessary tombstones actually poses a downstream
> problem beyond that of confusing behaviour, as I cannot think of one
> myself.  With that being said, I am far more inclined to actually then
> support just option #2 above and only have LEFT joins, forgoing INNER
> completely since it would not be a true inner join.
>
> Adam
>
>
>
>
>
>
>
>
> On Thu, Jul 4, 2019 at 8:50 AM Adam Bellemare <ad...@gmail.com>
> wrote:
>
> > 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,hashOf(b),PROPAGATE_OLD_VALUE) hits RHS-0, the following
> > occurs:
> >   1) Store the current (CombinedKey<FK,K>, Value=(Hash, ForeignValue)) in
> > a variable
> >   2) Delete the key from the store
> >   3) Publish the event from step A-1 downstream with an instruction:
> > (eventType = COMPARE_TO_OTHER) (or whatever)
> > *      (key, (hashOf(b),wasForeignValueNull, eventType))*
> >     //Don't need the old hashOf(b) as it is guaranteed to be out of date
> >     //We do need the hashOf(b) that came with the event to be passed
> > along. Will be used in resolution.
> >     //Don't need the actual value as we aren't joining or comparing the
> > values, just using it to determine nulls. Reduces payload size.
> >
> > B) When (k,b,PROPAGATE_NEW_VALUE) hits RHS-1, the following occurs:
> >   1) Store it in the prefix state store (as we currently do)
> >   2) Get the FK-value (as we currently do)
> >   3) Return the normal SubscriptionResponse payload (eventType = UPDATE)
> > (or whatever)
> > *     (key, (hashOf(b), foreignValue, eventType))*
> >
> >
> > C) The Resolver Table is keyed on (as per our example):
> > key = CombinedKey<k-hash(b)>, value =
> > NullValueResolution<wasForeignValueNull (set by RHS-0), foreignValue (set
> > by RHS-1)>
> >
> > Resolution Steps per event:
> >
> > When one of either the output events from A (eventType ==
> > COMPARE_TO_OTHER) or B (eventType == UPDATE) is received
> > 1) Check if this event matches the current hashOf(b). If not, discard it,
> > for it is stale and no longer matters.  Additionally, delete entry
> > CombinedKey<k-hash(b)> from the Resolver Table.
> >
> > 2) Lookup event in table on its CombinedKey:
> >   - If it's not in the table, create the  NullValueResolution value,
> > populate the field related to the eventType, and add it to the table.
> >   - If it already IS in the table, get the existing NullValueResolution
> > object and finish populating it:
> >
> > 3) If the NullValueResolution is fully populated, move on to the
> > resolution logic below.
> >
> > Format:
> > (wasForeignValueNull, foreignValue) -> Result
> > If:
> > ( false  , Null ) -> Send tombstone. Old value was not null, new one is,
> > send tombstone.
> > (  true  , Null ) -> Do nothing.  See * below for more details.
> > (  true  , NewValue ) -> Send the new result
> > (  true  , NewValue ) -> Send the new result
> >
> > * wasForeignValueNull may have been false at some very recent point, but
> > only just translated to true (race condition). In this case, the RHS table
> > was updated and the value was set to null due to a an RHS update of (a,
> > oldVal) -> (a, null). This event on its own will propagate a delete event
> > through to the resolver (of a different eventType), so we don't need to
> > handle this case from the LHS and doing nothing is OK.
> >
> > In the case that it's truly (true, Null), we also don't need to send a
> > tombstone because wasForeignKeyNull == true means that a tombstone was
> > previously sent.
> >
> > 4) Check the hashOf(b) one last time before sending the resolved message
> > out. If the hash is old, discard it.
> >
> > 5) Delete the row from the Resolver Table.
> >
> >
> > Takeaways:
> > 1) I believe this is only necessary for INNER joins when we transition
> > from (k, non-Null-val) -> (k, non-Null-new-val)
> > 2) We can maintain state until we get both events back from RHS-0 and
> > RHS-1, at which point we delete it and clean up.
> >   NOTE: I think this still works with at-least-once processing, but I am
> > not 100% on this. The concern is that we receive events from RHS-0, RHS-1
> > (causing a row deletion), then RHS-1 again from at-least once (creating an
> > entry that never gets deleted).
> > 3) My naive implementation requires SubscriptionResponseWrapper event
> > types.
> > 4) This shouldn't affect performance much, as it should only be
> > maxOf(RHS-0, RHS-1) event propagation and processing.
> >
> >
> > Adam
> >
> > On Wed, Jul 3, 2019 at 2:18 PM Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> >> 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, but want to throw out the idea first.
> >>
> >> > S-2 could probably be simplified to "for a given key, was the previous
> >> > propagated result a null/tombstone or not?"
> >>
> >> Yes, that's the idea.
> >>
> >> > However,
> >> > the table will grow indefinitely large as we can never remove keys from
> >> it.
> >>
> >> That a good point. I missed that... Seems that this rules out S-2
> >> because if we cannot provide _strict_ guarantees, it does not seem to be
> >> worth to even try?
> >>
> >>
> >> About the impact of unnecessary tombstones: There is a performance
> >> impact, as we increase the output data rate, potentially broker load if
> >> the result is written into a topic, and also increase downstream
> >> processing load. It's hard to judge, how much the overhead will be, as
> >> it will depend on the selectivity of the join. But it might be
> >> significant?
> >>
> >> Also, users have certain expectations on the result and it's unintuitive
> >> (even if semantically correct) to send those tombstones. From
> >> experience, we often have a hard time to explain semantics to people and
> >> I was hoping we could avoid introducing unintuitive behavior.
> >>
> >>
> >> Would be good to get input from others and how they judge the impact. I
> >> think it might be still worth to explore how complex S-1 would be. If we
> >> think it's too complex it might be a good argument to just accept the
> >> unnecessary tombstones?
> >>
> >>
> >> -Matthias
> >>
> >>
> >> On 7/3/19 8:03 AM, Adam Bellemare wrote:
> >> > 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 seems to me that the processor would need to block
> >> > while it waits for the unsubscribe to propagate and return, which would
> >> > cause throughput to drop significantly. Alternately, we would need to
> >> > maintain state anyways about which events were sent and which responses
> >> > returned, while being sure to respect the offset order in which they're
> >> > emitted. I think this would only reduce blocking slightly while
> >> increasing
> >> > complexity. If I am wrong in understanding this, please let me know
> >> where
> >> > my thinking is erroneous.
> >> >
> >> > S-2 could probably be simplified to "for a given key, was the previous
> >> > propagated result a null/tombstone or not?"
> >> > It would act very similarly to the hash value mechanism, where we
> >> discard
> >> > any events that are not of the correct hash. In this case, we simply
> >> store
> >> > (key, wasLastOutputATombstone) right before the event is output
> >> downstream
> >> > of the Join + Resolver. This ignores all the complexities of which
> >> event is
> >> > propagating over which wire and simply squelches any extra tombstones
> >> from
> >> > being sent.
> >> >
> >> > For storage, we need to use the full primary key and a boolean. However,
> >> > the table will grow indefinitely large as we can never remove keys from
> >> it.
> >> > If we delete key=k from the table and propagate a tombstone, but later
> >> (say
> >> > 3 weeks, 3 months, etc) we publish (k, baz), but baz does not exist on
> >> the
> >> > RHS, we will end up publishing an extra tombstone because we have no
> >> idea
> >> > what the previously sent record was for k. For this reason I think it's
> >> > worth asking if we really can maintain state, and if it's even necessary
> >> > (again, a full understanding of the impact of extra tombstones may help
> >> us
> >> > figure out a better solution).
> >> >
> >> > As it stands, I don't think either of these will work well. That being
> >> > said, I myself do not have any better ideas at the moment, but I would
> >> > still like to better understand circumstances where it has a negative
> >> > impact downstream as that may provide some insights.
> >> >
> >> >
> >> > Thanks
> >> >
> >> > Adam
> >> >
> >> >
> >> >
> >> >
> >> >
> >> > On Tue, Jul 2, 2019 at 11:18 PM Matthias J. Sax <ma...@confluent.io>
> >> > wrote:
> >> >
> >> >> 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 subscribe message to RHS
> >> >> a-2) RHS lookup and send result back if there is one
> >> >> a-3) emit result on LHS if any is returned
> >> >>
> >> >> b-1) delete subscription from RHS
> >> >> b-2) if there was a previous result (can easily be decided by looking
> >> up
> >> >> RHS table for an existing key), send tombstone back
> >> >> b-3) emit tombstone on LHS if any is returned
> >> >>
> >> >> c-1) delete old subscription from RHS
> >> >> c-2) send new subscription to RHS
> >> >> c-3) if there was no previous result and there is no new result emit
> >> >> nothing
> >> >> c-4) if there was a previous result and there is no new result emit a
> >> >> tombstone LHS
> >> >> c-5) if there is a new result (old result may or may not exist), emit
> >> >> only new result LHS (don't emit a tombstone)
> >> >>
> >> >>
> >> >> Case (a) and (b) are simple and could be implemented with a "fire and
> >> >> forget" strategy. The LHS just "blindly" updates the subscription, the
> >> >> RHS can process the subscription with local knowledge and may send
> >> >> something back. If something is sent back, the LHS blindly emits it.
> >> >>
> >> >> We know that for both cases, we never miss anything and we never emit
> >> >> anything unnecessary.
> >> >>
> >> >> However, using this pattern for (c), we don't get our expected result:
> >> >>
> >> >> Issues: LHS sends both subscription updates in parallel. It does not
> >> >> know if zero, one, or two result records will be produced by RHS. If
> >> RHS
> >> >> produces two results, their order is not known (however, LHS would need
> >> >> to emit them in the right order; also forcing RHS to always send a
> >> >> result back is not a sufficient solution). If only one result comes
> >> >> back, it's unclear if a second result may arrive later and thus the
> >> >> result may need to be buffered... Overall, local knowledge does not
> >> seem
> >> >> to be sufficient to tackle the case.
> >> >>
> >> >> The current proposal tries to address the issue with encoding
> >> additional
> >> >> information, to tell the RHS to send a tombstone back or not. But it
> >> >> does not seem to be perfect, and it might result in unnecessary
> >> >> tombstones as it still uses local knowledge only and thus misses some
> >> >> information.
> >> >>
> >> >> I think the main problem is, that the knowledge about the a potential
> >> >> previous result and a potential new result is sharded on the RHS.
> >> Hence,
> >> >> the "unsubscribe" does not know if it needs to send a tombstone back
> >> for
> >> >> the case that there was an old result but there is no new result.
> >> >> Similarly, the "new subscribe" cannot know if it needs to send a
> >> >> tombstone or not (as it does not know if there was a previous result)
> >> if
> >> >> it does not match.
> >> >>
> >> >> To really solve the issue, I see two possible solutions (both are not
> >> >> great, but I wanted to discuss them anyway):
> >> >>
> >> >> S-1: First unsubscribe, and send new subscription after result comes
> >> >> back. For this case, the RHS must always send something back to the LHS
> >> >> on unsubscribe. The answer if "previous result exists/not-exist" can be
> >> >> added to the new-subscription and hence RHS can either return nothing,
> >> a
> >> >> tombstone, or a new result. The LHS can blindly emit whatever RHS
> >> >> returns. This would also cover (a) and (b) cases. However, the overall
> >> >> time to emit the join result is doubled for the (common) update case...
> >> >> (we need two consecutive round-trips to the RHS).
> >> >>
> >> >> S-2: Remember/store if a previous result exists on LHS: for this case,
> >> >> (a) is handled straightforward, (b) is handled by telling RHS to send
> >> >> tombstone if previous result exits, and (c) can send both request in
> >> >> parallel letting the unsubscribe never return anything, and subscribe
> >> is
> >> >> handled as in (b). However, we need a second store on the LHS to
> >> >> remember if there was a previous result. (Also not sure how
> >> >> interleaving/inflight computation might affect the algorithm...)
> >> >>
> >> >> I think, sending unnecessary tombstones is quite bad (in very old
> >> >> releases we had a similar issue and fixed it). However, I am also not
> >> >> 100% sure if the solutions I came up with are good enough to justify
> >> >> them. (Personally, I slightly tend to prefer S-2 because I think that
> >> >> the additional store is less of an issue than the increase processing
> >> >> time).
> >> >>
> >> >> Would love to hear your thoughts.
> >> >>
> >> >>
> >> >> -Matthias
> >> >>
> >> >>
> >> >> On 6/28/19 6:19 AM, Adam Bellemare wrote:
> >> >>> 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 unnecessary tombstone is (that I can
> >> >>> tell...) when we do the following:
> >> >>> RHS:
> >> >>> (1, bar)
> >> >>>
> >> >>> LHS
> >> >>> (K,1)  -> Results in (K, 1, bar) being output
> >> >>> (K,1) -> (K,2) ->  Results in (K, null) being output for INNER (no
> >> >> matching
> >> >>> element on LHS)
> >> >>> (K,2) -> (K,3) ->  Results in (K, null) being output for INNER
> >> (because
> >> >> we
> >> >>> don't maintain state to know we already output the tombstone on the
> >> >>> previous transition).
> >> >>> (K,2) -> (K,9000) ->  Results in (K, null)... etc.
> >> >>>
> >> >>> Byte versioning is going in today, then I hope to get back to
> >> addressing
> >> >> a
> >> >>> number of John's previous questions in the PR.
> >> >>>
> >> >>> Adam
> >> >>>
> >> >>>
> >> >>> On Thu, Jun 27, 2019 at 5:47 PM Matthias J. Sax <
> >> matthias@confluent.io>
> >> >>> wrote:
> >> >>>
> >> >>>> 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 unnecessary tombstone? Seems I a missing case?
> >> >>>>
> >> >>>> Also for my own understanding: the `instruction` is only part of the
> >> >>>> message? It is no necessary to store it in the RHS auxiliary store,
> >> >> right?
> >> >>>>
> >> >>>> About right/full-outer joins. Agreed. Getting left-joins would be
> >> >> awesome!
> >> >>>>
> >> >>>> About upgrading: Good call John! Adding a version byte for
> >> subscription
> >> >>>> and response is good forward thinking. I personally prefer version
> >> >>>> numbers, too, as they carry more information.
> >> >>>>
> >> >>>> Thanks for all the hard to everybody involved!
> >> >>>>
> >> >>>>
> >> >>>> -Matthias
> >> >>>>
> >> >>>> On 6/27/19 1:44 PM, John Roesler wrote:
> >> >>>>> 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
> >> revisions,
> >> >>>>> we can just use one byte. The worst case is that we run out of
> >> numbers
> >> >>>>> and reserve the last one to mean, "consult another field for the
> >> >>>>> actual version number". It seems like a single byte on each message
> >> >>>>> isn't too much to pay.
> >> >>>>>
> >> >>>>> Since you point it out, we might as well put a version number on the
> >> >>>>> SubscriptionResponseWrapper as well. It may not be needed, but if we
> >> >>>>> ever need it, even just once, we'll be glad we have it.
> >> >>>>>
> >> >>>>> Regarding the instructions field, we can also serialize the enum
> >> very
> >> >>>>> compactly as a single byte (which is the same size a boolean takes
> >> >>>>> anyway), so it seems like an Enum in Java-land and a byte on the
> >> wire
> >> >>>>> is a good choice.
> >> >>>>>
> >> >>>>> Agreed on the right and full outer joins, it doesn't seem necessary
> >> >>>>> right now, although I am happy to see the left join "join" the
> >> party,
> >> >>>>> since as you said, we were so close to it anyway. Can you also add
> >> it
> >> >>>>> to the KIP?
> >> >>>>>
> >> >>>>> Thanks as always for your awesome efforts on this,
> >> >>>>> -John
> >> >>>>>
> >> >>>>> On Thu, Jun 27, 2019 at 3:04 PM Adam Bellemare <
> >> >> adam.bellemare@gmail.com>
> >> >>>> wrote:
> >> >>>>>>
> >> >>>>>> 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 microservices -
> >> there
> >> >> are
> >> >>>>>> indeed cases where a breaking change must be made, and forward
> >> >>>>>> compatibility will provide us with no out other than requiring a
> >> full
> >> >>>> stop
> >> >>>>>> and full upgrade for all nodes, shifting us back towards STRATEGY
> >> 1.
> >> >>>>>>
> >> >>>>>> My preference is STRATEGY 1 with instructions as an ENUM, and we
> >> can
> >> >>>>>> certainly include a version. Would it make sense to include a
> >> version
> >> >>>>>> number in  SubscriptionResponseWrapper as well? Currently we don't
> >> >> have
> >> >>>> any
> >> >>>>>> instructions in there, as I removed the boolean, but it is
> >> certainly
> >> >>>>>> plausible that it could happen in the future. I don't *think* we'll
> >> >> need
> >> >>>>>> it, but I also didn't think we'd need it for SubscriptionWrapper
> >> and
> >> >>>> here
> >> >>>>>> we are.
> >> >>>>>>
> >> >>>>>> Thanks for the thoughts, and the info on the right-key. That was
> >> >>>>>> enlightening, though I can't think of a use-case for it *at this
> >> point
> >> >>>> in
> >> >>>>>> time*. :)
> >> >>>>>>
> >> >>>>>> Adam
> >> >>>>>>
> >> >>>>>>
> >> >>>>>>
> >> >>>>>> On Thu, Jun 27, 2019 at 12:29 PM John Roesler <jo...@confluent.io>
> >> >>>> wrote:
> >> >>>>>>
> >> >>>>>>> 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 `nullKey: (null, rhsValue)`, but we
> >> >>>>>>> don't currently allow null keys in Streams. It actually is
> >> possible
> >> >> to
> >> >>>>>>> define them, and therefore to add right- and full-outer
> >> foreign-key
> >> >>>>>>> joins later, but it's non-trivial in a streaming context with
> >> >>>>>>> continuously updated results. (See the PS if you're curious what
> >> I'm
> >> >>>>>>> thinking). You're correct, right- and full-outer joins are
> >> trivial on
> >> >>>>>>> our current 1:1 table joins because they are equi-joins.
> >> >>>>>>>
> >> >>>>>>> Regarding the transition, it sounds like what you're proposing is
> >> >> that
> >> >>>>>>> we would say, "adding a foreign-key join to your topology
> >> requires a
> >> >>>>>>> full application reset (or a new application id)". This is also an
> >> >>>>>>> acceptable constraint to place on the feature, but not strictly
> >> >>>>>>> necessary. Since 2.3, it's now possible to give all the state in
> >> your
> >> >>>>>>> application stable names. This means that it's no longer true that
> >> >>>>>>> adding a node to your topology graph would break its structure,
> >> and
> >> >> it
> >> >>>>>>> does become possible to add new operators and simply restart the
> >> app.
> >> >>>>>>> Revisiting my prior thought, though, I think the problem is not
> >> >>>>>>> specific to your feature. For example, adding a new grouped
> >> >>>>>>> aggregation would produce a new repartition topic, but the
> >> >> repartition
> >> >>>>>>> topic partitions might get assigned to old nodes in the middle of
> >> a
> >> >>>>>>> rolling bounce, and they would need to just ignore them. This
> >> >>>>>>> requirement is the same for the repartition topics in the FK
> >> join, so
> >> >>>>>>> it's orthogonal to your design.
> >> >>>>>>>
> >> >>>>>>> Back to the first concern, though, I'm not sure I followed the
> >> >>>>>>> explanation. As a thought experiment, let's imagine that Joe
> >> hadn't
> >> >>>>>>> taken the time to experiment with your feature branch. We wouldn't
> >> >>>>>>> have noticed the problem until the feature was already released in
> >> >>>>>>> 2.4. So the wire protocol on that PK->FK subscription topic would
> >> >> have
> >> >>>>>>> been V1: "FK,PK,HASH,BOOLEAN". Then, Joe would have let us know
> >> the
> >> >>>>>>> problem once they picked up the feature, so we would want to
> >> >> implement
> >> >>>>>>> your proposed fix and change the wire protocol to V2:
> >> >>>>>>> "FK,PK,HASH,INSTRUCTIONS" in 2.5. Upon rolling out the update, we
> >> >>>>>>> would see both 2.4 nodes encountering V2 messages and 2.5 nodes
> >> >>>>>>> encountering V1 messages. How can they both detect that they are
> >> >>>>>>> attempting to process a newer or older protocol? If they can
> >> detect
> >> >>>>>>> it, then what should they do?
> >> >>>>>>>
> >> >>>>>>> From experience, there are two basic solutions to this problem:
> >> >>>>>>>
> >> >>>>>>> STRATEGY1. Add a protocol version to the message (could be a
> >> number
> >> >> at
> >> >>>>>>> the start of the message payload, or it could be a number in the
> >> >>>>>>> message headers, not sure if it matters much. Payload is probably
> >> >> more
> >> >>>>>>> compact, since the header would need a name.) In this case, the
> >> 2.4
> >> >>>>>>> worker would know that it's max protocol version is V1, and when
> >> it
> >> >>>>>>> sees the V2 message, it knows that it can't handle it properly.
> >> >> Rather
> >> >>>>>>> than doing something wrong, it would just not do anything. This
> >> means
> >> >>>>>>> it would stop the task, if not shut down the whole instance. On
> >> the
> >> >>>>>>> other hand, a 2.5 worker would have some defined logic for how to
> >> >>>>>>> handle all versions (V1 and V2), so once the upgrade is complete,
> >> all
> >> >>>>>>> messages can be processed.
> >> >>>>>>>
> >> >>>>>>> STRATEGY2. Make the schema forward-compatible. Basically, we
> >> ensure
> >> >>>>>>> that new fields can only be appended to the message schema, and
> >> that
> >> >>>>>>> older workers using only a prefix of the full message would still
> >> >>>>>>> behave correctly. Using the example above, we'd instead evolve the
> >> >>>>>>> schema to V2': "FK,PK,HASH,BOOLEAN,INSTRUCTIONS", and continue to
> >> set
> >> >>>>>>> the boolean field to true for the "new" foreign key. Then, 2.4
> >> >> workers
> >> >>>>>>> encountering the a "new FK" message would just see the prefix of
> >> the
> >> >>>>>>> payload that makes sense to them, and they would still continue
> >> >>>>>>> processing the messages as they always have. Only after the 2.5
> >> code
> >> >>>>>>> is fully rolled out to the cluster would we be sure to see the
> >> >> desired
> >> >>>>>>> behavior. Note: in the reverse case, a 2.5 worker knows how to
> >> fully
> >> >>>>>>> parse the new message format, even if it plans to ignore the
> >> BOOLEAN
> >> >>>>>>> field.
> >> >>>>>>>
> >> >>>>>>> There are some tradeoffs between these strategies: STRATEGY1
> >> ensures
> >> >>>>>>> that all messages are only handled by workers that can properly
> >> >> handle
> >> >>>>>>> them, although it results in processing stalls while there are
> >> still
> >> >>>>>>> old nodes in the cluster. STRATEGY2 ensures that all messages can
> >> be
> >> >>>>>>> processed by all nodes, so there are no stalls, but we can never
> >> >>>>>>> remove fields from the message, so if there are a lot of
> >> revisions in
> >> >>>>>>> the future, the payloads will become bloated. Also, it's not clear
> >> >>>>>>> that you can actually pull off STRATEGY2 in all cases. If there's
> >> >> some
> >> >>>>>>> new kind of message you want to send that has no way to be
> >> correctly
> >> >>>>>>> processed at all under the 2.4 code paths, the prefix thing simply
> >> >>>>>>> doesn't work. Etc.
> >> >>>>>>>
> >> >>>>>>> Also, note that you can modify the above strategies by instead
> >> >>>>>>> designing the message fields for extensibility. E.g., if you make
> >> the
> >> >>>>>>> instructions field an enum, then you can make sure that the
> >> default
> >> >>>>>>> case is handled sensibly (probably similarly to STRATEGY1, just
> >> choke
> >> >>>>>>> on unknown instructions) and that you never remove an instruction
> >> >> type
> >> >>>>>>> from the enum in future versions.
> >> >>>>>>>
> >> >>>>>>> Does this make sense?
> >> >>>>>>> -John
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>>
> >> >>>>>>> PS:
> >> >>>>>>> We can define null keys for streaming tables, but it's tricky.
> >> >>>>>>>
> >> >>>>>>> Specifically, you'd want to define some concept of null keys that
> >> >>>>>>> allows all null keys to be unique, but _also_ to have a fixed
> >> >>>>>>> identity, so that a particular null-key can be updated later. One
> >> >>>>>>> example could be to union the existing keyspace with a new
> >> >>>>>>> null-keyspace, where normal keys are like "key" and null-keys are
> >> >> like
> >> >>>>>>> "null(identity)". Then given a query like
> >> >>>>>>> "KTable<String,Integer>.rightJoin(KTable<Integer,Boolean>)", and
> >> >>>>>>> inputs like:
> >> >>>>>>> LHS:
> >> >>>>>>> "a": 1
> >> >>>>>>> "b": 2
> >> >>>>>>>
> >> >>>>>>> RHS:
> >> >>>>>>> 1: true
> >> >>>>>>> 3: false
> >> >>>>>>>
> >> >>>>>>> a full outer join would produce:
> >> >>>>>>> "a": (1, true)
> >> >>>>>>> "b": (2, null)
> >> >>>>>>> null(3): (null, false)
> >> >>>>>>>
> >> >>>>>>> which can be correctly updated later if we get an update on the
> >> LHS:
> >> >>>>>>> PUT("c": 3)
> >> >>>>>>>
> >> >>>>>>> We'd emit for the results:
> >> >>>>>>> DELETE(null(e))
> >> >>>>>>> EMIT("c": (3, false))
> >> >>>>>>>
> >> >>>>>>> Resulting in the correct result table of:
> >> >>>>>>> "a": (1, true)
> >> >>>>>>> "b": (2, null)
> >> >>>>>>> "c": (3, false)
> >> >>>>>>>
> >> >>>>>>> As mentioned, this is tricky, and I would avoid it until we have
> >> >>>>>>> evidence that it's actually useful to cover this part of the
> >> design
> >> >>>>>>> space. Certainly, it would be a separate KIP if it came to that.
> >> >>>>>>>
> >> >>>>>>> On Wed, Jun 26, 2019 at 8:57 PM Adam Bellemare <
> >> >>>> adam.bellemare@gmail.com>
> >> >>>>>>> wrote:
> >> >>>>>>>>
> >> >>>>>>>> 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
> >> >> the
> >> >>>>>>> last
> >> >>>>>>>> commit (about 10 min ago) along with INNER join. I do not think
> >> that
> >> >>>>>>> RIGHT
> >> >>>>>>>> join and OUTER are possible given that there is no LHS key
> >> >> available,
> >> >>>> so
> >> >>>>>>>> LHSTable.outerJoinOnForeignKey(RHSTable) wouldn't even make
> >> sense.
> >> >>>> This
> >> >>>>>>> is
> >> >>>>>>>> in contrast to the current LHSTable.outerJoin(RHSTable), as they
> >> are
> >> >>>> both
> >> >>>>>>>> keyed on the same key. I have buffed up the Integration tests and
> >> >> have
> >> >>>>>>>> tried to make them more readable to ensure that we're covering
> >> all
> >> >> the
> >> >>>>>>>> scenarios. I think that if we can get more eyes on the workflow
> >> >>>> showing
> >> >>>>>>> the
> >> >>>>>>>> various LHS and RHS events and outputs then that may help us
> >> >> validate
> >> >>>>>>> that
> >> >>>>>>>> we have all the scenarios covered.
> >> >>>>>>>>
> >> >>>>>>>> With regards to the 2.3->2.4 scenario you described, I'm not
> >> >> entirely
> >> >>>>>>> sure
> >> >>>>>>>> I follow. If they want to add a FK-join, they will need to rework
> >> >>>> their
> >> >>>>>>>> code in the KStreams app and make a new release, since the
> >> >> underlying
> >> >>>>>>>> topology would be different and new internal topics would need
> >> to be
> >> >>>>>>>> created. In other words, I don't think a rolling upgrade where
> >> the
> >> >>>> user
> >> >>>>>>>> introduces a FK join would be possible since their topology would
> >> >>>>>>>> necessitate a full KStreams reset. Is this what you meant?
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>>
> >> >>>>>>>> On Wed, Jun 26, 2019 at 4:10 PM John Roesler <jo...@confluent.io>
> >> >>>> wrote:
> >> >>>>>>>>
> >> >>>>>>>>> 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.bellemare@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.bellemare@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 <
> >> john@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
> >> >>>>>>>>>>>> <ad...@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 <
> >> >>>>>>> john@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
> >> >>>>>>>>>>>>>>
> >> >>>>>>>>>>>>
> >> >>>>>>>>>>>
> >> >>>>>>>>>
> >> >>>>>>>
> >> >>>>
> >> >>>>
> >> >>>
> >> >>
> >> >>
> >> >
> >>
> >>


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

Posted by Adam Bellemare <ad...@gmail.com>.
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-handed.

2) A very deft solution is to also just support left-joins but not
inner-joins. Practically speaking, either INNER or LEFT join as it
currently is would support all of my use-cases.

3) Accept that there may be some null tombstones (though this makes me want
to just go with LEFT only instead of LEFT and PSEUDO-INNER).

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 experience from our usage of our internal
implementation at my company, and that of many others who have reached out
to me.

What would help most at this point is if someone can come up with a
scenario where sending unnecessary tombstones actually poses a downstream
problem beyond that of confusing behaviour, as I cannot think of one
myself.  With that being said, I am far more inclined to actually then
support just option #2 above and only have LEFT joins, forgoing INNER
completely since it would not be a true inner join.

Adam








On Thu, Jul 4, 2019 at 8:50 AM Adam Bellemare <ad...@gmail.com>
wrote:

> 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,hashOf(b),PROPAGATE_OLD_VALUE) hits RHS-0, the following
> occurs:
>   1) Store the current (CombinedKey<FK,K>, Value=(Hash, ForeignValue)) in
> a variable
>   2) Delete the key from the store
>   3) Publish the event from step A-1 downstream with an instruction:
> (eventType = COMPARE_TO_OTHER) (or whatever)
> *      (key, (hashOf(b),wasForeignValueNull, eventType))*
>     //Don't need the old hashOf(b) as it is guaranteed to be out of date
>     //We do need the hashOf(b) that came with the event to be passed
> along. Will be used in resolution.
>     //Don't need the actual value as we aren't joining or comparing the
> values, just using it to determine nulls. Reduces payload size.
>
> B) When (k,b,PROPAGATE_NEW_VALUE) hits RHS-1, the following occurs:
>   1) Store it in the prefix state store (as we currently do)
>   2) Get the FK-value (as we currently do)
>   3) Return the normal SubscriptionResponse payload (eventType = UPDATE)
> (or whatever)
> *     (key, (hashOf(b), foreignValue, eventType))*
>
>
> C) The Resolver Table is keyed on (as per our example):
> key = CombinedKey<k-hash(b)>, value =
> NullValueResolution<wasForeignValueNull (set by RHS-0), foreignValue (set
> by RHS-1)>
>
> Resolution Steps per event:
>
> When one of either the output events from A (eventType ==
> COMPARE_TO_OTHER) or B (eventType == UPDATE) is received
> 1) Check if this event matches the current hashOf(b). If not, discard it,
> for it is stale and no longer matters.  Additionally, delete entry
> CombinedKey<k-hash(b)> from the Resolver Table.
>
> 2) Lookup event in table on its CombinedKey:
>   - If it's not in the table, create the  NullValueResolution value,
> populate the field related to the eventType, and add it to the table.
>   - If it already IS in the table, get the existing NullValueResolution
> object and finish populating it:
>
> 3) If the NullValueResolution is fully populated, move on to the
> resolution logic below.
>
> Format:
> (wasForeignValueNull, foreignValue) -> Result
> If:
> ( false  , Null ) -> Send tombstone. Old value was not null, new one is,
> send tombstone.
> (  true  , Null ) -> Do nothing.  See * below for more details.
> (  true  , NewValue ) -> Send the new result
> (  true  , NewValue ) -> Send the new result
>
> * wasForeignValueNull may have been false at some very recent point, but
> only just translated to true (race condition). In this case, the RHS table
> was updated and the value was set to null due to a an RHS update of (a,
> oldVal) -> (a, null). This event on its own will propagate a delete event
> through to the resolver (of a different eventType), so we don't need to
> handle this case from the LHS and doing nothing is OK.
>
> In the case that it's truly (true, Null), we also don't need to send a
> tombstone because wasForeignKeyNull == true means that a tombstone was
> previously sent.
>
> 4) Check the hashOf(b) one last time before sending the resolved message
> out. If the hash is old, discard it.
>
> 5) Delete the row from the Resolver Table.
>
>
> Takeaways:
> 1) I believe this is only necessary for INNER joins when we transition
> from (k, non-Null-val) -> (k, non-Null-new-val)
> 2) We can maintain state until we get both events back from RHS-0 and
> RHS-1, at which point we delete it and clean up.
>   NOTE: I think this still works with at-least-once processing, but I am
> not 100% on this. The concern is that we receive events from RHS-0, RHS-1
> (causing a row deletion), then RHS-1 again from at-least once (creating an
> entry that never gets deleted).
> 3) My naive implementation requires SubscriptionResponseWrapper event
> types.
> 4) This shouldn't affect performance much, as it should only be
> maxOf(RHS-0, RHS-1) event propagation and processing.
>
>
> Adam
>
> On Wed, Jul 3, 2019 at 2:18 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
>
>> 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, but want to throw out the idea first.
>>
>> > S-2 could probably be simplified to "for a given key, was the previous
>> > propagated result a null/tombstone or not?"
>>
>> Yes, that's the idea.
>>
>> > However,
>> > the table will grow indefinitely large as we can never remove keys from
>> it.
>>
>> That a good point. I missed that... Seems that this rules out S-2
>> because if we cannot provide _strict_ guarantees, it does not seem to be
>> worth to even try?
>>
>>
>> About the impact of unnecessary tombstones: There is a performance
>> impact, as we increase the output data rate, potentially broker load if
>> the result is written into a topic, and also increase downstream
>> processing load. It's hard to judge, how much the overhead will be, as
>> it will depend on the selectivity of the join. But it might be
>> significant?
>>
>> Also, users have certain expectations on the result and it's unintuitive
>> (even if semantically correct) to send those tombstones. From
>> experience, we often have a hard time to explain semantics to people and
>> I was hoping we could avoid introducing unintuitive behavior.
>>
>>
>> Would be good to get input from others and how they judge the impact. I
>> think it might be still worth to explore how complex S-1 would be. If we
>> think it's too complex it might be a good argument to just accept the
>> unnecessary tombstones?
>>
>>
>> -Matthias
>>
>>
>> On 7/3/19 8:03 AM, Adam Bellemare wrote:
>> > 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 seems to me that the processor would need to block
>> > while it waits for the unsubscribe to propagate and return, which would
>> > cause throughput to drop significantly. Alternately, we would need to
>> > maintain state anyways about which events were sent and which responses
>> > returned, while being sure to respect the offset order in which they're
>> > emitted. I think this would only reduce blocking slightly while
>> increasing
>> > complexity. If I am wrong in understanding this, please let me know
>> where
>> > my thinking is erroneous.
>> >
>> > S-2 could probably be simplified to "for a given key, was the previous
>> > propagated result a null/tombstone or not?"
>> > It would act very similarly to the hash value mechanism, where we
>> discard
>> > any events that are not of the correct hash. In this case, we simply
>> store
>> > (key, wasLastOutputATombstone) right before the event is output
>> downstream
>> > of the Join + Resolver. This ignores all the complexities of which
>> event is
>> > propagating over which wire and simply squelches any extra tombstones
>> from
>> > being sent.
>> >
>> > For storage, we need to use the full primary key and a boolean. However,
>> > the table will grow indefinitely large as we can never remove keys from
>> it.
>> > If we delete key=k from the table and propagate a tombstone, but later
>> (say
>> > 3 weeks, 3 months, etc) we publish (k, baz), but baz does not exist on
>> the
>> > RHS, we will end up publishing an extra tombstone because we have no
>> idea
>> > what the previously sent record was for k. For this reason I think it's
>> > worth asking if we really can maintain state, and if it's even necessary
>> > (again, a full understanding of the impact of extra tombstones may help
>> us
>> > figure out a better solution).
>> >
>> > As it stands, I don't think either of these will work well. That being
>> > said, I myself do not have any better ideas at the moment, but I would
>> > still like to better understand circumstances where it has a negative
>> > impact downstream as that may provide some insights.
>> >
>> >
>> > Thanks
>> >
>> > Adam
>> >
>> >
>> >
>> >
>> >
>> > On Tue, Jul 2, 2019 at 11:18 PM Matthias J. Sax <ma...@confluent.io>
>> > wrote:
>> >
>> >> 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 subscribe message to RHS
>> >> a-2) RHS lookup and send result back if there is one
>> >> a-3) emit result on LHS if any is returned
>> >>
>> >> b-1) delete subscription from RHS
>> >> b-2) if there was a previous result (can easily be decided by looking
>> up
>> >> RHS table for an existing key), send tombstone back
>> >> b-3) emit tombstone on LHS if any is returned
>> >>
>> >> c-1) delete old subscription from RHS
>> >> c-2) send new subscription to RHS
>> >> c-3) if there was no previous result and there is no new result emit
>> >> nothing
>> >> c-4) if there was a previous result and there is no new result emit a
>> >> tombstone LHS
>> >> c-5) if there is a new result (old result may or may not exist), emit
>> >> only new result LHS (don't emit a tombstone)
>> >>
>> >>
>> >> Case (a) and (b) are simple and could be implemented with a "fire and
>> >> forget" strategy. The LHS just "blindly" updates the subscription, the
>> >> RHS can process the subscription with local knowledge and may send
>> >> something back. If something is sent back, the LHS blindly emits it.
>> >>
>> >> We know that for both cases, we never miss anything and we never emit
>> >> anything unnecessary.
>> >>
>> >> However, using this pattern for (c), we don't get our expected result:
>> >>
>> >> Issues: LHS sends both subscription updates in parallel. It does not
>> >> know if zero, one, or two result records will be produced by RHS. If
>> RHS
>> >> produces two results, their order is not known (however, LHS would need
>> >> to emit them in the right order; also forcing RHS to always send a
>> >> result back is not a sufficient solution). If only one result comes
>> >> back, it's unclear if a second result may arrive later and thus the
>> >> result may need to be buffered... Overall, local knowledge does not
>> seem
>> >> to be sufficient to tackle the case.
>> >>
>> >> The current proposal tries to address the issue with encoding
>> additional
>> >> information, to tell the RHS to send a tombstone back or not. But it
>> >> does not seem to be perfect, and it might result in unnecessary
>> >> tombstones as it still uses local knowledge only and thus misses some
>> >> information.
>> >>
>> >> I think the main problem is, that the knowledge about the a potential
>> >> previous result and a potential new result is sharded on the RHS.
>> Hence,
>> >> the "unsubscribe" does not know if it needs to send a tombstone back
>> for
>> >> the case that there was an old result but there is no new result.
>> >> Similarly, the "new subscribe" cannot know if it needs to send a
>> >> tombstone or not (as it does not know if there was a previous result)
>> if
>> >> it does not match.
>> >>
>> >> To really solve the issue, I see two possible solutions (both are not
>> >> great, but I wanted to discuss them anyway):
>> >>
>> >> S-1: First unsubscribe, and send new subscription after result comes
>> >> back. For this case, the RHS must always send something back to the LHS
>> >> on unsubscribe. The answer if "previous result exists/not-exist" can be
>> >> added to the new-subscription and hence RHS can either return nothing,
>> a
>> >> tombstone, or a new result. The LHS can blindly emit whatever RHS
>> >> returns. This would also cover (a) and (b) cases. However, the overall
>> >> time to emit the join result is doubled for the (common) update case...
>> >> (we need two consecutive round-trips to the RHS).
>> >>
>> >> S-2: Remember/store if a previous result exists on LHS: for this case,
>> >> (a) is handled straightforward, (b) is handled by telling RHS to send
>> >> tombstone if previous result exits, and (c) can send both request in
>> >> parallel letting the unsubscribe never return anything, and subscribe
>> is
>> >> handled as in (b). However, we need a second store on the LHS to
>> >> remember if there was a previous result. (Also not sure how
>> >> interleaving/inflight computation might affect the algorithm...)
>> >>
>> >> I think, sending unnecessary tombstones is quite bad (in very old
>> >> releases we had a similar issue and fixed it). However, I am also not
>> >> 100% sure if the solutions I came up with are good enough to justify
>> >> them. (Personally, I slightly tend to prefer S-2 because I think that
>> >> the additional store is less of an issue than the increase processing
>> >> time).
>> >>
>> >> Would love to hear your thoughts.
>> >>
>> >>
>> >> -Matthias
>> >>
>> >>
>> >> On 6/28/19 6:19 AM, Adam Bellemare wrote:
>> >>> 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 unnecessary tombstone is (that I can
>> >>> tell...) when we do the following:
>> >>> RHS:
>> >>> (1, bar)
>> >>>
>> >>> LHS
>> >>> (K,1)  -> Results in (K, 1, bar) being output
>> >>> (K,1) -> (K,2) ->  Results in (K, null) being output for INNER (no
>> >> matching
>> >>> element on LHS)
>> >>> (K,2) -> (K,3) ->  Results in (K, null) being output for INNER
>> (because
>> >> we
>> >>> don't maintain state to know we already output the tombstone on the
>> >>> previous transition).
>> >>> (K,2) -> (K,9000) ->  Results in (K, null)... etc.
>> >>>
>> >>> Byte versioning is going in today, then I hope to get back to
>> addressing
>> >> a
>> >>> number of John's previous questions in the PR.
>> >>>
>> >>> Adam
>> >>>
>> >>>
>> >>> On Thu, Jun 27, 2019 at 5:47 PM Matthias J. Sax <
>> matthias@confluent.io>
>> >>> wrote:
>> >>>
>> >>>> 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 unnecessary tombstone? Seems I a missing case?
>> >>>>
>> >>>> Also for my own understanding: the `instruction` is only part of the
>> >>>> message? It is no necessary to store it in the RHS auxiliary store,
>> >> right?
>> >>>>
>> >>>> About right/full-outer joins. Agreed. Getting left-joins would be
>> >> awesome!
>> >>>>
>> >>>> About upgrading: Good call John! Adding a version byte for
>> subscription
>> >>>> and response is good forward thinking. I personally prefer version
>> >>>> numbers, too, as they carry more information.
>> >>>>
>> >>>> Thanks for all the hard to everybody involved!
>> >>>>
>> >>>>
>> >>>> -Matthias
>> >>>>
>> >>>> On 6/27/19 1:44 PM, John Roesler wrote:
>> >>>>> 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
>> revisions,
>> >>>>> we can just use one byte. The worst case is that we run out of
>> numbers
>> >>>>> and reserve the last one to mean, "consult another field for the
>> >>>>> actual version number". It seems like a single byte on each message
>> >>>>> isn't too much to pay.
>> >>>>>
>> >>>>> Since you point it out, we might as well put a version number on the
>> >>>>> SubscriptionResponseWrapper as well. It may not be needed, but if we
>> >>>>> ever need it, even just once, we'll be glad we have it.
>> >>>>>
>> >>>>> Regarding the instructions field, we can also serialize the enum
>> very
>> >>>>> compactly as a single byte (which is the same size a boolean takes
>> >>>>> anyway), so it seems like an Enum in Java-land and a byte on the
>> wire
>> >>>>> is a good choice.
>> >>>>>
>> >>>>> Agreed on the right and full outer joins, it doesn't seem necessary
>> >>>>> right now, although I am happy to see the left join "join" the
>> party,
>> >>>>> since as you said, we were so close to it anyway. Can you also add
>> it
>> >>>>> to the KIP?
>> >>>>>
>> >>>>> Thanks as always for your awesome efforts on this,
>> >>>>> -John
>> >>>>>
>> >>>>> On Thu, Jun 27, 2019 at 3:04 PM Adam Bellemare <
>> >> adam.bellemare@gmail.com>
>> >>>> wrote:
>> >>>>>>
>> >>>>>> 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 microservices -
>> there
>> >> are
>> >>>>>> indeed cases where a breaking change must be made, and forward
>> >>>>>> compatibility will provide us with no out other than requiring a
>> full
>> >>>> stop
>> >>>>>> and full upgrade for all nodes, shifting us back towards STRATEGY
>> 1.
>> >>>>>>
>> >>>>>> My preference is STRATEGY 1 with instructions as an ENUM, and we
>> can
>> >>>>>> certainly include a version. Would it make sense to include a
>> version
>> >>>>>> number in  SubscriptionResponseWrapper as well? Currently we don't
>> >> have
>> >>>> any
>> >>>>>> instructions in there, as I removed the boolean, but it is
>> certainly
>> >>>>>> plausible that it could happen in the future. I don't *think* we'll
>> >> need
>> >>>>>> it, but I also didn't think we'd need it for SubscriptionWrapper
>> and
>> >>>> here
>> >>>>>> we are.
>> >>>>>>
>> >>>>>> Thanks for the thoughts, and the info on the right-key. That was
>> >>>>>> enlightening, though I can't think of a use-case for it *at this
>> point
>> >>>> in
>> >>>>>> time*. :)
>> >>>>>>
>> >>>>>> Adam
>> >>>>>>
>> >>>>>>
>> >>>>>>
>> >>>>>> On Thu, Jun 27, 2019 at 12:29 PM John Roesler <jo...@confluent.io>
>> >>>> wrote:
>> >>>>>>
>> >>>>>>> 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 `nullKey: (null, rhsValue)`, but we
>> >>>>>>> don't currently allow null keys in Streams. It actually is
>> possible
>> >> to
>> >>>>>>> define them, and therefore to add right- and full-outer
>> foreign-key
>> >>>>>>> joins later, but it's non-trivial in a streaming context with
>> >>>>>>> continuously updated results. (See the PS if you're curious what
>> I'm
>> >>>>>>> thinking). You're correct, right- and full-outer joins are
>> trivial on
>> >>>>>>> our current 1:1 table joins because they are equi-joins.
>> >>>>>>>
>> >>>>>>> Regarding the transition, it sounds like what you're proposing is
>> >> that
>> >>>>>>> we would say, "adding a foreign-key join to your topology
>> requires a
>> >>>>>>> full application reset (or a new application id)". This is also an
>> >>>>>>> acceptable constraint to place on the feature, but not strictly
>> >>>>>>> necessary. Since 2.3, it's now possible to give all the state in
>> your
>> >>>>>>> application stable names. This means that it's no longer true that
>> >>>>>>> adding a node to your topology graph would break its structure,
>> and
>> >> it
>> >>>>>>> does become possible to add new operators and simply restart the
>> app.
>> >>>>>>> Revisiting my prior thought, though, I think the problem is not
>> >>>>>>> specific to your feature. For example, adding a new grouped
>> >>>>>>> aggregation would produce a new repartition topic, but the
>> >> repartition
>> >>>>>>> topic partitions might get assigned to old nodes in the middle of
>> a
>> >>>>>>> rolling bounce, and they would need to just ignore them. This
>> >>>>>>> requirement is the same for the repartition topics in the FK
>> join, so
>> >>>>>>> it's orthogonal to your design.
>> >>>>>>>
>> >>>>>>> Back to the first concern, though, I'm not sure I followed the
>> >>>>>>> explanation. As a thought experiment, let's imagine that Joe
>> hadn't
>> >>>>>>> taken the time to experiment with your feature branch. We wouldn't
>> >>>>>>> have noticed the problem until the feature was already released in
>> >>>>>>> 2.4. So the wire protocol on that PK->FK subscription topic would
>> >> have
>> >>>>>>> been V1: "FK,PK,HASH,BOOLEAN". Then, Joe would have let us know
>> the
>> >>>>>>> problem once they picked up the feature, so we would want to
>> >> implement
>> >>>>>>> your proposed fix and change the wire protocol to V2:
>> >>>>>>> "FK,PK,HASH,INSTRUCTIONS" in 2.5. Upon rolling out the update, we
>> >>>>>>> would see both 2.4 nodes encountering V2 messages and 2.5 nodes
>> >>>>>>> encountering V1 messages. How can they both detect that they are
>> >>>>>>> attempting to process a newer or older protocol? If they can
>> detect
>> >>>>>>> it, then what should they do?
>> >>>>>>>
>> >>>>>>> From experience, there are two basic solutions to this problem:
>> >>>>>>>
>> >>>>>>> STRATEGY1. Add a protocol version to the message (could be a
>> number
>> >> at
>> >>>>>>> the start of the message payload, or it could be a number in the
>> >>>>>>> message headers, not sure if it matters much. Payload is probably
>> >> more
>> >>>>>>> compact, since the header would need a name.) In this case, the
>> 2.4
>> >>>>>>> worker would know that it's max protocol version is V1, and when
>> it
>> >>>>>>> sees the V2 message, it knows that it can't handle it properly.
>> >> Rather
>> >>>>>>> than doing something wrong, it would just not do anything. This
>> means
>> >>>>>>> it would stop the task, if not shut down the whole instance. On
>> the
>> >>>>>>> other hand, a 2.5 worker would have some defined logic for how to
>> >>>>>>> handle all versions (V1 and V2), so once the upgrade is complete,
>> all
>> >>>>>>> messages can be processed.
>> >>>>>>>
>> >>>>>>> STRATEGY2. Make the schema forward-compatible. Basically, we
>> ensure
>> >>>>>>> that new fields can only be appended to the message schema, and
>> that
>> >>>>>>> older workers using only a prefix of the full message would still
>> >>>>>>> behave correctly. Using the example above, we'd instead evolve the
>> >>>>>>> schema to V2': "FK,PK,HASH,BOOLEAN,INSTRUCTIONS", and continue to
>> set
>> >>>>>>> the boolean field to true for the "new" foreign key. Then, 2.4
>> >> workers
>> >>>>>>> encountering the a "new FK" message would just see the prefix of
>> the
>> >>>>>>> payload that makes sense to them, and they would still continue
>> >>>>>>> processing the messages as they always have. Only after the 2.5
>> code
>> >>>>>>> is fully rolled out to the cluster would we be sure to see the
>> >> desired
>> >>>>>>> behavior. Note: in the reverse case, a 2.5 worker knows how to
>> fully
>> >>>>>>> parse the new message format, even if it plans to ignore the
>> BOOLEAN
>> >>>>>>> field.
>> >>>>>>>
>> >>>>>>> There are some tradeoffs between these strategies: STRATEGY1
>> ensures
>> >>>>>>> that all messages are only handled by workers that can properly
>> >> handle
>> >>>>>>> them, although it results in processing stalls while there are
>> still
>> >>>>>>> old nodes in the cluster. STRATEGY2 ensures that all messages can
>> be
>> >>>>>>> processed by all nodes, so there are no stalls, but we can never
>> >>>>>>> remove fields from the message, so if there are a lot of
>> revisions in
>> >>>>>>> the future, the payloads will become bloated. Also, it's not clear
>> >>>>>>> that you can actually pull off STRATEGY2 in all cases. If there's
>> >> some
>> >>>>>>> new kind of message you want to send that has no way to be
>> correctly
>> >>>>>>> processed at all under the 2.4 code paths, the prefix thing simply
>> >>>>>>> doesn't work. Etc.
>> >>>>>>>
>> >>>>>>> Also, note that you can modify the above strategies by instead
>> >>>>>>> designing the message fields for extensibility. E.g., if you make
>> the
>> >>>>>>> instructions field an enum, then you can make sure that the
>> default
>> >>>>>>> case is handled sensibly (probably similarly to STRATEGY1, just
>> choke
>> >>>>>>> on unknown instructions) and that you never remove an instruction
>> >> type
>> >>>>>>> from the enum in future versions.
>> >>>>>>>
>> >>>>>>> Does this make sense?
>> >>>>>>> -John
>> >>>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>>
>> >>>>>>> PS:
>> >>>>>>> We can define null keys for streaming tables, but it's tricky.
>> >>>>>>>
>> >>>>>>> Specifically, you'd want to define some concept of null keys that
>> >>>>>>> allows all null keys to be unique, but _also_ to have a fixed
>> >>>>>>> identity, so that a particular null-key can be updated later. One
>> >>>>>>> example could be to union the existing keyspace with a new
>> >>>>>>> null-keyspace, where normal keys are like "key" and null-keys are
>> >> like
>> >>>>>>> "null(identity)". Then given a query like
>> >>>>>>> "KTable<String,Integer>.rightJoin(KTable<Integer,Boolean>)", and
>> >>>>>>> inputs like:
>> >>>>>>> LHS:
>> >>>>>>> "a": 1
>> >>>>>>> "b": 2
>> >>>>>>>
>> >>>>>>> RHS:
>> >>>>>>> 1: true
>> >>>>>>> 3: false
>> >>>>>>>
>> >>>>>>> a full outer join would produce:
>> >>>>>>> "a": (1, true)
>> >>>>>>> "b": (2, null)
>> >>>>>>> null(3): (null, false)
>> >>>>>>>
>> >>>>>>> which can be correctly updated later if we get an update on the
>> LHS:
>> >>>>>>> PUT("c": 3)
>> >>>>>>>
>> >>>>>>> We'd emit for the results:
>> >>>>>>> DELETE(null(e))
>> >>>>>>> EMIT("c": (3, false))
>> >>>>>>>
>> >>>>>>> Resulting in the correct result table of:
>> >>>>>>> "a": (1, true)
>> >>>>>>> "b": (2, null)
>> >>>>>>> "c": (3, false)
>> >>>>>>>
>> >>>>>>> As mentioned, this is tricky, and I would avoid it until we have
>> >>>>>>> evidence that it's actually useful to cover this part of the
>> design
>> >>>>>>> space. Certainly, it would be a separate KIP if it came to that.
>> >>>>>>>
>> >>>>>>> On Wed, Jun 26, 2019 at 8:57 PM Adam Bellemare <
>> >>>> adam.bellemare@gmail.com>
>> >>>>>>> wrote:
>> >>>>>>>>
>> >>>>>>>> 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
>> >> the
>> >>>>>>> last
>> >>>>>>>> commit (about 10 min ago) along with INNER join. I do not think
>> that
>> >>>>>>> RIGHT
>> >>>>>>>> join and OUTER are possible given that there is no LHS key
>> >> available,
>> >>>> so
>> >>>>>>>> LHSTable.outerJoinOnForeignKey(RHSTable) wouldn't even make
>> sense.
>> >>>> This
>> >>>>>>> is
>> >>>>>>>> in contrast to the current LHSTable.outerJoin(RHSTable), as they
>> are
>> >>>> both
>> >>>>>>>> keyed on the same key. I have buffed up the Integration tests and
>> >> have
>> >>>>>>>> tried to make them more readable to ensure that we're covering
>> all
>> >> the
>> >>>>>>>> scenarios. I think that if we can get more eyes on the workflow
>> >>>> showing
>> >>>>>>> the
>> >>>>>>>> various LHS and RHS events and outputs then that may help us
>> >> validate
>> >>>>>>> that
>> >>>>>>>> we have all the scenarios covered.
>> >>>>>>>>
>> >>>>>>>> With regards to the 2.3->2.4 scenario you described, I'm not
>> >> entirely
>> >>>>>>> sure
>> >>>>>>>> I follow. If they want to add a FK-join, they will need to rework
>> >>>> their
>> >>>>>>>> code in the KStreams app and make a new release, since the
>> >> underlying
>> >>>>>>>> topology would be different and new internal topics would need
>> to be
>> >>>>>>>> created. In other words, I don't think a rolling upgrade where
>> the
>> >>>> user
>> >>>>>>>> introduces a FK join would be possible since their topology would
>> >>>>>>>> necessitate a full KStreams reset. Is this what you meant?
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>>
>> >>>>>>>> On Wed, Jun 26, 2019 at 4:10 PM John Roesler <jo...@confluent.io>
>> >>>> wrote:
>> >>>>>>>>
>> >>>>>>>>> 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.bellemare@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.bellemare@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 <
>> john@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
>> >>>>>>>>>>>> <ad...@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 <
>> >>>>>>> john@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
>> >>>>>>>>>>>>>>
>> >>>>>>>>>>>>
>> >>>>>>>>>>>
>> >>>>>>>>>
>> >>>>>>>
>> >>>>
>> >>>>
>> >>>
>> >>
>> >>
>> >
>>
>>

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

Posted by Adam Bellemare <ad...@gmail.com>.
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,hashOf(b),PROPAGATE_OLD_VALUE) hits RHS-0, the following
occurs:
  1) Store the current (CombinedKey<FK,K>, Value=(Hash, ForeignValue)) in a
variable
  2) Delete the key from the store
  3) Publish the event from step A-1 downstream with an instruction:
(eventType = COMPARE_TO_OTHER) (or whatever)
*      (key, (hashOf(b),wasForeignValueNull, eventType))*
    //Don't need the old hashOf(b) as it is guaranteed to be out of date
    //We do need the hashOf(b) that came with the event to be passed along.
Will be used in resolution.
    //Don't need the actual value as we aren't joining or comparing the
values, just using it to determine nulls. Reduces payload size.

B) When (k,b,PROPAGATE_NEW_VALUE) hits RHS-1, the following occurs:
  1) Store it in the prefix state store (as we currently do)
  2) Get the FK-value (as we currently do)
  3) Return the normal SubscriptionResponse payload (eventType = UPDATE)
(or whatever)
*     (key, (hashOf(b), foreignValue, eventType))*


C) The Resolver Table is keyed on (as per our example):
key = CombinedKey<k-hash(b)>, value =
NullValueResolution<wasForeignValueNull (set by RHS-0), foreignValue (set
by RHS-1)>

Resolution Steps per event:

When one of either the output events from A (eventType == COMPARE_TO_OTHER)
or B (eventType == UPDATE) is received
1) Check if this event matches the current hashOf(b). If not, discard it,
for it is stale and no longer matters.  Additionally, delete entry
CombinedKey<k-hash(b)> from the Resolver Table.

2) Lookup event in table on its CombinedKey:
  - If it's not in the table, create the  NullValueResolution value,
populate the field related to the eventType, and add it to the table.
  - If it already IS in the table, get the existing NullValueResolution
object and finish populating it:

3) If the NullValueResolution is fully populated, move on to the resolution
logic below.

Format:
(wasForeignValueNull, foreignValue) -> Result
If:
( false  , Null ) -> Send tombstone. Old value was not null, new one is,
send tombstone.
(  true  , Null ) -> Do nothing.  See * below for more details.
(  true  , NewValue ) -> Send the new result
(  true  , NewValue ) -> Send the new result

* wasForeignValueNull may have been false at some very recent point, but
only just translated to true (race condition). In this case, the RHS table
was updated and the value was set to null due to a an RHS update of (a,
oldVal) -> (a, null). This event on its own will propagate a delete event
through to the resolver (of a different eventType), so we don't need to
handle this case from the LHS and doing nothing is OK.

In the case that it's truly (true, Null), we also don't need to send a
tombstone because wasForeignKeyNull == true means that a tombstone was
previously sent.

4) Check the hashOf(b) one last time before sending the resolved message
out. If the hash is old, discard it.

5) Delete the row from the Resolver Table.


Takeaways:
1) I believe this is only necessary for INNER joins when we transition from
(k, non-Null-val) -> (k, non-Null-new-val)
2) We can maintain state until we get both events back from RHS-0 and
RHS-1, at which point we delete it and clean up.
  NOTE: I think this still works with at-least-once processing, but I am
not 100% on this. The concern is that we receive events from RHS-0, RHS-1
(causing a row deletion), then RHS-1 again from at-least once (creating an
entry that never gets deleted).
3) My naive implementation requires SubscriptionResponseWrapper event types.
4) This shouldn't affect performance much, as it should only be
maxOf(RHS-0, RHS-1) event propagation and processing.


Adam

On Wed, Jul 3, 2019 at 2:18 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> 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, but want to throw out the idea first.
>
> > S-2 could probably be simplified to "for a given key, was the previous
> > propagated result a null/tombstone or not?"
>
> Yes, that's the idea.
>
> > However,
> > the table will grow indefinitely large as we can never remove keys from
> it.
>
> That a good point. I missed that... Seems that this rules out S-2
> because if we cannot provide _strict_ guarantees, it does not seem to be
> worth to even try?
>
>
> About the impact of unnecessary tombstones: There is a performance
> impact, as we increase the output data rate, potentially broker load if
> the result is written into a topic, and also increase downstream
> processing load. It's hard to judge, how much the overhead will be, as
> it will depend on the selectivity of the join. But it might be significant?
>
> Also, users have certain expectations on the result and it's unintuitive
> (even if semantically correct) to send those tombstones. From
> experience, we often have a hard time to explain semantics to people and
> I was hoping we could avoid introducing unintuitive behavior.
>
>
> Would be good to get input from others and how they judge the impact. I
> think it might be still worth to explore how complex S-1 would be. If we
> think it's too complex it might be a good argument to just accept the
> unnecessary tombstones?
>
>
> -Matthias
>
>
> On 7/3/19 8:03 AM, Adam Bellemare wrote:
> > 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 seems to me that the processor would need to block
> > while it waits for the unsubscribe to propagate and return, which would
> > cause throughput to drop significantly. Alternately, we would need to
> > maintain state anyways about which events were sent and which responses
> > returned, while being sure to respect the offset order in which they're
> > emitted. I think this would only reduce blocking slightly while
> increasing
> > complexity. If I am wrong in understanding this, please let me know where
> > my thinking is erroneous.
> >
> > S-2 could probably be simplified to "for a given key, was the previous
> > propagated result a null/tombstone or not?"
> > It would act very similarly to the hash value mechanism, where we discard
> > any events that are not of the correct hash. In this case, we simply
> store
> > (key, wasLastOutputATombstone) right before the event is output
> downstream
> > of the Join + Resolver. This ignores all the complexities of which event
> is
> > propagating over which wire and simply squelches any extra tombstones
> from
> > being sent.
> >
> > For storage, we need to use the full primary key and a boolean. However,
> > the table will grow indefinitely large as we can never remove keys from
> it.
> > If we delete key=k from the table and propagate a tombstone, but later
> (say
> > 3 weeks, 3 months, etc) we publish (k, baz), but baz does not exist on
> the
> > RHS, we will end up publishing an extra tombstone because we have no idea
> > what the previously sent record was for k. For this reason I think it's
> > worth asking if we really can maintain state, and if it's even necessary
> > (again, a full understanding of the impact of extra tombstones may help
> us
> > figure out a better solution).
> >
> > As it stands, I don't think either of these will work well. That being
> > said, I myself do not have any better ideas at the moment, but I would
> > still like to better understand circumstances where it has a negative
> > impact downstream as that may provide some insights.
> >
> >
> > Thanks
> >
> > Adam
> >
> >
> >
> >
> >
> > On Tue, Jul 2, 2019 at 11:18 PM Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> >> 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 subscribe message to RHS
> >> a-2) RHS lookup and send result back if there is one
> >> a-3) emit result on LHS if any is returned
> >>
> >> b-1) delete subscription from RHS
> >> b-2) if there was a previous result (can easily be decided by looking up
> >> RHS table for an existing key), send tombstone back
> >> b-3) emit tombstone on LHS if any is returned
> >>
> >> c-1) delete old subscription from RHS
> >> c-2) send new subscription to RHS
> >> c-3) if there was no previous result and there is no new result emit
> >> nothing
> >> c-4) if there was a previous result and there is no new result emit a
> >> tombstone LHS
> >> c-5) if there is a new result (old result may or may not exist), emit
> >> only new result LHS (don't emit a tombstone)
> >>
> >>
> >> Case (a) and (b) are simple and could be implemented with a "fire and
> >> forget" strategy. The LHS just "blindly" updates the subscription, the
> >> RHS can process the subscription with local knowledge and may send
> >> something back. If something is sent back, the LHS blindly emits it.
> >>
> >> We know that for both cases, we never miss anything and we never emit
> >> anything unnecessary.
> >>
> >> However, using this pattern for (c), we don't get our expected result:
> >>
> >> Issues: LHS sends both subscription updates in parallel. It does not
> >> know if zero, one, or two result records will be produced by RHS. If RHS
> >> produces two results, their order is not known (however, LHS would need
> >> to emit them in the right order; also forcing RHS to always send a
> >> result back is not a sufficient solution). If only one result comes
> >> back, it's unclear if a second result may arrive later and thus the
> >> result may need to be buffered... Overall, local knowledge does not seem
> >> to be sufficient to tackle the case.
> >>
> >> The current proposal tries to address the issue with encoding additional
> >> information, to tell the RHS to send a tombstone back or not. But it
> >> does not seem to be perfect, and it might result in unnecessary
> >> tombstones as it still uses local knowledge only and thus misses some
> >> information.
> >>
> >> I think the main problem is, that the knowledge about the a potential
> >> previous result and a potential new result is sharded on the RHS. Hence,
> >> the "unsubscribe" does not know if it needs to send a tombstone back for
> >> the case that there was an old result but there is no new result.
> >> Similarly, the "new subscribe" cannot know if it needs to send a
> >> tombstone or not (as it does not know if there was a previous result) if
> >> it does not match.
> >>
> >> To really solve the issue, I see two possible solutions (both are not
> >> great, but I wanted to discuss them anyway):
> >>
> >> S-1: First unsubscribe, and send new subscription after result comes
> >> back. For this case, the RHS must always send something back to the LHS
> >> on unsubscribe. The answer if "previous result exists/not-exist" can be
> >> added to the new-subscription and hence RHS can either return nothing, a
> >> tombstone, or a new result. The LHS can blindly emit whatever RHS
> >> returns. This would also cover (a) and (b) cases. However, the overall
> >> time to emit the join result is doubled for the (common) update case...
> >> (we need two consecutive round-trips to the RHS).
> >>
> >> S-2: Remember/store if a previous result exists on LHS: for this case,
> >> (a) is handled straightforward, (b) is handled by telling RHS to send
> >> tombstone if previous result exits, and (c) can send both request in
> >> parallel letting the unsubscribe never return anything, and subscribe is
> >> handled as in (b). However, we need a second store on the LHS to
> >> remember if there was a previous result. (Also not sure how
> >> interleaving/inflight computation might affect the algorithm...)
> >>
> >> I think, sending unnecessary tombstones is quite bad (in very old
> >> releases we had a similar issue and fixed it). However, I am also not
> >> 100% sure if the solutions I came up with are good enough to justify
> >> them. (Personally, I slightly tend to prefer S-2 because I think that
> >> the additional store is less of an issue than the increase processing
> >> time).
> >>
> >> Would love to hear your thoughts.
> >>
> >>
> >> -Matthias
> >>
> >>
> >> On 6/28/19 6:19 AM, Adam Bellemare wrote:
> >>> 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 unnecessary tombstone is (that I can
> >>> tell...) when we do the following:
> >>> RHS:
> >>> (1, bar)
> >>>
> >>> LHS
> >>> (K,1)  -> Results in (K, 1, bar) being output
> >>> (K,1) -> (K,2) ->  Results in (K, null) being output for INNER (no
> >> matching
> >>> element on LHS)
> >>> (K,2) -> (K,3) ->  Results in (K, null) being output for INNER (because
> >> we
> >>> don't maintain state to know we already output the tombstone on the
> >>> previous transition).
> >>> (K,2) -> (K,9000) ->  Results in (K, null)... etc.
> >>>
> >>> Byte versioning is going in today, then I hope to get back to
> addressing
> >> a
> >>> number of John's previous questions in the PR.
> >>>
> >>> Adam
> >>>
> >>>
> >>> On Thu, Jun 27, 2019 at 5:47 PM Matthias J. Sax <matthias@confluent.io
> >
> >>> wrote:
> >>>
> >>>> 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 unnecessary tombstone? Seems I a missing case?
> >>>>
> >>>> Also for my own understanding: the `instruction` is only part of the
> >>>> message? It is no necessary to store it in the RHS auxiliary store,
> >> right?
> >>>>
> >>>> About right/full-outer joins. Agreed. Getting left-joins would be
> >> awesome!
> >>>>
> >>>> About upgrading: Good call John! Adding a version byte for
> subscription
> >>>> and response is good forward thinking. I personally prefer version
> >>>> numbers, too, as they carry more information.
> >>>>
> >>>> Thanks for all the hard to everybody involved!
> >>>>
> >>>>
> >>>> -Matthias
> >>>>
> >>>> On 6/27/19 1:44 PM, John Roesler wrote:
> >>>>> 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
> revisions,
> >>>>> we can just use one byte. The worst case is that we run out of
> numbers
> >>>>> and reserve the last one to mean, "consult another field for the
> >>>>> actual version number". It seems like a single byte on each message
> >>>>> isn't too much to pay.
> >>>>>
> >>>>> Since you point it out, we might as well put a version number on the
> >>>>> SubscriptionResponseWrapper as well. It may not be needed, but if we
> >>>>> ever need it, even just once, we'll be glad we have it.
> >>>>>
> >>>>> Regarding the instructions field, we can also serialize the enum very
> >>>>> compactly as a single byte (which is the same size a boolean takes
> >>>>> anyway), so it seems like an Enum in Java-land and a byte on the wire
> >>>>> is a good choice.
> >>>>>
> >>>>> Agreed on the right and full outer joins, it doesn't seem necessary
> >>>>> right now, although I am happy to see the left join "join" the party,
> >>>>> since as you said, we were so close to it anyway. Can you also add it
> >>>>> to the KIP?
> >>>>>
> >>>>> Thanks as always for your awesome efforts on this,
> >>>>> -John
> >>>>>
> >>>>> On Thu, Jun 27, 2019 at 3:04 PM Adam Bellemare <
> >> adam.bellemare@gmail.com>
> >>>> wrote:
> >>>>>>
> >>>>>> 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 microservices - there
> >> are
> >>>>>> indeed cases where a breaking change must be made, and forward
> >>>>>> compatibility will provide us with no out other than requiring a
> full
> >>>> stop
> >>>>>> and full upgrade for all nodes, shifting us back towards STRATEGY 1.
> >>>>>>
> >>>>>> My preference is STRATEGY 1 with instructions as an ENUM, and we can
> >>>>>> certainly include a version. Would it make sense to include a
> version
> >>>>>> number in  SubscriptionResponseWrapper as well? Currently we don't
> >> have
> >>>> any
> >>>>>> instructions in there, as I removed the boolean, but it is certainly
> >>>>>> plausible that it could happen in the future. I don't *think* we'll
> >> need
> >>>>>> it, but I also didn't think we'd need it for SubscriptionWrapper and
> >>>> here
> >>>>>> we are.
> >>>>>>
> >>>>>> Thanks for the thoughts, and the info on the right-key. That was
> >>>>>> enlightening, though I can't think of a use-case for it *at this
> point
> >>>> in
> >>>>>> time*. :)
> >>>>>>
> >>>>>> Adam
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On Thu, Jun 27, 2019 at 12:29 PM John Roesler <jo...@confluent.io>
> >>>> wrote:
> >>>>>>
> >>>>>>> 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 `nullKey: (null, rhsValue)`, but we
> >>>>>>> don't currently allow null keys in Streams. It actually is possible
> >> to
> >>>>>>> define them, and therefore to add right- and full-outer foreign-key
> >>>>>>> joins later, but it's non-trivial in a streaming context with
> >>>>>>> continuously updated results. (See the PS if you're curious what
> I'm
> >>>>>>> thinking). You're correct, right- and full-outer joins are trivial
> on
> >>>>>>> our current 1:1 table joins because they are equi-joins.
> >>>>>>>
> >>>>>>> Regarding the transition, it sounds like what you're proposing is
> >> that
> >>>>>>> we would say, "adding a foreign-key join to your topology requires
> a
> >>>>>>> full application reset (or a new application id)". This is also an
> >>>>>>> acceptable constraint to place on the feature, but not strictly
> >>>>>>> necessary. Since 2.3, it's now possible to give all the state in
> your
> >>>>>>> application stable names. This means that it's no longer true that
> >>>>>>> adding a node to your topology graph would break its structure, and
> >> it
> >>>>>>> does become possible to add new operators and simply restart the
> app.
> >>>>>>> Revisiting my prior thought, though, I think the problem is not
> >>>>>>> specific to your feature. For example, adding a new grouped
> >>>>>>> aggregation would produce a new repartition topic, but the
> >> repartition
> >>>>>>> topic partitions might get assigned to old nodes in the middle of a
> >>>>>>> rolling bounce, and they would need to just ignore them. This
> >>>>>>> requirement is the same for the repartition topics in the FK join,
> so
> >>>>>>> it's orthogonal to your design.
> >>>>>>>
> >>>>>>> Back to the first concern, though, I'm not sure I followed the
> >>>>>>> explanation. As a thought experiment, let's imagine that Joe hadn't
> >>>>>>> taken the time to experiment with your feature branch. We wouldn't
> >>>>>>> have noticed the problem until the feature was already released in
> >>>>>>> 2.4. So the wire protocol on that PK->FK subscription topic would
> >> have
> >>>>>>> been V1: "FK,PK,HASH,BOOLEAN". Then, Joe would have let us know the
> >>>>>>> problem once they picked up the feature, so we would want to
> >> implement
> >>>>>>> your proposed fix and change the wire protocol to V2:
> >>>>>>> "FK,PK,HASH,INSTRUCTIONS" in 2.5. Upon rolling out the update, we
> >>>>>>> would see both 2.4 nodes encountering V2 messages and 2.5 nodes
> >>>>>>> encountering V1 messages. How can they both detect that they are
> >>>>>>> attempting to process a newer or older protocol? If they can detect
> >>>>>>> it, then what should they do?
> >>>>>>>
> >>>>>>> From experience, there are two basic solutions to this problem:
> >>>>>>>
> >>>>>>> STRATEGY1. Add a protocol version to the message (could be a number
> >> at
> >>>>>>> the start of the message payload, or it could be a number in the
> >>>>>>> message headers, not sure if it matters much. Payload is probably
> >> more
> >>>>>>> compact, since the header would need a name.) In this case, the 2.4
> >>>>>>> worker would know that it's max protocol version is V1, and when it
> >>>>>>> sees the V2 message, it knows that it can't handle it properly.
> >> Rather
> >>>>>>> than doing something wrong, it would just not do anything. This
> means
> >>>>>>> it would stop the task, if not shut down the whole instance. On the
> >>>>>>> other hand, a 2.5 worker would have some defined logic for how to
> >>>>>>> handle all versions (V1 and V2), so once the upgrade is complete,
> all
> >>>>>>> messages can be processed.
> >>>>>>>
> >>>>>>> STRATEGY2. Make the schema forward-compatible. Basically, we ensure
> >>>>>>> that new fields can only be appended to the message schema, and
> that
> >>>>>>> older workers using only a prefix of the full message would still
> >>>>>>> behave correctly. Using the example above, we'd instead evolve the
> >>>>>>> schema to V2': "FK,PK,HASH,BOOLEAN,INSTRUCTIONS", and continue to
> set
> >>>>>>> the boolean field to true for the "new" foreign key. Then, 2.4
> >> workers
> >>>>>>> encountering the a "new FK" message would just see the prefix of
> the
> >>>>>>> payload that makes sense to them, and they would still continue
> >>>>>>> processing the messages as they always have. Only after the 2.5
> code
> >>>>>>> is fully rolled out to the cluster would we be sure to see the
> >> desired
> >>>>>>> behavior. Note: in the reverse case, a 2.5 worker knows how to
> fully
> >>>>>>> parse the new message format, even if it plans to ignore the
> BOOLEAN
> >>>>>>> field.
> >>>>>>>
> >>>>>>> There are some tradeoffs between these strategies: STRATEGY1
> ensures
> >>>>>>> that all messages are only handled by workers that can properly
> >> handle
> >>>>>>> them, although it results in processing stalls while there are
> still
> >>>>>>> old nodes in the cluster. STRATEGY2 ensures that all messages can
> be
> >>>>>>> processed by all nodes, so there are no stalls, but we can never
> >>>>>>> remove fields from the message, so if there are a lot of revisions
> in
> >>>>>>> the future, the payloads will become bloated. Also, it's not clear
> >>>>>>> that you can actually pull off STRATEGY2 in all cases. If there's
> >> some
> >>>>>>> new kind of message you want to send that has no way to be
> correctly
> >>>>>>> processed at all under the 2.4 code paths, the prefix thing simply
> >>>>>>> doesn't work. Etc.
> >>>>>>>
> >>>>>>> Also, note that you can modify the above strategies by instead
> >>>>>>> designing the message fields for extensibility. E.g., if you make
> the
> >>>>>>> instructions field an enum, then you can make sure that the default
> >>>>>>> case is handled sensibly (probably similarly to STRATEGY1, just
> choke
> >>>>>>> on unknown instructions) and that you never remove an instruction
> >> type
> >>>>>>> from the enum in future versions.
> >>>>>>>
> >>>>>>> Does this make sense?
> >>>>>>> -John
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>>
> >>>>>>> PS:
> >>>>>>> We can define null keys for streaming tables, but it's tricky.
> >>>>>>>
> >>>>>>> Specifically, you'd want to define some concept of null keys that
> >>>>>>> allows all null keys to be unique, but _also_ to have a fixed
> >>>>>>> identity, so that a particular null-key can be updated later. One
> >>>>>>> example could be to union the existing keyspace with a new
> >>>>>>> null-keyspace, where normal keys are like "key" and null-keys are
> >> like
> >>>>>>> "null(identity)". Then given a query like
> >>>>>>> "KTable<String,Integer>.rightJoin(KTable<Integer,Boolean>)", and
> >>>>>>> inputs like:
> >>>>>>> LHS:
> >>>>>>> "a": 1
> >>>>>>> "b": 2
> >>>>>>>
> >>>>>>> RHS:
> >>>>>>> 1: true
> >>>>>>> 3: false
> >>>>>>>
> >>>>>>> a full outer join would produce:
> >>>>>>> "a": (1, true)
> >>>>>>> "b": (2, null)
> >>>>>>> null(3): (null, false)
> >>>>>>>
> >>>>>>> which can be correctly updated later if we get an update on the
> LHS:
> >>>>>>> PUT("c": 3)
> >>>>>>>
> >>>>>>> We'd emit for the results:
> >>>>>>> DELETE(null(e))
> >>>>>>> EMIT("c": (3, false))
> >>>>>>>
> >>>>>>> Resulting in the correct result table of:
> >>>>>>> "a": (1, true)
> >>>>>>> "b": (2, null)
> >>>>>>> "c": (3, false)
> >>>>>>>
> >>>>>>> As mentioned, this is tricky, and I would avoid it until we have
> >>>>>>> evidence that it's actually useful to cover this part of the design
> >>>>>>> space. Certainly, it would be a separate KIP if it came to that.
> >>>>>>>
> >>>>>>> On Wed, Jun 26, 2019 at 8:57 PM Adam Bellemare <
> >>>> adam.bellemare@gmail.com>
> >>>>>>> wrote:
> >>>>>>>>
> >>>>>>>> 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
> >> the
> >>>>>>> last
> >>>>>>>> commit (about 10 min ago) along with INNER join. I do not think
> that
> >>>>>>> RIGHT
> >>>>>>>> join and OUTER are possible given that there is no LHS key
> >> available,
> >>>> so
> >>>>>>>> LHSTable.outerJoinOnForeignKey(RHSTable) wouldn't even make sense.
> >>>> This
> >>>>>>> is
> >>>>>>>> in contrast to the current LHSTable.outerJoin(RHSTable), as they
> are
> >>>> both
> >>>>>>>> keyed on the same key. I have buffed up the Integration tests and
> >> have
> >>>>>>>> tried to make them more readable to ensure that we're covering all
> >> the
> >>>>>>>> scenarios. I think that if we can get more eyes on the workflow
> >>>> showing
> >>>>>>> the
> >>>>>>>> various LHS and RHS events and outputs then that may help us
> >> validate
> >>>>>>> that
> >>>>>>>> we have all the scenarios covered.
> >>>>>>>>
> >>>>>>>> With regards to the 2.3->2.4 scenario you described, I'm not
> >> entirely
> >>>>>>> sure
> >>>>>>>> I follow. If they want to add a FK-join, they will need to rework
> >>>> their
> >>>>>>>> code in the KStreams app and make a new release, since the
> >> underlying
> >>>>>>>> topology would be different and new internal topics would need to
> be
> >>>>>>>> created. In other words, I don't think a rolling upgrade where the
> >>>> user
> >>>>>>>> introduces a FK join would be possible since their topology would
> >>>>>>>> necessitate a full KStreams reset. Is this what you meant?
> >>>>>>>>
> >>>>>>>>
> >>>>>>>>
> >>>>>>>> On Wed, Jun 26, 2019 at 4:10 PM John Roesler <jo...@confluent.io>
> >>>> wrote:
> >>>>>>>>
> >>>>>>>>> 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.bellemare@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.bellemare@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 <
> john@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
> >>>>>>>>>>>> <ad...@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 <
> >>>>>>> john@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
> >>>>>>>>>>>>>>
> >>>>>>>>>>>>
> >>>>>>>>>>>
> >>>>>>>>>
> >>>>>>>
> >>>>
> >>>>
> >>>
> >>
> >>
> >
>
>

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
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, but want to throw out the idea first.

> S-2 could probably be simplified to "for a given key, was the previous
> propagated result a null/tombstone or not?"

Yes, that's the idea.

> However,
> the table will grow indefinitely large as we can never remove keys from it.

That a good point. I missed that... Seems that this rules out S-2
because if we cannot provide _strict_ guarantees, it does not seem to be
worth to even try?


About the impact of unnecessary tombstones: There is a performance
impact, as we increase the output data rate, potentially broker load if
the result is written into a topic, and also increase downstream
processing load. It's hard to judge, how much the overhead will be, as
it will depend on the selectivity of the join. But it might be significant?

Also, users have certain expectations on the result and it's unintuitive
(even if semantically correct) to send those tombstones. From
experience, we often have a hard time to explain semantics to people and
I was hoping we could avoid introducing unintuitive behavior.


Would be good to get input from others and how they judge the impact. I
think it might be still worth to explore how complex S-1 would be. If we
think it's too complex it might be a good argument to just accept the
unnecessary tombstones?


-Matthias


On 7/3/19 8:03 AM, Adam Bellemare wrote:
> 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 seems to me that the processor would need to block
> while it waits for the unsubscribe to propagate and return, which would
> cause throughput to drop significantly. Alternately, we would need to
> maintain state anyways about which events were sent and which responses
> returned, while being sure to respect the offset order in which they're
> emitted. I think this would only reduce blocking slightly while increasing
> complexity. If I am wrong in understanding this, please let me know where
> my thinking is erroneous.
> 
> S-2 could probably be simplified to "for a given key, was the previous
> propagated result a null/tombstone or not?"
> It would act very similarly to the hash value mechanism, where we discard
> any events that are not of the correct hash. In this case, we simply store
> (key, wasLastOutputATombstone) right before the event is output downstream
> of the Join + Resolver. This ignores all the complexities of which event is
> propagating over which wire and simply squelches any extra tombstones from
> being sent.
> 
> For storage, we need to use the full primary key and a boolean. However,
> the table will grow indefinitely large as we can never remove keys from it.
> If we delete key=k from the table and propagate a tombstone, but later (say
> 3 weeks, 3 months, etc) we publish (k, baz), but baz does not exist on the
> RHS, we will end up publishing an extra tombstone because we have no idea
> what the previously sent record was for k. For this reason I think it's
> worth asking if we really can maintain state, and if it's even necessary
> (again, a full understanding of the impact of extra tombstones may help us
> figure out a better solution).
> 
> As it stands, I don't think either of these will work well. That being
> said, I myself do not have any better ideas at the moment, but I would
> still like to better understand circumstances where it has a negative
> impact downstream as that may provide some insights.
> 
> 
> Thanks
> 
> Adam
> 
> 
> 
> 
> 
> On Tue, Jul 2, 2019 at 11:18 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
> 
>> 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 subscribe message to RHS
>> a-2) RHS lookup and send result back if there is one
>> a-3) emit result on LHS if any is returned
>>
>> b-1) delete subscription from RHS
>> b-2) if there was a previous result (can easily be decided by looking up
>> RHS table for an existing key), send tombstone back
>> b-3) emit tombstone on LHS if any is returned
>>
>> c-1) delete old subscription from RHS
>> c-2) send new subscription to RHS
>> c-3) if there was no previous result and there is no new result emit
>> nothing
>> c-4) if there was a previous result and there is no new result emit a
>> tombstone LHS
>> c-5) if there is a new result (old result may or may not exist), emit
>> only new result LHS (don't emit a tombstone)
>>
>>
>> Case (a) and (b) are simple and could be implemented with a "fire and
>> forget" strategy. The LHS just "blindly" updates the subscription, the
>> RHS can process the subscription with local knowledge and may send
>> something back. If something is sent back, the LHS blindly emits it.
>>
>> We know that for both cases, we never miss anything and we never emit
>> anything unnecessary.
>>
>> However, using this pattern for (c), we don't get our expected result:
>>
>> Issues: LHS sends both subscription updates in parallel. It does not
>> know if zero, one, or two result records will be produced by RHS. If RHS
>> produces two results, their order is not known (however, LHS would need
>> to emit them in the right order; also forcing RHS to always send a
>> result back is not a sufficient solution). If only one result comes
>> back, it's unclear if a second result may arrive later and thus the
>> result may need to be buffered... Overall, local knowledge does not seem
>> to be sufficient to tackle the case.
>>
>> The current proposal tries to address the issue with encoding additional
>> information, to tell the RHS to send a tombstone back or not. But it
>> does not seem to be perfect, and it might result in unnecessary
>> tombstones as it still uses local knowledge only and thus misses some
>> information.
>>
>> I think the main problem is, that the knowledge about the a potential
>> previous result and a potential new result is sharded on the RHS. Hence,
>> the "unsubscribe" does not know if it needs to send a tombstone back for
>> the case that there was an old result but there is no new result.
>> Similarly, the "new subscribe" cannot know if it needs to send a
>> tombstone or not (as it does not know if there was a previous result) if
>> it does not match.
>>
>> To really solve the issue, I see two possible solutions (both are not
>> great, but I wanted to discuss them anyway):
>>
>> S-1: First unsubscribe, and send new subscription after result comes
>> back. For this case, the RHS must always send something back to the LHS
>> on unsubscribe. The answer if "previous result exists/not-exist" can be
>> added to the new-subscription and hence RHS can either return nothing, a
>> tombstone, or a new result. The LHS can blindly emit whatever RHS
>> returns. This would also cover (a) and (b) cases. However, the overall
>> time to emit the join result is doubled for the (common) update case...
>> (we need two consecutive round-trips to the RHS).
>>
>> S-2: Remember/store if a previous result exists on LHS: for this case,
>> (a) is handled straightforward, (b) is handled by telling RHS to send
>> tombstone if previous result exits, and (c) can send both request in
>> parallel letting the unsubscribe never return anything, and subscribe is
>> handled as in (b). However, we need a second store on the LHS to
>> remember if there was a previous result. (Also not sure how
>> interleaving/inflight computation might affect the algorithm...)
>>
>> I think, sending unnecessary tombstones is quite bad (in very old
>> releases we had a similar issue and fixed it). However, I am also not
>> 100% sure if the solutions I came up with are good enough to justify
>> them. (Personally, I slightly tend to prefer S-2 because I think that
>> the additional store is less of an issue than the increase processing
>> time).
>>
>> Would love to hear your thoughts.
>>
>>
>> -Matthias
>>
>>
>> On 6/28/19 6:19 AM, Adam Bellemare wrote:
>>> 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 unnecessary tombstone is (that I can
>>> tell...) when we do the following:
>>> RHS:
>>> (1, bar)
>>>
>>> LHS
>>> (K,1)  -> Results in (K, 1, bar) being output
>>> (K,1) -> (K,2) ->  Results in (K, null) being output for INNER (no
>> matching
>>> element on LHS)
>>> (K,2) -> (K,3) ->  Results in (K, null) being output for INNER (because
>> we
>>> don't maintain state to know we already output the tombstone on the
>>> previous transition).
>>> (K,2) -> (K,9000) ->  Results in (K, null)... etc.
>>>
>>> Byte versioning is going in today, then I hope to get back to addressing
>> a
>>> number of John's previous questions in the PR.
>>>
>>> Adam
>>>
>>>
>>> On Thu, Jun 27, 2019 at 5:47 PM Matthias J. Sax <ma...@confluent.io>
>>> wrote:
>>>
>>>> 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 unnecessary tombstone? Seems I a missing case?
>>>>
>>>> Also for my own understanding: the `instruction` is only part of the
>>>> message? It is no necessary to store it in the RHS auxiliary store,
>> right?
>>>>
>>>> About right/full-outer joins. Agreed. Getting left-joins would be
>> awesome!
>>>>
>>>> About upgrading: Good call John! Adding a version byte for subscription
>>>> and response is good forward thinking. I personally prefer version
>>>> numbers, too, as they carry more information.
>>>>
>>>> Thanks for all the hard to everybody involved!
>>>>
>>>>
>>>> -Matthias
>>>>
>>>> On 6/27/19 1:44 PM, John Roesler wrote:
>>>>> 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 revisions,
>>>>> we can just use one byte. The worst case is that we run out of numbers
>>>>> and reserve the last one to mean, "consult another field for the
>>>>> actual version number". It seems like a single byte on each message
>>>>> isn't too much to pay.
>>>>>
>>>>> Since you point it out, we might as well put a version number on the
>>>>> SubscriptionResponseWrapper as well. It may not be needed, but if we
>>>>> ever need it, even just once, we'll be glad we have it.
>>>>>
>>>>> Regarding the instructions field, we can also serialize the enum very
>>>>> compactly as a single byte (which is the same size a boolean takes
>>>>> anyway), so it seems like an Enum in Java-land and a byte on the wire
>>>>> is a good choice.
>>>>>
>>>>> Agreed on the right and full outer joins, it doesn't seem necessary
>>>>> right now, although I am happy to see the left join "join" the party,
>>>>> since as you said, we were so close to it anyway. Can you also add it
>>>>> to the KIP?
>>>>>
>>>>> Thanks as always for your awesome efforts on this,
>>>>> -John
>>>>>
>>>>> On Thu, Jun 27, 2019 at 3:04 PM Adam Bellemare <
>> adam.bellemare@gmail.com>
>>>> wrote:
>>>>>>
>>>>>> 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 microservices - there
>> are
>>>>>> indeed cases where a breaking change must be made, and forward
>>>>>> compatibility will provide us with no out other than requiring a full
>>>> stop
>>>>>> and full upgrade for all nodes, shifting us back towards STRATEGY 1.
>>>>>>
>>>>>> My preference is STRATEGY 1 with instructions as an ENUM, and we can
>>>>>> certainly include a version. Would it make sense to include a version
>>>>>> number in  SubscriptionResponseWrapper as well? Currently we don't
>> have
>>>> any
>>>>>> instructions in there, as I removed the boolean, but it is certainly
>>>>>> plausible that it could happen in the future. I don't *think* we'll
>> need
>>>>>> it, but I also didn't think we'd need it for SubscriptionWrapper and
>>>> here
>>>>>> we are.
>>>>>>
>>>>>> Thanks for the thoughts, and the info on the right-key. That was
>>>>>> enlightening, though I can't think of a use-case for it *at this point
>>>> in
>>>>>> time*. :)
>>>>>>
>>>>>> Adam
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Thu, Jun 27, 2019 at 12:29 PM John Roesler <jo...@confluent.io>
>>>> wrote:
>>>>>>
>>>>>>> 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 `nullKey: (null, rhsValue)`, but we
>>>>>>> don't currently allow null keys in Streams. It actually is possible
>> to
>>>>>>> define them, and therefore to add right- and full-outer foreign-key
>>>>>>> joins later, but it's non-trivial in a streaming context with
>>>>>>> continuously updated results. (See the PS if you're curious what I'm
>>>>>>> thinking). You're correct, right- and full-outer joins are trivial on
>>>>>>> our current 1:1 table joins because they are equi-joins.
>>>>>>>
>>>>>>> Regarding the transition, it sounds like what you're proposing is
>> that
>>>>>>> we would say, "adding a foreign-key join to your topology requires a
>>>>>>> full application reset (or a new application id)". This is also an
>>>>>>> acceptable constraint to place on the feature, but not strictly
>>>>>>> necessary. Since 2.3, it's now possible to give all the state in your
>>>>>>> application stable names. This means that it's no longer true that
>>>>>>> adding a node to your topology graph would break its structure, and
>> it
>>>>>>> does become possible to add new operators and simply restart the app.
>>>>>>> Revisiting my prior thought, though, I think the problem is not
>>>>>>> specific to your feature. For example, adding a new grouped
>>>>>>> aggregation would produce a new repartition topic, but the
>> repartition
>>>>>>> topic partitions might get assigned to old nodes in the middle of a
>>>>>>> rolling bounce, and they would need to just ignore them. This
>>>>>>> requirement is the same for the repartition topics in the FK join, so
>>>>>>> it's orthogonal to your design.
>>>>>>>
>>>>>>> Back to the first concern, though, I'm not sure I followed the
>>>>>>> explanation. As a thought experiment, let's imagine that Joe hadn't
>>>>>>> taken the time to experiment with your feature branch. We wouldn't
>>>>>>> have noticed the problem until the feature was already released in
>>>>>>> 2.4. So the wire protocol on that PK->FK subscription topic would
>> have
>>>>>>> been V1: "FK,PK,HASH,BOOLEAN". Then, Joe would have let us know the
>>>>>>> problem once they picked up the feature, so we would want to
>> implement
>>>>>>> your proposed fix and change the wire protocol to V2:
>>>>>>> "FK,PK,HASH,INSTRUCTIONS" in 2.5. Upon rolling out the update, we
>>>>>>> would see both 2.4 nodes encountering V2 messages and 2.5 nodes
>>>>>>> encountering V1 messages. How can they both detect that they are
>>>>>>> attempting to process a newer or older protocol? If they can detect
>>>>>>> it, then what should they do?
>>>>>>>
>>>>>>> From experience, there are two basic solutions to this problem:
>>>>>>>
>>>>>>> STRATEGY1. Add a protocol version to the message (could be a number
>> at
>>>>>>> the start of the message payload, or it could be a number in the
>>>>>>> message headers, not sure if it matters much. Payload is probably
>> more
>>>>>>> compact, since the header would need a name.) In this case, the 2.4
>>>>>>> worker would know that it's max protocol version is V1, and when it
>>>>>>> sees the V2 message, it knows that it can't handle it properly.
>> Rather
>>>>>>> than doing something wrong, it would just not do anything. This means
>>>>>>> it would stop the task, if not shut down the whole instance. On the
>>>>>>> other hand, a 2.5 worker would have some defined logic for how to
>>>>>>> handle all versions (V1 and V2), so once the upgrade is complete, all
>>>>>>> messages can be processed.
>>>>>>>
>>>>>>> STRATEGY2. Make the schema forward-compatible. Basically, we ensure
>>>>>>> that new fields can only be appended to the message schema, and that
>>>>>>> older workers using only a prefix of the full message would still
>>>>>>> behave correctly. Using the example above, we'd instead evolve the
>>>>>>> schema to V2': "FK,PK,HASH,BOOLEAN,INSTRUCTIONS", and continue to set
>>>>>>> the boolean field to true for the "new" foreign key. Then, 2.4
>> workers
>>>>>>> encountering the a "new FK" message would just see the prefix of the
>>>>>>> payload that makes sense to them, and they would still continue
>>>>>>> processing the messages as they always have. Only after the 2.5 code
>>>>>>> is fully rolled out to the cluster would we be sure to see the
>> desired
>>>>>>> behavior. Note: in the reverse case, a 2.5 worker knows how to fully
>>>>>>> parse the new message format, even if it plans to ignore the BOOLEAN
>>>>>>> field.
>>>>>>>
>>>>>>> There are some tradeoffs between these strategies: STRATEGY1 ensures
>>>>>>> that all messages are only handled by workers that can properly
>> handle
>>>>>>> them, although it results in processing stalls while there are still
>>>>>>> old nodes in the cluster. STRATEGY2 ensures that all messages can be
>>>>>>> processed by all nodes, so there are no stalls, but we can never
>>>>>>> remove fields from the message, so if there are a lot of revisions in
>>>>>>> the future, the payloads will become bloated. Also, it's not clear
>>>>>>> that you can actually pull off STRATEGY2 in all cases. If there's
>> some
>>>>>>> new kind of message you want to send that has no way to be correctly
>>>>>>> processed at all under the 2.4 code paths, the prefix thing simply
>>>>>>> doesn't work. Etc.
>>>>>>>
>>>>>>> Also, note that you can modify the above strategies by instead
>>>>>>> designing the message fields for extensibility. E.g., if you make the
>>>>>>> instructions field an enum, then you can make sure that the default
>>>>>>> case is handled sensibly (probably similarly to STRATEGY1, just choke
>>>>>>> on unknown instructions) and that you never remove an instruction
>> type
>>>>>>> from the enum in future versions.
>>>>>>>
>>>>>>> Does this make sense?
>>>>>>> -John
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> PS:
>>>>>>> We can define null keys for streaming tables, but it's tricky.
>>>>>>>
>>>>>>> Specifically, you'd want to define some concept of null keys that
>>>>>>> allows all null keys to be unique, but _also_ to have a fixed
>>>>>>> identity, so that a particular null-key can be updated later. One
>>>>>>> example could be to union the existing keyspace with a new
>>>>>>> null-keyspace, where normal keys are like "key" and null-keys are
>> like
>>>>>>> "null(identity)". Then given a query like
>>>>>>> "KTable<String,Integer>.rightJoin(KTable<Integer,Boolean>)", and
>>>>>>> inputs like:
>>>>>>> LHS:
>>>>>>> "a": 1
>>>>>>> "b": 2
>>>>>>>
>>>>>>> RHS:
>>>>>>> 1: true
>>>>>>> 3: false
>>>>>>>
>>>>>>> a full outer join would produce:
>>>>>>> "a": (1, true)
>>>>>>> "b": (2, null)
>>>>>>> null(3): (null, false)
>>>>>>>
>>>>>>> which can be correctly updated later if we get an update on the LHS:
>>>>>>> PUT("c": 3)
>>>>>>>
>>>>>>> We'd emit for the results:
>>>>>>> DELETE(null(e))
>>>>>>> EMIT("c": (3, false))
>>>>>>>
>>>>>>> Resulting in the correct result table of:
>>>>>>> "a": (1, true)
>>>>>>> "b": (2, null)
>>>>>>> "c": (3, false)
>>>>>>>
>>>>>>> As mentioned, this is tricky, and I would avoid it until we have
>>>>>>> evidence that it's actually useful to cover this part of the design
>>>>>>> space. Certainly, it would be a separate KIP if it came to that.
>>>>>>>
>>>>>>> On Wed, Jun 26, 2019 at 8:57 PM Adam Bellemare <
>>>> adam.bellemare@gmail.com>
>>>>>>> wrote:
>>>>>>>>
>>>>>>>> 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
>> the
>>>>>>> last
>>>>>>>> commit (about 10 min ago) along with INNER join. I do not think that
>>>>>>> RIGHT
>>>>>>>> join and OUTER are possible given that there is no LHS key
>> available,
>>>> so
>>>>>>>> LHSTable.outerJoinOnForeignKey(RHSTable) wouldn't even make sense.
>>>> This
>>>>>>> is
>>>>>>>> in contrast to the current LHSTable.outerJoin(RHSTable), as they are
>>>> both
>>>>>>>> keyed on the same key. I have buffed up the Integration tests and
>> have
>>>>>>>> tried to make them more readable to ensure that we're covering all
>> the
>>>>>>>> scenarios. I think that if we can get more eyes on the workflow
>>>> showing
>>>>>>> the
>>>>>>>> various LHS and RHS events and outputs then that may help us
>> validate
>>>>>>> that
>>>>>>>> we have all the scenarios covered.
>>>>>>>>
>>>>>>>> With regards to the 2.3->2.4 scenario you described, I'm not
>> entirely
>>>>>>> sure
>>>>>>>> I follow. If they want to add a FK-join, they will need to rework
>>>> their
>>>>>>>> code in the KStreams app and make a new release, since the
>> underlying
>>>>>>>> topology would be different and new internal topics would need to be
>>>>>>>> created. In other words, I don't think a rolling upgrade where the
>>>> user
>>>>>>>> introduces a FK join would be possible since their topology would
>>>>>>>> necessitate a full KStreams reset. Is this what you meant?
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> On Wed, Jun 26, 2019 at 4:10 PM John Roesler <jo...@confluent.io>
>>>> wrote:
>>>>>>>>
>>>>>>>>> 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.bellemare@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.bellemare@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 <jo...@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
>>>>>>>>>>>> <ad...@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 <
>>>>>>> john@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
>>>>>>>>>>>>>>
>>>>>>>>>>>>
>>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>
>>>>
>>>
>>
>>
> 


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

Posted by Adam Bellemare <ad...@gmail.com>.
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 seems to me that the processor would need to block
while it waits for the unsubscribe to propagate and return, which would
cause throughput to drop significantly. Alternately, we would need to
maintain state anyways about which events were sent and which responses
returned, while being sure to respect the offset order in which they're
emitted. I think this would only reduce blocking slightly while increasing
complexity. If I am wrong in understanding this, please let me know where
my thinking is erroneous.

S-2 could probably be simplified to "for a given key, was the previous
propagated result a null/tombstone or not?"
It would act very similarly to the hash value mechanism, where we discard
any events that are not of the correct hash. In this case, we simply store
(key, wasLastOutputATombstone) right before the event is output downstream
of the Join + Resolver. This ignores all the complexities of which event is
propagating over which wire and simply squelches any extra tombstones from
being sent.

For storage, we need to use the full primary key and a boolean. However,
the table will grow indefinitely large as we can never remove keys from it.
If we delete key=k from the table and propagate a tombstone, but later (say
3 weeks, 3 months, etc) we publish (k, baz), but baz does not exist on the
RHS, we will end up publishing an extra tombstone because we have no idea
what the previously sent record was for k. For this reason I think it's
worth asking if we really can maintain state, and if it's even necessary
(again, a full understanding of the impact of extra tombstones may help us
figure out a better solution).

As it stands, I don't think either of these will work well. That being
said, I myself do not have any better ideas at the moment, but I would
still like to better understand circumstances where it has a negative
impact downstream as that may provide some insights.


Thanks

Adam





On Tue, Jul 2, 2019 at 11:18 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> 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 subscribe message to RHS
> a-2) RHS lookup and send result back if there is one
> a-3) emit result on LHS if any is returned
>
> b-1) delete subscription from RHS
> b-2) if there was a previous result (can easily be decided by looking up
> RHS table for an existing key), send tombstone back
> b-3) emit tombstone on LHS if any is returned
>
> c-1) delete old subscription from RHS
> c-2) send new subscription to RHS
> c-3) if there was no previous result and there is no new result emit
> nothing
> c-4) if there was a previous result and there is no new result emit a
> tombstone LHS
> c-5) if there is a new result (old result may or may not exist), emit
> only new result LHS (don't emit a tombstone)
>
>
> Case (a) and (b) are simple and could be implemented with a "fire and
> forget" strategy. The LHS just "blindly" updates the subscription, the
> RHS can process the subscription with local knowledge and may send
> something back. If something is sent back, the LHS blindly emits it.
>
> We know that for both cases, we never miss anything and we never emit
> anything unnecessary.
>
> However, using this pattern for (c), we don't get our expected result:
>
> Issues: LHS sends both subscription updates in parallel. It does not
> know if zero, one, or two result records will be produced by RHS. If RHS
> produces two results, their order is not known (however, LHS would need
> to emit them in the right order; also forcing RHS to always send a
> result back is not a sufficient solution). If only one result comes
> back, it's unclear if a second result may arrive later and thus the
> result may need to be buffered... Overall, local knowledge does not seem
> to be sufficient to tackle the case.
>
> The current proposal tries to address the issue with encoding additional
> information, to tell the RHS to send a tombstone back or not. But it
> does not seem to be perfect, and it might result in unnecessary
> tombstones as it still uses local knowledge only and thus misses some
> information.
>
> I think the main problem is, that the knowledge about the a potential
> previous result and a potential new result is sharded on the RHS. Hence,
> the "unsubscribe" does not know if it needs to send a tombstone back for
> the case that there was an old result but there is no new result.
> Similarly, the "new subscribe" cannot know if it needs to send a
> tombstone or not (as it does not know if there was a previous result) if
> it does not match.
>
> To really solve the issue, I see two possible solutions (both are not
> great, but I wanted to discuss them anyway):
>
> S-1: First unsubscribe, and send new subscription after result comes
> back. For this case, the RHS must always send something back to the LHS
> on unsubscribe. The answer if "previous result exists/not-exist" can be
> added to the new-subscription and hence RHS can either return nothing, a
> tombstone, or a new result. The LHS can blindly emit whatever RHS
> returns. This would also cover (a) and (b) cases. However, the overall
> time to emit the join result is doubled for the (common) update case...
> (we need two consecutive round-trips to the RHS).
>
> S-2: Remember/store if a previous result exists on LHS: for this case,
> (a) is handled straightforward, (b) is handled by telling RHS to send
> tombstone if previous result exits, and (c) can send both request in
> parallel letting the unsubscribe never return anything, and subscribe is
> handled as in (b). However, we need a second store on the LHS to
> remember if there was a previous result. (Also not sure how
> interleaving/inflight computation might affect the algorithm...)
>
> I think, sending unnecessary tombstones is quite bad (in very old
> releases we had a similar issue and fixed it). However, I am also not
> 100% sure if the solutions I came up with are good enough to justify
> them. (Personally, I slightly tend to prefer S-2 because I think that
> the additional store is less of an issue than the increase processing
> time).
>
> Would love to hear your thoughts.
>
>
> -Matthias
>
>
> On 6/28/19 6:19 AM, Adam Bellemare wrote:
> > 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 unnecessary tombstone is (that I can
> > tell...) when we do the following:
> > RHS:
> > (1, bar)
> >
> > LHS
> > (K,1)  -> Results in (K, 1, bar) being output
> > (K,1) -> (K,2) ->  Results in (K, null) being output for INNER (no
> matching
> > element on LHS)
> > (K,2) -> (K,3) ->  Results in (K, null) being output for INNER (because
> we
> > don't maintain state to know we already output the tombstone on the
> > previous transition).
> > (K,2) -> (K,9000) ->  Results in (K, null)... etc.
> >
> > Byte versioning is going in today, then I hope to get back to addressing
> a
> > number of John's previous questions in the PR.
> >
> > Adam
> >
> >
> > On Thu, Jun 27, 2019 at 5:47 PM Matthias J. Sax <ma...@confluent.io>
> > wrote:
> >
> >> 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 unnecessary tombstone? Seems I a missing case?
> >>
> >> Also for my own understanding: the `instruction` is only part of the
> >> message? It is no necessary to store it in the RHS auxiliary store,
> right?
> >>
> >> About right/full-outer joins. Agreed. Getting left-joins would be
> awesome!
> >>
> >> About upgrading: Good call John! Adding a version byte for subscription
> >> and response is good forward thinking. I personally prefer version
> >> numbers, too, as they carry more information.
> >>
> >> Thanks for all the hard to everybody involved!
> >>
> >>
> >> -Matthias
> >>
> >> On 6/27/19 1:44 PM, John Roesler wrote:
> >>> 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 revisions,
> >>> we can just use one byte. The worst case is that we run out of numbers
> >>> and reserve the last one to mean, "consult another field for the
> >>> actual version number". It seems like a single byte on each message
> >>> isn't too much to pay.
> >>>
> >>> Since you point it out, we might as well put a version number on the
> >>> SubscriptionResponseWrapper as well. It may not be needed, but if we
> >>> ever need it, even just once, we'll be glad we have it.
> >>>
> >>> Regarding the instructions field, we can also serialize the enum very
> >>> compactly as a single byte (which is the same size a boolean takes
> >>> anyway), so it seems like an Enum in Java-land and a byte on the wire
> >>> is a good choice.
> >>>
> >>> Agreed on the right and full outer joins, it doesn't seem necessary
> >>> right now, although I am happy to see the left join "join" the party,
> >>> since as you said, we were so close to it anyway. Can you also add it
> >>> to the KIP?
> >>>
> >>> Thanks as always for your awesome efforts on this,
> >>> -John
> >>>
> >>> On Thu, Jun 27, 2019 at 3:04 PM Adam Bellemare <
> adam.bellemare@gmail.com>
> >> wrote:
> >>>>
> >>>> 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 microservices - there
> are
> >>>> indeed cases where a breaking change must be made, and forward
> >>>> compatibility will provide us with no out other than requiring a full
> >> stop
> >>>> and full upgrade for all nodes, shifting us back towards STRATEGY 1.
> >>>>
> >>>> My preference is STRATEGY 1 with instructions as an ENUM, and we can
> >>>> certainly include a version. Would it make sense to include a version
> >>>> number in  SubscriptionResponseWrapper as well? Currently we don't
> have
> >> any
> >>>> instructions in there, as I removed the boolean, but it is certainly
> >>>> plausible that it could happen in the future. I don't *think* we'll
> need
> >>>> it, but I also didn't think we'd need it for SubscriptionWrapper and
> >> here
> >>>> we are.
> >>>>
> >>>> Thanks for the thoughts, and the info on the right-key. That was
> >>>> enlightening, though I can't think of a use-case for it *at this point
> >> in
> >>>> time*. :)
> >>>>
> >>>> Adam
> >>>>
> >>>>
> >>>>
> >>>> On Thu, Jun 27, 2019 at 12:29 PM John Roesler <jo...@confluent.io>
> >> wrote:
> >>>>
> >>>>> 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 `nullKey: (null, rhsValue)`, but we
> >>>>> don't currently allow null keys in Streams. It actually is possible
> to
> >>>>> define them, and therefore to add right- and full-outer foreign-key
> >>>>> joins later, but it's non-trivial in a streaming context with
> >>>>> continuously updated results. (See the PS if you're curious what I'm
> >>>>> thinking). You're correct, right- and full-outer joins are trivial on
> >>>>> our current 1:1 table joins because they are equi-joins.
> >>>>>
> >>>>> Regarding the transition, it sounds like what you're proposing is
> that
> >>>>> we would say, "adding a foreign-key join to your topology requires a
> >>>>> full application reset (or a new application id)". This is also an
> >>>>> acceptable constraint to place on the feature, but not strictly
> >>>>> necessary. Since 2.3, it's now possible to give all the state in your
> >>>>> application stable names. This means that it's no longer true that
> >>>>> adding a node to your topology graph would break its structure, and
> it
> >>>>> does become possible to add new operators and simply restart the app.
> >>>>> Revisiting my prior thought, though, I think the problem is not
> >>>>> specific to your feature. For example, adding a new grouped
> >>>>> aggregation would produce a new repartition topic, but the
> repartition
> >>>>> topic partitions might get assigned to old nodes in the middle of a
> >>>>> rolling bounce, and they would need to just ignore them. This
> >>>>> requirement is the same for the repartition topics in the FK join, so
> >>>>> it's orthogonal to your design.
> >>>>>
> >>>>> Back to the first concern, though, I'm not sure I followed the
> >>>>> explanation. As a thought experiment, let's imagine that Joe hadn't
> >>>>> taken the time to experiment with your feature branch. We wouldn't
> >>>>> have noticed the problem until the feature was already released in
> >>>>> 2.4. So the wire protocol on that PK->FK subscription topic would
> have
> >>>>> been V1: "FK,PK,HASH,BOOLEAN". Then, Joe would have let us know the
> >>>>> problem once they picked up the feature, so we would want to
> implement
> >>>>> your proposed fix and change the wire protocol to V2:
> >>>>> "FK,PK,HASH,INSTRUCTIONS" in 2.5. Upon rolling out the update, we
> >>>>> would see both 2.4 nodes encountering V2 messages and 2.5 nodes
> >>>>> encountering V1 messages. How can they both detect that they are
> >>>>> attempting to process a newer or older protocol? If they can detect
> >>>>> it, then what should they do?
> >>>>>
> >>>>> From experience, there are two basic solutions to this problem:
> >>>>>
> >>>>> STRATEGY1. Add a protocol version to the message (could be a number
> at
> >>>>> the start of the message payload, or it could be a number in the
> >>>>> message headers, not sure if it matters much. Payload is probably
> more
> >>>>> compact, since the header would need a name.) In this case, the 2.4
> >>>>> worker would know that it's max protocol version is V1, and when it
> >>>>> sees the V2 message, it knows that it can't handle it properly.
> Rather
> >>>>> than doing something wrong, it would just not do anything. This means
> >>>>> it would stop the task, if not shut down the whole instance. On the
> >>>>> other hand, a 2.5 worker would have some defined logic for how to
> >>>>> handle all versions (V1 and V2), so once the upgrade is complete, all
> >>>>> messages can be processed.
> >>>>>
> >>>>> STRATEGY2. Make the schema forward-compatible. Basically, we ensure
> >>>>> that new fields can only be appended to the message schema, and that
> >>>>> older workers using only a prefix of the full message would still
> >>>>> behave correctly. Using the example above, we'd instead evolve the
> >>>>> schema to V2': "FK,PK,HASH,BOOLEAN,INSTRUCTIONS", and continue to set
> >>>>> the boolean field to true for the "new" foreign key. Then, 2.4
> workers
> >>>>> encountering the a "new FK" message would just see the prefix of the
> >>>>> payload that makes sense to them, and they would still continue
> >>>>> processing the messages as they always have. Only after the 2.5 code
> >>>>> is fully rolled out to the cluster would we be sure to see the
> desired
> >>>>> behavior. Note: in the reverse case, a 2.5 worker knows how to fully
> >>>>> parse the new message format, even if it plans to ignore the BOOLEAN
> >>>>> field.
> >>>>>
> >>>>> There are some tradeoffs between these strategies: STRATEGY1 ensures
> >>>>> that all messages are only handled by workers that can properly
> handle
> >>>>> them, although it results in processing stalls while there are still
> >>>>> old nodes in the cluster. STRATEGY2 ensures that all messages can be
> >>>>> processed by all nodes, so there are no stalls, but we can never
> >>>>> remove fields from the message, so if there are a lot of revisions in
> >>>>> the future, the payloads will become bloated. Also, it's not clear
> >>>>> that you can actually pull off STRATEGY2 in all cases. If there's
> some
> >>>>> new kind of message you want to send that has no way to be correctly
> >>>>> processed at all under the 2.4 code paths, the prefix thing simply
> >>>>> doesn't work. Etc.
> >>>>>
> >>>>> Also, note that you can modify the above strategies by instead
> >>>>> designing the message fields for extensibility. E.g., if you make the
> >>>>> instructions field an enum, then you can make sure that the default
> >>>>> case is handled sensibly (probably similarly to STRATEGY1, just choke
> >>>>> on unknown instructions) and that you never remove an instruction
> type
> >>>>> from the enum in future versions.
> >>>>>
> >>>>> Does this make sense?
> >>>>> -John
> >>>>>
> >>>>>
> >>>>>
> >>>>>
> >>>>> PS:
> >>>>> We can define null keys for streaming tables, but it's tricky.
> >>>>>
> >>>>> Specifically, you'd want to define some concept of null keys that
> >>>>> allows all null keys to be unique, but _also_ to have a fixed
> >>>>> identity, so that a particular null-key can be updated later. One
> >>>>> example could be to union the existing keyspace with a new
> >>>>> null-keyspace, where normal keys are like "key" and null-keys are
> like
> >>>>> "null(identity)". Then given a query like
> >>>>> "KTable<String,Integer>.rightJoin(KTable<Integer,Boolean>)", and
> >>>>> inputs like:
> >>>>> LHS:
> >>>>> "a": 1
> >>>>> "b": 2
> >>>>>
> >>>>> RHS:
> >>>>> 1: true
> >>>>> 3: false
> >>>>>
> >>>>> a full outer join would produce:
> >>>>> "a": (1, true)
> >>>>> "b": (2, null)
> >>>>> null(3): (null, false)
> >>>>>
> >>>>> which can be correctly updated later if we get an update on the LHS:
> >>>>> PUT("c": 3)
> >>>>>
> >>>>> We'd emit for the results:
> >>>>> DELETE(null(e))
> >>>>> EMIT("c": (3, false))
> >>>>>
> >>>>> Resulting in the correct result table of:
> >>>>> "a": (1, true)
> >>>>> "b": (2, null)
> >>>>> "c": (3, false)
> >>>>>
> >>>>> As mentioned, this is tricky, and I would avoid it until we have
> >>>>> evidence that it's actually useful to cover this part of the design
> >>>>> space. Certainly, it would be a separate KIP if it came to that.
> >>>>>
> >>>>> On Wed, Jun 26, 2019 at 8:57 PM Adam Bellemare <
> >> adam.bellemare@gmail.com>
> >>>>> wrote:
> >>>>>>
> >>>>>> 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
> the
> >>>>> last
> >>>>>> commit (about 10 min ago) along with INNER join. I do not think that
> >>>>> RIGHT
> >>>>>> join and OUTER are possible given that there is no LHS key
> available,
> >> so
> >>>>>> LHSTable.outerJoinOnForeignKey(RHSTable) wouldn't even make sense.
> >> This
> >>>>> is
> >>>>>> in contrast to the current LHSTable.outerJoin(RHSTable), as they are
> >> both
> >>>>>> keyed on the same key. I have buffed up the Integration tests and
> have
> >>>>>> tried to make them more readable to ensure that we're covering all
> the
> >>>>>> scenarios. I think that if we can get more eyes on the workflow
> >> showing
> >>>>> the
> >>>>>> various LHS and RHS events and outputs then that may help us
> validate
> >>>>> that
> >>>>>> we have all the scenarios covered.
> >>>>>>
> >>>>>> With regards to the 2.3->2.4 scenario you described, I'm not
> entirely
> >>>>> sure
> >>>>>> I follow. If they want to add a FK-join, they will need to rework
> >> their
> >>>>>> code in the KStreams app and make a new release, since the
> underlying
> >>>>>> topology would be different and new internal topics would need to be
> >>>>>> created. In other words, I don't think a rolling upgrade where the
> >> user
> >>>>>> introduces a FK join would be possible since their topology would
> >>>>>> necessitate a full KStreams reset. Is this what you meant?
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> On Wed, Jun 26, 2019 at 4:10 PM John Roesler <jo...@confluent.io>
> >> wrote:
> >>>>>>
> >>>>>>> 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.bellemare@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.bellemare@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 <jo...@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
> >>>>>>>>>> <ad...@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 <
> >>>>> john@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
> >>>>>>>>>>>>
> >>>>>>>>>>
> >>>>>>>>>
> >>>>>>>
> >>>>>
> >>
> >>
> >
>
>

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
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 subscribe message to RHS
a-2) RHS lookup and send result back if there is one
a-3) emit result on LHS if any is returned

b-1) delete subscription from RHS
b-2) if there was a previous result (can easily be decided by looking up
RHS table for an existing key), send tombstone back
b-3) emit tombstone on LHS if any is returned

c-1) delete old subscription from RHS
c-2) send new subscription to RHS
c-3) if there was no previous result and there is no new result emit nothing
c-4) if there was a previous result and there is no new result emit a
tombstone LHS
c-5) if there is a new result (old result may or may not exist), emit
only new result LHS (don't emit a tombstone)


Case (a) and (b) are simple and could be implemented with a "fire and
forget" strategy. The LHS just "blindly" updates the subscription, the
RHS can process the subscription with local knowledge and may send
something back. If something is sent back, the LHS blindly emits it.

We know that for both cases, we never miss anything and we never emit
anything unnecessary.

However, using this pattern for (c), we don't get our expected result:

Issues: LHS sends both subscription updates in parallel. It does not
know if zero, one, or two result records will be produced by RHS. If RHS
produces two results, their order is not known (however, LHS would need
to emit them in the right order; also forcing RHS to always send a
result back is not a sufficient solution). If only one result comes
back, it's unclear if a second result may arrive later and thus the
result may need to be buffered... Overall, local knowledge does not seem
to be sufficient to tackle the case.

The current proposal tries to address the issue with encoding additional
information, to tell the RHS to send a tombstone back or not. But it
does not seem to be perfect, and it might result in unnecessary
tombstones as it still uses local knowledge only and thus misses some
information.

I think the main problem is, that the knowledge about the a potential
previous result and a potential new result is sharded on the RHS. Hence,
the "unsubscribe" does not know if it needs to send a tombstone back for
the case that there was an old result but there is no new result.
Similarly, the "new subscribe" cannot know if it needs to send a
tombstone or not (as it does not know if there was a previous result) if
it does not match.

To really solve the issue, I see two possible solutions (both are not
great, but I wanted to discuss them anyway):

S-1: First unsubscribe, and send new subscription after result comes
back. For this case, the RHS must always send something back to the LHS
on unsubscribe. The answer if "previous result exists/not-exist" can be
added to the new-subscription and hence RHS can either return nothing, a
tombstone, or a new result. The LHS can blindly emit whatever RHS
returns. This would also cover (a) and (b) cases. However, the overall
time to emit the join result is doubled for the (common) update case...
(we need two consecutive round-trips to the RHS).

S-2: Remember/store if a previous result exists on LHS: for this case,
(a) is handled straightforward, (b) is handled by telling RHS to send
tombstone if previous result exits, and (c) can send both request in
parallel letting the unsubscribe never return anything, and subscribe is
handled as in (b). However, we need a second store on the LHS to
remember if there was a previous result. (Also not sure how
interleaving/inflight computation might affect the algorithm...)

I think, sending unnecessary tombstones is quite bad (in very old
releases we had a similar issue and fixed it). However, I am also not
100% sure if the solutions I came up with are good enough to justify
them. (Personally, I slightly tend to prefer S-2 because I think that
the additional store is less of an issue than the increase processing time).

Would love to hear your thoughts.


-Matthias


On 6/28/19 6:19 AM, Adam Bellemare wrote:
> 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 unnecessary tombstone is (that I can
> tell...) when we do the following:
> RHS:
> (1, bar)
> 
> LHS
> (K,1)  -> Results in (K, 1, bar) being output
> (K,1) -> (K,2) ->  Results in (K, null) being output for INNER (no matching
> element on LHS)
> (K,2) -> (K,3) ->  Results in (K, null) being output for INNER (because we
> don't maintain state to know we already output the tombstone on the
> previous transition).
> (K,2) -> (K,9000) ->  Results in (K, null)... etc.
> 
> Byte versioning is going in today, then I hope to get back to addressing a
> number of John's previous questions in the PR.
> 
> Adam
> 
> 
> On Thu, Jun 27, 2019 at 5:47 PM Matthias J. Sax <ma...@confluent.io>
> wrote:
> 
>> 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 unnecessary tombstone? Seems I a missing case?
>>
>> Also for my own understanding: the `instruction` is only part of the
>> message? It is no necessary to store it in the RHS auxiliary store, right?
>>
>> About right/full-outer joins. Agreed. Getting left-joins would be awesome!
>>
>> About upgrading: Good call John! Adding a version byte for subscription
>> and response is good forward thinking. I personally prefer version
>> numbers, too, as they carry more information.
>>
>> Thanks for all the hard to everybody involved!
>>
>>
>> -Matthias
>>
>> On 6/27/19 1:44 PM, John Roesler wrote:
>>> 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 revisions,
>>> we can just use one byte. The worst case is that we run out of numbers
>>> and reserve the last one to mean, "consult another field for the
>>> actual version number". It seems like a single byte on each message
>>> isn't too much to pay.
>>>
>>> Since you point it out, we might as well put a version number on the
>>> SubscriptionResponseWrapper as well. It may not be needed, but if we
>>> ever need it, even just once, we'll be glad we have it.
>>>
>>> Regarding the instructions field, we can also serialize the enum very
>>> compactly as a single byte (which is the same size a boolean takes
>>> anyway), so it seems like an Enum in Java-land and a byte on the wire
>>> is a good choice.
>>>
>>> Agreed on the right and full outer joins, it doesn't seem necessary
>>> right now, although I am happy to see the left join "join" the party,
>>> since as you said, we were so close to it anyway. Can you also add it
>>> to the KIP?
>>>
>>> Thanks as always for your awesome efforts on this,
>>> -John
>>>
>>> On Thu, Jun 27, 2019 at 3:04 PM Adam Bellemare <ad...@gmail.com>
>> wrote:
>>>>
>>>> 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 microservices - there are
>>>> indeed cases where a breaking change must be made, and forward
>>>> compatibility will provide us with no out other than requiring a full
>> stop
>>>> and full upgrade for all nodes, shifting us back towards STRATEGY 1.
>>>>
>>>> My preference is STRATEGY 1 with instructions as an ENUM, and we can
>>>> certainly include a version. Would it make sense to include a version
>>>> number in  SubscriptionResponseWrapper as well? Currently we don't have
>> any
>>>> instructions in there, as I removed the boolean, but it is certainly
>>>> plausible that it could happen in the future. I don't *think* we'll need
>>>> it, but I also didn't think we'd need it for SubscriptionWrapper and
>> here
>>>> we are.
>>>>
>>>> Thanks for the thoughts, and the info on the right-key. That was
>>>> enlightening, though I can't think of a use-case for it *at this point
>> in
>>>> time*. :)
>>>>
>>>> Adam
>>>>
>>>>
>>>>
>>>> On Thu, Jun 27, 2019 at 12:29 PM John Roesler <jo...@confluent.io>
>> wrote:
>>>>
>>>>> 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 `nullKey: (null, rhsValue)`, but we
>>>>> don't currently allow null keys in Streams. It actually is possible to
>>>>> define them, and therefore to add right- and full-outer foreign-key
>>>>> joins later, but it's non-trivial in a streaming context with
>>>>> continuously updated results. (See the PS if you're curious what I'm
>>>>> thinking). You're correct, right- and full-outer joins are trivial on
>>>>> our current 1:1 table joins because they are equi-joins.
>>>>>
>>>>> Regarding the transition, it sounds like what you're proposing is that
>>>>> we would say, "adding a foreign-key join to your topology requires a
>>>>> full application reset (or a new application id)". This is also an
>>>>> acceptable constraint to place on the feature, but not strictly
>>>>> necessary. Since 2.3, it's now possible to give all the state in your
>>>>> application stable names. This means that it's no longer true that
>>>>> adding a node to your topology graph would break its structure, and it
>>>>> does become possible to add new operators and simply restart the app.
>>>>> Revisiting my prior thought, though, I think the problem is not
>>>>> specific to your feature. For example, adding a new grouped
>>>>> aggregation would produce a new repartition topic, but the repartition
>>>>> topic partitions might get assigned to old nodes in the middle of a
>>>>> rolling bounce, and they would need to just ignore them. This
>>>>> requirement is the same for the repartition topics in the FK join, so
>>>>> it's orthogonal to your design.
>>>>>
>>>>> Back to the first concern, though, I'm not sure I followed the
>>>>> explanation. As a thought experiment, let's imagine that Joe hadn't
>>>>> taken the time to experiment with your feature branch. We wouldn't
>>>>> have noticed the problem until the feature was already released in
>>>>> 2.4. So the wire protocol on that PK->FK subscription topic would have
>>>>> been V1: "FK,PK,HASH,BOOLEAN". Then, Joe would have let us know the
>>>>> problem once they picked up the feature, so we would want to implement
>>>>> your proposed fix and change the wire protocol to V2:
>>>>> "FK,PK,HASH,INSTRUCTIONS" in 2.5. Upon rolling out the update, we
>>>>> would see both 2.4 nodes encountering V2 messages and 2.5 nodes
>>>>> encountering V1 messages. How can they both detect that they are
>>>>> attempting to process a newer or older protocol? If they can detect
>>>>> it, then what should they do?
>>>>>
>>>>> From experience, there are two basic solutions to this problem:
>>>>>
>>>>> STRATEGY1. Add a protocol version to the message (could be a number at
>>>>> the start of the message payload, or it could be a number in the
>>>>> message headers, not sure if it matters much. Payload is probably more
>>>>> compact, since the header would need a name.) In this case, the 2.4
>>>>> worker would know that it's max protocol version is V1, and when it
>>>>> sees the V2 message, it knows that it can't handle it properly. Rather
>>>>> than doing something wrong, it would just not do anything. This means
>>>>> it would stop the task, if not shut down the whole instance. On the
>>>>> other hand, a 2.5 worker would have some defined logic for how to
>>>>> handle all versions (V1 and V2), so once the upgrade is complete, all
>>>>> messages can be processed.
>>>>>
>>>>> STRATEGY2. Make the schema forward-compatible. Basically, we ensure
>>>>> that new fields can only be appended to the message schema, and that
>>>>> older workers using only a prefix of the full message would still
>>>>> behave correctly. Using the example above, we'd instead evolve the
>>>>> schema to V2': "FK,PK,HASH,BOOLEAN,INSTRUCTIONS", and continue to set
>>>>> the boolean field to true for the "new" foreign key. Then, 2.4 workers
>>>>> encountering the a "new FK" message would just see the prefix of the
>>>>> payload that makes sense to them, and they would still continue
>>>>> processing the messages as they always have. Only after the 2.5 code
>>>>> is fully rolled out to the cluster would we be sure to see the desired
>>>>> behavior. Note: in the reverse case, a 2.5 worker knows how to fully
>>>>> parse the new message format, even if it plans to ignore the BOOLEAN
>>>>> field.
>>>>>
>>>>> There are some tradeoffs between these strategies: STRATEGY1 ensures
>>>>> that all messages are only handled by workers that can properly handle
>>>>> them, although it results in processing stalls while there are still
>>>>> old nodes in the cluster. STRATEGY2 ensures that all messages can be
>>>>> processed by all nodes, so there are no stalls, but we can never
>>>>> remove fields from the message, so if there are a lot of revisions in
>>>>> the future, the payloads will become bloated. Also, it's not clear
>>>>> that you can actually pull off STRATEGY2 in all cases. If there's some
>>>>> new kind of message you want to send that has no way to be correctly
>>>>> processed at all under the 2.4 code paths, the prefix thing simply
>>>>> doesn't work. Etc.
>>>>>
>>>>> Also, note that you can modify the above strategies by instead
>>>>> designing the message fields for extensibility. E.g., if you make the
>>>>> instructions field an enum, then you can make sure that the default
>>>>> case is handled sensibly (probably similarly to STRATEGY1, just choke
>>>>> on unknown instructions) and that you never remove an instruction type
>>>>> from the enum in future versions.
>>>>>
>>>>> Does this make sense?
>>>>> -John
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> PS:
>>>>> We can define null keys for streaming tables, but it's tricky.
>>>>>
>>>>> Specifically, you'd want to define some concept of null keys that
>>>>> allows all null keys to be unique, but _also_ to have a fixed
>>>>> identity, so that a particular null-key can be updated later. One
>>>>> example could be to union the existing keyspace with a new
>>>>> null-keyspace, where normal keys are like "key" and null-keys are like
>>>>> "null(identity)". Then given a query like
>>>>> "KTable<String,Integer>.rightJoin(KTable<Integer,Boolean>)", and
>>>>> inputs like:
>>>>> LHS:
>>>>> "a": 1
>>>>> "b": 2
>>>>>
>>>>> RHS:
>>>>> 1: true
>>>>> 3: false
>>>>>
>>>>> a full outer join would produce:
>>>>> "a": (1, true)
>>>>> "b": (2, null)
>>>>> null(3): (null, false)
>>>>>
>>>>> which can be correctly updated later if we get an update on the LHS:
>>>>> PUT("c": 3)
>>>>>
>>>>> We'd emit for the results:
>>>>> DELETE(null(e))
>>>>> EMIT("c": (3, false))
>>>>>
>>>>> Resulting in the correct result table of:
>>>>> "a": (1, true)
>>>>> "b": (2, null)
>>>>> "c": (3, false)
>>>>>
>>>>> As mentioned, this is tricky, and I would avoid it until we have
>>>>> evidence that it's actually useful to cover this part of the design
>>>>> space. Certainly, it would be a separate KIP if it came to that.
>>>>>
>>>>> On Wed, Jun 26, 2019 at 8:57 PM Adam Bellemare <
>> adam.bellemare@gmail.com>
>>>>> wrote:
>>>>>>
>>>>>> 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 the
>>>>> last
>>>>>> commit (about 10 min ago) along with INNER join. I do not think that
>>>>> RIGHT
>>>>>> join and OUTER are possible given that there is no LHS key available,
>> so
>>>>>> LHSTable.outerJoinOnForeignKey(RHSTable) wouldn't even make sense.
>> This
>>>>> is
>>>>>> in contrast to the current LHSTable.outerJoin(RHSTable), as they are
>> both
>>>>>> keyed on the same key. I have buffed up the Integration tests and have
>>>>>> tried to make them more readable to ensure that we're covering all the
>>>>>> scenarios. I think that if we can get more eyes on the workflow
>> showing
>>>>> the
>>>>>> various LHS and RHS events and outputs then that may help us validate
>>>>> that
>>>>>> we have all the scenarios covered.
>>>>>>
>>>>>> With regards to the 2.3->2.4 scenario you described, I'm not entirely
>>>>> sure
>>>>>> I follow. If they want to add a FK-join, they will need to rework
>> their
>>>>>> code in the KStreams app and make a new release, since the underlying
>>>>>> topology would be different and new internal topics would need to be
>>>>>> created. In other words, I don't think a rolling upgrade where the
>> user
>>>>>> introduces a FK join would be possible since their topology would
>>>>>> necessitate a full KStreams reset. Is this what you meant?
>>>>>>
>>>>>>
>>>>>>
>>>>>> On Wed, Jun 26, 2019 at 4:10 PM John Roesler <jo...@confluent.io>
>> wrote:
>>>>>>
>>>>>>> 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.bellemare@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.bellemare@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 <jo...@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
>>>>>>>>>> <ad...@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 <
>>>>> john@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
>>>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>
>>
>>
> 


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

Posted by Adam Bellemare <ad...@gmail.com>.
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 unnecessary tombstone is (that I can
tell...) when we do the following:
RHS:
(1, bar)

LHS
(K,1)  -> Results in (K, 1, bar) being output
(K,1) -> (K,2) ->  Results in (K, null) being output for INNER (no matching
element on LHS)
(K,2) -> (K,3) ->  Results in (K, null) being output for INNER (because we
don't maintain state to know we already output the tombstone on the
previous transition).
(K,2) -> (K,9000) ->  Results in (K, null)... etc.

Byte versioning is going in today, then I hope to get back to addressing a
number of John's previous questions in the PR.

Adam


On Thu, Jun 27, 2019 at 5:47 PM Matthias J. Sax <ma...@confluent.io>
wrote:

> 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 unnecessary tombstone? Seems I a missing case?
>
> Also for my own understanding: the `instruction` is only part of the
> message? It is no necessary to store it in the RHS auxiliary store, right?
>
> About right/full-outer joins. Agreed. Getting left-joins would be awesome!
>
> About upgrading: Good call John! Adding a version byte for subscription
> and response is good forward thinking. I personally prefer version
> numbers, too, as they carry more information.
>
> Thanks for all the hard to everybody involved!
>
>
> -Matthias
>
> On 6/27/19 1:44 PM, John Roesler wrote:
> > 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 revisions,
> > we can just use one byte. The worst case is that we run out of numbers
> > and reserve the last one to mean, "consult another field for the
> > actual version number". It seems like a single byte on each message
> > isn't too much to pay.
> >
> > Since you point it out, we might as well put a version number on the
> > SubscriptionResponseWrapper as well. It may not be needed, but if we
> > ever need it, even just once, we'll be glad we have it.
> >
> > Regarding the instructions field, we can also serialize the enum very
> > compactly as a single byte (which is the same size a boolean takes
> > anyway), so it seems like an Enum in Java-land and a byte on the wire
> > is a good choice.
> >
> > Agreed on the right and full outer joins, it doesn't seem necessary
> > right now, although I am happy to see the left join "join" the party,
> > since as you said, we were so close to it anyway. Can you also add it
> > to the KIP?
> >
> > Thanks as always for your awesome efforts on this,
> > -John
> >
> > On Thu, Jun 27, 2019 at 3:04 PM Adam Bellemare <ad...@gmail.com>
> wrote:
> >>
> >> 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 microservices - there are
> >> indeed cases where a breaking change must be made, and forward
> >> compatibility will provide us with no out other than requiring a full
> stop
> >> and full upgrade for all nodes, shifting us back towards STRATEGY 1.
> >>
> >> My preference is STRATEGY 1 with instructions as an ENUM, and we can
> >> certainly include a version. Would it make sense to include a version
> >> number in  SubscriptionResponseWrapper as well? Currently we don't have
> any
> >> instructions in there, as I removed the boolean, but it is certainly
> >> plausible that it could happen in the future. I don't *think* we'll need
> >> it, but I also didn't think we'd need it for SubscriptionWrapper and
> here
> >> we are.
> >>
> >> Thanks for the thoughts, and the info on the right-key. That was
> >> enlightening, though I can't think of a use-case for it *at this point
> in
> >> time*. :)
> >>
> >> Adam
> >>
> >>
> >>
> >> On Thu, Jun 27, 2019 at 12:29 PM John Roesler <jo...@confluent.io>
> wrote:
> >>
> >>> 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 `nullKey: (null, rhsValue)`, but we
> >>> don't currently allow null keys in Streams. It actually is possible to
> >>> define them, and therefore to add right- and full-outer foreign-key
> >>> joins later, but it's non-trivial in a streaming context with
> >>> continuously updated results. (See the PS if you're curious what I'm
> >>> thinking). You're correct, right- and full-outer joins are trivial on
> >>> our current 1:1 table joins because they are equi-joins.
> >>>
> >>> Regarding the transition, it sounds like what you're proposing is that
> >>> we would say, "adding a foreign-key join to your topology requires a
> >>> full application reset (or a new application id)". This is also an
> >>> acceptable constraint to place on the feature, but not strictly
> >>> necessary. Since 2.3, it's now possible to give all the state in your
> >>> application stable names. This means that it's no longer true that
> >>> adding a node to your topology graph would break its structure, and it
> >>> does become possible to add new operators and simply restart the app.
> >>> Revisiting my prior thought, though, I think the problem is not
> >>> specific to your feature. For example, adding a new grouped
> >>> aggregation would produce a new repartition topic, but the repartition
> >>> topic partitions might get assigned to old nodes in the middle of a
> >>> rolling bounce, and they would need to just ignore them. This
> >>> requirement is the same for the repartition topics in the FK join, so
> >>> it's orthogonal to your design.
> >>>
> >>> Back to the first concern, though, I'm not sure I followed the
> >>> explanation. As a thought experiment, let's imagine that Joe hadn't
> >>> taken the time to experiment with your feature branch. We wouldn't
> >>> have noticed the problem until the feature was already released in
> >>> 2.4. So the wire protocol on that PK->FK subscription topic would have
> >>> been V1: "FK,PK,HASH,BOOLEAN". Then, Joe would have let us know the
> >>> problem once they picked up the feature, so we would want to implement
> >>> your proposed fix and change the wire protocol to V2:
> >>> "FK,PK,HASH,INSTRUCTIONS" in 2.5. Upon rolling out the update, we
> >>> would see both 2.4 nodes encountering V2 messages and 2.5 nodes
> >>> encountering V1 messages. How can they both detect that they are
> >>> attempting to process a newer or older protocol? If they can detect
> >>> it, then what should they do?
> >>>
> >>> From experience, there are two basic solutions to this problem:
> >>>
> >>> STRATEGY1. Add a protocol version to the message (could be a number at
> >>> the start of the message payload, or it could be a number in the
> >>> message headers, not sure if it matters much. Payload is probably more
> >>> compact, since the header would need a name.) In this case, the 2.4
> >>> worker would know that it's max protocol version is V1, and when it
> >>> sees the V2 message, it knows that it can't handle it properly. Rather
> >>> than doing something wrong, it would just not do anything. This means
> >>> it would stop the task, if not shut down the whole instance. On the
> >>> other hand, a 2.5 worker would have some defined logic for how to
> >>> handle all versions (V1 and V2), so once the upgrade is complete, all
> >>> messages can be processed.
> >>>
> >>> STRATEGY2. Make the schema forward-compatible. Basically, we ensure
> >>> that new fields can only be appended to the message schema, and that
> >>> older workers using only a prefix of the full message would still
> >>> behave correctly. Using the example above, we'd instead evolve the
> >>> schema to V2': "FK,PK,HASH,BOOLEAN,INSTRUCTIONS", and continue to set
> >>> the boolean field to true for the "new" foreign key. Then, 2.4 workers
> >>> encountering the a "new FK" message would just see the prefix of the
> >>> payload that makes sense to them, and they would still continue
> >>> processing the messages as they always have. Only after the 2.5 code
> >>> is fully rolled out to the cluster would we be sure to see the desired
> >>> behavior. Note: in the reverse case, a 2.5 worker knows how to fully
> >>> parse the new message format, even if it plans to ignore the BOOLEAN
> >>> field.
> >>>
> >>> There are some tradeoffs between these strategies: STRATEGY1 ensures
> >>> that all messages are only handled by workers that can properly handle
> >>> them, although it results in processing stalls while there are still
> >>> old nodes in the cluster. STRATEGY2 ensures that all messages can be
> >>> processed by all nodes, so there are no stalls, but we can never
> >>> remove fields from the message, so if there are a lot of revisions in
> >>> the future, the payloads will become bloated. Also, it's not clear
> >>> that you can actually pull off STRATEGY2 in all cases. If there's some
> >>> new kind of message you want to send that has no way to be correctly
> >>> processed at all under the 2.4 code paths, the prefix thing simply
> >>> doesn't work. Etc.
> >>>
> >>> Also, note that you can modify the above strategies by instead
> >>> designing the message fields for extensibility. E.g., if you make the
> >>> instructions field an enum, then you can make sure that the default
> >>> case is handled sensibly (probably similarly to STRATEGY1, just choke
> >>> on unknown instructions) and that you never remove an instruction type
> >>> from the enum in future versions.
> >>>
> >>> Does this make sense?
> >>> -John
> >>>
> >>>
> >>>
> >>>
> >>> PS:
> >>> We can define null keys for streaming tables, but it's tricky.
> >>>
> >>> Specifically, you'd want to define some concept of null keys that
> >>> allows all null keys to be unique, but _also_ to have a fixed
> >>> identity, so that a particular null-key can be updated later. One
> >>> example could be to union the existing keyspace with a new
> >>> null-keyspace, where normal keys are like "key" and null-keys are like
> >>> "null(identity)". Then given a query like
> >>> "KTable<String,Integer>.rightJoin(KTable<Integer,Boolean>)", and
> >>> inputs like:
> >>> LHS:
> >>> "a": 1
> >>> "b": 2
> >>>
> >>> RHS:
> >>> 1: true
> >>> 3: false
> >>>
> >>> a full outer join would produce:
> >>> "a": (1, true)
> >>> "b": (2, null)
> >>> null(3): (null, false)
> >>>
> >>> which can be correctly updated later if we get an update on the LHS:
> >>> PUT("c": 3)
> >>>
> >>> We'd emit for the results:
> >>> DELETE(null(e))
> >>> EMIT("c": (3, false))
> >>>
> >>> Resulting in the correct result table of:
> >>> "a": (1, true)
> >>> "b": (2, null)
> >>> "c": (3, false)
> >>>
> >>> As mentioned, this is tricky, and I would avoid it until we have
> >>> evidence that it's actually useful to cover this part of the design
> >>> space. Certainly, it would be a separate KIP if it came to that.
> >>>
> >>> On Wed, Jun 26, 2019 at 8:57 PM Adam Bellemare <
> adam.bellemare@gmail.com>
> >>> wrote:
> >>>>
> >>>> 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 the
> >>> last
> >>>> commit (about 10 min ago) along with INNER join. I do not think that
> >>> RIGHT
> >>>> join and OUTER are possible given that there is no LHS key available,
> so
> >>>> LHSTable.outerJoinOnForeignKey(RHSTable) wouldn't even make sense.
> This
> >>> is
> >>>> in contrast to the current LHSTable.outerJoin(RHSTable), as they are
> both
> >>>> keyed on the same key. I have buffed up the Integration tests and have
> >>>> tried to make them more readable to ensure that we're covering all the
> >>>> scenarios. I think that if we can get more eyes on the workflow
> showing
> >>> the
> >>>> various LHS and RHS events and outputs then that may help us validate
> >>> that
> >>>> we have all the scenarios covered.
> >>>>
> >>>> With regards to the 2.3->2.4 scenario you described, I'm not entirely
> >>> sure
> >>>> I follow. If they want to add a FK-join, they will need to rework
> their
> >>>> code in the KStreams app and make a new release, since the underlying
> >>>> topology would be different and new internal topics would need to be
> >>>> created. In other words, I don't think a rolling upgrade where the
> user
> >>>> introduces a FK join would be possible since their topology would
> >>>> necessitate a full KStreams reset. Is this what you meant?
> >>>>
> >>>>
> >>>>
> >>>> On Wed, Jun 26, 2019 at 4:10 PM John Roesler <jo...@confluent.io>
> wrote:
> >>>>
> >>>>> 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.bellemare@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.bellemare@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 <jo...@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
> >>>>>>>> <ad...@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 <
> >>> john@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
> >>>>>>>>>>
> >>>>>>>>
> >>>>>>>
> >>>>>
> >>>
>
>

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

Posted by "Matthias J. Sax" <ma...@confluent.io>.
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 unnecessary tombstone? Seems I a missing case?

Also for my own understanding: the `instruction` is only part of the
message? It is no necessary to store it in the RHS auxiliary store, right?

About right/full-outer joins. Agreed. Getting left-joins would be awesome!

About upgrading: Good call John! Adding a version byte for subscription
and response is good forward thinking. I personally prefer version
numbers, too, as they carry more information.

Thanks for all the hard to everybody involved!


-Matthias

On 6/27/19 1:44 PM, John Roesler wrote:
> 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 revisions,
> we can just use one byte. The worst case is that we run out of numbers
> and reserve the last one to mean, "consult another field for the
> actual version number". It seems like a single byte on each message
> isn't too much to pay.
> 
> Since you point it out, we might as well put a version number on the
> SubscriptionResponseWrapper as well. It may not be needed, but if we
> ever need it, even just once, we'll be glad we have it.
> 
> Regarding the instructions field, we can also serialize the enum very
> compactly as a single byte (which is the same size a boolean takes
> anyway), so it seems like an Enum in Java-land and a byte on the wire
> is a good choice.
> 
> Agreed on the right and full outer joins, it doesn't seem necessary
> right now, although I am happy to see the left join "join" the party,
> since as you said, we were so close to it anyway. Can you also add it
> to the KIP?
> 
> Thanks as always for your awesome efforts on this,
> -John
> 
> On Thu, Jun 27, 2019 at 3:04 PM Adam Bellemare <ad...@gmail.com> wrote:
>>
>> 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 microservices - there are
>> indeed cases where a breaking change must be made, and forward
>> compatibility will provide us with no out other than requiring a full stop
>> and full upgrade for all nodes, shifting us back towards STRATEGY 1.
>>
>> My preference is STRATEGY 1 with instructions as an ENUM, and we can
>> certainly include a version. Would it make sense to include a version
>> number in  SubscriptionResponseWrapper as well? Currently we don't have any
>> instructions in there, as I removed the boolean, but it is certainly
>> plausible that it could happen in the future. I don't *think* we'll need
>> it, but I also didn't think we'd need it for SubscriptionWrapper and here
>> we are.
>>
>> Thanks for the thoughts, and the info on the right-key. That was
>> enlightening, though I can't think of a use-case for it *at this point in
>> time*. :)
>>
>> Adam
>>
>>
>>
>> On Thu, Jun 27, 2019 at 12:29 PM John Roesler <jo...@confluent.io> wrote:
>>
>>> 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 `nullKey: (null, rhsValue)`, but we
>>> don't currently allow null keys in Streams. It actually is possible to
>>> define them, and therefore to add right- and full-outer foreign-key
>>> joins later, but it's non-trivial in a streaming context with
>>> continuously updated results. (See the PS if you're curious what I'm
>>> thinking). You're correct, right- and full-outer joins are trivial on
>>> our current 1:1 table joins because they are equi-joins.
>>>
>>> Regarding the transition, it sounds like what you're proposing is that
>>> we would say, "adding a foreign-key join to your topology requires a
>>> full application reset (or a new application id)". This is also an
>>> acceptable constraint to place on the feature, but not strictly
>>> necessary. Since 2.3, it's now possible to give all the state in your
>>> application stable names. This means that it's no longer true that
>>> adding a node to your topology graph would break its structure, and it
>>> does become possible to add new operators and simply restart the app.
>>> Revisiting my prior thought, though, I think the problem is not
>>> specific to your feature. For example, adding a new grouped
>>> aggregation would produce a new repartition topic, but the repartition
>>> topic partitions might get assigned to old nodes in the middle of a
>>> rolling bounce, and they would need to just ignore them. This
>>> requirement is the same for the repartition topics in the FK join, so
>>> it's orthogonal to your design.
>>>
>>> Back to the first concern, though, I'm not sure I followed the
>>> explanation. As a thought experiment, let's imagine that Joe hadn't
>>> taken the time to experiment with your feature branch. We wouldn't
>>> have noticed the problem until the feature was already released in
>>> 2.4. So the wire protocol on that PK->FK subscription topic would have
>>> been V1: "FK,PK,HASH,BOOLEAN". Then, Joe would have let us know the
>>> problem once they picked up the feature, so we would want to implement
>>> your proposed fix and change the wire protocol to V2:
>>> "FK,PK,HASH,INSTRUCTIONS" in 2.5. Upon rolling out the update, we
>>> would see both 2.4 nodes encountering V2 messages and 2.5 nodes
>>> encountering V1 messages. How can they both detect that they are
>>> attempting to process a newer or older protocol? If they can detect
>>> it, then what should they do?
>>>
>>> From experience, there are two basic solutions to this problem:
>>>
>>> STRATEGY1. Add a protocol version to the message (could be a number at
>>> the start of the message payload, or it could be a number in the
>>> message headers, not sure if it matters much. Payload is probably more
>>> compact, since the header would need a name.) In this case, the 2.4
>>> worker would know that it's max protocol version is V1, and when it
>>> sees the V2 message, it knows that it can't handle it properly. Rather
>>> than doing something wrong, it would just not do anything. This means
>>> it would stop the task, if not shut down the whole instance. On the
>>> other hand, a 2.5 worker would have some defined logic for how to
>>> handle all versions (V1 and V2), so once the upgrade is complete, all
>>> messages can be processed.
>>>
>>> STRATEGY2. Make the schema forward-compatible. Basically, we ensure
>>> that new fields can only be appended to the message schema, and that
>>> older workers using only a prefix of the full message would still
>>> behave correctly. Using the example above, we'd instead evolve the
>>> schema to V2': "FK,PK,HASH,BOOLEAN,INSTRUCTIONS", and continue to set
>>> the boolean field to true for the "new" foreign key. Then, 2.4 workers
>>> encountering the a "new FK" message would just see the prefix of the
>>> payload that makes sense to them, and they would still continue
>>> processing the messages as they always have. Only after the 2.5 code
>>> is fully rolled out to the cluster would we be sure to see the desired
>>> behavior. Note: in the reverse case, a 2.5 worker knows how to fully
>>> parse the new message format, even if it plans to ignore the BOOLEAN
>>> field.
>>>
>>> There are some tradeoffs between these strategies: STRATEGY1 ensures
>>> that all messages are only handled by workers that can properly handle
>>> them, although it results in processing stalls while there are still
>>> old nodes in the cluster. STRATEGY2 ensures that all messages can be
>>> processed by all nodes, so there are no stalls, but we can never
>>> remove fields from the message, so if there are a lot of revisions in
>>> the future, the payloads will become bloated. Also, it's not clear
>>> that you can actually pull off STRATEGY2 in all cases. If there's some
>>> new kind of message you want to send that has no way to be correctly
>>> processed at all under the 2.4 code paths, the prefix thing simply
>>> doesn't work. Etc.
>>>
>>> Also, note that you can modify the above strategies by instead
>>> designing the message fields for extensibility. E.g., if you make the
>>> instructions field an enum, then you can make sure that the default
>>> case is handled sensibly (probably similarly to STRATEGY1, just choke
>>> on unknown instructions) and that you never remove an instruction type
>>> from the enum in future versions.
>>>
>>> Does this make sense?
>>> -John
>>>
>>>
>>>
>>>
>>> PS:
>>> We can define null keys for streaming tables, but it's tricky.
>>>
>>> Specifically, you'd want to define some concept of null keys that
>>> allows all null keys to be unique, but _also_ to have a fixed
>>> identity, so that a particular null-key can be updated later. One
>>> example could be to union the existing keyspace with a new
>>> null-keyspace, where normal keys are like "key" and null-keys are like
>>> "null(identity)". Then given a query like
>>> "KTable<String,Integer>.rightJoin(KTable<Integer,Boolean>)", and
>>> inputs like:
>>> LHS:
>>> "a": 1
>>> "b": 2
>>>
>>> RHS:
>>> 1: true
>>> 3: false
>>>
>>> a full outer join would produce:
>>> "a": (1, true)
>>> "b": (2, null)
>>> null(3): (null, false)
>>>
>>> which can be correctly updated later if we get an update on the LHS:
>>> PUT("c": 3)
>>>
>>> We'd emit for the results:
>>> DELETE(null(e))
>>> EMIT("c": (3, false))
>>>
>>> Resulting in the correct result table of:
>>> "a": (1, true)
>>> "b": (2, null)
>>> "c": (3, false)
>>>
>>> As mentioned, this is tricky, and I would avoid it until we have
>>> evidence that it's actually useful to cover this part of the design
>>> space. Certainly, it would be a separate KIP if it came to that.
>>>
>>> On Wed, Jun 26, 2019 at 8:57 PM Adam Bellemare <ad...@gmail.com>
>>> wrote:
>>>>
>>>> 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 the
>>> last
>>>> commit (about 10 min ago) along with INNER join. I do not think that
>>> RIGHT
>>>> join and OUTER are possible given that there is no LHS key available, so
>>>> LHSTable.outerJoinOnForeignKey(RHSTable) wouldn't even make sense. This
>>> is
>>>> in contrast to the current LHSTable.outerJoin(RHSTable), as they are both
>>>> keyed on the same key. I have buffed up the Integration tests and have
>>>> tried to make them more readable to ensure that we're covering all the
>>>> scenarios. I think that if we can get more eyes on the workflow showing
>>> the
>>>> various LHS and RHS events and outputs then that may help us validate
>>> that
>>>> we have all the scenarios covered.
>>>>
>>>> With regards to the 2.3->2.4 scenario you described, I'm not entirely
>>> sure
>>>> I follow. If they want to add a FK-join, they will need to rework their
>>>> code in the KStreams app and make a new release, since the underlying
>>>> topology would be different and new internal topics would need to be
>>>> created. In other words, I don't think a rolling upgrade where the user
>>>> introduces a FK join would be possible since their topology would
>>>> necessitate a full KStreams reset. Is this what you meant?
>>>>
>>>>
>>>>
>>>> On Wed, Jun 26, 2019 at 4:10 PM John Roesler <jo...@confluent.io> wrote:
>>>>
>>>>> 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.bellemare@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.bellemare@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 <jo...@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
>>>>>>>> <ad...@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 <
>>> john@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
>>>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>>
>>>


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

Posted by John Roesler <jo...@confluent.io>.
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 revisions,
we can just use one byte. The worst case is that we run out of numbers
and reserve the last one to mean, "consult another field for the
actual version number". It seems like a single byte on each message
isn't too much to pay.

Since you point it out, we might as well put a version number on the
SubscriptionResponseWrapper as well. It may not be needed, but if we
ever need it, even just once, we'll be glad we have it.

Regarding the instructions field, we can also serialize the enum very
compactly as a single byte (which is the same size a boolean takes
anyway), so it seems like an Enum in Java-land and a byte on the wire
is a good choice.

Agreed on the right and full outer joins, it doesn't seem necessary
right now, although I am happy to see the left join "join" the party,
since as you said, we were so close to it anyway. Can you also add it
to the KIP?

Thanks as always for your awesome efforts on this,
-John

On Thu, Jun 27, 2019 at 3:04 PM Adam Bellemare <ad...@gmail.com> wrote:
>
> 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 microservices - there are
> indeed cases where a breaking change must be made, and forward
> compatibility will provide us with no out other than requiring a full stop
> and full upgrade for all nodes, shifting us back towards STRATEGY 1.
>
> My preference is STRATEGY 1 with instructions as an ENUM, and we can
> certainly include a version. Would it make sense to include a version
> number in  SubscriptionResponseWrapper as well? Currently we don't have any
> instructions in there, as I removed the boolean, but it is certainly
> plausible that it could happen in the future. I don't *think* we'll need
> it, but I also didn't think we'd need it for SubscriptionWrapper and here
> we are.
>
> Thanks for the thoughts, and the info on the right-key. That was
> enlightening, though I can't think of a use-case for it *at this point in
> time*. :)
>
> Adam
>
>
>
> On Thu, Jun 27, 2019 at 12:29 PM John Roesler <jo...@confluent.io> wrote:
>
> > 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 `nullKey: (null, rhsValue)`, but we
> > don't currently allow null keys in Streams. It actually is possible to
> > define them, and therefore to add right- and full-outer foreign-key
> > joins later, but it's non-trivial in a streaming context with
> > continuously updated results. (See the PS if you're curious what I'm
> > thinking). You're correct, right- and full-outer joins are trivial on
> > our current 1:1 table joins because they are equi-joins.
> >
> > Regarding the transition, it sounds like what you're proposing is that
> > we would say, "adding a foreign-key join to your topology requires a
> > full application reset (or a new application id)". This is also an
> > acceptable constraint to place on the feature, but not strictly
> > necessary. Since 2.3, it's now possible to give all the state in your
> > application stable names. This means that it's no longer true that
> > adding a node to your topology graph would break its structure, and it
> > does become possible to add new operators and simply restart the app.
> > Revisiting my prior thought, though, I think the problem is not
> > specific to your feature. For example, adding a new grouped
> > aggregation would produce a new repartition topic, but the repartition
> > topic partitions might get assigned to old nodes in the middle of a
> > rolling bounce, and they would need to just ignore them. This
> > requirement is the same for the repartition topics in the FK join, so
> > it's orthogonal to your design.
> >
> > Back to the first concern, though, I'm not sure I followed the
> > explanation. As a thought experiment, let's imagine that Joe hadn't
> > taken the time to experiment with your feature branch. We wouldn't
> > have noticed the problem until the feature was already released in
> > 2.4. So the wire protocol on that PK->FK subscription topic would have
> > been V1: "FK,PK,HASH,BOOLEAN". Then, Joe would have let us know the
> > problem once they picked up the feature, so we would want to implement
> > your proposed fix and change the wire protocol to V2:
> > "FK,PK,HASH,INSTRUCTIONS" in 2.5. Upon rolling out the update, we
> > would see both 2.4 nodes encountering V2 messages and 2.5 nodes
> > encountering V1 messages. How can they both detect that they are
> > attempting to process a newer or older protocol? If they can detect
> > it, then what should they do?
> >
> > From experience, there are two basic solutions to this problem:
> >
> > STRATEGY1. Add a protocol version to the message (could be a number at
> > the start of the message payload, or it could be a number in the
> > message headers, not sure if it matters much. Payload is probably more
> > compact, since the header would need a name.) In this case, the 2.4
> > worker would know that it's max protocol version is V1, and when it
> > sees the V2 message, it knows that it can't handle it properly. Rather
> > than doing something wrong, it would just not do anything. This means
> > it would stop the task, if not shut down the whole instance. On the
> > other hand, a 2.5 worker would have some defined logic for how to
> > handle all versions (V1 and V2), so once the upgrade is complete, all
> > messages can be processed.
> >
> > STRATEGY2. Make the schema forward-compatible. Basically, we ensure
> > that new fields can only be appended to the message schema, and that
> > older workers using only a prefix of the full message would still
> > behave correctly. Using the example above, we'd instead evolve the
> > schema to V2': "FK,PK,HASH,BOOLEAN,INSTRUCTIONS", and continue to set
> > the boolean field to true for the "new" foreign key. Then, 2.4 workers
> > encountering the a "new FK" message would just see the prefix of the
> > payload that makes sense to them, and they would still continue
> > processing the messages as they always have. Only after the 2.5 code
> > is fully rolled out to the cluster would we be sure to see the desired
> > behavior. Note: in the reverse case, a 2.5 worker knows how to fully
> > parse the new message format, even if it plans to ignore the BOOLEAN
> > field.
> >
> > There are some tradeoffs between these strategies: STRATEGY1 ensures
> > that all messages are only handled by workers that can properly handle
> > them, although it results in processing stalls while there are still
> > old nodes in the cluster. STRATEGY2 ensures that all messages can be
> > processed by all nodes, so there are no stalls, but we can never
> > remove fields from the message, so if there are a lot of revisions in
> > the future, the payloads will become bloated. Also, it's not clear
> > that you can actually pull off STRATEGY2 in all cases. If there's some
> > new kind of message you want to send that has no way to be correctly
> > processed at all under the 2.4 code paths, the prefix thing simply
> > doesn't work. Etc.
> >
> > Also, note that you can modify the above strategies by instead
> > designing the message fields for extensibility. E.g., if you make the
> > instructions field an enum, then you can make sure that the default
> > case is handled sensibly (probably similarly to STRATEGY1, just choke
> > on unknown instructions) and that you never remove an instruction type
> > from the enum in future versions.
> >
> > Does this make sense?
> > -John
> >
> >
> >
> >
> > PS:
> > We can define null keys for streaming tables, but it's tricky.
> >
> > Specifically, you'd want to define some concept of null keys that
> > allows all null keys to be unique, but _also_ to have a fixed
> > identity, so that a particular null-key can be updated later. One
> > example could be to union the existing keyspace with a new
> > null-keyspace, where normal keys are like "key" and null-keys are like
> > "null(identity)". Then given a query like
> > "KTable<String,Integer>.rightJoin(KTable<Integer,Boolean>)", and
> > inputs like:
> > LHS:
> > "a": 1
> > "b": 2
> >
> > RHS:
> > 1: true
> > 3: false
> >
> > a full outer join would produce:
> > "a": (1, true)
> > "b": (2, null)
> > null(3): (null, false)
> >
> > which can be correctly updated later if we get an update on the LHS:
> > PUT("c": 3)
> >
> > We'd emit for the results:
> > DELETE(null(e))
> > EMIT("c": (3, false))
> >
> > Resulting in the correct result table of:
> > "a": (1, true)
> > "b": (2, null)
> > "c": (3, false)
> >
> > As mentioned, this is tricky, and I would avoid it until we have
> > evidence that it's actually useful to cover this part of the design
> > space. Certainly, it would be a separate KIP if it came to that.
> >
> > On Wed, Jun 26, 2019 at 8:57 PM Adam Bellemare <ad...@gmail.com>
> > wrote:
> > >
> > > 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 the
> > last
> > > commit (about 10 min ago) along with INNER join. I do not think that
> > RIGHT
> > > join and OUTER are possible given that there is no LHS key available, so
> > > LHSTable.outerJoinOnForeignKey(RHSTable) wouldn't even make sense. This
> > is
> > > in contrast to the current LHSTable.outerJoin(RHSTable), as they are both
> > > keyed on the same key. I have buffed up the Integration tests and have
> > > tried to make them more readable to ensure that we're covering all the
> > > scenarios. I think that if we can get more eyes on the workflow showing
> > the
> > > various LHS and RHS events and outputs then that may help us validate
> > that
> > > we have all the scenarios covered.
> > >
> > > With regards to the 2.3->2.4 scenario you described, I'm not entirely
> > sure
> > > I follow. If they want to add a FK-join, they will need to rework their
> > > code in the KStreams app and make a new release, since the underlying
> > > topology would be different and new internal topics would need to be
> > > created. In other words, I don't think a rolling upgrade where the user
> > > introduces a FK join would be possible since their topology would
> > > necessitate a full KStreams reset. Is this what you meant?
> > >
> > >
> > >
> > > On Wed, Jun 26, 2019 at 4:10 PM John Roesler <jo...@confluent.io> wrote:
> > >
> > > > 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.bellemare@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.bellemare@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 <jo...@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
> > > > > >> <ad...@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 <
> > john@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
> > > > > >> > >
> > > > > >>
> > > > > >
> > > >
> >

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

Posted by Adam Bellemare <ad...@gmail.com>.
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 microservices - there are
indeed cases where a breaking change must be made, and forward
compatibility will provide us with no out other than requiring a full stop
and full upgrade for all nodes, shifting us back towards STRATEGY 1.

My preference is STRATEGY 1 with instructions as an ENUM, and we can
certainly include a version. Would it make sense to include a version
number in  SubscriptionResponseWrapper as well? Currently we don't have any
instructions in there, as I removed the boolean, but it is certainly
plausible that it could happen in the future. I don't *think* we'll need
it, but I also didn't think we'd need it for SubscriptionWrapper and here
we are.

Thanks for the thoughts, and the info on the right-key. That was
enlightening, though I can't think of a use-case for it *at this point in
time*. :)

Adam



On Thu, Jun 27, 2019 at 12:29 PM John Roesler <jo...@confluent.io> wrote:

> 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 `nullKey: (null, rhsValue)`, but we
> don't currently allow null keys in Streams. It actually is possible to
> define them, and therefore to add right- and full-outer foreign-key
> joins later, but it's non-trivial in a streaming context with
> continuously updated results. (See the PS if you're curious what I'm
> thinking). You're correct, right- and full-outer joins are trivial on
> our current 1:1 table joins because they are equi-joins.
>
> Regarding the transition, it sounds like what you're proposing is that
> we would say, "adding a foreign-key join to your topology requires a
> full application reset (or a new application id)". This is also an
> acceptable constraint to place on the feature, but not strictly
> necessary. Since 2.3, it's now possible to give all the state in your
> application stable names. This means that it's no longer true that
> adding a node to your topology graph would break its structure, and it
> does become possible to add new operators and simply restart the app.
> Revisiting my prior thought, though, I think the problem is not
> specific to your feature. For example, adding a new grouped
> aggregation would produce a new repartition topic, but the repartition
> topic partitions might get assigned to old nodes in the middle of a
> rolling bounce, and they would need to just ignore them. This
> requirement is the same for the repartition topics in the FK join, so
> it's orthogonal to your design.
>
> Back to the first concern, though, I'm not sure I followed the
> explanation. As a thought experiment, let's imagine that Joe hadn't
> taken the time to experiment with your feature branch. We wouldn't
> have noticed the problem until the feature was already released in
> 2.4. So the wire protocol on that PK->FK subscription topic would have
> been V1: "FK,PK,HASH,BOOLEAN". Then, Joe would have let us know the
> problem once they picked up the feature, so we would want to implement
> your proposed fix and change the wire protocol to V2:
> "FK,PK,HASH,INSTRUCTIONS" in 2.5. Upon rolling out the update, we
> would see both 2.4 nodes encountering V2 messages and 2.5 nodes
> encountering V1 messages. How can they both detect that they are
> attempting to process a newer or older protocol? If they can detect
> it, then what should they do?
>
> From experience, there are two basic solutions to this problem:
>
> STRATEGY1. Add a protocol version to the message (could be a number at
> the start of the message payload, or it could be a number in the
> message headers, not sure if it matters much. Payload is probably more
> compact, since the header would need a name.) In this case, the 2.4
> worker would know that it's max protocol version is V1, and when it
> sees the V2 message, it knows that it can't handle it properly. Rather
> than doing something wrong, it would just not do anything. This means
> it would stop the task, if not shut down the whole instance. On the
> other hand, a 2.5 worker would have some defined logic for how to
> handle all versions (V1 and V2), so once the upgrade is complete, all
> messages can be processed.
>
> STRATEGY2. Make the schema forward-compatible. Basically, we ensure
> that new fields can only be appended to the message schema, and that
> older workers using only a prefix of the full message would still
> behave correctly. Using the example above, we'd instead evolve the
> schema to V2': "FK,PK,HASH,BOOLEAN,INSTRUCTIONS", and continue to set
> the boolean field to true for the "new" foreign key. Then, 2.4 workers
> encountering the a "new FK" message would just see the prefix of the
> payload that makes sense to them, and they would still continue
> processing the messages as they always have. Only after the 2.5 code
> is fully rolled out to the cluster would we be sure to see the desired
> behavior. Note: in the reverse case, a 2.5 worker knows how to fully
> parse the new message format, even if it plans to ignore the BOOLEAN
> field.
>
> There are some tradeoffs between these strategies: STRATEGY1 ensures
> that all messages are only handled by workers that can properly handle
> them, although it results in processing stalls while there are still
> old nodes in the cluster. STRATEGY2 ensures that all messages can be
> processed by all nodes, so there are no stalls, but we can never
> remove fields from the message, so if there are a lot of revisions in
> the future, the payloads will become bloated. Also, it's not clear
> that you can actually pull off STRATEGY2 in all cases. If there's some
> new kind of message you want to send that has no way to be correctly
> processed at all under the 2.4 code paths, the prefix thing simply
> doesn't work. Etc.
>
> Also, note that you can modify the above strategies by instead
> designing the message fields for extensibility. E.g., if you make the
> instructions field an enum, then you can make sure that the default
> case is handled sensibly (probably similarly to STRATEGY1, just choke
> on unknown instructions) and that you never remove an instruction type
> from the enum in future versions.
>
> Does this make sense?
> -John
>
>
>
>
> PS:
> We can define null keys for streaming tables, but it's tricky.
>
> Specifically, you'd want to define some concept of null keys that
> allows all null keys to be unique, but _also_ to have a fixed
> identity, so that a particular null-key can be updated later. One
> example could be to union the existing keyspace with a new
> null-keyspace, where normal keys are like "key" and null-keys are like
> "null(identity)". Then given a query like
> "KTable<String,Integer>.rightJoin(KTable<Integer,Boolean>)", and
> inputs like:
> LHS:
> "a": 1
> "b": 2
>
> RHS:
> 1: true
> 3: false
>
> a full outer join would produce:
> "a": (1, true)
> "b": (2, null)
> null(3): (null, false)
>
> which can be correctly updated later if we get an update on the LHS:
> PUT("c": 3)
>
> We'd emit for the results:
> DELETE(null(e))
> EMIT("c": (3, false))
>
> Resulting in the correct result table of:
> "a": (1, true)
> "b": (2, null)
> "c": (3, false)
>
> As mentioned, this is tricky, and I would avoid it until we have
> evidence that it's actually useful to cover this part of the design
> space. Certainly, it would be a separate KIP if it came to that.
>
> On Wed, Jun 26, 2019 at 8:57 PM Adam Bellemare <ad...@gmail.com>
> wrote:
> >
> > 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 the
> last
> > commit (about 10 min ago) along with INNER join. I do not think that
> RIGHT
> > join and OUTER are possible given that there is no LHS key available, so
> > LHSTable.outerJoinOnForeignKey(RHSTable) wouldn't even make sense. This
> is
> > in contrast to the current LHSTable.outerJoin(RHSTable), as they are both
> > keyed on the same key. I have buffed up the Integration tests and have
> > tried to make them more readable to ensure that we're covering all the
> > scenarios. I think that if we can get more eyes on the workflow showing
> the
> > various LHS and RHS events and outputs then that may help us validate
> that
> > we have all the scenarios covered.
> >
> > With regards to the 2.3->2.4 scenario you described, I'm not entirely
> sure
> > I follow. If they want to add a FK-join, they will need to rework their
> > code in the KStreams app and make a new release, since the underlying
> > topology would be different and new internal topics would need to be
> > created. In other words, I don't think a rolling upgrade where the user
> > introduces a FK join would be possible since their topology would
> > necessitate a full KStreams reset. Is this what you meant?
> >
> >
> >
> > On Wed, Jun 26, 2019 at 4:10 PM John Roesler <jo...@confluent.io> wrote:
> >
> > > 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.bellemare@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.bellemare@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 <jo...@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
> > > > >> <ad...@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 <
> john@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
> > > > >> > >
> > > > >>
> > > > >
> > >
>

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

Posted by John Roesler <jo...@confluent.io>.
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 `nullKey: (null, rhsValue)`, but we
don't currently allow null keys in Streams. It actually is possible to
define them, and therefore to add right- and full-outer foreign-key
joins later, but it's non-trivial in a streaming context with
continuously updated results. (See the PS if you're curious what I'm
thinking). You're correct, right- and full-outer joins are trivial on
our current 1:1 table joins because they are equi-joins.

Regarding the transition, it sounds like what you're proposing is that
we would say, "adding a foreign-key join to your topology requires a
full application reset (or a new application id)". This is also an
acceptable constraint to place on the feature, but not strictly
necessary. Since 2.3, it's now possible to give all the state in your
application stable names. This means that it's no longer true that
adding a node to your topology graph would break its structure, and it
does become possible to add new operators and simply restart the app.
Revisiting my prior thought, though, I think the problem is not
specific to your feature. For example, adding a new grouped
aggregation would produce a new repartition topic, but the repartition
topic partitions might get assigned to old nodes in the middle of a
rolling bounce, and they would need to just ignore them. This
requirement is the same for the repartition topics in the FK join, so
it's orthogonal to your design.

Back to the first concern, though, I'm not sure I followed the
explanation. As a thought experiment, let's imagine that Joe hadn't
taken the time to experiment with your feature branch. We wouldn't
have noticed the problem until the feature was already released in
2.4. So the wire protocol on that PK->FK subscription topic would have
been V1: "FK,PK,HASH,BOOLEAN". Then, Joe would have let us know the
problem once they picked up the feature, so we would want to implement
your proposed fix and change the wire protocol to V2:
"FK,PK,HASH,INSTRUCTIONS" in 2.5. Upon rolling out the update, we
would see both 2.4 nodes encountering V2 messages and 2.5 nodes
encountering V1 messages. How can they both detect that they are
attempting to process a newer or older protocol? If they can detect
it, then what should they do?

From experience, there are two basic solutions to this problem:

STRATEGY1. Add a protocol version to the message (could be a number at
the start of the message payload, or it could be a number in the
message headers, not sure if it matters much. Payload is probably more
compact, since the header would need a name.) In this case, the 2.4
worker would know that it's max protocol version is V1, and when it
sees the V2 message, it knows that it can't handle it properly. Rather
than doing something wrong, it would just not do anything. This means
it would stop the task, if not shut down the whole instance. On the
other hand, a 2.5 worker would have some defined logic for how to
handle all versions (V1 and V2), so once the upgrade is complete, all
messages can be processed.

STRATEGY2. Make the schema forward-compatible. Basically, we ensure
that new fields can only be appended to the message schema, and that
older workers using only a prefix of the full message would still
behave correctly. Using the example above, we'd instead evolve the
schema to V2': "FK,PK,HASH,BOOLEAN,INSTRUCTIONS", and continue to set
the boolean field to true for the "new" foreign key. Then, 2.4 workers
encountering the a "new FK" message would just see the prefix of the
payload that makes sense to them, and they would still continue
processing the messages as they always have. Only after the 2.5 code
is fully rolled out to the cluster would we be sure to see the desired
behavior. Note: in the reverse case, a 2.5 worker knows how to fully
parse the new message format, even if it plans to ignore the BOOLEAN
field.

There are some tradeoffs between these strategies: STRATEGY1 ensures
that all messages are only handled by workers that can properly handle
them, although it results in processing stalls while there are still
old nodes in the cluster. STRATEGY2 ensures that all messages can be
processed by all nodes, so there are no stalls, but we can never
remove fields from the message, so if there are a lot of revisions in
the future, the payloads will become bloated. Also, it's not clear
that you can actually pull off STRATEGY2 in all cases. If there's some
new kind of message you want to send that has no way to be correctly
processed at all under the 2.4 code paths, the prefix thing simply
doesn't work. Etc.

Also, note that you can modify the above strategies by instead
designing the message fields for extensibility. E.g., if you make the
instructions field an enum, then you can make sure that the default
case is handled sensibly (probably similarly to STRATEGY1, just choke
on unknown instructions) and that you never remove an instruction type
from the enum in future versions.

Does this make sense?
-John




PS:
We can define null keys for streaming tables, but it's tricky.

Specifically, you'd want to define some concept of null keys that
allows all null keys to be unique, but _also_ to have a fixed
identity, so that a particular null-key can be updated later. One
example could be to union the existing keyspace with a new
null-keyspace, where normal keys are like "key" and null-keys are like
"null(identity)". Then given a query like
"KTable<String,Integer>.rightJoin(KTable<Integer,Boolean>)", and
inputs like:
LHS:
"a": 1
"b": 2

RHS:
1: true
3: false

a full outer join would produce:
"a": (1, true)
"b": (2, null)
null(3): (null, false)

which can be correctly updated later if we get an update on the LHS:
PUT("c": 3)

We'd emit for the results:
DELETE(null(e))
EMIT("c": (3, false))

Resulting in the correct result table of:
"a": (1, true)
"b": (2, null)
"c": (3, false)

As mentioned, this is tricky, and I would avoid it until we have
evidence that it's actually useful to cover this part of the design
space. Certainly, it would be a separate KIP if it came to that.

On Wed, Jun 26, 2019 at 8:57 PM Adam Bellemare <ad...@gmail.com> wrote:
>
> 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 the last
> commit (about 10 min ago) along with INNER join. I do not think that RIGHT
> join and OUTER are possible given that there is no LHS key available, so
> LHSTable.outerJoinOnForeignKey(RHSTable) wouldn't even make sense. This is
> in contrast to the current LHSTable.outerJoin(RHSTable), as they are both
> keyed on the same key. I have buffed up the Integration tests and have
> tried to make them more readable to ensure that we're covering all the
> scenarios. I think that if we can get more eyes on the workflow showing the
> various LHS and RHS events and outputs then that may help us validate that
> we have all the scenarios covered.
>
> With regards to the 2.3->2.4 scenario you described, I'm not entirely sure
> I follow. If they want to add a FK-join, they will need to rework their
> code in the KStreams app and make a new release, since the underlying
> topology would be different and new internal topics would need to be
> created. In other words, I don't think a rolling upgrade where the user
> introduces a FK join would be possible since their topology would
> necessitate a full KStreams reset. Is this what you meant?
>
>
>
> On Wed, Jun 26, 2019 at 4:10 PM John Roesler <jo...@confluent.io> wrote:
>
> > 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 <ad...@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.bellemare@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 <jo...@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
> > > >> <ad...@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 <jo...@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
> > > >> > >
> > > >>
> > > >
> >

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

Posted by Adam Bellemare <ad...@gmail.com>.
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 the last
commit (about 10 min ago) along with INNER join. I do not think that RIGHT
join and OUTER are possible given that there is no LHS key available, so
LHSTable.outerJoinOnForeignKey(RHSTable) wouldn't even make sense. This is
in contrast to the current LHSTable.outerJoin(RHSTable), as they are both
keyed on the same key. I have buffed up the Integration tests and have
tried to make them more readable to ensure that we're covering all the
scenarios. I think that if we can get more eyes on the workflow showing the
various LHS and RHS events and outputs then that may help us validate that
we have all the scenarios covered.

With regards to the 2.3->2.4 scenario you described, I'm not entirely sure
I follow. If they want to add a FK-join, they will need to rework their
code in the KStreams app and make a new release, since the underlying
topology would be different and new internal topics would need to be
created. In other words, I don't think a rolling upgrade where the user
introduces a FK join would be possible since their topology would
necessitate a full KStreams reset. Is this what you meant?



On Wed, Jun 26, 2019 at 4:10 PM John Roesler <jo...@confluent.io> wrote:

> 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 <ad...@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.bellemare@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 <jo...@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
> > >> <ad...@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 <jo...@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
> > >> > >
> > >>
> > >
>

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

Posted by John Roesler <jo...@confluent.io>.
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 <ad...@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 <ad...@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 <jo...@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
> >> <ad...@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 <jo...@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
> >> > >
> >>
> >

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

Posted by Adam Bellemare <ad...@gmail.com>.
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 <ad...@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 <jo...@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
>> <ad...@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 <jo...@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
>> > >
>>
>

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

Posted by Adam Bellemare <ad...@gmail.com>.
>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 <jo...@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
> <ad...@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 <jo...@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
> > >
>

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

Posted by John Roesler <jo...@confluent.io>.
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
<ad...@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 <jo...@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
> >

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

Posted by Adam Bellemare <ad...@gmail.com>.
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 <jo...@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
>