forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
/
IterationHeadPactTask.java
407 lines (334 loc) · 16.6 KB
/
IterationHeadPactTask.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
/***********************************************************************************************************************
* Copyright (C) 2010-2013 by the Stratosphere project (http:https://stratosphere.eu)
*
* Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
* an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
* specific language governing permissions and limitations under the License.
**********************************************************************************************************************/
package eu.stratosphere.pact.runtime.iterative.task;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import eu.stratosphere.pact.runtime.iterative.concurrent.BlockingBackChannel;
import eu.stratosphere.pact.runtime.iterative.concurrent.BlockingBackChannelBroker;
import eu.stratosphere.pact.runtime.iterative.concurrent.Broker;
import eu.stratosphere.pact.runtime.iterative.concurrent.IterationAggregatorBroker;
import eu.stratosphere.pact.runtime.iterative.concurrent.SolutionSetBroker;
import eu.stratosphere.pact.runtime.iterative.concurrent.SolutionSetUpdateBarrier;
import eu.stratosphere.pact.runtime.iterative.concurrent.SolutionSetUpdateBarrierBroker;
import eu.stratosphere.pact.runtime.iterative.concurrent.SuperstepBarrier;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import eu.stratosphere.api.common.functions.Function;
import eu.stratosphere.api.common.typeutils.TypeComparator;
import eu.stratosphere.api.common.typeutils.TypeComparatorFactory;
import eu.stratosphere.api.common.typeutils.TypePairComparator;
import eu.stratosphere.api.common.typeutils.TypePairComparatorFactory;
import eu.stratosphere.api.common.typeutils.TypeSerializer;
import eu.stratosphere.api.common.typeutils.TypeSerializerFactory;
import eu.stratosphere.core.io.IOReadableWritable;
import eu.stratosphere.core.memory.DataInputView;
import eu.stratosphere.core.memory.MemorySegment;
import eu.stratosphere.nephele.io.AbstractRecordWriter;
import eu.stratosphere.nephele.io.RecordWriter;
import eu.stratosphere.nephele.io.channels.bytebuffered.EndOfSuperstepEvent;
import eu.stratosphere.pact.runtime.hash.MutableHashTable;
import eu.stratosphere.pact.runtime.io.InputViewIterator;
import eu.stratosphere.pact.runtime.iterative.event.AllWorkersDoneEvent;
import eu.stratosphere.pact.runtime.iterative.event.TerminationEvent;
import eu.stratosphere.pact.runtime.iterative.event.WorkerDoneEvent;
import eu.stratosphere.pact.runtime.iterative.io.SerializedUpdateBuffer;
import eu.stratosphere.pact.runtime.task.RegularPactTask;
import eu.stratosphere.pact.runtime.task.util.TaskConfig;
import eu.stratosphere.pact.runtime.util.EmptyMutableObjectIterator;
import eu.stratosphere.types.Value;
import eu.stratosphere.util.Collector;
import eu.stratosphere.util.MutableObjectIterator;
/**
* The head is responsible for coordinating an iteration and can run a
* {@link eu.stratosphere.pact.runtime.task.PactDriver} inside. It will read
* the initial input and establish a {@link BlockingBackChannel} to the iteration's tail. After successfully processing
* the input, it will send {@link EndOfSuperstepEvent} events to its outputs. It must also be connected to a
* synchronization task and after each superstep, it will wait
* until it receives an {@link AllWorkersDoneEvent} from the sync, which signals that all other heads have also finished
* their iteration. Starting with
* the second iteration, the input for the head is the output of the tail, transmitted through the backchannel. Once the
* iteration is done, the head
* will send a {@link TerminationEvent} to all it's connected tasks, signaling them to shutdown.
* <p>
* Assumption on the ordering of the outputs: - The first n output gates write to channels that go to the tasks of the
* step function. - The next m output gates to to the tasks that consume the final solution. - The last output gate
* connects to the synchronization task.
*
* @param <X>
* The type of the bulk partial solution / solution set and the final output.
* @param <Y>
* The type of the feed-back data set (bulk partial solution / workset). For bulk iterations, {@code Y} is the
* same as {@code X}
*/
public class IterationHeadPactTask<X, Y, S extends Function, OT> extends AbstractIterativePactTask<S, OT> {
private static final Log log = LogFactory.getLog(IterationHeadPactTask.class);
private Collector<X> finalOutputCollector;
private List<AbstractRecordWriter<?>> finalOutputWriters;
private TypeSerializer<Y> feedbackTypeSerializer;
private TypeSerializer<X> solutionTypeSerializer;
private RecordWriter<?> toSync;
private int initialSolutionSetInput; // undefined for bulk iterations
private int feedbackDataInput; // workset or bulk partial solution
private RuntimeAggregatorRegistry aggregatorRegistry;
// --------------------------------------------------------------------------------------------
@Override
protected int getNumTaskInputs() {
// this task has an additional input in the workset case for the initial solution set
boolean isWorkset = config.getIsWorksetIteration();
return driver.getNumberOfInputs() + (isWorkset ? 1 : 0);
}
@Override
protected void initOutputs() throws Exception {
// initialize the regular outputs first (the ones into the step function).
super.initOutputs();
// at this time, the outputs to the step function are created
// add the outputs for the final solution
this.finalOutputWriters = new ArrayList<AbstractRecordWriter<?>>();
final TaskConfig finalOutConfig = this.config.getIterationHeadFinalOutputConfig();
this.finalOutputCollector = RegularPactTask.getOutputCollector(this, finalOutConfig,
this.userCodeClassLoader, this.finalOutputWriters, finalOutConfig.getNumOutputs());
// sanity check the setup
final int writersIntoStepFunction = this.eventualOutputs.size();
final int writersIntoFinalResult = this.finalOutputWriters.size();
final int syncGateIndex = this.config.getIterationHeadIndexOfSyncOutput();
if (writersIntoStepFunction + writersIntoFinalResult != syncGateIndex) {
throw new Exception("Error: Inconsistent head task setup - wrong mapping of output gates.");
}
// now, we can instantiate the sync gate
this.toSync = new RecordWriter<IOReadableWritable>(this, IOReadableWritable.class);
}
/**
* the iteration head prepares the backchannel: it allocates memory, instantiates a {@link BlockingBackChannel} and
* hands it to the iteration tail via a {@link Broker} singleton
**/
private BlockingBackChannel initBackChannel() throws Exception {
/* get the size of the memory available to the backchannel */
int backChannelMemoryPages = getMemoryManager().computeNumberOfPages(this.config.getBackChannelMemory());
/* allocate the memory available to the backchannel */
List<MemorySegment> segments = new ArrayList<MemorySegment>();
int segmentSize = getMemoryManager().getPageSize();
getMemoryManager().allocatePages(this, segments, backChannelMemoryPages);
/* instantiate the backchannel */
BlockingBackChannel backChannel = new BlockingBackChannel(new SerializedUpdateBuffer(segments, segmentSize,
getIOManager()));
/* hand the backchannel over to the iteration tail */
Broker<BlockingBackChannel> broker = BlockingBackChannelBroker.instance();
broker.handIn(brokerKey(), backChannel);
return backChannel;
}
private <BT, PT> MutableHashTable<BT, PT> initHashTable() throws Exception {
// get some memory
long hashjoinMemorySize = config.getSolutionSetMemory();
TypeSerializerFactory<BT> solutionTypeSerializerFactory = config.getSolutionSetSerializer(userCodeClassLoader);
TypeSerializerFactory<PT> probeSideSerializerFactory = config
.getSolutionSetProberSerializer(userCodeClassLoader);
TypeComparatorFactory<BT> solutionTypeComparatorFactory = config.getSolutionSetComparator(userCodeClassLoader);
TypeComparatorFactory<PT> probeSideComparatorFactory = config
.getSolutionSetProberComparator(userCodeClassLoader);
TypePairComparatorFactory<BT, PT> pairComparatorFactory = config
.getSolutionSetPairComparatorFactory(userCodeClassLoader);
TypeSerializer<BT> solutionTypeSerializer = solutionTypeSerializerFactory.getSerializer();
TypeSerializer<PT> probeSideSerializer = probeSideSerializerFactory.getSerializer();
TypeComparator<BT> solutionTypeComparator = solutionTypeComparatorFactory.createComparator();
TypeComparator<PT> probeSideComparator = probeSideComparatorFactory.createComparator();
TypePairComparator<PT, BT> pairComparator = pairComparatorFactory.createComparator21(solutionTypeComparator,
probeSideComparator);
MutableHashTable<BT, PT> hashTable = null;
List<MemorySegment> memSegments = null;
boolean success = false;
try {
int numPages = getMemoryManager().computeNumberOfPages(hashjoinMemorySize);
memSegments = getMemoryManager().allocatePages(getOwningNepheleTask(), numPages);
hashTable = new MutableHashTable<BT, PT>(solutionTypeSerializer, probeSideSerializer,
solutionTypeComparator,
probeSideComparator, pairComparator, memSegments, getIOManager());
success = true;
return hashTable;
} finally {
if (!success) {
if (hashTable != null) {
try {
hashTable.close();
} catch (Throwable t) {
log.error("Error closing the solution set hash table after unsuccessful creation.", t);
}
}
if (memSegments != null) {
try {
getMemoryManager().release(memSegments);
} catch (Throwable t) {
log.error("Error freeing memory after error during solution set hash table creation.", t);
}
}
}
}
}
private <T> void readInitialSolutionSet(MutableHashTable<X, T> solutionSet,
MutableObjectIterator<X> solutionSetInput) throws IOException {
MutableObjectIterator<T> emptyInput = EmptyMutableObjectIterator.get();
solutionSet.open(solutionSetInput, emptyInput);
}
private SuperstepBarrier initSuperstepBarrier() {
SuperstepBarrier barrier = new SuperstepBarrier(userCodeClassLoader);
this.toSync.subscribeToEvent(barrier, AllWorkersDoneEvent.class);
this.toSync.subscribeToEvent(barrier, TerminationEvent.class);
return barrier;
}
@Override
public void run() throws Exception {
final String brokerKey = brokerKey();
final int workerIndex = getEnvironment().getIndexInSubtaskGroup();
MutableHashTable<X, ?> solutionSet = null; // if workset iteration
boolean waitForSolutionSetUpdate = config.getWaitForSolutionSetUpdate();
boolean isWorksetIteration = config.getIsWorksetIteration();
try {
/* used for receiving the current iteration result from iteration tail */
BlockingBackChannel backChannel = initBackChannel();
SuperstepBarrier barrier = initSuperstepBarrier();
SolutionSetUpdateBarrier solutionSetUpdateBarrier = null;
feedbackDataInput = config.getIterationHeadPartialSolutionOrWorksetInputIndex();
feedbackTypeSerializer = getInputSerializer(feedbackDataInput);
excludeFromReset(feedbackDataInput);
if (isWorksetIteration) {
initialSolutionSetInput = config.getIterationHeadSolutionSetInputIndex();
TypeSerializerFactory<X> solutionTypeSerializerFactory = config
.getSolutionSetSerializer(userCodeClassLoader);
solutionTypeSerializer = solutionTypeSerializerFactory.getSerializer();
// setup the index for the solution set
solutionSet = initHashTable();
// read the initial solution set
// @SuppressWarnings("unchecked")
// MutableObjectIterator<X> solutionSetInput = (MutableObjectIterator<X>) createInputIterator(
// initialSolutionSetInput, inputReaders[initialSolutionSetInput], solutionTypeSerializer);
MutableObjectIterator<X> solutionSetInput = getInput(initialSolutionSetInput);
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")
TypeSerializer<X> solSer = (TypeSerializer<X>) feedbackTypeSerializer;
solutionTypeSerializer = solSer;
}
// 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()) {
log.info(formatLogString("finishing iteration [" + currentIteration() + "]"));
}
sendEventToSync(new WorkerDoneEvent(workerIndex, aggregatorRegistry.getAllAggregators()));
if (log.isInfoEnabled()) {
log.info(formatLogString("waiting for other workers in iteration [" + currentIteration() + "]"));
}
barrier.waitForOtherWorkers();
if (barrier.terminationSignaled()) {
if (log.isInfoEnabled()) {
log.info(formatLogString("head received termination request in iteration ["
+ currentIteration()
+ "]"));
}
requestTermination();
} else {
incrementIterationCounter();
String[] globalAggregateNames = barrier.getAggregatorNames();
Value[] globalAggregates = barrier.getAggregates();
aggregatorRegistry.updateGlobalAggregatesAndReset(globalAggregateNames, globalAggregates);
}
}
if (log.isInfoEnabled()) {
log.info(formatLogString("streaming out final result after [" + currentIteration() + "] iterations"));
}
if (isWorksetIteration) {
streamSolutionSetToFinalOutput(solutionSet);
} else {
streamOutFinalOutputBulk(new InputViewIterator<X>(superstepResult, this.solutionTypeSerializer));
}
} finally {
// make sure we unregister everything from the broker:
// - backchannel
// - aggregator registry
// - solution set index
IterationAggregatorBroker.instance().remove(brokerKey);
BlockingBackChannelBroker.instance().remove(brokerKey);
if (isWorksetIteration) {
SolutionSetBroker.instance().remove(brokerKey);
if (waitForSolutionSetUpdate) {
SolutionSetUpdateBarrierBroker.instance().remove(brokerKey);
}
}
if (solutionSet != null) {
solutionSet.close();
solutionSet = null;
}
}
}
private void streamOutFinalOutputBulk(MutableObjectIterator<X> results) throws IOException {
final Collector<X> out = this.finalOutputCollector;
final X record = this.solutionTypeSerializer.createInstance();
while (results.next(record)) {
out.collect(record);
}
}
private void streamSolutionSetToFinalOutput(MutableHashTable<X, ?> hashTable) throws IOException,
InterruptedException {
final MutableObjectIterator<X> results = hashTable.getPartitionEntryIterator();
final Collector<X> output = this.finalOutputCollector;
final X record = solutionTypeSerializer.createInstance();
while (results.next(record)) {
output.collect(record);
}
}
private void feedBackSuperstepResult(DataInputView superstepResult) {
this.inputs[this.feedbackDataInput] =
new InputViewIterator<Y>(superstepResult, this.feedbackTypeSerializer);
}
private void sendEndOfSuperstepToAllIterationOutputs() throws IOException, InterruptedException {
if (log.isDebugEnabled()) {
log.debug(formatLogString("Sending end-of-superstep to all iteration outputs."));
}
for (int outputIndex = 0; outputIndex < this.eventualOutputs.size(); outputIndex++) {
this.eventualOutputs.get(outputIndex).sendEndOfSuperstep();
}
}
private void sendEventToSync(WorkerDoneEvent event) throws IOException, InterruptedException {
if (log.isInfoEnabled()) {
log.info(formatLogString("sending " + WorkerDoneEvent.class.getSimpleName() + " to sync"));
}
this.toSync.publishEvent(event);
}
}