-
Notifications
You must be signed in to change notification settings - Fork 64
/
PipelineTranslator.java
439 lines (400 loc) · 20.7 KB
/
PipelineTranslator.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
/*
* 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.nemo.compiler.frontend.beam;
import com.google.common.collect.Iterables;
import org.apache.beam.runners.core.SystemReduceFn;
import org.apache.beam.runners.core.construction.ParDoTranslation;
import org.apache.beam.runners.core.construction.TransformInputs;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.runners.AppliedPTransform;
import org.apache.beam.sdk.runners.TransformHierarchy;
import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
import org.apache.nemo.common.ir.edge.IREdge;
import org.apache.nemo.common.ir.edge.executionproperty.CommunicationPatternProperty;
import org.apache.nemo.common.ir.vertex.IRVertex;
import org.apache.nemo.common.ir.vertex.OperatorVertex;
import org.apache.nemo.common.ir.vertex.transform.Transform;
import org.apache.nemo.compiler.frontend.beam.source.BeamBoundedSourceVertex;
import org.apache.nemo.compiler.frontend.beam.source.BeamUnboundedSourceVertex;
import org.apache.nemo.compiler.frontend.beam.transform.*;
import org.apache.beam.sdk.coders.*;
import org.apache.beam.sdk.io.Read;
import org.apache.beam.sdk.transforms.*;
import org.apache.beam.sdk.transforms.windowing.Window;
import org.apache.beam.sdk.transforms.windowing.WindowFn;
import org.apache.beam.sdk.values.*;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.lang.annotation.*;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.util.*;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
/**
* A collection of translators for the Beam PTransforms.
*/
final class PipelineTranslator {
public static final PipelineTranslator INSTANCE = new PipelineTranslator();
private static final Logger LOG = LoggerFactory.getLogger(PipelineTranslator.class.getName());
private final Map<Class<? extends PTransform>, Method> primitiveTransformToTranslator = new HashMap<>();
private final Map<Class<? extends PTransform>, Method> compositeTransformToTranslator = new HashMap<>();
/**
* Creates the translator, while building a map between {@link PTransform}s and the corresponding translators.
*/
private PipelineTranslator() {
for (final Method translator : getClass().getDeclaredMethods()) {
final PrimitiveTransformTranslator primitive = translator.getAnnotation(PrimitiveTransformTranslator.class);
final CompositeTransformTranslator composite = translator.getAnnotation(CompositeTransformTranslator.class);
if (primitive != null) {
for (final Class<? extends PTransform> transform : primitive.value()) {
if (primitiveTransformToTranslator.containsKey(transform)) {
throw new RuntimeException(String.format("Translator for primitive transform %s is"
+ "already registered: %s", transform, primitiveTransformToTranslator.get(transform)));
}
primitiveTransformToTranslator.put(transform, translator);
}
}
if (composite != null) {
for (final Class<? extends PTransform> transform : composite.value()) {
if (compositeTransformToTranslator.containsKey(transform)) {
throw new RuntimeException(String.format("Translator for composite transform %s is"
+ "already registered: %s", transform, compositeTransformToTranslator.get(transform)));
}
compositeTransformToTranslator.put(transform, translator);
}
}
}
}
void translatePrimitive(final PipelineTranslationContext context,
final TransformHierarchy.Node primitive) {
final PTransform<?, ?> transform = primitive.getTransform();
Class<?> clazz = transform.getClass();
final Method translator = primitiveTransformToTranslator.get(clazz);
if (translator == null) {
throw new UnsupportedOperationException(
String.format("Primitive transform %s is not supported", transform.getClass().getCanonicalName()));
} else {
try {
translator.setAccessible(true);
translator.invoke(null, context, primitive, transform);
} catch (final IllegalAccessException e) {
throw new RuntimeException(e);
} catch (final InvocationTargetException | RuntimeException e) {
throw new RuntimeException(String.format(
"Translator %s have failed to translate %s", translator, transform), e);
}
}
}
/**
* @param context context.
* @param composite transform.
* @return behavior.
*/
Pipeline.PipelineVisitor.CompositeBehavior translateComposite(final PipelineTranslationContext context,
final TransformHierarchy.Node composite) {
final PTransform<?, ?> transform = composite.getTransform();
if (transform == null) {
// root beam node
return Pipeline.PipelineVisitor.CompositeBehavior.ENTER_TRANSFORM;
}
Class<?> clazz = transform.getClass();
final Method translator = compositeTransformToTranslator.get(clazz);
if (translator == null) {
return Pipeline.PipelineVisitor.CompositeBehavior.ENTER_TRANSFORM;
} else {
try {
translator.setAccessible(true);
return (Pipeline.PipelineVisitor.CompositeBehavior) translator.invoke(null, context, composite, transform);
} catch (final IllegalAccessException e) {
throw new RuntimeException(e);
} catch (final InvocationTargetException | RuntimeException e) {
throw new RuntimeException(String.format(
"Translator %s have failed to translate %s", translator, transform), e);
}
}
}
/**
* Annotates translator for PrimitiveTransform.
*/
@Target(ElementType.METHOD)
@Retention(RetentionPolicy.RUNTIME)
private @interface PrimitiveTransformTranslator {
Class<? extends PTransform>[] value();
}
/**
* Annotates translator for CompositeTransform.
*/
@Target(ElementType.METHOD)
@Retention(RetentionPolicy.RUNTIME)
private @interface CompositeTransformTranslator {
Class<? extends PTransform>[] value();
}
////////////////////////////////////////////////////////////////////////////////////////////////////////
/////////////////////// PRIMITIVE TRANSFORMS
@PrimitiveTransformTranslator(Read.Unbounded.class)
private static void unboundedReadTranslator(final PipelineTranslationContext ctx,
final TransformHierarchy.Node beamNode,
final Read.Unbounded<?> transform) {
final IRVertex vertex = new BeamUnboundedSourceVertex<>(transform.getSource());
ctx.addVertex(vertex);
beamNode.getInputs().values().forEach(input -> ctx.addEdgeTo(vertex, input));
beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, vertex, output));
}
@PrimitiveTransformTranslator(Read.Bounded.class)
private static void boundedReadTranslator(final PipelineTranslationContext ctx,
final TransformHierarchy.Node beamNode,
final Read.Bounded<?> transform) {
final IRVertex vertex = new BeamBoundedSourceVertex<>(transform.getSource());
ctx.addVertex(vertex);
beamNode.getInputs().values().forEach(input -> ctx.addEdgeTo(vertex, input));
beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, vertex, output));
}
@PrimitiveTransformTranslator(ParDo.SingleOutput.class)
private static void parDoSingleOutputTranslator(final PipelineTranslationContext ctx,
final TransformHierarchy.Node beamNode,
final ParDo.SingleOutput<?, ?> transform) {
final Map<Integer, PCollectionView<?>> sideInputMap = getSideInputMap(transform.getSideInputs());
final AbstractDoFnTransform doFnTransform = createDoFnTransform(ctx, beamNode, sideInputMap);
final IRVertex vertex = new OperatorVertex(doFnTransform);
ctx.addVertex(vertex);
beamNode.getInputs().values().stream()
.filter(input -> !transform.getAdditionalInputs().values().contains(input))
.forEach(input -> ctx.addEdgeTo(vertex, input));
ctx.addSideInputEdges(vertex, sideInputMap);
beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, vertex, output));
}
@PrimitiveTransformTranslator(ParDo.MultiOutput.class)
private static void parDoMultiOutputTranslator(final PipelineTranslationContext ctx,
final TransformHierarchy.Node beamNode,
final ParDo.MultiOutput<?, ?> transform) {
final Map<Integer, PCollectionView<?>> sideInputMap = getSideInputMap(transform.getSideInputs());
final AbstractDoFnTransform doFnTransform = createDoFnTransform(ctx, beamNode, sideInputMap);
final IRVertex vertex = new OperatorVertex(doFnTransform);
ctx.addVertex(vertex);
beamNode.getInputs().values().stream()
.filter(input -> !transform.getAdditionalInputs().values().contains(input))
.forEach(input -> ctx.addEdgeTo(vertex, input));
ctx.addSideInputEdges(vertex, sideInputMap);
beamNode.getOutputs().entrySet().stream()
.filter(pValueWithTupleTag -> pValueWithTupleTag.getKey().equals(transform.getMainOutputTag()))
.forEach(pValueWithTupleTag -> ctx.registerMainOutputFrom(beamNode, vertex, pValueWithTupleTag.getValue()));
beamNode.getOutputs().entrySet().stream()
.filter(pValueWithTupleTag -> !pValueWithTupleTag.getKey().equals(transform.getMainOutputTag()))
.forEach(pValueWithTupleTag -> ctx.registerAdditionalOutputFrom(beamNode, vertex, pValueWithTupleTag.getValue(),
pValueWithTupleTag.getKey()));
}
@PrimitiveTransformTranslator(GroupByKey.class)
private static void groupByKeyTranslator(final PipelineTranslationContext ctx,
final TransformHierarchy.Node beamNode,
final GroupByKey<?, ?> transform) {
final IRVertex vertex = new OperatorVertex(createGBKTransform(ctx, beamNode));
ctx.addVertex(vertex);
beamNode.getInputs().values().forEach(input -> ctx.addEdgeTo(vertex, input));
beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, vertex, output));
}
@PrimitiveTransformTranslator({Window.class, Window.Assign.class})
private static void windowTranslator(final PipelineTranslationContext ctx,
final TransformHierarchy.Node beamNode,
final PTransform<?, ?> transform) {
final WindowFn windowFn;
if (transform instanceof Window) {
windowFn = ((Window) transform).getWindowFn();
} else if (transform instanceof Window.Assign) {
windowFn = ((Window.Assign) transform).getWindowFn();
} else {
throw new UnsupportedOperationException(String.format("%s is not supported", transform));
}
final IRVertex vertex = new OperatorVertex(new WindowFnTransform(windowFn));
ctx.addVertex(vertex);
beamNode.getInputs().values().forEach(input -> ctx.addEdgeTo(vertex, input));
beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, vertex, output));
}
@PrimitiveTransformTranslator(View.CreatePCollectionView.class)
private static void createPCollectionViewTranslator(final PipelineTranslationContext ctx,
final TransformHierarchy.Node beamNode,
final View.CreatePCollectionView<?, ?> transform) {
final IRVertex vertex = new OperatorVertex(new CreateViewTransform(transform.getView().getViewFn()));
ctx.addVertex(vertex);
beamNode.getInputs().values().forEach(input -> ctx.addEdgeTo(vertex, input));
ctx.registerMainOutputFrom(beamNode, vertex, transform.getView());
beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, vertex, output));
}
@PrimitiveTransformTranslator(Flatten.PCollections.class)
private static void flattenTranslator(final PipelineTranslationContext ctx,
final TransformHierarchy.Node beamNode,
final Flatten.PCollections<?> transform) {
final IRVertex vertex = new OperatorVertex(new FlattenTransform());
ctx.addVertex(vertex);
beamNode.getInputs().values().forEach(input -> ctx.addEdgeTo(vertex, input));
beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, vertex, output));
}
////////////////////////////////////////////////////////////////////////////////////////////////////////
/////////////////////// COMPOSITE TRANSFORMS
/**
* {@link Combine.PerKey} = {@link GroupByKey} + {@link Combine.GroupedValues}
* ({@link Combine.Globally} internally uses {@link Combine.PerKey} which will also be optimized by this translator)
* Here, we translate this composite transform as a whole, exploiting its accumulator semantics.
*
* @param ctx provides translation context
* @param beamNode the given CompositeTransform to translate
* @param transform transform which can be obtained from {@code beamNode}
*/
@CompositeTransformTranslator(Combine.PerKey.class)
private static Pipeline.PipelineVisitor.CompositeBehavior combinePerKeyTranslator(
final PipelineTranslationContext ctx,
final TransformHierarchy.Node beamNode,
final PTransform<?, ?> transform) {
// Check if the partial combining optimization can be applied.
// If not, simply use the default Combine implementation by entering into it.
if (!isGlobalWindow(beamNode, ctx.getPipeline())) {
// TODO #263: Partial Combining for Beam Streaming
return Pipeline.PipelineVisitor.CompositeBehavior.ENTER_TRANSFORM;
}
final Combine.PerKey perKey = (Combine.PerKey) transform;
if (!perKey.getSideInputs().isEmpty()) {
// TODO #264: Partial Combining with Beam SideInputs
return Pipeline.PipelineVisitor.CompositeBehavior.ENTER_TRANSFORM;
}
// This Combine can be optimized as the following sequence of Nemo IRVertices.
// Combine Input -> Combine(Partial Combine -> KV<InputT, AccumT> -> Final Combine) -> Combine Output
final CombineFnBase.GlobalCombineFn combineFn = perKey.getFn();
// (Step 1) To Partial Combine
final IRVertex partialCombine = new OperatorVertex(new CombineFnPartialTransform<>(combineFn));
ctx.addVertex(partialCombine);
beamNode.getInputs().values().forEach(input -> ctx.addEdgeTo(partialCombine, input));
// (Step 2) To Final Combine
final PCollection input = (PCollection) Iterables.getOnlyElement(
TransformInputs.nonAdditionalInputs(beamNode.toAppliedPTransform(ctx.getPipeline())));
final KvCoder inputCoder = (KvCoder) input.getCoder();
final Coder accumulatorCoder;
try {
accumulatorCoder =
combineFn.getAccumulatorCoder(ctx.getPipeline().getCoderRegistry(), inputCoder.getValueCoder());
} catch (CannotProvideCoderException e) {
throw new RuntimeException(e);
}
final IRVertex finalCombine = new OperatorVertex(new CombineFnFinalTransform<>(combineFn));
ctx.addVertex(finalCombine);
final IREdge edge = new IREdge(CommunicationPatternProperty.Value.Shuffle, partialCombine, finalCombine);
ctx.addEdge(
edge,
KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder),
input.getWindowingStrategy().getWindowFn().windowCoder());
// (Step 3) To Combine Output
beamNode.getOutputs().values().forEach(output -> ctx.registerMainOutputFrom(beamNode, finalCombine, output));
// This composite transform has been translated in its entirety.
return Pipeline.PipelineVisitor.CompositeBehavior.DO_NOT_ENTER_TRANSFORM;
}
/**
* @param ctx provides translation context
* @param beamNode the given CompositeTransform to translate
* @param transform transform which can be obtained from {@code beamNode}
* @
*/
@CompositeTransformTranslator(LoopCompositeTransform.class)
private static Pipeline.PipelineVisitor.CompositeBehavior loopTranslator(
final PipelineTranslationContext ctx,
final TransformHierarchy.Node beamNode,
final LoopCompositeTransform<?, ?> transform) {
// Do nothing here, as the context handles the loop vertex stack.
// We just keep this method to signal that the loop vertex is acknowledged.
return Pipeline.PipelineVisitor.CompositeBehavior.ENTER_TRANSFORM;
}
////////////////////////////////////////////////////////////////////////////////////////////////////////
/////////////////////// HELPER METHODS
private static Map<Integer, PCollectionView<?>> getSideInputMap(final List<PCollectionView<?>> viewList) {
return IntStream.range(0, viewList.size()).boxed().collect(Collectors.toMap(Function.identity(), viewList::get));
}
private static AbstractDoFnTransform createDoFnTransform(final PipelineTranslationContext ctx,
final TransformHierarchy.Node beamNode,
final Map<Integer, PCollectionView<?>> sideInputMap) {
try {
final AppliedPTransform pTransform = beamNode.toAppliedPTransform(ctx.getPipeline());
final DoFn doFn = ParDoTranslation.getDoFn(pTransform);
final TupleTag mainOutputTag = ParDoTranslation.getMainOutputTag(pTransform);
final TupleTagList additionalOutputTags = ParDoTranslation.getAdditionalOutputTags(pTransform);
final PCollection<?> mainInput = (PCollection<?>)
Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(pTransform));
if (sideInputMap.isEmpty()) {
return new DoFnTransform(
doFn,
mainInput.getCoder(),
getOutputCoders(pTransform),
mainOutputTag,
additionalOutputTags.getAll(),
mainInput.getWindowingStrategy(),
ctx.getPipelineOptions());
} else {
return new PushBackDoFnTransform(
doFn,
mainInput.getCoder(),
getOutputCoders(pTransform),
mainOutputTag,
additionalOutputTags.getAll(),
mainInput.getWindowingStrategy(),
sideInputMap,
ctx.getPipelineOptions());
}
} catch (final IOException e) {
throw new RuntimeException(e);
}
}
private static Map<TupleTag<?>, Coder<?>> getOutputCoders(final AppliedPTransform<?, ?, ?> ptransform) {
return ptransform
.getOutputs()
.entrySet()
.stream()
.filter(e -> e.getValue() instanceof PCollection)
.collect(Collectors.toMap(e -> e.getKey(), e -> ((PCollection) e.getValue()).getCoder()));
}
/**
* Create a group by key transform.
* It returns GroupByKeyAndWindowDoFnTransform if window function is not default.
* @param ctx translation context
* @param beamNode transform vertex
* @return group by key transform
*/
private static Transform createGBKTransform(
final PipelineTranslationContext ctx,
final TransformHierarchy.Node beamNode) {
final AppliedPTransform pTransform = beamNode.toAppliedPTransform(ctx.getPipeline());
final PCollection<?> mainInput = (PCollection<?>)
Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(pTransform));
final TupleTag mainOutputTag = new TupleTag<>();
if (isGlobalWindow(beamNode, ctx.getPipeline())) {
return new GroupByKeyTransform();
} else {
return new GroupByKeyAndWindowDoFnTransform(
getOutputCoders(pTransform),
mainOutputTag,
mainInput.getWindowingStrategy(),
ctx.getPipelineOptions(),
SystemReduceFn.buffering(mainInput.getCoder()));
}
}
private static boolean isGlobalWindow(final TransformHierarchy.Node beamNode, final Pipeline pipeline) {
final AppliedPTransform pTransform = beamNode.toAppliedPTransform(pipeline);
final PCollection<?> mainInput = (PCollection<?>)
Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(pTransform));
return mainInput.getWindowingStrategy().getWindowFn() instanceof GlobalWindows;
}
}