New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
KAFKA-3856: Refactoring for KIP-120 #3536
KAFKA-3856: Refactoring for KIP-120 #3536
Conversation
mjsax
commented
Jul 17, 2017
•
edited
edited
- extract InternalTopologyBuilder from TopologyBuilder
- deprecate all "leaking" methods from public TopologyBuilder API
- changed TopologyDescription and all nested classed into interfaces
b963505
to
9783943
Compare
Call for review @bbejeck @dguy @enothereska @guozhangwang |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
@@ -222,9 +222,9 @@ public void shouldDriveGlobalStore() throws Exception { | |||
} | |||
|
|||
@Test | |||
public void testDrivingSimpleMultiSourceTopology() { | |||
public void testDrivingSimpleMultiSourceTopology() throws Exception { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Which part throws the exception? (same question for the other tests below). Could we use a specific exception, or is it broad?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I guess, this is some "left over" as I cut down this PR to make is smaller... (ie, did remove some other changes that will follow in another RP -- just forgot to undo this changes to -- should be ok to just leave it IMHO) For test in general it's ok to just declare "Exception" as nobody catches it anyway (it's best practice in Java AFAIK).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
+1 to leave in. IMHO it makes the test more readable without the try/catch boilerplate.
Massive cleanup, thanks @mjsax. I couldn't find |
@enothereska I'll split up the whole KIP in multiple PRs. Otherwise it's a mess to review. There is already #2301 that does add |
Looks good, thanks for the big cleanup. One question (sorry if this is an obvious one): the |
@mjsax I'm wondering if could you try breaking this PR into to commits for easier reviewing? One commit for just extracting out the EDIT: actually nvm, assuming that the |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Minor comments, otherwise LGTM.
import org.apache.kafka.common.metrics.stats.Max; | ||
import org.apache.kafka.common.metrics.stats.Rate; | ||
import org.apache.kafka.common.metrics.stats.Sum; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Never know how do you find these: good catch :)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Intellij does -- it automatically sort imports if you enable it :)
@@ -36,7 +36,7 @@ | |||
* Factory for creating state stores in Kafka Streams. | |||
*/ | |||
@InterfaceStability.Evolving | |||
public class Stores { | |||
public class Stores { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This seems not intentional.
public synchronized final TopologyBuilder addSource(final AutoOffsetReset offsetReset, final String name, final String... topics) { | ||
return addSource(offsetReset, name, null, null, null, topics); | ||
public synchronized final TopologyBuilder addSource(final AutoOffsetReset offsetReset, | ||
final String name, final String... topics) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: if we want to break into multi-lines better do that one parameter per line.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good catch! That was the intent. Just missed this one.
throw new TopologyBuilderException("Source store " + sourceStoreName + " is already added."); | ||
} | ||
storeToChangelogTopic.put(sourceStoreName, topic); | ||
@Deprecated |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: add similar NOTE as in 668 as well, also mentioning it has been deprecated?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ditto for all other deprecated functions: mention it should only be used internally and has been deprecated.
@mjsax could you rebase? |
Yes. |
- extract InternalTopologyBuilder from TopologyBuilder - deprecate all "leaking" methods from public TopologyBuilder API
80653c4
to
bfc2527
Compare
Refer to this link for build results (access rights to CI server needed): |
bfc2527
to
d53157d
Compare
Did rebase this and addressed "dangling comments" from #2301 |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
Updates with regard to both comments on #2301 |
Refer to this link for build results (access rights to CI server needed): |
Refer to this link for build results (access rights to CI server needed): |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@mjsax I left a comment on the public APIs of describe()
, as described in https://cwiki.apache.org/confluence/display/KAFKA/KIP-120%3A+Cleanup+Kafka+Streams+builder+API. Will be merging as is since I understand this is not the last PR of KIP-120 so you may have some ideas about modifying such a bit later, but just as a reminder..
return false; | ||
} | ||
|
||
public TopologyDescription describe() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmm.. The function is aimed to be a public API in Topology
in KIP-120 (which I think will be renamed from the current TopologyBuilder
?), but here it is only used as the function of the InternalTopologyBuilder
which seems incorrect.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
the function is aimed to be a public API in Topology in KIP-120
correct
which I think will be renamed from the current TopologyBuilder?
No. We need to keep TopologyBuilder
for backward compatibility (just deprecating it) and add Topology
next to it. And Topology
will also have describe()
in it's public API and than call InternalTopologyBuilder#describe()
(we might also be able to move this method completely from InternalTopologyBuilder
to Topology
, but this would be just an internal refactoring.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for the clarification.
Also one thing I notice from the unit tests is that in StreamThreadTest
we are accessing the builder's internalTopologyBuilder
object directly which should be deprecated and removed eventually from the TopologyBuilder
. I believe you are going to update on that with the next follow-up PRs right?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I agree, that we will need to do some refactoring there. Not sure though if this should be part of KIP-120 or its own JIRA. (I would prefer own JIRA.)
With InternalTopologyBuilder
being in package internals
we can do any kind of refactoring we want without impacting users. There are couple of things we need to think through carefully for this part. I also hope, that we might be able to break some code into more classes that would also simplify testing (hopefully). But this part is hard to design on a whiteboard and must be done in an exploration way "playing" with the code.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sounds good.
Merged to trunk. |