forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
/
MessagePassingIteration.java
493 lines (403 loc) · 19.2 KB
/
MessagePassingIteration.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
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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://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 org.apache.flink.graph.spargelnew;
import java.util.Iterator;
import java.util.Map;
import org.apache.flink.api.common.aggregators.Aggregator;
import org.apache.flink.api.common.functions.FlatJoinFunction;
import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.common.functions.RichGroupReduceFunction;
import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields;
import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsFirst;
import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFieldsSecond;
import org.apache.flink.api.java.operators.DeltaIteration;
import org.apache.flink.api.common.functions.RichCoGroupFunction;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.operators.CustomUnaryOperation;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.Either;
import org.apache.flink.api.java.typeutils.EitherTypeInfo;
import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.graph.Edge;
import org.apache.flink.graph.Graph;
import org.apache.flink.graph.Vertex;
import org.apache.flink.types.NullValue;
import org.apache.flink.util.Collector;
import com.google.common.base.Preconditions;
/**
* This class represents iterative graph computations, programmed in a vertex-centric perspective.
* It is a special case of <i>Bulk Synchronous Parallel</i> computation. The paradigm has also been
* implemented by Google's <i>Pregel</i> system and by <i>Apache Giraph</i>.
* <p>
* Vertex centric algorithms operate on graphs, which are defined through vertices and edges. The
* algorithms send messages along the edges and update the state of vertices based on
* the old state and the incoming messages. All vertices have an initial state.
* The computation terminates once no vertex receives any message anymore.
* Additionally, a maximum number of iterations (supersteps) may be specified.
* <p>
* The computation is here represented by one function:
* <ul>
* <li>The {@link ComputeFunction} receives incoming messages, may update the state for
* the vertex, and sends messages along the edges of the vertex.
* </li>
* </ul>
* <p>
*
* Vertex-centric graph iterations are are run by calling
* {@link Graph#runVertexCentricIteration(VertexUpdateFunction, MessagingFunction, int)}.
*
* @param <K> The type of the vertex key (the vertex identifier).
* @param <VV> The type of the vertex value (the state of the vertex).
* @param <Message> The type of the message sent between vertices along the edges.
* @param <EV> The type of the values that are associated with the edges.
*/
public class MessagePassingIteration<K, VV, EV, Message>
implements CustomUnaryOperation<Vertex<K, VV>, Vertex<K, VV>>
{
private final ComputeFunction<K, VV, EV, Message> computeFunction;
private final MessageCombiner<K, Message> combineFunction;
private final DataSet<Edge<K, EV>> edgesWithValue;
private final int maximumNumberOfIterations;
private final TypeInformation<Message> messageType;
private DataSet<Vertex<K, VV>> initialVertices;
private VertexCentricConfiguration configuration;
// ----------------------------------------------------------------------------------
private MessagePassingIteration(ComputeFunction<K, VV, EV, Message> cf,
DataSet<Edge<K, EV>> edgesWithValue, MessageCombiner<K, Message> mc,
int maximumNumberOfIterations)
{
Preconditions.checkNotNull(cf);
Preconditions.checkNotNull(edgesWithValue);
Preconditions.checkArgument(maximumNumberOfIterations > 0,
"The maximum number of iterations must be at least one.");
this.computeFunction = cf;
this.edgesWithValue = edgesWithValue;
this.combineFunction = mc;
this.maximumNumberOfIterations = maximumNumberOfIterations;
this.messageType = getMessageType(cf);
}
private TypeInformation<Message> getMessageType(ComputeFunction<K, VV, EV, Message> cf) {
return TypeExtractor.createTypeInfo(ComputeFunction.class, cf.getClass(), 3, null, null);
}
// --------------------------------------------------------------------------------------------
// Custom Operator behavior
// --------------------------------------------------------------------------------------------
/**
* Sets the input data set for this operator. In the case of this operator this input data set represents
* the set of vertices with their initial state.
*
* @param inputData The input data set, which in the case of this operator represents the set of
* vertices with their initial state.
*
* @see org.apache.flink.api.java.operators.CustomUnaryOperation#setInput(org.apache.flink.api.java.DataSet)
*/
@Override
public void setInput(DataSet<Vertex<K, VV>> inputData) {
this.initialVertices = inputData;
}
/**
* Creates the operator that represents this vertex-centric graph computation.
*
* @return The operator that represents this vertex-centric graph computation.
*/
@Override
public DataSet<Vertex<K, VV>> createResult() {
if (this.initialVertices == null) {
throw new IllegalStateException("The input data set has not been set.");
}
// prepare the type information
TypeInformation<K> keyType = ((TupleTypeInfo<?>) initialVertices.getType()).getTypeAt(0);
TypeInformation<Tuple2<K, Message>> messageTypeInfo =
new TupleTypeInfo<Tuple2<K, Message>>(keyType, messageType);
TypeInformation<Vertex<K, VV>> vertexType = initialVertices.getType();
TypeInformation<Either<Vertex<K, VV>, Tuple2<K, Message>>> intermediateTypeInfo =
new EitherTypeInfo<Vertex<K, VV>, Tuple2<K, Message>>(vertexType, messageTypeInfo);
TypeInformation<Either<NullValue, Message>> nullableMsgTypeInfo =
new EitherTypeInfo<NullValue, Message>(TypeExtractor.getForClass(NullValue.class), messageType);
TypeInformation<Tuple2<K, Either<NullValue, Message>>> workSetTypeInfo =
new TupleTypeInfo<Tuple2<K, Either<NullValue, Message>>>(keyType, nullableMsgTypeInfo);
DataSet<Tuple2<K, Either<NullValue, Message>>> initialWorkSet = initialVertices.map(
new InitializeWorkSet<K, VV, Message>()).returns(workSetTypeInfo);
final DeltaIteration<Vertex<K, VV>, Tuple2<K, Either<NullValue, Message>>> iteration =
initialVertices.iterateDelta(initialWorkSet, this.maximumNumberOfIterations, 0);
setUpIteration(iteration);
// join with the current state to get vertex values
DataSet<Tuple2<Vertex<K, VV>, Either<NullValue, Message>>> verticesWithMsgs =
iteration.getSolutionSet().join(iteration.getWorkset())
.where(0).equalTo(0)
.with(new AppendVertexState<K, VV, Message>())
.returns(new TupleTypeInfo<Tuple2<Vertex<K, VV>, Either<NullValue, Message>>>(
vertexType, nullableMsgTypeInfo));
VertexComputeUdf<K, VV, EV, Message> vertexUdf =
new VertexComputeUdf<K, VV, EV, Message>(computeFunction, intermediateTypeInfo);
DataSet<Either<Vertex<K, VV>, Tuple2<K, Message>>> superstepComputation =
verticesWithMsgs.coGroup(edgesWithValue)
.where("f0.f0").equalTo(0)
.with(vertexUdf);
// compute the solution set delta
DataSet<Vertex<K, VV>> solutionSetDelta = superstepComputation.flatMap(
new ProjectNewVertexValue<K, VV, Message>()).returns(vertexType);
// compute the inbox of each vertex for the next superstep (new workset)
DataSet<Tuple2<K, Either<NullValue, Message>>> allMessages = superstepComputation.flatMap(
new ProjectMessages<K, VV, Message>()).returns(workSetTypeInfo);
DataSet<Tuple2<K, Either<NullValue, Message>>> newWorkSet = allMessages;
// check if a combiner has been provided
if (combineFunction != null) {
MessageCombinerUdf<K, Message> combinerUdf =
new MessageCombinerUdf<K, Message>(combineFunction, workSetTypeInfo);
DataSet<Tuple2<K, Either<NullValue, Message>>> combinedMessages = allMessages
.groupBy(0).reduceGroup(combinerUdf)
.setCombinable(true);
newWorkSet = combinedMessages;
}
// configureComputeFunction(superstepComputation);
return iteration.closeWith(solutionSetDelta, newWorkSet);
}
/**
* Creates a new vertex-centric iteration operator for graphs where the edges are associated with a value (such as
* a weight or distance).
*
* @param edgesWithValue The data set containing edges.
* @param uf The function that updates the state of the vertices from the incoming messages.
* @param mf The function that turns changed vertex states into messages along the edges.
*
* @param <K> The type of the vertex key (the vertex identifier).
* @param <VV> The type of the vertex value (the state of the vertex).
* @param <Message> The type of the message sent between vertices along the edges.
* @param <EV> The type of the values that are associated with the edges.
*
* @return An in stance of the vertex-centric graph computation operator.
*/
public static final <K, VV, EV, Message> MessagePassingIteration<K, VV, EV, Message> withEdges(
DataSet<Edge<K, EV>> edgesWithValue, ComputeFunction<K, VV, EV, Message> cf,
int maximumNumberOfIterations) {
return new MessagePassingIteration<K, VV, EV, Message>(cf, edgesWithValue, null,
maximumNumberOfIterations);
}
/**
* Creates a new vertex-centric iteration operator for graphs where the edges are associated with a value (such as
* a weight or distance).
*
* @param edgesWithValue The data set containing edges.
* @param uf The function that updates the state of the vertices from the incoming messages.
* @param mf The function that turns changed vertex states into messages along the edges.
* @param mc The function that combines messages sent to a vertex during a superstep.
*
* @param <K> The type of the vertex key (the vertex identifier).
* @param <VV> The type of the vertex value (the state of the vertex).
* @param <Message> The type of the message sent between vertices along the edges.
* @param <EV> The type of the values that are associated with the edges.
*
* @return An in stance of the vertex-centric graph computation operator.
*/
public static final <K, VV, EV, Message> MessagePassingIteration<K, VV, EV, Message> withEdges(
DataSet<Edge<K, EV>> edgesWithValue, ComputeFunction<K, VV, EV, Message> cf,
MessageCombiner<K, Message> mc, int maximumNumberOfIterations) {
return new MessagePassingIteration<K, VV, EV, Message>(cf, edgesWithValue, mc,
maximumNumberOfIterations);
}
/**
* Configures this vertex-centric iteration with the provided parameters.
*
* @param parameters the configuration parameters
*/
public void configure(VertexCentricConfiguration parameters) {
this.configuration = parameters;
}
/**
* @return the configuration parameters of this vertex-centric iteration
*/
public VertexCentricConfiguration getIterationConfiguration() {
return this.configuration;
}
// --------------------------------------------------------------------------------------------
// Wrapping UDFs
// --------------------------------------------------------------------------------------------
@SuppressWarnings("serial")
private static class InitializeWorkSet<K, VV, Message> implements
MapFunction<Vertex<K, VV>, Tuple2<K, Either<NullValue, Message>>> {
private Tuple2<K, Either<NullValue, Message>> outTuple = new Tuple2<K, Either<NullValue, Message>>();
private Either<NullValue, Message> nullMessage = Either.left(NullValue.getInstance());
public Tuple2<K, Either<NullValue, Message>> map(Vertex<K, VV> vertex) {
outTuple.setField(vertex.getId(), 0);
outTuple.setField(nullMessage, 1);
return outTuple;
}
}
@SuppressWarnings("serial")
private static class VertexComputeUdf<K, VV, EV, Message> extends RichCoGroupFunction<
Tuple2<Vertex<K, VV>, Either<NullValue, Message>>, Edge<K, EV>,
Either<Vertex<K, VV>, Tuple2<K, Message>>>
implements ResultTypeQueryable<Either<Vertex<K, VV>, Tuple2<K, Message>>> {
final ComputeFunction<K, VV, EV, Message> computeFunction;
private transient TypeInformation<Either<Vertex<K, VV>, Tuple2<K, Message>>> resultType;
private VertexComputeUdf(ComputeFunction<K, VV, EV, Message> compute,
TypeInformation<Either<Vertex<K, VV>, Tuple2<K, Message>>> typeInfo) {
this.computeFunction = compute;
this.resultType = typeInfo;
}
@Override
public TypeInformation<Either<Vertex<K, VV>, Tuple2<K, Message>>> getProducedType() {
return this.resultType;
}
@Override
public void open(Configuration parameters) throws Exception {
if (getIterationRuntimeContext().getSuperstepNumber() == 1) {
this.computeFunction.init(getIterationRuntimeContext());
}
this.computeFunction.preSuperstep();
}
@Override
public void close() throws Exception {
this.computeFunction.postSuperstep();
}
@Override
public void coGroup(
Iterable<Tuple2<Vertex<K, VV>, Either<NullValue, Message>>> messages,
Iterable<Edge<K, EV>> edgesIterator,
Collector<Either<Vertex<K, VV>, Tuple2<K, Message>>> out) throws Exception {
final Iterator<Tuple2<Vertex<K, VV>, Either<NullValue, Message>>> vertexIter =
messages.iterator();
if (vertexIter.hasNext()) {
final Tuple2<Vertex<K, VV>, Either<NullValue, Message>> first = vertexIter.next();
final Vertex<K, VV> vertexState = first.f0;
final MessageIterator<Message> messageIter = new MessageIterator<Message>();
if (getIterationRuntimeContext().getSuperstepNumber() == 1) {
}
else {
messageIter.setFirst(first.f1.right());
@SuppressWarnings("unchecked")
Iterator<Tuple2<?, Either<NullValue, Message>>> downcastIter =
(Iterator<Tuple2<?, Either<NullValue, Message>>>) (Iterator<?>) vertexIter;
messageIter.setSource(downcastIter);
}
computeFunction.set(vertexState.getId(), edgesIterator.iterator(), out);
computeFunction.compute(vertexState, messageIter);
}
}
}
@SuppressWarnings("serial")
@ForwardedFields("f0")
public static class MessageCombinerUdf<K, Message> extends RichGroupReduceFunction<
Tuple2<K, Either<NullValue, Message>>, Tuple2<K, Either<NullValue, Message>>>
implements ResultTypeQueryable<Tuple2<K, Either<NullValue, Message>>> {
final MessageCombiner<K, Message> combinerFunction;
private transient TypeInformation<Tuple2<K, Either<NullValue, Message>>> resultType;
private MessageCombinerUdf(MessageCombiner<K, Message> combineFunction,
TypeInformation<Tuple2<K, Either<NullValue, Message>>> messageTypeInfo) {
this.combinerFunction = combineFunction;
this.resultType = messageTypeInfo;
}
@Override
public TypeInformation<Tuple2<K, Either<NullValue, Message>>> getProducedType() {
return resultType;
}
@Override
public void reduce(Iterable<Tuple2<K, Either<NullValue, Message>>> messages,
Collector<Tuple2<K, Either<NullValue, Message>>> out) throws Exception {
final Iterator<Tuple2<K, Either<NullValue, Message>>> messageIterator = messages.iterator();
if (messageIterator.hasNext()) {
final Tuple2<K, Either<NullValue, Message>> first = messageIterator.next();
final K vertexID = first.f0;
final MessageIterator<Message> messageIter = new MessageIterator<Message>();
messageIter.setFirst(first.f1.right());
@SuppressWarnings("unchecked")
Iterator<Tuple2<?, Either<NullValue, Message>>> downcastIter =
(Iterator<Tuple2<?, Either<NullValue, Message>>>) (Iterator<?>) messageIterator;
messageIter.setSource(downcastIter);
combinerFunction.set(vertexID, out);
combinerFunction.combineMessages(messageIter);
}
}
@Override
public void combine(Iterable<Tuple2<K, Either<NullValue, Message>>> values,
Collector<Tuple2<K, Either<NullValue, Message>>> out) throws Exception {
this.reduce(values, out);
}
}
// --------------------------------------------------------------------------------------------
// UTIL methods
// --------------------------------------------------------------------------------------------
/**
* Helper method which sets up an iteration with the given vertex value
*
* @param iteration
*/
private void setUpIteration(DeltaIteration<?, ?> iteration) {
// set up the iteration operator
if (this.configuration != null) {
iteration.name(this.configuration.getName("Vertex-centric iteration (" + computeFunction + ")"));
iteration.parallelism(this.configuration.getParallelism());
iteration.setSolutionSetUnManaged(this.configuration.isSolutionSetUnmanagedMemory());
// register all aggregators
for (Map.Entry<String, Aggregator<?>> entry : this.configuration.getAggregators().entrySet()) {
iteration.registerAggregator(entry.getKey(), entry.getValue());
}
}
else {
// no configuration provided; set default name
iteration.name("Vertex-centric iteration (" + computeFunction + ")");
}
}
@SuppressWarnings("serial")
@ForwardedFieldsFirst("*->f0")
@ForwardedFieldsSecond("f1->f1")
private static final class AppendVertexState<K, VV, Message> implements
FlatJoinFunction<Vertex<K, VV>, Tuple2<K, Either<NullValue, Message>>,
Tuple2<Vertex<K, VV>, Either<NullValue, Message>>> {
private Tuple2<Vertex<K, VV>, Either<NullValue, Message>> outTuple =
new Tuple2<Vertex<K, VV>, Either<NullValue, Message>>();
public void join(Vertex<K, VV> vertex, Tuple2<K, Either<NullValue, Message>> message,
Collector<Tuple2<Vertex<K, VV>, Either<NullValue, Message>>> out) {
outTuple.setField(vertex, 0);
outTuple.setField(message.f1, 1);
out.collect(outTuple);
}
}
@SuppressWarnings("serial")
// @ForwardedFields("f0->*")
private static final class ProjectNewVertexValue<K, VV, Message> implements
FlatMapFunction<Either<Vertex<K, VV>, Tuple2<K, Message>>, Vertex<K, VV>> {
public void flatMap(Either<Vertex<K, VV>, Tuple2<K, Message>> value,
Collector<Vertex<K, VV>> out) {
if (value.isLeft()) {
out.collect(value.left());
}
}
}
@SuppressWarnings("serial")
// @ForwardedFields("f1->*")
private static final class ProjectMessages<K, VV, Message> implements
FlatMapFunction<Either<Vertex<K, VV>, Tuple2<K, Message>>, Tuple2<K, Either<NullValue, Message>>> {
private Tuple2<K, Either<NullValue, Message>> outTuple = new Tuple2<K, Either<NullValue, Message>>();
public void flatMap(Either<Vertex<K, VV>, Tuple2<K, Message>> value,
Collector<Tuple2<K, Either<NullValue, Message>>> out) {
if (value.isRight()) {
Tuple2<K, Message> message = value.right();
outTuple.setField(message.f0, 0);
outTuple.setField(Either.right(message.f1), 1);
out.collect(outTuple);
}
}
}
}