Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[NEMO-216,251,259] Support side inputs and windowing #159

Merged
merged 27 commits into from Nov 21, 2018
Merged
Show file tree
Hide file tree
Changes from 21 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Expand Up @@ -20,7 +20,6 @@

import org.apache.nemo.common.exception.CompileTimeOptimizationException;
import org.apache.nemo.common.ir.edge.IREdge;
import org.apache.nemo.common.ir.edge.executionproperty.BroadcastVariableIdProperty;
import org.apache.nemo.common.ir.edge.executionproperty.DataFlowProperty;
import org.apache.nemo.common.ir.edge.executionproperty.MetricCollectionProperty;
import org.apache.nemo.common.ir.vertex.*;
Expand Down Expand Up @@ -258,14 +257,6 @@ private void sinkCheck() {
* Helper method to check that all execution properties are correct and makes sense.
*/
private void executionPropertyCheck() {
// SideInput is not compatible with Push
vertices.forEach(v -> incomingEdges.get(v).stream().filter(e -> e instanceof IREdge).map(e -> (IREdge) e)
.filter(e -> e.getPropertyValue(BroadcastVariableIdProperty.class).isPresent())
.filter(e -> DataFlowProperty.Value.Push.equals(e.getPropertyValue(DataFlowProperty.class).get()))
.forEach(e -> {
throw new CompileTimeOptimizationException("DAG execution property check: "
+ "Broadcast edge is not compatible with push" + e.getId());
}));
// DataSizeMetricCollection is not compatible with Push (All data have to be stored before the data collection)
vertices.forEach(v -> incomingEdges.get(v).stream().filter(e -> e instanceof IREdge).map(e -> (IREdge) e)
.filter(e -> Optional.of(MetricCollectionProperty.Value.DataSkewRuntimePass)
Expand Down
Expand Up @@ -49,7 +49,7 @@ public boolean equals(final Object o) {

@Override
public String toString() {
return String.valueOf(timestamp);
return String.valueOf("Watermark(" + timestamp + ")");
}

@Override
Expand Down
@@ -0,0 +1,108 @@
/*
* 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 org.apache.beam.runners.core.ReadyCheckingSideInputReader;
import org.apache.beam.sdk.transforms.ViewFn;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.values.PCollectionView;
import org.apache.nemo.common.Pair;
import org.apache.nemo.compiler.frontend.beam.transform.CreateViewTransform;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.annotation.Nullable;
import java.util.*;

/**
* Accumulates and provides side inputs in memory.
*/
public final class InMemorySideInputReader implements ReadyCheckingSideInputReader {
private static final Logger LOG = LoggerFactory.getLogger(InMemorySideInputReader.class.getName());

private long curWatermark = Long.MIN_VALUE;

private final Collection<PCollectionView<?>> sideInputsToRead;
private final Map<Pair<PCollectionView<?>, BoundedWindow>, Object> inMemorySideInputs;

public InMemorySideInputReader(final Collection<PCollectionView<?>> sideInputsToRead) {
this.sideInputsToRead = sideInputsToRead;
this.inMemorySideInputs = new HashMap<>();
}

@Override
public boolean isReady(final PCollectionView view, final BoundedWindow window) {
return window.maxTimestamp().getMillis() < curWatermark
|| inMemorySideInputs.containsKey(Pair.of(view, window));
}

@Nullable
@Override
public <T> T get(final PCollectionView<T> view, final BoundedWindow window) {
// This gets called after isReady()
final T sideInputData = (T) inMemorySideInputs.get(Pair.of(view, window));
return sideInputData == null
// The upstream gave us an empty sideInput
? ((ViewFn<Object, T>) view.getViewFn()).apply(new CreateViewTransform.MultiView<T>(Collections.emptyList()))
// The upstream gave us a concrete sideInput
: sideInputData;
}

@Override
public <T> boolean contains(final PCollectionView<T> view) {
return sideInputsToRead.contains(view);
}

@Override
public boolean isEmpty() {
return sideInputsToRead.isEmpty();
}

/**
* Stores the side input in memory to be used with main inputs.
* @param view of the side input.
* @param sideInputElement to add.
*/
public void addSideInputElement(final PCollectionView<?> view,
final WindowedValue<SideInputElement<?>> sideInputElement) {
for (final BoundedWindow bw : sideInputElement.getWindows()) {
inMemorySideInputs.put(Pair.of(view, bw), sideInputElement.getValue().getSideInputValue());
}
}

/**
* Say a DoFn of this reader has 3 main inputs and 4 side inputs.
* {@link org.apache.nemo.runtime.executor.datatransfer.InputWatermarkManager} guarantees that the watermark here
* is the minimum of the all 7 input streams.
* @param newWatermark to set.
*/
public void setCurrentWatermarkOfAllMainAndSideInputs(final long newWatermark) {
if (curWatermark > newWatermark) {
// Cannot go backwards in time.
throw new IllegalStateException(curWatermark + " > " + newWatermark);
}

this.curWatermark = newWatermark;
// TODO #282: Handle late data
inMemorySideInputs.entrySet().removeIf(entry -> {
return entry.getKey().right().maxTimestamp().getMillis() <= this.curWatermark; // Discard old sideinputs.
});
}
}
Expand Up @@ -35,6 +35,7 @@
import org.apache.nemo.common.ir.vertex.transform.Transform;
import org.apache.nemo.compiler.frontend.beam.coder.BeamDecoderFactory;
import org.apache.nemo.compiler.frontend.beam.coder.BeamEncoderFactory;
import org.apache.nemo.compiler.frontend.beam.coder.SideInputCoder;
import org.apache.nemo.compiler.frontend.beam.transform.*;

import java.util.*;
Expand Down Expand Up @@ -91,69 +92,84 @@ void addVertex(final IRVertex vertex) {
builder.addVertex(vertex, loopVertexStack);
}

/**
* Say the dstIRVertex consumes three views: view0, view1, and view2.
*
* We translate that as the following:
* view0 -> SideInputTransform(index=0) ->
* view1 -> SideInputTransform(index=1) -> dstIRVertex(with a map from indices to PCollectionViews)
* view2 -> SideInputTransform(index=2) ->
*
* @param dstVertex vertex.
* @param sideInputs of the vertex.
*/
void addSideInputEdges(final IRVertex dstVertex, final Map<Integer, PCollectionView<?>> sideInputs) {
for (final Map.Entry<Integer, PCollectionView<?>> entry : sideInputs.entrySet()) {
final int index = entry.getKey();
final PCollectionView view = entry.getValue();

final IRVertex srcVertex = pValueToProducerVertex.get(view);
final IRVertex sideInputTransformVertex = new OperatorVertex(new SideInputTransform(index));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why don't you use the view, instead of index? I think using view is more explicit, instead of the index.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

SideInputElement(Int, Object) is cheaper to encode than SideInputElement(PCollectionView, Object)
(The Flink runner also uses Int rather than PCollectionView)

addVertex(sideInputTransformVertex);
final Coder viewCoder = getCoderForView(view, this);
final Coder windowCoder = view.getPCollection().getWindowingStrategy().getWindowFn().windowCoder();

// First edge: view to transform
final IREdge firstEdge =
new IREdge(CommunicationPatternProperty.Value.OneToOne, srcVertex, sideInputTransformVertex);
addEdge(firstEdge, viewCoder, windowCoder);

// Second edge: transform to the dstIRVertex
final IREdge secondEdge =
new IREdge(CommunicationPatternProperty.Value.OneToOne, sideInputTransformVertex, dstVertex);
final WindowedValue.FullWindowedValueCoder sideInputElementCoder =
WindowedValue.getFullCoder(SideInputCoder.of(viewCoder), windowCoder);

secondEdge.setProperty(EncoderProperty.of(new BeamEncoderFactory(sideInputElementCoder)));
secondEdge.setProperty(DecoderProperty.of(new BeamDecoderFactory(sideInputElementCoder)));
builder.connectVertices(secondEdge);
}
}

/**
* Add IR edge to the builder.
*
* @param dst the destination IR vertex.
* @param input the {@link PValue} {@code dst} consumes
*/
void addEdgeTo(final IRVertex dst, final PValue input) {
final Coder coder;
if (input instanceof PCollection) {
coder = ((PCollection) input).getCoder();
} else if (input instanceof PCollectionView) {
coder = getCoderForView((PCollectionView) input, this);
} else {
throw new RuntimeException(String.format("While adding an edge to %s, coder for PValue %s cannot "
+ "be determined", dst, input));
}
addEdgeTo(dst, input, coder);
}
final Coder elementCoder = ((PCollection) input).getCoder();
final Coder windowCoder = ((PCollection) input).getWindowingStrategy().getWindowFn().windowCoder();
final IRVertex src = pValueToProducerVertex.get(input);
if (src == null) {
throw new IllegalStateException(String.format("Cannot find a vertex that emits pValue %s", input));
}

void addEdgeTo(final IRVertex dst, final PValue input, final Coder elementCoder) {
final IRVertex src = pValueToProducerVertex.get(input);
if (src == null) {
throw new IllegalStateException(String.format("Cannot find a vertex that emits pValue %s", input));
}
final CommunicationPatternProperty.Value communicationPattern = getCommPattern(src, dst);
final IREdge edge = new IREdge(communicationPattern, src, dst);

final Coder windowCoder;
final CommunicationPatternProperty.Value communicationPattern = getCommPattern(src, dst);
final IREdge edge = new IREdge(communicationPattern, src, dst);
if (pValueToTag.containsKey(input)) {
edge.setProperty(AdditionalOutputTagProperty.of(pValueToTag.get(input).getId()));
}

if (pValueToTag.containsKey(input)) {
edge.setProperty(AdditionalOutputTagProperty.of(pValueToTag.get(input).getId()));
}
if (input instanceof PCollectionView) {
edge.setProperty(BroadcastVariableIdProperty.of((PCollectionView) input));
}
if (input instanceof PCollection) {
windowCoder = ((PCollection) input).getWindowingStrategy().getWindowFn().windowCoder();
} else if (input instanceof PCollectionView) {
windowCoder = ((PCollectionView) input).getPCollection()
.getWindowingStrategy().getWindowFn().windowCoder();
addEdge(edge, elementCoder, windowCoder);
} else {
throw new RuntimeException(String.format("While adding an edge from %s, to %s, coder for PValue %s cannot "
+ "be determined", src, dst, input));
throw new IllegalStateException(input.toString());
}

addEdgeTo(edge, elementCoder, windowCoder);
}

void addEdgeTo(final IREdge edge,
final Coder elementCoder,
final Coder windowCoder) {
void addEdge(final IREdge edge, final Coder elementCoder, final Coder windowCoder) {
edge.setProperty(KeyExtractorProperty.of(new BeamKeyExtractor()));

if (elementCoder instanceof KvCoder) {
Coder keyCoder = ((KvCoder) elementCoder).getKeyCoder();
edge.setProperty(KeyEncoderProperty.of(new BeamEncoderFactory(keyCoder)));
edge.setProperty(KeyDecoderProperty.of(new BeamDecoderFactory(keyCoder)));
}

edge.setProperty(EncoderProperty.of(
new BeamEncoderFactory<>(WindowedValue.getFullCoder(elementCoder, windowCoder))));
edge.setProperty(DecoderProperty.of(
new BeamDecoderFactory<>(WindowedValue.getFullCoder(elementCoder, windowCoder))));
final WindowedValue.FullWindowedValueCoder coder = WindowedValue.getFullCoder(elementCoder, windowCoder);
edge.setProperty(EncoderProperty.of(new BeamEncoderFactory<>(coder)));
edge.setProperty(DecoderProperty.of(new BeamDecoderFactory<>(coder)));

builder.connectVertices(edge);
}
Expand Down
Expand Up @@ -48,7 +48,9 @@
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.
Expand Down Expand Up @@ -184,28 +186,30 @@ private static void boundedReadTranslator(final PipelineTranslationContext ctx,
private static void parDoSingleOutputTranslator(final PipelineTranslationContext ctx,
final TransformHierarchy.Node beamNode,
final ParDo.SingleOutput<?, ?> transform) {
final DoFnTransform doFnTransform = createDoFnTransform(ctx, beamNode);
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));
transform.getSideInputs().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 DoFnTransform doFnTransform = createDoFnTransform(ctx, beamNode);
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));
transform.getSideInputs().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()));
Expand Down Expand Up @@ -317,7 +321,7 @@ private static Pipeline.PipelineVisitor.CompositeBehavior combinePerKeyTranslato
final IRVertex finalCombine = new OperatorVertex(new CombineFnFinalTransform<>(combineFn));
ctx.addVertex(finalCombine);
final IREdge edge = new IREdge(CommunicationPatternProperty.Value.Shuffle, partialCombine, finalCombine);
ctx.addEdgeTo(
ctx.addEdge(
edge,
KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder),
input.getWindowingStrategy().getWindowFn().windowCoder());
Expand Down Expand Up @@ -348,27 +352,43 @@ private static Pipeline.PipelineVisitor.CompositeBehavior loopTranslator(
////////////////////////////////////////////////////////////////////////////////////////////////////////
/////////////////////// HELPER METHODS

private static DoFnTransform createDoFnTransform(final PipelineTranslationContext ctx,
final TransformHierarchy.Node beamNode) {
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 List<PCollectionView<?>> sideInputs = ParDoTranslation.getSideInputs(pTransform);
final TupleTagList additionalOutputTags = ParDoTranslation.getAdditionalOutputTags(pTransform);

final PCollection<?> mainInput = (PCollection<?>)
Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(pTransform));

return new DoFnTransform(
doFn,
mainInput.getCoder(),
getOutputCoders(pTransform),
mainOutputTag,
additionalOutputTags.getAll(),
mainInput.getWindowingStrategy(),
sideInputs,
ctx.getPipelineOptions());
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);
}
Expand Down Expand Up @@ -404,9 +424,7 @@ private static Transform createGBKTransform(
return new GroupByKeyAndWindowDoFnTransform(
getOutputCoders(pTransform),
mainOutputTag,
Collections.emptyList(), /* GBK does not have additional outputs */
mainInput.getWindowingStrategy(),
Collections.emptyList(), /* GBK does not have additional side inputs */
ctx.getPipelineOptions(),
SystemReduceFn.buffering(mainInput.getCoder()));
}
Expand Down