KAFKA-7758: Reuse KGroupedStream/KGroupedTable with named repartition topics#6265
Conversation
There was a problem hiding this comment.
On the first invocation, the repartitionNode is always null, so we'll build the repartition node and set the variable. But on subsequent calls, only if the user has provided a name for the repartition topic will we re-use the repartition graph node.
This maintains compatibility as users not naming the repartition topic will get the same topology and re-using the KGroupedStream results in an InvalidToplogyException when building the topology
There was a problem hiding this comment.
and re-using the
KGroupedStreamresults in anInvalidToplogyExceptionwhen building the topology
I thought, if there is no user topic-name, old code would create multiple repartition topics? And re-using KGroupedStream only throughs if there is a user topic-name (and this restriction is lifted with this PR)
There was a problem hiding this comment.
I had a hard time reasoning about the or condition and what exactly happens when there is a name / isn't a name / repartition isn't required at all.
Can we consider breaking these up into three separate cases to make the code more linear?
There was a problem hiding this comment.
I thought, if there is no user topic-name, old code would create multiple repartition topics? And re-using
KGroupedStreamonly throughs if there is a user topic-name (and this restriction is lifted with this PR)
That's correct. I may not have been clear above, but what I meant is that this change won't break compatibility with users currently not providing a repartition topic name as it will create multiple repartition topics thus keep their topology the same. Does that make sense?
There was a problem hiding this comment.
Can we consider breaking these up into three separate cases to make the code more linear?
Repartitons are required when the repartitionRequired flag is true as passed in from the parent KStream instance https://github.com/apache/kafka/pull/6265/files#diff-2585c8864b9121454af6d88adba60975R81. A name for the repartition topic is always required; it's just a matter of using a generated one or the user provided one.
So we are left with just two cases:
- On the first call the to
GroupedStreamAggregateBuilder#build()method thereparitionNodewill always benull, so we need to create it, regardless if the user has provided a reparation topic name or not. - The other case is when we make subsequent
GroupedStreamAggregateBuilder#build()calls to the sameGroupedStreamAggregateBuilderinstance; then the name does come into play. If the user has provided a repartition topic name, we would reuse the same repartition node. Otherwise, it's null, so we know to create a new repartition node.
I originally had this:
if (repartitionNode == null) {
repartitionNode = repartitionNodeBuilder.build();
} else if (userProvidedRepartitionTopicName == null) {
repartitionNode = repartitionNodeBuilder.build();
}But I felt that having multiple lines with the same variable assignment did not make sense. What do think about me adding my explanation here to line 86 as comments?
There was a problem hiding this comment.
Yeah, comments would help. I think the logic is correct, but it's just a little complex to reason about. Some stuff I came to realize only by poking around the code:
- there is a 1:1 relationship between a GroupedStreamAggregateBuilder instance and a KGroupedStreamImpl instance (i.e., we construct the builder in the constructor of the grouped stream node), so when we cache a repartition node in the builder, it's exactly like caching it in the KGroupedStream node, which is intuitively what we wanted to do.
- If repartition is not required, we don't build a repartition node at all, but just bypass it in the graph.
- Although we cache the repartition node in an instance field, we actually only use it if the node is user-named. If the name is auto-generated, we overwrite the repartition node every time before adding it to the graph.
2 and 3 were basically what I had in mind as separate branches. Something like:
- if repartition not required, just connect the agg node directly to the source node
- if repartition is required and not user-named, make a brand-new repartition node, and hook it up to the source node and hook the agg node up to the repartition node
- if the repartition is required and user-named, then check if there's already a repartition node (if not, create one, hook it up to the source node, and cache it), then hook the agg node up to the repartition node
This is just to clarify my comment above. If you prefer to clarify the flow with the explanation you gave, that's fine as well.
There was a problem hiding this comment.
That's correct. I may not have been clear above, but what I meant is that this change won't break compatibility with users currently not providing a repartition topic name as it will create multiple repartition topics thus keep their topology the same. Does that make sense?
Ack. Just wanted to make sure we are on the same page :)
There was a problem hiding this comment.
2 and 3 were basically what I had in mind as separate branches.
We already have this
- Line 84 is the guard for checking if a repartition is not required, if not then we do skip all the repartition node creation steps and just add the agg node to the source node.
- If a repartition is required we drop into the block and make the necessary node creation and graph links.
I'll add a comment on the boolean statement just to clarify what's going on.
There was a problem hiding this comment.
I did not in-line this method as I feel it helps with the readability of the code.
There was a problem hiding this comment.
Does it? It's a one liner calling createRepartitionedSource, so what does we gain by calling createRepartitionSource instead?
There was a problem hiding this comment.
I guess it's personal preference if you insist I can inline it.
There was a problem hiding this comment.
Don't insist -- would prefer though (for this case) -- in general, this pattern can be useful -- just don't see it for this particular case.
There was a problem hiding this comment.
On the first invocation, the repartitionNode is always null, so we'll build the repartition node and set the variable. But on subsequent calls, only if the user has provided a name for the repartition topic will we re-use the repartition graph node.
This maintains compatibility as users not naming the repartition topic will get the same topology and re-using the KGroupedTable results in an InvalidToplogyException when building the topology
|
@ableegoldman, @guozhangwang, @mjsax, @vvcephei for reviews |
|
Java 11 failure unrelated retest this please |
mjsax
left a comment
There was a problem hiding this comment.
Overall LGTM -- just couple of nits.
One related by orthogonal question: what happens if a TimeWindowedKStream or SessionWindowedKStream is reused?
There was a problem hiding this comment.
and re-using the
KGroupedStreamresults in anInvalidToplogyExceptionwhen building the topology
I thought, if there is no user topic-name, old code would create multiple repartition topics? And re-using KGroupedStream only throughs if there is a user topic-name (and this restriction is lifted with this PR)
There was a problem hiding this comment.
nit: use static import to shorten line length?
There was a problem hiding this comment.
Does it? It's a one liner calling createRepartitionedSource, so what does we gain by calling createRepartitionSource instead?
There was a problem hiding this comment.
Thanks for renaming! Much better!
There was a problem hiding this comment.
nit: break long line into multiple
vvcephei
left a comment
There was a problem hiding this comment.
LGTM. I just left one comment about readability, but I think it's also fine if you prefer it the way it is.
There was a problem hiding this comment.
Is this javadoc still accurate?
There was a problem hiding this comment.
Not your code, but did we want the assert on L73 to execute all the time, rather than just when java is executed with -ea?
There was a problem hiding this comment.
I had a hard time reasoning about the or condition and what exactly happens when there is a name / isn't a name / repartition isn't required at all.
Can we consider breaking these up into three separate cases to make the code more linear?
Good question! When calling either |
befacb6 to
f09bfe1
Compare
mjsax
left a comment
There was a problem hiding this comment.
LGTM.
I don't see any issue with the logic (and don't think comments are necessary -- also, it might be better to refactor code before adding comments, so the code documents itself.) Leave it up to you if you want to address John's comment.
If you refactor the code, I can also have a look again.
|
Java 11 had an unrelated failure retest this please |
|
both Java 11 and Java 8 timed out retest this please |
|
Java11 passed. and and Retest this please. |
|
This time Java8 passed. https://builds.apache.org/job/kafka-pr-jdk11-scala2.12/2422/ Thus, we got one green build each. Merging to |
… topics (#6265) Reviewers: John Roesler <john@confluent.io>, Matthias J. Sax <matthias@confluent.io>
|
Merged to |
* ak/trunk: (45 commits) KAFKA-7487: DumpLogSegments misreports offset mismatches (apache#5756) MINOR: improve JavaDocs about auto-repartitioning in Streams DSL (apache#6269) KAFKA-7935: UNSUPPORTED_COMPRESSION_TYPE if ReplicaManager.getLogConfig returns None (apache#6274) KAFKA-7895: Fix stream-time reckoning for suppress (apache#6278) KAFKA-6569: Move OffsetIndex/TimeIndex logger to companion object (apache#4586) MINOR: add log indicating the suppression time (apache#6260) MINOR: Make info logs for KafkaConsumer a bit more verbose (apache#6279) KAFKA-7758: Reuse KGroupedStream/KGroupedTable with named repartition topics (apache#6265) KAFKA-7884; Docs for message.format.version should display valid values (apache#6209) MINOR: Save failed test output to build output directory MINOR: add test for StreamsSmokeTestDriver (apache#6231) MINOR: Fix bugs identified by compiler warnings (apache#6258) KAFKA-6474: Rewrite tests to use new public TopologyTestDriver [part 4] (apache#5433) MINOR: fix bypasses in ChangeLogging stores (apache#6266) MINOR: Make MockClient#poll() more thread-safe (apache#5942) MINOR: drop dbAccessor reference on close (apache#6254) KAFKA-7811: Avoid unnecessary lock acquire when KafkaConsumer commits offsets (apache#6119) KAFKA-7916: Unify store wrapping code for clarity (apache#6255) MINOR: Add missing Alter Operation to Topic supported operations list in AclCommand KAFKA-7921: log at error level for missing source topic (apache#6262) ...
… topics (apache#6265) Reviewers: John Roesler <john@confluent.io>, Matthias J. Sax <matthias@confluent.io>
This PR adds support for re-using a
KGroupedStreamor `KGroupedTable object after executing an aggregation operation with a named repartition topic.KGroupedStreamexampleKGroupedTableexampleThis approach will not cause any compatibility issues for two reasons
KGroupedStreamorKGroupedTableobject as Kafka Streams throws anInvalidTopologyExceptionwhen building the topology. Hence you can't even deploy the application.I've added unit tests for each case and ran our existing suite of streams tests.
Committer Checklist (excluded from commit message)