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鈥檒l occasionally send you account related emails.

Already on GitHub? Sign in to your account

Replace WindowAssignment OldDoFn by FlatMap in FLink Runner #1435

Merged
merged 1 commit into from
Nov 24, 2016
Merged
Changes from all 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
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@

package org.apache.beam.runners.flink.translation;

import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import java.nio.ByteBuffer;
Expand All @@ -31,6 +30,7 @@
import java.util.Set;
import org.apache.beam.runners.core.SystemReduceFn;
import org.apache.beam.runners.flink.FlinkRunner;
import org.apache.beam.runners.flink.translation.functions.FlinkAssignWindows;
import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
import org.apache.beam.runners.flink.translation.types.FlinkCoder;
import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator;
Expand All @@ -53,7 +53,6 @@
import org.apache.beam.sdk.transforms.DoFn;
import org.apache.beam.sdk.transforms.Flatten;
import org.apache.beam.sdk.transforms.GroupByKey;
import org.apache.beam.sdk.transforms.OldDoFn;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.ParDo;
import org.apache.beam.sdk.transforms.join.RawUnionValue;
Expand Down Expand Up @@ -637,64 +636,20 @@ public void translateNode(
TypeInformation<WindowedValue<T>> typeInfo =
context.getTypeInfo(context.getOutput(transform));

OldDoFn<T, T> windowAssignerDoFn =
createWindowAssigner(windowingStrategy.getWindowFn());

@SuppressWarnings("unchecked")
PCollection<T> inputPCollection = context.getInput(transform);

TypeInformation<WindowedValue<T>> inputTypeInfo =
context.getTypeInfo(inputPCollection);

DoFnOperator<T, T, WindowedValue<T>> doFnOperator = new DoFnOperator<>(
windowAssignerDoFn,
inputTypeInfo,
new TupleTag<T>("main output"),
Collections.<TupleTag<?>>emptyList(),
new DoFnOperator.DefaultOutputManagerFactory<WindowedValue<T>>(),
windowingStrategy,
new HashMap<Integer, PCollectionView<?>>(), /* side-input mapping */
Collections.<PCollectionView<?>>emptyList(), /* side inputs */
context.getPipelineOptions());

DataStream<WindowedValue<T>> inputDataStream =
context.getInputDataStream(context.getInput(transform));

SingleOutputStreamOperator<WindowedValue<T>> outDataStream = inputDataStream
.transform(transform.getName(), typeInfo, doFnOperator);

context.setOutputDataStream(context.getOutput(transform), outDataStream);
}
WindowFn<T, ? extends BoundedWindow> windowFn = windowingStrategy.getWindowFn();

private static <T, W extends BoundedWindow> OldDoFn<T, T> createWindowAssigner(
final WindowFn<T, W> windowFn) {
FlinkAssignWindows<T, ? extends BoundedWindow> assignWindowsFunction =
new FlinkAssignWindows<>(windowFn);

return new OldDoFn<T, T>() {
SingleOutputStreamOperator<WindowedValue<T>> outputDataStream = inputDataStream
.flatMap(assignWindowsFunction)
.name(context.getOutput(transform).getName())
.returns(typeInfo);

@Override
public void processElement(final ProcessContext c) throws Exception {
Collection<W> windows = windowFn.assignWindows(
windowFn.new AssignContext() {
@Override
public T element() {
return c.element();
}

@Override
public Instant timestamp() {
return c.timestamp();
}

@Override
public BoundedWindow window() {
return Iterables.getOnlyElement(c.windowingInternals().windows());
}
});

c.windowingInternals().outputWindowedValue(
c.element(), c.timestamp(), windows, c.pane());
}
};
context.setOutputDataStream(context.getOutput(transform), outputDataStream);
}
}

Expand Down