this.sources.add((SourcePlanNode) visitable);
}
else if (visitable instanceof BulkPartialSolutionPlanNode) {
// tell the partial solution about the iteration node that contains it
final BulkPartialSolutionPlanNode pspn = (BulkPartialSolutionPlanNode) visitable;
final IterationPlanNode iteration = this.stackOfIterationNodes.peekLast();
// sanity check!
if (iteration == null || !(iteration instanceof BulkIterationPlanNode)) {
throw new CompilerException("Bug: Error finalizing the plan. " +
"Cannot associate the node for a partial solutions with its containing iteration.");
}
pspn.setContainingIterationNode((BulkIterationPlanNode) iteration);
}
else if (visitable instanceof WorksetPlanNode) {
// tell the partial solution about the iteration node that contains it
final WorksetPlanNode wspn = (WorksetPlanNode) visitable;
final IterationPlanNode iteration = this.stackOfIterationNodes.peekLast();
// sanity check!
if (iteration == null || !(iteration instanceof WorksetIterationPlanNode)) {
throw new CompilerException("Bug: Error finalizing the plan. " +
"Cannot associate the node for a partial solutions with its containing iteration.");
}
wspn.setContainingIterationNode((WorksetIterationPlanNode) iteration);
}
else if (visitable instanceof SolutionSetPlanNode) {
// tell the partial solution about the iteration node that contains it
final SolutionSetPlanNode sspn = (SolutionSetPlanNode) visitable;
final IterationPlanNode iteration = this.stackOfIterationNodes.peekLast();
// sanity check!
if (iteration == null || !(iteration instanceof WorksetIterationPlanNode)) {
throw new CompilerException("Bug: Error finalizing the plan. " +
"Cannot associate the node for a partial solutions with its containing iteration.");
}
sspn.setContainingIterationNode((WorksetIterationPlanNode) iteration);
}
// double-connect the connections. previously, only parents knew their children, because
// one child candidate could have been referenced by multiple parents.
for (Channel conn : visitable.getInputs()) {
conn.setTarget(visitable);
conn.getSource().addOutgoingChannel(conn);
}
for (Channel c : visitable.getBroadcastInputs()) {
c.setTarget(visitable);
c.getSource().addOutgoingChannel(c);
}
// count the memory consumption
this.memoryConsumerWeights += visitable.getMemoryConsumerWeight();
for (Channel c : visitable.getInputs()) {
if (c.getLocalStrategy().dams()) {
this.memoryConsumerWeights++;
}
if (c.getTempMode() != TempMode.NONE) {
this.memoryConsumerWeights++;
}
}
for (Channel c : visitable.getBroadcastInputs()) {
if (c.getLocalStrategy().dams()) {
this.memoryConsumerWeights++;
}
if (c.getTempMode() != TempMode.NONE) {
this.memoryConsumerWeights++;
}
}
// pass the visitor to the iteraton's step function
if (visitable instanceof IterationPlanNode) {
// push the iteration node onto the stack
final IterationPlanNode iterNode = (IterationPlanNode) visitable;
this.stackOfIterationNodes.addLast(iterNode);
// recurse
((IterationPlanNode) visitable).acceptForStepFunction(this);