MINOR: remove unnecessary store info from TopologyBuilder#2338
MINOR: remove unnecessary store info from TopologyBuilder#2338guozhangwang wants to merge 8 commits intoapache:trunkfrom
Conversation
|
ping @dguy @mjsax @enothereska for reviews. |
|
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): |
|
Refer to this link for build results (access rights to CI server needed): |
| } finally { | ||
| currNode = current; | ||
|
|
||
| for (StateStore stateStore : context.allStateStores().values()) { |
There was a problem hiding this comment.
Why do we not need currNode to map to the store we are flushing any more? I don't see any change in StateStore#flush() -- or did we not need this in the first place and this is just a cleanup of unnecessary code?
There was a problem hiding this comment.
This is a leftover work that was not introduced in this PR: before we use the topology.storeToProcessorNodeMap() to remember the linked processor node of the state store, so we can set it while flushing and possibly forwarding to children processor nodes. When we add KIP-63 we store that information in DirtyEntry so we do not need this any more, but forgot to remove it.
This cleanup is mainly for removing those not-needed-anymore maps from topology builder
| final KStreamBuilder builder = new KStreamBuilder(); | ||
| builder.stream("topic").groupByKey().count("my-store"); | ||
| final ProcessorTopology topology = builder.build(0); | ||
| final ProcessorTopology topology = builder.setApplicationId(applicationId).build(0); |
There was a problem hiding this comment.
Why was a missing application ID not a problem before the change? I don't see the connection.
There was a problem hiding this comment.
Explained as above. We are now solely depending on the topology builder to give us the information about which state stores should be changelog enabled and what are their changelog topic names. Before that we have a separate loggingEnabled parameter in register to indicate that.
| put(storeName1, storeChangelogTopicName1); | ||
| put(storeName2, storeChangelogTopicName2); | ||
| } | ||
| }); |
There was a problem hiding this comment.
I don't get this change? (same blow). Why do we to provide this now?
There was a problem hiding this comment.
We are now passing the map from store names to its changlog topics as the last parameter into ProcessorStateManager, since we can actually gather that information beforehand in the topology builder already (any stores that are not changelog enabled won't have an entry in the map). So in ProcessorStateManager it only needs to access this map to decide whether we need to get a changelog for the state store and what would be the changelog name.
In these tests since we are adding two stores persistentXX and nonPersistentXX which are all changelogged, we need to pass in this map.
There was a problem hiding this comment.
Yes. But in the original version of this test we pass in two empty maps (sourceStoreToSourceTopics and storeToProcessorNodeMap) -- so I am wondering why we do need to provide this information now but not before (or where did this information come from before). Still confused.
| Set<TopicPartition> sourcePartitions = Utils.mkSet(new TopicPartition(storeTopicName3, 1)); | ||
|
|
||
| ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, taskId, sourcePartitions, restoreConsumer, true, stateDirectory, null, Collections.<StateStore, ProcessorNode>emptyMap()); // standby | ||
| ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, taskId, sourcePartitions, restoreConsumer, true, stateDirectory, storeToChangelogTopic); // standby |
There was a problem hiding this comment.
The test worked with an empty map before -- why do we need to provide the mapping now? (same below)
| final KTableImpl<K, ?, V> kTable = new KTableImpl<>(this, name, processorSupplier, Collections.singleton(source), storeName); | ||
|
|
||
| // only materialize the KTable into a state store if the storeName is not null | ||
| if (storeName != null) { |
There was a problem hiding this comment.
My bad, this is not committed.
…into KMinor-refactor-state-to-changelogtopic
|
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): |
|
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): |
|
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): |
|
retest this please. |
|
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): |
|
Refer to this link for build results (access rights to CI server needed): |
|
retest this please |
|
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): |
|
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): |
ewencp
left a comment
There was a problem hiding this comment.
@guozhangwang One trivial typo, but LGTM
| private final Map<String, Set<String>> stateStoreNameToSourceTopics = new HashMap<>(); | ||
|
|
||
| // map from state store names that are directly associated with source processors to their subscribed topics, | ||
| // map from state store names to ths state store's corresponding changelog topic if possible, |
|
Merged to trunk. |
|
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): |
|
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): |
This PR is extracted from apache#2333 as an incremental fix to ease the reviewing: 1. Removed `storeToProcessorNodeMap` from ProcessorTopology since it was previously used to set the context current record, and can now be replaced with the dirty entry in the named cache. 2. Replaced `sourceStoreToSourceTopic` from ProcessorTopology with `storeToChangelogTopic` map, which includes the corresponding changelog topic name for all stores that are changelog enabled. 3. Modified `ProcessorStateManager` to rely on `sourceStoreToSourceTopic` when retrieving the changelog topic; this makes the second parameter `loggingEnabled` in `register` not needed any more, and we can deprecate the old API with a new one. 4. Also fixed a minor issue in `KStreamBuilder`: if the storeName is not provided in the `table(..)` function, do not create the underlying materialized store. Modified the unit tests to cover this case. 5. Fixed a bunch of other unit tests failures that are exposed by this refactoring, in which we are not setting the applicationId correctly when constructing the mocking processor topology. Author: Guozhang Wang <wangguoz@gmail.com> Reviewers: Damian Guy, Matthias J. Sax, Ewen Cheslack-Postava Closes apache#2338 from guozhangwang/KMinor-refactor-state-to-changelogtopic
This PR is extracted from #2333 as an incremental fix to ease the reviewing:
Removed
storeToProcessorNodeMapfrom ProcessorTopology since it was previously used to set the context current record, and can now be replaced with the dirty entry in the named cache.Replaced
sourceStoreToSourceTopicfrom ProcessorTopology withstoreToChangelogTopicmap, which includes the corresponding changelog topic name for all stores that are changelog enabled.Modified
ProcessorStateManagerto rely onsourceStoreToSourceTopicwhen retrieving the changelog topic; this makes the second parameterloggingEnabledinregisternot needed any more, and we can deprecate the old API with a new one.Also fixed a minor issue in
KStreamBuilder: if the storeName is not provided in thetable(..)function, do not create the underlying materialized store. Modified the unit tests to cover this case.Fixed a bunch of other unit tests failures that are exposed by this refactoring, in which we are not setting the applicationId correctly when constructing the mocking processor topology.