[jira] [Commented] (KAFKA-9517) KTable Joins Without Materialized Argument Yield Results That Further Joins NPE On

2020-02-13 Thread Paul Snively (Jira)


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

Paul Snively commented on KAFKA-9517:
-

John,

Thanks so much for your prompt attention, attention to detail, and pointers to 
the relevant issues and PRs. I'm glad to hear they've been merged and look 
forward to the 2.4.1 release when the last blocker is resolved. We'll continue 
pushing on our specific use-cases for the remainder of the week and let you 
know ASAP if anything continues to arise.

> KTable Joins Without Materialized Argument Yield Results That Further Joins 
> NPE On
> --
>
> Key: KAFKA-9517
> URL: https://issues.apache.org/jira/browse/KAFKA-9517
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.4.0
>Reporter: Paul Snively
>Assignee: John Roesler
>Priority: Blocker
> Fix For: 2.5.0, 2.4.1
>
> Attachments: test.tar.xz
>
>
> The `KTable` API implemented [[here||#L842-L844]] 
> [https://github.com/apache/kafka/blob/2.4.0/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L842-L844]
>  []|#L842-L844]] calls `doJoinOnForeignKey` with an argument of 
> `Materialized.with(null, null)`, as apparently do several other APIs. As the 
> comment spanning [these lines|#L1098-L1099]] makes clear, the result is a 
> `KTable` whose `valueSerde` (as a `KTableImpl`) is `null`. Therefore, 
> attempts to `join` etc. on the resulting `KTable` fail with a 
> `NullPointerException`.
> While there is an obvious workaround—explicitly construct the required 
> `Materialized` and use the APIs that take it as an argument—I have to admit I 
> find the existence of public APIs with this sort of bug, particularly when 
> the bug is literally documented as a comment in the source code, astonishing 
> to the point of incredulity. It calls the quality and trustworthiness of 
> Kafka Streams into serious question, and if a resolution is not forthcoming 
> within a week, we will be left with no other option but to consider technical 
> alternatives.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9517) KTable Joins Without Materialized Argument Yield Results That Further Joins NPE On

2020-02-12 Thread John Roesler (Jira)


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

John Roesler commented on KAFKA-9517:
-

Hi [~psnively],

Just another quick update: now all the reported bugs for foreign-key joins are 
fixed and merged to the branches: trunk, 2.5, and 2.4 . If you want to test the 
fixes, you can now just check out and build the 2.4 branch without any 
cherry-picking.

There is still a blocker for 2.4.1, so [~bbejeck] hasn't created an actual 
release candidate yet.

Thanks,
-John

> KTable Joins Without Materialized Argument Yield Results That Further Joins 
> NPE On
> --
>
> Key: KAFKA-9517
> URL: https://issues.apache.org/jira/browse/KAFKA-9517
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.4.0
>Reporter: Paul Snively
>Assignee: John Roesler
>Priority: Blocker
> Fix For: 2.5.0, 2.4.1
>
> Attachments: test.tar.xz
>
>
> The `KTable` API implemented [[here||#L842-L844]] 
> [https://github.com/apache/kafka/blob/2.4.0/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L842-L844]
>  []|#L842-L844]] calls `doJoinOnForeignKey` with an argument of 
> `Materialized.with(null, null)`, as apparently do several other APIs. As the 
> comment spanning [these lines|#L1098-L1099]] makes clear, the result is a 
> `KTable` whose `valueSerde` (as a `KTableImpl`) is `null`. Therefore, 
> attempts to `join` etc. on the resulting `KTable` fail with a 
> `NullPointerException`.
> While there is an obvious workaround—explicitly construct the required 
> `Materialized` and use the APIs that take it as an argument—I have to admit I 
> find the existence of public APIs with this sort of bug, particularly when 
> the bug is literally documented as a comment in the source code, astonishing 
> to the point of incredulity. It calls the quality and trustworthiness of 
> Kafka Streams into serious question, and if a resolution is not forthcoming 
> within a week, we will be left with no other option but to consider technical 
> alternatives.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9517) KTable Joins Without Materialized Argument Yield Results That Further Joins NPE On

2020-02-10 Thread John Roesler (Jira)


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

John Roesler commented on KAFKA-9517:
-

Quick update, I've merged this fix to trunk, 2.5, and 2.4. I'll go ahead and 
mark this ticket resolved so that it doesn't prevent the creation of 2.4.1 or 
2.5.0 release candidates.

I'd still very much appreciate it if you can test it to make sure it resolves 
the issue for your use case. 

> KTable Joins Without Materialized Argument Yield Results That Further Joins 
> NPE On
> --
>
> Key: KAFKA-9517
> URL: https://issues.apache.org/jira/browse/KAFKA-9517
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.4.0
>Reporter: Paul Snively
>Assignee: John Roesler
>Priority: Blocker
> Fix For: 2.5.0, 2.4.1
>
> Attachments: test.tar.xz
>
>
> The `KTable` API implemented [[here||#L842-L844]] 
> [https://github.com/apache/kafka/blob/2.4.0/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L842-L844]
>  []|#L842-L844]] calls `doJoinOnForeignKey` with an argument of 
> `Materialized.with(null, null)`, as apparently do several other APIs. As the 
> comment spanning [these lines|#L1098-L1099]] makes clear, the result is a 
> `KTable` whose `valueSerde` (as a `KTableImpl`) is `null`. Therefore, 
> attempts to `join` etc. on the resulting `KTable` fail with a 
> `NullPointerException`.
> While there is an obvious workaround—explicitly construct the required 
> `Materialized` and use the APIs that take it as an argument—I have to admit I 
> find the existence of public APIs with this sort of bug, particularly when 
> the bug is literally documented as a comment in the source code, astonishing 
> to the point of incredulity. It calls the quality and trustworthiness of 
> Kafka Streams into serious question, and if a resolution is not forthcoming 
> within a week, we will be left with no other option but to consider technical 
> alternatives.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9517) KTable Joins Without Materialized Argument Yield Results That Further Joins NPE On

2020-02-10 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on KAFKA-9517:
---

vvcephei commented on pull request #8061: KAFKA-9517: Fix default serdes with 
FK join
URL: https://github.com/apache/kafka/pull/8061
 
 
   
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> KTable Joins Without Materialized Argument Yield Results That Further Joins 
> NPE On
> --
>
> Key: KAFKA-9517
> URL: https://issues.apache.org/jira/browse/KAFKA-9517
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.4.0
>Reporter: Paul Snively
>Assignee: John Roesler
>Priority: Blocker
> Fix For: 2.5.0, 2.4.1
>
> Attachments: test.tar.xz
>
>
> The `KTable` API implemented [[here||#L842-L844]] 
> [https://github.com/apache/kafka/blob/2.4.0/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L842-L844]
>  []|#L842-L844]] calls `doJoinOnForeignKey` with an argument of 
> `Materialized.with(null, null)`, as apparently do several other APIs. As the 
> comment spanning [these lines|#L1098-L1099]] makes clear, the result is a 
> `KTable` whose `valueSerde` (as a `KTableImpl`) is `null`. Therefore, 
> attempts to `join` etc. on the resulting `KTable` fail with a 
> `NullPointerException`.
> While there is an obvious workaround—explicitly construct the required 
> `Materialized` and use the APIs that take it as an argument—I have to admit I 
> find the existence of public APIs with this sort of bug, particularly when 
> the bug is literally documented as a comment in the source code, astonishing 
> to the point of incredulity. It calls the quality and trustworthiness of 
> Kafka Streams into serious question, and if a resolution is not forthcoming 
> within a week, we will be left with no other option but to consider technical 
> alternatives.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9517) KTable Joins Without Materialized Argument Yield Results That Further Joins NPE On

2020-02-10 Thread John Roesler (Jira)


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

John Roesler commented on KAFKA-9517:
-

Hi Paul,

Ah, those failures shouldn't affect your testing. #8015 depends on a fix I 
added for the TopologyTestDriver (https://github.com/apache/kafka/pull/8065). 
If you want to clear the error, you can cherry-pick that one also, or you can 
just skip the tests and build the artifacts directly with `installAll`.

Sorry for neglecting to mention that initially; both changes were part of 8015 
to begin with, but the reviewers rightly suggested I should pull out the 
TopologyTestDriver fix into a separately verified PR.

Thanks,
-John

> KTable Joins Without Materialized Argument Yield Results That Further Joins 
> NPE On
> --
>
> Key: KAFKA-9517
> URL: https://issues.apache.org/jira/browse/KAFKA-9517
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.4.0
>Reporter: Paul Snively
>Assignee: John Roesler
>Priority: Blocker
> Fix For: 2.5.0, 2.4.1
>
> Attachments: test.tar.xz
>
>
> The `KTable` API implemented [[here||#L842-L844]] 
> [https://github.com/apache/kafka/blob/2.4.0/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L842-L844]
>  []|#L842-L844]] calls `doJoinOnForeignKey` with an argument of 
> `Materialized.with(null, null)`, as apparently do several other APIs. As the 
> comment spanning [these lines|#L1098-L1099]] makes clear, the result is a 
> `KTable` whose `valueSerde` (as a `KTableImpl`) is `null`. Therefore, 
> attempts to `join` etc. on the resulting `KTable` fail with a 
> `NullPointerException`.
> While there is an obvious workaround—explicitly construct the required 
> `Materialized` and use the APIs that take it as an argument—I have to admit I 
> find the existence of public APIs with this sort of bug, particularly when 
> the bug is literally documented as a comment in the source code, astonishing 
> to the point of incredulity. It calls the quality and trustworthiness of 
> Kafka Streams into serious question, and if a resolution is not forthcoming 
> within a week, we will be left with no other option but to consider technical 
> alternatives.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9517) KTable Joins Without Materialized Argument Yield Results That Further Joins NPE On

2020-02-10 Thread Paul Snively (Jira)


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

Paul Snively commented on KAFKA-9517:
-

Thanks for the suggestion to squash and cherry-pick #8015 and #8061. I've done 
that, and `.gradlew test` is giving me four errors that seem related to the 
cherry-picked PRs. I'm attaching the test report for others to perhaps analyze. 
My colleague and I will also attempt to reproduce the issues we specifically 
encountered in using 2.4.0. I'm reasonably confident we can also take some time 
to review these two PRs, but that seems somewhat unlikely to happen today.

[^test.tar.xz]

> KTable Joins Without Materialized Argument Yield Results That Further Joins 
> NPE On
> --
>
> Key: KAFKA-9517
> URL: https://issues.apache.org/jira/browse/KAFKA-9517
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.4.0
>Reporter: Paul Snively
>Assignee: John Roesler
>Priority: Blocker
> Fix For: 2.5.0, 2.4.1
>
> Attachments: test.tar.xz
>
>
> The `KTable` API implemented [[here||#L842-L844]] 
> [https://github.com/apache/kafka/blob/2.4.0/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L842-L844]
>  []|#L842-L844]] calls `doJoinOnForeignKey` with an argument of 
> `Materialized.with(null, null)`, as apparently do several other APIs. As the 
> comment spanning [these lines|#L1098-L1099]] makes clear, the result is a 
> `KTable` whose `valueSerde` (as a `KTableImpl`) is `null`. Therefore, 
> attempts to `join` etc. on the resulting `KTable` fail with a 
> `NullPointerException`.
> While there is an obvious workaround—explicitly construct the required 
> `Materialized` and use the APIs that take it as an argument—I have to admit I 
> find the existence of public APIs with this sort of bug, particularly when 
> the bug is literally documented as a comment in the source code, astonishing 
> to the point of incredulity. It calls the quality and trustworthiness of 
> Kafka Streams into serious question, and if a resolution is not forthcoming 
> within a week, we will be left with no other option but to consider technical 
> alternatives.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9517) KTable Joins Without Materialized Argument Yield Results That Further Joins NPE On

2020-02-07 Thread John Roesler (Jira)


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

John Roesler commented on KAFKA-9517:
-

Hi [~psnively], thanks for your kind words.

Actually, the 2.4.1 release is currently in progress. Fortunately, we caught 
these issues early enough to have them included as blockers for the release. 
The release plan is here: 
https://cwiki.apache.org/confluence/display/KAFKA/Release+Plan+2.4.1
Of course, it cannot progress until the blockers are resolved, which is the 
biggest wild-card in the timeline. As soon as the blockers are cleared, Bill 
(who volunteered to drive the release) will be able to give a better estimate 
about the timeline.

As you said, the biggest thing you can do to help is to check out the PRs and 
test them. I think you'll need both #8015 and #8061. As per Apache Kafka 
policies, the PRs are actually based on trunk, so you'll want to squash them 
and cherry-pick them onto 2.4 to get an accurate proxy for 2.4.1 . This is 
actually a huge help, since even extensive testing can have subtle but 
important gaps (which is how we wound up with these bugs to begin with).

The other bug thing you can do if you have time is review the PRs. You've 
already become familiar enough with the code to identify the root cause even 
before I saw it, and a fresh perspective is always helpful.

Thanks again,
-John

> KTable Joins Without Materialized Argument Yield Results That Further Joins 
> NPE On
> --
>
> Key: KAFKA-9517
> URL: https://issues.apache.org/jira/browse/KAFKA-9517
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.4.0
>Reporter: Paul Snively
>Priority: Blocker
> Fix For: 2.5.0, 2.4.1
>
>
> The `KTable` API implemented [[here||#L842-L844]] 
> [https://github.com/apache/kafka/blob/2.4.0/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L842-L844]
>  []|#L842-L844]] calls `doJoinOnForeignKey` with an argument of 
> `Materialized.with(null, null)`, as apparently do several other APIs. As the 
> comment spanning [these lines|#L1098-L1099]] makes clear, the result is a 
> `KTable` whose `valueSerde` (as a `KTableImpl`) is `null`. Therefore, 
> attempts to `join` etc. on the resulting `KTable` fail with a 
> `NullPointerException`.
> While there is an obvious workaround—explicitly construct the required 
> `Materialized` and use the APIs that take it as an argument—I have to admit I 
> find the existence of public APIs with this sort of bug, particularly when 
> the bug is literally documented as a comment in the source code, astonishing 
> to the point of incredulity. It calls the quality and trustworthiness of 
> Kafka Streams into serious question, and if a resolution is not forthcoming 
> within a week, we will be left with no other option but to consider technical 
> alternatives.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9517) KTable Joins Without Materialized Argument Yield Results That Further Joins NPE On

2020-02-07 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on KAFKA-9517:
---

vvcephei commented on pull request #8061: KAFKA-9517: Fix default serdes with 
FK join
URL: https://github.com/apache/kafka/pull/8061
 
 
   During the KIP-213 implementation and verification, we neglected to test the 
code path for falling back to default serdes if none are given in the topology.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> KTable Joins Without Materialized Argument Yield Results That Further Joins 
> NPE On
> --
>
> Key: KAFKA-9517
> URL: https://issues.apache.org/jira/browse/KAFKA-9517
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.4.0
>Reporter: Paul Snively
>Priority: Critical
>
> The `KTable` API implemented [[here||#L842-L844]] 
> [https://github.com/apache/kafka/blob/2.4.0/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L842-L844]
>  []|#L842-L844]] calls `doJoinOnForeignKey` with an argument of 
> `Materialized.with(null, null)`, as apparently do several other APIs. As the 
> comment spanning [these lines|#L1098-L1099]] makes clear, the result is a 
> `KTable` whose `valueSerde` (as a `KTableImpl`) is `null`. Therefore, 
> attempts to `join` etc. on the resulting `KTable` fail with a 
> `NullPointerException`.
> While there is an obvious workaround—explicitly construct the required 
> `Materialized` and use the APIs that take it as an argument—I have to admit I 
> find the existence of public APIs with this sort of bug, particularly when 
> the bug is literally documented as a comment in the source code, astonishing 
> to the point of incredulity. It calls the quality and trustworthiness of 
> Kafka Streams into serious question, and if a resolution is not forthcoming 
> within a week, we will be left with no other option but to consider technical 
> alternatives.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9517) KTable Joins Without Materialized Argument Yield Results That Further Joins NPE On

2020-02-07 Thread Paul Snively (Jira)


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

Paul Snively commented on KAFKA-9517:
-

Speaking of things I can do: I am downloading PR #8015 as a patch, and will 
apply it locally, build the appropriate `.jar`s, and we will attempt to 
reproduce the issues we've seen given that PR, and report back.

> KTable Joins Without Materialized Argument Yield Results That Further Joins 
> NPE On
> --
>
> Key: KAFKA-9517
> URL: https://issues.apache.org/jira/browse/KAFKA-9517
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.4.0
>Reporter: Paul Snively
>Priority: Critical
>
> The `KTable` API implemented [[here||#L842-L844]] 
> [https://github.com/apache/kafka/blob/2.4.0/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L842-L844]
>  []|#L842-L844]] calls `doJoinOnForeignKey` with an argument of 
> `Materialized.with(null, null)`, as apparently do several other APIs. As the 
> comment spanning [these lines|#L1098-L1099]] makes clear, the result is a 
> `KTable` whose `valueSerde` (as a `KTableImpl`) is `null`. Therefore, 
> attempts to `join` etc. on the resulting `KTable` fail with a 
> `NullPointerException`.
> While there is an obvious workaround—explicitly construct the required 
> `Materialized` and use the APIs that take it as an argument—I have to admit I 
> find the existence of public APIs with this sort of bug, particularly when 
> the bug is literally documented as a comment in the source code, astonishing 
> to the point of incredulity. It calls the quality and trustworthiness of 
> Kafka Streams into serious question, and if a resolution is not forthcoming 
> within a week, we will be left with no other option but to consider technical 
> alternatives.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9517) KTable Joins Without Materialized Argument Yield Results That Further Joins NPE On

2020-02-07 Thread Paul Snively (Jira)


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

Paul Snively commented on KAFKA-9517:
-

[~vvcephei], that's great news! First, let me thank you again for your prompt 
attention. It's done a great deal to restore my confidence in Kafka Streams. 
Second, and I hate to ask because I know it's a big project with many other 
customers with various needs, but do you happen to have some idea when a 2.4.1 
might be available with fixes for these, and is there anything I can do to help 
with the process?

Thanks again!

> KTable Joins Without Materialized Argument Yield Results That Further Joins 
> NPE On
> --
>
> Key: KAFKA-9517
> URL: https://issues.apache.org/jira/browse/KAFKA-9517
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.4.0
>Reporter: Paul Snively
>Priority: Critical
>
> The `KTable` API implemented [[here||#L842-L844]] 
> [https://github.com/apache/kafka/blob/2.4.0/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L842-L844]
>  []|#L842-L844]] calls `doJoinOnForeignKey` with an argument of 
> `Materialized.with(null, null)`, as apparently do several other APIs. As the 
> comment spanning [these lines|#L1098-L1099]] makes clear, the result is a 
> `KTable` whose `valueSerde` (as a `KTableImpl`) is `null`. Therefore, 
> attempts to `join` etc. on the resulting `KTable` fail with a 
> `NullPointerException`.
> While there is an obvious workaround—explicitly construct the required 
> `Materialized` and use the APIs that take it as an argument—I have to admit I 
> find the existence of public APIs with this sort of bug, particularly when 
> the bug is literally documented as a comment in the source code, astonishing 
> to the point of incredulity. It calls the quality and trustworthiness of 
> Kafka Streams into serious question, and if a resolution is not forthcoming 
> within a week, we will be left with no other option but to consider technical 
> alternatives.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9517) KTable Joins Without Materialized Argument Yield Results That Further Joins NPE On

2020-02-06 Thread John Roesler (Jira)


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

John Roesler commented on KAFKA-9517:
-

Aha! I think I've found it:

{noformat}
java.lang.NullPointerException
at 
org.apache.kafka.streams.kstream.internals.foreignkeyjoin.SubscriptionWrapperSerde.(SubscriptionWrapperSerde.java:31)
at 
org.apache.kafka.streams.kstream.internals.KTableImpl.doJoinOnForeignKey(KTableImpl.java:956)
at 
org.apache.kafka.streams.kstream.internals.KTableImpl.join(KTableImpl.java:845)
at 
org.apache.kafka.streams.TopologyTestDriverTest.shouldProduceOutputsInTheRightOrder(TopologyTestDriverTest.java:1533)
{noformat}

produced with the following test (that I'm working on for KAFKA-9503):

{noformat}
public void shouldProduceOutputsInTheRightOrder() {
final StreamsBuilder builder = new StreamsBuilder();
final KTable aTable = builder.table("A");
final KTable bTable = builder.table("B");

final KTable fkJoinResult = aTable.join(
bTable,
value -> value.split("-")[0],
(aVal, bVal) -> "(" + aVal + "," + bVal + ")"
);

final KTable finalJoinResult = aTable.join(
fkJoinResult,
(aVal, fkJoinVal) -> "(" + aVal + "," + fkJoinVal + ")"
);

finalJoinResult.toStream().to("output");

System.out.println(builder.build().describe());

final Properties config = new Properties();
config.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "dummy");
config.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy");
config.setProperty(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, 
Serdes.StringSerde.class.getName());
config.setProperty(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, 
Serdes.StringSerde.class.getName());
try (final TopologyTestDriver topologyTestDriver = new 
TopologyTestDriver(builder.build(), config)) {
final TestInputTopic aTopic = 
topologyTestDriver.createInputTopic("A", new StringSerializer(), new 
StringSerializer());
final TestInputTopic bTopic = 
topologyTestDriver.createInputTopic("B", new StringSerializer(), new 
StringSerializer());
final TestOutputTopic output = 
topologyTestDriver.createOutputTopic("output", new StringDeserializer(), new 
StringDeserializer());
aTopic.pipeInput("a1", "b1-alpha");
aTopic.pipeInput("b1", "beta");
System.out.println(output.readKeyValuesToMap());
}
}
{noformat}



> KTable Joins Without Materialized Argument Yield Results That Further Joins 
> NPE On
> --
>
> Key: KAFKA-9517
> URL: https://issues.apache.org/jira/browse/KAFKA-9517
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.4.0
>Reporter: Paul Snively
>Priority: Critical
>
> The `KTable` API implemented [[here||#L842-L844]] 
> [https://github.com/apache/kafka/blob/2.4.0/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L842-L844]
>  []|#L842-L844]] calls `doJoinOnForeignKey` with an argument of 
> `Materialized.with(null, null)`, as apparently do several other APIs. As the 
> comment spanning [these lines|#L1098-L1099]] makes clear, the result is a 
> `KTable` whose `valueSerde` (as a `KTableImpl`) is `null`. Therefore, 
> attempts to `join` etc. on the resulting `KTable` fail with a 
> `NullPointerException`.
> While there is an obvious workaround—explicitly construct the required 
> `Materialized` and use the APIs that take it as an argument—I have to admit I 
> find the existence of public APIs with this sort of bug, particularly when 
> the bug is literally documented as a comment in the source code, astonishing 
> to the point of incredulity. It calls the quality and trustworthiness of 
> Kafka Streams into serious question, and if a resolution is not forthcoming 
> within a week, we will be left with no other option but to consider technical 
> alternatives.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9517) KTable Joins Without Materialized Argument Yield Results That Further Joins NPE On

2020-02-06 Thread John Roesler (Jira)


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

John Roesler commented on KAFKA-9517:
-

Thanks for the update, [~psnively].

I've been trying to reproduce the issue you've reported (building on the fix 
already for KAFKA-9500), and I haven't been able to produce an NPE. Do you 
happen to have the stacktrace handy?

To address the concern you voiced about the comment in the code, the valueSerde 
is set to `null` to indicate that we do not know the serde a priori. This sets 
us up to use the following precedence rules at run time.

If an operator needs to serialize some data, it will use:
#1 The serde explicily passed to it, via Materialized, Produced, Grouped, etc.
#2 If no explicit serde is passed, then use the serde passed from the upstream 
operator, if applicable*
#3 If no explicit or upstream serde is available, use the "default" serde 
provided in config

* this is where passing a null downstream indicates that no applicable serde is 
available.

So, what I would expect to see is a ClassCastException if the "default" serde 
isn't for the same type as the foreign key join result. But I think maybe I 
misunderstood the scenario you provided.

Thanks,
-John

> KTable Joins Without Materialized Argument Yield Results That Further Joins 
> NPE On
> --
>
> Key: KAFKA-9517
> URL: https://issues.apache.org/jira/browse/KAFKA-9517
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.4.0
>Reporter: Paul Snively
>Priority: Critical
>
> The `KTable` API implemented [[here||#L842-L844]] 
> [https://github.com/apache/kafka/blob/2.4.0/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L842-L844]
>  []|#L842-L844]] calls `doJoinOnForeignKey` with an argument of 
> `Materialized.with(null, null)`, as apparently do several other APIs. As the 
> comment spanning [these lines|#L1098-L1099]] makes clear, the result is a 
> `KTable` whose `valueSerde` (as a `KTableImpl`) is `null`. Therefore, 
> attempts to `join` etc. on the resulting `KTable` fail with a 
> `NullPointerException`.
> While there is an obvious workaround—explicitly construct the required 
> `Materialized` and use the APIs that take it as an argument—I have to admit I 
> find the existence of public APIs with this sort of bug, particularly when 
> the bug is literally documented as a comment in the source code, astonishing 
> to the point of incredulity. It calls the quality and trustworthiness of 
> Kafka Streams into serious question, and if a resolution is not forthcoming 
> within a week, we will be left with no other option but to consider technical 
> alternatives.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9517) KTable Joins Without Materialized Argument Yield Results That Further Joins NPE On

2020-02-06 Thread Paul Snively (Jira)


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

Paul Snively commented on KAFKA-9517:
-

John,

Having been sharply critical of the project, now let me thank you for the 
speedy response. I'll look at the related issue and PR immediately.

> KTable Joins Without Materialized Argument Yield Results That Further Joins 
> NPE On
> --
>
> Key: KAFKA-9517
> URL: https://issues.apache.org/jira/browse/KAFKA-9517
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.4.0
>Reporter: Paul Snively
>Priority: Critical
>
> The `KTable` API implemented [[here||#L842-L844]] 
> [https://github.com/apache/kafka/blob/2.4.0/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L842-L844]
>  []|#L842-L844]] calls `doJoinOnForeignKey` with an argument of 
> `Materialized.with(null, null)`, as apparently do several other APIs. As the 
> comment spanning [these lines|#L1098-L1099]] makes clear, the result is a 
> `KTable` whose `valueSerde` (as a `KTableImpl`) is `null`. Therefore, 
> attempts to `join` etc. on the resulting `KTable` fail with a 
> `NullPointerException`.
> While there is an obvious workaround—explicitly construct the required 
> `Materialized` and use the APIs that take it as an argument—I have to admit I 
> find the existence of public APIs with this sort of bug, particularly when 
> the bug is literally documented as a comment in the source code, astonishing 
> to the point of incredulity. It calls the quality and trustworthiness of 
> Kafka Streams into serious question, and if a resolution is not forthcoming 
> within a week, we will be left with no other option but to consider technical 
> alternatives.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (KAFKA-9517) KTable Joins Without Materialized Argument Yield Results That Further Joins NPE On

2020-02-06 Thread John Roesler (Jira)


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

John Roesler commented on KAFKA-9517:
-

Hey [~psnively],

Thanks for the report. I think this is the same as KAFKA-9500 . 

I already have a PR to fix it, although I need to address some comments.

If you like, you could take a look at https://github.com/apache/kafka/pull/8015 
to see if it fixes the issue for you.

> KTable Joins Without Materialized Argument Yield Results That Further Joins 
> NPE On
> --
>
> Key: KAFKA-9517
> URL: https://issues.apache.org/jira/browse/KAFKA-9517
> Project: Kafka
>  Issue Type: Bug
>  Components: streams
>Affects Versions: 2.4.0
>Reporter: Paul Snively
>Priority: Critical
>
> The `KTable` API implemented [[here||#L842-L844]] 
> [https://github.com/apache/kafka/blob/2.4.0/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java#L842-L844]
>  []|#L842-L844]] calls `doJoinOnForeignKey` with an argument of 
> `Materialized.with(null, null)`, as apparently do several other APIs. As the 
> comment spanning [these lines|#L1098-L1099]] makes clear, the result is a 
> `KTable` whose `valueSerde` (as a `KTableImpl`) is `null`. Therefore, 
> attempts to `join` etc. on the resulting `KTable` fail with a 
> `NullPointerException`.
> While there is an obvious workaround—explicitly construct the required 
> `Materialized` and use the APIs that take it as an argument—I have to admit I 
> find the existence of public APIs with this sort of bug, particularly when 
> the bug is literally documented as a comment in the source code, astonishing 
> to the point of incredulity. It calls the quality and trustworthiness of 
> Kafka Streams into serious question, and if a resolution is not forthcoming 
> within a week, we will be left with no other option but to consider technical 
> alternatives.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)