-
Notifications
You must be signed in to change notification settings - Fork 7
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
refactor TopologyBuilder #8
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -27,88 +27,60 @@ | |
|
||
public class ProcessorTopology { | ||
|
||
private Map<String, ProcessorNode> processors = new HashMap<>(); | ||
private Map<String, SourceNode> sourceTopics = new HashMap<>(); | ||
private Map<String, SinkNode> sinkTopics = new HashMap<>(); | ||
private List<ProcessorNode> processors; | ||
private Map<String, SourceNode> sourceByTopics; | ||
private Map<String, SinkNode> sinkByTopics; | ||
|
||
public ProcessorTopology(Map<String, ProcessorNode> processors, | ||
Map<String, SourceNode> sourceTopics, | ||
Map<String, SinkNode> sinkTopics) { | ||
public ProcessorTopology(List<ProcessorNode> processors, | ||
Map<String, SourceNode> sourceByTopics, | ||
Map<String, SinkNode> sinkByTopics) { | ||
this.processors = processors; | ||
this.sourceTopics = sourceTopics; | ||
this.sinkTopics = sinkTopics; | ||
this.sourceByTopics = sourceByTopics; | ||
this.sinkByTopics = sinkByTopics; | ||
} | ||
|
||
public Set<String> sourceTopics() { | ||
return sourceTopics.keySet(); | ||
return sourceByTopics.keySet(); | ||
} | ||
|
||
public Set<String> sinkTopics() { | ||
return sinkTopics.keySet(); | ||
return sinkByTopics.keySet(); | ||
} | ||
|
||
public SourceNode source(String topic) { | ||
return sourceTopics.get(topic); | ||
return sourceByTopics.get(topic); | ||
} | ||
|
||
public SinkNode sink(String topic) { | ||
return sinkTopics.get(topic); | ||
return sinkByTopics.get(topic); | ||
} | ||
|
||
public Collection<SourceNode> sources() { | ||
return sourceTopics.values(); | ||
return sourceByTopics.values(); | ||
} | ||
|
||
public Collection<SinkNode> sinks() { | ||
return sinkTopics.values(); | ||
return sinkByTopics.values(); | ||
} | ||
|
||
/** | ||
* Initialize the processors following the DAG reverse ordering | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Edit the comments accordingly. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The comment is still correct. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It is not the DAG reverse ordering but just topological ordering right? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I may be missing something. What do you mean by DAG reverse ordering exactly? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. My bad, in an older version of the code I try to do initialization recursively by starting at sink nodes, and call parent.init() first before initializing itself, I later dropped that idea but did not change the comments. Will fix it in #9. |
||
* such that parents are always initialized before children | ||
*/ | ||
@SuppressWarnings("unchecked") | ||
public void init(ProcessorContext context) { | ||
// initialize sources | ||
for (String topic : sourceTopics.keySet()) { | ||
SourceNode source = sourceTopics.get(topic); | ||
|
||
init(source, context); | ||
} | ||
} | ||
|
||
/** | ||
* Initialize the current processor node by first initializing | ||
* its parent nodes first, then the processor itself | ||
*/ | ||
@SuppressWarnings("unchecked") | ||
private void init(ProcessorNode node, ProcessorContext context) { | ||
for (ProcessorNode parentNode : (List<ProcessorNode>) node.parents()) { | ||
if (!parentNode.initialized) { | ||
init(parentNode, context); | ||
} | ||
} | ||
|
||
node.init(context); | ||
node.initialized = true; | ||
|
||
// try to initialize its children | ||
for (ProcessorNode childNode : (List<ProcessorNode>) node.children()) { | ||
if (!childNode.initialized) { | ||
init(childNode, context); | ||
} | ||
for (ProcessorNode node : processors) { | ||
node.init(context); | ||
} | ||
} | ||
|
||
public final void close() { | ||
// close the processors | ||
// TODO: do we need to follow the DAG ordering? | ||
for (ProcessorNode processorNode : processors.values()) { | ||
processorNode.close(); | ||
for (ProcessorNode node : processors) { | ||
node.close(); | ||
} | ||
|
||
processors.clear(); | ||
sourceTopics.clear(); | ||
sinkTopics.clear(); | ||
sourceByTopics.clear(); | ||
sinkByTopics.clear(); | ||
} | ||
} |
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.
topics are not added to sourceTopicNames.
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.
fixed