try {
/* used for receiving the current iteration result from iteration tail */
BlockingBackChannel backChannel = initBackChannel();
SuperstepBarrier barrier = initSuperstepBarrier();
SolutionSetUpdateBarrier solutionSetUpdateBarrier = null;
feedbackDataInput = config.getIterationHeadPartialSolutionOrWorksetInputIndex();
feedbackTypeSerializer = this.<Y>getInputSerializer(feedbackDataInput);
excludeFromReset(feedbackDataInput);
if (isWorksetIteration) {
initialSolutionSetInput = config.getIterationHeadSolutionSetInputIndex();
TypeSerializerFactory<X> solutionTypeSerializerFactory = config.getSolutionSetSerializer(userCodeClassLoader);
solutionTypeSerializer = solutionTypeSerializerFactory;
// setup the index for the solution set
//solutionSet = initHashTable();
solutionSet = initCompactingHashTable();
// read the initial solution set
@SuppressWarnings("unchecked")
MutableObjectIterator<X> solutionSetInput = (MutableObjectIterator<X>) createInputIterator(inputReaders[initialSolutionSetInput], solutionTypeSerializer);
readInitialSolutionSet(solutionSet, solutionSetInput);
SolutionSetBroker.instance().handIn(brokerKey, solutionSet);
if (waitForSolutionSetUpdate) {
solutionSetUpdateBarrier = new SolutionSetUpdateBarrier();
SolutionSetUpdateBarrierBroker.instance().handIn(brokerKey, solutionSetUpdateBarrier);
}
} else {
// bulk iteration case
initialSolutionSetInput = -1;
@SuppressWarnings("unchecked")
TypeSerializerFactory<X> solSer = (TypeSerializerFactory<X>) feedbackTypeSerializer;
solutionTypeSerializer = solSer;
// = termination Criterion tail
if (waitForSolutionSetUpdate) {
solutionSetUpdateBarrier = new SolutionSetUpdateBarrier();
SolutionSetUpdateBarrierBroker.instance().handIn(brokerKey, solutionSetUpdateBarrier);
}
}
// instantiate all aggregators and register them at the iteration global registry
aggregatorRegistry = new RuntimeAggregatorRegistry(config.getIterationAggregators());
IterationAggregatorBroker.instance().handIn(brokerKey, aggregatorRegistry);
DataInputView superstepResult = null;
while (this.running && !terminationRequested()) {
if (log.isInfoEnabled()) {
log.info(formatLogString("starting iteration [" + currentIteration() + "]"));
}
barrier.setup();
if (waitForSolutionSetUpdate) {
solutionSetUpdateBarrier.setup();
}
if (!inFirstIteration()) {
feedBackSuperstepResult(superstepResult);
}
super.run();
// signal to connected tasks that we are done with the superstep
sendEndOfSuperstepToAllIterationOutputs();
if (waitForSolutionSetUpdate) {
solutionSetUpdateBarrier.waitForSolutionSetUpdate();
}
// blocking call to wait for the result
superstepResult = backChannel.getReadEndAfterSuperstepEnded();
if (log.isInfoEnabled()) {