[jira] [Commented] (KAFKA-7577) Semantics of Table-Table Join with Null Message Are Incorrect

2018-11-20 Thread Matthias J. Sax (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-7577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16693778#comment-16693778
 ] 

Matthias J. Sax commented on KAFKA-7577:


Glad to hear. We have a ticket  KAFKA-7658 to add KStream#toTable already.

> Semantics of Table-Table Join with Null Message Are Incorrect
> -
>
> Key: KAFKA-7577
> URL: https://issues.apache.org/jira/browse/KAFKA-7577
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 1.1.0
>Reporter: Daren Thomas
>Priority: Major
>
> Observed behavior of Table-Table join with a Null Message does not match the 
> semantics described in the documentation 
> ([https://docs.confluent.io/current/streams/developer-guide/dsl-api.html#ktable-ktable-join).]
>   The expectation is:
>  * Message A results in [A, null] from the Left Join
>  * Message null (tombstone) results in null (tombstone) from the Left Join
>  The observed behavior was that the null (tombstone) message did not pass 
> through the Left Join to the output topic like expected.  This behavior was 
> observed with and without caching enabled, against a test harness, and 
> against a local Confluent 5.0.0 platform.  It was also observed that the 
> KTableKTableLeftJoinProcessor.process() function was not called for the null 
> (tombstone) message.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-7577) Semantics of Table-Table Join with Null Message Are Incorrect

2018-11-20 Thread Daren Thomas (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-7577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16693770#comment-16693770
 ] 

Daren Thomas commented on KAFKA-7577:
-

Yes, thank you, that does make sense.  I verified (in a portion of my code) 
that the following works as expected:
{code:java}
stream.map(/select new key/).to("my-new-changelog-topic"); KTable table = 
builder.table("my-new-changelog-topic"){code}
I'll change the rest of the code because this is much simpler than I was doing 
before and it makes more sense now that I understand the nuance.  My incoming 
data is a series of table updates so it should be treated as such.

One final note, adding a `KStream#toTable()` would be very helpful.  In my 
situation, I'm reading topics as streams so I can rekey/repartition them, but 
the end goal is to convert them back into tables before performing joins on the 
tables.  That started me down the road of using a dummy aggregation and led to 
this issue.

 

> Semantics of Table-Table Join with Null Message Are Incorrect
> -
>
> Key: KAFKA-7577
> URL: https://issues.apache.org/jira/browse/KAFKA-7577
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 1.1.0
>Reporter: Daren Thomas
>Priority: Major
>
> Observed behavior of Table-Table join with a Null Message does not match the 
> semantics described in the documentation 
> ([https://docs.confluent.io/current/streams/developer-guide/dsl-api.html#ktable-ktable-join).]
>   The expectation is:
>  * Message A results in [A, null] from the Left Join
>  * Message null (tombstone) results in null (tombstone) from the Left Join
>  The observed behavior was that the null (tombstone) message did not pass 
> through the Left Join to the output topic like expected.  This behavior was 
> observed with and without caching enabled, against a test harness, and 
> against a local Confluent 5.0.0 platform.  It was also observed that the 
> KTableKTableLeftJoinProcessor.process() function was not called for the null 
> (tombstone) message.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-7577) Semantics of Table-Table Join with Null Message Are Incorrect

2018-11-19 Thread Matthias J. Sax (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-7577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16692535#comment-16692535
 ] 

Matthias J. Sax commented on KAFKA-7577:


An `aggregation()` is designed like in SQL, that also drops `null`s.

Note, that while an `aggregation` *returns* a KTable, the *input* is a KStream 
that represents fact, but not updates... That is the main difference: if you 
read a topic directly into a `KTable`, you tell KafkaStreams to use update 
semantics including tombstone semantics. However, when you aggregate a 
`KStream` there is no update/tombstone semantics for the *input* records.
{quote}how are deletions suppose to propagate 
{quote}
There is no reason to propagate them, as this concept does not exist in a 
KStream. Also note, that `KStream#groupByKey()` returns a `KGrouped*Stream*` 
but not a `KGroupedTable` (that would be result of `KTable#groupBy()`). It 
might seem subtle, but from my point of view an `aggregate()` is not the same 
as an UPSERT (what you want).

If you want to treat the data as table updates, you should read the topic as a 
table directly (from a semantic point of view) and create a derived table with 
`KTable#groupBy()` to repartition accordingly. If you want to avoid the 
materialization overhead for the first table, you can also do 
`stream.map(/*select new key*/).to("my-new-changelog-topic"); KTable table = 
builder.table("my-new-changelog-topic")` instead.

We are working on an optimization atm, that will allow to read a topic as 
`KTable` directly, while the KTable will only be materialized if necessary. 
This should address your use case, as we would only materialize the second 
KTable. For now, you can use your own workaround (even if I think you miss use 
the API, as I believe an aggregation is not the right operator to implement an 
UPSERT—my personal opinion only), or with the intermediate topic as suggested. 
I also want to point out, that having an UPSERT operator might actually be 
useful. But instead of changing `aggregate()` I would rather add 
`KStream#toTable()` operator (similar to, or reverse of, `KTable#toStream()`) 
for this use case. Because it's semantically two different things, there should 
be two different operators for both IMHO.

Does this make sense?

> Semantics of Table-Table Join with Null Message Are Incorrect
> -
>
> Key: KAFKA-7577
> URL: https://issues.apache.org/jira/browse/KAFKA-7577
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 1.1.0
>Reporter: Daren Thomas
>Priority: Major
>
> Observed behavior of Table-Table join with a Null Message does not match the 
> semantics described in the documentation 
> ([https://docs.confluent.io/current/streams/developer-guide/dsl-api.html#ktable-ktable-join).]
>   The expectation is:
>  * Message A results in [A, null] from the Left Join
>  * Message null (tombstone) results in null (tombstone) from the Left Join
>  The observed behavior was that the null (tombstone) message did not pass 
> through the Left Join to the output topic like expected.  This behavior was 
> observed with and without caching enabled, against a test harness, and 
> against a local Confluent 5.0.0 platform.  It was also observed that the 
> KTableKTableLeftJoinProcessor.process() function was not called for the null 
> (tombstone) message.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-7577) Semantics of Table-Table Join with Null Message Are Incorrect

2018-11-19 Thread Daren Thomas (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-7577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16692143#comment-16692143
 ] 

Daren Thomas commented on KAFKA-7577:
-

This is the detailed behavior of KGroupedTable:
 * When a tombstone record – i.e. a record with a {{null}} value – is received 
for a key (e.g., DELETE), then only the subtractor is called. Note that, 
whenever the subtractor returns a {{null}} value itself, then the corresponding 
key is removed from the resulting {{KTable}}. If that happens, any next input 
record for that key will trigger the initializer again.

That is the behavior I hoped to see.  It seems that my aggregation is following 
the semantics of KGroupedStream instead.

> Semantics of Table-Table Join with Null Message Are Incorrect
> -
>
> Key: KAFKA-7577
> URL: https://issues.apache.org/jira/browse/KAFKA-7577
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 1.1.0
>Reporter: Daren Thomas
>Priority: Major
>
> Observed behavior of Table-Table join with a Null Message does not match the 
> semantics described in the documentation 
> ([https://docs.confluent.io/current/streams/developer-guide/dsl-api.html#ktable-ktable-join).]
>   The expectation is:
>  * Message A results in [A, null] from the Left Join
>  * Message null (tombstone) results in null (tombstone) from the Left Join
>  The observed behavior was that the null (tombstone) message did not pass 
> through the Left Join to the output topic like expected.  This behavior was 
> observed with and without caching enabled, against a test harness, and 
> against a local Confluent 5.0.0 platform.  It was also observed that the 
> KTableKTableLeftJoinProcessor.process() function was not called for the null 
> (tombstone) message.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-7577) Semantics of Table-Table Join with Null Message Are Incorrect

2018-11-19 Thread Daren Thomas (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-7577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16692136#comment-16692136
 ] 

Daren Thomas commented on KAFKA-7577:
-

What is the reasoning behind that being the design?  

I'm using Kafka Connect to stream database updates to a Kafka topic.  I then 
rekey and repartition the streams so I can perform various table joins.  If 
aggregate() drops any `null`-value input records, how are deletions suppose to 
propagate into the system and remove entries from the joins?  Right now I'm 
catching those deletions and creating my own messages that are specifically 
constructed to generate the desired output from the joins.  

> Semantics of Table-Table Join with Null Message Are Incorrect
> -
>
> Key: KAFKA-7577
> URL: https://issues.apache.org/jira/browse/KAFKA-7577
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 1.1.0
>Reporter: Daren Thomas
>Priority: Major
>
> Observed behavior of Table-Table join with a Null Message does not match the 
> semantics described in the documentation 
> ([https://docs.confluent.io/current/streams/developer-guide/dsl-api.html#ktable-ktable-join).]
>   The expectation is:
>  * Message A results in [A, null] from the Left Join
>  * Message null (tombstone) results in null (tombstone) from the Left Join
>  The observed behavior was that the null (tombstone) message did not pass 
> through the Left Join to the output topic like expected.  This behavior was 
> observed with and without caching enabled, against a test harness, and 
> against a local Confluent 5.0.0 platform.  It was also observed that the 
> KTableKTableLeftJoinProcessor.process() function was not called for the null 
> (tombstone) message.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-7577) Semantics of Table-Table Join with Null Message Are Incorrect

2018-11-19 Thread Matthias J. Sax (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-7577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16692066#comment-16692066
 ] 

Matthias J. Sax commented on KAFKA-7577:


`aggregate()` will drop any `null`-value input record as pointed out in the 
JavaDocs. Can this be issue you are facing? If yes, it's not a bug, but by 
design.

> Semantics of Table-Table Join with Null Message Are Incorrect
> -
>
> Key: KAFKA-7577
> URL: https://issues.apache.org/jira/browse/KAFKA-7577
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 1.1.0
>Reporter: Daren Thomas
>Priority: Major
>
> Observed behavior of Table-Table join with a Null Message does not match the 
> semantics described in the documentation 
> ([https://docs.confluent.io/current/streams/developer-guide/dsl-api.html#ktable-ktable-join).]
>   The expectation is:
>  * Message A results in [A, null] from the Left Join
>  * Message null (tombstone) results in null (tombstone) from the Left Join
>  The observed behavior was that the null (tombstone) message did not pass 
> through the Left Join to the output topic like expected.  This behavior was 
> observed with and without caching enabled, against a test harness, and 
> against a local Confluent 5.0.0 platform.  It was also observed that the 
> KTableKTableLeftJoinProcessor.process() function was not called for the null 
> (tombstone) message.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-7577) Semantics of Table-Table Join with Null Message Are Incorrect

2018-11-19 Thread Daren Thomas (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-7577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16691821#comment-16691821
 ] 

Daren Thomas commented on KAFKA-7577:
-

[~guozhang] I have unit tests for the issue, but not ones I could easily break 
out in order to provide them.  Unfortunately due to schedule pressure I've had 
to just work around the issue.

My best guess for the root cause of the issue resides with the aggregate I'm 
performing on a stream to create the tables used in the join.  I don't think 
it's passing on the null message.  Since I'm just trying to pass messages along 
after changing the key (not adding or subtracting values as shown in the DSL 
documentation), I have the following:
{code:java}
.aggregate(() -> myMessageValue.newBuilder().build(), (aggKey, newValue, 
aggValue) -> newValue, ...{code}
Perhaps there is something wrong with that and not the join?

> Semantics of Table-Table Join with Null Message Are Incorrect
> -
>
> Key: KAFKA-7577
> URL: https://issues.apache.org/jira/browse/KAFKA-7577
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 1.1.0
>Reporter: Daren Thomas
>Priority: Major
>
> Observed behavior of Table-Table join with a Null Message does not match the 
> semantics described in the documentation 
> ([https://docs.confluent.io/current/streams/developer-guide/dsl-api.html#ktable-ktable-join).]
>   The expectation is:
>  * Message A results in [A, null] from the Left Join
>  * Message null (tombstone) results in null (tombstone) from the Left Join
>  The observed behavior was that the null (tombstone) message did not pass 
> through the Left Join to the output topic like expected.  This behavior was 
> observed with and without caching enabled, against a test harness, and 
> against a local Confluent 5.0.0 platform.  It was also observed that the 
> KTableKTableLeftJoinProcessor.process() function was not called for the null 
> (tombstone) message.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-7577) Semantics of Table-Table Join with Null Message Are Incorrect

2018-11-16 Thread Guozhang Wang (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-7577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16690175#comment-16690175
 ] 

Guozhang Wang commented on KAFKA-7577:
--

Hi [~dthomas-trimble], any luck trying to reproduce the issue as a unit test 
with TopologyTestDriver so that we can further investigate?

> Semantics of Table-Table Join with Null Message Are Incorrect
> -
>
> Key: KAFKA-7577
> URL: https://issues.apache.org/jira/browse/KAFKA-7577
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 1.1.0
>Reporter: Daren Thomas
>Priority: Major
>
> Observed behavior of Table-Table join with a Null Message does not match the 
> semantics described in the documentation 
> ([https://docs.confluent.io/current/streams/developer-guide/dsl-api.html#ktable-ktable-join).]
>   The expectation is:
>  * Message A results in [A, null] from the Left Join
>  * Message null (tombstone) results in null (tombstone) from the Left Join
>  The observed behavior was that the null (tombstone) message did not pass 
> through the Left Join to the output topic like expected.  This behavior was 
> observed with and without caching enabled, against a test harness, and 
> against a local Confluent 5.0.0 platform.  It was also observed that the 
> KTableKTableLeftJoinProcessor.process() function was not called for the null 
> (tombstone) message.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-7577) Semantics of Table-Table Join with Null Message Are Incorrect

2018-11-02 Thread Matthias J. Sax (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-7577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16673444#comment-16673444
 ] 

Matthias J. Sax commented on KAFKA-7577:


TopologyTestDriver docs: 
[https://kafka.apache.org/20/documentation/streams/developer-guide/testing.html]

> Semantics of Table-Table Join with Null Message Are Incorrect
> -
>
> Key: KAFKA-7577
> URL: https://issues.apache.org/jira/browse/KAFKA-7577
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 1.1.0
>Reporter: Daren Thomas
>Priority: Major
>
> Observed behavior of Table-Table join with a Null Message does not match the 
> semantics described in the documentation 
> ([https://docs.confluent.io/current/streams/developer-guide/dsl-api.html#ktable-ktable-join).]
>   The expectation is:
>  * Message A results in [A, null] from the Left Join
>  * Message null (tombstone) results in null (tombstone) from the Left Join
>  The observed behavior was that the null (tombstone) message did not pass 
> through the Left Join to the output topic like expected.  This behavior was 
> observed with and without caching enabled, against a test harness, and 
> against a local Confluent 5.0.0 platform.  It was also observed that the 
> KTableKTableLeftJoinProcessor.process() function was not called for the null 
> (tombstone) message.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-7577) Semantics of Table-Table Join with Null Message Are Incorrect

2018-11-02 Thread Daren Thomas (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-7577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16673137#comment-16673137
 ] 

Daren Thomas commented on KAFKA-7577:
-

{quote}Can you verify that Y.leftJoin(Z) produces an output record that 
contains B ?
{quote}
I can try.  It is correct that B comes from input Z.  Table Y does not 
contribute anything to the final output.  Its only purpose is to link Table X 
with Table Z.  So the output of Y.leftJoin(Z) is a key capable of joining with 
X and the relevant information from Z.  Does that make sense?
{quote}Can you reproduce the issue using `TopologyTestDriver`?
{quote}
My apologies, but I'm not familiar with the TopologyTestDriver.  If I can 
figure out how to use it, I will certainly pass that information along.

> Semantics of Table-Table Join with Null Message Are Incorrect
> -
>
> Key: KAFKA-7577
> URL: https://issues.apache.org/jira/browse/KAFKA-7577
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 1.1.0
>Reporter: Daren Thomas
>Priority: Major
>
> Observed behavior of Table-Table join with a Null Message does not match the 
> semantics described in the documentation 
> ([https://docs.confluent.io/current/streams/developer-guide/dsl-api.html#ktable-ktable-join).]
>   The expectation is:
>  * Message A results in [A, null] from the Left Join
>  * Message null (tombstone) results in null (tombstone) from the Left Join
>  The observed behavior was that the null (tombstone) message did not pass 
> through the Left Join to the output topic like expected.  This behavior was 
> observed with and without caching enabled, against a test harness, and 
> against a local Confluent 5.0.0 platform.  It was also observed that the 
> KTableKTableLeftJoinProcessor.process() function was not called for the null 
> (tombstone) message.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-7577) Semantics of Table-Table Join with Null Message Are Incorrect

2018-11-01 Thread Matthias J. Sax (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-7577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16672341#comment-16672341
 ] 

Matthias J. Sax commented on KAFKA-7577:


Thanks for clarification. Still try to understand the scenario. For regular 
left join, we have test and those pass and thus leftjoin itself seems to be 
correct 
([https://github.com/apache/kafka/blob/trunk/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java).]
 Just trying to figure out what the issue could be. There are two other bug 
reports about KTable joins, but they seem to be different 
(https://issues.apache.org/jira/browse/KAFKA-4609 and 
https://issues.apache.org/jira/browse/KAFKA-6599).

What I am wondering: if you do Y.leftJoin(Z), it is only guaranteed that the Y 
input will be in the output, while your example shows the B comes from input Z. 
Can you verify that Y.leftJoin(Z) produces an output record that contains B ?

Can you reproduce the issue using `TopologyTestDriver`? This would be most 
helpful to track down the root cause. Thanks a lot for your help!

> Semantics of Table-Table Join with Null Message Are Incorrect
> -
>
> Key: KAFKA-7577
> URL: https://issues.apache.org/jira/browse/KAFKA-7577
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 1.1.0
>Reporter: Daren Thomas
>Priority: Major
>
> Observed behavior of Table-Table join with a Null Message does not match the 
> semantics described in the documentation 
> ([https://docs.confluent.io/current/streams/developer-guide/dsl-api.html#ktable-ktable-join).]
>   The expectation is:
>  * Message A results in [A, null] from the Left Join
>  * Message null (tombstone) results in null (tombstone) from the Left Join
>  The observed behavior was that the null (tombstone) message did not pass 
> through the Left Join to the output topic like expected.  This behavior was 
> observed with and without caching enabled, against a test harness, and 
> against a local Confluent 5.0.0 platform.  It was also observed that the 
> KTableKTableLeftJoinProcessor.process() function was not called for the null 
> (tombstone) message.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-7577) Semantics of Table-Table Join with Null Message Are Incorrect

2018-11-01 Thread Daren Thomas (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-7577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16672245#comment-16672245
 ] 

Daren Thomas commented on KAFKA-7577:
-

{quote}Note, that a tombstone message is only produced, if there is anything to 
be deleted.
{quote}
My test case is set up to perform line 3 and line 7 from the data.  The table 
is not empty when the tombstone is sent.

The database tables backing the topics used by the stream processor look like 
this:
 * Table X with xid (primary key) and data A or null
 * Table Y with yid (primary key), xid, and zid
 * Table Z with zid (primary key) and data B or null

The topology looks like this:
 * Repartition Y using groupBy() and aggregate()
 * Repartition Z using groupBy() and aggregate()
 * Perform Y.leftJoin(Z).  Repartition using groupBy and aggregate()
 * Repartition X using flatMap()
 * Perform X.leftJoin(YZ), convert results to a stream, and send to the output 
topic.  Intended result is [A, B] for the value.

> Semantics of Table-Table Join with Null Message Are Incorrect
> -
>
> Key: KAFKA-7577
> URL: https://issues.apache.org/jira/browse/KAFKA-7577
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 1.1.0
>Reporter: Daren Thomas
>Priority: Major
>
> Observed behavior of Table-Table join with a Null Message does not match the 
> semantics described in the documentation 
> ([https://docs.confluent.io/current/streams/developer-guide/dsl-api.html#ktable-ktable-join).]
>   The expectation is:
>  * Message A results in [A, null] from the Left Join
>  * Message null (tombstone) results in null (tombstone) from the Left Join
>  The observed behavior was that the null (tombstone) message did not pass 
> through the Left Join to the output topic like expected.  This behavior was 
> observed with and without caching enabled, against a test harness, and 
> against a local Confluent 5.0.0 platform.  It was also observed that the 
> KTableKTableLeftJoinProcessor.process() function was not called for the null 
> (tombstone) message.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-7577) Semantics of Table-Table Join with Null Message Are Incorrect

2018-11-01 Thread Matthias J. Sax (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-7577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=1667#comment-1667
 ] 

Matthias J. Sax commented on KAFKA-7577:


Can you describe your test data your are using and your ValueJoiner 
implementation? It's unclear to me what your setup is so we can reproduce the 
error.

Note, that a tomstone message is only produced, if there is anything to be 
deleted. Compare the semantics as describe in the wiki: 
[https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+Join+Semantics#KafkaStreamsJoinSemantics-KTable-KTableJoin]
 Note, that there is no output for line 1,2, and 13, because the result table 
is empty in those cases and thus it's not required to send another tombstone 
(for line 13, pervious line 11/12, generated already a tombstone).

> Semantics of Table-Table Join with Null Message Are Incorrect
> -
>
> Key: KAFKA-7577
> URL: https://issues.apache.org/jira/browse/KAFKA-7577
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 1.1.0
>Reporter: Daren Thomas
>Priority: Major
>
> Observed behavior of Table-Table join with a Null Message does not match the 
> semantics described in the documentation 
> ([https://docs.confluent.io/current/streams/developer-guide/dsl-api.html#ktable-ktable-join).]
>   The expectation is:
>  * Message A results in [A, null] from the Left Join
>  * Message null (tombstone) results in null (tombstone) from the Left Join
>  The observed behavior was that the null (tombstone) message did not pass 
> through the Left Join to the output topic like expected.  This behavior was 
> observed with and without caching enabled, against a test harness, and 
> against a local Confluent 5.0.0 platform.  It was also observed that the 
> KTableKTableLeftJoinProcessor.process() function was not called for the null 
> (tombstone) message.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (KAFKA-7577) Semantics of Table-Table Join with Null Message Are Incorrect

2018-11-01 Thread Daren Thomas (JIRA)


[ 
https://issues.apache.org/jira/browse/KAFKA-7577?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16671844#comment-16671844
 ] 

Daren Thomas commented on KAFKA-7577:
-

I see similar behavior with the right side of the Left Join.  If the message 
value is null, the Left Join is not processed.

> Semantics of Table-Table Join with Null Message Are Incorrect
> -
>
> Key: KAFKA-7577
> URL: https://issues.apache.org/jira/browse/KAFKA-7577
> Project: Kafka
>  Issue Type: Bug
>Affects Versions: 1.1.0
>Reporter: Daren Thomas
>Priority: Major
>
> Observed behavior of Table-Table join with a Null Message does not match the 
> semantics described in the documentation 
> ([https://docs.confluent.io/current/streams/developer-guide/dsl-api.html#ktable-ktable-join).]
>   The expectation is:
>  * Message A results in [A, null] from the Left Join
>  * Message null (tombstone) results in null (tombstone) from the Left Join
>  The observed behavior was that the null (tombstone) message did not pass 
> through the Left Join to the output topic like expected.  This behavior was 
> observed with and without caching enabled, against a test harness, and 
> against a local Confluent 5.0.0 platform.  It was also observed that the 
> KTableKTableLeftJoinProcessor.process() function was not called for the null 
> (tombstone) message.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)