You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by da...@apache.org on 2016/03/04 19:10:56 UTC
[01/50] [abbrv] incubator-beam git commit: [readme] add a section on
how to submit cluster programs
Repository: incubator-beam
Updated Branches:
refs/heads/master 052857023 -> 4da935b65
[readme] add a section on how to submit cluster programs
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/d7bd7f4e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/d7bd7f4e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/d7bd7f4e
Branch: refs/heads/master
Commit: d7bd7f4e020dd678cf3d89e4b68efe18a0bb3814
Parents: 336d394
Author: Max <ma...@posteo.de>
Authored: Tue Jan 19 14:30:26 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
runners/flink/README.md | 81 +++++++++++++++++++++++++++++++++++++++++---
1 file changed, 76 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/d7bd7f4e/runners/flink/README.md
----------------------------------------------------------------------
diff --git a/runners/flink/README.md b/runners/flink/README.md
index a4b1d7e..e8c2170 100644
--- a/runners/flink/README.md
+++ b/runners/flink/README.md
@@ -60,11 +60,82 @@ Congratulations, you have run your first Google Dataflow program on top of Apach
# Running Dataflow on Flink on a cluster
-You can run your Dataflow program on a Apache Flink cluster as well. For more
-information, please visit the [Apache Flink Website](http://flink.apache.org) or
-contact the
-[Mailinglists](http://flink.apache.org/community.html#mailing-lists).
+You can run your Dataflow program on an Apache Flink cluster. Please start off by creating a new
+Maven project.
+
+ mvn archetype:generate -DgroupId=com.mycompany.dataflow -DartifactId=dataflow-test \
+ -DarchetypeArtifactId=maven-archetype-quickstart -DinteractiveMode=false
+
+The contents of the root `pom.xml` should be slightly changed aftewards (explanation below):
+
+```xml
+<?xml version="1.0" encoding="UTF-8"?>
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+ <modelVersion>4.0.0</modelVersion>
+
+ <groupId>com.mycompany.dataflow</groupId>
+ <artifactId>dataflow-test</artifactId>
+ <version>1.0</version>
+
+ <dependencies>
+ <dependency>
+ <groupId>com.dataartisans</groupId>
+ <artifactId>flink-dataflow</artifactId>
+ <version>0.2</version>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-shade-plugin</artifactId>
+ <version>2.4.1</version>
+ <executions>
+ <execution>
+ <phase>package</phase>
+ <goals>
+ <goal>shade</goal>
+ </goals>
+ <configuration>
+ <transformers>
+ <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+ <mainClass>WordCount</mainClass>
+ </transformer>
+ </transformers>
+ <artifactSet>
+ <excludes>
+ <exclude>org.apache.flink:*</exclude>
+ </excludes>
+ </artifactSet>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+
+ </plugins>
+
+ </build>
+
+</project>
+```
+
+The following changes have been made:
+
+1. The Flink Dataflow Runner was added as a dependency.
+
+2. The Maven Shade plugin was added to build a fat jar.
+
+A fat jar is necessary if you want to submit your Dataflow code to a Flink cluster. The fat jar
+includes your program code but also Dataflow code which is necessary during runtime. Note that this
+step is necessary because the Dataflow Runner is not part of Flink.
+
+For more information, please visit the [Apache Flink Website](http://flink.apache.org) or contact
+the [Mailinglists](http://flink.apache.org/community.html#mailing-lists).
# Streaming
-Streaming support is currently under development. See the `streaming` branch for the current version.
+Streaming support is currently under development. See the `streaming_new` branch for the current
+work in progress version.
[18/50] [abbrv] incubator-beam git commit: [maven] correct license
and formatting
Posted by da...@apache.org.
[maven] correct license and formatting
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/ea1095af
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/ea1095af
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/ea1095af
Branch: refs/heads/master
Commit: ea1095af96cfdb910e3ce907e5cbc608c052bb79
Parents: 9f70112
Author: Max <ma...@posteo.de>
Authored: Fri Jan 15 18:52:59 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
runners/flink/pom.xml | 75 +++++++++++++++++++++++-----------------------
1 file changed, 38 insertions(+), 37 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ea1095af/runners/flink/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml
index 288de1b..6102d74 100644
--- a/runners/flink/pom.xml
+++ b/runners/flink/pom.xml
@@ -1,18 +1,20 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
-Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans
- 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
+ Copyright 2015 Data Artisans GmbH
- http://www.apache.org/licenses/LICENSE-2.0
+ 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://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.
- 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.
-->
<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
@@ -25,7 +27,7 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans
<name>Flink Dataflow Runner</name>
<packaging>jar</packaging>
-
+
<inceptionYear>2015</inceptionYear>
<licenses>
@@ -39,13 +41,12 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans
<properties>
<project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
<project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
- <flink.version>0.10.1</flink.version>
-
- <!-- Default parameters for mvn exec:exec -->
- <clazz>com.dataartisans.flink.dataflow.examples.WordCount</clazz>
- <input>kinglear.txt</input>
- <output>wordcounts.txt</output>
- <parallelism>1</parallelism>
+ <flink.version>0.10.1</flink.version>
+ <!-- Default parameters for mvn exec:exec -->
+ <clazz>com.dataartisans.flink.dataflow.examples.WordCount</clazz>
+ <input>kinglear.txt</input>
+ <output>wordcounts.txt</output>
+ <parallelism>1</parallelism>
</properties>
<repositories>
@@ -114,7 +115,7 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans
</exclusions>
</dependency>
</dependencies>
-
+
<build>
<plugins>
<!-- JAR Packaging -->
@@ -134,7 +135,7 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans
<!-- Java compiler -->
<plugin>
-
+
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
<version>3.1</version><!--$NO-MVN-MAN-VER$-->
@@ -164,7 +165,7 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans
<artifactId>maven-surefire-plugin</artifactId>
<version>2.17</version><!--$NO-MVN-MAN-VER$-->
</plugin>
-
+
<!-- Eclipse Integration -->
<plugin>
<groupId>org.apache.maven.plugins</groupId>
@@ -205,22 +206,22 @@ Copyright 2015 Stephan Ewen, Aljoscha Krettek, Max Michels, dataArtisans
</executions>
</plugin>
- <plugin>
- <groupId>org.codehaus.mojo</groupId>
- <artifactId>exec-maven-plugin</artifactId>
- <version>1.2.1</version>
- <configuration>
- <executable>java</executable>
- <arguments>
- <argument>-classpath</argument>
- <classpath />
- <argument>${clazz}</argument>
- <argument>--input=${input}</argument>
- <argument>--output=${output}</argument>
- <argument>--parallelism=${parallelism}</argument>
- </arguments>
- </configuration>
- </plugin>
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>exec-maven-plugin</artifactId>
+ <version>1.2.1</version>
+ <configuration>
+ <executable>java</executable>
+ <arguments>
+ <argument>-classpath</argument>
+ <classpath />
+ <argument>${clazz}</argument>
+ <argument>--input=${input}</argument>
+ <argument>--output=${output}</argument>
+ <argument>--parallelism=${parallelism}</argument>
+ </arguments>
+ </configuration>
+ </plugin>
</plugins>
[43/50] [abbrv] incubator-beam git commit: [flink] convert tabs to 2
spaces
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
index eabc307..7dae0b0 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
@@ -44,221 +44,221 @@ import java.util.Collection;
* */
public abstract class FlinkAbstractParDoWrapper<IN, OUTDF, OUTFL> extends RichFlatMapFunction<WindowedValue<IN>, WindowedValue<OUTFL>> {
- private final DoFn<IN, OUTDF> doFn;
- private final WindowingStrategy<?, ?> windowingStrategy;
- private transient PipelineOptions options;
-
- private DoFnProcessContext context;
-
- public FlinkAbstractParDoWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUTDF> doFn) {
- Preconditions.checkNotNull(options);
- Preconditions.checkNotNull(windowingStrategy);
- Preconditions.checkNotNull(doFn);
-
- this.doFn = doFn;
- this.options = options;
- this.windowingStrategy = windowingStrategy;
- }
-
- private void initContext(DoFn<IN, OUTDF> function, Collector<WindowedValue<OUTFL>> outCollector) {
- if (this.context == null) {
- this.context = new DoFnProcessContext(function, outCollector);
- }
- }
-
- @Override
- public void flatMap(WindowedValue<IN> value, Collector<WindowedValue<OUTFL>> out) throws Exception {
- this.initContext(doFn, out);
-
- // for each window the element belongs to, create a new copy here.
- Collection<? extends BoundedWindow> windows = value.getWindows();
- if (windows.size() <= 1) {
- processElement(value);
- } else {
- for (BoundedWindow window : windows) {
- processElement(WindowedValue.of(
- value.getValue(), value.getTimestamp(), window, value.getPane()));
- }
- }
- }
-
- private void processElement(WindowedValue<IN> value) throws Exception {
- this.context.setElement(value);
- this.doFn.startBundle(context);
- doFn.processElement(context);
- this.doFn.finishBundle(context);
- }
-
- private class DoFnProcessContext extends DoFn<IN, OUTDF>.ProcessContext {
-
- private final DoFn<IN, OUTDF> fn;
-
- protected final Collector<WindowedValue<OUTFL>> collector;
-
- private WindowedValue<IN> element;
-
- private DoFnProcessContext(DoFn<IN, OUTDF> function, Collector<WindowedValue<OUTFL>> outCollector) {
- function.super();
- super.setupDelegateAggregators();
-
- this.fn = function;
- this.collector = outCollector;
- }
-
- public void setElement(WindowedValue<IN> value) {
- this.element = value;
- }
-
- @Override
- public IN element() {
- return this.element.getValue();
- }
-
- @Override
- public Instant timestamp() {
- return this.element.getTimestamp();
- }
-
- @Override
- public BoundedWindow window() {
- if (!(fn instanceof DoFn.RequiresWindowAccess)) {
- throw new UnsupportedOperationException(
- "window() is only available in the context of a DoFn marked as RequiresWindow.");
- }
-
- Collection<? extends BoundedWindow> windows = this.element.getWindows();
- if (windows.size() != 1) {
- throw new IllegalArgumentException("Each element is expected to belong to 1 window. " +
- "This belongs to " + windows.size() + ".");
- }
- return windows.iterator().next();
- }
-
- @Override
- public PaneInfo pane() {
- return this.element.getPane();
- }
-
- @Override
- public WindowingInternals<IN, OUTDF> windowingInternals() {
- return windowingInternalsHelper(element, collector);
- }
-
- @Override
- public PipelineOptions getPipelineOptions() {
- return options;
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view) {
- throw new RuntimeException("sideInput() is not supported in Streaming mode.");
- }
-
- @Override
- public void output(OUTDF output) {
- outputWithTimestamp(output, this.element.getTimestamp());
- }
-
- @Override
- public void outputWithTimestamp(OUTDF output, Instant timestamp) {
- outputWithTimestampHelper(element, output, timestamp, collector);
- }
-
- @Override
- public <T> void sideOutput(TupleTag<T> tag, T output) {
- sideOutputWithTimestamp(tag, output, this.element.getTimestamp());
- }
-
- @Override
- public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
- sideOutputWithTimestampHelper(element, output, timestamp, collector, tag);
- }
-
- @Override
- protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
- Accumulator acc = getRuntimeContext().getAccumulator(name);
- if (acc != null) {
- AccumulatorHelper.compareAccumulatorTypes(name,
- SerializableFnAggregatorWrapper.class, acc.getClass());
- return (Aggregator<AggInputT, AggOutputT>) acc;
- }
-
- SerializableFnAggregatorWrapper<AggInputT, AggOutputT> accumulator =
- new SerializableFnAggregatorWrapper<>(combiner);
- getRuntimeContext().addAccumulator(name, accumulator);
- return accumulator;
- }
- }
-
- protected void checkTimestamp(WindowedValue<IN> ref, Instant timestamp) {
- if (timestamp.isBefore(ref.getTimestamp().minus(doFn.getAllowedTimestampSkew()))) {
- throw new IllegalArgumentException(String.format(
- "Cannot output with timestamp %s. Output timestamps must be no earlier than the "
- + "timestamp of the current input (%s) minus the allowed skew (%s). See the "
- + "DoFn#getAllowedTimestmapSkew() Javadoc for details on changing the allowed skew.",
- timestamp, ref.getTimestamp(),
- PeriodFormat.getDefault().print(doFn.getAllowedTimestampSkew().toPeriod())));
- }
- }
-
- protected <T> WindowedValue<T> makeWindowedValue(
- T output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
- final Instant inputTimestamp = timestamp;
- final WindowFn windowFn = windowingStrategy.getWindowFn();
-
- if (timestamp == null) {
- timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
- }
-
- if (windows == null) {
- try {
- windows = windowFn.assignWindows(windowFn.new AssignContext() {
- @Override
- public Object element() {
- throw new UnsupportedOperationException(
- "WindowFn attempted to access input element when none was available");
- }
-
- @Override
- public Instant timestamp() {
- if (inputTimestamp == null) {
- throw new UnsupportedOperationException(
- "WindowFn attempted to access input timestamp when none was available");
- }
- return inputTimestamp;
- }
-
- @Override
- public Collection<? extends BoundedWindow> windows() {
- throw new UnsupportedOperationException(
- "WindowFn attempted to access input windows when none were available");
- }
- });
- } catch (Exception e) {
- throw UserCodeException.wrap(e);
- }
- }
-
- return WindowedValue.of(output, timestamp, windows, pane);
- }
-
- /////////// ABSTRACT METHODS TO BE IMPLEMENTED BY SUBCLASSES /////////////////
-
- public abstract void outputWithTimestampHelper(
- WindowedValue<IN> inElement,
- OUTDF output,
- Instant timestamp,
- Collector<WindowedValue<OUTFL>> outCollector);
-
- public abstract <T> void sideOutputWithTimestampHelper(
- WindowedValue<IN> inElement,
- T output,
- Instant timestamp,
- Collector<WindowedValue<OUTFL>> outCollector,
- TupleTag<T> tag);
-
- public abstract WindowingInternals<IN, OUTDF> windowingInternalsHelper(
- WindowedValue<IN> inElement,
- Collector<WindowedValue<OUTFL>> outCollector);
+ private final DoFn<IN, OUTDF> doFn;
+ private final WindowingStrategy<?, ?> windowingStrategy;
+ private transient PipelineOptions options;
+
+ private DoFnProcessContext context;
+
+ public FlinkAbstractParDoWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUTDF> doFn) {
+ Preconditions.checkNotNull(options);
+ Preconditions.checkNotNull(windowingStrategy);
+ Preconditions.checkNotNull(doFn);
+
+ this.doFn = doFn;
+ this.options = options;
+ this.windowingStrategy = windowingStrategy;
+ }
+
+ private void initContext(DoFn<IN, OUTDF> function, Collector<WindowedValue<OUTFL>> outCollector) {
+ if (this.context == null) {
+ this.context = new DoFnProcessContext(function, outCollector);
+ }
+ }
+
+ @Override
+ public void flatMap(WindowedValue<IN> value, Collector<WindowedValue<OUTFL>> out) throws Exception {
+ this.initContext(doFn, out);
+
+ // for each window the element belongs to, create a new copy here.
+ Collection<? extends BoundedWindow> windows = value.getWindows();
+ if (windows.size() <= 1) {
+ processElement(value);
+ } else {
+ for (BoundedWindow window : windows) {
+ processElement(WindowedValue.of(
+ value.getValue(), value.getTimestamp(), window, value.getPane()));
+ }
+ }
+ }
+
+ private void processElement(WindowedValue<IN> value) throws Exception {
+ this.context.setElement(value);
+ this.doFn.startBundle(context);
+ doFn.processElement(context);
+ this.doFn.finishBundle(context);
+ }
+
+ private class DoFnProcessContext extends DoFn<IN, OUTDF>.ProcessContext {
+
+ private final DoFn<IN, OUTDF> fn;
+
+ protected final Collector<WindowedValue<OUTFL>> collector;
+
+ private WindowedValue<IN> element;
+
+ private DoFnProcessContext(DoFn<IN, OUTDF> function, Collector<WindowedValue<OUTFL>> outCollector) {
+ function.super();
+ super.setupDelegateAggregators();
+
+ this.fn = function;
+ this.collector = outCollector;
+ }
+
+ public void setElement(WindowedValue<IN> value) {
+ this.element = value;
+ }
+
+ @Override
+ public IN element() {
+ return this.element.getValue();
+ }
+
+ @Override
+ public Instant timestamp() {
+ return this.element.getTimestamp();
+ }
+
+ @Override
+ public BoundedWindow window() {
+ if (!(fn instanceof DoFn.RequiresWindowAccess)) {
+ throw new UnsupportedOperationException(
+ "window() is only available in the context of a DoFn marked as RequiresWindow.");
+ }
+
+ Collection<? extends BoundedWindow> windows = this.element.getWindows();
+ if (windows.size() != 1) {
+ throw new IllegalArgumentException("Each element is expected to belong to 1 window. " +
+ "This belongs to " + windows.size() + ".");
+ }
+ return windows.iterator().next();
+ }
+
+ @Override
+ public PaneInfo pane() {
+ return this.element.getPane();
+ }
+
+ @Override
+ public WindowingInternals<IN, OUTDF> windowingInternals() {
+ return windowingInternalsHelper(element, collector);
+ }
+
+ @Override
+ public PipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view) {
+ throw new RuntimeException("sideInput() is not supported in Streaming mode.");
+ }
+
+ @Override
+ public void output(OUTDF output) {
+ outputWithTimestamp(output, this.element.getTimestamp());
+ }
+
+ @Override
+ public void outputWithTimestamp(OUTDF output, Instant timestamp) {
+ outputWithTimestampHelper(element, output, timestamp, collector);
+ }
+
+ @Override
+ public <T> void sideOutput(TupleTag<T> tag, T output) {
+ sideOutputWithTimestamp(tag, output, this.element.getTimestamp());
+ }
+
+ @Override
+ public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+ sideOutputWithTimestampHelper(element, output, timestamp, collector, tag);
+ }
+
+ @Override
+ protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
+ Accumulator acc = getRuntimeContext().getAccumulator(name);
+ if (acc != null) {
+ AccumulatorHelper.compareAccumulatorTypes(name,
+ SerializableFnAggregatorWrapper.class, acc.getClass());
+ return (Aggregator<AggInputT, AggOutputT>) acc;
+ }
+
+ SerializableFnAggregatorWrapper<AggInputT, AggOutputT> accumulator =
+ new SerializableFnAggregatorWrapper<>(combiner);
+ getRuntimeContext().addAccumulator(name, accumulator);
+ return accumulator;
+ }
+ }
+
+ protected void checkTimestamp(WindowedValue<IN> ref, Instant timestamp) {
+ if (timestamp.isBefore(ref.getTimestamp().minus(doFn.getAllowedTimestampSkew()))) {
+ throw new IllegalArgumentException(String.format(
+ "Cannot output with timestamp %s. Output timestamps must be no earlier than the "
+ + "timestamp of the current input (%s) minus the allowed skew (%s). See the "
+ + "DoFn#getAllowedTimestmapSkew() Javadoc for details on changing the allowed skew.",
+ timestamp, ref.getTimestamp(),
+ PeriodFormat.getDefault().print(doFn.getAllowedTimestampSkew().toPeriod())));
+ }
+ }
+
+ protected <T> WindowedValue<T> makeWindowedValue(
+ T output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+ final Instant inputTimestamp = timestamp;
+ final WindowFn windowFn = windowingStrategy.getWindowFn();
+
+ if (timestamp == null) {
+ timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+ }
+
+ if (windows == null) {
+ try {
+ windows = windowFn.assignWindows(windowFn.new AssignContext() {
+ @Override
+ public Object element() {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input element when none was available");
+ }
+
+ @Override
+ public Instant timestamp() {
+ if (inputTimestamp == null) {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input timestamp when none was available");
+ }
+ return inputTimestamp;
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input windows when none were available");
+ }
+ });
+ } catch (Exception e) {
+ throw UserCodeException.wrap(e);
+ }
+ }
+
+ return WindowedValue.of(output, timestamp, windows, pane);
+ }
+
+ /////////// ABSTRACT METHODS TO BE IMPLEMENTED BY SUBCLASSES /////////////////
+
+ public abstract void outputWithTimestampHelper(
+ WindowedValue<IN> inElement,
+ OUTDF output,
+ Instant timestamp,
+ Collector<WindowedValue<OUTFL>> outCollector);
+
+ public abstract <T> void sideOutputWithTimestampHelper(
+ WindowedValue<IN> inElement,
+ T output,
+ Instant timestamp,
+ Collector<WindowedValue<OUTFL>> outCollector,
+ TupleTag<T> tag);
+
+ public abstract WindowingInternals<IN, OUTDF> windowingInternalsHelper(
+ WindowedValue<IN> inElement,
+ Collector<WindowedValue<OUTFL>> outCollector);
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
index fb3d329..55235c9 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
@@ -68,562 +68,562 @@ import java.util.*;
* for furhter processing.
*/
public class FlinkGroupAlsoByWindowWrapper<K, VIN, VACC, VOUT>
- extends AbstractStreamOperator<WindowedValue<KV<K, VOUT>>>
- implements OneInputStreamOperator<WindowedValue<KV<K, VIN>>, WindowedValue<KV<K, VOUT>>> {
-
- private static final long serialVersionUID = 1L;
-
- private transient PipelineOptions options;
-
- private transient CoderRegistry coderRegistry;
-
- private DoFn<KeyedWorkItem<K, VIN>, KV<K, VOUT>> operator;
-
- private ProcessContext context;
-
- private final WindowingStrategy<KV<K, VIN>, BoundedWindow> windowingStrategy;
-
- private final Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combineFn;
-
- private final KvCoder<K, VIN> inputKvCoder;
-
- /**
- * State is kept <b>per-key</b>. This data structure keeps this mapping between an active key, i.e. a
- * key whose elements are currently waiting to be processed, and its associated state.
- */
- private Map<K, FlinkStateInternals<K>> perKeyStateInternals = new HashMap<>();
-
- /**
- * Timers waiting to be processed.
- */
- private Map<K, Set<TimerInternals.TimerData>> activeTimers = new HashMap<>();
-
- private FlinkTimerInternals timerInternals = new FlinkTimerInternals();
-
- /**
- * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy.
- * This method assumes that <b>elements are already grouped by key</b>.
- * <p/>
- * The difference with {@link #createForIterable(PipelineOptions, PCollection, KeyedStream)}
- * is that this method assumes that a combiner function is provided
- * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}).
- * A combiner helps at increasing the speed and, in most of the cases, reduce the per-window state.
- *
- * @param options the general job configuration options.
- * @param input the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}.
- * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key.
- * @param combiner the combiner to be used.
- * @param outputKvCoder the type of the output values.
- */
- public static <K, VIN, VACC, VOUT> DataStream<WindowedValue<KV<K, VOUT>>> create(
- PipelineOptions options,
- PCollection input,
- KeyedStream<WindowedValue<KV<K, VIN>>, K> groupedStreamByKey,
- Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner,
- KvCoder<K, VOUT> outputKvCoder) {
- Preconditions.checkNotNull(options);
-
- KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) input.getCoder();
- FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper<>(options,
- input.getPipeline().getCoderRegistry(), input.getWindowingStrategy(), inputKvCoder, combiner);
-
- Coder<WindowedValue<KV<K, VOUT>>> windowedOutputElemCoder = WindowedValue.FullWindowedValueCoder.of(
- outputKvCoder,
- input.getWindowingStrategy().getWindowFn().windowCoder());
-
- CoderTypeInformation<WindowedValue<KV<K, VOUT>>> outputTypeInfo =
- new CoderTypeInformation<>(windowedOutputElemCoder);
-
- DataStream<WindowedValue<KV<K, VOUT>>> groupedByKeyAndWindow = groupedStreamByKey
- .transform("GroupByWindowWithCombiner",
- new CoderTypeInformation<>(outputKvCoder),
- windower)
- .returns(outputTypeInfo);
-
- return groupedByKeyAndWindow;
- }
-
- /**
- * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy.
- * This method assumes that <b>elements are already grouped by key</b>.
- * <p/>
- * The difference with {@link #create(PipelineOptions, PCollection, KeyedStream, Combine.KeyedCombineFn, KvCoder)}
- * is that this method assumes no combiner function
- * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}).
- *
- * @param options the general job configuration options.
- * @param input the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}.
- * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key.
- */
- public static <K, VIN> DataStream<WindowedValue<KV<K, Iterable<VIN>>>> createForIterable(
- PipelineOptions options,
- PCollection input,
- KeyedStream<WindowedValue<KV<K, VIN>>, K> groupedStreamByKey) {
- Preconditions.checkNotNull(options);
-
- KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) input.getCoder();
- Coder<K> keyCoder = inputKvCoder.getKeyCoder();
- Coder<VIN> inputValueCoder = inputKvCoder.getValueCoder();
-
- FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper(options,
- input.getPipeline().getCoderRegistry(), input.getWindowingStrategy(), inputKvCoder, null);
-
- Coder<Iterable<VIN>> valueIterCoder = IterableCoder.of(inputValueCoder);
- KvCoder<K, Iterable<VIN>> outputElemCoder = KvCoder.of(keyCoder, valueIterCoder);
-
- Coder<WindowedValue<KV<K, Iterable<VIN>>>> windowedOutputElemCoder = WindowedValue.FullWindowedValueCoder.of(
- outputElemCoder,
- input.getWindowingStrategy().getWindowFn().windowCoder());
-
- CoderTypeInformation<WindowedValue<KV<K, Iterable<VIN>>>> outputTypeInfo =
- new CoderTypeInformation<>(windowedOutputElemCoder);
-
- DataStream<WindowedValue<KV<K, Iterable<VIN>>>> groupedByKeyAndWindow = groupedStreamByKey
- .transform("GroupByWindow",
- new CoderTypeInformation<>(windowedOutputElemCoder),
- windower)
- .returns(outputTypeInfo);
-
- return groupedByKeyAndWindow;
- }
-
- public static <K, VIN, VACC, VOUT> FlinkGroupAlsoByWindowWrapper
- createForTesting(PipelineOptions options,
- CoderRegistry registry,
- WindowingStrategy<KV<K, VIN>, BoundedWindow> windowingStrategy,
- KvCoder<K, VIN> inputCoder,
- Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner) {
- Preconditions.checkNotNull(options);
-
- return new FlinkGroupAlsoByWindowWrapper(options, registry, windowingStrategy, inputCoder, combiner);
- }
-
- private FlinkGroupAlsoByWindowWrapper(PipelineOptions options,
- CoderRegistry registry,
- WindowingStrategy<KV<K, VIN>, BoundedWindow> windowingStrategy,
- KvCoder<K, VIN> inputCoder,
- Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner) {
- Preconditions.checkNotNull(options);
-
- this.options = Preconditions.checkNotNull(options);
- this.coderRegistry = Preconditions.checkNotNull(registry);
- this.inputKvCoder = Preconditions.checkNotNull(inputCoder);//(KvCoder<K, VIN>) input.getCoder();
- this.windowingStrategy = Preconditions.checkNotNull(windowingStrategy);//input.getWindowingStrategy();
- this.combineFn = combiner;
- this.operator = createGroupAlsoByWindowOperator();
- this.chainingStrategy = ChainingStrategy.ALWAYS;
- }
-
- @Override
- public void open() throws Exception {
- super.open();
- this.context = new ProcessContext(operator, new TimestampedCollector<>(output), this.timerInternals);
- }
-
- /**
- * Create the adequate {@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn},
- * <b> if not already created</b>.
- * If a {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn} was provided, then
- * a function with that combiner is created, so that elements are combined as they arrive. This is
- * done for speed and (in most of the cases) for reduction of the per-window state.
- */
- private <W extends BoundedWindow> DoFn<KeyedWorkItem<K, VIN>, KV<K, VOUT>> createGroupAlsoByWindowOperator() {
- if (this.operator == null) {
- if (this.combineFn == null) {
- // Thus VOUT == Iterable<VIN>
- Coder<VIN> inputValueCoder = inputKvCoder.getValueCoder();
-
- this.operator = (DoFn) GroupAlsoByWindowViaWindowSetDoFn.create(
- (WindowingStrategy<?, W>) this.windowingStrategy, SystemReduceFn.<K, VIN, W>buffering(inputValueCoder));
- } else {
- Coder<K> inputKeyCoder = inputKvCoder.getKeyCoder();
-
- AppliedCombineFn<K, VIN, VACC, VOUT> appliedCombineFn = AppliedCombineFn
- .withInputCoder(combineFn, coderRegistry, inputKvCoder);
-
- this.operator = GroupAlsoByWindowViaWindowSetDoFn.create(
- (WindowingStrategy<?, W>) this.windowingStrategy, SystemReduceFn.<K, VIN, VACC, VOUT, W>combining(inputKeyCoder, appliedCombineFn));
- }
- }
- return this.operator;
- }
-
- private void processKeyedWorkItem(KeyedWorkItem<K, VIN> workItem) throws Exception {
- context.setElement(workItem, getStateInternalsForKey(workItem.key()));
-
- // TODO: Ideally startBundle/finishBundle would be called when the operator is first used / about to be discarded.
- operator.startBundle(context);
- operator.processElement(context);
- operator.finishBundle(context);
- }
-
- @Override
- public void processElement(StreamRecord<WindowedValue<KV<K, VIN>>> element) throws Exception {
- ArrayList<WindowedValue<VIN>> elements = new ArrayList<>();
- elements.add(WindowedValue.of(element.getValue().getValue().getValue(), element.getValue().getTimestamp(),
- element.getValue().getWindows(), element.getValue().getPane()));
- processKeyedWorkItem(KeyedWorkItems.elementsWorkItem(element.getValue().getValue().getKey(), elements));
- }
-
- @Override
- public void processWatermark(Watermark mark) throws Exception {
- context.setCurrentInputWatermark(new Instant(mark.getTimestamp()));
-
- Multimap<K, TimerInternals.TimerData> timers = getTimersReadyToProcess(mark.getTimestamp());
- if (!timers.isEmpty()) {
- for (K key : timers.keySet()) {
- processKeyedWorkItem(KeyedWorkItems.<K, VIN>timersWorkItem(key, timers.get(key)));
- }
- }
-
- /**
- * This is to take into account the different semantics of the Watermark in Flink and
- * in Dataflow. To understand the reasoning behind the Dataflow semantics and its
- * watermark holding logic, see the documentation of
- * {@link WatermarkHold#addHold(ReduceFn.ProcessValueContext, boolean)}
- * */
- long millis = Long.MAX_VALUE;
- for (FlinkStateInternals state : perKeyStateInternals.values()) {
- Instant watermarkHold = state.getWatermarkHold();
- if (watermarkHold != null && watermarkHold.getMillis() < millis) {
- millis = watermarkHold.getMillis();
- }
- }
-
- if (mark.getTimestamp() < millis) {
- millis = mark.getTimestamp();
- }
-
- context.setCurrentOutputWatermark(new Instant(millis));
-
- // Don't forget to re-emit the watermark for further operators down the line.
- // This is critical for jobs with multiple aggregation steps.
- // Imagine a job with a groupByKey() on key K1, followed by a map() that changes
- // the key K1 to K2, and another groupByKey() on K2. In this case, if the watermark
- // is not re-emitted, the second aggregation would never be triggered, and no result
- // will be produced.
- output.emitWatermark(new Watermark(millis));
- }
-
- @Override
- public void close() throws Exception {
- super.close();
- }
-
- private void registerActiveTimer(K key, TimerInternals.TimerData timer) {
- Set<TimerInternals.TimerData> timersForKey = activeTimers.get(key);
- if (timersForKey == null) {
- timersForKey = new HashSet<>();
- }
- timersForKey.add(timer);
- activeTimers.put(key, timersForKey);
- }
-
- private void unregisterActiveTimer(K key, TimerInternals.TimerData timer) {
- Set<TimerInternals.TimerData> timersForKey = activeTimers.get(key);
- if (timersForKey != null) {
- timersForKey.remove(timer);
- if (timersForKey.isEmpty()) {
- activeTimers.remove(key);
- } else {
- activeTimers.put(key, timersForKey);
- }
- }
- }
-
- /**
- * Returns the list of timers that are ready to fire. These are the timers
- * that are registered to be triggered at a time before the current watermark.
- * We keep these timers in a Set, so that they are deduplicated, as the same
- * timer can be registered multiple times.
- */
- private Multimap<K, TimerInternals.TimerData> getTimersReadyToProcess(long currentWatermark) {
-
- // we keep the timers to return in a different list and launch them later
- // because we cannot prevent a trigger from registering another trigger,
- // which would lead to concurrent modification exception.
- Multimap<K, TimerInternals.TimerData> toFire = HashMultimap.create();
-
- Iterator<Map.Entry<K, Set<TimerInternals.TimerData>>> it = activeTimers.entrySet().iterator();
- while (it.hasNext()) {
- Map.Entry<K, Set<TimerInternals.TimerData>> keyWithTimers = it.next();
-
- Iterator<TimerInternals.TimerData> timerIt = keyWithTimers.getValue().iterator();
- while (timerIt.hasNext()) {
- TimerInternals.TimerData timerData = timerIt.next();
- if (timerData.getTimestamp().isBefore(currentWatermark)) {
- toFire.put(keyWithTimers.getKey(), timerData);
- timerIt.remove();
- }
- }
-
- if (keyWithTimers.getValue().isEmpty()) {
- it.remove();
- }
- }
- return toFire;
- }
-
- /**
- * Gets the state associated with the specified key.
- *
- * @param key the key whose state we want.
- * @return The {@link FlinkStateInternals}
- * associated with that key.
- */
- private FlinkStateInternals<K> getStateInternalsForKey(K key) {
- FlinkStateInternals<K> stateInternals = perKeyStateInternals.get(key);
- if (stateInternals == null) {
- Coder<? extends BoundedWindow> windowCoder = this.windowingStrategy.getWindowFn().windowCoder();
- OutputTimeFn<? super BoundedWindow> outputTimeFn = this.windowingStrategy.getWindowFn().getOutputTimeFn();
- stateInternals = new FlinkStateInternals<>(key, inputKvCoder.getKeyCoder(), windowCoder, outputTimeFn);
- perKeyStateInternals.put(key, stateInternals);
- }
- return stateInternals;
- }
-
- private class FlinkTimerInternals extends AbstractFlinkTimerInternals<K, VIN> {
- @Override
- public void setTimer(TimerData timerKey) {
- registerActiveTimer(context.element().key(), timerKey);
- }
-
- @Override
- public void deleteTimer(TimerData timerKey) {
- unregisterActiveTimer(context.element().key(), timerKey);
- }
- }
-
- private class ProcessContext extends GroupAlsoByWindowViaWindowSetDoFn<K, VIN, VOUT, ?, KeyedWorkItem<K, VIN>>.ProcessContext {
-
- private final FlinkTimerInternals timerInternals;
-
- private final TimestampedCollector<WindowedValue<KV<K, VOUT>>> collector;
-
- private FlinkStateInternals<K> stateInternals;
-
- private KeyedWorkItem<K, VIN> element;
-
- public ProcessContext(DoFn<KeyedWorkItem<K, VIN>, KV<K, VOUT>> function,
- TimestampedCollector<WindowedValue<KV<K, VOUT>>> outCollector,
- FlinkTimerInternals timerInternals) {
- function.super();
- super.setupDelegateAggregators();
-
- this.collector = Preconditions.checkNotNull(outCollector);
- this.timerInternals = Preconditions.checkNotNull(timerInternals);
- }
-
- public void setElement(KeyedWorkItem<K, VIN> element,
- FlinkStateInternals<K> stateForKey) {
- this.element = element;
- this.stateInternals = stateForKey;
- }
-
- public void setCurrentInputWatermark(Instant watermark) {
- this.timerInternals.setCurrentInputWatermark(watermark);
- }
-
- public void setCurrentOutputWatermark(Instant watermark) {
- this.timerInternals.setCurrentOutputWatermark(watermark);
- }
-
- @Override
- public KeyedWorkItem<K, VIN> element() {
- return this.element;
- }
-
- @Override
- public Instant timestamp() {
- throw new UnsupportedOperationException("timestamp() is not available when processing KeyedWorkItems.");
- }
-
- @Override
- public PipelineOptions getPipelineOptions() {
- // TODO: PipelineOptions need to be available on the workers.
- // Ideally they are captured as part of the pipeline.
- // For now, construct empty options so that StateContexts.createFromComponents
- // will yield a valid StateContext, which is needed to support the StateContext.window().
- if (options == null) {
- options = new PipelineOptions() {
- @Override
- public <T extends PipelineOptions> T as(Class<T> kls) {
- return null;
- }
-
- @Override
- public <T extends PipelineOptions> T cloneAs(Class<T> kls) {
- return null;
- }
-
- @Override
- public Class<? extends PipelineRunner<?>> getRunner() {
- return null;
- }
-
- @Override
- public void setRunner(Class<? extends PipelineRunner<?>> kls) {
-
- }
-
- @Override
- public CheckEnabled getStableUniqueNames() {
- return null;
- }
-
- @Override
- public void setStableUniqueNames(CheckEnabled enabled) {
- }
- };
- }
- return options;
- }
-
- @Override
- public void output(KV<K, VOUT> output) {
- throw new UnsupportedOperationException(
- "output() is not available when processing KeyedWorkItems.");
- }
-
- @Override
- public void outputWithTimestamp(KV<K, VOUT> output, Instant timestamp) {
- throw new UnsupportedOperationException(
- "outputWithTimestamp() is not available when processing KeyedWorkItems.");
- }
-
- @Override
- public PaneInfo pane() {
- throw new UnsupportedOperationException("pane() is not available when processing KeyedWorkItems.");
- }
-
- @Override
- public BoundedWindow window() {
- throw new UnsupportedOperationException(
- "window() is not available when processing KeyedWorkItems.");
- }
-
- @Override
- public WindowingInternals<KeyedWorkItem<K, VIN>, KV<K, VOUT>> windowingInternals() {
- return new WindowingInternals<KeyedWorkItem<K, VIN>, KV<K, VOUT>>() {
-
- @Override
- public com.google.cloud.dataflow.sdk.util.state.StateInternals stateInternals() {
- return stateInternals;
- }
-
- @Override
- public void outputWindowedValue(KV<K, VOUT> output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
- // TODO: No need to represent timestamp twice.
- collector.setAbsoluteTimestamp(timestamp.getMillis());
- collector.collect(WindowedValue.of(output, timestamp, windows, pane));
-
- }
-
- @Override
- public TimerInternals timerInternals() {
- return timerInternals;
- }
-
- @Override
- public Collection<? extends BoundedWindow> windows() {
- throw new UnsupportedOperationException("windows() is not available in Streaming mode.");
- }
-
- @Override
- public PaneInfo pane() {
- throw new UnsupportedOperationException("pane() is not available in Streaming mode.");
- }
-
- @Override
- public <T> void writePCollectionViewData(TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
- throw new RuntimeException("writePCollectionViewData() not available in Streaming mode.");
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
- throw new RuntimeException("sideInput() is not available in Streaming mode.");
- }
- };
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view) {
- throw new RuntimeException("sideInput() is not supported in Streaming mode.");
- }
-
- @Override
- public <T> void sideOutput(TupleTag<T> tag, T output) {
- // ignore the side output, this can happen when a user does not register
- // side outputs but then outputs using a freshly created TupleTag.
- throw new RuntimeException("sideOutput() is not available when grouping by window.");
- }
-
- @Override
- public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
- sideOutput(tag, output);
- }
-
- @Override
- protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
- Accumulator acc = getRuntimeContext().getAccumulator(name);
- if (acc != null) {
- AccumulatorHelper.compareAccumulatorTypes(name,
- SerializableFnAggregatorWrapper.class, acc.getClass());
- return (Aggregator<AggInputT, AggOutputT>) acc;
- }
-
- SerializableFnAggregatorWrapper<AggInputT, AggOutputT> accumulator =
- new SerializableFnAggregatorWrapper<>(combiner);
- getRuntimeContext().addAccumulator(name, accumulator);
- return accumulator;
- }
- }
-
- ////////////// Checkpointing implementation ////////////////
-
- @Override
- public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception {
- StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
- AbstractStateBackend.CheckpointStateOutputView out = getStateBackend().createCheckpointStateOutputView(checkpointId, timestamp);
- StateCheckpointWriter writer = StateCheckpointWriter.create(out);
- Coder<K> keyCoder = inputKvCoder.getKeyCoder();
-
- // checkpoint the timers
- StateCheckpointUtils.encodeTimers(activeTimers, writer, keyCoder);
-
- // checkpoint the state
- StateCheckpointUtils.encodeState(perKeyStateInternals, writer, keyCoder);
-
- // checkpoint the timerInternals
- context.timerInternals.encodeTimerInternals(context, writer,
- inputKvCoder, windowingStrategy.getWindowFn().windowCoder());
-
- taskState.setOperatorState(out.closeAndGetHandle());
- return taskState;
- }
-
- @Override
- public void restoreState(StreamTaskState taskState, long recoveryTimestamp) throws Exception {
- super.restoreState(taskState, recoveryTimestamp);
-
- final ClassLoader userClassloader = getUserCodeClassloader();
-
- Coder<? extends BoundedWindow> windowCoder = this.windowingStrategy.getWindowFn().windowCoder();
- Coder<K> keyCoder = inputKvCoder.getKeyCoder();
-
- @SuppressWarnings("unchecked")
- StateHandle<DataInputView> inputState = (StateHandle<DataInputView>) taskState.getOperatorState();
- DataInputView in = inputState.getState(userClassloader);
- StateCheckpointReader reader = new StateCheckpointReader(in);
-
- // restore the timers
- this.activeTimers = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder);
-
- // restore the state
- this.perKeyStateInternals = StateCheckpointUtils.decodeState(
- reader, windowingStrategy.getOutputTimeFn(), keyCoder, windowCoder, userClassloader);
-
- // restore the timerInternals.
- this.timerInternals.restoreTimerInternals(reader, inputKvCoder, windowCoder);
- }
+ extends AbstractStreamOperator<WindowedValue<KV<K, VOUT>>>
+ implements OneInputStreamOperator<WindowedValue<KV<K, VIN>>, WindowedValue<KV<K, VOUT>>> {
+
+ private static final long serialVersionUID = 1L;
+
+ private transient PipelineOptions options;
+
+ private transient CoderRegistry coderRegistry;
+
+ private DoFn<KeyedWorkItem<K, VIN>, KV<K, VOUT>> operator;
+
+ private ProcessContext context;
+
+ private final WindowingStrategy<KV<K, VIN>, BoundedWindow> windowingStrategy;
+
+ private final Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combineFn;
+
+ private final KvCoder<K, VIN> inputKvCoder;
+
+ /**
+ * State is kept <b>per-key</b>. This data structure keeps this mapping between an active key, i.e. a
+ * key whose elements are currently waiting to be processed, and its associated state.
+ */
+ private Map<K, FlinkStateInternals<K>> perKeyStateInternals = new HashMap<>();
+
+ /**
+ * Timers waiting to be processed.
+ */
+ private Map<K, Set<TimerInternals.TimerData>> activeTimers = new HashMap<>();
+
+ private FlinkTimerInternals timerInternals = new FlinkTimerInternals();
+
+ /**
+ * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy.
+ * This method assumes that <b>elements are already grouped by key</b>.
+ * <p/>
+ * The difference with {@link #createForIterable(PipelineOptions, PCollection, KeyedStream)}
+ * is that this method assumes that a combiner function is provided
+ * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}).
+ * A combiner helps at increasing the speed and, in most of the cases, reduce the per-window state.
+ *
+ * @param options the general job configuration options.
+ * @param input the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}.
+ * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key.
+ * @param combiner the combiner to be used.
+ * @param outputKvCoder the type of the output values.
+ */
+ public static <K, VIN, VACC, VOUT> DataStream<WindowedValue<KV<K, VOUT>>> create(
+ PipelineOptions options,
+ PCollection input,
+ KeyedStream<WindowedValue<KV<K, VIN>>, K> groupedStreamByKey,
+ Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner,
+ KvCoder<K, VOUT> outputKvCoder) {
+ Preconditions.checkNotNull(options);
+
+ KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) input.getCoder();
+ FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper<>(options,
+ input.getPipeline().getCoderRegistry(), input.getWindowingStrategy(), inputKvCoder, combiner);
+
+ Coder<WindowedValue<KV<K, VOUT>>> windowedOutputElemCoder = WindowedValue.FullWindowedValueCoder.of(
+ outputKvCoder,
+ input.getWindowingStrategy().getWindowFn().windowCoder());
+
+ CoderTypeInformation<WindowedValue<KV<K, VOUT>>> outputTypeInfo =
+ new CoderTypeInformation<>(windowedOutputElemCoder);
+
+ DataStream<WindowedValue<KV<K, VOUT>>> groupedByKeyAndWindow = groupedStreamByKey
+ .transform("GroupByWindowWithCombiner",
+ new CoderTypeInformation<>(outputKvCoder),
+ windower)
+ .returns(outputTypeInfo);
+
+ return groupedByKeyAndWindow;
+ }
+
+ /**
+ * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy.
+ * This method assumes that <b>elements are already grouped by key</b>.
+ * <p/>
+ * The difference with {@link #create(PipelineOptions, PCollection, KeyedStream, Combine.KeyedCombineFn, KvCoder)}
+ * is that this method assumes no combiner function
+ * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}).
+ *
+ * @param options the general job configuration options.
+ * @param input the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}.
+ * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key.
+ */
+ public static <K, VIN> DataStream<WindowedValue<KV<K, Iterable<VIN>>>> createForIterable(
+ PipelineOptions options,
+ PCollection input,
+ KeyedStream<WindowedValue<KV<K, VIN>>, K> groupedStreamByKey) {
+ Preconditions.checkNotNull(options);
+
+ KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) input.getCoder();
+ Coder<K> keyCoder = inputKvCoder.getKeyCoder();
+ Coder<VIN> inputValueCoder = inputKvCoder.getValueCoder();
+
+ FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper(options,
+ input.getPipeline().getCoderRegistry(), input.getWindowingStrategy(), inputKvCoder, null);
+
+ Coder<Iterable<VIN>> valueIterCoder = IterableCoder.of(inputValueCoder);
+ KvCoder<K, Iterable<VIN>> outputElemCoder = KvCoder.of(keyCoder, valueIterCoder);
+
+ Coder<WindowedValue<KV<K, Iterable<VIN>>>> windowedOutputElemCoder = WindowedValue.FullWindowedValueCoder.of(
+ outputElemCoder,
+ input.getWindowingStrategy().getWindowFn().windowCoder());
+
+ CoderTypeInformation<WindowedValue<KV<K, Iterable<VIN>>>> outputTypeInfo =
+ new CoderTypeInformation<>(windowedOutputElemCoder);
+
+ DataStream<WindowedValue<KV<K, Iterable<VIN>>>> groupedByKeyAndWindow = groupedStreamByKey
+ .transform("GroupByWindow",
+ new CoderTypeInformation<>(windowedOutputElemCoder),
+ windower)
+ .returns(outputTypeInfo);
+
+ return groupedByKeyAndWindow;
+ }
+
+ public static <K, VIN, VACC, VOUT> FlinkGroupAlsoByWindowWrapper
+ createForTesting(PipelineOptions options,
+ CoderRegistry registry,
+ WindowingStrategy<KV<K, VIN>, BoundedWindow> windowingStrategy,
+ KvCoder<K, VIN> inputCoder,
+ Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner) {
+ Preconditions.checkNotNull(options);
+
+ return new FlinkGroupAlsoByWindowWrapper(options, registry, windowingStrategy, inputCoder, combiner);
+ }
+
+ private FlinkGroupAlsoByWindowWrapper(PipelineOptions options,
+ CoderRegistry registry,
+ WindowingStrategy<KV<K, VIN>, BoundedWindow> windowingStrategy,
+ KvCoder<K, VIN> inputCoder,
+ Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner) {
+ Preconditions.checkNotNull(options);
+
+ this.options = Preconditions.checkNotNull(options);
+ this.coderRegistry = Preconditions.checkNotNull(registry);
+ this.inputKvCoder = Preconditions.checkNotNull(inputCoder);//(KvCoder<K, VIN>) input.getCoder();
+ this.windowingStrategy = Preconditions.checkNotNull(windowingStrategy);//input.getWindowingStrategy();
+ this.combineFn = combiner;
+ this.operator = createGroupAlsoByWindowOperator();
+ this.chainingStrategy = ChainingStrategy.ALWAYS;
+ }
+
+ @Override
+ public void open() throws Exception {
+ super.open();
+ this.context = new ProcessContext(operator, new TimestampedCollector<>(output), this.timerInternals);
+ }
+
+ /**
+ * Create the adequate {@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn},
+ * <b> if not already created</b>.
+ * If a {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn} was provided, then
+ * a function with that combiner is created, so that elements are combined as they arrive. This is
+ * done for speed and (in most of the cases) for reduction of the per-window state.
+ */
+ private <W extends BoundedWindow> DoFn<KeyedWorkItem<K, VIN>, KV<K, VOUT>> createGroupAlsoByWindowOperator() {
+ if (this.operator == null) {
+ if (this.combineFn == null) {
+ // Thus VOUT == Iterable<VIN>
+ Coder<VIN> inputValueCoder = inputKvCoder.getValueCoder();
+
+ this.operator = (DoFn) GroupAlsoByWindowViaWindowSetDoFn.create(
+ (WindowingStrategy<?, W>) this.windowingStrategy, SystemReduceFn.<K, VIN, W>buffering(inputValueCoder));
+ } else {
+ Coder<K> inputKeyCoder = inputKvCoder.getKeyCoder();
+
+ AppliedCombineFn<K, VIN, VACC, VOUT> appliedCombineFn = AppliedCombineFn
+ .withInputCoder(combineFn, coderRegistry, inputKvCoder);
+
+ this.operator = GroupAlsoByWindowViaWindowSetDoFn.create(
+ (WindowingStrategy<?, W>) this.windowingStrategy, SystemReduceFn.<K, VIN, VACC, VOUT, W>combining(inputKeyCoder, appliedCombineFn));
+ }
+ }
+ return this.operator;
+ }
+
+ private void processKeyedWorkItem(KeyedWorkItem<K, VIN> workItem) throws Exception {
+ context.setElement(workItem, getStateInternalsForKey(workItem.key()));
+
+ // TODO: Ideally startBundle/finishBundle would be called when the operator is first used / about to be discarded.
+ operator.startBundle(context);
+ operator.processElement(context);
+ operator.finishBundle(context);
+ }
+
+ @Override
+ public void processElement(StreamRecord<WindowedValue<KV<K, VIN>>> element) throws Exception {
+ ArrayList<WindowedValue<VIN>> elements = new ArrayList<>();
+ elements.add(WindowedValue.of(element.getValue().getValue().getValue(), element.getValue().getTimestamp(),
+ element.getValue().getWindows(), element.getValue().getPane()));
+ processKeyedWorkItem(KeyedWorkItems.elementsWorkItem(element.getValue().getValue().getKey(), elements));
+ }
+
+ @Override
+ public void processWatermark(Watermark mark) throws Exception {
+ context.setCurrentInputWatermark(new Instant(mark.getTimestamp()));
+
+ Multimap<K, TimerInternals.TimerData> timers = getTimersReadyToProcess(mark.getTimestamp());
+ if (!timers.isEmpty()) {
+ for (K key : timers.keySet()) {
+ processKeyedWorkItem(KeyedWorkItems.<K, VIN>timersWorkItem(key, timers.get(key)));
+ }
+ }
+
+ /**
+ * This is to take into account the different semantics of the Watermark in Flink and
+ * in Dataflow. To understand the reasoning behind the Dataflow semantics and its
+ * watermark holding logic, see the documentation of
+ * {@link WatermarkHold#addHold(ReduceFn.ProcessValueContext, boolean)}
+ * */
+ long millis = Long.MAX_VALUE;
+ for (FlinkStateInternals state : perKeyStateInternals.values()) {
+ Instant watermarkHold = state.getWatermarkHold();
+ if (watermarkHold != null && watermarkHold.getMillis() < millis) {
+ millis = watermarkHold.getMillis();
+ }
+ }
+
+ if (mark.getTimestamp() < millis) {
+ millis = mark.getTimestamp();
+ }
+
+ context.setCurrentOutputWatermark(new Instant(millis));
+
+ // Don't forget to re-emit the watermark for further operators down the line.
+ // This is critical for jobs with multiple aggregation steps.
+ // Imagine a job with a groupByKey() on key K1, followed by a map() that changes
+ // the key K1 to K2, and another groupByKey() on K2. In this case, if the watermark
+ // is not re-emitted, the second aggregation would never be triggered, and no result
+ // will be produced.
+ output.emitWatermark(new Watermark(millis));
+ }
+
+ @Override
+ public void close() throws Exception {
+ super.close();
+ }
+
+ private void registerActiveTimer(K key, TimerInternals.TimerData timer) {
+ Set<TimerInternals.TimerData> timersForKey = activeTimers.get(key);
+ if (timersForKey == null) {
+ timersForKey = new HashSet<>();
+ }
+ timersForKey.add(timer);
+ activeTimers.put(key, timersForKey);
+ }
+
+ private void unregisterActiveTimer(K key, TimerInternals.TimerData timer) {
+ Set<TimerInternals.TimerData> timersForKey = activeTimers.get(key);
+ if (timersForKey != null) {
+ timersForKey.remove(timer);
+ if (timersForKey.isEmpty()) {
+ activeTimers.remove(key);
+ } else {
+ activeTimers.put(key, timersForKey);
+ }
+ }
+ }
+
+ /**
+ * Returns the list of timers that are ready to fire. These are the timers
+ * that are registered to be triggered at a time before the current watermark.
+ * We keep these timers in a Set, so that they are deduplicated, as the same
+ * timer can be registered multiple times.
+ */
+ private Multimap<K, TimerInternals.TimerData> getTimersReadyToProcess(long currentWatermark) {
+
+ // we keep the timers to return in a different list and launch them later
+ // because we cannot prevent a trigger from registering another trigger,
+ // which would lead to concurrent modification exception.
+ Multimap<K, TimerInternals.TimerData> toFire = HashMultimap.create();
+
+ Iterator<Map.Entry<K, Set<TimerInternals.TimerData>>> it = activeTimers.entrySet().iterator();
+ while (it.hasNext()) {
+ Map.Entry<K, Set<TimerInternals.TimerData>> keyWithTimers = it.next();
+
+ Iterator<TimerInternals.TimerData> timerIt = keyWithTimers.getValue().iterator();
+ while (timerIt.hasNext()) {
+ TimerInternals.TimerData timerData = timerIt.next();
+ if (timerData.getTimestamp().isBefore(currentWatermark)) {
+ toFire.put(keyWithTimers.getKey(), timerData);
+ timerIt.remove();
+ }
+ }
+
+ if (keyWithTimers.getValue().isEmpty()) {
+ it.remove();
+ }
+ }
+ return toFire;
+ }
+
+ /**
+ * Gets the state associated with the specified key.
+ *
+ * @param key the key whose state we want.
+ * @return The {@link FlinkStateInternals}
+ * associated with that key.
+ */
+ private FlinkStateInternals<K> getStateInternalsForKey(K key) {
+ FlinkStateInternals<K> stateInternals = perKeyStateInternals.get(key);
+ if (stateInternals == null) {
+ Coder<? extends BoundedWindow> windowCoder = this.windowingStrategy.getWindowFn().windowCoder();
+ OutputTimeFn<? super BoundedWindow> outputTimeFn = this.windowingStrategy.getWindowFn().getOutputTimeFn();
+ stateInternals = new FlinkStateInternals<>(key, inputKvCoder.getKeyCoder(), windowCoder, outputTimeFn);
+ perKeyStateInternals.put(key, stateInternals);
+ }
+ return stateInternals;
+ }
+
+ private class FlinkTimerInternals extends AbstractFlinkTimerInternals<K, VIN> {
+ @Override
+ public void setTimer(TimerData timerKey) {
+ registerActiveTimer(context.element().key(), timerKey);
+ }
+
+ @Override
+ public void deleteTimer(TimerData timerKey) {
+ unregisterActiveTimer(context.element().key(), timerKey);
+ }
+ }
+
+ private class ProcessContext extends GroupAlsoByWindowViaWindowSetDoFn<K, VIN, VOUT, ?, KeyedWorkItem<K, VIN>>.ProcessContext {
+
+ private final FlinkTimerInternals timerInternals;
+
+ private final TimestampedCollector<WindowedValue<KV<K, VOUT>>> collector;
+
+ private FlinkStateInternals<K> stateInternals;
+
+ private KeyedWorkItem<K, VIN> element;
+
+ public ProcessContext(DoFn<KeyedWorkItem<K, VIN>, KV<K, VOUT>> function,
+ TimestampedCollector<WindowedValue<KV<K, VOUT>>> outCollector,
+ FlinkTimerInternals timerInternals) {
+ function.super();
+ super.setupDelegateAggregators();
+
+ this.collector = Preconditions.checkNotNull(outCollector);
+ this.timerInternals = Preconditions.checkNotNull(timerInternals);
+ }
+
+ public void setElement(KeyedWorkItem<K, VIN> element,
+ FlinkStateInternals<K> stateForKey) {
+ this.element = element;
+ this.stateInternals = stateForKey;
+ }
+
+ public void setCurrentInputWatermark(Instant watermark) {
+ this.timerInternals.setCurrentInputWatermark(watermark);
+ }
+
+ public void setCurrentOutputWatermark(Instant watermark) {
+ this.timerInternals.setCurrentOutputWatermark(watermark);
+ }
+
+ @Override
+ public KeyedWorkItem<K, VIN> element() {
+ return this.element;
+ }
+
+ @Override
+ public Instant timestamp() {
+ throw new UnsupportedOperationException("timestamp() is not available when processing KeyedWorkItems.");
+ }
+
+ @Override
+ public PipelineOptions getPipelineOptions() {
+ // TODO: PipelineOptions need to be available on the workers.
+ // Ideally they are captured as part of the pipeline.
+ // For now, construct empty options so that StateContexts.createFromComponents
+ // will yield a valid StateContext, which is needed to support the StateContext.window().
+ if (options == null) {
+ options = new PipelineOptions() {
+ @Override
+ public <T extends PipelineOptions> T as(Class<T> kls) {
+ return null;
+ }
+
+ @Override
+ public <T extends PipelineOptions> T cloneAs(Class<T> kls) {
+ return null;
+ }
+
+ @Override
+ public Class<? extends PipelineRunner<?>> getRunner() {
+ return null;
+ }
+
+ @Override
+ public void setRunner(Class<? extends PipelineRunner<?>> kls) {
+
+ }
+
+ @Override
+ public CheckEnabled getStableUniqueNames() {
+ return null;
+ }
+
+ @Override
+ public void setStableUniqueNames(CheckEnabled enabled) {
+ }
+ };
+ }
+ return options;
+ }
+
+ @Override
+ public void output(KV<K, VOUT> output) {
+ throw new UnsupportedOperationException(
+ "output() is not available when processing KeyedWorkItems.");
+ }
+
+ @Override
+ public void outputWithTimestamp(KV<K, VOUT> output, Instant timestamp) {
+ throw new UnsupportedOperationException(
+ "outputWithTimestamp() is not available when processing KeyedWorkItems.");
+ }
+
+ @Override
+ public PaneInfo pane() {
+ throw new UnsupportedOperationException("pane() is not available when processing KeyedWorkItems.");
+ }
+
+ @Override
+ public BoundedWindow window() {
+ throw new UnsupportedOperationException(
+ "window() is not available when processing KeyedWorkItems.");
+ }
+
+ @Override
+ public WindowingInternals<KeyedWorkItem<K, VIN>, KV<K, VOUT>> windowingInternals() {
+ return new WindowingInternals<KeyedWorkItem<K, VIN>, KV<K, VOUT>>() {
+
+ @Override
+ public com.google.cloud.dataflow.sdk.util.state.StateInternals stateInternals() {
+ return stateInternals;
+ }
+
+ @Override
+ public void outputWindowedValue(KV<K, VOUT> output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+ // TODO: No need to represent timestamp twice.
+ collector.setAbsoluteTimestamp(timestamp.getMillis());
+ collector.collect(WindowedValue.of(output, timestamp, windows, pane));
+
+ }
+
+ @Override
+ public TimerInternals timerInternals() {
+ return timerInternals;
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ throw new UnsupportedOperationException("windows() is not available in Streaming mode.");
+ }
+
+ @Override
+ public PaneInfo pane() {
+ throw new UnsupportedOperationException("pane() is not available in Streaming mode.");
+ }
+
+ @Override
+ public <T> void writePCollectionViewData(TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
+ throw new RuntimeException("writePCollectionViewData() not available in Streaming mode.");
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+ throw new RuntimeException("sideInput() is not available in Streaming mode.");
+ }
+ };
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view) {
+ throw new RuntimeException("sideInput() is not supported in Streaming mode.");
+ }
+
+ @Override
+ public <T> void sideOutput(TupleTag<T> tag, T output) {
+ // ignore the side output, this can happen when a user does not register
+ // side outputs but then outputs using a freshly created TupleTag.
+ throw new RuntimeException("sideOutput() is not available when grouping by window.");
+ }
+
+ @Override
+ public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+ sideOutput(tag, output);
+ }
+
+ @Override
+ protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
+ Accumulator acc = getRuntimeContext().getAccumulator(name);
+ if (acc != null) {
+ AccumulatorHelper.compareAccumulatorTypes(name,
+ SerializableFnAggregatorWrapper.class, acc.getClass());
+ return (Aggregator<AggInputT, AggOutputT>) acc;
+ }
+
+ SerializableFnAggregatorWrapper<AggInputT, AggOutputT> accumulator =
+ new SerializableFnAggregatorWrapper<>(combiner);
+ getRuntimeContext().addAccumulator(name, accumulator);
+ return accumulator;
+ }
+ }
+
+ ////////////// Checkpointing implementation ////////////////
+
+ @Override
+ public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception {
+ StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
+ AbstractStateBackend.CheckpointStateOutputView out = getStateBackend().createCheckpointStateOutputView(checkpointId, timestamp);
+ StateCheckpointWriter writer = StateCheckpointWriter.create(out);
+ Coder<K> keyCoder = inputKvCoder.getKeyCoder();
+
+ // checkpoint the timers
+ StateCheckpointUtils.encodeTimers(activeTimers, writer, keyCoder);
+
+ // checkpoint the state
+ StateCheckpointUtils.encodeState(perKeyStateInternals, writer, keyCoder);
+
+ // checkpoint the timerInternals
+ context.timerInternals.encodeTimerInternals(context, writer,
+ inputKvCoder, windowingStrategy.getWindowFn().windowCoder());
+
+ taskState.setOperatorState(out.closeAndGetHandle());
+ return taskState;
+ }
+
+ @Override
+ public void restoreState(StreamTaskState taskState, long recoveryTimestamp) throws Exception {
+ super.restoreState(taskState, recoveryTimestamp);
+
+ final ClassLoader userClassloader = getUserCodeClassloader();
+
+ Coder<? extends BoundedWindow> windowCoder = this.windowingStrategy.getWindowFn().windowCoder();
+ Coder<K> keyCoder = inputKvCoder.getKeyCoder();
+
+ @SuppressWarnings("unchecked")
+ StateHandle<DataInputView> inputState = (StateHandle<DataInputView>) taskState.getOperatorState();
+ DataInputView in = inputState.getState(userClassloader);
+ StateCheckpointReader reader = new StateCheckpointReader(in);
+
+ // restore the timers
+ this.activeTimers = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder);
+
+ // restore the state
+ this.perKeyStateInternals = StateCheckpointUtils.decodeState(
+ reader, windowingStrategy.getOutputTimeFn(), keyCoder, windowCoder, userClassloader);
+
+ // restore the timerInternals.
+ this.timerInternals.restoreTimerInternals(reader, inputKvCoder, windowCoder);
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
index 24f6d40..d01cf81 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
@@ -34,31 +34,31 @@ import org.apache.flink.streaming.api.datastream.KeyedStream;
* */
public class FlinkGroupByKeyWrapper {
- /**
- * Just an auxiliary interface to bypass the fact that java anonymous classes cannot implement
- * multiple interfaces.
- */
- private interface KeySelectorWithQueryableResultType<K, V> extends KeySelector<WindowedValue<KV<K, V>>, K>, ResultTypeQueryable<K> {
- }
+ /**
+ * Just an auxiliary interface to bypass the fact that java anonymous classes cannot implement
+ * multiple interfaces.
+ */
+ private interface KeySelectorWithQueryableResultType<K, V> extends KeySelector<WindowedValue<KV<K, V>>, K>, ResultTypeQueryable<K> {
+ }
- public static <K, V> KeyedStream<WindowedValue<KV<K, V>>, K> groupStreamByKey(DataStream<WindowedValue<KV<K, V>>> inputDataStream, KvCoder<K, V> inputKvCoder) {
- final Coder<K> keyCoder = inputKvCoder.getKeyCoder();
- final TypeInformation<K> keyTypeInfo = new CoderTypeInformation<>(keyCoder);
- final boolean isKeyVoid = keyCoder instanceof VoidCoder;
+ public static <K, V> KeyedStream<WindowedValue<KV<K, V>>, K> groupStreamByKey(DataStream<WindowedValue<KV<K, V>>> inputDataStream, KvCoder<K, V> inputKvCoder) {
+ final Coder<K> keyCoder = inputKvCoder.getKeyCoder();
+ final TypeInformation<K> keyTypeInfo = new CoderTypeInformation<>(keyCoder);
+ final boolean isKeyVoid = keyCoder instanceof VoidCoder;
- return inputDataStream.keyBy(
- new KeySelectorWithQueryableResultType<K, V>() {
+ return inputDataStream.keyBy(
+ new KeySelectorWithQueryableResultType<K, V>() {
- @Override
- public K getKey(WindowedValue<KV<K, V>> value) throws Exception {
- return isKeyVoid ? (K) VoidCoderTypeSerializer.VoidValue.INSTANCE :
- value.getValue().getKey();
- }
+ @Override
+ public K getKey(WindowedValue<KV<K, V>> value) throws Exception {
+ return isKeyVoid ? (K) VoidCoderTypeSerializer.VoidValue.INSTANCE :
+ value.getValue().getKey();
+ }
- @Override
- public TypeInformation<K> getProducedType() {
- return keyTypeInfo;
- }
- });
- }
+ @Override
+ public TypeInformation<K> getProducedType() {
+ return keyTypeInfo;
+ }
+ });
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
index d65cbc3..066a55c 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
@@ -33,43 +33,43 @@ import java.util.Map;
* */
public class FlinkParDoBoundMultiWrapper<IN, OUT> extends FlinkAbstractParDoWrapper<IN, OUT, RawUnionValue> {
- private final TupleTag<?> mainTag;
- private final Map<TupleTag<?>, Integer> outputLabels;
+ private final TupleTag<?> mainTag;
+ private final Map<TupleTag<?>, Integer> outputLabels;
- public FlinkParDoBoundMultiWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUT> doFn, TupleTag<?> mainTag, Map<TupleTag<?>, Integer> tagsToLabels) {
- super(options, windowingStrategy, doFn);
- this.mainTag = Preconditions.checkNotNull(mainTag);
- this.outputLabels = Preconditions.checkNotNull(tagsToLabels);
- }
+ public FlinkParDoBoundMultiWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUT> doFn, TupleTag<?> mainTag, Map<TupleTag<?>, Integer> tagsToLabels) {
+ super(options, windowingStrategy, doFn);
+ this.mainTag = Preconditions.checkNotNull(mainTag);
+ this.outputLabels = Preconditions.checkNotNull(tagsToLabels);
+ }
- @Override
- public void outputWithTimestampHelper(WindowedValue<IN> inElement, OUT output, Instant timestamp, Collector<WindowedValue<RawUnionValue>> collector) {
- checkTimestamp(inElement, timestamp);
- Integer index = outputLabels.get(mainTag);
- collector.collect(makeWindowedValue(
- new RawUnionValue(index, output),
- timestamp,
- inElement.getWindows(),
- inElement.getPane()));
- }
+ @Override
+ public void outputWithTimestampHelper(WindowedValue<IN> inElement, OUT output, Instant timestamp, Collector<WindowedValue<RawUnionValue>> collector) {
+ checkTimestamp(inElement, timestamp);
+ Integer index = outputLabels.get(mainTag);
+ collector.collect(makeWindowedValue(
+ new RawUnionValue(index, output),
+ timestamp,
+ inElement.getWindows(),
+ inElement.getPane()));
+ }
- @Override
- public <T> void sideOutputWithTimestampHelper(WindowedValue<IN> inElement, T output, Instant timestamp, Collector<WindowedValue<RawUnionValue>> collector, TupleTag<T> tag) {
- checkTimestamp(inElement, timestamp);
- Integer index = outputLabels.get(tag);
- if (index != null) {
- collector.collect(makeWindowedValue(
- new RawUnionValue(index, output),
- timestamp,
- inElement.getWindows(),
- inElement.getPane()));
- }
- }
+ @Override
+ public <T> void sideOutputWithTimestampHelper(WindowedValue<IN> inElement, T output, Instant timestamp, Collector<WindowedValue<RawUnionValue>> collector, TupleTag<T> tag) {
+ checkTimestamp(inElement, timestamp);
+ Integer index = outputLabels.get(tag);
+ if (index != null) {
+ collector.collect(makeWindowedValue(
+ new RawUnionValue(index, output),
+ timestamp,
+ inElement.getWindows(),
+ inElement.getPane()));
+ }
+ }
- @Override
- public WindowingInternals<IN, OUT> windowingInternalsHelper(WindowedValue<IN> inElement, Collector<WindowedValue<RawUnionValue>> outCollector) {
- throw new RuntimeException("FlinkParDoBoundMultiWrapper is just an internal operator serving as " +
- "an intermediate transformation for the ParDo.BoundMulti translation. windowingInternals() " +
- "is not available in this class.");
- }
+ @Override
+ public WindowingInternals<IN, OUT> windowingInternalsHelper(WindowedValue<IN> inElement, Collector<WindowedValue<RawUnionValue>> outCollector) {
+ throw new RuntimeException("FlinkParDoBoundMultiWrapper is just an internal operator serving as " +
+ "an intermediate transformation for the ParDo.BoundMulti translation. windowingInternals() " +
+ "is not available in this class.");
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
index b0d8a76..b3a7090 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
@@ -35,64 +35,64 @@ import java.util.*;
* */
public class FlinkParDoBoundWrapper<IN, OUT> extends FlinkAbstractParDoWrapper<IN, OUT, OUT> {
- public FlinkParDoBoundWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUT> doFn) {
- super(options, windowingStrategy, doFn);
- }
+ public FlinkParDoBoundWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUT> doFn) {
+ super(options, windowingStrategy, doFn);
+ }
- @Override
- public void outputWithTimestampHelper(WindowedValue<IN> inElement, OUT output, Instant timestamp, Collector<WindowedValue<OUT>> collector) {
- checkTimestamp(inElement, timestamp);
- collector.collect(makeWindowedValue(
- output,
- timestamp,
- inElement.getWindows(),
- inElement.getPane()));
- }
+ @Override
+ public void outputWithTimestampHelper(WindowedValue<IN> inElement, OUT output, Instant timestamp, Collector<WindowedValue<OUT>> collector) {
+ checkTimestamp(inElement, timestamp);
+ collector.collect(makeWindowedValue(
+ output,
+ timestamp,
+ inElement.getWindows(),
+ inElement.getPane()));
+ }
- @Override
- public <T> void sideOutputWithTimestampHelper(WindowedValue<IN> inElement, T output, Instant timestamp, Collector<WindowedValue<OUT>> outCollector, TupleTag<T> tag) {
- // ignore the side output, this can happen when a user does not register
- // side outputs but then outputs using a freshly created TupleTag.
- throw new RuntimeException("sideOutput() not not available in ParDo.Bound().");
- }
+ @Override
+ public <T> void sideOutputWithTimestampHelper(WindowedValue<IN> inElement, T output, Instant timestamp, Collector<WindowedValue<OUT>> outCollector, TupleTag<T> tag) {
+ // ignore the side output, this can happen when a user does not register
+ // side outputs but then outputs using a freshly created TupleTag.
+ throw new RuntimeException("sideOutput() not not available in ParDo.Bound().");
+ }
- @Override
- public WindowingInternals<IN, OUT> windowingInternalsHelper(final WindowedValue<IN> inElement, final Collector<WindowedValue<OUT>> collector) {
- return new WindowingInternals<IN, OUT>() {
- @Override
- public StateInternals stateInternals() {
- throw new NullPointerException("StateInternals are not available for ParDo.Bound().");
- }
+ @Override
+ public WindowingInternals<IN, OUT> windowingInternalsHelper(final WindowedValue<IN> inElement, final Collector<WindowedValue<OUT>> collector) {
+ return new WindowingInternals<IN, OUT>() {
+ @Override
+ public StateInternals stateInternals() {
+ throw new NullPointerException("StateInternals are not available for ParDo.Bound().");
+ }
- @Override
- public void outputWindowedValue(OUT output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
- collector.collect(makeWindowedValue(output, timestamp, windows, pane));
- }
+ @Override
+ public void outputWindowedValue(OUT output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+ collector.collect(makeWindowedValue(output, timestamp, windows, pane));
+ }
- @Override
- public TimerInternals timerInternals() {
- throw new NullPointerException("TimeInternals are not available for ParDo.Bound().");
- }
+ @Override
+ public TimerInternals timerInternals() {
+ throw new NullPointerException("TimeInternals are not available for ParDo.Bound().");
+ }
- @Override
- public Collection<? extends BoundedWindow> windows() {
- return inElement.getWindows();
- }
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ return inElement.getWindows();
+ }
- @Override
- public PaneInfo pane() {
- return inElement.getPane();
- }
+ @Override
+ public PaneInfo pane() {
+ return inElement.getPane();
+ }
- @Override
- public <T> void writePCollectionViewData(TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
- throw new RuntimeException("writePCollectionViewData() not supported in Streaming mode.");
- }
+ @Override
+ public <T> void writePCollectionViewData(TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
+ throw new RuntimeException("writePCollectionViewData() not supported in Streaming mode.");
+ }
- @Override
- public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
- throw new RuntimeException("sideInput() not implemented.");
- }
- };
- }
+ @Override
+ public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+ throw new RuntimeException("sideInput() not implemented.");
+ }
+ };
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
index dc8e05a..39770c9 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
@@ -34,30 +34,30 @@ import java.util.List;
*/
public class FlinkStreamingCreateFunction<IN, OUT> implements FlatMapFunction<IN, WindowedValue<OUT>> {
- private final List<byte[]> elements;
- private final Coder<OUT> coder;
-
- public FlinkStreamingCreateFunction(List<byte[]> elements, Coder<OUT> coder) {
- this.elements = elements;
- this.coder = coder;
- }
-
- @Override
- public void flatMap(IN value, Collector<WindowedValue<OUT>> out) throws Exception {
-
- @SuppressWarnings("unchecked")
- OUT voidValue = (OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE;
- for (byte[] element : elements) {
- ByteArrayInputStream bai = new ByteArrayInputStream(element);
- OUT outValue = coder.decode(bai, Coder.Context.OUTER);
-
- if (outValue == null) {
- out.collect(WindowedValue.of(voidValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
- } else {
- out.collect(WindowedValue.of(outValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
- }
- }
-
- out.close();
- }
+ private final List<byte[]> elements;
+ private final Coder<OUT> coder;
+
+ public FlinkStreamingCreateFunction(List<byte[]> elements, Coder<OUT> coder) {
+ this.elements = elements;
+ this.coder = coder;
+ }
+
+ @Override
+ public void flatMap(IN value, Collector<WindowedValue<OUT>> out) throws Exception {
+
+ @SuppressWarnings("unchecked")
+ OUT voidValue = (OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE;
+ for (byte[] element : elements) {
+ ByteArrayInputStream bai = new ByteArrayInputStream(element);
+ OUT outValue = coder.decode(bai, Coder.Context.OUTER);
+
+ if (outValue == null) {
+ out.collect(WindowedValue.of(voidValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
+ } else {
+ out.collect(WindowedValue.of(outValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
+ }
+ }
+
+ out.close();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java
index 699d256..4d6f4e2 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java
@@ -31,50 +31,50 @@ import java.util.List;
* */
public class UnboundedFlinkSource<T, C extends UnboundedSource.CheckpointMark> extends UnboundedSource<T, C> {
- private final PipelineOptions options;
- private final RichParallelSourceFunction<T> flinkSource;
+ private final PipelineOptions options;
+ private final RichParallelSourceFunction<T> flinkSource;
- public UnboundedFlinkSource(PipelineOptions pipelineOptions, RichParallelSourceFunction<T> source) {
- if(!pipelineOptions.getRunner().equals(FlinkPipelineRunner.class)) {
- throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
- }
- options = Preconditions.checkNotNull(pipelineOptions);
- flinkSource = Preconditions.checkNotNull(source);
- validate();
- }
+ public UnboundedFlinkSource(PipelineOptions pipelineOptions, RichParallelSourceFunction<T> source) {
+ if(!pipelineOptions.getRunner().equals(FlinkPipelineRunner.class)) {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
+ options = Preconditions.checkNotNull(pipelineOptions);
+ flinkSource = Preconditions.checkNotNull(source);
+ validate();
+ }
- public RichParallelSourceFunction<T> getFlinkSource() {
- return this.flinkSource;
- }
+ public RichParallelSourceFunction<T> getFlinkSource() {
+ return this.flinkSource;
+ }
- @Override
- public List<? extends UnboundedSource<T, C>> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception {
- throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
- }
+ @Override
+ public List<? extends UnboundedSource<T, C>> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
- @Override
- public UnboundedReader<T> createReader(PipelineOptions options, @Nullable C checkpointMark) {
- throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
- }
+ @Override
+ public UnboundedReader<T> createReader(PipelineOptions options, @Nullable C checkpointMark) {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
- @Nullable
- @Override
- public Coder<C> getCheckpointMarkCoder() {
- throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
- }
+ @Nullable
+ @Override
+ public Coder<C> getCheckpointMarkCoder() {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
- @Override
- public void validate() {
- Preconditions.checkNotNull(options);
- Preconditions.checkNotNull(flinkSource);
- if(!options.getRunner().equals(FlinkPipelineRunner.class)) {
- throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
- }
- }
+ @Override
+ public void validate() {
+ Preconditions.checkNotNull(options);
+ Preconditions.checkNotNull(flinkSource);
+ if(!options.getRunner().equals(FlinkPipelineRunner.class)) {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
+ }
- @Override
- public Coder<T> getDefaultOutputCoder() {
- throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
- }
+ @Override
+ public Coder<T> getDefaultOutputCoder() {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
}
[05/50] [abbrv] incubator-beam git commit: [runner] add streaming
support with checkpointing
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
new file mode 100644
index 0000000..0a0e301
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation.wrappers.streaming;
+
+import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.util.*;
+import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.KeyedStream;
+
+public class FlinkGroupByKeyWrapper {
+
+ /**
+ * Just an auxiliary interface to bypass the fact that java anonymous classes cannot implement
+ * multiple interfaces.
+ */
+ private interface KeySelectorWithQueryableResultType<K, V> extends KeySelector<WindowedValue<KV<K, V>>, K>, ResultTypeQueryable<K> {
+ }
+
+ public static <K, V> KeyedStream<WindowedValue<KV<K, V>>, K> groupStreamByKey(DataStream<WindowedValue<KV<K, V>>> inputDataStream, KvCoder<K, V> inputKvCoder) {
+ final Coder<K> keyCoder = inputKvCoder.getKeyCoder();
+ final TypeInformation<K> keyTypeInfo = new CoderTypeInformation<>(keyCoder);
+
+ return inputDataStream.keyBy(
+ new KeySelectorWithQueryableResultType<K, V>() {
+
+ @Override
+ public K getKey(WindowedValue<KV<K, V>> value) throws Exception {
+ return value.getValue().getKey();
+ }
+
+ @Override
+ public TypeInformation<K> getProducedType() {
+ return keyTypeInfo;
+ }
+ });
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
new file mode 100644
index 0000000..200c397
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation.wrappers.streaming;
+
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.util.WindowingInternals;
+import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+
+import java.util.Map;
+
+public class FlinkParDoBoundMultiWrapper<IN, OUT> extends FlinkAbstractParDoWrapper<IN, OUT, RawUnionValue> {
+
+ private final TupleTag<?> mainTag;
+ private final Map<TupleTag<?>, Integer> outputLabels;
+
+ public FlinkParDoBoundMultiWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUT> doFn, TupleTag<?> mainTag, Map<TupleTag<?>, Integer> tagsToLabels) {
+ super(options, windowingStrategy, doFn);
+ this.mainTag = Preconditions.checkNotNull(mainTag);
+ this.outputLabels = Preconditions.checkNotNull(tagsToLabels);
+ }
+
+ @Override
+ public void outputWithTimestampHelper(WindowedValue<IN> inElement, OUT output, Instant timestamp, Collector<WindowedValue<RawUnionValue>> collector) {
+ checkTimestamp(inElement, timestamp);
+ Integer index = outputLabels.get(mainTag);
+ collector.collect(makeWindowedValue(
+ new RawUnionValue(index, output),
+ timestamp,
+ inElement.getWindows(),
+ inElement.getPane()));
+ }
+
+ @Override
+ public <T> void sideOutputWithTimestampHelper(WindowedValue<IN> inElement, T output, Instant timestamp, Collector<WindowedValue<RawUnionValue>> collector, TupleTag<T> tag) {
+ checkTimestamp(inElement, timestamp);
+ Integer index = outputLabels.get(tag);
+ if (index != null) {
+ collector.collect(makeWindowedValue(
+ new RawUnionValue(index, output),
+ timestamp,
+ inElement.getWindows(),
+ inElement.getPane()));
+ }
+ }
+
+ @Override
+ public WindowingInternals<IN, OUT> windowingInternalsHelper(WindowedValue<IN> inElement, Collector<WindowedValue<RawUnionValue>> outCollector) {
+ throw new RuntimeException("FlinkParDoBoundMultiWrapper is just an internal operator serving as " +
+ "an intermediate transformation for the ParDo.BoundMulti translation. windowingInternals() " +
+ "is not available in this class.");
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
new file mode 100644
index 0000000..18d4249
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation.wrappers.streaming;
+
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
+import com.google.cloud.dataflow.sdk.util.*;
+import com.google.cloud.dataflow.sdk.util.state.StateInternals;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.util.*;
+
+public class FlinkParDoBoundWrapper<IN, OUT> extends FlinkAbstractParDoWrapper<IN, OUT, OUT> {
+
+ public FlinkParDoBoundWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUT> doFn) {
+ super(options, windowingStrategy, doFn);
+ }
+
+ @Override
+ public void outputWithTimestampHelper(WindowedValue<IN> inElement, OUT output, Instant timestamp, Collector<WindowedValue<OUT>> collector) {
+ checkTimestamp(inElement, timestamp);
+ collector.collect(makeWindowedValue(
+ output,
+ timestamp,
+ inElement.getWindows(),
+ inElement.getPane()));
+ }
+
+ @Override
+ public <T> void sideOutputWithTimestampHelper(WindowedValue<IN> inElement, T output, Instant timestamp, Collector<WindowedValue<OUT>> outCollector, TupleTag<T> tag) {
+ // ignore the side output, this can happen when a user does not register
+ // side outputs but then outputs using a freshly created TupleTag.
+ throw new RuntimeException("sideOutput() not not available in ParDo.Bound().");
+ }
+
+ @Override
+ public WindowingInternals<IN, OUT> windowingInternalsHelper(final WindowedValue<IN> inElement, final Collector<WindowedValue<OUT>> collector) {
+ return new WindowingInternals<IN, OUT>() {
+ @Override
+ public StateInternals stateInternals() {
+ throw new NullPointerException("StateInternals are not available for ParDo.Bound().");
+ }
+
+ @Override
+ public void outputWindowedValue(OUT output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+ collector.collect(makeWindowedValue(output, timestamp, windows, pane));
+ }
+
+ @Override
+ public TimerInternals timerInternals() {
+ throw new NullPointerException("TimeInternals are not available for ParDo.Bound().");
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ return inElement.getWindows();
+ }
+
+ @Override
+ public PaneInfo pane() {
+ return inElement.getPane();
+ }
+
+ @Override
+ public <T> void writePCollectionViewData(TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
+ throw new RuntimeException("writePCollectionViewData() not supported in Streaming mode.");
+ }
+ };
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java
new file mode 100644
index 0000000..17e0746
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.io;
+
+import com.dataartisans.flink.dataflow.FlinkPipelineRunner;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.io.UnboundedSource;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+
+import javax.annotation.Nullable;
+import java.util.List;
+
+public class UnboundedFlinkSource<T, C extends UnboundedSource.CheckpointMark> extends UnboundedSource<T, C> {
+
+ private final PipelineOptions options;
+ private final RichParallelSourceFunction<T> flinkSource;
+
+ public UnboundedFlinkSource(PipelineOptions pipelineOptions, RichParallelSourceFunction<T> source) {
+ if(!pipelineOptions.getRunner().equals(FlinkPipelineRunner.class)) {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
+ options = Preconditions.checkNotNull(pipelineOptions);
+ flinkSource = Preconditions.checkNotNull(source);
+ validate();
+ }
+
+ public RichParallelSourceFunction<T> getFlinkSource() {
+ return this.flinkSource;
+ }
+
+ @Override
+ public List<? extends UnboundedSource<T, C>> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
+
+ @Override
+ public UnboundedReader<T> createReader(PipelineOptions options, @Nullable C checkpointMark) {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
+
+ @Nullable
+ @Override
+ public Coder<C> getCheckpointMarkCoder() {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
+
+
+ @Override
+ public void validate() {
+ Preconditions.checkNotNull(options);
+ Preconditions.checkNotNull(flinkSource);
+ if(!options.getRunner().equals(FlinkPipelineRunner.class)) {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
+ }
+
+ @Override
+ public Coder<T> getDefaultOutputCoder() {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java
new file mode 100644
index 0000000..2b0d6dc
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java
@@ -0,0 +1,228 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.io;
+
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.io.UnboundedSource;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Serializable;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Collections;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+public class UnboundedSocketSource<C extends UnboundedSource.CheckpointMark> extends UnboundedSource<String, C> {
+
+ private static final Coder<String> DEFAULT_SOCKET_CODER = StringUtf8Coder.of();
+
+ private static final long serialVersionUID = 1L;
+
+ private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 500;
+
+ private static final int CONNECTION_TIMEOUT_TIME = 0;
+
+ private final String hostname;
+ private final int port;
+ private final char delimiter;
+ private final long maxNumRetries;
+ private final long delayBetweenRetries;
+
+ public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries) {
+ this(hostname, port, delimiter, maxNumRetries, DEFAULT_CONNECTION_RETRY_SLEEP);
+ }
+
+ public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries, long delayBetweenRetries) {
+ this.hostname = hostname;
+ this.port = port;
+ this.delimiter = delimiter;
+ this.maxNumRetries = maxNumRetries;
+ this.delayBetweenRetries = delayBetweenRetries;
+ }
+
+ public String getHostname() {
+ return this.hostname;
+ }
+
+ public int getPort() {
+ return this.port;
+ }
+
+ public char getDelimiter() {
+ return this.delimiter;
+ }
+
+ public long getMaxNumRetries() {
+ return this.maxNumRetries;
+ }
+
+ public long getDelayBetweenRetries() {
+ return this.delayBetweenRetries;
+ }
+
+ @Override
+ public List<? extends UnboundedSource<String, C>> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception {
+ return Collections.<UnboundedSource<String, C>>singletonList(this);
+ }
+
+ @Override
+ public UnboundedReader<String> createReader(PipelineOptions options, @Nullable C checkpointMark) {
+ return new UnboundedSocketReader(this);
+ }
+
+ @Nullable
+ @Override
+ public Coder getCheckpointMarkCoder() {
+ // Flink and Dataflow have different checkpointing mechanisms.
+ // In our case we do not need a coder.
+ return null;
+ }
+
+ @Override
+ public void validate() {
+ checkArgument(port > 0 && port < 65536, "port is out of range");
+ checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), or -1 (infinite retries)");
+ checkArgument(delayBetweenRetries >= 0, "delayBetweenRetries must be zero or positive");
+ }
+
+ @Override
+ public Coder getDefaultOutputCoder() {
+ return DEFAULT_SOCKET_CODER;
+ }
+
+ public static class UnboundedSocketReader extends UnboundedSource.UnboundedReader<String> implements Serializable {
+
+ private static final long serialVersionUID = 7526472295622776147L;
+ private static final Logger LOG = LoggerFactory.getLogger(UnboundedSocketReader.class);
+
+ private final UnboundedSocketSource source;
+
+ private Socket socket;
+ private BufferedReader reader;
+
+ private boolean isRunning;
+
+ private String currentRecord;
+
+ public UnboundedSocketReader(UnboundedSocketSource source) {
+ this.source = source;
+ }
+
+ private void openConnection() throws IOException {
+ this.socket = new Socket();
+ this.socket.connect(new InetSocketAddress(this.source.getHostname(), this.source.getPort()), CONNECTION_TIMEOUT_TIME);
+ this.reader = new BufferedReader(new InputStreamReader(this.socket.getInputStream()));
+ this.isRunning = true;
+ }
+
+ @Override
+ public boolean start() throws IOException {
+ int attempt = 0;
+ while (!isRunning) {
+ try {
+ openConnection();
+ LOG.info("Connected to server socket " + this.source.getHostname() + ':' + this.source.getPort());
+
+ return advance();
+ } catch (IOException e) {
+ LOG.info("Lost connection to server socket " + this.source.getHostname() + ':' + this.source.getPort() + ". Retrying in " + this.source.getDelayBetweenRetries() + " msecs...");
+
+ if (this.source.getMaxNumRetries() == -1 || attempt++ < this.source.getMaxNumRetries()) {
+ try {
+ Thread.sleep(this.source.getDelayBetweenRetries());
+ } catch (InterruptedException e1) {
+ e1.printStackTrace();
+ }
+ } else {
+ this.isRunning = false;
+ break;
+ }
+ }
+ }
+ LOG.error("Unable to connect to host " + this.source.getHostname() + " : " + this.source.getPort());
+ return false;
+ }
+
+ @Override
+ public boolean advance() throws IOException {
+ final StringBuilder buffer = new StringBuilder();
+ int data;
+ while (isRunning && (data = reader.read()) != -1) {
+ // check if the string is complete
+ if (data != this.source.getDelimiter()) {
+ buffer.append((char) data);
+ } else {
+ if (buffer.length() > 0 && buffer.charAt(buffer.length() - 1) == '\r') {
+ buffer.setLength(buffer.length() - 1);
+ }
+ this.currentRecord = buffer.toString();
+ buffer.setLength(0);
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public byte[] getCurrentRecordId() throws NoSuchElementException {
+ return new byte[0];
+ }
+
+ @Override
+ public String getCurrent() throws NoSuchElementException {
+ return this.currentRecord;
+ }
+
+ @Override
+ public Instant getCurrentTimestamp() throws NoSuchElementException {
+ return Instant.now();
+ }
+
+ @Override
+ public void close() throws IOException {
+ this.reader.close();
+ this.socket.close();
+ this.isRunning = false;
+ LOG.info("Closed connection to server socket at " + this.source.getHostname() + ":" + this.source.getPort() + ".");
+ }
+
+ @Override
+ public Instant getWatermark() {
+ return Instant.now();
+ }
+
+ @Override
+ public CheckpointMark getCheckpointMark() {
+ return null;
+ }
+
+ @Override
+ public UnboundedSource<String, ?> getCurrentSource() {
+ return this.source;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
new file mode 100644
index 0000000..3e248a6
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
@@ -0,0 +1,120 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.io;
+
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.io.UnboundedSource;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import org.apache.flink.streaming.api.functions.source.EventTimeSourceFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.operators.Triggerable;
+import org.joda.time.Instant;
+
+import java.util.Collection;
+
+public class UnboundedSourceWrapper<T> extends RichSourceFunction<WindowedValue<T>> implements EventTimeSourceFunction<WindowedValue<T>>, Triggerable {
+
+ private final String name;
+ private final UnboundedSource.UnboundedReader<T> reader;
+
+ private StreamingRuntimeContext runtime = null;
+ private StreamSource.ManualWatermarkContext<T> context = null;
+
+ private volatile boolean isRunning = false;
+
+ public UnboundedSourceWrapper(PipelineOptions options, Read.Unbounded<T> transform) {
+ this.name = transform.getName();
+ this.reader = transform.getSource().createReader(options, null);
+ }
+
+ public String getName() {
+ return this.name;
+ }
+
+ WindowedValue<T> makeWindowedValue(
+ T output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+ if (timestamp == null) {
+ timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+ }
+ return WindowedValue.of(output, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING);
+ }
+
+ @Override
+ public void run(SourceContext<WindowedValue<T>> ctx) throws Exception {
+ if (!(ctx instanceof StreamSource.ManualWatermarkContext)) {
+ throw new RuntimeException("We assume that all sources in Dataflow are EventTimeSourceFunction. " +
+ "Apparently " + this.name + " is not. Probably you should consider writing your own Wrapper for this source.");
+ }
+
+ context = (StreamSource.ManualWatermarkContext<T>) ctx;
+ runtime = (StreamingRuntimeContext) getRuntimeContext();
+
+ this.isRunning = reader.start();
+ setNextWatermarkTimer(this.runtime);
+
+ while (isRunning) {
+
+ // get it and its timestamp from the source
+ T item = reader.getCurrent();
+ Instant timestamp = reader.getCurrentTimestamp();
+
+ long milliseconds = timestamp.getMillis();
+
+ // write it to the output collector
+ synchronized (ctx.getCheckpointLock()) {
+ ctx.collectWithTimestamp(makeWindowedValue(item, timestamp, null, PaneInfo.NO_FIRING), milliseconds);
+ }
+
+ // try to go to the next record
+ this.isRunning = reader.advance();
+ }
+ }
+
+ @Override
+ public void cancel() {
+ isRunning = false;
+ }
+
+ @Override
+ public void trigger(long timestamp) throws Exception {
+ if (this.isRunning) {
+ synchronized (context.getCheckpointLock()) {
+ long watermarkMillis = this.reader.getWatermark().getMillis();
+ context.emitWatermark(new Watermark(watermarkMillis));
+ }
+ setNextWatermarkTimer(this.runtime);
+ }
+ }
+
+ private void setNextWatermarkTimer(StreamingRuntimeContext runtime) {
+ if (this.isRunning) {
+ long watermarkInterval = runtime.getExecutionConfig().getAutoWatermarkInterval();
+ long timeToNextWatermark = getTimeToNextWaternark(watermarkInterval);
+ runtime.registerTimer(timeToNextWatermark, this);
+ }
+ }
+
+ private long getTimeToNextWaternark(long watermarkInterval) {
+ return System.currentTimeMillis() + watermarkInterval;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
new file mode 100644
index 0000000..4401eb3
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
@@ -0,0 +1,139 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.state;
+
+import com.dataartisans.flink.dataflow.translation.types.CoderTypeSerializer;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.util.TimerInternals;
+import com.google.cloud.dataflow.sdk.util.TimerOrElement;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.values.KV;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+public abstract class AbstractFlinkTimerInternals<K, VIN> implements TimerInternals, Serializable {
+
+ private TimerOrElement<WindowedValue<KV<K, VIN>>> element;
+
+ private Instant currentWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE;
+
+ public TimerOrElement<WindowedValue<KV<K, VIN>>> getElement() {
+ return this.element;
+ }
+
+ public void setElement(TimerOrElement<WindowedValue<KV<K, VIN>>> value) {
+ this.element = value;
+ }
+
+ public void setCurrentWatermark(Instant watermark) {
+ checkIfValidWatermark(watermark);
+ this.currentWatermark = watermark;
+ }
+
+ private void setCurrentWatermarkAfterRecovery(Instant watermark) {
+ if(!currentWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) {
+ throw new RuntimeException("Explicitly setting the watermark is only allowed on " +
+ "initialization after recovery from a node failure. Apparently this is not " +
+ "the case here as the watermark is already set.");
+ }
+ this.currentWatermark = watermark;
+ }
+
+ @Override
+ public void setTimer(com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData timerKey) {
+ K key = element.isTimer() ? (K) element.key() : element.element().getValue().getKey();
+ registerTimer(key, timerKey);
+ }
+
+ protected abstract void registerTimer(K key, TimerData timerKey);
+
+ @Override
+ public void deleteTimer(com.google.cloud.dataflow.sdk.util.TimerInternals.TimerData timerKey) {
+ K key = element.isTimer() ? (K) element.key() : element.element().getValue().getKey();
+ unregisterTimer(key, timerKey);
+ }
+
+ protected abstract void unregisterTimer(K key, TimerData timerKey);
+
+ @Override
+ public Instant currentProcessingTime() {
+ return Instant.now();
+ }
+
+ @Override
+ public Instant currentWatermarkTime() {
+ return this.currentWatermark;
+ }
+
+ private void checkIfValidWatermark(Instant newWatermark) {
+ if (currentWatermark.isAfter(newWatermark)) {
+ throw new IllegalArgumentException(String.format(
+ "Cannot set current watermark to %s. Newer watermarks " +
+ "must be no earlier than the current one (%s).",
+ newWatermark, this.currentWatermark));
+ }
+ }
+
+ public void encodeTimerInternals(DoFn.ProcessContext context,
+ StateCheckpointWriter writer,
+ KvCoder<K, VIN> kvCoder,
+ Coder<? extends BoundedWindow> windowCoder) throws IOException {
+ if (context == null) {
+ throw new RuntimeException("The Context has not been initialized.");
+ }
+
+ if (element != null && !element.isTimer()) {
+ // create the element coder
+ WindowedValue.WindowedValueCoder<KV<K, VIN>> elementCoder = WindowedValue
+ .getFullCoder(kvCoder, windowCoder);
+
+ CoderTypeSerializer<WindowedValue<KV<K, VIN>>> serializer =
+ new CoderTypeSerializer<>(elementCoder);
+
+ writer.writeByte((byte) 1);
+ writer.serializeObject(element.element(), serializer);
+ } else {
+ // just setting a flag to 0, meaning that there is no value.
+ writer.writeByte((byte) 0);
+ }
+ writer.setTimestamp(currentWatermark);
+ }
+
+ public void restoreTimerInternals(StateCheckpointReader reader,
+ KvCoder<K, VIN> kvCoder,
+ Coder<? extends BoundedWindow> windowCoder) throws IOException {
+
+ boolean isSet = (reader.getByte() == (byte) 1);
+ if (!isSet) {
+ this.element = null;
+ } else {
+ WindowedValue.WindowedValueCoder<KV<K, VIN>> elementCoder = WindowedValue
+ .getFullCoder(kvCoder, windowCoder);
+
+ CoderTypeSerializer<WindowedValue<KV<K, VIN>>> serializer =
+ new CoderTypeSerializer<>(elementCoder);
+
+ WindowedValue<KV<K, VIN>> elem = reader.deserializeObject(serializer);
+ this.element = TimerOrElement.element(elem);
+ }
+ setCurrentWatermarkAfterRecovery(reader.getTimestamp());
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java
new file mode 100644
index 0000000..03b8bb5
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java
@@ -0,0 +1,533 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.state;
+
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.util.state.*;
+import com.google.protobuf.ByteString;
+import org.apache.flink.util.InstantiationUtil;
+import org.joda.time.Instant;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.*;
+
+public class FlinkStateInternals<K> extends MergingStateInternals {
+
+ private final K key;
+
+ private final Coder<K> keyCoder;
+
+ private final Combine.KeyedCombineFn<K, ?, ?, ?> combineFn;
+
+ private final Coder<? extends BoundedWindow> windowCoder;
+
+ private Instant watermarkHoldAccessor;
+
+ public FlinkStateInternals(K key,
+ Coder<K> keyCoder,
+ Coder<? extends BoundedWindow> windowCoder,
+ Combine.KeyedCombineFn<K, ?, ?, ?> combineFn) {
+ this.key = key;
+ this.combineFn = combineFn;
+ this.windowCoder = windowCoder;
+ this.keyCoder = keyCoder;
+ }
+
+ public Instant getWatermarkHold() {
+ return watermarkHoldAccessor;
+ }
+
+ /**
+ * This is the interface state has to implement in order for it to be fault tolerant when
+ * executed by the FlinkPipelineRunner.
+ */
+ private interface CheckpointableIF {
+
+ boolean shouldPersist();
+
+ void persistState(StateCheckpointWriter checkpointBuilder) throws IOException;
+ }
+
+ protected final StateTable inMemoryState = new StateTable() {
+
+ @Override
+ protected StateTag.StateBinder binderForNamespace(final StateNamespace namespace) {
+ return new StateTag.StateBinder() {
+
+ @Override
+ public <T> ValueState<T> bindValue(StateTag<ValueState<T>> address, Coder<T> coder) {
+ return new FlinkInMemoryValue<>(encodeKey(namespace, address), coder);
+ }
+
+ @Override
+ public <T> BagState<T> bindBag(StateTag<BagState<T>> address, Coder<T> elemCoder) {
+ return new FlinkInMemoryBag<>(encodeKey(namespace, address), elemCoder);
+ }
+
+ @Override
+ public <InputT, AccumT, OutputT> CombiningValueStateInternal<InputT, AccumT, OutputT> bindCombiningValue(
+ StateTag<CombiningValueStateInternal<InputT, AccumT, OutputT>> address,
+ Coder<AccumT> accumCoder, Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
+ return new FlinkInMemoryCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder);
+ }
+
+ @Override
+ public <T> WatermarkStateInternal bindWatermark(StateTag<WatermarkStateInternal> address) {
+ return new FlinkWatermarkStateInternalImpl(encodeKey(namespace, address));
+ }
+ };
+ }
+ };
+
+ @Override
+ public <T extends State> T state(StateNamespace namespace, StateTag<T> address) {
+ return inMemoryState.get(namespace, address);
+ }
+
+ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
+ checkpointBuilder.writeInt(getNoOfElements());
+
+ for (State location : inMemoryState.values()) {
+ if (!(location instanceof CheckpointableIF)) {
+ throw new IllegalStateException(String.format(
+ "%s wasn't created by %s -- unable to persist it",
+ location.getClass().getSimpleName(),
+ getClass().getSimpleName()));
+ }
+ ((CheckpointableIF) location).persistState(checkpointBuilder);
+ }
+ }
+
+ public void restoreState(StateCheckpointReader checkpointReader, ClassLoader loader)
+ throws IOException, ClassNotFoundException {
+
+ // the number of elements to read.
+ int noOfElements = checkpointReader.getInt();
+ for (int i = 0; i < noOfElements; i++) {
+ decodeState(checkpointReader, loader);
+ }
+ }
+
+ /**
+ * We remove the first character which encodes the type of the stateTag ('s' for system
+ * and 'u' for user). For more details check out the source of
+ * {@link StateTags.StateTagBase#getId()}.
+ */
+ private void decodeState(StateCheckpointReader reader, ClassLoader loader)
+ throws IOException, ClassNotFoundException {
+
+ StateType stateItemType = StateType.deserialize(reader);
+ ByteString stateKey = reader.getTag();
+
+ // first decode the namespace and the tagId...
+ String[] namespaceAndTag = stateKey.toStringUtf8().split("\\+");
+ if (namespaceAndTag.length != 2) {
+ throw new IllegalArgumentException("Invalid stateKey " + stateKey.toString() + ".");
+ }
+ StateNamespace namespace = StateNamespaces.fromString(namespaceAndTag[0], windowCoder);
+
+ // ... decide if it is a system or user stateTag...
+ char ownerTag = namespaceAndTag[1].charAt(0);
+ if (ownerTag != 's' && ownerTag != 'u') {
+ throw new RuntimeException("Invalid StateTag name.");
+ }
+ boolean isSystemTag = ownerTag == 's';
+ String tagId = namespaceAndTag[1].substring(1);
+
+ // ...then decode the coder (if there is one)...
+ Coder coder = null;
+ if (!stateItemType.equals(StateType.WATERMARK)) {
+ ByteString coderBytes = reader.getData();
+ coder = InstantiationUtil.deserializeObject(coderBytes.toByteArray(), loader);
+ }
+
+ //... and finally, depending on the type of the state being decoded,
+ // 1) create the adequate stateTag,
+ // 2) create the state container,
+ // 3) restore the actual content.
+ switch (stateItemType) {
+ case VALUE: {
+ StateTag stateTag = StateTags.value(tagId, coder);
+ stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
+ FlinkInMemoryValue<?> value = (FlinkInMemoryValue<?>) inMemoryState.get(namespace, stateTag);
+ value.restoreState(reader);
+ break;
+ }
+ case WATERMARK: {
+ StateTag<WatermarkStateInternal> stateTag = StateTags.watermarkStateInternal(tagId);
+ stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
+ FlinkWatermarkStateInternalImpl watermark = (FlinkWatermarkStateInternalImpl) inMemoryState.get(namespace, stateTag);
+ watermark.restoreState(reader);
+ break;
+ }
+ case LIST: {
+ StateTag stateTag = StateTags.bag(tagId, coder);
+ stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
+ FlinkInMemoryBag<?> bag = (FlinkInMemoryBag<?>) inMemoryState.get(namespace, stateTag);
+ bag.restoreState(reader);
+ break;
+ }
+ case ACCUMULATOR: {
+ StateTag stateTag = StateTags.combiningValue(tagId, coder, combineFn.forKey(this.key, keyCoder));
+ stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
+ FlinkInMemoryCombiningValue<?, ?, ?> combiningValue = (FlinkInMemoryCombiningValue<?, ?, ?>) inMemoryState.get(namespace, stateTag);
+ combiningValue.restoreState(reader);
+ break;
+ }
+ default:
+ throw new RuntimeException("Unknown State Type " + stateItemType + ".");
+ }
+ }
+
+ private ByteString encodeKey(StateNamespace namespace, StateTag<?> address) {
+ return ByteString.copyFromUtf8(namespace.stringKey() + "+" + address.getId());
+ }
+
+ private int getNoOfElements() {
+ int noOfElements = 0;
+ for (State state : inMemoryState.values()) {
+ if (!(state instanceof CheckpointableIF)) {
+ throw new RuntimeException("State Implementations used by the " +
+ "Flink Dataflow Runner should implement the CheckpointableIF interface.");
+ }
+
+ if (((CheckpointableIF) state).shouldPersist()) {
+ noOfElements++;
+ }
+ }
+ return noOfElements;
+ }
+
+ private final class FlinkInMemoryValue<T> implements ValueState<T>, CheckpointableIF {
+
+ private final ByteString stateKey;
+ private final Coder<T> elemCoder;
+
+ private T value = null;
+
+ public FlinkInMemoryValue(ByteString stateKey, Coder<T> elemCoder) {
+ this.stateKey = stateKey;
+ this.elemCoder = elemCoder;
+ }
+
+ @Override
+ public void clear() {
+ value = null;
+ }
+
+ @Override
+ public StateContents<T> get() {
+ return new StateContents<T>() {
+ @Override
+ public T read() {
+ return value;
+ }
+ };
+ }
+
+ @Override
+ public void set(T input) {
+ this.value = input;
+ }
+
+ @Override
+ public boolean shouldPersist() {
+ return value != null;
+ }
+
+ @Override
+ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
+ if (value != null) {
+
+ // serialize the coder.
+ byte[] coder = InstantiationUtil.serializeObject(elemCoder);
+
+ // encode the value into a ByteString
+ ByteString.Output stream = ByteString.newOutput();
+ elemCoder.encode(value, stream, Coder.Context.OUTER);
+ ByteString data = stream.toByteString();
+
+ checkpointBuilder.addValueBuilder()
+ .setTag(stateKey)
+ .setData(coder)
+ .setData(data);
+ }
+ }
+
+ public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
+ ByteString valueContent = checkpointReader.getData();
+ T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER);
+ set(outValue);
+ }
+ }
+
+ private final class FlinkWatermarkStateInternalImpl
+ implements WatermarkStateInternal, CheckpointableIF {
+
+ private final ByteString stateKey;
+
+ private Instant minimumHold = null;
+
+ public FlinkWatermarkStateInternalImpl(ByteString stateKey) {
+ this.stateKey = stateKey;
+ }
+
+ @Override
+ public void clear() {
+ // Even though we're clearing we can't remove this from the in-memory state map, since
+ // other users may already have a handle on this WatermarkBagInternal.
+ minimumHold = null;
+ watermarkHoldAccessor = null;
+ }
+
+ @Override
+ public StateContents<Instant> get() {
+ return new StateContents<Instant>() {
+ @Override
+ public Instant read() {
+ return minimumHold;
+ }
+ };
+ }
+
+ @Override
+ public void add(Instant watermarkHold) {
+ if (minimumHold == null || minimumHold.isAfter(watermarkHold)) {
+ watermarkHoldAccessor = watermarkHold;
+ minimumHold = watermarkHold;
+ }
+ }
+
+ @Override
+ public StateContents<Boolean> isEmpty() {
+ return new StateContents<Boolean>() {
+ @Override
+ public Boolean read() {
+ return minimumHold == null;
+ }
+ };
+ }
+
+ @Override
+ public String toString() {
+ return Objects.toString(minimumHold);
+ }
+
+ @Override
+ public boolean shouldPersist() {
+ return minimumHold != null;
+ }
+
+ @Override
+ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
+ if (minimumHold != null) {
+ checkpointBuilder.addWatermarkHoldsBuilder()
+ .setTag(stateKey)
+ .setTimestamp(minimumHold);
+ }
+ }
+
+ public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
+ Instant watermark = checkpointReader.getTimestamp();
+ add(watermark);
+ }
+ }
+
+ private final class FlinkInMemoryCombiningValue<InputT, AccumT, OutputT>
+ implements CombiningValueStateInternal<InputT, AccumT, OutputT>, CheckpointableIF {
+
+ private final ByteString stateKey;
+ private final Combine.CombineFn<InputT, AccumT, OutputT> combineFn;
+ private final Coder<AccumT> accumCoder;
+
+ private AccumT accum;
+ private boolean isCleared = true;
+
+ private FlinkInMemoryCombiningValue(ByteString stateKey,
+ Combine.CombineFn<InputT, AccumT, OutputT> combineFn,
+ Coder<AccumT> accumCoder) {
+ Preconditions.checkNotNull(combineFn);
+ Preconditions.checkNotNull(accumCoder);
+
+ this.stateKey = stateKey;
+ this.combineFn = combineFn;
+ this.accumCoder = accumCoder;
+ accum = combineFn.createAccumulator();
+ }
+
+ @Override
+ public void clear() {
+ accum = combineFn.createAccumulator();
+ isCleared = true;
+ }
+
+ @Override
+ public StateContents<OutputT> get() {
+ return new StateContents<OutputT>() {
+ @Override
+ public OutputT read() {
+ return combineFn.extractOutput(accum);
+ }
+ };
+ }
+
+ @Override
+ public void add(InputT input) {
+ isCleared = false;
+ accum = combineFn.addInput(accum, input);
+ }
+
+ @Override
+ public StateContents<AccumT> getAccum() {
+ return new StateContents<AccumT>() {
+ @Override
+ public AccumT read() {
+ return accum;
+ }
+ };
+ }
+
+ @Override
+ public StateContents<Boolean> isEmpty() {
+ return new StateContents<Boolean>() {
+ @Override
+ public Boolean read() {
+ return isCleared;
+ }
+ };
+ }
+
+ @Override
+ public void addAccum(AccumT accum) {
+ isCleared = false;
+ this.accum = combineFn.mergeAccumulators(Arrays.asList(this.accum, accum));
+ }
+
+ @Override
+ public boolean shouldPersist() {
+ return accum != null;
+ }
+
+ @Override
+ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
+ if (accum != null) {
+
+ // serialize the coder.
+ byte[] coder = InstantiationUtil.serializeObject(accumCoder);
+
+ // encode the accumulator into a ByteString
+ ByteString.Output stream = ByteString.newOutput();
+ accumCoder.encode(accum, stream, Coder.Context.OUTER);
+ ByteString data = stream.toByteString();
+
+ // put the flag that the next serialized element is an accumulator
+ checkpointBuilder.addAccumulatorBuilder()
+ .setTag(stateKey)
+ .setData(coder)
+ .setData(data);
+ }
+ }
+
+ public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
+ ByteString valueContent = checkpointReader.getData();
+ AccumT accum = this.accumCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER);
+ addAccum(accum);
+ }
+ }
+
+ private static final class FlinkInMemoryBag<T> implements BagState<T>, CheckpointableIF {
+ private final List<T> contents = new ArrayList<>();
+
+ private final ByteString stateKey;
+ private final Coder<T> elemCoder;
+
+ public FlinkInMemoryBag(ByteString stateKey, Coder<T> elemCoder) {
+ this.stateKey = stateKey;
+ this.elemCoder = elemCoder;
+ }
+
+ @Override
+ public void clear() {
+ contents.clear();
+ }
+
+ @Override
+ public StateContents<Iterable<T>> get() {
+ return new StateContents<Iterable<T>>() {
+ @Override
+ public Iterable<T> read() {
+ return contents;
+ }
+ };
+ }
+
+ @Override
+ public void add(T input) {
+ contents.add(input);
+ }
+
+ @Override
+ public StateContents<Boolean> isEmpty() {
+ return new StateContents<Boolean>() {
+ @Override
+ public Boolean read() {
+ return contents.isEmpty();
+ }
+ };
+ }
+
+ @Override
+ public boolean shouldPersist() {
+ return !contents.isEmpty();
+ }
+
+ @Override
+ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
+ if (!contents.isEmpty()) {
+ // serialize the coder.
+ byte[] coder = InstantiationUtil.serializeObject(elemCoder);
+
+ checkpointBuilder.addListUpdatesBuilder()
+ .setTag(stateKey)
+ .setData(coder)
+ .writeInt(contents.size());
+
+ for (T item : contents) {
+ // encode the element
+ ByteString.Output stream = ByteString.newOutput();
+ elemCoder.encode(item, stream, Coder.Context.OUTER);
+ ByteString data = stream.toByteString();
+
+ // add the data to the checkpoint.
+ checkpointBuilder.setData(data);
+ }
+ }
+ }
+
+ public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
+ int noOfValues = checkpointReader.getInt();
+ for (int j = 0; j < noOfValues; j++) {
+ ByteString valueContent = checkpointReader.getData();
+ T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER);
+ add(outValue);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java
new file mode 100644
index 0000000..ba8ef89
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.state;
+
+import com.dataartisans.flink.dataflow.translation.types.CoderTypeSerializer;
+import com.google.protobuf.ByteString;
+import org.apache.flink.core.memory.DataInputView;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+public class StateCheckpointReader {
+
+ private final DataInputView input;
+
+ public StateCheckpointReader(DataInputView in) {
+ this.input = in;
+ }
+
+ public ByteString getTag() throws IOException {
+ return ByteString.copyFrom(readRawData());
+ }
+
+ public String getTagToString() throws IOException {
+ return input.readUTF();
+ }
+
+ public ByteString getData() throws IOException {
+ return ByteString.copyFrom(readRawData());
+ }
+
+ public int getInt() throws IOException {
+ validate();
+ return input.readInt();
+ }
+
+ public byte getByte() throws IOException {
+ validate();
+ return input.readByte();
+ }
+
+ public Instant getTimestamp() throws IOException {
+ validate();
+ Long watermarkMillis = input.readLong();
+ return new Instant(TimeUnit.MICROSECONDS.toMillis(watermarkMillis));
+ }
+
+ public <K> K deserializeKey(CoderTypeSerializer<K> keySerializer) throws IOException {
+ return deserializeObject(keySerializer);
+ }
+
+ public <T> T deserializeObject(CoderTypeSerializer<T> objectSerializer) throws IOException {
+ return objectSerializer.deserialize(input);
+ }
+
+ ///////// Helper Methods ///////
+
+ private byte[] readRawData() throws IOException {
+ validate();
+ int size = input.readInt();
+
+ byte[] serData = new byte[size];
+ int bytesRead = input.read(serData);
+ if (bytesRead != size) {
+ throw new RuntimeException("Error while deserializing checkpoint. Not enough bytes in the input stream.");
+ }
+ return serData;
+ }
+
+ private void validate() {
+ if (this.input == null) {
+ throw new RuntimeException("StateBackend not initialized yet.");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java
new file mode 100644
index 0000000..6bc8662
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java
@@ -0,0 +1,152 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.state;
+
+import com.dataartisans.flink.dataflow.translation.types.CoderTypeSerializer;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.util.TimeDomain;
+import com.google.cloud.dataflow.sdk.util.TimerInternals;
+import com.google.cloud.dataflow.sdk.util.state.StateNamespace;
+import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class StateCheckpointUtils {
+
+ public static <K> void encodeState(Map<K, FlinkStateInternals<K>> perKeyStateInternals,
+ StateCheckpointWriter writer, Coder<K> keyCoder) throws IOException {
+ CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
+
+ int noOfKeys = perKeyStateInternals.size();
+ writer.writeInt(noOfKeys);
+ for (Map.Entry<K, FlinkStateInternals<K>> keyStatePair : perKeyStateInternals.entrySet()) {
+ K key = keyStatePair.getKey();
+ FlinkStateInternals<K> state = keyStatePair.getValue();
+
+ // encode the key
+ writer.serializeKey(key, keySerializer);
+
+ // write the associated state
+ state.persistState(writer);
+ }
+ }
+
+ public static <K> Map<K, FlinkStateInternals<K>> decodeState(
+ StateCheckpointReader reader,
+ Combine.KeyedCombineFn<K, ?, ?, ?> combineFn,
+ Coder<K> keyCoder,
+ Coder<? extends BoundedWindow> windowCoder,
+ ClassLoader classLoader) throws IOException, ClassNotFoundException {
+
+ int noOfKeys = reader.getInt();
+ Map<K, FlinkStateInternals<K>> perKeyStateInternals = new HashMap<>(noOfKeys);
+ perKeyStateInternals.clear();
+
+ CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
+ for (int i = 0; i < noOfKeys; i++) {
+
+ // decode the key.
+ K key = reader.deserializeKey(keySerializer);
+
+ //decode the state associated to the key.
+ FlinkStateInternals<K> stateForKey =
+ new FlinkStateInternals<>(key, keyCoder, windowCoder, combineFn);
+ stateForKey.restoreState(reader, classLoader);
+ perKeyStateInternals.put(key, stateForKey);
+ }
+ return perKeyStateInternals;
+ }
+
+ ////////////// Encoding/Decoding the Timers ////////////////
+
+
+ public static <K> void encodeTimers(Map<K, Set<TimerInternals.TimerData>> allTimers,
+ StateCheckpointWriter writer,
+ Coder<K> keyCoder) throws IOException {
+ CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
+
+ int noOfKeys = allTimers.size();
+ writer.writeInt(noOfKeys);
+ for (Map.Entry<K, Set<TimerInternals.TimerData>> timersPerKey : allTimers.entrySet()) {
+ K key = timersPerKey.getKey();
+
+ // encode the key
+ writer.serializeKey(key, keySerializer);
+
+ // write the associated timers
+ Set<TimerInternals.TimerData> timers = timersPerKey.getValue();
+ encodeTimerDataForKey(writer, timers);
+ }
+ }
+
+ public static <K> Map<K, Set<TimerInternals.TimerData>> decodeTimers(
+ StateCheckpointReader reader,
+ Coder<? extends BoundedWindow> windowCoder,
+ Coder<K> keyCoder) throws IOException {
+
+ int noOfKeys = reader.getInt();
+ Map<K, Set<TimerInternals.TimerData>> activeTimers = new HashMap<>(noOfKeys);
+ activeTimers.clear();
+
+ CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
+ for (int i = 0; i < noOfKeys; i++) {
+
+ // decode the key.
+ K key = reader.deserializeKey(keySerializer);
+
+ // decode the associated timers.
+ Set<TimerInternals.TimerData> timers = decodeTimerDataForKey(reader, windowCoder);
+ activeTimers.put(key, timers);
+ }
+ return activeTimers;
+ }
+
+ private static void encodeTimerDataForKey(StateCheckpointWriter writer, Set<TimerInternals.TimerData> timers) throws IOException {
+ // encode timers
+ writer.writeInt(timers.size());
+ for (TimerInternals.TimerData timer : timers) {
+ String stringKey = timer.getNamespace().stringKey();
+
+ writer.setTag(stringKey);
+ writer.setTimestamp(timer.getTimestamp());
+ writer.writeInt(timer.getDomain().ordinal());
+ }
+ }
+
+ private static Set<TimerInternals.TimerData> decodeTimerDataForKey(
+ StateCheckpointReader reader, Coder<? extends BoundedWindow> windowCoder) throws IOException {
+
+ // decode the timers: first their number and then the content itself.
+ int noOfTimers = reader.getInt();
+ Set<TimerInternals.TimerData> timers = new HashSet<>(noOfTimers);
+ for (int i = 0; i < noOfTimers; i++) {
+ String stringKey = reader.getTagToString();
+ Instant instant = reader.getTimestamp();
+ TimeDomain domain = TimeDomain.values()[reader.getInt()];
+
+ StateNamespace namespace = StateNamespaces.fromString(stringKey, windowCoder);
+ timers.add(TimerInternals.TimerData.of(namespace, instant, domain));
+ }
+ return timers;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java
new file mode 100644
index 0000000..7201112
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java
@@ -0,0 +1,127 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.state;
+
+import com.dataartisans.flink.dataflow.translation.types.CoderTypeSerializer;
+import com.google.protobuf.ByteString;
+import org.apache.flink.runtime.state.StateBackend;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+public class StateCheckpointWriter {
+
+ private final StateBackend.CheckpointStateOutputView output;
+
+ public static StateCheckpointWriter create(StateBackend.CheckpointStateOutputView output) {
+ return new StateCheckpointWriter(output);
+ }
+
+ private StateCheckpointWriter(StateBackend.CheckpointStateOutputView output) {
+ this.output = output;
+ }
+
+ ///////// Creating the serialized versions of the different types of state held by dataflow ///////
+
+ public StateCheckpointWriter addValueBuilder() throws IOException {
+ validate();
+ StateType.serialize(StateType.VALUE, this);
+ return this;
+ }
+
+ public StateCheckpointWriter addWatermarkHoldsBuilder() throws IOException {
+ validate();
+ StateType.serialize(StateType.WATERMARK, this);
+ return this;
+ }
+
+ public StateCheckpointWriter addListUpdatesBuilder() throws IOException {
+ validate();
+ StateType.serialize(StateType.LIST, this);
+ return this;
+ }
+
+ public StateCheckpointWriter addAccumulatorBuilder() throws IOException {
+ validate();
+ StateType.serialize(StateType.ACCUMULATOR, this);
+ return this;
+ }
+
+ ///////// Setting the tag for a given state element ///////
+
+ public StateCheckpointWriter setTag(ByteString stateKey) throws IOException {
+ return writeData(stateKey.toByteArray());
+ }
+
+ public StateCheckpointWriter setTag(String stateKey) throws IOException {
+ output.writeUTF(stateKey);
+ return this;
+ }
+
+
+ public <K> StateCheckpointWriter serializeKey(K key, CoderTypeSerializer<K> keySerializer) throws IOException {
+ return serializeObject(key, keySerializer);
+ }
+
+ public <T> StateCheckpointWriter serializeObject(T object, CoderTypeSerializer<T> objectSerializer) throws IOException {
+ objectSerializer.serialize(object, output);
+ return this;
+ }
+
+ ///////// Write the actual serialized data //////////
+
+ public StateCheckpointWriter setData(ByteString data) throws IOException {
+ return writeData(data.toByteArray());
+ }
+
+ public StateCheckpointWriter setData(byte[] data) throws IOException {
+ return writeData(data);
+ }
+
+ public StateCheckpointWriter setTimestamp(Instant timestamp) throws IOException {
+ validate();
+ output.writeLong(TimeUnit.MILLISECONDS.toMicros(timestamp.getMillis()));
+ return this;
+ }
+
+ public StateCheckpointWriter writeInt(int number) throws IOException {
+ validate();
+ output.writeInt(number);
+ return this;
+ }
+
+ public StateCheckpointWriter writeByte(byte b) throws IOException {
+ validate();
+ output.writeByte(b);
+ return this;
+ }
+
+ ///////// Helper Methods ///////
+
+ private StateCheckpointWriter writeData(byte[] data) throws IOException {
+ validate();
+ output.writeInt(data.length);
+ output.write(data);
+ return this;
+ }
+
+ private void validate() {
+ if (this.output == null) {
+ throw new RuntimeException("StateBackend not initialized yet.");
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java
new file mode 100644
index 0000000..11446ea
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.state;
+
+import java.io.IOException;
+
+public enum StateType {
+
+ VALUE(0),
+
+ WATERMARK(1),
+
+ LIST(2),
+
+ ACCUMULATOR(3);
+
+ private final int numVal;
+
+ StateType(int value) {
+ this.numVal = value;
+ }
+
+ public static void serialize(StateType type, StateCheckpointWriter output) throws IOException {
+ if (output == null) {
+ throw new IllegalArgumentException("Cannot write to a null output.");
+ }
+
+ if(type.numVal < 0 || type.numVal > 3) {
+ throw new RuntimeException("Unknown State Type " + type + ".");
+ }
+
+ output.writeByte((byte) type.numVal);
+ }
+
+ public static StateType deserialize(StateCheckpointReader input) throws IOException {
+ if (input == null) {
+ throw new IllegalArgumentException("Cannot read from a null input.");
+ }
+
+ int typeInt = (int) input.getByte();
+ if(typeInt < 0 || typeInt > 3) {
+ throw new RuntimeException("Unknown State Type " + typeInt + ".");
+ }
+
+ StateType resultType = null;
+ for(StateType st: values()) {
+ if(st.numVal == typeInt) {
+ resultType = st;
+ break;
+ }
+ }
+ return resultType;
+ }
+}
[31/50] [abbrv] incubator-beam git commit: [flink] adjust directories
according to package name
Posted by da...@apache.org.
[flink] adjust directories according to package name
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/51bec310
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/51bec310
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/51bec310
Branch: refs/heads/master
Commit: 51bec310d3a68cd87071aff1b59d8353fc2c40ac
Parents: 028a55f
Author: Maximilian Michels <mx...@apache.org>
Authored: Wed Mar 2 23:15:22 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
.../FlinkPipelineExecutionEnvironment.java | 267 -------
.../flink/dataflow/FlinkPipelineOptions.java | 91 ---
.../flink/dataflow/FlinkPipelineRunner.java | 204 ------
.../flink/dataflow/FlinkRunnerResult.java | 66 --
.../flink/dataflow/examples/TFIDF.java | 452 ------------
.../flink/dataflow/examples/WordCount.java | 111 ---
.../examples/streaming/AutoComplete.java | 387 ----------
.../examples/streaming/JoinExamples.java | 158 ----
.../KafkaWindowedWordCountExample.java | 141 ----
.../examples/streaming/WindowedWordCount.java | 128 ----
.../flink/dataflow/io/ConsoleIO.java | 80 ---
.../FlinkBatchPipelineTranslator.java | 149 ----
.../FlinkBatchTransformTranslators.java | 594 ---------------
.../FlinkBatchTranslationContext.java | 129 ----
.../translation/FlinkPipelineTranslator.java | 34 -
.../FlinkStreamingPipelineTranslator.java | 143 ----
.../FlinkStreamingTransformTranslators.java | 404 -----------
.../FlinkStreamingTranslationContext.java | 87 ---
.../FlinkCoGroupKeyedListAggregator.java | 58 --
.../functions/FlinkCreateFunction.java | 60 --
.../functions/FlinkDoFnFunction.java | 202 ------
.../FlinkKeyedListAggregationFunction.java | 75 --
.../functions/FlinkMultiOutputDoFnFunction.java | 175 -----
.../FlinkMultiOutputPruningFunction.java | 41 --
.../functions/FlinkPartialReduceFunction.java | 60 --
.../functions/FlinkReduceFunction.java | 57 --
.../translation/functions/UnionCoder.java | 150 ----
.../translation/types/CoderComparator.java | 216 ------
.../translation/types/CoderTypeInformation.java | 116 ---
.../translation/types/CoderTypeSerializer.java | 152 ----
.../types/InspectableByteArrayOutputStream.java | 34 -
.../translation/types/KvCoderComperator.java | 264 -------
.../types/KvCoderTypeInformation.java | 186 -----
.../types/VoidCoderTypeSerializer.java | 112 ---
.../wrappers/CombineFnAggregatorWrapper.java | 92 ---
.../wrappers/DataInputViewWrapper.java | 59 --
.../wrappers/DataOutputViewWrapper.java | 52 --
.../SerializableFnAggregatorWrapper.java | 91 ---
.../translation/wrappers/SinkOutputFormat.java | 121 ----
.../translation/wrappers/SourceInputFormat.java | 164 -----
.../translation/wrappers/SourceInputSplit.java | 52 --
.../streaming/FlinkAbstractParDoWrapper.java | 264 -------
.../FlinkGroupAlsoByWindowWrapper.java | 629 ----------------
.../streaming/FlinkGroupByKeyWrapper.java | 64 --
.../streaming/FlinkParDoBoundMultiWrapper.java | 75 --
.../streaming/FlinkParDoBoundWrapper.java | 98 ---
.../io/FlinkStreamingCreateFunction.java | 63 --
.../streaming/io/UnboundedFlinkSource.java | 80 ---
.../streaming/io/UnboundedSocketSource.java | 231 ------
.../streaming/io/UnboundedSourceWrapper.java | 132 ----
.../state/AbstractFlinkTimerInternals.java | 126 ----
.../streaming/state/FlinkStateInternals.java | 713 -------------------
.../streaming/state/StateCheckpointReader.java | 89 ---
.../streaming/state/StateCheckpointUtils.java | 153 ----
.../streaming/state/StateCheckpointWriter.java | 127 ----
.../wrappers/streaming/state/StateType.java | 71 --
.../FlinkPipelineExecutionEnvironment.java | 267 +++++++
.../runners/flink/FlinkPipelineOptions.java | 91 +++
.../beam/runners/flink/FlinkPipelineRunner.java | 204 ++++++
.../beam/runners/flink/FlinkRunnerResult.java | 66 ++
.../beam/runners/flink/examples/TFIDF.java | 452 ++++++++++++
.../beam/runners/flink/examples/WordCount.java | 111 +++
.../flink/examples/streaming/AutoComplete.java | 387 ++++++++++
.../flink/examples/streaming/JoinExamples.java | 158 ++++
.../KafkaWindowedWordCountExample.java | 141 ++++
.../examples/streaming/WindowedWordCount.java | 128 ++++
.../apache/beam/runners/flink/io/ConsoleIO.java | 80 +++
.../FlinkBatchPipelineTranslator.java | 149 ++++
.../FlinkBatchTransformTranslators.java | 594 +++++++++++++++
.../FlinkBatchTranslationContext.java | 129 ++++
.../translation/FlinkPipelineTranslator.java | 34 +
.../FlinkStreamingPipelineTranslator.java | 143 ++++
.../FlinkStreamingTransformTranslators.java | 404 +++++++++++
.../FlinkStreamingTranslationContext.java | 87 +++
.../FlinkCoGroupKeyedListAggregator.java | 58 ++
.../functions/FlinkCreateFunction.java | 60 ++
.../functions/FlinkDoFnFunction.java | 202 ++++++
.../FlinkKeyedListAggregationFunction.java | 75 ++
.../functions/FlinkMultiOutputDoFnFunction.java | 175 +++++
.../FlinkMultiOutputPruningFunction.java | 41 ++
.../functions/FlinkPartialReduceFunction.java | 60 ++
.../functions/FlinkReduceFunction.java | 57 ++
.../flink/translation/functions/UnionCoder.java | 150 ++++
.../translation/types/CoderComparator.java | 216 ++++++
.../translation/types/CoderTypeInformation.java | 116 +++
.../translation/types/CoderTypeSerializer.java | 152 ++++
.../types/InspectableByteArrayOutputStream.java | 34 +
.../translation/types/KvCoderComperator.java | 264 +++++++
.../types/KvCoderTypeInformation.java | 186 +++++
.../types/VoidCoderTypeSerializer.java | 112 +++
.../wrappers/CombineFnAggregatorWrapper.java | 92 +++
.../wrappers/DataInputViewWrapper.java | 59 ++
.../wrappers/DataOutputViewWrapper.java | 52 ++
.../SerializableFnAggregatorWrapper.java | 91 +++
.../translation/wrappers/SinkOutputFormat.java | 121 ++++
.../translation/wrappers/SourceInputFormat.java | 164 +++++
.../translation/wrappers/SourceInputSplit.java | 52 ++
.../streaming/FlinkAbstractParDoWrapper.java | 264 +++++++
.../FlinkGroupAlsoByWindowWrapper.java | 629 ++++++++++++++++
.../streaming/FlinkGroupByKeyWrapper.java | 64 ++
.../streaming/FlinkParDoBoundMultiWrapper.java | 75 ++
.../streaming/FlinkParDoBoundWrapper.java | 98 +++
.../io/FlinkStreamingCreateFunction.java | 63 ++
.../streaming/io/UnboundedFlinkSource.java | 80 +++
.../streaming/io/UnboundedSocketSource.java | 231 ++++++
.../streaming/io/UnboundedSourceWrapper.java | 132 ++++
.../state/AbstractFlinkTimerInternals.java | 126 ++++
.../streaming/state/FlinkStateInternals.java | 713 +++++++++++++++++++
.../streaming/state/StateCheckpointReader.java | 89 +++
.../streaming/state/StateCheckpointUtils.java | 153 ++++
.../streaming/state/StateCheckpointWriter.java | 127 ++++
.../wrappers/streaming/state/StateType.java | 71 ++
.../dataartisans/flink/dataflow/AvroITCase.java | 99 ---
.../flink/dataflow/FlattenizeITCase.java | 72 --
.../flink/dataflow/FlinkTestPipeline.java | 70 --
.../flink/dataflow/JoinExamplesITCase.java | 99 ---
.../flink/dataflow/MaybeEmptyTestITCase.java | 63 --
.../flink/dataflow/ParDoMultiOutputITCase.java | 98 ---
.../flink/dataflow/ReadSourceITCase.java | 163 -----
.../dataflow/RemoveDuplicatesEmptyITCase.java | 68 --
.../flink/dataflow/RemoveDuplicatesITCase.java | 69 --
.../flink/dataflow/SideInputITCase.java | 67 --
.../flink/dataflow/TfIdfITCase.java | 76 --
.../flink/dataflow/WordCountITCase.java | 74 --
.../flink/dataflow/WordCountJoin2ITCase.java | 136 ----
.../flink/dataflow/WordCountJoin3ITCase.java | 154 ----
.../flink/dataflow/WriteSinkITCase.java | 156 ----
.../streaming/GroupAlsoByWindowTest.java | 506 -------------
.../dataflow/streaming/GroupByNullKeyTest.java | 121 ----
.../streaming/StateSerializationTest.java | 303 --------
.../streaming/TopWikipediaSessionsITCase.java | 132 ----
.../flink/dataflow/util/JoinExamples.java | 158 ----
.../apache/beam/runners/flink/AvroITCase.java | 99 +++
.../beam/runners/flink/FlattenizeITCase.java | 72 ++
.../beam/runners/flink/FlinkTestPipeline.java | 70 ++
.../beam/runners/flink/JoinExamplesITCase.java | 99 +++
.../runners/flink/MaybeEmptyTestITCase.java | 63 ++
.../runners/flink/ParDoMultiOutputITCase.java | 98 +++
.../beam/runners/flink/ReadSourceITCase.java | 163 +++++
.../flink/RemoveDuplicatesEmptyITCase.java | 68 ++
.../runners/flink/RemoveDuplicatesITCase.java | 69 ++
.../beam/runners/flink/SideInputITCase.java | 67 ++
.../apache/beam/runners/flink/TfIdfITCase.java | 76 ++
.../beam/runners/flink/WordCountITCase.java | 74 ++
.../runners/flink/WordCountJoin2ITCase.java | 136 ++++
.../runners/flink/WordCountJoin3ITCase.java | 154 ++++
.../beam/runners/flink/WriteSinkITCase.java | 156 ++++
.../flink/streaming/GroupAlsoByWindowTest.java | 506 +++++++++++++
.../flink/streaming/GroupByNullKeyTest.java | 121 ++++
.../flink/streaming/StateSerializationTest.java | 303 ++++++++
.../streaming/TopWikipediaSessionsITCase.java | 132 ++++
.../beam/runners/flink/util/JoinExamples.java | 158 ++++
152 files changed, 11813 insertions(+), 11813 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java
deleted file mode 100644
index 02a49b9..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java
+++ /dev/null
@@ -1,267 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import org.apache.beam.runners.flink.translation.FlinkPipelineTranslator;
-import org.apache.beam.runners.flink.translation.FlinkBatchPipelineTranslator;
-import org.apache.beam.runners.flink.translation.FlinkStreamingPipelineTranslator;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.common.base.Preconditions;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.java.CollectionEnvironment;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.List;
-
-/**
- * The class that instantiates and manages the execution of a given job.
- * Depending on if the job is a Streaming or Batch processing one, it creates
- * the adequate execution environment ({@link ExecutionEnvironment} or {@link StreamExecutionEnvironment}),
- * the necessary {@link FlinkPipelineTranslator} ({@link FlinkBatchPipelineTranslator} or
- * {@link FlinkStreamingPipelineTranslator})to transform the Beam job into a Flink one, and
- * executes the (translated) job.
- */
-public class FlinkPipelineExecutionEnvironment {
-
- private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class);
-
- private final FlinkPipelineOptions options;
-
- /**
- * The Flink Batch execution environment. This is instantiated to either a
- * {@link org.apache.flink.api.java.CollectionEnvironment},
- * a {@link org.apache.flink.api.java.LocalEnvironment} or
- * a {@link org.apache.flink.api.java.RemoteEnvironment}, depending on the configuration
- * options.
- */
- private ExecutionEnvironment flinkBatchEnv;
-
-
- /**
- * The Flink Streaming execution environment. This is instantiated to either a
- * {@link org.apache.flink.streaming.api.environment.LocalStreamEnvironment} or
- * a {@link org.apache.flink.streaming.api.environment.RemoteStreamEnvironment}, depending
- * on the configuration options, and more specifically, the url of the master.
- */
- private StreamExecutionEnvironment flinkStreamEnv;
-
- /**
- * Translator for this FlinkPipelineRunner. Its role is to translate the Beam operators to
- * their Flink counterparts. Based on the options provided by the user, if we have a streaming job,
- * this is instantiated as a {@link FlinkStreamingPipelineTranslator}. In other case, i.e. a batch job,
- * a {@link FlinkBatchPipelineTranslator} is created.
- */
- private FlinkPipelineTranslator flinkPipelineTranslator;
-
- /**
- * Creates a {@link FlinkPipelineExecutionEnvironment} with the user-specified parameters in the
- * provided {@link FlinkPipelineOptions}.
- *
- * @param options the user-defined pipeline options.
- * */
- public FlinkPipelineExecutionEnvironment(FlinkPipelineOptions options) {
- this.options = Preconditions.checkNotNull(options);
- this.createPipelineExecutionEnvironment();
- this.createPipelineTranslator();
- }
-
- /**
- * Depending on the type of job (Streaming or Batch) and the user-specified options,
- * this method creates the adequate ExecutionEnvironment.
- */
- private void createPipelineExecutionEnvironment() {
- if (options.isStreaming()) {
- createStreamExecutionEnvironment();
- } else {
- createBatchExecutionEnvironment();
- }
- }
-
- /**
- * Depending on the type of job (Streaming or Batch), this method creates the adequate job graph
- * translator. In the case of batch, it will work with {@link org.apache.flink.api.java.DataSet},
- * while for streaming, it will work with {@link org.apache.flink.streaming.api.datastream.DataStream}.
- */
- private void createPipelineTranslator() {
- checkInitializationState();
- if (this.flinkPipelineTranslator != null) {
- throw new IllegalStateException("FlinkPipelineTranslator already initialized.");
- }
-
- this.flinkPipelineTranslator = options.isStreaming() ?
- new FlinkStreamingPipelineTranslator(flinkStreamEnv, options) :
- new FlinkBatchPipelineTranslator(flinkBatchEnv, options);
- }
-
- /**
- * Depending on if the job is a Streaming or a Batch one, this method creates
- * the necessary execution environment and pipeline translator, and translates
- * the {@link com.google.cloud.dataflow.sdk.values.PCollection} program into
- * a {@link org.apache.flink.api.java.DataSet} or {@link org.apache.flink.streaming.api.datastream.DataStream}
- * one.
- * */
- public void translate(Pipeline pipeline) {
- checkInitializationState();
- if(this.flinkBatchEnv == null && this.flinkStreamEnv == null) {
- createPipelineExecutionEnvironment();
- }
- if (this.flinkPipelineTranslator == null) {
- createPipelineTranslator();
- }
- this.flinkPipelineTranslator.translate(pipeline);
- }
-
- /**
- * Launches the program execution.
- * */
- public JobExecutionResult executePipeline() throws Exception {
- if (options.isStreaming()) {
- if (this.flinkStreamEnv == null) {
- throw new RuntimeException("FlinkPipelineExecutionEnvironment not initialized.");
- }
- if (this.flinkPipelineTranslator == null) {
- throw new RuntimeException("FlinkPipelineTranslator not initialized.");
- }
- return this.flinkStreamEnv.execute();
- } else {
- if (this.flinkBatchEnv == null) {
- throw new RuntimeException("FlinkPipelineExecutionEnvironment not initialized.");
- }
- if (this.flinkPipelineTranslator == null) {
- throw new RuntimeException("FlinkPipelineTranslator not initialized.");
- }
- return this.flinkBatchEnv.execute();
- }
- }
-
- /**
- * If the submitted job is a batch processing job, this method creates the adequate
- * Flink {@link org.apache.flink.api.java.ExecutionEnvironment} depending
- * on the user-specified options.
- */
- private void createBatchExecutionEnvironment() {
- if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) {
- throw new RuntimeException("FlinkPipelineExecutionEnvironment already initialized.");
- }
-
- LOG.info("Creating the required Batch Execution Environment.");
-
- String masterUrl = options.getFlinkMaster();
- this.flinkStreamEnv = null;
-
- // depending on the master, create the right environment.
- if (masterUrl.equals("[local]")) {
- this.flinkBatchEnv = ExecutionEnvironment.createLocalEnvironment();
- } else if (masterUrl.equals("[collection]")) {
- this.flinkBatchEnv = new CollectionEnvironment();
- } else if (masterUrl.equals("[auto]")) {
- this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
- } else if (masterUrl.matches(".*:\\d*")) {
- String[] parts = masterUrl.split(":");
- List<String> stagingFiles = options.getFilesToStage();
- this.flinkBatchEnv = ExecutionEnvironment.createRemoteEnvironment(parts[0],
- Integer.parseInt(parts[1]),
- stagingFiles.toArray(new String[stagingFiles.size()]));
- } else {
- LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
- this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
- }
-
- // set the correct parallelism.
- if (options.getParallelism() != -1 && !(this.flinkBatchEnv instanceof CollectionEnvironment)) {
- this.flinkBatchEnv.setParallelism(options.getParallelism());
- }
-
- // set parallelism in the options (required by some execution code)
- options.setParallelism(flinkBatchEnv.getParallelism());
- }
-
- /**
- * If the submitted job is a stream processing job, this method creates the adequate
- * Flink {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment} depending
- * on the user-specified options.
- */
- private void createStreamExecutionEnvironment() {
- if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) {
- throw new RuntimeException("FlinkPipelineExecutionEnvironment already initialized.");
- }
-
- LOG.info("Creating the required Streaming Environment.");
-
- String masterUrl = options.getFlinkMaster();
- this.flinkBatchEnv = null;
-
- // depending on the master, create the right environment.
- if (masterUrl.equals("[local]")) {
- this.flinkStreamEnv = StreamExecutionEnvironment.createLocalEnvironment();
- } else if (masterUrl.equals("[auto]")) {
- this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
- } else if (masterUrl.matches(".*:\\d*")) {
- String[] parts = masterUrl.split(":");
- List<String> stagingFiles = options.getFilesToStage();
- this.flinkStreamEnv = StreamExecutionEnvironment.createRemoteEnvironment(parts[0],
- Integer.parseInt(parts[1]), stagingFiles.toArray(new String[stagingFiles.size()]));
- } else {
- LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
- this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
- }
-
- // set the correct parallelism.
- if (options.getParallelism() != -1) {
- this.flinkStreamEnv.setParallelism(options.getParallelism());
- }
-
- // set parallelism in the options (required by some execution code)
- options.setParallelism(flinkStreamEnv.getParallelism());
-
- // default to event time
- this.flinkStreamEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-
- // for the following 2 parameters, a value of -1 means that Flink will use
- // the default values as specified in the configuration.
- int numRetries = options.getNumberOfExecutionRetries();
- if (numRetries != -1) {
- this.flinkStreamEnv.setNumberOfExecutionRetries(numRetries);
- }
- long retryDelay = options.getExecutionRetryDelay();
- if (retryDelay != -1) {
- this.flinkStreamEnv.getConfig().setExecutionRetryDelay(retryDelay);
- }
-
- // A value of -1 corresponds to disabled checkpointing (see CheckpointConfig in Flink).
- // If the value is not -1, then the validity checks are applied.
- // By default, checkpointing is disabled.
- long checkpointInterval = options.getCheckpointingInterval();
- if(checkpointInterval != -1) {
- if (checkpointInterval < 1) {
- throw new IllegalArgumentException("The checkpoint interval must be positive");
- }
- this.flinkStreamEnv.enableCheckpointing(checkpointInterval);
- }
- }
-
- private void checkInitializationState() {
- if (options.isStreaming() && this.flinkBatchEnv != null) {
- throw new IllegalStateException("Attempted to run a Streaming Job with a Batch Execution Environment.");
- } else if (!options.isStreaming() && this.flinkStreamEnv != null) {
- throw new IllegalStateException("Attempted to run a Batch Job with a Streaming Execution Environment.");
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java
deleted file mode 100644
index bf83353..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.google.cloud.dataflow.sdk.options.ApplicationNameOptions;
-import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.StreamingOptions;
-
-import java.util.List;
-
-/**
- * Options which can be used to configure a Flink PipelineRunner.
- */
-public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOptions, StreamingOptions {
-
- /**
- * List of local files to make available to workers.
- * <p>
- * Jars are placed on the worker's classpath.
- * <p>
- * The default value is the list of jars from the main program's classpath.
- */
- @Description("Jar-Files to send to all workers and put on the classpath. " +
- "The default value is all files from the classpath.")
- @JsonIgnore
- List<String> getFilesToStage();
- void setFilesToStage(List<String> value);
-
- /**
- * The job name is used to identify jobs running on a Flink cluster.
- */
- @Description("Dataflow job name, to uniquely identify active jobs. "
- + "Defaults to using the ApplicationName-UserName-Date.")
- @Default.InstanceFactory(DataflowPipelineOptions.JobNameFactory.class)
- String getJobName();
- void setJobName(String value);
-
- /**
- * The url of the Flink JobManager on which to execute pipelines. This can either be
- * the the address of a cluster JobManager, in the form "host:port" or one of the special
- * Strings "[local]", "[collection]" or "[auto]". "[local]" will start a local Flink
- * Cluster in the JVM, "[collection]" will execute the pipeline on Java Collections while
- * "[auto]" will let the system decide where to execute the pipeline based on the environment.
- */
- @Description("Address of the Flink Master where the Pipeline should be executed. Can" +
- " either be of the form \"host:port\" or one of the special values [local], " +
- "[collection] or [auto].")
- String getFlinkMaster();
- void setFlinkMaster(String value);
-
- @Description("The degree of parallelism to be used when distributing operations onto workers.")
- @Default.Integer(-1)
- Integer getParallelism();
- void setParallelism(Integer value);
-
- @Description("The interval between consecutive checkpoints (i.e. snapshots of the current pipeline state used for " +
- "fault tolerance).")
- @Default.Long(-1L)
- Long getCheckpointingInterval();
- void setCheckpointingInterval(Long interval);
-
- @Description("Sets the number of times that failed tasks are re-executed. " +
- "A value of zero effectively disables fault tolerance. A value of -1 indicates " +
- "that the system default value (as defined in the configuration) should be used.")
- @Default.Integer(-1)
- Integer getNumberOfExecutionRetries();
- void setNumberOfExecutionRetries(Integer retries);
-
- @Description("Sets the delay between executions. A value of {@code -1} indicates that the default value should be used.")
- @Default.Long(-1L)
- Long getExecutionRetryDelay();
- void setExecutionRetryDelay(Long delay);
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
deleted file mode 100644
index 3c33d20..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
+++ /dev/null
@@ -1,204 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator;
-import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.net.URLClassLoader;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-/**
- * A {@link PipelineRunner} that executes the operations in the
- * pipeline by first translating them to a Flink Plan and then executing them either locally
- * or on a Flink cluster, depending on the configuration.
- * <p>
- * This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}.
- */
-public class FlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {
-
- private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineRunner.class);
-
- /**
- * Provided options.
- */
- private final FlinkPipelineOptions options;
-
- private final FlinkPipelineExecutionEnvironment flinkJobEnv;
-
- /**
- * Construct a runner from the provided options.
- *
- * @param options Properties which configure the runner.
- * @return The newly created runner.
- */
- public static FlinkPipelineRunner fromOptions(PipelineOptions options) {
- FlinkPipelineOptions flinkOptions =
- PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options);
- ArrayList<String> missing = new ArrayList<>();
-
- if (flinkOptions.getAppName() == null) {
- missing.add("appName");
- }
- if (missing.size() > 0) {
- throw new IllegalArgumentException(
- "Missing required values: " + Joiner.on(',').join(missing));
- }
-
- if (flinkOptions.getFilesToStage() == null) {
- flinkOptions.setFilesToStage(detectClassPathResourcesToStage(
- DataflowPipelineRunner.class.getClassLoader()));
- LOG.info("PipelineOptions.filesToStage was not specified. "
- + "Defaulting to files from the classpath: will stage {} files. "
- + "Enable logging at DEBUG level to see which files will be staged.",
- flinkOptions.getFilesToStage().size());
- LOG.debug("Classpath elements: {}", flinkOptions.getFilesToStage());
- }
-
- // Verify jobName according to service requirements.
- String jobName = flinkOptions.getJobName().toLowerCase();
- Preconditions.checkArgument(jobName.matches("[a-z]([-a-z0-9]*[a-z0-9])?"), "JobName invalid; " +
- "the name must consist of only the characters " + "[-a-z0-9], starting with a letter " +
- "and ending with a letter " + "or number");
- Preconditions.checkArgument(jobName.length() <= 40,
- "JobName too long; must be no more than 40 characters in length");
-
- // Set Flink Master to [auto] if no option was specified.
- if (flinkOptions.getFlinkMaster() == null) {
- flinkOptions.setFlinkMaster("[auto]");
- }
-
- return new FlinkPipelineRunner(flinkOptions);
- }
-
- private FlinkPipelineRunner(FlinkPipelineOptions options) {
- this.options = options;
- this.flinkJobEnv = new FlinkPipelineExecutionEnvironment(options);
- }
-
- @Override
- public FlinkRunnerResult run(Pipeline pipeline) {
- LOG.info("Executing pipeline using FlinkPipelineRunner.");
-
- LOG.info("Translating pipeline to Flink program.");
-
- this.flinkJobEnv.translate(pipeline);
-
- LOG.info("Starting execution of Flink program.");
-
- JobExecutionResult result;
- try {
- result = this.flinkJobEnv.executePipeline();
- } catch (Exception e) {
- LOG.error("Pipeline execution failed", e);
- throw new RuntimeException("Pipeline execution failed", e);
- }
-
- LOG.info("Execution finished in {} msecs", result.getNetRuntime());
-
- Map<String, Object> accumulators = result.getAllAccumulatorResults();
- if (accumulators != null && !accumulators.isEmpty()) {
- LOG.info("Final aggregator values:");
-
- for (Map.Entry<String, Object> entry : result.getAllAccumulatorResults().entrySet()) {
- LOG.info("{} : {}", entry.getKey(), entry.getValue());
- }
- }
-
- return new FlinkRunnerResult(accumulators, result.getNetRuntime());
- }
-
- /**
- * For testing.
- */
- public FlinkPipelineOptions getPipelineOptions() {
- return options;
- }
-
- /**
- * Constructs a runner with default properties for testing.
- *
- * @return The newly created runner.
- */
- public static FlinkPipelineRunner createForTest(boolean streaming) {
- FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
- // we use [auto] for testing since this will make it pick up the Testing
- // ExecutionEnvironment
- options.setFlinkMaster("[auto]");
- options.setStreaming(streaming);
- return new FlinkPipelineRunner(options);
- }
-
- @Override
- public <Output extends POutput, Input extends PInput> Output apply(
- PTransform<Input, Output> transform, Input input) {
- return super.apply(transform, input);
- }
-
- /////////////////////////////////////////////////////////////////////////////
-
- @Override
- public String toString() {
- return "DataflowPipelineRunner#" + hashCode();
- }
-
- /**
- * Attempts to detect all the resources the class loader has access to. This does not recurse
- * to class loader parents stopping it from pulling in resources from the system class loader.
- *
- * @param classLoader The URLClassLoader to use to detect resources to stage.
- * @return A list of absolute paths to the resources the class loader uses.
- * @throws IllegalArgumentException If either the class loader is not a URLClassLoader or one
- * of the resources the class loader exposes is not a file resource.
- */
- protected static List<String> detectClassPathResourcesToStage(ClassLoader classLoader) {
- if (!(classLoader instanceof URLClassLoader)) {
- String message = String.format("Unable to use ClassLoader to detect classpath elements. "
- + "Current ClassLoader is %s, only URLClassLoaders are supported.", classLoader);
- LOG.error(message);
- throw new IllegalArgumentException(message);
- }
-
- List<String> files = new ArrayList<>();
- for (URL url : ((URLClassLoader) classLoader).getURLs()) {
- try {
- files.add(new File(url.toURI()).getAbsolutePath());
- } catch (IllegalArgumentException | URISyntaxException e) {
- String message = String.format("Unable to convert url (%s) to file.", url);
- LOG.error(message);
- throw new IllegalArgumentException(message, e);
- }
- }
- return files;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java
deleted file mode 100644
index c2329a6..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException;
-import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-
-import java.util.Collections;
-import java.util.Map;
-
-/**
- * Result of executing a {@link com.google.cloud.dataflow.sdk.Pipeline} with Flink. This
- * has methods to query to job runtime and the final values of
- * {@link com.google.cloud.dataflow.sdk.transforms.Aggregator}s.
- */
-public class FlinkRunnerResult implements PipelineResult {
-
- private final Map<String, Object> aggregators;
-
- private final long runtime;
-
- public FlinkRunnerResult(Map<String, Object> aggregators, long runtime) {
- this.aggregators = (aggregators == null || aggregators.isEmpty()) ?
- Collections.<String, Object>emptyMap() :
- Collections.unmodifiableMap(aggregators);
-
- this.runtime = runtime;
- }
-
- @Override
- public State getState() {
- return null;
- }
-
- @Override
- public <T> AggregatorValues<T> getAggregatorValues(final Aggregator<?, T> aggregator) throws AggregatorRetrievalException {
- // TODO provide a list of all accumulator step values
- Object value = aggregators.get(aggregator.getName());
- if (value != null) {
- return new AggregatorValues<T>() {
- @Override
- public Map<String, T> getValuesAtSteps() {
- return (Map<String, T>) aggregators;
- }
- };
- } else {
- throw new AggregatorRetrievalException("Accumulator results not found.",
- new RuntimeException("Accumulator does not exist."));
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java
deleted file mode 100644
index ab23b92..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java
+++ /dev/null
@@ -1,452 +0,0 @@
-/*
- * Copyright (C) 2015 Google Inc.
- *
- * 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://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.beam.runners.flink.examples;
-
-import org.apache.beam.runners.flink.FlinkPipelineOptions;
-import org.apache.beam.runners.flink.FlinkPipelineRunner;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.GcsOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.Keys;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.transforms.Values;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.WithKeys;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.util.GcsUtil;
-import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PDone;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.HashSet;
-import java.util.Set;
-
-/**
- * An example that computes a basic TF-IDF search table for a directory or GCS prefix.
- *
- * <p> Concepts: joining data; side inputs; logging
- *
- * <p> To execute this pipeline locally, specify general pipeline configuration:
- * <pre>{@code
- * --project=YOUR_PROJECT_ID
- * }</pre>
- * and a local output file or output prefix on GCS:
- * <pre>{@code
- * --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
- * }</pre>
- *
- * <p> To execute this pipeline using the Dataflow service, specify pipeline configuration:
- * <pre>{@code
- * --project=YOUR_PROJECT_ID
- * --stagingLocation=gs://YOUR_STAGING_DIRECTORY
- * --runner=BlockingDataflowPipelineRunner
- * and an output prefix on GCS:
- * --output=gs://YOUR_OUTPUT_PREFIX
- * }</pre>
- *
- * <p> The default input is {@code gs://dataflow-samples/shakespeare/} and can be overridden with
- * {@code --input}.
- */
-public class TFIDF {
- /**
- * Options supported by {@link TFIDF}.
- * <p>
- * Inherits standard configuration options.
- */
- private interface Options extends PipelineOptions, FlinkPipelineOptions {
- @Description("Path to the directory or GCS prefix containing files to read from")
- @Default.String("gs://dataflow-samples/shakespeare/")
- String getInput();
- void setInput(String value);
-
- @Description("Prefix of output URI to write to")
- @Validation.Required
- String getOutput();
- void setOutput(String value);
- }
-
- /**
- * Lists documents contained beneath the {@code options.input} prefix/directory.
- */
- public static Set<URI> listInputDocuments(Options options)
- throws URISyntaxException, IOException {
- URI baseUri = new URI(options.getInput());
-
- // List all documents in the directory or GCS prefix.
- URI absoluteUri;
- if (baseUri.getScheme() != null) {
- absoluteUri = baseUri;
- } else {
- absoluteUri = new URI(
- "file",
- baseUri.getAuthority(),
- baseUri.getPath(),
- baseUri.getQuery(),
- baseUri.getFragment());
- }
-
- Set<URI> uris = new HashSet<>();
- if (absoluteUri.getScheme().equals("file")) {
- File directory = new File(absoluteUri);
- for (String entry : directory.list()) {
- File path = new File(directory, entry);
- uris.add(path.toURI());
- }
- } else if (absoluteUri.getScheme().equals("gs")) {
- GcsUtil gcsUtil = options.as(GcsOptions.class).getGcsUtil();
- URI gcsUriGlob = new URI(
- absoluteUri.getScheme(),
- absoluteUri.getAuthority(),
- absoluteUri.getPath() + "*",
- absoluteUri.getQuery(),
- absoluteUri.getFragment());
- for (GcsPath entry : gcsUtil.expand(GcsPath.fromUri(gcsUriGlob))) {
- uris.add(entry.toUri());
- }
- }
-
- return uris;
- }
-
- /**
- * Reads the documents at the provided uris and returns all lines
- * from the documents tagged with which document they are from.
- */
- public static class ReadDocuments
- extends PTransform<PInput, PCollection<KV<URI, String>>> {
- private static final long serialVersionUID = 0;
-
- private Iterable<URI> uris;
-
- public ReadDocuments(Iterable<URI> uris) {
- this.uris = uris;
- }
-
- @Override
- public Coder<?> getDefaultOutputCoder() {
- return KvCoder.of(StringDelegateCoder.of(URI.class), StringUtf8Coder.of());
- }
-
- @Override
- public PCollection<KV<URI, String>> apply(PInput input) {
- Pipeline pipeline = input.getPipeline();
-
- // Create one TextIO.Read transform for each document
- // and add its output to a PCollectionList
- PCollectionList<KV<URI, String>> urisToLines =
- PCollectionList.empty(pipeline);
-
- // TextIO.Read supports:
- // - file: URIs and paths locally
- // - gs: URIs on the service
- for (final URI uri : uris) {
- String uriString;
- if (uri.getScheme().equals("file")) {
- uriString = new File(uri).getPath();
- } else {
- uriString = uri.toString();
- }
-
- PCollection<KV<URI, String>> oneUriToLines = pipeline
- .apply(TextIO.Read.from(uriString)
- .named("TextIO.Read(" + uriString + ")"))
- .apply("WithKeys(" + uriString + ")", WithKeys.<URI, String>of(uri));
-
- urisToLines = urisToLines.and(oneUriToLines);
- }
-
- return urisToLines.apply(Flatten.<KV<URI, String>>pCollections());
- }
- }
-
- /**
- * A transform containing a basic TF-IDF pipeline. The input consists of KV objects
- * where the key is the document's URI and the value is a piece
- * of the document's content. The output is mapping from terms to
- * scores for each document URI.
- */
- public static class ComputeTfIdf
- extends PTransform<PCollection<KV<URI, String>>, PCollection<KV<String, KV<URI, Double>>>> {
- private static final long serialVersionUID = 0;
-
- public ComputeTfIdf() { }
-
- @Override
- public PCollection<KV<String, KV<URI, Double>>> apply(
- PCollection<KV<URI, String>> uriToContent) {
-
- // Compute the total number of documents, and
- // prepare this singleton PCollectionView for
- // use as a side input.
- final PCollectionView<Long> totalDocuments =
- uriToContent
- .apply("GetURIs", Keys.<URI>create())
- .apply("RemoveDuplicateDocs", RemoveDuplicates.<URI>create())
- .apply(Count.<URI>globally())
- .apply(View.<Long>asSingleton());
-
- // Create a collection of pairs mapping a URI to each
- // of the words in the document associated with that that URI.
- PCollection<KV<URI, String>> uriToWords = uriToContent
- .apply(ParDo.named("SplitWords").of(
- new DoFn<KV<URI, String>, KV<URI, String>>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- URI uri = c.element().getKey();
- String line = c.element().getValue();
- for (String word : line.split("\\W+")) {
- // Log INFO messages when the word “love” is found.
- if (word.toLowerCase().equals("love")) {
- LOG.info("Found {}", word.toLowerCase());
- }
-
- if (!word.isEmpty()) {
- c.output(KV.of(uri, word.toLowerCase()));
- }
- }
- }
- }));
-
- // Compute a mapping from each word to the total
- // number of documents in which it appears.
- PCollection<KV<String, Long>> wordToDocCount = uriToWords
- .apply("RemoveDuplicateWords", RemoveDuplicates.<KV<URI, String>>create())
- .apply(Values.<String>create())
- .apply("CountDocs", Count.<String>perElement());
-
- // Compute a mapping from each URI to the total
- // number of words in the document associated with that URI.
- PCollection<KV<URI, Long>> uriToWordTotal = uriToWords
- .apply("GetURIs2", Keys.<URI>create())
- .apply("CountWords", Count.<URI>perElement());
-
- // Count, for each (URI, word) pair, the number of
- // occurrences of that word in the document associated
- // with the URI.
- PCollection<KV<KV<URI, String>, Long>> uriAndWordToCount = uriToWords
- .apply("CountWordDocPairs", Count.<KV<URI, String>>perElement());
-
- // Adjust the above collection to a mapping from
- // (URI, word) pairs to counts into an isomorphic mapping
- // from URI to (word, count) pairs, to prepare for a join
- // by the URI key.
- PCollection<KV<URI, KV<String, Long>>> uriToWordAndCount = uriAndWordToCount
- .apply(ParDo.named("ShiftKeys").of(
- new DoFn<KV<KV<URI, String>, Long>, KV<URI, KV<String, Long>>>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- URI uri = c.element().getKey().getKey();
- String word = c.element().getKey().getValue();
- Long occurrences = c.element().getValue();
- c.output(KV.of(uri, KV.of(word, occurrences)));
- }
- }));
-
- // Prepare to join the mapping of URI to (word, count) pairs with
- // the mapping of URI to total word counts, by associating
- // each of the input PCollection<KV<URI, ...>> with
- // a tuple tag. Each input must have the same key type, URI
- // in this case. The type parameter of the tuple tag matches
- // the types of the values for each collection.
- final TupleTag<Long> wordTotalsTag = new TupleTag<>();
- final TupleTag<KV<String, Long>> wordCountsTag = new TupleTag<>();
- KeyedPCollectionTuple<URI> coGbkInput = KeyedPCollectionTuple
- .of(wordTotalsTag, uriToWordTotal)
- .and(wordCountsTag, uriToWordAndCount);
-
- // Perform a CoGroupByKey (a sort of pre-join) on the prepared
- // inputs. This yields a mapping from URI to a CoGbkResult
- // (CoGroupByKey Result). The CoGbkResult is a mapping
- // from the above tuple tags to the values in each input
- // associated with a particular URI. In this case, each
- // KV<URI, CoGbkResult> group a URI with the total number of
- // words in that document as well as all the (word, count)
- // pairs for particular words.
- PCollection<KV<URI, CoGbkResult>> uriToWordAndCountAndTotal = coGbkInput
- .apply("CoGroupByUri", CoGroupByKey.<URI>create());
-
- // Compute a mapping from each word to a (URI, term frequency)
- // pair for each URI. A word's term frequency for a document
- // is simply the number of times that word occurs in the document
- // divided by the total number of words in the document.
- PCollection<KV<String, KV<URI, Double>>> wordToUriAndTf = uriToWordAndCountAndTotal
- .apply(ParDo.named("ComputeTermFrequencies").of(
- new DoFn<KV<URI, CoGbkResult>, KV<String, KV<URI, Double>>>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- URI uri = c.element().getKey();
- Long wordTotal = c.element().getValue().getOnly(wordTotalsTag);
-
- for (KV<String, Long> wordAndCount
- : c.element().getValue().getAll(wordCountsTag)) {
- String word = wordAndCount.getKey();
- Long wordCount = wordAndCount.getValue();
- Double termFrequency = wordCount.doubleValue() / wordTotal.doubleValue();
- c.output(KV.of(word, KV.of(uri, termFrequency)));
- }
- }
- }));
-
- // Compute a mapping from each word to its document frequency.
- // A word's document frequency in a corpus is the number of
- // documents in which the word appears divided by the total
- // number of documents in the corpus. Note how the total number of
- // documents is passed as a side input; the same value is
- // presented to each invocation of the DoFn.
- PCollection<KV<String, Double>> wordToDf = wordToDocCount
- .apply(ParDo
- .named("ComputeDocFrequencies")
- .withSideInputs(totalDocuments)
- .of(new DoFn<KV<String, Long>, KV<String, Double>>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- String word = c.element().getKey();
- Long documentCount = c.element().getValue();
- Long documentTotal = c.sideInput(totalDocuments);
- Double documentFrequency = documentCount.doubleValue()
- / documentTotal.doubleValue();
-
- c.output(KV.of(word, documentFrequency));
- }
- }));
-
- // Join the term frequency and document frequency
- // collections, each keyed on the word.
- final TupleTag<KV<URI, Double>> tfTag = new TupleTag<>();
- final TupleTag<Double> dfTag = new TupleTag<>();
- PCollection<KV<String, CoGbkResult>> wordToUriAndTfAndDf = KeyedPCollectionTuple
- .of(tfTag, wordToUriAndTf)
- .and(dfTag, wordToDf)
- .apply(CoGroupByKey.<String>create());
-
- // Compute a mapping from each word to a (URI, TF-IDF) score
- // for each URI. There are a variety of definitions of TF-IDF
- // ("term frequency - inverse document frequency") score;
- // here we use a basic version that is the term frequency
- // divided by the log of the document frequency.
-
- return wordToUriAndTfAndDf
- .apply(ParDo.named("ComputeTfIdf").of(
- new DoFn<KV<String, CoGbkResult>, KV<String, KV<URI, Double>>>() {
- private static final long serialVersionUID1 = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- String word = c.element().getKey();
- Double df = c.element().getValue().getOnly(dfTag);
-
- for (KV<URI, Double> uriAndTf : c.element().getValue().getAll(tfTag)) {
- URI uri = uriAndTf.getKey();
- Double tf = uriAndTf.getValue();
- Double tfIdf = tf * Math.log(1 / df);
- c.output(KV.of(word, KV.of(uri, tfIdf)));
- }
- }
- }));
- }
-
- // Instantiate Logger.
- // It is suggested that the user specify the class name of the containing class
- // (in this case ComputeTfIdf).
- private static final Logger LOG = LoggerFactory.getLogger(ComputeTfIdf.class);
- }
-
- /**
- * A {@link PTransform} to write, in CSV format, a mapping from term and URI
- * to score.
- */
- public static class WriteTfIdf
- extends PTransform<PCollection<KV<String, KV<URI, Double>>>, PDone> {
- private static final long serialVersionUID = 0;
-
- private String output;
-
- public WriteTfIdf(String output) {
- this.output = output;
- }
-
- @Override
- public PDone apply(PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf) {
- return wordToUriAndTfIdf
- .apply(ParDo.named("Format").of(new DoFn<KV<String, KV<URI, Double>>, String>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- c.output(String.format("%s,\t%s,\t%f",
- c.element().getKey(),
- c.element().getValue().getKey(),
- c.element().getValue().getValue()));
- }
- }))
- .apply(TextIO.Write
- .to(output)
- .withSuffix(".csv"));
- }
- }
-
- public static void main(String[] args) throws Exception {
- Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-
- options.setRunner(FlinkPipelineRunner.class);
-
- Pipeline pipeline = Pipeline.create(options);
- pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
-
- pipeline
- .apply(new ReadDocuments(listInputDocuments(options)))
- .apply(new ComputeTfIdf())
- .apply(new WriteTfIdf(options.getOutput()));
-
- pipeline.run();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java
deleted file mode 100644
index ba46301..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.examples;
-
-import org.apache.beam.runners.flink.FlinkPipelineOptions;
-import org.apache.beam.runners.flink.FlinkPipelineRunner;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-public class WordCount {
-
- public static class ExtractWordsFn extends DoFn<String, String> {
- private final Aggregator<Long, Long> emptyLines =
- createAggregator("emptyLines", new Sum.SumLongFn());
-
- @Override
- public void processElement(ProcessContext c) {
- if (c.element().trim().isEmpty()) {
- emptyLines.addValue(1L);
- }
-
- // Split the line into words.
- String[] words = c.element().split("[^a-zA-Z']+");
-
- // Output each word encountered into the output PCollection.
- for (String word : words) {
- if (!word.isEmpty()) {
- c.output(word);
- }
- }
- }
- }
-
- public static class CountWords extends PTransform<PCollection<String>,
- PCollection<KV<String, Long>>> {
- @Override
- public PCollection<KV<String, Long>> apply(PCollection<String> lines) {
-
- // Convert lines of text into individual words.
- PCollection<String> words = lines.apply(
- ParDo.of(new ExtractWordsFn()));
-
- // Count the number of times each word occurs.
- PCollection<KV<String, Long>> wordCounts =
- words.apply(Count.<String>perElement());
-
- return wordCounts;
- }
- }
-
- /** A SimpleFunction that converts a Word and Count into a printable string. */
- public static class FormatAsTextFn extends SimpleFunction<KV<String, Long>, String> {
- @Override
- public String apply(KV<String, Long> input) {
- return input.getKey() + ": " + input.getValue();
- }
- }
-
- /**
- * Options supported by {@link WordCount}.
- * <p>
- * Inherits standard configuration options.
- */
- public interface Options extends PipelineOptions, FlinkPipelineOptions {
- @Description("Path of the file to read from")
- @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt")
- String getInput();
- void setInput(String value);
-
- @Description("Path of the file to write to")
- String getOutput();
- void setOutput(String value);
- }
-
- public static void main(String[] args) {
-
- Options options = PipelineOptionsFactory.fromArgs(args).withValidation()
- .as(Options.class);
- options.setRunner(FlinkPipelineRunner.class);
-
- Pipeline p = Pipeline.create(options);
-
- p.apply(TextIO.Read.named("ReadLines").from(options.getInput()))
- .apply(new CountWords())
- .apply(MapElements.via(new FormatAsTextFn()))
- .apply(TextIO.Write.named("WriteCounts").to(options.getOutput()));
-
- p.run();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java
deleted file mode 100644
index 8168122..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java
+++ /dev/null
@@ -1,387 +0,0 @@
-/*
- * Copyright (C) 2015 Google Inc.
- *
- * 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://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.beam.runners.flink.examples.streaming;
-
-import org.apache.beam.runners.flink.FlinkPipelineRunner;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
-import com.google.cloud.dataflow.sdk.io.*;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.transforms.Partition.PartitionFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.*;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import org.joda.time.Duration;
-
-import java.io.IOException;
-import java.util.List;
-
-/**
- * To run the example, first open a socket on a terminal by executing the command:
- * <li>
- * <li>
- * <code>nc -lk 9999</code>
- * </li>
- * </li>
- * and then launch the example. Now whatever you type in the terminal is going to be
- * the input to the program.
- * */
-public class AutoComplete {
-
- /**
- * A PTransform that takes as input a list of tokens and returns
- * the most common tokens per prefix.
- */
- public static class ComputeTopCompletions
- extends PTransform<PCollection<String>, PCollection<KV<String, List<CompletionCandidate>>>> {
- private static final long serialVersionUID = 0;
-
- private final int candidatesPerPrefix;
- private final boolean recursive;
-
- protected ComputeTopCompletions(int candidatesPerPrefix, boolean recursive) {
- this.candidatesPerPrefix = candidatesPerPrefix;
- this.recursive = recursive;
- }
-
- public static ComputeTopCompletions top(int candidatesPerPrefix, boolean recursive) {
- return new ComputeTopCompletions(candidatesPerPrefix, recursive);
- }
-
- @Override
- public PCollection<KV<String, List<CompletionCandidate>>> apply(PCollection<String> input) {
- PCollection<CompletionCandidate> candidates = input
- // First count how often each token appears.
- .apply(new Count.PerElement<String>())
-
- // Map the KV outputs of Count into our own CompletionCandiate class.
- .apply(ParDo.named("CreateCompletionCandidates").of(
- new DoFn<KV<String, Long>, CompletionCandidate>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- CompletionCandidate cand = new CompletionCandidate(c.element().getKey(), c.element().getValue());
- c.output(cand);
- }
- }));
-
- // Compute the top via either a flat or recursive algorithm.
- if (recursive) {
- return candidates
- .apply(new ComputeTopRecursive(candidatesPerPrefix, 1))
- .apply(Flatten.<KV<String, List<CompletionCandidate>>>pCollections());
- } else {
- return candidates
- .apply(new ComputeTopFlat(candidatesPerPrefix, 1));
- }
- }
- }
-
- /**
- * Lower latency, but more expensive.
- */
- private static class ComputeTopFlat
- extends PTransform<PCollection<CompletionCandidate>,
- PCollection<KV<String, List<CompletionCandidate>>>> {
- private static final long serialVersionUID = 0;
-
- private final int candidatesPerPrefix;
- private final int minPrefix;
-
- public ComputeTopFlat(int candidatesPerPrefix, int minPrefix) {
- this.candidatesPerPrefix = candidatesPerPrefix;
- this.minPrefix = minPrefix;
- }
-
- @Override
- public PCollection<KV<String, List<CompletionCandidate>>> apply(
- PCollection<CompletionCandidate> input) {
- return input
- // For each completion candidate, map it to all prefixes.
- .apply(ParDo.of(new AllPrefixes(minPrefix)))
-
- // Find and return the top candiates for each prefix.
- .apply(Top.<String, CompletionCandidate>largestPerKey(candidatesPerPrefix)
- .withHotKeyFanout(new HotKeyFanout()));
- }
-
- private static class HotKeyFanout implements SerializableFunction<String, Integer> {
- private static final long serialVersionUID = 0;
-
- @Override
- public Integer apply(String input) {
- return (int) Math.pow(4, 5 - input.length());
- }
- }
- }
-
- /**
- * Cheaper but higher latency.
- *
- * <p> Returns two PCollections, the first is top prefixes of size greater
- * than minPrefix, and the second is top prefixes of size exactly
- * minPrefix.
- */
- private static class ComputeTopRecursive
- extends PTransform<PCollection<CompletionCandidate>,
- PCollectionList<KV<String, List<CompletionCandidate>>>> {
- private static final long serialVersionUID = 0;
-
- private final int candidatesPerPrefix;
- private final int minPrefix;
-
- public ComputeTopRecursive(int candidatesPerPrefix, int minPrefix) {
- this.candidatesPerPrefix = candidatesPerPrefix;
- this.minPrefix = minPrefix;
- }
-
- private class KeySizePartitionFn implements PartitionFn<KV<String, List<CompletionCandidate>>> {
- private static final long serialVersionUID = 0;
-
- @Override
- public int partitionFor(KV<String, List<CompletionCandidate>> elem, int numPartitions) {
- return elem.getKey().length() > minPrefix ? 0 : 1;
- }
- }
-
- private static class FlattenTops
- extends DoFn<KV<String, List<CompletionCandidate>>, CompletionCandidate> {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- for (CompletionCandidate cc : c.element().getValue()) {
- c.output(cc);
- }
- }
- }
-
- @Override
- public PCollectionList<KV<String, List<CompletionCandidate>>> apply(
- PCollection<CompletionCandidate> input) {
- if (minPrefix > 10) {
- // Base case, partitioning to return the output in the expected format.
- return input
- .apply(new ComputeTopFlat(candidatesPerPrefix, minPrefix))
- .apply(Partition.of(2, new KeySizePartitionFn()));
- } else {
- // If a candidate is in the top N for prefix a...b, it must also be in the top
- // N for a...bX for every X, which is typlically a much smaller set to consider.
- // First, compute the top candidate for prefixes of size at least minPrefix + 1.
- PCollectionList<KV<String, List<CompletionCandidate>>> larger = input
- .apply(new ComputeTopRecursive(candidatesPerPrefix, minPrefix + 1));
- // Consider the top candidates for each prefix of length minPrefix + 1...
- PCollection<KV<String, List<CompletionCandidate>>> small =
- PCollectionList
- .of(larger.get(1).apply(ParDo.of(new FlattenTops())))
- // ...together with those (previously excluded) candidates of length
- // exactly minPrefix...
- .and(input.apply(Filter.by(new SerializableFunction<CompletionCandidate, Boolean>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public Boolean apply(CompletionCandidate c) {
- return c.getValue().length() == minPrefix;
- }
- })))
- .apply("FlattenSmall", Flatten.<CompletionCandidate>pCollections())
- // ...set the key to be the minPrefix-length prefix...
- .apply(ParDo.of(new AllPrefixes(minPrefix, minPrefix)))
- // ...and (re)apply the Top operator to all of them together.
- .apply(Top.<String, CompletionCandidate>largestPerKey(candidatesPerPrefix));
-
- PCollection<KV<String, List<CompletionCandidate>>> flattenLarger = larger
- .apply("FlattenLarge", Flatten.<KV<String, List<CompletionCandidate>>>pCollections());
-
- return PCollectionList.of(flattenLarger).and(small);
- }
- }
- }
-
- /**
- * A DoFn that keys each candidate by all its prefixes.
- */
- private static class AllPrefixes
- extends DoFn<CompletionCandidate, KV<String, CompletionCandidate>> {
- private static final long serialVersionUID = 0;
-
- private final int minPrefix;
- private final int maxPrefix;
- public AllPrefixes(int minPrefix) {
- this(minPrefix, Integer.MAX_VALUE);
- }
- public AllPrefixes(int minPrefix, int maxPrefix) {
- this.minPrefix = minPrefix;
- this.maxPrefix = maxPrefix;
- }
- @Override
- public void processElement(ProcessContext c) {
- String word = c.element().value;
- for (int i = minPrefix; i <= Math.min(word.length(), maxPrefix); i++) {
- KV<String, CompletionCandidate> kv = KV.of(word.substring(0, i), c.element());
- c.output(kv);
- }
- }
- }
-
- /**
- * Class used to store tag-count pairs.
- */
- @DefaultCoder(AvroCoder.class)
- static class CompletionCandidate implements Comparable<CompletionCandidate> {
- private long count;
- private String value;
-
- public CompletionCandidate(String value, long count) {
- this.value = value;
- this.count = count;
- }
-
- public String getValue() {
- return value;
- }
-
- // Empty constructor required for Avro decoding.
- @SuppressWarnings("unused")
- public CompletionCandidate() {}
-
- @Override
- public int compareTo(CompletionCandidate o) {
- if (this.count < o.count) {
- return -1;
- } else if (this.count == o.count) {
- return this.value.compareTo(o.value);
- } else {
- return 1;
- }
- }
-
- @Override
- public boolean equals(Object other) {
- if (other instanceof CompletionCandidate) {
- CompletionCandidate that = (CompletionCandidate) other;
- return this.count == that.count && this.value.equals(that.value);
- } else {
- return false;
- }
- }
-
- @Override
- public int hashCode() {
- return Long.valueOf(count).hashCode() ^ value.hashCode();
- }
-
- @Override
- public String toString() {
- return "CompletionCandidate[" + value + ", " + count + "]";
- }
- }
-
- static class ExtractWordsFn extends DoFn<String, String> {
- private final Aggregator<Long, Long> emptyLines =
- createAggregator("emptyLines", new Sum.SumLongFn());
-
- @Override
- public void processElement(ProcessContext c) {
- if (c.element().trim().isEmpty()) {
- emptyLines.addValue(1L);
- }
-
- // Split the line into words.
- String[] words = c.element().split("[^a-zA-Z']+");
-
- // Output each word encountered into the output PCollection.
- for (String word : words) {
- if (!word.isEmpty()) {
- c.output(word);
- }
- }
- }
- }
-
- /**
- * Takes as input a the top candidates per prefix, and emits an entity
- * suitable for writing to Datastore.
- */
- static class FormatForPerTaskLocalFile extends DoFn<KV<String, List<CompletionCandidate>>, String>
- implements DoFn.RequiresWindowAccess{
-
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- StringBuilder str = new StringBuilder();
- KV<String, List<CompletionCandidate>> elem = c.element();
-
- str.append(elem.getKey() +" @ "+ c.window() +" -> ");
- for(CompletionCandidate cand: elem.getValue()) {
- str.append(cand.toString() + " ");
- }
- System.out.println(str.toString());
- c.output(str.toString());
- }
- }
-
- /**
- * Options supported by this class.
- *
- * <p> Inherits standard Dataflow configuration options.
- */
- private interface Options extends WindowedWordCount.StreamingWordCountOptions {
- @Description("Whether to use the recursive algorithm")
- @Default.Boolean(true)
- Boolean getRecursive();
- void setRecursive(Boolean value);
- }
-
- public static void main(String[] args) throws IOException {
- Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
- options.setStreaming(true);
- options.setCheckpointingInterval(1000L);
- options.setNumberOfExecutionRetries(5);
- options.setExecutionRetryDelay(3000L);
- options.setRunner(FlinkPipelineRunner.class);
-
- PTransform<? super PBegin, PCollection<String>> readSource =
- Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("WordStream");
- WindowFn<Object, ?> windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize()));
-
- // Create the pipeline.
- Pipeline p = Pipeline.create(options);
- PCollection<KV<String, List<CompletionCandidate>>> toWrite = p
- .apply(readSource)
- .apply(ParDo.of(new ExtractWordsFn()))
- .apply(Window.<String>into(windowFn)
- .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
- .discardingFiredPanes())
- .apply(ComputeTopCompletions.top(10, options.getRecursive()));
-
- toWrite
- .apply(ParDo.named("FormatForPerTaskFile").of(new FormatForPerTaskLocalFile()))
- .apply(TextIO.Write.to("./outputAutoComplete.txt"));
-
- p.run();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
deleted file mode 100644
index 3a8bdb0..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
- * Copyright (C) 2015 Google Inc.
- *
- * 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://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.beam.runners.flink.examples.streaming;
-
-import org.apache.beam.runners.flink.FlinkPipelineRunner;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.transforms.windowing.*;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PBegin;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import org.joda.time.Duration;
-
-/**
- * To run the example, first open two sockets on two terminals by executing the commands:
- * <li>
- * <li>
- * <code>nc -lk 9999</code>, and
- * </li>
- * <li>
- * <code>nc -lk 9998</code>
- * </li>
- * </li>
- * and then launch the example. Now whatever you type in the terminal is going to be
- * the input to the program.
- * */
-public class JoinExamples {
-
- static PCollection<String> joinEvents(PCollection<String> streamA,
- PCollection<String> streamB) throws Exception {
-
- final TupleTag<String> firstInfoTag = new TupleTag<>();
- final TupleTag<String> secondInfoTag = new TupleTag<>();
-
- // transform both input collections to tuple collections, where the keys are country
- // codes in both cases.
- PCollection<KV<String, String>> firstInfo = streamA.apply(
- ParDo.of(new ExtractEventDataFn()));
- PCollection<KV<String, String>> secondInfo = streamB.apply(
- ParDo.of(new ExtractEventDataFn()));
-
- // country code 'key' -> CGBKR (<event info>, <country name>)
- PCollection<KV<String, CoGbkResult>> kvpCollection = KeyedPCollectionTuple
- .of(firstInfoTag, firstInfo)
- .and(secondInfoTag, secondInfo)
- .apply(CoGroupByKey.<String>create());
-
- // Process the CoGbkResult elements generated by the CoGroupByKey transform.
- // country code 'key' -> string of <event info>, <country name>
- PCollection<KV<String, String>> finalResultCollection =
- kvpCollection.apply(ParDo.named("Process").of(
- new DoFn<KV<String, CoGbkResult>, KV<String, String>>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- KV<String, CoGbkResult> e = c.element();
- String key = e.getKey();
-
- String defaultA = "NO_VALUE";
-
- // the following getOnly is a bit tricky because it expects to have
- // EXACTLY ONE value in the corresponding stream and for the corresponding key.
-
- String lineA = e.getValue().getOnly(firstInfoTag, defaultA);
- for (String lineB : c.element().getValue().getAll(secondInfoTag)) {
- // Generate a string that combines information from both collection values
- c.output(KV.of(key, "Value A: " + lineA + " - Value B: " + lineB));
- }
- }
- }));
-
- return finalResultCollection
- .apply(ParDo.named("Format").of(new DoFn<KV<String, String>, String>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- String result = c.element().getKey() + " -> " + c.element().getValue();
- System.out.println(result);
- c.output(result);
- }
- }));
- }
-
- static class ExtractEventDataFn extends DoFn<String, KV<String, String>> {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- String line = c.element().toLowerCase();
- String key = line.split("\\s")[0];
- c.output(KV.of(key, line));
- }
- }
-
- private interface Options extends WindowedWordCount.StreamingWordCountOptions {
-
- }
-
- public static void main(String[] args) throws Exception {
- Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
- options.setStreaming(true);
- options.setCheckpointingInterval(1000L);
- options.setNumberOfExecutionRetries(5);
- options.setExecutionRetryDelay(3000L);
- options.setRunner(FlinkPipelineRunner.class);
-
- PTransform<? super PBegin, PCollection<String>> readSourceA =
- Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("FirstStream");
- PTransform<? super PBegin, PCollection<String>> readSourceB =
- Read.from(new UnboundedSocketSource<>("localhost", 9998, '\n', 3)).named("SecondStream");
-
- WindowFn<Object, ?> windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize()));
-
- Pipeline p = Pipeline.create(options);
-
- // the following two 'applys' create multiple inputs to our pipeline, one for each
- // of our two input sources.
- PCollection<String> streamA = p.apply(readSourceA)
- .apply(Window.<String>into(windowFn)
- .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
- .discardingFiredPanes());
- PCollection<String> streamB = p.apply(readSourceB)
- .apply(Window.<String>into(windowFn)
- .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
- .discardingFiredPanes());
-
- PCollection<String> formattedResults = joinEvents(streamA, streamB);
- formattedResults.apply(TextIO.Write.to("./outputJoin.txt"));
- p.run();
- }
-
-}
[37/50] [abbrv] incubator-beam git commit: Fixes Void handling
Posted by da...@apache.org.
Fixes Void handling
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/a4e9b09f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/a4e9b09f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/a4e9b09f
Branch: refs/heads/master
Commit: a4e9b09fb4690b4e110afa6bc5744b3646980115
Parents: 067837f
Author: kl0u <kk...@gmail.com>
Authored: Mon Feb 29 16:26:12 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
.../streaming/FlinkGroupByKeyWrapper.java | 8 +-
.../flink/dataflow/FlinkTestPipeline.java | 4 +-
.../dataflow/streaming/GroupByNullKeyTest.java | 121 +++++++++++++++++++
3 files changed, 128 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a4e9b09f/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
index b0d9e48..24f6d40 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
@@ -16,9 +16,11 @@
package com.dataartisans.flink.dataflow.translation.wrappers.streaming;
import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation;
+import com.dataartisans.flink.dataflow.translation.types.VoidCoderTypeSerializer;
import com.google.cloud.dataflow.sdk.coders.Coder;
import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.util.*;
+import com.google.cloud.dataflow.sdk.coders.VoidCoder;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
import com.google.cloud.dataflow.sdk.values.KV;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.functions.KeySelector;
@@ -42,13 +44,15 @@ public class FlinkGroupByKeyWrapper {
public static <K, V> KeyedStream<WindowedValue<KV<K, V>>, K> groupStreamByKey(DataStream<WindowedValue<KV<K, V>>> inputDataStream, KvCoder<K, V> inputKvCoder) {
final Coder<K> keyCoder = inputKvCoder.getKeyCoder();
final TypeInformation<K> keyTypeInfo = new CoderTypeInformation<>(keyCoder);
+ final boolean isKeyVoid = keyCoder instanceof VoidCoder;
return inputDataStream.keyBy(
new KeySelectorWithQueryableResultType<K, V>() {
@Override
public K getKey(WindowedValue<KV<K, V>> value) throws Exception {
- return value.getValue().getKey();
+ return isKeyVoid ? (K) VoidCoderTypeSerializer.VoidValue.INSTANCE :
+ value.getValue().getKey();
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a4e9b09f/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
index 56af3f1..59c3b69 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
@@ -59,9 +59,7 @@ public class FlinkTestPipeline extends Pipeline {
*/
private static FlinkTestPipeline create(boolean streaming) {
FlinkPipelineRunner flinkRunner = FlinkPipelineRunner.createForTest(streaming);
- FlinkPipelineOptions pipelineOptions = flinkRunner.getPipelineOptions();
- pipelineOptions.setStreaming(streaming);
- return new FlinkTestPipeline(flinkRunner, pipelineOptions);
+ return new FlinkTestPipeline(flinkRunner, flinkRunner.getPipelineOptions());
}
private FlinkTestPipeline(PipelineRunner<? extends PipelineResult> runner,
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/a4e9b09f/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java
new file mode 100644
index 0000000..5a412aa
--- /dev/null
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java
@@ -0,0 +1,121 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.streaming;
+
+import com.dataartisans.flink.dataflow.FlinkTestPipeline;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Joiner;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.io.Serializable;
+import java.util.Arrays;
+
+public class GroupByNullKeyTest extends StreamingProgramTestBase implements Serializable {
+
+
+ protected String resultPath;
+
+ static final String[] EXPECTED_RESULT = new String[] {
+ "k: null v: user1 user1 user1 user2 user2 user2 user2 user3"
+ };
+
+ public GroupByNullKeyTest(){
+ }
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
+
+ public static class ExtractUserAndTimestamp extends DoFn<KV<Integer, String>, String> {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ KV<Integer, String> record = c.element();
+ long now = System.currentTimeMillis();
+ int timestamp = record.getKey();
+ String userName = record.getValue();
+ if (userName != null) {
+ // Sets the implicit timestamp field to be used in windowing.
+ c.outputWithTimestamp(userName, new Instant(timestamp + now));
+ }
+ }
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+
+ Pipeline p = FlinkTestPipeline.createForStreaming();
+
+ PCollection<String> output =
+ p.apply(Create.of(Arrays.asList(
+ KV.<Integer, String>of(0, "user1"),
+ KV.<Integer, String>of(1, "user1"),
+ KV.<Integer, String>of(2, "user1"),
+ KV.<Integer, String>of(10, "user2"),
+ KV.<Integer, String>of(1, "user2"),
+ KV.<Integer, String>of(15000, "user2"),
+ KV.<Integer, String>of(12000, "user2"),
+ KV.<Integer, String>of(25000, "user3"))))
+ .apply(ParDo.of(new ExtractUserAndTimestamp()))
+ .apply(Window.<String>into(FixedWindows.of(Duration.standardHours(1)))
+ .triggering(AfterWatermark.pastEndOfWindow())
+ .withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes())
+
+ .apply(ParDo.of(new DoFn<String, KV<Void, String>>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ String elem = c.element();
+ c.output(KV.<Void, String>of((Void) null, elem));
+ }
+ }))
+ .apply(GroupByKey.<Void, String>create())
+ .apply(ParDo.of(new DoFn<KV<Void, Iterable<String>>, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ KV<Void, Iterable<String>> elem = c.element();
+ StringBuilder str = new StringBuilder();
+ str.append("k: " + elem.getKey() + " v:");
+ for (String v : elem.getValue()) {
+ str.append(" " + v);
+ }
+ c.output(str.toString());
+ }
+ }));
+ output.apply(TextIO.Write.to(resultPath));
+ p.run();
+ }
+}
[20/50] [abbrv] incubator-beam git commit: [flink] adjust directories
according to package name
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java
new file mode 100644
index 0000000..e39b81d
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.VoidCoder;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import java.io.Serializable;
+
+public class MaybeEmptyTestITCase extends JavaProgramTestBase implements Serializable {
+
+ protected String resultPath;
+
+ protected final String expected = "test";
+
+ public MaybeEmptyTestITCase() {
+ }
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(expected, resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ p.apply(Create.of((Void) null)).setCoder(VoidCoder.of())
+ .apply(ParDo.of(
+ new DoFn<Void, String>() {
+ @Override
+ public void processElement(DoFn<Void, String>.ProcessContext c) {
+ c.output(expected);
+ }
+ })).apply(TextIO.Write.to(resultPath));
+ p.run();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java
new file mode 100644
index 0000000..08e5323
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.cloud.dataflow.sdk.values.TupleTagList;
+import com.google.common.base.Joiner;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import java.io.Serializable;
+
+public class ParDoMultiOutputITCase extends JavaProgramTestBase implements Serializable {
+
+ private String resultPath;
+
+ private static String[] expectedWords = {"MAAA", "MAAFOOO"};
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on("\n").join(expectedWords), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ PCollection<String> words = p.apply(Create.of("Hello", "Whatupmyman", "hey", "SPECIALthere", "MAAA", "MAAFOOO"));
+
+ // Select words whose length is below a cut off,
+ // plus the lengths of words that are above the cut off.
+ // Also select words starting with "MARKER".
+ final int wordLengthCutOff = 3;
+ // Create tags to use for the main and side outputs.
+ final TupleTag<String> wordsBelowCutOffTag = new TupleTag<String>(){};
+ final TupleTag<Integer> wordLengthsAboveCutOffTag = new TupleTag<Integer>(){};
+ final TupleTag<String> markedWordsTag = new TupleTag<String>(){};
+
+ PCollectionTuple results =
+ words.apply(ParDo
+ .withOutputTags(wordsBelowCutOffTag, TupleTagList.of(wordLengthsAboveCutOffTag)
+ .and(markedWordsTag))
+ .of(new DoFn<String, String>() {
+ final TupleTag<String> specialWordsTag = new TupleTag<String>() {
+ };
+
+ public void processElement(ProcessContext c) {
+ String word = c.element();
+ if (word.length() <= wordLengthCutOff) {
+ c.output(word);
+ } else {
+ c.sideOutput(wordLengthsAboveCutOffTag, word.length());
+ }
+ if (word.startsWith("MAA")) {
+ c.sideOutput(markedWordsTag, word);
+ }
+
+ if (word.startsWith("SPECIAL")) {
+ c.sideOutput(specialWordsTag, word);
+ }
+ }
+ }));
+
+ // Extract the PCollection results, by tag.
+ PCollection<String> wordsBelowCutOff = results.get(wordsBelowCutOffTag);
+ PCollection<Integer> wordLengthsAboveCutOff = results.get
+ (wordLengthsAboveCutOffTag);
+ PCollection<String> markedWords = results.get(markedWordsTag);
+
+ markedWords.apply(TextIO.Write.to(resultPath));
+
+ p.run();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java
new file mode 100644
index 0000000..7202417
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java
@@ -0,0 +1,163 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.io.BoundedSource;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+
+public class ReadSourceITCase extends JavaProgramTestBase {
+
+ protected String resultPath;
+
+ public ReadSourceITCase(){
+ }
+
+ static final String[] EXPECTED_RESULT = new String[] {
+ "1", "2", "3", "4", "5", "6", "7", "8", "9"};
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+ runProgram(resultPath);
+ }
+
+ private static void runProgram(String resultPath) {
+
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ PCollection<String> result = p
+ .apply(Read.from(new ReadSource(1, 10)))
+ .apply(ParDo.of(new DoFn<Integer, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ c.output(c.element().toString());
+ }
+ }));
+
+ result.apply(TextIO.Write.to(resultPath));
+ p.run();
+ }
+
+
+ private static class ReadSource extends BoundedSource<Integer> {
+ final int from;
+ final int to;
+
+ ReadSource(int from, int to) {
+ this.from = from;
+ this.to = to;
+ }
+
+ @Override
+ public List<ReadSource> splitIntoBundles(long desiredShardSizeBytes, PipelineOptions options)
+ throws Exception {
+ List<ReadSource> res = new ArrayList<>();
+ FlinkPipelineOptions flinkOptions = options.as(FlinkPipelineOptions.class);
+ int numWorkers = flinkOptions.getParallelism();
+ Preconditions.checkArgument(numWorkers > 0, "Number of workers should be larger than 0.");
+
+ float step = 1.0f * (to - from) / numWorkers;
+ for (int i = 0; i < numWorkers; ++i) {
+ res.add(new ReadSource(Math.round(from + i * step), Math.round(from + (i + 1) * step)));
+ }
+ return res;
+ }
+
+ @Override
+ public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
+ return 8 * (to - from);
+ }
+
+ @Override
+ public boolean producesSortedKeys(PipelineOptions options) throws Exception {
+ return true;
+ }
+
+ @Override
+ public BoundedReader<Integer> createReader(PipelineOptions options) throws IOException {
+ return new RangeReader(this);
+ }
+
+ @Override
+ public void validate() {}
+
+ @Override
+ public Coder<Integer> getDefaultOutputCoder() {
+ return BigEndianIntegerCoder.of();
+ }
+
+ private class RangeReader extends BoundedReader<Integer> {
+ private int current;
+
+ public RangeReader(ReadSource source) {
+ this.current = source.from - 1;
+ }
+
+ @Override
+ public boolean start() throws IOException {
+ return true;
+ }
+
+ @Override
+ public boolean advance() throws IOException {
+ current++;
+ return (current < to);
+ }
+
+ @Override
+ public Integer getCurrent() {
+ return current;
+ }
+
+ @Override
+ public void close() throws IOException {
+ // Nothing
+ }
+
+ @Override
+ public BoundedSource<Integer> getCurrentSource() {
+ return ReadSource.this;
+ }
+ }
+ }
+}
+
+
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java
new file mode 100644
index 0000000..dc82d7d
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java
@@ -0,0 +1,68 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Joiner;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import java.util.Collections;
+import java.util.List;
+
+
+public class RemoveDuplicatesEmptyITCase extends JavaProgramTestBase {
+
+ protected String resultPath;
+
+ public RemoveDuplicatesEmptyITCase(){
+ }
+
+ static final String[] EXPECTED_RESULT = new String[] {};
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+
+ List<String> strings = Collections.emptyList();
+
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ PCollection<String> input =
+ p.apply(Create.of(strings))
+ .setCoder(StringUtf8Coder.of());
+
+ PCollection<String> output =
+ input.apply(RemoveDuplicates.<String>create());
+
+ output.apply(TextIO.Write.to(resultPath));
+ p.run();
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java
new file mode 100644
index 0000000..78b48b5
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java
@@ -0,0 +1,69 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Joiner;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import java.util.Arrays;
+import java.util.List;
+
+
+public class RemoveDuplicatesITCase extends JavaProgramTestBase {
+
+ protected String resultPath;
+
+ public RemoveDuplicatesITCase(){
+ }
+
+ static final String[] EXPECTED_RESULT = new String[] {
+ "k1", "k5", "k2", "k3"};
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+
+ List<String> strings = Arrays.asList("k1", "k5", "k5", "k2", "k1", "k2", "k3");
+
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ PCollection<String> input =
+ p.apply(Create.of(strings))
+ .setCoder(StringUtf8Coder.of());
+
+ PCollection<String> output =
+ input.apply(RemoveDuplicates.<String>create());
+
+ output.apply(TextIO.Write.to(resultPath));
+ p.run();
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java
new file mode 100644
index 0000000..5cd7d78
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java
@@ -0,0 +1,67 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import java.io.Serializable;
+
+public class SideInputITCase extends JavaProgramTestBase implements Serializable {
+
+ private static final String expected = "Hello!";
+
+ protected String resultPath;
+
+ @Override
+ protected void testProgram() throws Exception {
+
+
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+
+ final PCollectionView<String> sidesInput = p
+ .apply(Create.of(expected))
+ .apply(View.<String>asSingleton());
+
+ p.apply(Create.of("bli"))
+ .apply(ParDo.of(new DoFn<String, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ String s = c.sideInput(sidesInput);
+ c.output(s);
+ }
+ }).withSideInputs(sidesInput)).apply(TextIO.Write.to(resultPath));
+
+ p.run();
+ }
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(expected, resultPath);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java
new file mode 100644
index 0000000..ceb0a3f
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java
@@ -0,0 +1,76 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import org.apache.beam.runners.flink.examples.TFIDF;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.Keys;
+import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Joiner;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import java.net.URI;
+
+
+public class TfIdfITCase extends JavaProgramTestBase {
+
+ protected String resultPath;
+
+ public TfIdfITCase(){
+ }
+
+ static final String[] EXPECTED_RESULT = new String[] {
+ "a", "m", "n", "b", "c", "d"};
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+
+ Pipeline pipeline = FlinkTestPipeline.createForBatch();
+
+ pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
+
+ PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf = pipeline
+ .apply(Create.of(
+ KV.of(new URI("x"), "a b c d"),
+ KV.of(new URI("y"), "a b c"),
+ KV.of(new URI("z"), "a m n")))
+ .apply(new TFIDF.ComputeTfIdf());
+
+ PCollection<String> words = wordToUriAndTfIdf
+ .apply(Keys.<String>create())
+ .apply(RemoveDuplicates.<String>create());
+
+ words.apply(TextIO.Write.to(resultPath));
+
+ pipeline.run();
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java
new file mode 100644
index 0000000..c2b6fdd
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java
@@ -0,0 +1,74 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import org.apache.beam.runners.flink.examples.WordCount;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.MapElements;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Joiner;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import java.util.Arrays;
+import java.util.List;
+
+
+public class WordCountITCase extends JavaProgramTestBase {
+
+ protected String resultPath;
+
+ public WordCountITCase(){
+ }
+
+ static final String[] WORDS_ARRAY = new String[] {
+ "hi there", "hi", "hi sue bob",
+ "hi sue", "", "bob hi"};
+
+ static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
+
+ static final String[] COUNTS_ARRAY = new String[] {
+ "hi: 5", "there: 1", "sue: 2", "bob: 2"};
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(COUNTS_ARRAY), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ PCollection<String> input = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of());
+
+ input
+ .apply(new WordCount.CountWords())
+ .apply(MapElements.via(new WordCount.FormatAsTextFn()))
+ .apply(TextIO.Write.to(resultPath));
+
+ p.run();
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java
new file mode 100644
index 0000000..d78434b
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java
@@ -0,0 +1,136 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Count;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.base.Joiner;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+
+public class WordCountJoin2ITCase extends JavaProgramTestBase {
+
+ static final String[] WORDS_1 = new String[] {
+ "hi there", "hi", "hi sue bob",
+ "hi sue", "", "bob hi"};
+
+ static final String[] WORDS_2 = new String[] {
+ "hi tim", "beauty", "hooray sue bob",
+ "hi there", "", "please say hi"};
+
+ static final String[] RESULTS = new String[] {
+ "beauty -> Tag1: Tag2: 1",
+ "bob -> Tag1: 2 Tag2: 1",
+ "hi -> Tag1: 5 Tag2: 3",
+ "hooray -> Tag1: Tag2: 1",
+ "please -> Tag1: Tag2: 1",
+ "say -> Tag1: Tag2: 1",
+ "sue -> Tag1: 2 Tag2: 1",
+ "there -> Tag1: 1 Tag2: 1",
+ "tim -> Tag1: Tag2: 1"
+ };
+
+ static final TupleTag<Long> tag1 = new TupleTag<>("Tag1");
+ static final TupleTag<Long> tag2 = new TupleTag<>("Tag2");
+
+ protected String resultPath;
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(RESULTS), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ /* Create two PCollections and join them */
+ PCollection<KV<String,Long>> occurences1 = p.apply(Create.of(WORDS_1))
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Count.<String>perElement());
+
+ PCollection<KV<String,Long>> occurences2 = p.apply(Create.of(WORDS_2))
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Count.<String>perElement());
+
+ /* CoGroup the two collections */
+ PCollection<KV<String, CoGbkResult>> mergedOccurences = KeyedPCollectionTuple
+ .of(tag1, occurences1)
+ .and(tag2, occurences2)
+ .apply(CoGroupByKey.<String>create());
+
+ /* Format output */
+ mergedOccurences.apply(ParDo.of(new FormatCountsFn()))
+ .apply(TextIO.Write.named("test").to(resultPath));
+
+ p.run();
+ }
+
+
+ static class ExtractWordsFn extends DoFn<String, String> {
+
+ @Override
+ public void startBundle(Context c) {
+ }
+
+ @Override
+ public void processElement(ProcessContext c) {
+ // Split the line into words.
+ String[] words = c.element().split("[^a-zA-Z']+");
+
+ // Output each word encountered into the output PCollection.
+ for (String word : words) {
+ if (!word.isEmpty()) {
+ c.output(word);
+ }
+ }
+ }
+ }
+
+ static class FormatCountsFn extends DoFn<KV<String, CoGbkResult>, String> {
+ @Override
+ public void processElement(ProcessContext c) {
+ CoGbkResult value = c.element().getValue();
+ String key = c.element().getKey();
+ String countTag1 = tag1.getId() + ": ";
+ String countTag2 = tag2.getId() + ": ";
+ for (Long count : value.getAll(tag1)) {
+ countTag1 += count + " ";
+ }
+ for (Long count : value.getAll(tag2)) {
+ countTag2 += count;
+ }
+ c.output(key + " -> " + countTag1 + countTag2);
+ }
+ }
+
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java
new file mode 100644
index 0000000..0836279
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java
@@ -0,0 +1,154 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Count;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.base.Joiner;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+
+public class WordCountJoin3ITCase extends JavaProgramTestBase {
+
+ static final String[] WORDS_1 = new String[] {
+ "hi there", "hi", "hi sue bob",
+ "hi sue", "", "bob hi"};
+
+ static final String[] WORDS_2 = new String[] {
+ "hi tim", "beauty", "hooray sue bob",
+ "hi there", "", "please say hi"};
+
+ static final String[] WORDS_3 = new String[] {
+ "hi stephan", "beauty", "hooray big fabian",
+ "hi yo", "", "please say hi"};
+
+ static final String[] RESULTS = new String[] {
+ "beauty -> Tag1: Tag2: 1 Tag3: 1",
+ "bob -> Tag1: 2 Tag2: 1 Tag3: ",
+ "hi -> Tag1: 5 Tag2: 3 Tag3: 3",
+ "hooray -> Tag1: Tag2: 1 Tag3: 1",
+ "please -> Tag1: Tag2: 1 Tag3: 1",
+ "say -> Tag1: Tag2: 1 Tag3: 1",
+ "sue -> Tag1: 2 Tag2: 1 Tag3: ",
+ "there -> Tag1: 1 Tag2: 1 Tag3: ",
+ "tim -> Tag1: Tag2: 1 Tag3: ",
+ "stephan -> Tag1: Tag2: Tag3: 1",
+ "yo -> Tag1: Tag2: Tag3: 1",
+ "fabian -> Tag1: Tag2: Tag3: 1",
+ "big -> Tag1: Tag2: Tag3: 1"
+ };
+
+ static final TupleTag<Long> tag1 = new TupleTag<>("Tag1");
+ static final TupleTag<Long> tag2 = new TupleTag<>("Tag2");
+ static final TupleTag<Long> tag3 = new TupleTag<>("Tag3");
+
+ protected String resultPath;
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(RESULTS), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ /* Create two PCollections and join them */
+ PCollection<KV<String,Long>> occurences1 = p.apply(Create.of(WORDS_1))
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Count.<String>perElement());
+
+ PCollection<KV<String,Long>> occurences2 = p.apply(Create.of(WORDS_2))
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Count.<String>perElement());
+
+ PCollection<KV<String,Long>> occurences3 = p.apply(Create.of(WORDS_3))
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Count.<String>perElement());
+
+ /* CoGroup the two collections */
+ PCollection<KV<String, CoGbkResult>> mergedOccurences = KeyedPCollectionTuple
+ .of(tag1, occurences1)
+ .and(tag2, occurences2)
+ .and(tag3, occurences3)
+ .apply(CoGroupByKey.<String>create());
+
+ /* Format output */
+ mergedOccurences.apply(ParDo.of(new FormatCountsFn()))
+ .apply(TextIO.Write.named("test").to(resultPath));
+
+ p.run();
+ }
+
+
+ static class ExtractWordsFn extends DoFn<String, String> {
+
+ @Override
+ public void startBundle(Context c) {
+ }
+
+ @Override
+ public void processElement(ProcessContext c) {
+ // Split the line into words.
+ String[] words = c.element().split("[^a-zA-Z']+");
+
+ // Output each word encountered into the output PCollection.
+ for (String word : words) {
+ if (!word.isEmpty()) {
+ c.output(word);
+ }
+ }
+ }
+ }
+
+ static class FormatCountsFn extends DoFn<KV<String, CoGbkResult>, String> {
+ @Override
+ public void processElement(ProcessContext c) {
+ CoGbkResult value = c.element().getValue();
+ String key = c.element().getKey();
+ String countTag1 = tag1.getId() + ": ";
+ String countTag2 = tag2.getId() + ": ";
+ String countTag3 = tag3.getId() + ": ";
+ for (Long count : value.getAll(tag1)) {
+ countTag1 += count + " ";
+ }
+ for (Long count : value.getAll(tag2)) {
+ countTag2 += count + " ";
+ }
+ for (Long count : value.getAll(tag3)) {
+ countTag3 += count;
+ }
+ c.output(key + " -> " + countTag1 + countTag2 + countTag3);
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
new file mode 100644
index 0000000..497a5bb
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
@@ -0,0 +1,156 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.io.Sink;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.Write;
+import com.google.common.base.Joiner;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import java.io.File;
+import java.io.PrintWriter;
+import java.net.URI;
+
+import static org.junit.Assert.*;
+
+/**
+ * Tests the translation of custom Write.Bound sinks.
+ */
+public class WriteSinkITCase extends JavaProgramTestBase {
+
+ protected String resultPath;
+
+ public WriteSinkITCase(){
+ }
+
+ static final String[] EXPECTED_RESULT = new String[] {
+ "Joe red 3", "Mary blue 4", "Max yellow 23"};
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+ runProgram(resultPath);
+ }
+
+ private static void runProgram(String resultPath) {
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ p.apply(Create.of(EXPECTED_RESULT)).setCoder(StringUtf8Coder.of())
+ .apply("CustomSink", Write.to(new MyCustomSink(resultPath)));
+
+ p.run();
+ }
+
+ /**
+ * Simple custom sink which writes to a file.
+ */
+ private static class MyCustomSink extends Sink<String> {
+
+ private final String resultPath;
+
+ public MyCustomSink(String resultPath) {
+ this.resultPath = resultPath;
+ }
+
+ @Override
+ public void validate(PipelineOptions options) {
+ assertNotNull(options);
+ }
+
+ @Override
+ public WriteOperation<String, ?> createWriteOperation(PipelineOptions options) {
+ return new MyWriteOperation();
+ }
+
+ private class MyWriteOperation extends WriteOperation<String, String> {
+
+ @Override
+ public Coder<String> getWriterResultCoder() {
+ return StringUtf8Coder.of();
+ }
+
+ @Override
+ public void initialize(PipelineOptions options) throws Exception {
+
+ }
+
+ @Override
+ public void finalize(Iterable<String> writerResults, PipelineOptions options) throws Exception {
+
+ }
+
+ @Override
+ public Writer<String, String> createWriter(PipelineOptions options) throws Exception {
+ return new MyWriter();
+ }
+
+ @Override
+ public Sink<String> getSink() {
+ return MyCustomSink.this;
+ }
+
+ /**
+ * Simple Writer which writes to a file.
+ */
+ private class MyWriter extends Writer<String, String> {
+
+ private PrintWriter internalWriter;
+
+ @Override
+ public void open(String uId) throws Exception {
+ Path path = new Path(resultPath + "/" + uId);
+ FileSystem.get(new URI("file:///")).create(path, false);
+ internalWriter = new PrintWriter(new File(path.toUri()));
+ }
+
+ @Override
+ public void write(String value) throws Exception {
+ internalWriter.println(value);
+ }
+
+ @Override
+ public String close() throws Exception {
+ internalWriter.close();
+ return resultPath;
+ }
+
+ @Override
+ public WriteOperation<String, String> getWriteOperation() {
+ return MyWriteOperation.this;
+ }
+ }
+ }
+ }
+
+}
+
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java
new file mode 100644
index 0000000..27ddc83
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java
@@ -0,0 +1,506 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.streaming;
+
+import org.apache.beam.runners.flink.FlinkTestPipeline;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkGroupAlsoByWindowWrapper;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.Sum;
+import com.google.cloud.dataflow.sdk.transforms.windowing.*;
+import com.google.cloud.dataflow.sdk.util.UserCodeException;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.common.base.Throwables;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.TestHarnessUtil;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+public class GroupAlsoByWindowTest {
+
+ private final Combine.CombineFn combiner = new Sum.SumIntegerFn();
+
+ private final WindowingStrategy slidingWindowWithAfterWatermarkTriggerStrategy =
+ WindowingStrategy.of(SlidingWindows.of(Duration.standardSeconds(10)).every(Duration.standardSeconds(5)))
+ .withTrigger(AfterWatermark.pastEndOfWindow()).withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES);
+
+ private final WindowingStrategy sessionWindowingStrategy =
+ WindowingStrategy.of(Sessions.withGapDuration(Duration.standardSeconds(2)))
+ .withTrigger(Repeatedly.forever(AfterWatermark.pastEndOfWindow()))
+ .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES)
+ .withAllowedLateness(Duration.standardSeconds(100));
+
+ private final WindowingStrategy fixedWindowingStrategy =
+ WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(10)));
+
+ private final WindowingStrategy fixedWindowWithCountTriggerStrategy =
+ fixedWindowingStrategy.withTrigger(AfterPane.elementCountAtLeast(5));
+
+ private final WindowingStrategy fixedWindowWithAfterWatermarkTriggerStrategy =
+ fixedWindowingStrategy.withTrigger(AfterWatermark.pastEndOfWindow());
+
+ private final WindowingStrategy fixedWindowWithCompoundTriggerStrategy =
+ fixedWindowingStrategy.withTrigger(
+ AfterWatermark.pastEndOfWindow().withEarlyFirings(AfterPane.elementCountAtLeast(5))
+ .withLateFirings(AfterPane.elementCountAtLeast(5)).buildTrigger());
+
+ /**
+ * The default accumulation mode is
+ * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#DISCARDING_FIRED_PANES}.
+ * This strategy changes it to
+ * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#ACCUMULATING_FIRED_PANES}
+ */
+ private final WindowingStrategy fixedWindowWithCompoundTriggerStrategyAcc =
+ fixedWindowWithCompoundTriggerStrategy
+ .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES);
+
+ @Test
+ public void testWithLateness() throws Exception {
+ WindowingStrategy strategy = WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(2)))
+ .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES)
+ .withAllowedLateness(Duration.millis(1000));
+ long initialTime = 0L;
+ Pipeline pipeline = FlinkTestPipeline.createForStreaming();
+
+ KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
+
+ FlinkGroupAlsoByWindowWrapper gbwOperaror =
+ FlinkGroupAlsoByWindowWrapper.createForTesting(
+ pipeline.getOptions(),
+ pipeline.getCoderRegistry(),
+ strategy,
+ inputCoder,
+ combiner.<String>asKeyedFn());
+
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ new OneInputStreamOperatorTestHarness<>(gbwOperaror);
+ testHarness.open();
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processWatermark(new Watermark(initialTime + 2000));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processWatermark(new Watermark(initialTime + 4000));
+
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 4),
+ new Instant(initialTime + 1),
+ new IntervalWindow(new Instant(0), new Instant(2000)),
+ PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
+ , initialTime + 1));
+ expectedOutput.add(new Watermark(initialTime + 2000));
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 5),
+ new Instant(initialTime + 1999),
+ new IntervalWindow(new Instant(0), new Instant(2000)),
+ PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 1, 1))
+ , initialTime + 1999));
+
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 6),
+ new Instant(initialTime + 1999),
+ new IntervalWindow(new Instant(0), new Instant(2000)),
+ PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 2, 2))
+ , initialTime + 1999));
+ expectedOutput.add(new Watermark(initialTime + 4000));
+
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+ testHarness.close();
+ }
+
+ @Test
+ public void testSessionWindows() throws Exception {
+ WindowingStrategy strategy = sessionWindowingStrategy;
+
+ long initialTime = 0L;
+ Pipeline pipeline = FlinkTestPipeline.createForStreaming();
+
+ KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
+
+ FlinkGroupAlsoByWindowWrapper gbwOperaror =
+ FlinkGroupAlsoByWindowWrapper.createForTesting(
+ pipeline.getOptions(),
+ pipeline.getCoderRegistry(),
+ strategy,
+ inputCoder,
+ combiner.<String>asKeyedFn());
+
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ new OneInputStreamOperatorTestHarness<>(gbwOperaror);
+ testHarness.open();
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 3500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 3700), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 2700), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processWatermark(new Watermark(initialTime + 6000));
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 6700), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 6800), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 8900), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 7600), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 5600), null, PaneInfo.NO_FIRING), initialTime + 20));
+
+ testHarness.processWatermark(new Watermark(initialTime + 12000));
+
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 6),
+ new Instant(initialTime + 1),
+ new IntervalWindow(new Instant(1), new Instant(5700)),
+ PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
+ , initialTime + 1));
+ expectedOutput.add(new Watermark(initialTime + 6000));
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 11),
+ new Instant(initialTime + 6700),
+ new IntervalWindow(new Instant(1), new Instant(10900)),
+ PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
+ , initialTime + 6700));
+ expectedOutput.add(new Watermark(initialTime + 12000));
+
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+ testHarness.close();
+ }
+
+ @Test
+ public void testSlidingWindows() throws Exception {
+ WindowingStrategy strategy = slidingWindowWithAfterWatermarkTriggerStrategy;
+ long initialTime = 0L;
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ createTestingOperatorAndState(strategy, initialTime);
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+ testHarness.processWatermark(new Watermark(initialTime + 25000));
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 6),
+ new Instant(initialTime + 5000),
+ new IntervalWindow(new Instant(0), new Instant(10000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime + 5000));
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 6),
+ new Instant(initialTime + 1),
+ new IntervalWindow(new Instant(-5000), new Instant(5000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime + 1));
+ expectedOutput.add(new Watermark(initialTime + 10000));
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 11),
+ new Instant(initialTime + 15000),
+ new IntervalWindow(new Instant(10000), new Instant(20000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime + 15000));
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 3),
+ new Instant(initialTime + 10000),
+ new IntervalWindow(new Instant(5000), new Instant(15000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime + 10000));
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key2", 1),
+ new Instant(initialTime + 19500),
+ new IntervalWindow(new Instant(10000), new Instant(20000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime + 19500));
+ expectedOutput.add(new Watermark(initialTime + 20000));
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key2", 1),
+ new Instant(initialTime + 20000),
+ /**
+ * this is 20000 and not 19500 because of a convention in dataflow where
+ * timestamps of windowed values in a window cannot be smaller than the
+ * end of a previous window. Checkout the documentation of the
+ * {@link WindowFn#getOutputTime(Instant, BoundedWindow)}
+ */
+ new IntervalWindow(new Instant(15000), new Instant(25000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime + 20000));
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 8),
+ new Instant(initialTime + 20000),
+ new IntervalWindow(new Instant(15000), new Instant(25000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime + 20000));
+ expectedOutput.add(new Watermark(initialTime + 25000));
+
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+ testHarness.close();
+ }
+
+ @Test
+ public void testAfterWatermarkProgram() throws Exception {
+ WindowingStrategy strategy = fixedWindowWithAfterWatermarkTriggerStrategy;
+ long initialTime = 0L;
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ createTestingOperatorAndState(strategy, initialTime);
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6),
+ new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 1));
+ expectedOutput.add(new Watermark(initialTime + 10000));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11),
+ new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 10000));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500));
+ expectedOutput.add(new Watermark(initialTime + 20000));
+
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+ testHarness.close();
+ }
+
+ @Test
+ public void testAfterCountProgram() throws Exception {
+ WindowingStrategy strategy = fixedWindowWithCountTriggerStrategy;
+
+ long initialTime = 0L;
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ createTestingOperatorAndState(strategy, initialTime);
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime + 1));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime + 10000));
+ expectedOutput.add(new Watermark(initialTime + 10000));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME, 0, 0)), initialTime + 19500));
+ expectedOutput.add(new Watermark(initialTime + 20000));
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+ testHarness.close();
+ }
+
+ @Test
+ public void testCompoundProgram() throws Exception {
+ WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategy;
+
+ long initialTime = 0L;
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ createTestingOperatorAndState(strategy, initialTime);
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+ /**
+ * PaneInfo are:
+ * isFirst (pane in window),
+ * isLast, Timing (of triggering),
+ * index (of pane in the window),
+ * onTimeIndex (if it the 1st,2nd, ... pane that was fired on time)
+ * */
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 1));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 10000));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime + 19500));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1),
+ new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime + 1200));
+
+ expectedOutput.add(new Watermark(initialTime + 10000));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime + 19500));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500));
+
+ expectedOutput.add(new Watermark(initialTime + 20000));
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+ testHarness.close();
+ }
+
+ @Test
+ public void testCompoundAccumulatingPanesProgram() throws Exception {
+ WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategyAcc;
+ long initialTime = 0L;
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ createTestingOperatorAndState(strategy, initialTime);
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 1));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 10000));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 10),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime + 19500));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6),
+ new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime + 1200));
+
+ expectedOutput.add(new Watermark(initialTime + 10000));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime + 19500));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500));
+
+ expectedOutput.add(new Watermark(initialTime + 20000));
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+ testHarness.close();
+ }
+
+ private OneInputStreamOperatorTestHarness createTestingOperatorAndState(WindowingStrategy strategy, long initialTime) throws Exception {
+ Pipeline pipeline = FlinkTestPipeline.createForStreaming();
+
+ KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
+
+ FlinkGroupAlsoByWindowWrapper gbwOperaror =
+ FlinkGroupAlsoByWindowWrapper.createForTesting(
+ pipeline.getOptions(),
+ pipeline.getCoderRegistry(),
+ strategy,
+ inputCoder,
+ combiner.<String>asKeyedFn());
+
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ new OneInputStreamOperatorTestHarness<>(gbwOperaror);
+ testHarness.open();
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 10000), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 12100), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 14200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 15300), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 16500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+
+ testHarness.processWatermark(new Watermark(initialTime + 10000));
+ testHarness.processWatermark(new Watermark(initialTime + 20000));
+
+ return testHarness;
+ }
+
+ private static class ResultSortComparator implements Comparator<Object> {
+ @Override
+ public int compare(Object o1, Object o2) {
+ if (o1 instanceof Watermark && o2 instanceof Watermark) {
+ Watermark w1 = (Watermark) o1;
+ Watermark w2 = (Watermark) o2;
+ return (int) (w1.getTimestamp() - w2.getTimestamp());
+ } else {
+ StreamRecord<WindowedValue<KV<String, Integer>>> sr0 = (StreamRecord<WindowedValue<KV<String, Integer>>>) o1;
+ StreamRecord<WindowedValue<KV<String, Integer>>> sr1 = (StreamRecord<WindowedValue<KV<String, Integer>>>) o2;
+
+ int comparison = (int) (sr0.getValue().getTimestamp().getMillis() - sr1.getValue().getTimestamp().getMillis());
+ if (comparison != 0) {
+ return comparison;
+ }
+
+ comparison = sr0.getValue().getValue().getKey().compareTo(sr1.getValue().getValue().getKey());
+ if(comparison == 0) {
+ comparison = Integer.compare(
+ sr0.getValue().getValue().getValue(),
+ sr1.getValue().getValue().getValue());
+ }
+ if(comparison == 0) {
+ Collection windowsA = sr0.getValue().getWindows();
+ Collection windowsB = sr1.getValue().getWindows();
+
+ if(windowsA.size() != 1 || windowsB.size() != 1) {
+ throw new IllegalStateException("A value cannot belong to more than one windows after grouping.");
+ }
+
+ BoundedWindow windowA = (BoundedWindow) windowsA.iterator().next();
+ BoundedWindow windowB = (BoundedWindow) windowsB.iterator().next();
+ comparison = Long.compare(windowA.maxTimestamp().getMillis(), windowB.maxTimestamp().getMillis());
+ }
+ return comparison;
+ }
+ }
+ }
+
+ private <T> WindowedValue<T> makeWindowedValue(WindowingStrategy strategy,
+ T output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+ final Instant inputTimestamp = timestamp;
+ final WindowFn windowFn = strategy.getWindowFn();
+
+ if (timestamp == null) {
+ timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+ }
+
+ if (windows == null) {
+ try {
+ windows = windowFn.assignWindows(windowFn.new AssignContext() {
+ @Override
+ public Object element() {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input element when none was available");
+ }
+
+ @Override
+ public Instant timestamp() {
+ if (inputTimestamp == null) {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input timestamp when none was available");
+ }
+ return inputTimestamp;
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input windows when none were available");
+ }
+ });
+ } catch (Exception e) {
+ throw UserCodeException.wrap(e);
+ }
+ }
+
+ return WindowedValue.of(output, timestamp, windows, pane);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
new file mode 100644
index 0000000..80d78b9
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
@@ -0,0 +1,121 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.streaming;
+
+import org.apache.beam.runners.flink.FlinkTestPipeline;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
+import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Joiner;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.io.Serializable;
+import java.util.Arrays;
+
+public class GroupByNullKeyTest extends StreamingProgramTestBase implements Serializable {
+
+
+ protected String resultPath;
+
+ static final String[] EXPECTED_RESULT = new String[] {
+ "k: null v: user1 user1 user1 user2 user2 user2 user2 user3"
+ };
+
+ public GroupByNullKeyTest(){
+ }
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
+
+ public static class ExtractUserAndTimestamp extends DoFn<KV<Integer, String>, String> {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ KV<Integer, String> record = c.element();
+ long now = System.currentTimeMillis();
+ int timestamp = record.getKey();
+ String userName = record.getValue();
+ if (userName != null) {
+ // Sets the implicit timestamp field to be used in windowing.
+ c.outputWithTimestamp(userName, new Instant(timestamp + now));
+ }
+ }
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+
+ Pipeline p = FlinkTestPipeline.createForStreaming();
+
+ PCollection<String> output =
+ p.apply(Create.of(Arrays.asList(
+ KV.<Integer, String>of(0, "user1"),
+ KV.<Integer, String>of(1, "user1"),
+ KV.<Integer, String>of(2, "user1"),
+ KV.<Integer, String>of(10, "user2"),
+ KV.<Integer, String>of(1, "user2"),
+ KV.<Integer, String>of(15000, "user2"),
+ KV.<Integer, String>of(12000, "user2"),
+ KV.<Integer, String>of(25000, "user3"))))
+ .apply(ParDo.of(new ExtractUserAndTimestamp()))
+ .apply(Window.<String>into(FixedWindows.of(Duration.standardHours(1)))
+ .triggering(AfterWatermark.pastEndOfWindow())
+ .withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes())
+
+ .apply(ParDo.of(new DoFn<String, KV<Void, String>>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ String elem = c.element();
+ c.output(KV.<Void, String>of((Void) null, elem));
+ }
+ }))
+ .apply(GroupByKey.<Void, String>create())
+ .apply(ParDo.of(new DoFn<KV<Void, Iterable<String>>, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ KV<Void, Iterable<String>> elem = c.element();
+ StringBuilder str = new StringBuilder();
+ str.append("k: " + elem.getKey() + " v:");
+ for (String v : elem.getValue()) {
+ str.append(" " + v);
+ }
+ c.output(str.toString());
+ }
+ }));
+ output.apply(TextIO.Write.to(resultPath));
+ p.run();
+ }
+}
[02/50] [abbrv] incubator-beam git commit: [sink] generate unique id
for writer initialization
Posted by da...@apache.org.
[sink] generate unique id for writer initialization
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/336d394e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/336d394e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/336d394e
Branch: refs/heads/master
Commit: 336d394e9cb4d68143def6027574b56f617080d2
Parents: 8e859e2
Author: Max <ma...@posteo.de>
Authored: Tue Jan 19 10:38:49 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
.../flink/dataflow/translation/wrappers/SinkOutputFormat.java | 7 ++++++-
1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/336d394e/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java
index d87b240..b10c86f 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java
@@ -23,13 +23,16 @@ import com.google.cloud.dataflow.sdk.io.Sink;
import com.google.cloud.dataflow.sdk.options.PipelineOptions;
import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions;
import com.google.cloud.dataflow.sdk.transforms.Write;
+import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.io.OutputFormat;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.AbstractID;
import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.lang.reflect.Field;
+import java.util.UUID;
/**
* Wrapper class to use generic Write.Bound transforms as sinks.
@@ -44,6 +47,8 @@ public class SinkOutputFormat<T> implements OutputFormat<T> {
private Sink.WriteOperation<T, ?> writeOperation;
private Sink.Writer<T, ?> writer;
+ private AbstractID uid = new AbstractID();
+
public SinkOutputFormat(Write.Bound<T> transform, PipelineOptions pipelineOptions) {
this.sink = extractSink(transform);
this.pipelineOptions = Preconditions.checkNotNull(pipelineOptions);
@@ -80,7 +85,7 @@ public class SinkOutputFormat<T> implements OutputFormat<T> {
throw new IOException("Couldn't create writer.", e);
}
try {
- writer.open(String.valueOf(taskNumber));
+ writer.open(uid + "-" + String.valueOf(taskNumber));
} catch (Exception e) {
throw new IOException("Couldn't open writer.", e);
}
[36/50] [abbrv] incubator-beam git commit: [flink] adjust root
pom.xml to Beam
Posted by da...@apache.org.
[flink] adjust root pom.xml to Beam
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/39f08fa7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/39f08fa7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/39f08fa7
Branch: refs/heads/master
Commit: 39f08fa7406a4f30ceed558aa58577f52df94975
Parents: 730f092
Author: Maximilian Michels <mx...@apache.org>
Authored: Fri Mar 4 15:08:26 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
runners/flink/pom.xml | 475 +++++++++++++++++++++++----------------------
1 file changed, 243 insertions(+), 232 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/39f08fa7/runners/flink/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml
index 6b12f2d..2110c2c 100644
--- a/runners/flink/pom.xml
+++ b/runners/flink/pom.xml
@@ -1,253 +1,264 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
+ 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
- Copyright 2015 Data Artisans GmbH
+ http://www.apache.org/licenses/LICENSE-2.0
- 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
+ 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.
+-->
+<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+ xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
- http://www.apache.org/licenses/LICENSE-2.0
+ <modelVersion>4.0.0</modelVersion>
- 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.
+ <parent>
+ <groupId>org.apache.beam</groupId>
+ <artifactId>runners</artifactId>
+ <version>1.5.0-SNAPSHOT</version>
+ </parent>
--->
-<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
- xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-
- <modelVersion>4.0.0</modelVersion>
-
- <groupId>com.dataartisans</groupId>
- <artifactId>flink-dataflow</artifactId>
- <version>0.3-SNAPSHOT</version>
-
- <name>Flink Beam Runner</name>
- <packaging>jar</packaging>
-
- <inceptionYear>2015</inceptionYear>
-
- <licenses>
- <license>
- <name>The Apache Software License, Version 2.0</name>
- <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
- <distribution>repo</distribution>
- </license>
- </licenses>
-
- <properties>
- <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
- <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
- <flink.version>1.0-SNAPSHOT</flink.version>
- <beam.version>1.5.0-SNAPSHOT</beam.version>
- <!-- Default parameters for mvn exec:exec -->
- <clazz>com.dataartisans.flink.dataflow.examples.WordCount</clazz>
- <input>kinglear.txt</input>
- <output>wordcounts.txt</output>
- <parallelism>1</parallelism>
- </properties>
-
- <repositories>
- <repository>
- <id>apache.snapshots</id>
- <name>Apache Development Snapshot Repository</name>
- <url>https://repository.apache.org/content/repositories/snapshots/</url>
- <releases>
- <enabled>false</enabled>
- </releases>
- <snapshots>
- <enabled>true</enabled>
- </snapshots>
- </repository>
- </repositories>
-
- <dependencies>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-core</artifactId>
- <version>${flink.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-streaming-java_2.10</artifactId>
- <version>${flink.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-streaming-java_2.10</artifactId>
- <version>${flink.version}</version>
- <scope>test</scope>
- <type>test-jar</type>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-java</artifactId>
- <version>${flink.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-avro_2.10</artifactId>
- <version>${flink.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-clients_2.10</artifactId>
- <version>${flink.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-test-utils_2.10</artifactId>
- <version>${flink.version}</version>
- <scope>test</scope>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-connector-kafka-0.8_2.10</artifactId>
- <version>${flink.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-avro</artifactId>
- <version>${flink.version}</version>
- </dependency>
- <dependency>
- <groupId>com.google.cloud.dataflow</groupId>
- <artifactId>google-cloud-dataflow-java-sdk-all</artifactId>
- <version>${beam.version}</version>
- <exclusions>
- <exclusion>
- <groupId>org.slf4j</groupId>
- <artifactId>slf4j-jdk14</artifactId>
- </exclusion>
- </exclusions>
- </dependency>
- <dependency>
- <groupId>org.mockito</groupId>
- <artifactId>mockito-all</artifactId>
- <version>1.9.5</version>
- <scope>test</scope>
- </dependency>
- </dependencies>
-
- <build>
- <plugins>
-
- <!-- JAR Packaging -->
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-jar-plugin</artifactId>
- <version>2.6</version><!--$NO-MVN-MAN-VER$-->
- <configuration>
- <archive>
- <manifest>
- <addDefaultImplementationEntries>true</addDefaultImplementationEntries>
- <addDefaultSpecificationEntries>true</addDefaultSpecificationEntries>
- </manifest>
- </archive>
- </configuration>
- </plugin>
+ <artifactId>flink-runner</artifactId>
+ <version>0.3-SNAPSHOT</version>
- <!-- Java compiler -->
- <plugin>
+ <name>Flink Beam Runner</name>
+ <packaging>jar</packaging>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-compiler-plugin</artifactId>
- <version>3.1</version><!--$NO-MVN-MAN-VER$-->
- <configuration>
- <source>1.7</source>
- <target>1.7</target>
- </configuration>
- </plugin>
-
- <!-- Integration Tests -->
- <plugin>
- <artifactId>maven-failsafe-plugin</artifactId>
- <version>2.17</version><!--$NO-MVN-MAN-VER$-->
- <executions>
- <execution>
- <goals>
- <goal>integration-test</goal>
- <goal>verify</goal>
- </goals>
- </execution>
- </executions>
+ <inceptionYear>2015</inceptionYear>
+
+ <licenses>
+ <license>
+ <name>The Apache Software License, Version 2.0</name>
+ <url>http://www.apache.org/licenses/LICENSE-2.0.txt</url>
+ <distribution>repo</distribution>
+ </license>
+ </licenses>
+
+ <properties>
+ <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
+ <project.reporting.outputEncoding>UTF-8</project.reporting.outputEncoding>
+ <flink.version>1.0-SNAPSHOT</flink.version>
+ <beam.version>1.5.0-SNAPSHOT</beam.version>
+ <!-- Default parameters for mvn exec:exec -->
+ <clazz>org.apache.beam.runners.flink.examples.WordCount</clazz>
+ <input>kinglear.txt</input>
+ <output>wordcounts.txt</output>
+ <parallelism>1</parallelism>
+ </properties>
+
+ <repositories>
+ <repository>
+ <id>apache.snapshots</id>
+ <name>Apache Development Snapshot Repository</name>
+ <url>https://repository.apache.org/content/repositories/snapshots/</url>
+ <releases>
+ <enabled>false</enabled>
+ </releases>
+ <snapshots>
+ <enabled>true</enabled>
+ </snapshots>
+ </repository>
+ </repositories>
+
+ <dependencies>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-core</artifactId>
+ <version>${flink.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-streaming-java_2.10</artifactId>
+ <version>${flink.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-streaming-java_2.10</artifactId>
+ <version>${flink.version}</version>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-java</artifactId>
+ <version>${flink.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-avro_2.10</artifactId>
+ <version>${flink.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-clients_2.10</artifactId>
+ <version>${flink.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-test-utils_2.10</artifactId>
+ <version>${flink.version}</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-kafka-0.8_2.10</artifactId>
+ <version>${flink.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-avro</artifactId>
+ <version>${flink.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>com.google.cloud.dataflow</groupId>
+ <artifactId>google-cloud-dataflow-java-sdk-all</artifactId>
+ <version>${beam.version}</version>
+ <exclusions>
+ <exclusion>
+ <groupId>org.slf4j</groupId>
+ <artifactId>slf4j-jdk14</artifactId>
+ </exclusion>
+ </exclusions>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <version>1.9.5</version>
+ <scope>test</scope>
+ </dependency>
+ </dependencies>
+
+ <build>
+ <plugins>
+
+ <!-- JAR Packaging -->
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-jar-plugin</artifactId>
+ <version>2.6</version><!--$NO-MVN-MAN-VER$-->
+ <configuration>
+ <archive>
+ <manifest>
+ <addDefaultImplementationEntries>true</addDefaultImplementationEntries>
+ <addDefaultSpecificationEntries>true</addDefaultSpecificationEntries>
+ </manifest>
+ </archive>
+ </configuration>
+ </plugin>
+
+ <!-- Java compiler -->
+ <plugin>
+
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-compiler-plugin</artifactId>
+ <version>3.1</version><!--$NO-MVN-MAN-VER$-->
+ <configuration>
+ <source>1.7</source>
+ <target>1.7</target>
+ </configuration>
+ </plugin>
+
+ <!-- Integration Tests -->
+ <plugin>
+ <artifactId>maven-failsafe-plugin</artifactId>
+ <version>2.17</version><!--$NO-MVN-MAN-VER$-->
+ <executions>
+ <execution>
+ <goals>
+ <goal>integration-test</goal>
+ <goal>verify</goal>
+ </goals>
+ </execution>
+ </executions>
<configuration>
<argLine>-Dlog4j.configuration=log4j-test.properties -XX:-UseGCOverheadLimit</argLine>
</configuration>
- </plugin>
+ </plugin>
- <!-- Unit Tests -->
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-surefire-plugin</artifactId>
- <version>2.17</version><!--$NO-MVN-MAN-VER$-->
+ <!-- Unit Tests -->
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-surefire-plugin</artifactId>
+ <version>2.17</version><!--$NO-MVN-MAN-VER$-->
<configuration>
<argLine>-Dlog4j.configuration=log4j-test.properties -XX:-UseGCOverheadLimit</argLine>
</configuration>
- </plugin>
+ </plugin>
- <!-- Eclipse Integration -->
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-eclipse-plugin</artifactId>
- <version>2.8</version>
- <configuration>
- <classpathContainers>
- <classpathContainer>org.eclipse.jdt.launching.JRE_CONTAINER</classpathContainer>
- </classpathContainers>
- <downloadSources>true</downloadSources>
- <downloadJavadocs>true</downloadJavadocs>
- </configuration>
- </plugin>
-
- <!-- Maven minimum version check -->
- <plugin>
- <groupId>org.apache.maven.plugins</groupId>
- <artifactId>maven-enforcer-plugin</artifactId>
- <version>1.3.1</version><!--$NO-MVN-MAN-VER$-->
- <executions>
- <execution>
- <id>enforce-maven</id>
- <goals>
- <goal>enforce</goal>
- </goals>
- <configuration>
- <rules>
- <requireJavaVersion>
- <version>[1.7,)</version>
- </requireJavaVersion>
- <requireMavenVersion>
- <!-- enforce at least mvn version 3.0.3 -->
- <version>[3.0.3,)</version>
- </requireMavenVersion>
- </rules>
- </configuration>
- </execution>
- </executions>
- </plugin>
-
- <plugin>
- <groupId>org.codehaus.mojo</groupId>
- <artifactId>exec-maven-plugin</artifactId>
- <version>1.2.1</version>
- <configuration>
- <executable>java</executable>
- <arguments>
- <argument>-classpath</argument>
- <classpath/>
- <argument>${clazz}</argument>
- <argument>--input=${input}</argument>
- <argument>--output=${output}</argument>
- <argument>--parallelism=${parallelism}</argument>
- </arguments>
- </configuration>
- </plugin>
+ <!-- Eclipse Integration -->
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-eclipse-plugin</artifactId>
+ <version>2.8</version>
+ <configuration>
+ <classpathContainers>
+ <classpathContainer>org.eclipse.jdt.launching.JRE_CONTAINER</classpathContainer>
+ </classpathContainers>
+ <downloadSources>true</downloadSources>
+ <downloadJavadocs>true</downloadJavadocs>
+ </configuration>
+ </plugin>
+
+ <!-- Maven minimum version check -->
+ <plugin>
+ <groupId>org.apache.maven.plugins</groupId>
+ <artifactId>maven-enforcer-plugin</artifactId>
+ <version>1.3.1</version><!--$NO-MVN-MAN-VER$-->
+ <executions>
+ <execution>
+ <id>enforce-maven</id>
+ <goals>
+ <goal>enforce</goal>
+ </goals>
+ <configuration>
+ <rules>
+ <requireJavaVersion>
+ <version>[1.7,)</version>
+ </requireJavaVersion>
+ <requireMavenVersion>
+ <!-- enforce at least mvn version 3.0.3 -->
+ <version>[3.0.3,)</version>
+ </requireMavenVersion>
+ </rules>
+ </configuration>
+ </execution>
+ </executions>
+ </plugin>
+
+ <plugin>
+ <groupId>org.codehaus.mojo</groupId>
+ <artifactId>exec-maven-plugin</artifactId>
+ <version>1.2.1</version>
+ <executions>
+ <execution>
+ <phase>none</phase>
+ </execution>
+ </executions>
+ <configuration>
+ <executable>java</executable>
+ <arguments>
+ <argument>-classpath</argument>
+ <classpath/>
+ <argument>${clazz}</argument>
+ <argument>--input=${input}</argument>
+ <argument>--output=${output}</argument>
+ <argument>--parallelism=${parallelism}</argument>
+ </arguments>
+ </configuration>
+ </plugin>
- </plugins>
+ </plugins>
- </build>
+ </build>
</project>
[14/50] [abbrv] incubator-beam git commit: [runner] add Create
transform
Posted by da...@apache.org.
[runner] add Create transform
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/3227fccd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/3227fccd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/3227fccd
Branch: refs/heads/master
Commit: 3227fccdb0c4e09c6367b25d436c7e89cd881a2d
Parents: 0be42cb
Author: Max <ma...@posteo.de>
Authored: Mon Feb 22 18:25:23 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
.../FlinkStreamingTransformTranslators.java | 60 +++++++++++++++++--
.../io/FlinkStreamingCreateFunction.java | 61 ++++++++++++++++++++
2 files changed, 117 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3227fccd/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
index 17583cd..46d3e36 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
@@ -13,11 +13,13 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package com.dataartisans.flink.dataflow.translation;
import com.dataartisans.flink.dataflow.translation.functions.UnionCoder;
import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation;
import com.dataartisans.flink.dataflow.translation.wrappers.streaming.*;
+import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.FlinkStreamingCreateFunction;
import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedFlinkSource;
import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedSourceWrapper;
import com.google.api.client.util.Maps;
@@ -37,6 +39,7 @@ import com.google.cloud.dataflow.sdk.values.TupleTag;
import com.google.common.collect.Lists;
import org.apache.flink.api.common.functions.FilterFunction;
import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.streaming.api.datastream.*;
import org.apache.flink.util.Collector;
@@ -44,6 +47,8 @@ import org.joda.time.Instant;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
import java.util.*;
/**
@@ -64,6 +69,8 @@ public class FlinkStreamingTransformTranslators {
// here you can find all the available translators.
static {
+
+ TRANSLATORS.put(Create.Values.class, new CreateStreamingTranslator());
TRANSLATORS.put(Read.Unbounded.class, new UnboundedReadSourceTranslator());
TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator());
TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteBoundStreamingTranslator());
@@ -83,6 +90,47 @@ public class FlinkStreamingTransformTranslators {
// Transformation Implementations
// --------------------------------------------------------------------------------------------
+ private static class CreateStreamingTranslator<OUT> implements
+ FlinkStreamingPipelineTranslator.StreamTransformTranslator<Create.Values<OUT>> {
+
+ @Override
+ public void translateNode(Create.Values<OUT> transform, FlinkStreamingTranslationContext context) {
+ PCollection<OUT> output = context.getOutput(transform);
+ Iterable<OUT> elements = transform.getElements();
+
+ // we need to serialize the elements to byte arrays, since they might contain
+ // elements that are not serializable by Java serialization. We deserialize them
+ // in the FlatMap function using the Coder.
+
+ List<byte[]> serializedElements = Lists.newArrayList();
+ Coder<OUT> elementCoder = context.getOutput(transform).getCoder();
+ for (OUT element: elements) {
+ ByteArrayOutputStream bao = new ByteArrayOutputStream();
+ try {
+ elementCoder.encode(element, bao, Coder.Context.OUTER);
+ serializedElements.add(bao.toByteArray());
+ } catch (IOException e) {
+ throw new RuntimeException("Could not serialize Create elements using Coder: " + e);
+ }
+ }
+
+
+ DataStream<Integer> initDataSet = context.getExecutionEnvironment().fromElements(1);
+
+ FlinkStreamingCreateFunction<Integer, OUT> createFunction =
+ new FlinkStreamingCreateFunction<>(serializedElements, elementCoder);
+
+ WindowedValue.ValueOnlyWindowedValueCoder<OUT> windowCoder = WindowedValue.getValueOnlyCoder(elementCoder);
+ TypeInformation<WindowedValue<OUT>> outputType = new CoderTypeInformation<>(windowCoder);
+
+ DataStream<WindowedValue<OUT>> outputDataStream = initDataSet.flatMap(createFunction)
+ .returns(outputType);
+
+ context.setOutputDataStream(context.getOutput(transform), outputDataStream);
+ }
+ }
+
+
private static class TextIOWriteBoundStreamingTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<TextIO.Write.Bound<T>> {
private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteBoundStreamingTranslator.class);
@@ -151,12 +199,16 @@ public class FlinkStreamingTransformTranslators {
(WindowingStrategy<OUT, ? extends BoundedWindow>)
context.getOutput(transform).getWindowingStrategy();
- WindowedValue.WindowedValueCoder<OUT> outputStreamCoder = WindowedValue.getFullCoder(output.getCoder(), windowingStrategy.getWindowFn().windowCoder());
- CoderTypeInformation<WindowedValue<OUT>> outputWindowedValueCoder = new CoderTypeInformation<>(outputStreamCoder);
+ WindowedValue.WindowedValueCoder<OUT> outputStreamCoder = WindowedValue.getFullCoder(output.getCoder(),
+ windowingStrategy.getWindowFn().windowCoder());
+ CoderTypeInformation<WindowedValue<OUT>> outputWindowedValueCoder =
+ new CoderTypeInformation<>(outputStreamCoder);
- FlinkParDoBoundWrapper<IN, OUT> doFnWrapper = new FlinkParDoBoundWrapper<>(context.getPipelineOptions(), windowingStrategy, transform.getFn());
+ FlinkParDoBoundWrapper<IN, OUT> doFnWrapper = new FlinkParDoBoundWrapper<>(
+ context.getPipelineOptions(), windowingStrategy, transform.getFn());
DataStream<WindowedValue<IN>> inputDataStream = context.getInputDataStream(context.getInput(transform));
- SingleOutputStreamOperator<WindowedValue<OUT>, ?> outDataStream = inputDataStream.flatMap(doFnWrapper).returns(outputWindowedValueCoder);
+ SingleOutputStreamOperator<WindowedValue<OUT>, ?> outDataStream = inputDataStream.flatMap(doFnWrapper)
+ .returns(outputWindowedValueCoder);
context.setOutputDataStream(context.getOutput(transform), outDataStream);
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/3227fccd/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
new file mode 100644
index 0000000..b8824f5
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
@@ -0,0 +1,61 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation.wrappers.streaming.io;
+
+import com.dataartisans.flink.dataflow.translation.types.VoidCoderTypeSerializer;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.util.Collector;
+
+import java.io.ByteArrayInputStream;
+import java.util.List;
+
+/**
+ * This flat map function bootstraps from collection elements and turns them into WindowedValues
+ * (as required by the Flink runner).
+ */
+public class FlinkStreamingCreateFunction<IN, OUT> implements FlatMapFunction<IN, WindowedValue<OUT>> {
+
+ private final List<byte[]> elements;
+ private final Coder<OUT> coder;
+
+ public FlinkStreamingCreateFunction(List<byte[]> elements, Coder<OUT> coder) {
+ this.elements = elements;
+ this.coder = coder;
+ }
+
+ @Override
+ public void flatMap(IN value, Collector<WindowedValue<OUT>> out) throws Exception {
+
+ @SuppressWarnings("unchecked")
+ // TODO Flink doesn't allow null values in records
+ OUT voidValue = (OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE;
+
+ for (byte[] element : elements) {
+ ByteArrayInputStream bai = new ByteArrayInputStream(element);
+ OUT outValue = coder.decode(bai, Coder.Context.OUTER);
+
+ if (outValue == null) {
+ out.collect(WindowedValue.of(voidValue, GlobalWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
+ } else {
+ out.collect(WindowedValue.of(outValue, GlobalWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
+ }
+ }
+ }
+}
[35/50] [abbrv] incubator-beam git commit: [maven] add project for
Runners
Posted by da...@apache.org.
[maven] add project for Runners
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/730f092c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/730f092c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/730f092c
Branch: refs/heads/master
Commit: 730f092c842c8575c961398559b6ff4173f40b5f
Parents: dc7786f
Author: Maximilian Michels <mx...@apache.org>
Authored: Tue Mar 1 17:57:02 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
pom.xml | 1 +
runners/pom.xml | 43 +++++++++++++++++++++++++++++++++++++++++++
2 files changed, 44 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/730f092c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ba130d2..de47ff5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -91,6 +91,7 @@
<packaging>pom</packaging>
<modules>
<module>sdk</module>
+ <module>runners</module>
<module>examples</module>
<module>maven-archetypes/starter</module>
<module>maven-archetypes/examples</module>
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/730f092c/runners/pom.xml
----------------------------------------------------------------------
diff --git a/runners/pom.xml b/runners/pom.xml
new file mode 100644
index 0000000..757e208
--- /dev/null
+++ b/runners/pom.xml
@@ -0,0 +1,43 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ 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.
+-->
+<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+ xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+ <modelVersion>4.0.0</modelVersion>
+
+ <parent>
+ <groupId>com.google.cloud.dataflow</groupId>
+ <artifactId>google-cloud-dataflow-java-sdk-parent</artifactId>
+ <version>1.5.0-SNAPSHOT</version>
+ </parent>
+
+ <groupId>org.apache.beam</groupId>
+ <artifactId>runners</artifactId>
+ <version>1.5.0-SNAPSHOT</version>
+
+ <packaging>pom</packaging>
+
+ <name>Beam Runners</name>
+
+ <modules>
+ <module>flink</module>
+ </modules>
+
+</project>
[08/50] [abbrv] incubator-beam git commit: [runner] add streaming
support with checkpointing
Posted by da...@apache.org.
[runner] add streaming support with checkpointing
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/edff0785
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/edff0785
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/edff0785
Branch: refs/heads/master
Commit: edff0785a82d2c6c01abeb3c64ca0d2958ccd0fd
Parents: 517c1bd
Author: Kostas Kloudas <kk...@gmail.com>
Authored: Wed Dec 9 17:30:53 2015 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
runners/flink/pom.xml | 28 +
.../dataflow/FlinkJobExecutionEnvironment.java | 238 ++++++++
.../flink/dataflow/FlinkPipelineRunner.java | 99 +--
.../flink/dataflow/examples/WordCount.java | 2 +-
.../examples/streaming/AutoComplete.java | 384 ++++++++++++
.../examples/streaming/JoinExamples.java | 157 +++++
.../KafkaWindowedWordCountExample.java | 138 +++++
.../examples/streaming/WindowedWordCount.java | 126 ++++
.../FlinkBatchPipelineTranslator.java | 152 +++++
.../FlinkBatchTransformTranslators.java | 593 ++++++++++++++++++
.../FlinkBatchTranslationContext.java | 129 ++++
.../translation/FlinkPipelineTranslator.java | 145 +----
.../FlinkStreamingPipelineTranslator.java | 138 +++++
.../FlinkStreamingTransformTranslators.java | 356 +++++++++++
.../FlinkStreamingTranslationContext.java | 86 +++
.../translation/FlinkTransformTranslators.java | 594 ------------------
.../translation/TranslationContext.java | 129 ----
.../translation/types/CoderComparator.java | 216 +++++++
.../translation/types/CoderComperator.java | 218 -------
.../translation/types/CoderTypeInformation.java | 6 +-
.../translation/types/CoderTypeSerializer.java | 2 -
.../translation/types/KvCoderComperator.java | 2 +-
.../types/VoidCoderTypeSerializer.java | 1 -
.../translation/wrappers/SourceInputFormat.java | 4 +-
.../streaming/FlinkAbstractParDoWrapper.java | 274 +++++++++
.../FlinkGroupAlsoByWindowWrapper.java | 601 +++++++++++++++++++
.../streaming/FlinkGroupByKeyWrapper.java | 56 ++
.../streaming/FlinkParDoBoundMultiWrapper.java | 72 +++
.../streaming/FlinkParDoBoundWrapper.java | 89 +++
.../streaming/io/UnboundedFlinkSource.java | 76 +++
.../streaming/io/UnboundedSocketSource.java | 228 +++++++
.../streaming/io/UnboundedSourceWrapper.java | 120 ++++
.../state/AbstractFlinkTimerInternals.java | 139 +++++
.../streaming/state/FlinkStateInternals.java | 533 ++++++++++++++++
.../streaming/state/StateCheckpointReader.java | 89 +++
.../streaming/state/StateCheckpointUtils.java | 152 +++++
.../streaming/state/StateCheckpointWriter.java | 127 ++++
.../wrappers/streaming/state/StateType.java | 67 +++
.../streaming/GroupAlsoByWindowTest.java | 507 ++++++++++++++++
.../streaming/StateSerializationTest.java | 257 ++++++++
.../flink/dataflow/util/JoinExamples.java | 4 +-
41 files changed, 6157 insertions(+), 1177 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml
index 6102d74..14693b8 100644
--- a/runners/flink/pom.xml
+++ b/runners/flink/pom.xml
@@ -71,6 +71,18 @@
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
+ <artifactId>flink-streaming-java</artifactId>
+ <version>${flink.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-streaming-java</artifactId>
+ <version>${flink.version}</version>
+ <scope>test</scope>
+ <type>test-jar</type>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
<artifactId>flink-java</artifactId>
<version>${flink.version}</version>
</dependency>
@@ -114,6 +126,22 @@
</exclusion>
</exclusions>
</dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-connector-kafka</artifactId>
+ <version>${flink.version}</version>
+ </dependency>
+ <dependency>
+ <groupId>org.mockito</groupId>
+ <artifactId>mockito-all</artifactId>
+ <version>1.9.5</version>
+ <scope>test</scope>
+ </dependency>
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-streaming-java</artifactId>
+ <version>${flink.version}</version>
+ </dependency>
</dependencies>
<build>
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java
new file mode 100644
index 0000000..66d60fa
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java
@@ -0,0 +1,238 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow;
+
+import com.dataartisans.flink.dataflow.translation.FlinkPipelineTranslator;
+import com.dataartisans.flink.dataflow.translation.FlinkBatchPipelineTranslator;
+import com.dataartisans.flink.dataflow.translation.FlinkStreamingPipelineTranslator;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.java.CollectionEnvironment;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.environment.RemoteStreamEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class FlinkJobExecutionEnvironment {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkJobExecutionEnvironment.class);
+
+ private final FlinkPipelineOptions options;
+
+ /**
+ * The Flink Batch execution environment. This is instantiated to either a
+ * {@link org.apache.flink.api.java.CollectionEnvironment},
+ * a {@link org.apache.flink.api.java.LocalEnvironment} or
+ * a {@link org.apache.flink.api.java.RemoteEnvironment}, depending on the configuration
+ * options.
+ */
+ private ExecutionEnvironment flinkBatchEnv;
+
+
+ /**
+ * The Flink Streaming execution environment. This is instantiated to either a
+ * {@link org.apache.flink.streaming.api.environment.LocalStreamEnvironment} or
+ * a {@link org.apache.flink.streaming.api.environment.RemoteStreamEnvironment}, depending
+ * on the configuration options, and more specifically, the url of the master url.
+ */
+ private StreamExecutionEnvironment flinkStreamEnv;
+
+ /**
+ * Translator for this FlinkPipelineRunner. Its role is to translate the Dataflow operators to
+ * their Flink based counterparts. Based on the options provided by the user, if we have a streaming job,
+ * this is instantiated to a FlinkStreamingPipelineTranslator. In other case, i.e. a batch job,
+ * a FlinkBatchPipelineTranslator is created.
+ */
+ private FlinkPipelineTranslator flinkPipelineTranslator;
+
+ public FlinkJobExecutionEnvironment(FlinkPipelineOptions options) {
+ if (options == null) {
+ throw new IllegalArgumentException("Options in the FlinkJobExecutionEnvironment cannot be NULL.");
+ }
+ this.options = options;
+ this.createJobEnvironment();
+ this.createJobGraphTranslator();
+ }
+
+ /**
+ * Depending on the type of job (Streaming or Batch) and the user-specified options,
+ * this method creates the adequate ExecutionEnvironment.
+ */
+ private void createJobEnvironment() {
+ if (options.isStreaming()) {
+ LOG.info("Creating the required STREAMING Environment.");
+ createStreamExecutionEnvironment();
+ } else {
+ LOG.info("Creating the required BATCH Environment.");
+ createBatchExecutionEnvironment();
+ }
+ }
+
+ /**
+ * Depending on the type of job (Streaming or Batch), this method creates the adequate job graph
+ * translator. In the case of batch, it will work with DataSets, while for streaming, it will work
+ * with DataStreams.
+ */
+ private void createJobGraphTranslator() {
+ checkInitializationState();
+ if (this.flinkPipelineTranslator != null) {
+ throw new IllegalStateException("JobGraphTranslator already initialized.");
+ }
+
+ this.flinkPipelineTranslator = options.isStreaming() ?
+ new FlinkStreamingPipelineTranslator(flinkStreamEnv, options) :
+ new FlinkBatchPipelineTranslator(flinkBatchEnv, options);
+ }
+
+ public void translate(Pipeline pipeline) {
+ checkInitializationState();
+ if(this.flinkBatchEnv == null && this.flinkStreamEnv == null) {
+ createJobEnvironment();
+ }
+ if (this.flinkPipelineTranslator == null) {
+ createJobGraphTranslator();
+ }
+ this.flinkPipelineTranslator.translate(pipeline);
+ }
+
+ public JobExecutionResult executeJob() throws Exception {
+ if (options.isStreaming()) {
+
+ System.out.println("Plan: " + this.flinkStreamEnv.getExecutionPlan());
+
+ if (this.flinkStreamEnv == null) {
+ throw new RuntimeException("JobExecutionEnvironment not initialized.");
+ }
+ if (this.flinkPipelineTranslator == null) {
+ throw new RuntimeException("JobGraphTranslator not initialized.");
+ }
+ return this.flinkStreamEnv.execute();
+ } else {
+ if (this.flinkBatchEnv == null) {
+ throw new RuntimeException("JobExecutionEnvironment not initialized.");
+ }
+ if (this.flinkPipelineTranslator == null) {
+ throw new RuntimeException("JobGraphTranslator not initialized.");
+ }
+ return this.flinkBatchEnv.execute();
+ }
+ }
+
+ /**
+ * If the submitted job is a batch processing job, this method creates the adequate
+ * Flink {@link org.apache.flink.api.java.ExecutionEnvironment} depending
+ * on the user-specified options.
+ */
+ private void createBatchExecutionEnvironment() {
+ if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) {
+ throw new RuntimeException("JobExecutionEnvironment already initialized.");
+ }
+
+ String masterUrl = options.getFlinkMaster();
+ this.flinkStreamEnv = null;
+
+ // depending on the master, create the right environment.
+ if (masterUrl.equals("[local]")) {
+ this.flinkBatchEnv = ExecutionEnvironment.createLocalEnvironment();
+ } else if (masterUrl.equals("[collection]")) {
+ this.flinkBatchEnv = new CollectionEnvironment();
+ } else if (masterUrl.equals("[auto]")) {
+ this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
+ } else if (masterUrl.matches(".*:\\d*")) {
+ String[] parts = masterUrl.split(":");
+ List<String> stagingFiles = options.getFilesToStage();
+ this.flinkBatchEnv = ExecutionEnvironment.createRemoteEnvironment(parts[0],
+ Integer.parseInt(parts[1]),
+ stagingFiles.toArray(new String[stagingFiles.size()]));
+ } else {
+ LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
+ this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
+ }
+
+ // set the correct parallelism.
+ if (options.getParallelism() != -1 && !(this.flinkBatchEnv instanceof CollectionEnvironment)) {
+ this.flinkBatchEnv.setParallelism(options.getParallelism());
+ }
+
+ // set parallelism in the options (required by some execution code)
+ options.setParallelism(flinkBatchEnv.getParallelism());
+ }
+
+ /**
+ * If the submitted job is a stream processing job, this method creates the adequate
+ * Flink {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment} depending
+ * on the user-specified options.
+ */
+ private void createStreamExecutionEnvironment() {
+ if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) {
+ throw new RuntimeException("JobExecutionEnvironment already initialized.");
+ }
+
+ String masterUrl = options.getFlinkMaster();
+ this.flinkBatchEnv = null;
+
+ // depending on the master, create the right environment.
+ if (masterUrl.equals("[local]")) {
+ this.flinkStreamEnv = StreamExecutionEnvironment.createLocalEnvironment();
+ } else if (masterUrl.equals("[auto]")) {
+ this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+ } else if (masterUrl.matches(".*:\\d*")) {
+ String[] parts = masterUrl.split(":");
+ List<String> stagingFiles = options.getFilesToStage();
+ this.flinkStreamEnv = StreamExecutionEnvironment.createRemoteEnvironment(parts[0],
+ Integer.parseInt(parts[1]), stagingFiles.toArray(new String[stagingFiles.size()]));
+ } else {
+ LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
+ this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+ }
+
+ // set the correct parallelism.
+ if (options.getParallelism() != -1) {
+ this.flinkStreamEnv.setParallelism(options.getParallelism());
+ }
+
+ // set parallelism in the options (required by some execution code)
+ options.setParallelism(flinkStreamEnv.getParallelism());
+
+ // although we do not use the generated timestamps,
+ // enabling timestamps is needed for the watermarks.
+ this.flinkStreamEnv.getConfig().enableTimestamps();
+
+ this.flinkStreamEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+
+ this.flinkStreamEnv.enableCheckpointing(1000);
+ this.flinkStreamEnv.setNumberOfExecutionRetries(5);
+
+ LOG.info("Setting execution retry delay to 3 sec");
+ this.flinkStreamEnv.getConfig().setExecutionRetryDelay(3000);
+ }
+
+ private final void checkInitializationState() {
+ if (this.options == null) {
+ throw new IllegalStateException("FlinkJobExecutionEnvironment is not initialized yet.");
+ }
+
+ if (options.isStreaming() && this.flinkBatchEnv != null) {
+ throw new IllegalStateException("Attempted to run a Streaming Job with a Batch Execution Environment.");
+ } else if (!options.isStreaming() && this.flinkStreamEnv != null) {
+ throw new IllegalStateException("Attempted to run a Batch Job with a Streaming Execution Environment.");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
index ae31f48..f57fed2 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
@@ -15,7 +15,6 @@
*/
package com.dataartisans.flink.dataflow;
-import com.dataartisans.flink.dataflow.translation.FlinkPipelineTranslator;
import com.google.cloud.dataflow.sdk.Pipeline;
import com.google.cloud.dataflow.sdk.options.PipelineOptions;
import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
@@ -28,8 +27,6 @@ import com.google.cloud.dataflow.sdk.values.POutput;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.java.CollectionEnvironment;
-import org.apache.flink.api.java.ExecutionEnvironment;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -45,27 +42,19 @@ import java.util.Map;
* A {@link PipelineRunner} that executes the operations in the
* pipeline by first translating them to a Flink Plan and then executing them either locally
* or on a Flink cluster, depending on the configuration.
- *
+ * <p>
* This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}.
*/
public class FlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {
private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineRunner.class);
- /** Provided options. */
- private final FlinkPipelineOptions options;
-
/**
- * The Flink execution environment. This is instantiated to either a
- * {@link org.apache.flink.api.java.CollectionEnvironment},
- * a {@link org.apache.flink.api.java.LocalEnvironment} or
- * a {@link org.apache.flink.api.java.RemoteEnvironment}, depending on the configuration
- * options.
+ * Provided options.
*/
- private final ExecutionEnvironment flinkEnv;
+ private final FlinkPipelineOptions options;
- /** Translator for this FlinkPipelineRunner, based on options. */
- private final FlinkPipelineTranslator translator;
+ private final FlinkJobExecutionEnvironment flinkJobEnv;
/**
* Construct a runner from the provided options.
@@ -109,90 +98,38 @@ public class FlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {
flinkOptions.setFlinkMaster("[auto]");
}
- if (flinkOptions.isStreaming()) {
- throw new RuntimeException("Streaming is currently not supported.");
- }
-
return new FlinkPipelineRunner(flinkOptions);
}
private FlinkPipelineRunner(FlinkPipelineOptions options) {
this.options = options;
- this.flinkEnv = createExecutionEnvironment(options);
-
- // set parallelism in the options (required by some execution code)
- options.setParallelism(flinkEnv.getParallelism());
-
- this.translator = new FlinkPipelineTranslator(flinkEnv, options);
- }
-
- /**
- * Create Flink {@link org.apache.flink.api.java.ExecutionEnvironment} depending
- * on the options.
- */
- private ExecutionEnvironment createExecutionEnvironment(FlinkPipelineOptions options) {
- String masterUrl = options.getFlinkMaster();
-
-
- if (masterUrl.equals("[local]")) {
- ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
- if (options.getParallelism() != -1) {
- env.setParallelism(options.getParallelism());
- }
- return env;
- } else if (masterUrl.equals("[collection]")) {
- return new CollectionEnvironment();
- } else if (masterUrl.equals("[auto]")) {
- ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
- if (options.getParallelism() != -1) {
- env.setParallelism(options.getParallelism());
- }
- return env;
- } else if (masterUrl.matches(".*:\\d*")) {
- String[] parts = masterUrl.split(":");
- List<String> stagingFiles = options.getFilesToStage();
- ExecutionEnvironment env = ExecutionEnvironment.createRemoteEnvironment(parts[0],
- Integer.parseInt(parts[1]),
- stagingFiles.toArray(new String[stagingFiles.size()]));
- if (options.getParallelism() != -1) {
- env.setParallelism(options.getParallelism());
- }
- return env;
- } else {
- LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
- ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
- if (options.getParallelism() != -1) {
- env.setParallelism(options.getParallelism());
- }
- return env;
- }
+ this.flinkJobEnv = new FlinkJobExecutionEnvironment(options);
}
@Override
public FlinkRunnerResult run(Pipeline pipeline) {
LOG.info("Executing pipeline using FlinkPipelineRunner.");
-
+
LOG.info("Translating pipeline to Flink program.");
-
- translator.translate(pipeline);
-
+
+ this.flinkJobEnv.translate(pipeline);
+
LOG.info("Starting execution of Flink program.");
JobExecutionResult result;
try {
- result = flinkEnv.execute();
- }
- catch (Exception e) {
+ result = this.flinkJobEnv.executeJob();
+ } catch (Exception e) {
LOG.error("Pipeline execution failed", e);
throw new RuntimeException("Pipeline execution failed", e);
}
-
+
LOG.info("Execution finished in {} msecs", result.getNetRuntime());
-
+
Map<String, Object> accumulators = result.getAllAccumulatorResults();
if (accumulators != null && !accumulators.isEmpty()) {
LOG.info("Final aggregator values:");
-
+
for (Map.Entry<String, Object> entry : result.getAllAccumulatorResults().entrySet()) {
LOG.info("{} : {}", entry.getKey(), entry.getValue());
}
@@ -230,16 +167,18 @@ public class FlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {
/////////////////////////////////////////////////////////////////////////////
@Override
- public String toString() { return "DataflowPipelineRunner#" + hashCode(); }
+ public String toString() {
+ return "DataflowPipelineRunner#" + hashCode();
+ }
/**
* Attempts to detect all the resources the class loader has access to. This does not recurse
* to class loader parents stopping it from pulling in resources from the system class loader.
*
* @param classLoader The URLClassLoader to use to detect resources to stage.
- * @throws IllegalArgumentException If either the class loader is not a URLClassLoader or one
- * of the resources the class loader exposes is not a file resource.
* @return A list of absolute paths to the resources the class loader uses.
+ * @throws IllegalArgumentException If either the class loader is not a URLClassLoader or one
+ * of the resources the class loader exposes is not a file resource.
*/
protected static List<String> detectClassPathResourcesToStage(ClassLoader classLoader) {
if (!(classLoader instanceof URLClassLoader)) {
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java
index 82f1e46..7857778 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java
@@ -43,7 +43,7 @@ public class WordCount {
String getOutput();
void setOutput(String value);
}
-
+
public static void main(String[] args) {
Options options = PipelineOptionsFactory.fromArgs(args).withValidation()
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java
new file mode 100644
index 0000000..0245a7b
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java
@@ -0,0 +1,384 @@
+/*
+ * Copyright (C) 2015 Google Inc.
+ *
+ * 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://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 com.dataartisans.flink.dataflow.examples.streaming;
+
+import com.dataartisans.flink.dataflow.FlinkPipelineRunner;
+import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedSocketSource;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.AvroCoder;
+import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
+import com.google.cloud.dataflow.sdk.io.*;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.*;
+import com.google.cloud.dataflow.sdk.transforms.Partition.PartitionFn;
+import com.google.cloud.dataflow.sdk.transforms.windowing.*;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PBegin;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import org.joda.time.Duration;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * To run the example, first open a socket on a terminal by executing the command:
+ * <li>
+ * <li>
+ * <code>nc -lk 9999</code>
+ * </li>
+ * </li>
+ * and then launch the example. Now whatever you type in the terminal is going to be
+ * the input to the program.
+ * */
+public class AutoComplete {
+
+ /**
+ * A PTransform that takes as input a list of tokens and returns
+ * the most common tokens per prefix.
+ */
+ public static class ComputeTopCompletions
+ extends PTransform<PCollection<String>, PCollection<KV<String, List<CompletionCandidate>>>> {
+ private static final long serialVersionUID = 0;
+
+ private final int candidatesPerPrefix;
+ private final boolean recursive;
+
+ protected ComputeTopCompletions(int candidatesPerPrefix, boolean recursive) {
+ this.candidatesPerPrefix = candidatesPerPrefix;
+ this.recursive = recursive;
+ }
+
+ public static ComputeTopCompletions top(int candidatesPerPrefix, boolean recursive) {
+ return new ComputeTopCompletions(candidatesPerPrefix, recursive);
+ }
+
+ @Override
+ public PCollection<KV<String, List<CompletionCandidate>>> apply(PCollection<String> input) {
+ PCollection<CompletionCandidate> candidates = input
+ // First count how often each token appears.
+ .apply(new Count.PerElement<String>())
+
+ // Map the KV outputs of Count into our own CompletionCandiate class.
+ .apply(ParDo.named("CreateCompletionCandidates").of(
+ new DoFn<KV<String, Long>, CompletionCandidate>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ CompletionCandidate cand = new CompletionCandidate(c.element().getKey(), c.element().getValue());
+ c.output(cand);
+ }
+ }));
+
+ // Compute the top via either a flat or recursive algorithm.
+ if (recursive) {
+ return candidates
+ .apply(new ComputeTopRecursive(candidatesPerPrefix, 1))
+ .apply(Flatten.<KV<String, List<CompletionCandidate>>>pCollections());
+ } else {
+ return candidates
+ .apply(new ComputeTopFlat(candidatesPerPrefix, 1));
+ }
+ }
+ }
+
+ /**
+ * Lower latency, but more expensive.
+ */
+ private static class ComputeTopFlat
+ extends PTransform<PCollection<CompletionCandidate>,
+ PCollection<KV<String, List<CompletionCandidate>>>> {
+ private static final long serialVersionUID = 0;
+
+ private final int candidatesPerPrefix;
+ private final int minPrefix;
+
+ public ComputeTopFlat(int candidatesPerPrefix, int minPrefix) {
+ this.candidatesPerPrefix = candidatesPerPrefix;
+ this.minPrefix = minPrefix;
+ }
+
+ @Override
+ public PCollection<KV<String, List<CompletionCandidate>>> apply(
+ PCollection<CompletionCandidate> input) {
+ return input
+ // For each completion candidate, map it to all prefixes.
+ .apply(ParDo.of(new AllPrefixes(minPrefix)))
+
+ // Find and return the top candiates for each prefix.
+ .apply(Top.<String, CompletionCandidate>largestPerKey(candidatesPerPrefix)
+ .withHotKeyFanout(new HotKeyFanout()));
+ }
+
+ private static class HotKeyFanout implements SerializableFunction<String, Integer> {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public Integer apply(String input) {
+ return (int) Math.pow(4, 5 - input.length());
+ }
+ }
+ }
+
+ /**
+ * Cheaper but higher latency.
+ *
+ * <p> Returns two PCollections, the first is top prefixes of size greater
+ * than minPrefix, and the second is top prefixes of size exactly
+ * minPrefix.
+ */
+ private static class ComputeTopRecursive
+ extends PTransform<PCollection<CompletionCandidate>,
+ PCollectionList<KV<String, List<CompletionCandidate>>>> {
+ private static final long serialVersionUID = 0;
+
+ private final int candidatesPerPrefix;
+ private final int minPrefix;
+
+ public ComputeTopRecursive(int candidatesPerPrefix, int minPrefix) {
+ this.candidatesPerPrefix = candidatesPerPrefix;
+ this.minPrefix = minPrefix;
+ }
+
+ private class KeySizePartitionFn implements PartitionFn<KV<String, List<CompletionCandidate>>> {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public int partitionFor(KV<String, List<CompletionCandidate>> elem, int numPartitions) {
+ return elem.getKey().length() > minPrefix ? 0 : 1;
+ }
+ }
+
+ private static class FlattenTops
+ extends DoFn<KV<String, List<CompletionCandidate>>, CompletionCandidate> {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ for (CompletionCandidate cc : c.element().getValue()) {
+ c.output(cc);
+ }
+ }
+ }
+
+ @Override
+ public PCollectionList<KV<String, List<CompletionCandidate>>> apply(
+ PCollection<CompletionCandidate> input) {
+ if (minPrefix > 10) {
+ // Base case, partitioning to return the output in the expected format.
+ return input
+ .apply(new ComputeTopFlat(candidatesPerPrefix, minPrefix))
+ .apply(Partition.of(2, new KeySizePartitionFn()));
+ } else {
+ // If a candidate is in the top N for prefix a...b, it must also be in the top
+ // N for a...bX for every X, which is typlically a much smaller set to consider.
+ // First, compute the top candidate for prefixes of size at least minPrefix + 1.
+ PCollectionList<KV<String, List<CompletionCandidate>>> larger = input
+ .apply(new ComputeTopRecursive(candidatesPerPrefix, minPrefix + 1));
+ // Consider the top candidates for each prefix of length minPrefix + 1...
+ PCollection<KV<String, List<CompletionCandidate>>> small =
+ PCollectionList
+ .of(larger.get(1).apply(ParDo.of(new FlattenTops())))
+ // ...together with those (previously excluded) candidates of length
+ // exactly minPrefix...
+ .and(input.apply(Filter.by(new SerializableFunction<CompletionCandidate, Boolean>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public Boolean apply(CompletionCandidate c) {
+ return c.getValue().length() == minPrefix;
+ }
+ })))
+ .apply("FlattenSmall", Flatten.<CompletionCandidate>pCollections())
+ // ...set the key to be the minPrefix-length prefix...
+ .apply(ParDo.of(new AllPrefixes(minPrefix, minPrefix)))
+ // ...and (re)apply the Top operator to all of them together.
+ .apply(Top.<String, CompletionCandidate>largestPerKey(candidatesPerPrefix));
+
+ PCollection<KV<String, List<CompletionCandidate>>> flattenLarger = larger
+ .apply("FlattenLarge", Flatten.<KV<String, List<CompletionCandidate>>>pCollections());
+
+ return PCollectionList.of(flattenLarger).and(small);
+ }
+ }
+ }
+
+ /**
+ * A DoFn that keys each candidate by all its prefixes.
+ */
+ private static class AllPrefixes
+ extends DoFn<CompletionCandidate, KV<String, CompletionCandidate>> {
+ private static final long serialVersionUID = 0;
+
+ private final int minPrefix;
+ private final int maxPrefix;
+ public AllPrefixes(int minPrefix) {
+ this(minPrefix, Integer.MAX_VALUE);
+ }
+ public AllPrefixes(int minPrefix, int maxPrefix) {
+ this.minPrefix = minPrefix;
+ this.maxPrefix = maxPrefix;
+ }
+ @Override
+ public void processElement(ProcessContext c) {
+ String word = c.element().value;
+ for (int i = minPrefix; i <= Math.min(word.length(), maxPrefix); i++) {
+ KV kv = KV.of(word.substring(0, i), c.element());
+ c.output(kv);
+ }
+ }
+ }
+
+ /**
+ * Class used to store tag-count pairs.
+ */
+ @DefaultCoder(AvroCoder.class)
+ static class CompletionCandidate implements Comparable<CompletionCandidate> {
+ private long count;
+ private String value;
+
+ public CompletionCandidate(String value, long count) {
+ this.value = value;
+ this.count = count;
+ }
+
+ public String getValue() {
+ return value;
+ }
+
+ // Empty constructor required for Avro decoding.
+ @SuppressWarnings("unused")
+ public CompletionCandidate() {}
+
+ @Override
+ public int compareTo(CompletionCandidate o) {
+ if (this.count < o.count) {
+ return -1;
+ } else if (this.count == o.count) {
+ return this.value.compareTo(o.value);
+ } else {
+ return 1;
+ }
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (other instanceof CompletionCandidate) {
+ CompletionCandidate that = (CompletionCandidate) other;
+ return this.count == that.count && this.value.equals(that.value);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return Long.valueOf(count).hashCode() ^ value.hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return "CompletionCandidate[" + value + ", " + count + "]";
+ }
+ }
+
+ static class ExtractWordsFn extends DoFn<String, String> {
+ private final Aggregator<Long, Long> emptyLines =
+ createAggregator("emptyLines", new Sum.SumLongFn());
+
+ @Override
+ public void processElement(ProcessContext c) {
+ if (c.element().trim().isEmpty()) {
+ emptyLines.addValue(1L);
+ }
+
+ // Split the line into words.
+ String[] words = c.element().split("[^a-zA-Z']+");
+
+ // Output each word encountered into the output PCollection.
+ for (String word : words) {
+ if (!word.isEmpty()) {
+ c.output(word);
+ }
+ }
+ }
+ }
+
+ /**
+ * Takes as input a the top candidates per prefix, and emits an entity
+ * suitable for writing to Datastore.
+ */
+ static class FormatForPerTaskLocalFile extends DoFn<KV<String, List<CompletionCandidate>>, String> {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ StringBuilder str = new StringBuilder();
+ KV<String, List<CompletionCandidate>> elem = c.element();
+
+ str.append(elem.getKey() +" @ "+ c.window() +" -> ");
+ for(CompletionCandidate cand: elem.getValue()) {
+ str.append(cand.toString() + " ");
+ }
+ System.out.println(str.toString());
+ c.output(str.toString());
+ }
+ }
+
+ /**
+ * Options supported by this class.
+ *
+ * <p> Inherits standard Dataflow configuration options.
+ */
+ private static interface Options extends WindowedWordCount.StreamingWordCountOptions {
+ @Description("Whether to use the recursive algorithm")
+ @Default.Boolean(true)
+ Boolean getRecursive();
+ void setRecursive(Boolean value);
+ }
+
+ public static void main(String[] args) throws IOException {
+ Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+ options.setStreaming(true);
+ options.setRunner(FlinkPipelineRunner.class);
+
+ PTransform<? super PBegin, PCollection<String>> readSource =
+ Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("WordStream");
+ WindowFn<Object, ?> windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize()));
+
+ // Create the pipeline.
+ Pipeline p = Pipeline.create(options);
+ PCollection<KV<String, List<CompletionCandidate>>> toWrite = p
+ .apply(readSource)
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Window.<String>into(windowFn)
+ .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes())
+ .apply(ComputeTopCompletions.top(10, options.getRecursive()));
+
+ toWrite
+ .apply(ParDo.named("FormatForPerTaskFile").of(new FormatForPerTaskLocalFile()))
+ .apply(TextIO.Write.to("./outputAutoComplete.txt"));
+
+ p.run();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
new file mode 100644
index 0000000..b0cc4fa
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
@@ -0,0 +1,157 @@
+/*
+ * Copyright (C) 2015 Google Inc.
+ *
+ * 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://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 com.dataartisans.flink.dataflow.examples.streaming;
+
+import com.dataartisans.flink.dataflow.FlinkPipelineRunner;
+import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedSocketSource;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
+import com.google.cloud.dataflow.sdk.transforms.windowing.*;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PBegin;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.joda.time.Duration;
+
+/**
+ * To run the example, first open two sockets on two terminals by executing the commands:
+ * <li>
+ * <li>
+ * <code>nc -lk 9999</code>, and
+ * </li>
+ * <li>
+ * <code>nc -lk 9998</code>
+ * </li>
+ * </li>
+ * and then launch the example. Now whatever you type in the terminal is going to be
+ * the input to the program.
+ * */
+public class JoinExamples {
+
+ static PCollection<String> joinEvents(PCollection<String> streamA,
+ PCollection<String> streamB) throws Exception {
+
+ final TupleTag<String> firstInfoTag = new TupleTag<String>();
+ final TupleTag<String> secondInfoTag = new TupleTag<String>();
+
+ // transform both input collections to tuple collections, where the keys are country
+ // codes in both cases.
+ PCollection<KV<String, String>> firstInfo = streamA.apply(
+ ParDo.of(new ExtractEventDataFn()));
+ PCollection<KV<String, String>> secondInfo = streamB.apply(
+ ParDo.of(new ExtractEventDataFn()));
+
+ // country code 'key' -> CGBKR (<event info>, <country name>)
+ PCollection<KV<String, CoGbkResult>> kvpCollection = KeyedPCollectionTuple
+ .of(firstInfoTag, firstInfo)
+ .and(secondInfoTag, secondInfo)
+ .apply(CoGroupByKey.<String>create());
+
+ // Process the CoGbkResult elements generated by the CoGroupByKey transform.
+ // country code 'key' -> string of <event info>, <country name>
+ PCollection<KV<String, String>> finalResultCollection =
+ kvpCollection.apply(ParDo.named("Process").of(
+ new DoFn<KV<String, CoGbkResult>, KV<String, String>>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ KV<String, CoGbkResult> e = c.element();
+ String key = e.getKey();
+
+ String defaultA = "NO_VALUE";
+
+ // the following getOnly is a bit tricky because it expects to have
+ // EXACTLY ONE value in the corresponding stream and for the corresponding key.
+
+ String lineA = e.getValue().getOnly(firstInfoTag, defaultA);
+ for (String lineB : c.element().getValue().getAll(secondInfoTag)) {
+ // Generate a string that combines information from both collection values
+ c.output(KV.of(key, "Value A: " + lineA + " - Value B: " + lineB));
+ }
+ }
+ }));
+
+ return finalResultCollection
+ .apply(ParDo.named("Format").of(new DoFn<KV<String, String>, String>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ String result = c.element().getKey() + " -> " + c.element().getValue();
+ System.out.println(result);
+ c.output(result);
+ }
+ }));
+ }
+
+ static class ExtractEventDataFn extends DoFn<String, KV<String, String>> {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ String line = c.element().toLowerCase();
+ String key = line.split("\\s")[0];
+ c.output(KV.of(key, line));
+ }
+ }
+
+ private static interface Options extends WindowedWordCount.StreamingWordCountOptions {
+
+ }
+
+ public static void main(String[] args) throws Exception {
+ Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+
+ // make it a streaming example.
+ options.setStreaming(true);
+ options.setRunner(FlinkPipelineRunner.class);
+
+ PTransform<? super PBegin, PCollection<String>> readSourceA =
+ Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("FirstStream");
+ PTransform<? super PBegin, PCollection<String>> readSourceB =
+ Read.from(new UnboundedSocketSource<>("localhost", 9998, '\n', 3)).named("SecondStream");
+
+ WindowFn<Object, ?> windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize()));
+
+ Pipeline p = Pipeline.create(options);
+
+ // the following two 'applys' create multiple inputs to our pipeline, one for each
+ // of our two input sources.
+ PCollection<String> streamA = p.apply(readSourceA)
+ .apply(Window.<String>into(windowFn)
+ .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes());
+ PCollection<String> streamB = p.apply(readSourceB)
+ .apply(Window.<String>into(windowFn)
+ .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes());
+
+ PCollection<String> formattedResults = joinEvents(streamA, streamB);
+ formattedResults.apply(TextIO.Write.to("./outputJoin.txt"));
+ p.run();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
new file mode 100644
index 0000000..46c9bd6
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
@@ -0,0 +1,138 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.examples.streaming;
+
+import com.dataartisans.flink.dataflow.FlinkPipelineRunner;
+import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedFlinkSource;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.io.UnboundedSource;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.*;
+import com.google.cloud.dataflow.sdk.transforms.windowing.*;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer082;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+import org.joda.time.Duration;
+
+import java.util.Properties;
+
+public class KafkaWindowedWordCountExample {
+
+ static final String KAFKA_TOPIC = "test"; // Default kafka topic to read from
+ static final String KAFKA_BROKER = "localhost:9092"; // Default kafka broker to contact
+ static final String GROUP_ID = "myGroup"; // Default groupId
+ static final String ZOOKEEPER = "localhost:2181"; // Default zookeeper to connect to for Kafka
+
+ public static class ExtractWordsFn extends DoFn<String, String> {
+ private final Aggregator<Long, Long> emptyLines =
+ createAggregator("emptyLines", new Sum.SumLongFn());
+
+ @Override
+ public void processElement(ProcessContext c) {
+ if (c.element().trim().isEmpty()) {
+ emptyLines.addValue(1L);
+ }
+
+ // Split the line into words.
+ String[] words = c.element().split("[^a-zA-Z']+");
+
+ // Output each word encountered into the output PCollection.
+ for (String word : words) {
+ if (!word.isEmpty()) {
+ c.output(word);
+ }
+ }
+ }
+ }
+
+ public static class FormatAsStringFn extends DoFn<KV<String, Long>, String> {
+ @Override
+ public void processElement(ProcessContext c) {
+ String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString();
+ System.out.println(row);
+ c.output(row);
+ }
+ }
+
+ public static interface KafkaStreamingWordCountOptions extends WindowedWordCount.StreamingWordCountOptions {
+ @Description("The Kafka topic to read from")
+ @Default.String(KAFKA_TOPIC)
+ String getKafkaTopic();
+
+ void setKafkaTopic(String value);
+
+ @Description("The Kafka Broker to read from")
+ @Default.String(KAFKA_BROKER)
+ String getBroker();
+
+ void setBroker(String value);
+
+ @Description("The Zookeeper server to connect to")
+ @Default.String(ZOOKEEPER)
+ String getZookeeper();
+
+ void setZookeeper(String value);
+
+ @Description("The groupId")
+ @Default.String(GROUP_ID)
+ String getGroup();
+
+ void setGroup(String value);
+
+ }
+
+ public static void main(String[] args) {
+ PipelineOptionsFactory.register(KafkaStreamingWordCountOptions.class);
+ KafkaStreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).as(KafkaStreamingWordCountOptions.class);
+ options.setJobName("KafkaExample");
+ options.setStreaming(true);
+ options.setRunner(FlinkPipelineRunner.class);
+
+ System.out.println(options.getKafkaTopic() +" "+ options.getZookeeper() +" "+ options.getBroker() +" "+ options.getGroup() );
+ Pipeline pipeline = Pipeline.create(options);
+
+ Properties p = new Properties();
+ p.setProperty("zookeeper.connect", options.getZookeeper());
+ p.setProperty("bootstrap.servers", options.getBroker());
+ p.setProperty("group.id", options.getGroup());
+
+ // this is the Flink consumer that reads the input to
+ // the program from a kafka topic.
+ FlinkKafkaConsumer082 kafkaConsumer = new FlinkKafkaConsumer082<>(
+ options.getKafkaTopic(),
+ new SimpleStringSchema(), p);
+
+ PCollection<String> words = pipeline
+ .apply(Read.from(new UnboundedFlinkSource<String, UnboundedSource.CheckpointMark>(options, kafkaConsumer)).named("StreamingWordCount"))
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(options.getWindowSize())))
+ .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes());
+
+ PCollection<KV<String, Long>> wordCounts =
+ words.apply(Count.<String>perElement());
+
+ wordCounts.apply(ParDo.of(new FormatAsStringFn()))
+ .apply(TextIO.Write.to("./outputKafka.txt"));
+
+ pipeline.run();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
new file mode 100644
index 0000000..1d4a44b
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
@@ -0,0 +1,126 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.examples.streaming;
+
+import com.dataartisans.flink.dataflow.FlinkPipelineRunner;
+import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedSocketSource;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.*;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.*;
+import com.google.cloud.dataflow.sdk.transforms.windowing.*;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * To run the example, first open a socket on a terminal by executing the command:
+ * <li>
+ * <li>
+ * <code>nc -lk 9999</code>
+ * </li>
+ * </li>
+ * and then launch the example. Now whatever you type in the terminal is going to be
+ * the input to the program.
+ * */
+public class WindowedWordCount {
+
+ private static final Logger LOG = LoggerFactory.getLogger(WindowedWordCount.class);
+
+ static final long WINDOW_SIZE = 10; // Default window duration in seconds
+ static final long SLIDE_SIZE = 5; // Default window slide in seconds
+
+ static class FormatAsStringFn extends DoFn<KV<String, Long>, String> {
+ @Override
+ public void processElement(ProcessContext c) {
+ String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString();
+ c.output(row);
+ }
+ }
+
+ static class ExtractWordsFn extends DoFn<String, String> {
+ private final Aggregator<Long, Long> emptyLines =
+ createAggregator("emptyLines", new Sum.SumLongFn());
+
+ @Override
+ public void processElement(ProcessContext c) {
+ if (c.element().trim().isEmpty()) {
+ emptyLines.addValue(1L);
+ }
+
+ // Split the line into words.
+ String[] words = c.element().split("[^a-zA-Z']+");
+
+ // Output each word encountered into the output PCollection.
+ for (String word : words) {
+ if (!word.isEmpty()) {
+ c.output(word);
+ }
+ }
+ }
+ }
+
+ public static interface StreamingWordCountOptions extends com.dataartisans.flink.dataflow.examples.WordCount.Options {
+ @Description("Sliding window duration, in seconds")
+ @Default.Long(WINDOW_SIZE)
+ Long getWindowSize();
+
+ void setWindowSize(Long value);
+
+ @Description("Window slide, in seconds")
+ @Default.Long(SLIDE_SIZE)
+ Long getSlide();
+
+ void setSlide(Long value);
+ }
+
+ public static void main(String[] args) throws IOException {
+ StreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(StreamingWordCountOptions.class);
+ options.setStreaming(true);
+ options.setWindowSize(10L);
+ options.setSlide(5L);
+ options.setRunner(FlinkPipelineRunner.class);
+
+ LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize() +
+ " sec. and a slide of " + options.getSlide());
+
+ Pipeline pipeline = Pipeline.create(options);
+
+ PCollection<String> words = pipeline
+ .apply(Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("StreamingWordCount"))
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Window.<String>into(SlidingWindows.of(Duration.standardSeconds(options.getWindowSize()))
+ .every(Duration.standardSeconds(options.getSlide())))
+ .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes());
+
+ PCollection<KV<String, Long>> wordCounts =
+ words.apply(Count.<String>perElement());
+
+ wordCounts.apply(ParDo.of(new FormatAsStringFn()))
+ .apply(TextIO.Write.to("./outputWordCount.txt"));
+
+ pipeline.run();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java
new file mode 100644
index 0000000..8c0183e
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java
@@ -0,0 +1,152 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation;
+
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import org.apache.flink.api.java.ExecutionEnvironment;
+
+/**
+ * FlinkBatchPipelineTranslator knows how to translate Pipeline objects into Flink Jobs.
+ * This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator}
+ */
+public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
+
+ /**
+ * The necessary context in the case of a batch job.
+ */
+ private final FlinkBatchTranslationContext batchContext;
+
+ private int depth = 0;
+
+ /**
+ * Composite transform that we want to translate before proceeding with other transforms.
+ */
+ private PTransform<?, ?> currentCompositeTransform;
+
+ public FlinkBatchPipelineTranslator(ExecutionEnvironment env, PipelineOptions options) {
+ this.batchContext = new FlinkBatchTranslationContext(env, options);
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Pipeline Visitor Methods
+ // --------------------------------------------------------------------------------------------
+
+ @Override
+ public void enterCompositeTransform(TransformTreeNode node) {
+ System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node));
+
+ PTransform<?, ?> transform = node.getTransform();
+ if (transform != null && currentCompositeTransform == null) {
+
+ BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
+ if (translator != null) {
+ currentCompositeTransform = transform;
+ if (transform instanceof CoGroupByKey && node.getInput().expand().size() != 2) {
+ // we can only optimize CoGroupByKey for input size 2
+ currentCompositeTransform = null;
+ }
+ }
+ }
+ this.depth++;
+ }
+
+ @Override
+ public void leaveCompositeTransform(TransformTreeNode node) {
+ PTransform<?, ?> transform = node.getTransform();
+ if (transform != null && currentCompositeTransform == transform) {
+
+ BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
+ if (translator != null) {
+ System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node));
+ applyBatchTransform(transform, node, translator);
+ currentCompositeTransform = null;
+ } else {
+ throw new IllegalStateException("Attempted to translate composite transform " +
+ "but no translator was found: " + currentCompositeTransform);
+ }
+ }
+ this.depth--;
+ System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node));
+ }
+
+ @Override
+ public void visitTransform(TransformTreeNode node) {
+ System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node));
+ if (currentCompositeTransform != null) {
+ // ignore it
+ return;
+ }
+
+ // get the transformation corresponding to hte node we are
+ // currently visiting and translate it into its Flink alternative.
+
+ PTransform<?, ?> transform = node.getTransform();
+ BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
+ if (translator == null) {
+ System.out.println(node.getTransform().getClass());
+ throw new UnsupportedOperationException("The transform " + transform + " is currently not supported.");
+ }
+ applyBatchTransform(transform, node, translator);
+ }
+
+ @Override
+ public void visitValue(PValue value, TransformTreeNode producer) {
+ // do nothing here
+ }
+
+ private <T extends PTransform<?, ?>> void applyBatchTransform(PTransform<?, ?> transform, TransformTreeNode node, BatchTransformTranslator<?> translator) {
+ if (this.batchContext == null) {
+ throw new IllegalStateException("The FlinkPipelineTranslator is not yet initialized.");
+ }
+
+ @SuppressWarnings("unchecked")
+ T typedTransform = (T) transform;
+
+ @SuppressWarnings("unchecked")
+ BatchTransformTranslator<T> typedTranslator = (BatchTransformTranslator<T>) translator;
+
+ // create the applied PTransform on the batchContext
+ batchContext.setCurrentTransform(AppliedPTransform.of(
+ node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform));
+ typedTranslator.translateNode(typedTransform, batchContext);
+ }
+
+ /**
+ * A translator of a {@link PTransform}.
+ */
+ public interface BatchTransformTranslator<Type extends PTransform> {
+ void translateNode(Type transform, FlinkBatchTranslationContext context);
+ }
+
+ private static String genSpaces(int n) {
+ String s = "";
+ for (int i = 0; i < n; i++) {
+ s += "| ";
+ }
+ return s;
+ }
+
+ private static String formatNodeName(TransformTreeNode node) {
+ return node.toString().split("@")[1] + node.getTransform();
+ }
+}
[46/50] [abbrv] incubator-beam git commit: [flink] convert tabs to 2
spaces
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
index 60f6788..a6e1e37 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
@@ -50,109 +50,109 @@ import org.joda.time.Duration;
* */
public class JoinExamples {
- static PCollection<String> joinEvents(PCollection<String> streamA,
- PCollection<String> streamB) throws Exception {
-
- final TupleTag<String> firstInfoTag = new TupleTag<>();
- final TupleTag<String> secondInfoTag = new TupleTag<>();
-
- // transform both input collections to tuple collections, where the keys are country
- // codes in both cases.
- PCollection<KV<String, String>> firstInfo = streamA.apply(
- ParDo.of(new ExtractEventDataFn()));
- PCollection<KV<String, String>> secondInfo = streamB.apply(
- ParDo.of(new ExtractEventDataFn()));
-
- // country code 'key' -> CGBKR (<event info>, <country name>)
- PCollection<KV<String, CoGbkResult>> kvpCollection = KeyedPCollectionTuple
- .of(firstInfoTag, firstInfo)
- .and(secondInfoTag, secondInfo)
- .apply(CoGroupByKey.<String>create());
-
- // Process the CoGbkResult elements generated by the CoGroupByKey transform.
- // country code 'key' -> string of <event info>, <country name>
- PCollection<KV<String, String>> finalResultCollection =
- kvpCollection.apply(ParDo.named("Process").of(
- new DoFn<KV<String, CoGbkResult>, KV<String, String>>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- KV<String, CoGbkResult> e = c.element();
- String key = e.getKey();
-
- String defaultA = "NO_VALUE";
-
- // the following getOnly is a bit tricky because it expects to have
- // EXACTLY ONE value in the corresponding stream and for the corresponding key.
-
- String lineA = e.getValue().getOnly(firstInfoTag, defaultA);
- for (String lineB : c.element().getValue().getAll(secondInfoTag)) {
- // Generate a string that combines information from both collection values
- c.output(KV.of(key, "Value A: " + lineA + " - Value B: " + lineB));
- }
- }
- }));
-
- return finalResultCollection
- .apply(ParDo.named("Format").of(new DoFn<KV<String, String>, String>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- String result = c.element().getKey() + " -> " + c.element().getValue();
- System.out.println(result);
- c.output(result);
- }
- }));
- }
-
- static class ExtractEventDataFn extends DoFn<String, KV<String, String>> {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- String line = c.element().toLowerCase();
- String key = line.split("\\s")[0];
- c.output(KV.of(key, line));
- }
- }
-
- private interface Options extends WindowedWordCount.StreamingWordCountOptions {
-
- }
-
- public static void main(String[] args) throws Exception {
- Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
- options.setStreaming(true);
- options.setCheckpointingInterval(1000L);
- options.setNumberOfExecutionRetries(5);
- options.setExecutionRetryDelay(3000L);
- options.setRunner(FlinkPipelineRunner.class);
-
- PTransform<? super PBegin, PCollection<String>> readSourceA =
- Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("FirstStream");
- PTransform<? super PBegin, PCollection<String>> readSourceB =
- Read.from(new UnboundedSocketSource<>("localhost", 9998, '\n', 3)).named("SecondStream");
-
- WindowFn<Object, ?> windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize()));
-
- Pipeline p = Pipeline.create(options);
-
- // the following two 'applys' create multiple inputs to our pipeline, one for each
- // of our two input sources.
- PCollection<String> streamA = p.apply(readSourceA)
- .apply(Window.<String>into(windowFn)
- .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
- .discardingFiredPanes());
- PCollection<String> streamB = p.apply(readSourceB)
- .apply(Window.<String>into(windowFn)
- .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
- .discardingFiredPanes());
-
- PCollection<String> formattedResults = joinEvents(streamA, streamB);
- formattedResults.apply(TextIO.Write.to("./outputJoin.txt"));
- p.run();
- }
+ static PCollection<String> joinEvents(PCollection<String> streamA,
+ PCollection<String> streamB) throws Exception {
+
+ final TupleTag<String> firstInfoTag = new TupleTag<>();
+ final TupleTag<String> secondInfoTag = new TupleTag<>();
+
+ // transform both input collections to tuple collections, where the keys are country
+ // codes in both cases.
+ PCollection<KV<String, String>> firstInfo = streamA.apply(
+ ParDo.of(new ExtractEventDataFn()));
+ PCollection<KV<String, String>> secondInfo = streamB.apply(
+ ParDo.of(new ExtractEventDataFn()));
+
+ // country code 'key' -> CGBKR (<event info>, <country name>)
+ PCollection<KV<String, CoGbkResult>> kvpCollection = KeyedPCollectionTuple
+ .of(firstInfoTag, firstInfo)
+ .and(secondInfoTag, secondInfo)
+ .apply(CoGroupByKey.<String>create());
+
+ // Process the CoGbkResult elements generated by the CoGroupByKey transform.
+ // country code 'key' -> string of <event info>, <country name>
+ PCollection<KV<String, String>> finalResultCollection =
+ kvpCollection.apply(ParDo.named("Process").of(
+ new DoFn<KV<String, CoGbkResult>, KV<String, String>>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ KV<String, CoGbkResult> e = c.element();
+ String key = e.getKey();
+
+ String defaultA = "NO_VALUE";
+
+ // the following getOnly is a bit tricky because it expects to have
+ // EXACTLY ONE value in the corresponding stream and for the corresponding key.
+
+ String lineA = e.getValue().getOnly(firstInfoTag, defaultA);
+ for (String lineB : c.element().getValue().getAll(secondInfoTag)) {
+ // Generate a string that combines information from both collection values
+ c.output(KV.of(key, "Value A: " + lineA + " - Value B: " + lineB));
+ }
+ }
+ }));
+
+ return finalResultCollection
+ .apply(ParDo.named("Format").of(new DoFn<KV<String, String>, String>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ String result = c.element().getKey() + " -> " + c.element().getValue();
+ System.out.println(result);
+ c.output(result);
+ }
+ }));
+ }
+
+ static class ExtractEventDataFn extends DoFn<String, KV<String, String>> {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ String line = c.element().toLowerCase();
+ String key = line.split("\\s")[0];
+ c.output(KV.of(key, line));
+ }
+ }
+
+ private interface Options extends WindowedWordCount.StreamingWordCountOptions {
+
+ }
+
+ public static void main(String[] args) throws Exception {
+ Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+ options.setStreaming(true);
+ options.setCheckpointingInterval(1000L);
+ options.setNumberOfExecutionRetries(5);
+ options.setExecutionRetryDelay(3000L);
+ options.setRunner(FlinkPipelineRunner.class);
+
+ PTransform<? super PBegin, PCollection<String>> readSourceA =
+ Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("FirstStream");
+ PTransform<? super PBegin, PCollection<String>> readSourceB =
+ Read.from(new UnboundedSocketSource<>("localhost", 9998, '\n', 3)).named("SecondStream");
+
+ WindowFn<Object, ?> windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize()));
+
+ Pipeline p = Pipeline.create(options);
+
+ // the following two 'applys' create multiple inputs to our pipeline, one for each
+ // of our two input sources.
+ PCollection<String> streamA = p.apply(readSourceA)
+ .apply(Window.<String>into(windowFn)
+ .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes());
+ PCollection<String> streamB = p.apply(readSourceB)
+ .apply(Window.<String>into(windowFn)
+ .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes());
+
+ PCollection<String> formattedResults = joinEvents(streamA, streamB);
+ formattedResults.apply(TextIO.Write.to("./outputJoin.txt"));
+ p.run();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
index dba2721..b97c35c 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
@@ -36,106 +36,106 @@ import java.util.Properties;
public class KafkaWindowedWordCountExample {
- static final String KAFKA_TOPIC = "test"; // Default kafka topic to read from
- static final String KAFKA_BROKER = "localhost:9092"; // Default kafka broker to contact
- static final String GROUP_ID = "myGroup"; // Default groupId
- static final String ZOOKEEPER = "localhost:2181"; // Default zookeeper to connect to for Kafka
-
- public static class ExtractWordsFn extends DoFn<String, String> {
- private final Aggregator<Long, Long> emptyLines =
- createAggregator("emptyLines", new Sum.SumLongFn());
-
- @Override
- public void processElement(ProcessContext c) {
- if (c.element().trim().isEmpty()) {
- emptyLines.addValue(1L);
- }
-
- // Split the line into words.
- String[] words = c.element().split("[^a-zA-Z']+");
-
- // Output each word encountered into the output PCollection.
- for (String word : words) {
- if (!word.isEmpty()) {
- c.output(word);
- }
- }
- }
- }
-
- public static class FormatAsStringFn extends DoFn<KV<String, Long>, String> {
- @Override
- public void processElement(ProcessContext c) {
- String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString();
- System.out.println(row);
- c.output(row);
- }
- }
-
- public interface KafkaStreamingWordCountOptions extends WindowedWordCount.StreamingWordCountOptions {
- @Description("The Kafka topic to read from")
- @Default.String(KAFKA_TOPIC)
- String getKafkaTopic();
-
- void setKafkaTopic(String value);
-
- @Description("The Kafka Broker to read from")
- @Default.String(KAFKA_BROKER)
- String getBroker();
-
- void setBroker(String value);
-
- @Description("The Zookeeper server to connect to")
- @Default.String(ZOOKEEPER)
- String getZookeeper();
-
- void setZookeeper(String value);
-
- @Description("The groupId")
- @Default.String(GROUP_ID)
- String getGroup();
-
- void setGroup(String value);
-
- }
-
- public static void main(String[] args) {
- PipelineOptionsFactory.register(KafkaStreamingWordCountOptions.class);
- KafkaStreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).as(KafkaStreamingWordCountOptions.class);
- options.setJobName("KafkaExample");
- options.setStreaming(true);
- options.setCheckpointingInterval(1000L);
- options.setNumberOfExecutionRetries(5);
- options.setExecutionRetryDelay(3000L);
- options.setRunner(FlinkPipelineRunner.class);
-
- System.out.println(options.getKafkaTopic() +" "+ options.getZookeeper() +" "+ options.getBroker() +" "+ options.getGroup() );
- Pipeline pipeline = Pipeline.create(options);
-
- Properties p = new Properties();
- p.setProperty("zookeeper.connect", options.getZookeeper());
- p.setProperty("bootstrap.servers", options.getBroker());
- p.setProperty("group.id", options.getGroup());
-
- // this is the Flink consumer that reads the input to
- // the program from a kafka topic.
- FlinkKafkaConsumer082 kafkaConsumer = new FlinkKafkaConsumer082<>(
- options.getKafkaTopic(),
- new SimpleStringSchema(), p);
-
- PCollection<String> words = pipeline
- .apply(Read.from(new UnboundedFlinkSource<String, UnboundedSource.CheckpointMark>(options, kafkaConsumer)).named("StreamingWordCount"))
- .apply(ParDo.of(new ExtractWordsFn()))
- .apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(options.getWindowSize())))
- .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
- .discardingFiredPanes());
-
- PCollection<KV<String, Long>> wordCounts =
- words.apply(Count.<String>perElement());
-
- wordCounts.apply(ParDo.of(new FormatAsStringFn()))
- .apply(TextIO.Write.to("./outputKafka.txt"));
-
- pipeline.run();
- }
+ static final String KAFKA_TOPIC = "test"; // Default kafka topic to read from
+ static final String KAFKA_BROKER = "localhost:9092"; // Default kafka broker to contact
+ static final String GROUP_ID = "myGroup"; // Default groupId
+ static final String ZOOKEEPER = "localhost:2181"; // Default zookeeper to connect to for Kafka
+
+ public static class ExtractWordsFn extends DoFn<String, String> {
+ private final Aggregator<Long, Long> emptyLines =
+ createAggregator("emptyLines", new Sum.SumLongFn());
+
+ @Override
+ public void processElement(ProcessContext c) {
+ if (c.element().trim().isEmpty()) {
+ emptyLines.addValue(1L);
+ }
+
+ // Split the line into words.
+ String[] words = c.element().split("[^a-zA-Z']+");
+
+ // Output each word encountered into the output PCollection.
+ for (String word : words) {
+ if (!word.isEmpty()) {
+ c.output(word);
+ }
+ }
+ }
+ }
+
+ public static class FormatAsStringFn extends DoFn<KV<String, Long>, String> {
+ @Override
+ public void processElement(ProcessContext c) {
+ String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString();
+ System.out.println(row);
+ c.output(row);
+ }
+ }
+
+ public interface KafkaStreamingWordCountOptions extends WindowedWordCount.StreamingWordCountOptions {
+ @Description("The Kafka topic to read from")
+ @Default.String(KAFKA_TOPIC)
+ String getKafkaTopic();
+
+ void setKafkaTopic(String value);
+
+ @Description("The Kafka Broker to read from")
+ @Default.String(KAFKA_BROKER)
+ String getBroker();
+
+ void setBroker(String value);
+
+ @Description("The Zookeeper server to connect to")
+ @Default.String(ZOOKEEPER)
+ String getZookeeper();
+
+ void setZookeeper(String value);
+
+ @Description("The groupId")
+ @Default.String(GROUP_ID)
+ String getGroup();
+
+ void setGroup(String value);
+
+ }
+
+ public static void main(String[] args) {
+ PipelineOptionsFactory.register(KafkaStreamingWordCountOptions.class);
+ KafkaStreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).as(KafkaStreamingWordCountOptions.class);
+ options.setJobName("KafkaExample");
+ options.setStreaming(true);
+ options.setCheckpointingInterval(1000L);
+ options.setNumberOfExecutionRetries(5);
+ options.setExecutionRetryDelay(3000L);
+ options.setRunner(FlinkPipelineRunner.class);
+
+ System.out.println(options.getKafkaTopic() +" "+ options.getZookeeper() +" "+ options.getBroker() +" "+ options.getGroup() );
+ Pipeline pipeline = Pipeline.create(options);
+
+ Properties p = new Properties();
+ p.setProperty("zookeeper.connect", options.getZookeeper());
+ p.setProperty("bootstrap.servers", options.getBroker());
+ p.setProperty("group.id", options.getGroup());
+
+ // this is the Flink consumer that reads the input to
+ // the program from a kafka topic.
+ FlinkKafkaConsumer082 kafkaConsumer = new FlinkKafkaConsumer082<>(
+ options.getKafkaTopic(),
+ new SimpleStringSchema(), p);
+
+ PCollection<String> words = pipeline
+ .apply(Read.from(new UnboundedFlinkSource<String, UnboundedSource.CheckpointMark>(options, kafkaConsumer)).named("StreamingWordCount"))
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(options.getWindowSize())))
+ .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes());
+
+ PCollection<KV<String, Long>> wordCounts =
+ words.apply(Count.<String>perElement());
+
+ wordCounts.apply(ParDo.of(new FormatAsStringFn()))
+ .apply(TextIO.Write.to("./outputKafka.txt"));
+
+ pipeline.run();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
index 37dc39a..753cbc3 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
@@ -45,84 +45,84 @@ import java.io.IOException;
* */
public class WindowedWordCount {
- private static final Logger LOG = LoggerFactory.getLogger(WindowedWordCount.class);
-
- static final long WINDOW_SIZE = 10; // Default window duration in seconds
- static final long SLIDE_SIZE = 5; // Default window slide in seconds
-
- static class FormatAsStringFn extends DoFn<KV<String, Long>, String> {
- @Override
- public void processElement(ProcessContext c) {
- String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString();
- c.output(row);
- }
- }
-
- static class ExtractWordsFn extends DoFn<String, String> {
- private final Aggregator<Long, Long> emptyLines =
- createAggregator("emptyLines", new Sum.SumLongFn());
-
- @Override
- public void processElement(ProcessContext c) {
- if (c.element().trim().isEmpty()) {
- emptyLines.addValue(1L);
- }
-
- // Split the line into words.
- String[] words = c.element().split("[^a-zA-Z']+");
-
- // Output each word encountered into the output PCollection.
- for (String word : words) {
- if (!word.isEmpty()) {
- c.output(word);
- }
- }
- }
- }
-
- public interface StreamingWordCountOptions extends com.dataartisans.flink.dataflow.examples.WordCount.Options {
- @Description("Sliding window duration, in seconds")
- @Default.Long(WINDOW_SIZE)
- Long getWindowSize();
-
- void setWindowSize(Long value);
-
- @Description("Window slide, in seconds")
- @Default.Long(SLIDE_SIZE)
- Long getSlide();
-
- void setSlide(Long value);
- }
-
- public static void main(String[] args) throws IOException {
- StreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(StreamingWordCountOptions.class);
- options.setStreaming(true);
- options.setWindowSize(10L);
- options.setSlide(5L);
- options.setCheckpointingInterval(1000L);
- options.setNumberOfExecutionRetries(5);
- options.setExecutionRetryDelay(3000L);
- options.setRunner(FlinkPipelineRunner.class);
-
- LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize() +
- " sec. and a slide of " + options.getSlide());
-
- Pipeline pipeline = Pipeline.create(options);
-
- PCollection<String> words = pipeline
- .apply(Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("StreamingWordCount"))
- .apply(ParDo.of(new ExtractWordsFn()))
- .apply(Window.<String>into(SlidingWindows.of(Duration.standardSeconds(options.getWindowSize()))
- .every(Duration.standardSeconds(options.getSlide())))
- .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
- .discardingFiredPanes());
-
- PCollection<KV<String, Long>> wordCounts =
- words.apply(Count.<String>perElement());
-
- wordCounts.apply(ParDo.of(new FormatAsStringFn()))
- .apply(TextIO.Write.to("./outputWordCount.txt"));
-
- pipeline.run();
- }
+ private static final Logger LOG = LoggerFactory.getLogger(WindowedWordCount.class);
+
+ static final long WINDOW_SIZE = 10; // Default window duration in seconds
+ static final long SLIDE_SIZE = 5; // Default window slide in seconds
+
+ static class FormatAsStringFn extends DoFn<KV<String, Long>, String> {
+ @Override
+ public void processElement(ProcessContext c) {
+ String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString();
+ c.output(row);
+ }
+ }
+
+ static class ExtractWordsFn extends DoFn<String, String> {
+ private final Aggregator<Long, Long> emptyLines =
+ createAggregator("emptyLines", new Sum.SumLongFn());
+
+ @Override
+ public void processElement(ProcessContext c) {
+ if (c.element().trim().isEmpty()) {
+ emptyLines.addValue(1L);
+ }
+
+ // Split the line into words.
+ String[] words = c.element().split("[^a-zA-Z']+");
+
+ // Output each word encountered into the output PCollection.
+ for (String word : words) {
+ if (!word.isEmpty()) {
+ c.output(word);
+ }
+ }
+ }
+ }
+
+ public interface StreamingWordCountOptions extends com.dataartisans.flink.dataflow.examples.WordCount.Options {
+ @Description("Sliding window duration, in seconds")
+ @Default.Long(WINDOW_SIZE)
+ Long getWindowSize();
+
+ void setWindowSize(Long value);
+
+ @Description("Window slide, in seconds")
+ @Default.Long(SLIDE_SIZE)
+ Long getSlide();
+
+ void setSlide(Long value);
+ }
+
+ public static void main(String[] args) throws IOException {
+ StreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(StreamingWordCountOptions.class);
+ options.setStreaming(true);
+ options.setWindowSize(10L);
+ options.setSlide(5L);
+ options.setCheckpointingInterval(1000L);
+ options.setNumberOfExecutionRetries(5);
+ options.setExecutionRetryDelay(3000L);
+ options.setRunner(FlinkPipelineRunner.class);
+
+ LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize() +
+ " sec. and a slide of " + options.getSlide());
+
+ Pipeline pipeline = Pipeline.create(options);
+
+ PCollection<String> words = pipeline
+ .apply(Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("StreamingWordCount"))
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Window.<String>into(SlidingWindows.of(Duration.standardSeconds(options.getWindowSize()))
+ .every(Duration.standardSeconds(options.getSlide())))
+ .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes());
+
+ PCollection<KV<String, Long>> wordCounts =
+ words.apply(Count.<String>perElement());
+
+ wordCounts.apply(ParDo.of(new FormatAsStringFn()))
+ .apply(TextIO.Write.to("./outputWordCount.txt"));
+
+ pipeline.run();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java
index 90fb635..3f3492c 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java
@@ -28,53 +28,53 @@ import com.google.cloud.dataflow.sdk.values.PDone;
*/
public class ConsoleIO {
- /**
- * A PTransform that writes a PCollection to a standard output.
- */
- public static class Write {
+ /**
+ * A PTransform that writes a PCollection to a standard output.
+ */
+ public static class Write {
- /**
- * Returns a ConsoleIO.Write PTransform with a default step name.
- */
- public static Bound create() {
- return new Bound();
- }
+ /**
+ * Returns a ConsoleIO.Write PTransform with a default step name.
+ */
+ public static Bound create() {
+ return new Bound();
+ }
- /**
- * Returns a ConsoleIO.Write PTransform with the given step name.
- */
- public static Bound named(String name) {
- return new Bound().named(name);
- }
+ /**
+ * Returns a ConsoleIO.Write PTransform with the given step name.
+ */
+ public static Bound named(String name) {
+ return new Bound().named(name);
+ }
- /**
- * A PTransform that writes a bounded PCollection to standard output.
- */
- public static class Bound extends PTransform<PCollection<?>, PDone> {
- private static final long serialVersionUID = 0;
+ /**
+ * A PTransform that writes a bounded PCollection to standard output.
+ */
+ public static class Bound extends PTransform<PCollection<?>, PDone> {
+ private static final long serialVersionUID = 0;
- Bound() {
- super("ConsoleIO.Write");
- }
+ Bound() {
+ super("ConsoleIO.Write");
+ }
- Bound(String name) {
- super(name);
- }
+ Bound(String name) {
+ super(name);
+ }
- /**
- * Returns a new ConsoleIO.Write PTransform that's like this one but with the given
- * step
- * name. Does not modify this object.
- */
- public Bound named(String name) {
- return new Bound(name);
- }
+ /**
+ * Returns a new ConsoleIO.Write PTransform that's like this one but with the given
+ * step
+ * name. Does not modify this object.
+ */
+ public Bound named(String name) {
+ return new Bound(name);
+ }
- @Override
- public PDone apply(PCollection<?> input) {
- return PDone.in(input.getPipeline());
- }
- }
- }
+ @Override
+ public PDone apply(PCollection<?> input) {
+ return PDone.in(input.getPipeline());
+ }
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java
index a1e4410..82b7e97 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java
@@ -31,119 +31,119 @@ import org.apache.flink.api.java.ExecutionEnvironment;
*/
public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
- /**
- * The necessary context in the case of a batch job.
- */
- private final FlinkBatchTranslationContext batchContext;
-
- private int depth = 0;
-
- /**
- * Composite transform that we want to translate before proceeding with other transforms.
- */
- private PTransform<?, ?> currentCompositeTransform;
-
- public FlinkBatchPipelineTranslator(ExecutionEnvironment env, PipelineOptions options) {
- this.batchContext = new FlinkBatchTranslationContext(env, options);
- }
-
- // --------------------------------------------------------------------------------------------
- // Pipeline Visitor Methods
- // --------------------------------------------------------------------------------------------
-
- @Override
- public void enterCompositeTransform(TransformTreeNode node) {
- System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node));
-
- PTransform<?, ?> transform = node.getTransform();
- if (transform != null && currentCompositeTransform == null) {
-
- BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
- if (translator != null) {
- currentCompositeTransform = transform;
- if (transform instanceof CoGroupByKey && node.getInput().expand().size() != 2) {
- // we can only optimize CoGroupByKey for input size 2
- currentCompositeTransform = null;
- }
- }
- }
- this.depth++;
- }
-
- @Override
- public void leaveCompositeTransform(TransformTreeNode node) {
- PTransform<?, ?> transform = node.getTransform();
- if (transform != null && currentCompositeTransform == transform) {
-
- BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
- if (translator != null) {
- System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node));
- applyBatchTransform(transform, node, translator);
- currentCompositeTransform = null;
- } else {
- throw new IllegalStateException("Attempted to translate composite transform " +
- "but no translator was found: " + currentCompositeTransform);
- }
- }
- this.depth--;
- System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node));
- }
-
- @Override
- public void visitTransform(TransformTreeNode node) {
- System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node));
- if (currentCompositeTransform != null) {
- // ignore it
- return;
- }
-
- // get the transformation corresponding to hte node we are
- // currently visiting and translate it into its Flink alternative.
-
- PTransform<?, ?> transform = node.getTransform();
- BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
- if (translator == null) {
- System.out.println(node.getTransform().getClass());
- throw new UnsupportedOperationException("The transform " + transform + " is currently not supported.");
- }
- applyBatchTransform(transform, node, translator);
- }
-
- @Override
- public void visitValue(PValue value, TransformTreeNode producer) {
- // do nothing here
- }
-
- private <T extends PTransform<?, ?>> void applyBatchTransform(PTransform<?, ?> transform, TransformTreeNode node, BatchTransformTranslator<?> translator) {
-
- @SuppressWarnings("unchecked")
- T typedTransform = (T) transform;
-
- @SuppressWarnings("unchecked")
- BatchTransformTranslator<T> typedTranslator = (BatchTransformTranslator<T>) translator;
-
- // create the applied PTransform on the batchContext
- batchContext.setCurrentTransform(AppliedPTransform.of(
- node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform));
- typedTranslator.translateNode(typedTransform, batchContext);
- }
-
- /**
- * A translator of a {@link PTransform}.
- */
- public interface BatchTransformTranslator<Type extends PTransform> {
- void translateNode(Type transform, FlinkBatchTranslationContext context);
- }
-
- private static String genSpaces(int n) {
- String s = "";
- for (int i = 0; i < n; i++) {
- s += "| ";
- }
- return s;
- }
-
- private static String formatNodeName(TransformTreeNode node) {
- return node.toString().split("@")[1] + node.getTransform();
- }
+ /**
+ * The necessary context in the case of a batch job.
+ */
+ private final FlinkBatchTranslationContext batchContext;
+
+ private int depth = 0;
+
+ /**
+ * Composite transform that we want to translate before proceeding with other transforms.
+ */
+ private PTransform<?, ?> currentCompositeTransform;
+
+ public FlinkBatchPipelineTranslator(ExecutionEnvironment env, PipelineOptions options) {
+ this.batchContext = new FlinkBatchTranslationContext(env, options);
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Pipeline Visitor Methods
+ // --------------------------------------------------------------------------------------------
+
+ @Override
+ public void enterCompositeTransform(TransformTreeNode node) {
+ System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node));
+
+ PTransform<?, ?> transform = node.getTransform();
+ if (transform != null && currentCompositeTransform == null) {
+
+ BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
+ if (translator != null) {
+ currentCompositeTransform = transform;
+ if (transform instanceof CoGroupByKey && node.getInput().expand().size() != 2) {
+ // we can only optimize CoGroupByKey for input size 2
+ currentCompositeTransform = null;
+ }
+ }
+ }
+ this.depth++;
+ }
+
+ @Override
+ public void leaveCompositeTransform(TransformTreeNode node) {
+ PTransform<?, ?> transform = node.getTransform();
+ if (transform != null && currentCompositeTransform == transform) {
+
+ BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
+ if (translator != null) {
+ System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node));
+ applyBatchTransform(transform, node, translator);
+ currentCompositeTransform = null;
+ } else {
+ throw new IllegalStateException("Attempted to translate composite transform " +
+ "but no translator was found: " + currentCompositeTransform);
+ }
+ }
+ this.depth--;
+ System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node));
+ }
+
+ @Override
+ public void visitTransform(TransformTreeNode node) {
+ System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node));
+ if (currentCompositeTransform != null) {
+ // ignore it
+ return;
+ }
+
+ // get the transformation corresponding to hte node we are
+ // currently visiting and translate it into its Flink alternative.
+
+ PTransform<?, ?> transform = node.getTransform();
+ BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
+ if (translator == null) {
+ System.out.println(node.getTransform().getClass());
+ throw new UnsupportedOperationException("The transform " + transform + " is currently not supported.");
+ }
+ applyBatchTransform(transform, node, translator);
+ }
+
+ @Override
+ public void visitValue(PValue value, TransformTreeNode producer) {
+ // do nothing here
+ }
+
+ private <T extends PTransform<?, ?>> void applyBatchTransform(PTransform<?, ?> transform, TransformTreeNode node, BatchTransformTranslator<?> translator) {
+
+ @SuppressWarnings("unchecked")
+ T typedTransform = (T) transform;
+
+ @SuppressWarnings("unchecked")
+ BatchTransformTranslator<T> typedTranslator = (BatchTransformTranslator<T>) translator;
+
+ // create the applied PTransform on the batchContext
+ batchContext.setCurrentTransform(AppliedPTransform.of(
+ node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform));
+ typedTranslator.translateNode(typedTransform, batchContext);
+ }
+
+ /**
+ * A translator of a {@link PTransform}.
+ */
+ public interface BatchTransformTranslator<Type extends PTransform> {
+ void translateNode(Type transform, FlinkBatchTranslationContext context);
+ }
+
+ private static String genSpaces(int n) {
+ String s = "";
+ for (int i = 0; i < n; i++) {
+ s += "| ";
+ }
+ return s;
+ }
+
+ private static String formatNodeName(TransformTreeNode node) {
+ return node.toString().split("@")[1] + node.getTransform();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
index 0e45a21..6a8409c 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
@@ -93,502 +93,502 @@ import java.util.Map;
*/
public class FlinkBatchTransformTranslators {
- // --------------------------------------------------------------------------------------------
- // Transform Translator Registry
- // --------------------------------------------------------------------------------------------
-
- @SuppressWarnings("rawtypes")
- private static final Map<Class<? extends PTransform>, FlinkBatchPipelineTranslator.BatchTransformTranslator> TRANSLATORS = new HashMap<>();
+ // --------------------------------------------------------------------------------------------
+ // Transform Translator Registry
+ // --------------------------------------------------------------------------------------------
+
+ @SuppressWarnings("rawtypes")
+ private static final Map<Class<? extends PTransform>, FlinkBatchPipelineTranslator.BatchTransformTranslator> TRANSLATORS = new HashMap<>();
- // register the known translators
- static {
- TRANSLATORS.put(View.CreatePCollectionView.class, new CreatePCollectionViewTranslatorBatch());
+ // register the known translators
+ static {
+ TRANSLATORS.put(View.CreatePCollectionView.class, new CreatePCollectionViewTranslatorBatch());
- TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslatorBatch());
- // we don't need this because we translate the Combine.PerKey directly
- //TRANSLATORS.put(Combine.GroupedValues.class, new CombineGroupedValuesTranslator());
+ TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslatorBatch());
+ // we don't need this because we translate the Combine.PerKey directly
+ //TRANSLATORS.put(Combine.GroupedValues.class, new CombineGroupedValuesTranslator());
- TRANSLATORS.put(Create.Values.class, new CreateTranslatorBatch());
+ TRANSLATORS.put(Create.Values.class, new CreateTranslatorBatch());
- TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslatorBatch());
+ TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslatorBatch());
- TRANSLATORS.put(GroupByKey.GroupByKeyOnly.class, new GroupByKeyOnlyTranslatorBatch());
- // TODO we're currently ignoring windows here but that has to change in the future
- TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch());
-
- TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslatorBatch());
- TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslatorBatch());
-
- TRANSLATORS.put(CoGroupByKey.class, new CoGroupByKeyTranslatorBatch());
-
- TRANSLATORS.put(AvroIO.Read.Bound.class, new AvroIOReadTranslatorBatch());
- TRANSLATORS.put(AvroIO.Write.Bound.class, new AvroIOWriteTranslatorBatch());
-
- TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslatorBatch());
- TRANSLATORS.put(Write.Bound.class, new WriteSinkTranslatorBatch());
-
- TRANSLATORS.put(TextIO.Read.Bound.class, new TextIOReadTranslatorBatch());
- TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteTranslatorBatch());
-
- // Flink-specific
- TRANSLATORS.put(ConsoleIO.Write.Bound.class, new ConsoleIOWriteTranslatorBatch());
-
- }
-
-
- public static FlinkBatchPipelineTranslator.BatchTransformTranslator<?> getTranslator(PTransform<?, ?> transform) {
- return TRANSLATORS.get(transform.getClass());
- }
-
- private static class ReadSourceTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Read.Bounded<T>> {
-
- @Override
- public void translateNode(Read.Bounded<T> transform, FlinkBatchTranslationContext context) {
- String name = transform.getName();
- BoundedSource<T> source = transform.getSource();
- PCollection<T> output = context.getOutput(transform);
- Coder<T> coder = output.getCoder();
-
- TypeInformation<T> typeInformation = context.getTypeInfo(output);
-
- DataSource<T> dataSource = new DataSource<>(context.getExecutionEnvironment(),
- new SourceInputFormat<>(source, context.getPipelineOptions()), typeInformation, name);
-
- context.setOutputDataSet(output, dataSource);
- }
- }
-
- private static class AvroIOReadTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<AvroIO.Read.Bound<T>> {
- private static final Logger LOG = LoggerFactory.getLogger(AvroIOReadTranslatorBatch.class);
-
- @Override
- public void translateNode(AvroIO.Read.Bound<T> transform, FlinkBatchTranslationContext context) {
- String path = transform.getFilepattern();
- String name = transform.getName();
-// Schema schema = transform.getSchema();
- PValue output = context.getOutput(transform);
-
- TypeInformation<T> typeInformation = context.getTypeInfo(output);
-
- // This is super hacky, but unfortunately we cannot get the type otherwise
- Class<T> extractedAvroType;
- try {
- Field typeField = transform.getClass().getDeclaredField("type");
- typeField.setAccessible(true);
- @SuppressWarnings("unchecked")
- Class<T> avroType = (Class<T>) typeField.get(transform);
- extractedAvroType = avroType;
- } catch (NoSuchFieldException | IllegalAccessException e) {
- // we know that the field is there and it is accessible
- throw new RuntimeException("Could not access type from AvroIO.Bound", e);
- }
-
- DataSource<T> source = new DataSource<>(context.getExecutionEnvironment(),
- new AvroInputFormat<>(new Path(path), extractedAvroType),
- typeInformation, name);
-
- context.setOutputDataSet(output, source);
- }
- }
-
- private static class AvroIOWriteTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<AvroIO.Write.Bound<T>> {
- private static final Logger LOG = LoggerFactory.getLogger(AvroIOWriteTranslatorBatch.class);
-
- @Override
- public void translateNode(AvroIO.Write.Bound<T> transform, FlinkBatchTranslationContext context) {
- DataSet<T> inputDataSet = context.getInputDataSet(context.getInput(transform));
- String filenamePrefix = transform.getFilenamePrefix();
- String filenameSuffix = transform.getFilenameSuffix();
- int numShards = transform.getNumShards();
- String shardNameTemplate = transform.getShardNameTemplate();
-
- // TODO: Implement these. We need Flink support for this.
- LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.",
- filenameSuffix);
- LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
-
- // This is super hacky, but unfortunately we cannot get the type otherwise
- Class<T> extractedAvroType;
- try {
- Field typeField = transform.getClass().getDeclaredField("type");
- typeField.setAccessible(true);
- @SuppressWarnings("unchecked")
- Class<T> avroType = (Class<T>) typeField.get(transform);
- extractedAvroType = avroType;
- } catch (NoSuchFieldException | IllegalAccessException e) {
- // we know that the field is there and it is accessible
- throw new RuntimeException("Could not access type from AvroIO.Bound", e);
- }
-
- DataSink<T> dataSink = inputDataSet.output(new AvroOutputFormat<>(new Path
- (filenamePrefix), extractedAvroType));
-
- if (numShards > 0) {
- dataSink.setParallelism(numShards);
- }
- }
- }
-
- private static class TextIOReadTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator<TextIO.Read.Bound<String>> {
- private static final Logger LOG = LoggerFactory.getLogger(TextIOReadTranslatorBatch.class);
-
- @Override
- public void translateNode(TextIO.Read.Bound<String> transform, FlinkBatchTranslationContext context) {
- String path = transform.getFilepattern();
- String name = transform.getName();
-
- TextIO.CompressionType compressionType = transform.getCompressionType();
- boolean needsValidation = transform.needsValidation();
-
- // TODO: Implement these. We need Flink support for this.
- LOG.warn("Translation of TextIO.CompressionType not yet supported. Is: {}.", compressionType);
- LOG.warn("Translation of TextIO.Read.needsValidation not yet supported. Is: {}.", needsValidation);
-
- PValue output = context.getOutput(transform);
-
- TypeInformation<String> typeInformation = context.getTypeInfo(output);
- DataSource<String> source = new DataSource<>(context.getExecutionEnvironment(), new TextInputFormat(new Path(path)), typeInformation, name);
-
- context.setOutputDataSet(output, source);
- }
- }
-
- private static class TextIOWriteTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<TextIO.Write.Bound<T>> {
- private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteTranslatorBatch.class);
-
- @Override
- public void translateNode(TextIO.Write.Bound<T> transform, FlinkBatchTranslationContext context) {
- PValue input = context.getInput(transform);
- DataSet<T> inputDataSet = context.getInputDataSet(input);
-
- String filenamePrefix = transform.getFilenamePrefix();
- String filenameSuffix = transform.getFilenameSuffix();
- boolean needsValidation = transform.needsValidation();
- int numShards = transform.getNumShards();
- String shardNameTemplate = transform.getShardNameTemplate();
-
- // TODO: Implement these. We need Flink support for this.
- LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation);
- LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix);
- LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
-
- //inputDataSet.print();
- DataSink<T> dataSink = inputDataSet.writeAsText(filenamePrefix);
-
- if (numShards > 0) {
- dataSink.setParallelism(numShards);
- }
- }
- }
-
- private static class ConsoleIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator<ConsoleIO.Write.Bound> {
- @Override
- public void translateNode(ConsoleIO.Write.Bound transform, FlinkBatchTranslationContext context) {
- PValue input = context.getInput(transform);
- DataSet<?> inputDataSet = context.getInputDataSet(input);
- inputDataSet.printOnTaskManager(transform.getName());
- }
- }
+ TRANSLATORS.put(GroupByKey.GroupByKeyOnly.class, new GroupByKeyOnlyTranslatorBatch());
+ // TODO we're currently ignoring windows here but that has to change in the future
+ TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch());
+
+ TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslatorBatch());
+ TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslatorBatch());
+
+ TRANSLATORS.put(CoGroupByKey.class, new CoGroupByKeyTranslatorBatch());
+
+ TRANSLATORS.put(AvroIO.Read.Bound.class, new AvroIOReadTranslatorBatch());
+ TRANSLATORS.put(AvroIO.Write.Bound.class, new AvroIOWriteTranslatorBatch());
+
+ TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslatorBatch());
+ TRANSLATORS.put(Write.Bound.class, new WriteSinkTranslatorBatch());
+
+ TRANSLATORS.put(TextIO.Read.Bound.class, new TextIOReadTranslatorBatch());
+ TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteTranslatorBatch());
+
+ // Flink-specific
+ TRANSLATORS.put(ConsoleIO.Write.Bound.class, new ConsoleIOWriteTranslatorBatch());
+
+ }
+
+
+ public static FlinkBatchPipelineTranslator.BatchTransformTranslator<?> getTranslator(PTransform<?, ?> transform) {
+ return TRANSLATORS.get(transform.getClass());
+ }
+
+ private static class ReadSourceTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Read.Bounded<T>> {
+
+ @Override
+ public void translateNode(Read.Bounded<T> transform, FlinkBatchTranslationContext context) {
+ String name = transform.getName();
+ BoundedSource<T> source = transform.getSource();
+ PCollection<T> output = context.getOutput(transform);
+ Coder<T> coder = output.getCoder();
+
+ TypeInformation<T> typeInformation = context.getTypeInfo(output);
+
+ DataSource<T> dataSource = new DataSource<>(context.getExecutionEnvironment(),
+ new SourceInputFormat<>(source, context.getPipelineOptions()), typeInformation, name);
+
+ context.setOutputDataSet(output, dataSource);
+ }
+ }
+
+ private static class AvroIOReadTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<AvroIO.Read.Bound<T>> {
+ private static final Logger LOG = LoggerFactory.getLogger(AvroIOReadTranslatorBatch.class);
+
+ @Override
+ public void translateNode(AvroIO.Read.Bound<T> transform, FlinkBatchTranslationContext context) {
+ String path = transform.getFilepattern();
+ String name = transform.getName();
+// Schema schema = transform.getSchema();
+ PValue output = context.getOutput(transform);
+
+ TypeInformation<T> typeInformation = context.getTypeInfo(output);
+
+ // This is super hacky, but unfortunately we cannot get the type otherwise
+ Class<T> extractedAvroType;
+ try {
+ Field typeField = transform.getClass().getDeclaredField("type");
+ typeField.setAccessible(true);
+ @SuppressWarnings("unchecked")
+ Class<T> avroType = (Class<T>) typeField.get(transform);
+ extractedAvroType = avroType;
+ } catch (NoSuchFieldException | IllegalAccessException e) {
+ // we know that the field is there and it is accessible
+ throw new RuntimeException("Could not access type from AvroIO.Bound", e);
+ }
+
+ DataSource<T> source = new DataSource<>(context.getExecutionEnvironment(),
+ new AvroInputFormat<>(new Path(path), extractedAvroType),
+ typeInformation, name);
+
+ context.setOutputDataSet(output, source);
+ }
+ }
+
+ private static class AvroIOWriteTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<AvroIO.Write.Bound<T>> {
+ private static final Logger LOG = LoggerFactory.getLogger(AvroIOWriteTranslatorBatch.class);
+
+ @Override
+ public void translateNode(AvroIO.Write.Bound<T> transform, FlinkBatchTranslationContext context) {
+ DataSet<T> inputDataSet = context.getInputDataSet(context.getInput(transform));
+ String filenamePrefix = transform.getFilenamePrefix();
+ String filenameSuffix = transform.getFilenameSuffix();
+ int numShards = transform.getNumShards();
+ String shardNameTemplate = transform.getShardNameTemplate();
+
+ // TODO: Implement these. We need Flink support for this.
+ LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.",
+ filenameSuffix);
+ LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
+
+ // This is super hacky, but unfortunately we cannot get the type otherwise
+ Class<T> extractedAvroType;
+ try {
+ Field typeField = transform.getClass().getDeclaredField("type");
+ typeField.setAccessible(true);
+ @SuppressWarnings("unchecked")
+ Class<T> avroType = (Class<T>) typeField.get(transform);
+ extractedAvroType = avroType;
+ } catch (NoSuchFieldException | IllegalAccessException e) {
+ // we know that the field is there and it is accessible
+ throw new RuntimeException("Could not access type from AvroIO.Bound", e);
+ }
+
+ DataSink<T> dataSink = inputDataSet.output(new AvroOutputFormat<>(new Path
+ (filenamePrefix), extractedAvroType));
+
+ if (numShards > 0) {
+ dataSink.setParallelism(numShards);
+ }
+ }
+ }
+
+ private static class TextIOReadTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator<TextIO.Read.Bound<String>> {
+ private static final Logger LOG = LoggerFactory.getLogger(TextIOReadTranslatorBatch.class);
+
+ @Override
+ public void translateNode(TextIO.Read.Bound<String> transform, FlinkBatchTranslationContext context) {
+ String path = transform.getFilepattern();
+ String name = transform.getName();
+
+ TextIO.CompressionType compressionType = transform.getCompressionType();
+ boolean needsValidation = transform.needsValidation();
+
+ // TODO: Implement these. We need Flink support for this.
+ LOG.warn("Translation of TextIO.CompressionType not yet supported. Is: {}.", compressionType);
+ LOG.warn("Translation of TextIO.Read.needsValidation not yet supported. Is: {}.", needsValidation);
+
+ PValue output = context.getOutput(transform);
+
+ TypeInformation<String> typeInformation = context.getTypeInfo(output);
+ DataSource<String> source = new DataSource<>(context.getExecutionEnvironment(), new TextInputFormat(new Path(path)), typeInformation, name);
+
+ context.setOutputDataSet(output, source);
+ }
+ }
+
+ private static class TextIOWriteTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<TextIO.Write.Bound<T>> {
+ private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteTranslatorBatch.class);
+
+ @Override
+ public void translateNode(TextIO.Write.Bound<T> transform, FlinkBatchTranslationContext context) {
+ PValue input = context.getInput(transform);
+ DataSet<T> inputDataSet = context.getInputDataSet(input);
+
+ String filenamePrefix = transform.getFilenamePrefix();
+ String filenameSuffix = transform.getFilenameSuffix();
+ boolean needsValidation = transform.needsValidation();
+ int numShards = transform.getNumShards();
+ String shardNameTemplate = transform.getShardNameTemplate();
+
+ // TODO: Implement these. We need Flink support for this.
+ LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation);
+ LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix);
+ LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
+
+ //inputDataSet.print();
+ DataSink<T> dataSink = inputDataSet.writeAsText(filenamePrefix);
+
+ if (numShards > 0) {
+ dataSink.setParallelism(numShards);
+ }
+ }
+ }
+
+ private static class ConsoleIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator<ConsoleIO.Write.Bound> {
+ @Override
+ public void translateNode(ConsoleIO.Write.Bound transform, FlinkBatchTranslationContext context) {
+ PValue input = context.getInput(transform);
+ DataSet<?> inputDataSet = context.getInputDataSet(input);
+ inputDataSet.printOnTaskManager(transform.getName());
+ }
+ }
- private static class WriteSinkTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Write.Bound<T>> {
-
- @Override
- public void translateNode(Write.Bound<T> transform, FlinkBatchTranslationContext context) {
- String name = transform.getName();
- PValue input = context.getInput(transform);
- DataSet<T> inputDataSet = context.getInputDataSet(input);
+ private static class WriteSinkTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Write.Bound<T>> {
+
+ @Override
+ public void translateNode(Write.Bound<T> transform, FlinkBatchTranslationContext context) {
+ String name = transform.getName();
+ PValue input = context.getInput(transform);
+ DataSet<T> inputDataSet = context.getInputDataSet(input);
- inputDataSet.output(new SinkOutputFormat<>(transform, context.getPipelineOptions())).name(name);
- }
- }
+ inputDataSet.output(new SinkOutputFormat<>(transform, context.getPipelineOptions())).name(name);
+ }
+ }
- private static class GroupByKeyOnlyTranslatorBatch<K, V> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<GroupByKey.GroupByKeyOnly<K, V>> {
+ private static class GroupByKeyOnlyTranslatorBatch<K, V> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<GroupByKey.GroupByKeyOnly<K, V>> {
- @Override
- public void translateNode(GroupByKey.GroupByKeyOnly<K, V> transform, FlinkBatchTranslationContext context) {
- DataSet<KV<K, V>> inputDataSet = context.getInputDataSet(context.getInput(transform));
- GroupReduceFunction<KV<K, V>, KV<K, Iterable<V>>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>();
+ @Override
+ public void translateNode(GroupByKey.GroupByKeyOnly<K, V> transform, FlinkBatchTranslationContext context) {
+ DataSet<KV<K, V>> inputDataSet = context.getInputDataSet(context.getInput(transform));
+ GroupReduceFunction<KV<K, V>, KV<K, Iterable<V>>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>();
- TypeInformation<KV<K, Iterable<V>>> typeInformation = context.getTypeInfo(context.getOutput(transform));
+ TypeInformation<KV<K, Iterable<V>>> typeInformation = context.getTypeInfo(context.getOutput(transform));
- Grouping<KV<K, V>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType()));
+ Grouping<KV<K, V>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType()));
- GroupReduceOperator<KV<K, V>, KV<K, Iterable<V>>> outputDataSet =
- new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
- context.setOutputDataSet(context.getOutput(transform), outputDataSet);
- }
- }
+ GroupReduceOperator<KV<K, V>, KV<K, Iterable<V>>> outputDataSet =
+ new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
+ context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+ }
+ }
- /**
- * Translates a GroupByKey while ignoring window assignments. This is identical to the {@link GroupByKeyOnlyTranslatorBatch}
- */
- private static class GroupByKeyTranslatorBatch<K, V> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<GroupByKey<K, V>> {
+ /**
+ * Translates a GroupByKey while ignoring window assignments. This is identical to the {@link GroupByKeyOnlyTranslatorBatch}
+ */
+ private static class GroupByKeyTranslatorBatch<K, V> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<GroupByKey<K, V>> {
- @Override
- public void translateNode(GroupByKey<K, V> transform, FlinkBatchTranslationContext context) {
- DataSet<KV<K, V>> inputDataSet = context.getInputDataSet(context.getInput(transform));
- GroupReduceFunction<KV<K, V>, KV<K, Iterable<V>>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>();
+ @Override
+ public void translateNode(GroupByKey<K, V> transform, FlinkBatchTranslationContext context) {
+ DataSet<KV<K, V>> inputDataSet = context.getInputDataSet(context.getInput(transform));
+ GroupReduceFunction<KV<K, V>, KV<K, Iterable<V>>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>();
- TypeInformation<KV<K, Iterable<V>>> typeInformation = context.getTypeInfo(context.getOutput(transform));
+ TypeInformation<KV<K, Iterable<V>>> typeInformation = context.getTypeInfo(context.getOutput(transform));
- Grouping<KV<K, V>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType()));
+ Grouping<KV<K, V>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType()));
- GroupReduceOperator<KV<K, V>, KV<K, Iterable<V>>> outputDataSet =
- new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
+ GroupReduceOperator<KV<K, V>, KV<K, Iterable<V>>> outputDataSet =
+ new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
- context.setOutputDataSet(context.getOutput(transform), outputDataSet);
- }
- }
+ context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+ }
+ }
- private static class CombinePerKeyTranslatorBatch<K, VI, VA, VO> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Combine.PerKey<K, VI, VO>> {
+ private static class CombinePerKeyTranslatorBatch<K, VI, VA, VO> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Combine.PerKey<K, VI, VO>> {
- @Override
- public void translateNode(Combine.PerKey<K, VI, VO> transform, FlinkBatchTranslationContext context) {
- DataSet<KV<K, VI>> inputDataSet = context.getInputDataSet(context.getInput(transform));
+ @Override
+ public void translateNode(Combine.PerKey<K, VI, VO> transform, FlinkBatchTranslationContext context) {
+ DataSet<KV<K, VI>> inputDataSet = context.getInputDataSet(context.getInput(transform));
- @SuppressWarnings("unchecked")
- Combine.KeyedCombineFn<K, VI, VA, VO> keyedCombineFn = (Combine.KeyedCombineFn<K, VI, VA, VO>) transform.getFn();
+ @SuppressWarnings("unchecked")
+ Combine.KeyedCombineFn<K, VI, VA, VO> keyedCombineFn = (Combine.KeyedCombineFn<K, VI, VA, VO>) transform.getFn();
- KvCoder<K, VI> inputCoder = (KvCoder<K, VI>) context.getInput(transform).getCoder();
+ KvCoder<K, VI> inputCoder = (KvCoder<K, VI>) context.getInput(transform).getCoder();
- Coder<VA> accumulatorCoder =
- null;
- try {
- accumulatorCoder = keyedCombineFn.getAccumulatorCoder(context.getInput(transform).getPipeline().getCoderRegistry(), inputCoder.getKeyCoder(), inputCoder.getValueCoder());
- } catch (CannotProvideCoderException e) {
- e.printStackTrace();
- // TODO
- }
+ Coder<VA> accumulatorCoder =
+ null;
+ try {
+ accumulatorCoder = keyedCombineFn.getAccumulatorCoder(context.getInput(transform).getPipeline().getCoderRegistry(), inputCoder.getKeyCoder(), inputCoder.getValueCoder());
+ } catch (CannotProvideCoderException e) {
+ e.printStackTrace();
+ // TODO
+ }
- TypeInformation<KV<K, VI>> kvCoderTypeInformation = new KvCoderTypeInformation<>(inputCoder);
- TypeInformation<KV<K, VA>> partialReduceTypeInfo = new KvCoderTypeInformation<>(KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder));
+ TypeInformation<KV<K, VI>> kvCoderTypeInformation = new KvCoderTypeInformation<>(inputCoder);
+ TypeInformation<KV<K, VA>> partialReduceTypeInfo = new KvCoderTypeInformation<>(KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder));
- Grouping<KV<K, VI>> inputGrouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation));
+ Grouping<KV<K, VI>> inputGrouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation));
- FlinkPartialReduceFunction<K, VI, VA> partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn);
+ FlinkPartialReduceFunction<K, VI, VA> partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn);
- // Partially GroupReduce the values into the intermediate format VA (combine)
- GroupCombineOperator<KV<K, VI>, KV<K, VA>> groupCombine =
- new GroupCombineOperator<>(inputGrouping, partialReduceTypeInfo, partialReduceFunction,
- "GroupCombine: " + transform.getName());
+ // Partially GroupReduce the values into the intermediate format VA (combine)
+ GroupCombineOperator<KV<K, VI>, KV<K, VA>> groupCombine =
+ new GroupCombineOperator<>(inputGrouping, partialReduceTypeInfo, partialReduceFunction,
+ "GroupCombine: " + transform.getName());
- // Reduce fully to VO
- GroupReduceFunction<KV<K, VA>, KV<K, VO>> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn);
+ // Reduce fully to VO
+ GroupReduceFunction<KV<K, VA>, KV<K, VO>> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn);
- TypeInformation<KV<K, VO>> reduceTypeInfo = context.getTypeInfo(context.getOutput(transform));
+ TypeInformation<KV<K, VO>> reduceTypeInfo = context.getTypeInfo(context.getOutput(transform));
- Grouping<KV<K, VA>> intermediateGrouping = new UnsortedGrouping<>(groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType()));
+ Grouping<KV<K, VA>> intermediateGrouping = new UnsortedGrouping<>(groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType()));
- // Fully reduce the values and create output format VO
- GroupReduceOperator<KV<K, VA>, KV<K, VO>> outputDataSet =
- new GroupReduceOperator<>(intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName());
+ // Fully reduce the values and create output format VO
+ GroupReduceOperator<KV<K, VA>, KV<K, VO>> outputDataSet =
+ new GroupReduceOperator<>(intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName());
- context.setOutputDataSet(context.getOutput(transform), outputDataSet);
- }
- }
+ context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+ }
+ }
-// private static class CombineGroupedValuesTranslator<K, VI, VO> implements FlinkPipelineTranslator.TransformTranslator<Combine.GroupedValues<K, VI, VO>> {
+// private static class CombineGroupedValuesTranslator<K, VI, VO> implements FlinkPipelineTranslator.TransformTranslator<Combine.GroupedValues<K, VI, VO>> {
//
-// @Override
-// public void translateNode(Combine.GroupedValues<K, VI, VO> transform, TranslationContext context) {
-// DataSet<KV<K, VI>> inputDataSet = context.getInputDataSet(transform.getInput());
+// @Override
+// public void translateNode(Combine.GroupedValues<K, VI, VO> transform, TranslationContext context) {
+// DataSet<KV<K, VI>> inputDataSet = context.getInputDataSet(transform.getInput());
//
-// Combine.KeyedCombineFn<? super K, ? super VI, ?, VO> keyedCombineFn = transform.getFn();
+// Combine.KeyedCombineFn<? super K, ? super VI, ?, VO> keyedCombineFn = transform.getFn();
//
-// GroupReduceFunction<KV<K, VI>, KV<K, VO>> groupReduceFunction = new FlinkCombineFunction<>(keyedCombineFn);
+// GroupReduceFunction<KV<K, VI>, KV<K, VO>> groupReduceFunction = new FlinkCombineFunction<>(keyedCombineFn);
//
-// TypeInformation<KV<K, VO>> typeInformation = context.getTypeInfo(transform.getOutput());
+// TypeInformation<KV<K, VO>> typeInformation = context.getTypeInfo(transform.getOutput());
//
-// Grouping<KV<K, VI>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{""}, inputDataSet.getType()));
+// Grouping<KV<K, VI>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{""}, inputDataSet.getType()));
//
-// GroupReduceOperator<KV<K, VI>, KV<K, VO>> outputDataSet =
-// new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
-// context.setOutputDataSet(transform.getOutput(), outputDataSet);
-// }
-// }
-
- private static class ParDoBoundTranslatorBatch<IN, OUT> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<ParDo.Bound<IN, OUT>> {
- private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundTranslatorBatch.class);
-
- @Override
- public void translateNode(ParDo.Bound<IN, OUT> transform, FlinkBatchTranslationContext context) {
- DataSet<IN> inputDataSet = context.getInputDataSet(context.getInput(transform));
-
- final DoFn<IN, OUT> doFn = transform.getFn();
-
- TypeInformation<OUT> typeInformation = context.getTypeInfo(context.getOutput(transform));
-
- FlinkDoFnFunction<IN, OUT> doFnWrapper = new FlinkDoFnFunction<>(doFn, context.getPipelineOptions());
- MapPartitionOperator<IN, OUT> outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName());
-
- transformSideInputs(transform.getSideInputs(), outputDataSet, context);
-
- context.setOutputDataSet(context.getOutput(transform), outputDataSet);
- }
- }
-
- private static class ParDoBoundMultiTranslatorBatch<IN, OUT> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<ParDo.BoundMulti<IN, OUT>> {
- private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundMultiTranslatorBatch.class);
-
- @Override
- public void translateNode(ParDo.BoundMulti<IN, OUT> transform, FlinkBatchTranslationContext context) {
- DataSet<IN> inputDataSet = context.getInputDataSet(context.getInput(transform));
-
- final DoFn<IN, OUT> doFn = transform.getFn();
-
- Map<TupleTag<?>, PCollection<?>> outputs = context.getOutput(transform).getAll();
-
- Map<TupleTag<?>, Integer> outputMap = Maps.newHashMap();
- // put the main output at index 0, FlinkMultiOutputDoFnFunction also expects this
- outputMap.put(transform.getMainOutputTag(), 0);
- int count = 1;
- for (TupleTag<?> tag: outputs.keySet()) {
- if (!outputMap.containsKey(tag)) {
- outputMap.put(tag, count++);
- }
- }
-
- // collect all output Coders and create a UnionCoder for our tagged outputs
- List<Coder<?>> outputCoders = Lists.newArrayList();
- for (PCollection<?> coll: outputs.values()) {
- outputCoders.add(coll.getCoder());
- }
-
- UnionCoder unionCoder = UnionCoder.of(outputCoders);
-
- @SuppressWarnings("unchecked")
- TypeInformation<RawUnionValue> typeInformation = new CoderTypeInformation<>(unionCoder);
-
- @SuppressWarnings("unchecked")
- FlinkMultiOutputDoFnFunction<IN, OUT> doFnWrapper = new FlinkMultiOutputDoFnFunction(doFn, context.getPipelineOptions(), outputMap);
- MapPartitionOperator<IN, RawUnionValue> outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName());
-
- transformSideInputs(transform.getSideInputs(), outputDataSet, context);
-
- for (Map.Entry<TupleTag<?>, PCollection<?>> output: outputs.entrySet()) {
- TypeInformation<Object> outputType = context.getTypeInfo(output.getValue());
- int outputTag = outputMap.get(output.getKey());
- FlinkMultiOutputPruningFunction<Object> pruningFunction = new FlinkMultiOutputPruningFunction<>(outputTag);
- FlatMapOperator<RawUnionValue, Object> pruningOperator = new
- FlatMapOperator<>(outputDataSet, outputType,
- pruningFunction, output.getValue().getName());
- context.setOutputDataSet(output.getValue(), pruningOperator);
-
- }
- }
- }
-
- private static class FlattenPCollectionTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Flatten.FlattenPCollectionList<T>> {
-
- @Override
- public void translateNode(Flatten.FlattenPCollectionList<T> transform, FlinkBatchTranslationContext context) {
- List<PCollection<T>> allInputs = context.getInput(transform).getAll();
- DataSet<T> result = null;
- for(PCollection<T> collection : allInputs) {
- DataSet<T> current = context.getInputDataSet(collection);
- if (result == null) {
- result = current;
- } else {
- result = result.union(current);
- }
- }
- context.setOutputDataSet(context.getOutput(transform), result);
- }
- }
-
- private static class CreatePCollectionViewTranslatorBatch<R, T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<View.CreatePCollectionView<R, T>> {
- @Override
- public void translateNode(View.CreatePCollectionView<R, T> transform, FlinkBatchTranslationContext context) {
- DataSet<T> inputDataSet = context.getInputDataSet(context.getInput(transform));
- PCollectionView<T> input = transform.apply(null);
- context.setSideInputDataSet(input, inputDataSet);
- }
- }
-
- private static class CreateTranslatorBatch<OUT> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Create.Values<OUT>> {
-
- @Override
- public void translateNode(Create.Values<OUT> transform, FlinkBatchTranslationContext context) {
- TypeInformation<OUT> typeInformation = context.getOutputTypeInfo();
- Iterable<OUT> elements = transform.getElements();
-
- // we need to serialize the elements to byte arrays, since they might contain
- // elements that are not serializable by Java serialization. We deserialize them
- // in the FlatMap function using the Coder.
-
- List<byte[]> serializedElements = Lists.newArrayList();
- Coder<OUT> coder = context.getOutput(transform).getCoder();
- for (OUT element: elements) {
- ByteArrayOutputStream bao = new ByteArrayOutputStream();
- try {
- coder.encode(element, bao, Coder.Context.OUTER);
- serializedElements.add(bao.toByteArray());
- } catch (IOException e) {
- throw new RuntimeException("Could not serialize Create elements using Coder: " + e);
- }
- }
-
- DataSet<Integer> initDataSet = context.getExecutionEnvironment().fromElements(1);
- FlinkCreateFunction<Integer, OUT> flatMapFunction = new FlinkCreateFunction<>(serializedElements, coder);
- FlatMapOperator<Integer, OUT> outputDataSet = new FlatMapOperator<>(initDataSet, typeInformation, flatMapFunction, transform.getName());
-
- context.setOutputDataSet(context.getOutput(transform), outputDataSet);
- }
- }
-
- private static void transformSideInputs(List<PCollectionView<?>> sideInputs,
- MapPartitionOperator<?, ?> outputDataSet,
- FlinkBatchTranslationContext context) {
- // get corresponding Flink broadcast DataSets
- for(PCollectionView<?> input : sideInputs) {
- DataSet<?> broadcastSet = context.getSideInputDataSet(input);
- outputDataSet.withBroadcastSet(broadcastSet, input.getTagInternal().getId());
- }
- }
+// GroupReduceOperator<KV<K, VI>, KV<K, VO>> outputDataSet =
+// new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
+// context.setOutputDataSet(transform.getOutput(), outputDataSet);
+// }
+// }
+
+ private static class ParDoBoundTranslatorBatch<IN, OUT> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<ParDo.Bound<IN, OUT>> {
+ private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundTranslatorBatch.class);
+
+ @Override
+ public void translateNode(ParDo.Bound<IN, OUT> transform, FlinkBatchTranslationContext context) {
+ DataSet<IN> inputDataSet = context.getInputDataSet(context.getInput(transform));
+
+ final DoFn<IN, OUT> doFn = transform.getFn();
+
+ TypeInformation<OUT> typeInformation = context.getTypeInfo(context.getOutput(transform));
+
+ FlinkDoFnFunction<IN, OUT> doFnWrapper = new FlinkDoFnFunction<>(doFn, context.getPipelineOptions());
+ MapPartitionOperator<IN, OUT> outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName());
+
+ transformSideInputs(transform.getSideInputs(), outputDataSet, context);
+
+ context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+ }
+ }
+
+ private static class ParDoBoundMultiTranslatorBatch<IN, OUT> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<ParDo.BoundMulti<IN, OUT>> {
+ private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundMultiTranslatorBatch.class);
+
+ @Override
+ public void translateNode(ParDo.BoundMulti<IN, OUT> transform, FlinkBatchTranslationContext context) {
+ DataSet<IN> inputDataSet = context.getInputDataSet(context.getInput(transform));
+
+ final DoFn<IN, OUT> doFn = transform.getFn();
+
+ Map<TupleTag<?>, PCollection<?>> outputs = context.getOutput(transform).getAll();
+
+ Map<TupleTag<?>, Integer> outputMap = Maps.newHashMap();
+ // put the main output at index 0, FlinkMultiOutputDoFnFunction also expects this
+ outputMap.put(transform.getMainOutputTag(), 0);
+ int count = 1;
+ for (TupleTag<?> tag: outputs.keySet()) {
+ if (!outputMap.containsKey(tag)) {
+ outputMap.put(tag, count++);
+ }
+ }
+
+ // collect all output Coders and create a UnionCoder for our tagged outputs
+ List<Coder<?>> outputCoders = Lists.newArrayList();
+ for (PCollection<?> coll: outputs.values()) {
+ outputCoders.add(coll.getCoder());
+ }
+
+ UnionCoder unionCoder = UnionCoder.of(outputCoders);
+
+ @SuppressWarnings("unchecked")
+ TypeInformation<RawUnionValue> typeInformation = new CoderTypeInformation<>(unionCoder);
+
+ @SuppressWarnings("unchecked")
+ FlinkMultiOutputDoFnFunction<IN, OUT> doFnWrapper = new FlinkMultiOutputDoFnFunction(doFn, context.getPipelineOptions(), outputMap);
+ MapPartitionOperator<IN, RawUnionValue> outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName());
+
+ transformSideInputs(transform.getSideInputs(), outputDataSet, context);
+
+ for (Map.Entry<TupleTag<?>, PCollection<?>> output: outputs.entrySet()) {
+ TypeInformation<Object> outputType = context.getTypeInfo(output.getValue());
+ int outputTag = outputMap.get(output.getKey());
+ FlinkMultiOutputPruningFunction<Object> pruningFunction = new FlinkMultiOutputPruningFunction<>(outputTag);
+ FlatMapOperator<RawUnionValue, Object> pruningOperator = new
+ FlatMapOperator<>(outputDataSet, outputType,
+ pruningFunction, output.getValue().getName());
+ context.setOutputDataSet(output.getValue(), pruningOperator);
+
+ }
+ }
+ }
+
+ private static class FlattenPCollectionTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Flatten.FlattenPCollectionList<T>> {
+
+ @Override
+ public void translateNode(Flatten.FlattenPCollectionList<T> transform, FlinkBatchTranslationContext context) {
+ List<PCollection<T>> allInputs = context.getInput(transform).getAll();
+ DataSet<T> result = null;
+ for(PCollection<T> collection : allInputs) {
+ DataSet<T> current = context.getInputDataSet(collection);
+ if (result == null) {
+ result = current;
+ } else {
+ result = result.union(current);
+ }
+ }
+ context.setOutputDataSet(context.getOutput(transform), result);
+ }
+ }
+
+ private static class CreatePCollectionViewTranslatorBatch<R, T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<View.CreatePCollectionView<R, T>> {
+ @Override
+ public void translateNode(View.CreatePCollectionView<R, T> transform, FlinkBatchTranslationContext context) {
+ DataSet<T> inputDataSet = context.getInputDataSet(context.getInput(transform));
+ PCollectionView<T> input = transform.apply(null);
+ context.setSideInputDataSet(input, inputDataSet);
+ }
+ }
+
+ private static class CreateTranslatorBatch<OUT> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Create.Values<OUT>> {
+
+ @Override
+ public void translateNode(Create.Values<OUT> transform, FlinkBatchTranslationContext context) {
+ TypeInformation<OUT> typeInformation = context.getOutputTypeInfo();
+ Iterable<OUT> elements = transform.getElements();
+
+ // we need to serialize the elements to byte arrays, since they might contain
+ // elements that are not serializable by Java serialization. We deserialize them
+ // in the FlatMap function using the Coder.
+
+ List<byte[]> serializedElements = Lists.newArrayList();
+ Coder<OUT> coder = context.getOutput(transform).getCoder();
+ for (OUT element: elements) {
+ ByteArrayOutputStream bao = new ByteArrayOutputStream();
+ try {
+ coder.encode(element, bao, Coder.Context.OUTER);
+ serializedElements.add(bao.toByteArray());
+ } catch (IOException e) {
+ throw new RuntimeException("Could not serialize Create elements using Coder: " + e);
+ }
+ }
+
+ DataSet<Integer> initDataSet = context.getExecutionEnvironment().fromElements(1);
+ FlinkCreateFunction<Integer, OUT> flatMapFunction = new FlinkCreateFunction<>(serializedElements, coder);
+ FlatMapOperator<Integer, OUT> outputDataSet = new FlatMapOperator<>(initDataSet, typeInformation, flatMapFunction, transform.getName());
+
+ context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+ }
+ }
+
+ private static void transformSideInputs(List<PCollectionView<?>> sideInputs,
+ MapPartitionOperator<?, ?> outputDataSet,
+ FlinkBatchTranslationContext context) {
+ // get corresponding Flink broadcast DataSets
+ for(PCollectionView<?> input : sideInputs) {
+ DataSet<?> broadcastSet = context.getSideInputDataSet(input);
+ outputDataSet.withBroadcastSet(broadcastSet, input.getTagInternal().getId());
+ }
+ }
// Disabled because it depends on a pending pull request to the DataFlowSDK
- /**
- * Special composite transform translator. Only called if the CoGroup is two dimensional.
- * @param <K>
- */
- private static class CoGroupByKeyTranslatorBatch<K, V1, V2> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<CoGroupByKey<K>> {
+ /**
+ * Special composite transform translator. Only called if the CoGroup is two dimensional.
+ * @param <K>
+ */
+ private static class CoGroupByKeyTranslatorBatch<K, V1, V2> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<CoGroupByKey<K>> {
- @Override
- public void translateNode(CoGroupByKey<K> transform, FlinkBatchTranslationContext context) {
- KeyedPCollectionTuple<K> input = context.getInput(transform);
+ @Override
+ public void translateNode(CoGroupByKey<K> transform, FlinkBatchTranslationContext context) {
+ KeyedPCollectionTuple<K> input = context.getInput(transform);
- CoGbkResultSchema schema = input.getCoGbkResultSchema();
- List<KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?>> keyedCollections = input.getKeyedCollections();
+ CoGbkResultSchema schema = input.getCoGbkResultSchema();
+ List<KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?>> keyedCollections = input.getKeyedCollections();
- KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?> taggedCollection1 = keyedCollections.get(0);
- KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?> taggedCollection2 = keyedCollections.get(1);
+ KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?> taggedCollection1 = keyedCollections.get(0);
+ KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?> taggedCollection2 = keyedCollections.get(1);
- TupleTag<?> tupleTag1 = taggedCollection1.getTupleTag();
- TupleTag<?> tupleTag2 = taggedCollection2.getTupleTag();
+ TupleTag<?> tupleTag1 = taggedCollection1.getTupleTag();
+ TupleTag<?> tupleTag2 = taggedCollection2.getTupleTag();
- PCollection<? extends KV<K, ?>> collection1 = taggedCollection1.getCollection();
- PCollection<? extends KV<K, ?>> collection2 = taggedCollection2.getCollection();
+ PCollection<? extends KV<K, ?>> collection1 = taggedCollection1.getCollection();
+ PCollection<? extends KV<K, ?>> collection2 = taggedCollection2.getCollection();
- DataSet<KV<K,V1>> inputDataSet1 = context.getInputDataSet(collection1);
- DataSet<KV<K,V2>> inputDataSet2 = context.getInputDataSet(collection2);
+ DataSet<KV<K,V1>> inputDataSet1 = context.getInputDataSet(collection1);
+ DataSet<KV<K,V2>> inputDataSet2 = context.getInputDataSet(collection2);
- TypeInformation<KV<K,CoGbkResult>> typeInfo = context.getOutputTypeInfo();
+ TypeInformation<KV<K,CoGbkResult>> typeInfo = context.getOutputTypeInfo();
- FlinkCoGroupKeyedListAggregator<K,V1,V2> aggregator = new FlinkCoGroupKeyedListAggregator<>(schema, tupleTag1, tupleTag2);
+ FlinkCoGroupKeyedListAggregator<K,V1,V2> aggregator = new FlinkCoGroupKeyedListAggregator<>(schema, tupleTag1, tupleTag2);
- Keys.ExpressionKeys<KV<K,V1>> keySelector1 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet1.getType());
- Keys.ExpressionKeys<KV<K,V2>> keySelector2 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet2.getType());
+ Keys.ExpressionKeys<KV<K,V1>> keySelector1 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet1.getType());
+ Keys.ExpressionKeys<KV<K,V2>> keySelector2 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet2.getType());
- DataSet<KV<K, CoGbkResult>> out = new CoGroupOperator<>(inputDataSet1, inputDataSet2,
- keySelector1, keySelector2,
- aggregator, typeInfo, null, transform.getName());
- context.setOutputDataSet(context.getOutput(transform), out);
- }
- }
+ DataSet<KV<K, CoGbkResult>> out = new CoGroupOperator<>(inputDataSet1, inputDataSet2,
+ keySelector1, keySelector2,
+ aggregator, typeInfo, null, transform.getName());
+ context.setOutputDataSet(context.getOutput(transform), out);
+ }
+ }
- // --------------------------------------------------------------------------------------------
- // Miscellaneous
- // --------------------------------------------------------------------------------------------
-
- private FlinkBatchTransformTranslators() {}
+ // --------------------------------------------------------------------------------------------
+ // Miscellaneous
+ // --------------------------------------------------------------------------------------------
+
+ private FlinkBatchTransformTranslators() {}
}
[41/50] [abbrv] incubator-beam git commit: [flink] convert tabs to 2
spaces
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java
index 1525c80..9f602fd 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java
@@ -25,103 +25,103 @@ import java.util.concurrent.TimeUnit;
public class StateCheckpointWriter {
- private final AbstractStateBackend.CheckpointStateOutputView output;
-
- public static StateCheckpointWriter create(AbstractStateBackend.CheckpointStateOutputView output) {
- return new StateCheckpointWriter(output);
- }
-
- private StateCheckpointWriter(AbstractStateBackend.CheckpointStateOutputView output) {
- this.output = output;
- }
-
- ///////// Creating the serialized versions of the different types of state held by dataflow ///////
-
- public StateCheckpointWriter addValueBuilder() throws IOException {
- validate();
- StateType.serialize(StateType.VALUE, this);
- return this;
- }
-
- public StateCheckpointWriter addWatermarkHoldsBuilder() throws IOException {
- validate();
- StateType.serialize(StateType.WATERMARK, this);
- return this;
- }
-
- public StateCheckpointWriter addListUpdatesBuilder() throws IOException {
- validate();
- StateType.serialize(StateType.LIST, this);
- return this;
- }
-
- public StateCheckpointWriter addAccumulatorBuilder() throws IOException {
- validate();
- StateType.serialize(StateType.ACCUMULATOR, this);
- return this;
- }
-
- ///////// Setting the tag for a given state element ///////
-
- public StateCheckpointWriter setTag(ByteString stateKey) throws IOException {
- return writeData(stateKey.toByteArray());
- }
-
- public StateCheckpointWriter setTag(String stateKey) throws IOException {
- output.writeUTF(stateKey);
- return this;
- }
-
-
- public <K> StateCheckpointWriter serializeKey(K key, CoderTypeSerializer<K> keySerializer) throws IOException {
- return serializeObject(key, keySerializer);
- }
-
- public <T> StateCheckpointWriter serializeObject(T object, CoderTypeSerializer<T> objectSerializer) throws IOException {
- objectSerializer.serialize(object, output);
- return this;
- }
-
- ///////// Write the actual serialized data //////////
-
- public StateCheckpointWriter setData(ByteString data) throws IOException {
- return writeData(data.toByteArray());
- }
-
- public StateCheckpointWriter setData(byte[] data) throws IOException {
- return writeData(data);
- }
-
- public StateCheckpointWriter setTimestamp(Instant timestamp) throws IOException {
- validate();
- output.writeLong(TimeUnit.MILLISECONDS.toMicros(timestamp.getMillis()));
- return this;
- }
-
- public StateCheckpointWriter writeInt(int number) throws IOException {
- validate();
- output.writeInt(number);
- return this;
- }
-
- public StateCheckpointWriter writeByte(byte b) throws IOException {
- validate();
- output.writeByte(b);
- return this;
- }
-
- ///////// Helper Methods ///////
-
- private StateCheckpointWriter writeData(byte[] data) throws IOException {
- validate();
- output.writeInt(data.length);
- output.write(data);
- return this;
- }
-
- private void validate() {
- if (this.output == null) {
- throw new RuntimeException("StateBackend not initialized yet.");
- }
- }
+ private final AbstractStateBackend.CheckpointStateOutputView output;
+
+ public static StateCheckpointWriter create(AbstractStateBackend.CheckpointStateOutputView output) {
+ return new StateCheckpointWriter(output);
+ }
+
+ private StateCheckpointWriter(AbstractStateBackend.CheckpointStateOutputView output) {
+ this.output = output;
+ }
+
+ ///////// Creating the serialized versions of the different types of state held by dataflow ///////
+
+ public StateCheckpointWriter addValueBuilder() throws IOException {
+ validate();
+ StateType.serialize(StateType.VALUE, this);
+ return this;
+ }
+
+ public StateCheckpointWriter addWatermarkHoldsBuilder() throws IOException {
+ validate();
+ StateType.serialize(StateType.WATERMARK, this);
+ return this;
+ }
+
+ public StateCheckpointWriter addListUpdatesBuilder() throws IOException {
+ validate();
+ StateType.serialize(StateType.LIST, this);
+ return this;
+ }
+
+ public StateCheckpointWriter addAccumulatorBuilder() throws IOException {
+ validate();
+ StateType.serialize(StateType.ACCUMULATOR, this);
+ return this;
+ }
+
+ ///////// Setting the tag for a given state element ///////
+
+ public StateCheckpointWriter setTag(ByteString stateKey) throws IOException {
+ return writeData(stateKey.toByteArray());
+ }
+
+ public StateCheckpointWriter setTag(String stateKey) throws IOException {
+ output.writeUTF(stateKey);
+ return this;
+ }
+
+
+ public <K> StateCheckpointWriter serializeKey(K key, CoderTypeSerializer<K> keySerializer) throws IOException {
+ return serializeObject(key, keySerializer);
+ }
+
+ public <T> StateCheckpointWriter serializeObject(T object, CoderTypeSerializer<T> objectSerializer) throws IOException {
+ objectSerializer.serialize(object, output);
+ return this;
+ }
+
+ ///////// Write the actual serialized data //////////
+
+ public StateCheckpointWriter setData(ByteString data) throws IOException {
+ return writeData(data.toByteArray());
+ }
+
+ public StateCheckpointWriter setData(byte[] data) throws IOException {
+ return writeData(data);
+ }
+
+ public StateCheckpointWriter setTimestamp(Instant timestamp) throws IOException {
+ validate();
+ output.writeLong(TimeUnit.MILLISECONDS.toMicros(timestamp.getMillis()));
+ return this;
+ }
+
+ public StateCheckpointWriter writeInt(int number) throws IOException {
+ validate();
+ output.writeInt(number);
+ return this;
+ }
+
+ public StateCheckpointWriter writeByte(byte b) throws IOException {
+ validate();
+ output.writeByte(b);
+ return this;
+ }
+
+ ///////// Helper Methods ///////
+
+ private StateCheckpointWriter writeData(byte[] data) throws IOException {
+ validate();
+ output.writeInt(data.length);
+ output.write(data);
+ return this;
+ }
+
+ private void validate() {
+ if (this.output == null) {
+ throw new RuntimeException("StateBackend not initialized yet.");
+ }
+ }
}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java
index aa049ef..9e2c9f8 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java
@@ -23,49 +23,49 @@ import java.io.IOException;
* */
public enum StateType {
- VALUE(0),
+ VALUE(0),
- WATERMARK(1),
+ WATERMARK(1),
- LIST(2),
+ LIST(2),
- ACCUMULATOR(3);
+ ACCUMULATOR(3);
- private final int numVal;
+ private final int numVal;
- StateType(int value) {
- this.numVal = value;
- }
+ StateType(int value) {
+ this.numVal = value;
+ }
- public static void serialize(StateType type, StateCheckpointWriter output) throws IOException {
- if (output == null) {
- throw new IllegalArgumentException("Cannot write to a null output.");
- }
+ public static void serialize(StateType type, StateCheckpointWriter output) throws IOException {
+ if (output == null) {
+ throw new IllegalArgumentException("Cannot write to a null output.");
+ }
- if(type.numVal < 0 || type.numVal > 3) {
- throw new RuntimeException("Unknown State Type " + type + ".");
- }
+ if(type.numVal < 0 || type.numVal > 3) {
+ throw new RuntimeException("Unknown State Type " + type + ".");
+ }
- output.writeByte((byte) type.numVal);
- }
+ output.writeByte((byte) type.numVal);
+ }
- public static StateType deserialize(StateCheckpointReader input) throws IOException {
- if (input == null) {
- throw new IllegalArgumentException("Cannot read from a null input.");
- }
+ public static StateType deserialize(StateCheckpointReader input) throws IOException {
+ if (input == null) {
+ throw new IllegalArgumentException("Cannot read from a null input.");
+ }
- int typeInt = (int) input.getByte();
- if(typeInt < 0 || typeInt > 3) {
- throw new RuntimeException("Unknown State Type " + typeInt + ".");
- }
+ int typeInt = (int) input.getByte();
+ if(typeInt < 0 || typeInt > 3) {
+ throw new RuntimeException("Unknown State Type " + typeInt + ".");
+ }
- StateType resultType = null;
- for(StateType st: values()) {
- if(st.numVal == typeInt) {
- resultType = st;
- break;
- }
- }
- return resultType;
- }
+ StateType resultType = null;
+ for(StateType st: values()) {
+ if(st.numVal == typeInt) {
+ resultType = st;
+ break;
+ }
+ }
+ return resultType;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java
index ce53d44..3272975 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java
@@ -30,70 +30,70 @@ import org.apache.flink.test.util.JavaProgramTestBase;
public class AvroITCase extends JavaProgramTestBase {
- protected String resultPath;
- protected String tmpPath;
-
- public AvroITCase(){
- }
-
- static final String[] EXPECTED_RESULT = new String[] {
- "Joe red 3",
- "Mary blue 4",
- "Mark green 1",
- "Julia purple 5"
- };
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- tmpPath = getTempDirPath("tmp");
-
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
- runProgram(tmpPath, resultPath);
- }
-
- private static void runProgram(String tmpPath, String resultPath) {
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- p
- .apply(Create.of(
- new User("Joe", 3, "red"),
- new User("Mary", 4, "blue"),
- new User("Mark", 1, "green"),
- new User("Julia", 5, "purple"))
- .withCoder(AvroCoder.of(User.class)))
-
- .apply(AvroIO.Write.to(tmpPath)
- .withSchema(User.class));
-
- p.run();
-
- p = FlinkTestPipeline.createForBatch();
-
- p
- .apply(AvroIO.Read.from(tmpPath).withSchema(User.class).withoutValidation())
-
- .apply(ParDo.of(new DoFn<User, String>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- User u = c.element();
- String result = u.getName() + " " + u.getFavoriteColor() + " " + u.getFavoriteNumber();
- c.output(result);
- }
- }))
-
- .apply(TextIO.Write.to(resultPath));
-
- p.run();
- }
+ protected String resultPath;
+ protected String tmpPath;
+
+ public AvroITCase(){
+ }
+
+ static final String[] EXPECTED_RESULT = new String[] {
+ "Joe red 3",
+ "Mary blue 4",
+ "Mark green 1",
+ "Julia purple 5"
+ };
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ tmpPath = getTempDirPath("tmp");
+
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+ runProgram(tmpPath, resultPath);
+ }
+
+ private static void runProgram(String tmpPath, String resultPath) {
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ p
+ .apply(Create.of(
+ new User("Joe", 3, "red"),
+ new User("Mary", 4, "blue"),
+ new User("Mark", 1, "green"),
+ new User("Julia", 5, "purple"))
+ .withCoder(AvroCoder.of(User.class)))
+
+ .apply(AvroIO.Write.to(tmpPath)
+ .withSchema(User.class));
+
+ p.run();
+
+ p = FlinkTestPipeline.createForBatch();
+
+ p
+ .apply(AvroIO.Read.from(tmpPath).withSchema(User.class).withoutValidation())
+
+ .apply(ParDo.of(new DoFn<User, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ User u = c.element();
+ String result = u.getName() + " " + u.getFavoriteColor() + " " + u.getFavoriteNumber();
+ c.output(result);
+ }
+ }))
+
+ .apply(TextIO.Write.to(resultPath));
+
+ p.run();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java
index 928388c..e65e497 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java
@@ -26,47 +26,47 @@ import org.apache.flink.test.util.JavaProgramTestBase;
public class FlattenizeITCase extends JavaProgramTestBase {
- private String resultPath;
- private String resultPath2;
+ private String resultPath;
+ private String resultPath2;
- private static final String[] words = {"hello", "this", "is", "a", "DataSet!"};
- private static final String[] words2 = {"hello", "this", "is", "another", "DataSet!"};
- private static final String[] words3 = {"hello", "this", "is", "yet", "another", "DataSet!"};
+ private static final String[] words = {"hello", "this", "is", "a", "DataSet!"};
+ private static final String[] words2 = {"hello", "this", "is", "another", "DataSet!"};
+ private static final String[] words3 = {"hello", "this", "is", "yet", "another", "DataSet!"};
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- resultPath2 = getTempDirPath("result2");
- }
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ resultPath2 = getTempDirPath("result2");
+ }
- @Override
- protected void postSubmit() throws Exception {
- String join = Joiner.on('\n').join(words);
- String join2 = Joiner.on('\n').join(words2);
- String join3 = Joiner.on('\n').join(words3);
- compareResultsByLinesInMemory(join + "\n" + join2, resultPath);
- compareResultsByLinesInMemory(join + "\n" + join2 + "\n" + join3, resultPath2);
- }
+ @Override
+ protected void postSubmit() throws Exception {
+ String join = Joiner.on('\n').join(words);
+ String join2 = Joiner.on('\n').join(words2);
+ String join3 = Joiner.on('\n').join(words3);
+ compareResultsByLinesInMemory(join + "\n" + join2, resultPath);
+ compareResultsByLinesInMemory(join + "\n" + join2 + "\n" + join3, resultPath2);
+ }
- @Override
- protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.createForBatch();
+ @Override
+ protected void testProgram() throws Exception {
+ Pipeline p = FlinkTestPipeline.createForBatch();
- PCollection<String> p1 = p.apply(Create.of(words));
- PCollection<String> p2 = p.apply(Create.of(words2));
+ PCollection<String> p1 = p.apply(Create.of(words));
+ PCollection<String> p2 = p.apply(Create.of(words2));
- PCollectionList<String> list = PCollectionList.of(p1).and(p2);
+ PCollectionList<String> list = PCollectionList.of(p1).and(p2);
- list.apply(Flatten.<String>pCollections()).apply(TextIO.Write.to(resultPath));
+ list.apply(Flatten.<String>pCollections()).apply(TextIO.Write.to(resultPath));
- PCollection<String> p3 = p.apply(Create.of(words3));
+ PCollection<String> p3 = p.apply(Create.of(words3));
- PCollectionList<String> list2 = list.and(p3);
+ PCollectionList<String> list2 = list.and(p3);
- list2.apply(Flatten.<String>pCollections()).apply(TextIO.Write.to(resultPath2));
+ list2.apply(Flatten.<String>pCollections()).apply(TextIO.Write.to(resultPath2));
- p.run();
- }
+ p.run();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
index 59c3b69..578e0e1 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
@@ -26,45 +26,45 @@ import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
*/
public class FlinkTestPipeline extends Pipeline {
- /**
- * Creates and returns a new test pipeline for batch execution.
- *
- * <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
- * {@link Pipeline#run} to execute the pipeline and check the tests.
- */
- public static FlinkTestPipeline createForBatch() {
- return create(false);
- }
+ /**
+ * Creates and returns a new test pipeline for batch execution.
+ *
+ * <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
+ * {@link Pipeline#run} to execute the pipeline and check the tests.
+ */
+ public static FlinkTestPipeline createForBatch() {
+ return create(false);
+ }
- /**
- * Creates and returns a new test pipeline for streaming execution.
- *
- * <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
- * {@link Pipeline#run} to execute the pipeline and check the tests.
- *
- * @return The Test Pipeline
- */
- public static FlinkTestPipeline createForStreaming() {
- return create(true);
- }
+ /**
+ * Creates and returns a new test pipeline for streaming execution.
+ *
+ * <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
+ * {@link Pipeline#run} to execute the pipeline and check the tests.
+ *
+ * @return The Test Pipeline
+ */
+ public static FlinkTestPipeline createForStreaming() {
+ return create(true);
+ }
- /**
- * Creates and returns a new test pipeline for streaming or batch execution.
- *
- * <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
- * {@link Pipeline#run} to execute the pipeline and check the tests.
- *
- * @param streaming <code>True</code> for streaming mode, <code>False</code> for batch.
- * @return The Test Pipeline.
- */
- private static FlinkTestPipeline create(boolean streaming) {
- FlinkPipelineRunner flinkRunner = FlinkPipelineRunner.createForTest(streaming);
- return new FlinkTestPipeline(flinkRunner, flinkRunner.getPipelineOptions());
- }
+ /**
+ * Creates and returns a new test pipeline for streaming or batch execution.
+ *
+ * <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
+ * {@link Pipeline#run} to execute the pipeline and check the tests.
+ *
+ * @param streaming <code>True</code> for streaming mode, <code>False</code> for batch.
+ * @return The Test Pipeline.
+ */
+ private static FlinkTestPipeline create(boolean streaming) {
+ FlinkPipelineRunner flinkRunner = FlinkPipelineRunner.createForTest(streaming);
+ return new FlinkTestPipeline(flinkRunner, flinkRunner.getPipelineOptions());
+ }
- private FlinkTestPipeline(PipelineRunner<? extends PipelineResult> runner,
- PipelineOptions options) {
- super(runner, options);
- }
+ private FlinkTestPipeline(PipelineRunner<? extends PipelineResult> runner,
+ PipelineOptions options) {
+ super(runner, options);
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java
index af0f217..28861ea 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java
@@ -34,66 +34,66 @@ import java.util.List;
*/
public class JoinExamplesITCase extends JavaProgramTestBase {
- protected String resultPath;
-
- public JoinExamplesITCase(){
- }
-
- private static final TableRow row1 = new TableRow()
- .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212")
- .set("Actor1Name", "BANGKOK").set("SOURCEURL", "http://cnn.com");
- private static final TableRow row2 = new TableRow()
- .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212")
- .set("Actor1Name", "LAOS").set("SOURCEURL", "http://www.chicagotribune.com");
- private static final TableRow row3 = new TableRow()
- .set("ActionGeo_CountryCode", "BE").set("SQLDATE", "20141213")
- .set("Actor1Name", "AFGHANISTAN").set("SOURCEURL", "http://cnn.com");
- static final TableRow[] EVENTS = new TableRow[] {
- row1, row2, row3
- };
- static final List<TableRow> EVENT_ARRAY = Arrays.asList(EVENTS);
-
- private static final TableRow cc1 = new TableRow()
- .set("FIPSCC", "VM").set("HumanName", "Vietnam");
- private static final TableRow cc2 = new TableRow()
- .set("FIPSCC", "BE").set("HumanName", "Belgium");
- static final TableRow[] CCS = new TableRow[] {
- cc1, cc2
- };
- static final List<TableRow> CC_ARRAY = Arrays.asList(CCS);
-
- static final String[] JOINED_EVENTS = new String[] {
- "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: LAOS, "
- + "url: http://www.chicagotribune.com",
- "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: BANGKOK, "
- + "url: http://cnn.com",
- "Country code: BE, Country name: Belgium, Event info: Date: 20141213, Actor1: AFGHANISTAN, "
- + "url: http://cnn.com"
- };
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(JOINED_EVENTS), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
-
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- PCollection<TableRow> input1 = p.apply(Create.of(EVENT_ARRAY));
- PCollection<TableRow> input2 = p.apply(Create.of(CC_ARRAY));
-
- PCollection<String> output = JoinExamples.joinEvents(input1, input2);
-
- output.apply(TextIO.Write.to(resultPath));
-
- p.run();
- }
+ protected String resultPath;
+
+ public JoinExamplesITCase(){
+ }
+
+ private static final TableRow row1 = new TableRow()
+ .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212")
+ .set("Actor1Name", "BANGKOK").set("SOURCEURL", "http://cnn.com");
+ private static final TableRow row2 = new TableRow()
+ .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212")
+ .set("Actor1Name", "LAOS").set("SOURCEURL", "http://www.chicagotribune.com");
+ private static final TableRow row3 = new TableRow()
+ .set("ActionGeo_CountryCode", "BE").set("SQLDATE", "20141213")
+ .set("Actor1Name", "AFGHANISTAN").set("SOURCEURL", "http://cnn.com");
+ static final TableRow[] EVENTS = new TableRow[] {
+ row1, row2, row3
+ };
+ static final List<TableRow> EVENT_ARRAY = Arrays.asList(EVENTS);
+
+ private static final TableRow cc1 = new TableRow()
+ .set("FIPSCC", "VM").set("HumanName", "Vietnam");
+ private static final TableRow cc2 = new TableRow()
+ .set("FIPSCC", "BE").set("HumanName", "Belgium");
+ static final TableRow[] CCS = new TableRow[] {
+ cc1, cc2
+ };
+ static final List<TableRow> CC_ARRAY = Arrays.asList(CCS);
+
+ static final String[] JOINED_EVENTS = new String[] {
+ "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: LAOS, "
+ + "url: http://www.chicagotribune.com",
+ "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: BANGKOK, "
+ + "url: http://cnn.com",
+ "Country code: BE, Country name: Belgium, Event info: Date: 20141213, Actor1: AFGHANISTAN, "
+ + "url: http://cnn.com"
+ };
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(JOINED_EVENTS), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ PCollection<TableRow> input1 = p.apply(Create.of(EVENT_ARRAY));
+ PCollection<TableRow> input2 = p.apply(Create.of(CC_ARRAY));
+
+ PCollection<String> output = JoinExamples.joinEvents(input1, input2);
+
+ output.apply(TextIO.Write.to(resultPath));
+
+ p.run();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java
index 35f2eaf..d1652e7 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java
@@ -27,37 +27,37 @@ import java.io.Serializable;
public class MaybeEmptyTestITCase extends JavaProgramTestBase implements Serializable {
- protected String resultPath;
-
- protected final String expected = "test";
-
- public MaybeEmptyTestITCase() {
- }
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(expected, resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
-
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- p.apply(Create.of((Void) null)).setCoder(VoidCoder.of())
- .apply(ParDo.of(
- new DoFn<Void, String>() {
- @Override
- public void processElement(DoFn<Void, String>.ProcessContext c) {
- c.output(expected);
- }
- })).apply(TextIO.Write.to(resultPath));
- p.run();
- }
+ protected String resultPath;
+
+ protected final String expected = "test";
+
+ public MaybeEmptyTestITCase() {
+ }
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(expected, resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ p.apply(Create.of((Void) null)).setCoder(VoidCoder.of())
+ .apply(ParDo.of(
+ new DoFn<Void, String>() {
+ @Override
+ public void processElement(DoFn<Void, String>.ProcessContext c) {
+ c.output(expected);
+ }
+ })).apply(TextIO.Write.to(resultPath));
+ p.run();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java
index ccdbbf9..d8087d6 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java
@@ -31,68 +31,68 @@ import java.io.Serializable;
public class ParDoMultiOutputITCase extends JavaProgramTestBase implements Serializable {
- private String resultPath;
-
- private static String[] expectedWords = {"MAAA", "MAAFOOO"};
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on("\n").join(expectedWords), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- PCollection<String> words = p.apply(Create.of("Hello", "Whatupmyman", "hey", "SPECIALthere", "MAAA", "MAAFOOO"));
-
- // Select words whose length is below a cut off,
- // plus the lengths of words that are above the cut off.
- // Also select words starting with "MARKER".
- final int wordLengthCutOff = 3;
- // Create tags to use for the main and side outputs.
- final TupleTag<String> wordsBelowCutOffTag = new TupleTag<String>(){};
- final TupleTag<Integer> wordLengthsAboveCutOffTag = new TupleTag<Integer>(){};
- final TupleTag<String> markedWordsTag = new TupleTag<String>(){};
-
- PCollectionTuple results =
- words.apply(ParDo
- .withOutputTags(wordsBelowCutOffTag, TupleTagList.of(wordLengthsAboveCutOffTag)
- .and(markedWordsTag))
- .of(new DoFn<String, String>() {
- final TupleTag<String> specialWordsTag = new TupleTag<String>() {
- };
-
- public void processElement(ProcessContext c) {
- String word = c.element();
- if (word.length() <= wordLengthCutOff) {
- c.output(word);
- } else {
- c.sideOutput(wordLengthsAboveCutOffTag, word.length());
- }
- if (word.startsWith("MAA")) {
- c.sideOutput(markedWordsTag, word);
- }
-
- if (word.startsWith("SPECIAL")) {
- c.sideOutput(specialWordsTag, word);
- }
- }
- }));
-
- // Extract the PCollection results, by tag.
- PCollection<String> wordsBelowCutOff = results.get(wordsBelowCutOffTag);
- PCollection<Integer> wordLengthsAboveCutOff = results.get
- (wordLengthsAboveCutOffTag);
- PCollection<String> markedWords = results.get(markedWordsTag);
-
- markedWords.apply(TextIO.Write.to(resultPath));
-
- p.run();
- }
+ private String resultPath;
+
+ private static String[] expectedWords = {"MAAA", "MAAFOOO"};
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on("\n").join(expectedWords), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ PCollection<String> words = p.apply(Create.of("Hello", "Whatupmyman", "hey", "SPECIALthere", "MAAA", "MAAFOOO"));
+
+ // Select words whose length is below a cut off,
+ // plus the lengths of words that are above the cut off.
+ // Also select words starting with "MARKER".
+ final int wordLengthCutOff = 3;
+ // Create tags to use for the main and side outputs.
+ final TupleTag<String> wordsBelowCutOffTag = new TupleTag<String>(){};
+ final TupleTag<Integer> wordLengthsAboveCutOffTag = new TupleTag<Integer>(){};
+ final TupleTag<String> markedWordsTag = new TupleTag<String>(){};
+
+ PCollectionTuple results =
+ words.apply(ParDo
+ .withOutputTags(wordsBelowCutOffTag, TupleTagList.of(wordLengthsAboveCutOffTag)
+ .and(markedWordsTag))
+ .of(new DoFn<String, String>() {
+ final TupleTag<String> specialWordsTag = new TupleTag<String>() {
+ };
+
+ public void processElement(ProcessContext c) {
+ String word = c.element();
+ if (word.length() <= wordLengthCutOff) {
+ c.output(word);
+ } else {
+ c.sideOutput(wordLengthsAboveCutOffTag, word.length());
+ }
+ if (word.startsWith("MAA")) {
+ c.sideOutput(markedWordsTag, word);
+ }
+
+ if (word.startsWith("SPECIAL")) {
+ c.sideOutput(specialWordsTag, word);
+ }
+ }
+ }));
+
+ // Extract the PCollection results, by tag.
+ PCollection<String> wordsBelowCutOff = results.get(wordsBelowCutOffTag);
+ PCollection<Integer> wordLengthsAboveCutOff = results.get
+ (wordLengthsAboveCutOffTag);
+ PCollection<String> markedWords = results.get(markedWordsTag);
+
+ markedWords.apply(TextIO.Write.to(resultPath));
+
+ p.run();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java
index 3569a78..5a46359 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java
@@ -36,128 +36,128 @@ import java.util.List;
public class ReadSourceITCase extends JavaProgramTestBase {
- protected String resultPath;
-
- public ReadSourceITCase(){
- }
-
- static final String[] EXPECTED_RESULT = new String[] {
- "1", "2", "3", "4", "5", "6", "7", "8", "9"};
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
- runProgram(resultPath);
- }
-
- private static void runProgram(String resultPath) {
-
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- PCollection<String> result = p
- .apply(Read.from(new ReadSource(1, 10)))
- .apply(ParDo.of(new DoFn<Integer, String>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- c.output(c.element().toString());
- }
- }));
-
- result.apply(TextIO.Write.to(resultPath));
- p.run();
- }
-
-
- private static class ReadSource extends BoundedSource<Integer> {
- final int from;
- final int to;
-
- ReadSource(int from, int to) {
- this.from = from;
- this.to = to;
- }
-
- @Override
- public List<ReadSource> splitIntoBundles(long desiredShardSizeBytes, PipelineOptions options)
- throws Exception {
- List<ReadSource> res = new ArrayList<>();
- FlinkPipelineOptions flinkOptions = options.as(FlinkPipelineOptions.class);
- int numWorkers = flinkOptions.getParallelism();
- Preconditions.checkArgument(numWorkers > 0, "Number of workers should be larger than 0.");
-
- float step = 1.0f * (to - from) / numWorkers;
- for (int i = 0; i < numWorkers; ++i) {
- res.add(new ReadSource(Math.round(from + i * step), Math.round(from + (i + 1) * step)));
- }
- return res;
- }
-
- @Override
- public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
- return 8 * (to - from);
- }
-
- @Override
- public boolean producesSortedKeys(PipelineOptions options) throws Exception {
- return true;
- }
-
- @Override
- public BoundedReader<Integer> createReader(PipelineOptions options) throws IOException {
- return new RangeReader(this);
- }
-
- @Override
- public void validate() {}
-
- @Override
- public Coder<Integer> getDefaultOutputCoder() {
- return BigEndianIntegerCoder.of();
- }
-
- private class RangeReader extends BoundedReader<Integer> {
- private int current;
-
- public RangeReader(ReadSource source) {
- this.current = source.from - 1;
- }
-
- @Override
- public boolean start() throws IOException {
- return true;
- }
-
- @Override
- public boolean advance() throws IOException {
- current++;
- return (current < to);
- }
-
- @Override
- public Integer getCurrent() {
- return current;
- }
-
- @Override
- public void close() throws IOException {
- // Nothing
- }
-
- @Override
- public BoundedSource<Integer> getCurrentSource() {
- return ReadSource.this;
- }
- }
- }
+ protected String resultPath;
+
+ public ReadSourceITCase(){
+ }
+
+ static final String[] EXPECTED_RESULT = new String[] {
+ "1", "2", "3", "4", "5", "6", "7", "8", "9"};
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+ runProgram(resultPath);
+ }
+
+ private static void runProgram(String resultPath) {
+
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ PCollection<String> result = p
+ .apply(Read.from(new ReadSource(1, 10)))
+ .apply(ParDo.of(new DoFn<Integer, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ c.output(c.element().toString());
+ }
+ }));
+
+ result.apply(TextIO.Write.to(resultPath));
+ p.run();
+ }
+
+
+ private static class ReadSource extends BoundedSource<Integer> {
+ final int from;
+ final int to;
+
+ ReadSource(int from, int to) {
+ this.from = from;
+ this.to = to;
+ }
+
+ @Override
+ public List<ReadSource> splitIntoBundles(long desiredShardSizeBytes, PipelineOptions options)
+ throws Exception {
+ List<ReadSource> res = new ArrayList<>();
+ FlinkPipelineOptions flinkOptions = options.as(FlinkPipelineOptions.class);
+ int numWorkers = flinkOptions.getParallelism();
+ Preconditions.checkArgument(numWorkers > 0, "Number of workers should be larger than 0.");
+
+ float step = 1.0f * (to - from) / numWorkers;
+ for (int i = 0; i < numWorkers; ++i) {
+ res.add(new ReadSource(Math.round(from + i * step), Math.round(from + (i + 1) * step)));
+ }
+ return res;
+ }
+
+ @Override
+ public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
+ return 8 * (to - from);
+ }
+
+ @Override
+ public boolean producesSortedKeys(PipelineOptions options) throws Exception {
+ return true;
+ }
+
+ @Override
+ public BoundedReader<Integer> createReader(PipelineOptions options) throws IOException {
+ return new RangeReader(this);
+ }
+
+ @Override
+ public void validate() {}
+
+ @Override
+ public Coder<Integer> getDefaultOutputCoder() {
+ return BigEndianIntegerCoder.of();
+ }
+
+ private class RangeReader extends BoundedReader<Integer> {
+ private int current;
+
+ public RangeReader(ReadSource source) {
+ this.current = source.from - 1;
+ }
+
+ @Override
+ public boolean start() throws IOException {
+ return true;
+ }
+
+ @Override
+ public boolean advance() throws IOException {
+ current++;
+ return (current < to);
+ }
+
+ @Override
+ public Integer getCurrent() {
+ return current;
+ }
+
+ @Override
+ public void close() throws IOException {
+ // Nothing
+ }
+
+ @Override
+ public BoundedSource<Integer> getCurrentSource() {
+ return ReadSource.this;
+ }
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java
index db794f7..615f194 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java
@@ -30,39 +30,39 @@ import java.util.List;
public class RemoveDuplicatesEmptyITCase extends JavaProgramTestBase {
- protected String resultPath;
+ protected String resultPath;
- public RemoveDuplicatesEmptyITCase(){
- }
+ public RemoveDuplicatesEmptyITCase(){
+ }
- static final String[] EXPECTED_RESULT = new String[] {};
+ static final String[] EXPECTED_RESULT = new String[] {};
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
- @Override
- protected void testProgram() throws Exception {
+ @Override
+ protected void testProgram() throws Exception {
- List<String> strings = Collections.emptyList();
+ List<String> strings = Collections.emptyList();
- Pipeline p = FlinkTestPipeline.createForBatch();
+ Pipeline p = FlinkTestPipeline.createForBatch();
- PCollection<String> input =
- p.apply(Create.of(strings))
- .setCoder(StringUtf8Coder.of());
+ PCollection<String> input =
+ p.apply(Create.of(strings))
+ .setCoder(StringUtf8Coder.of());
- PCollection<String> output =
- input.apply(RemoveDuplicates.<String>create());
+ PCollection<String> output =
+ input.apply(RemoveDuplicates.<String>create());
- output.apply(TextIO.Write.to(resultPath));
- p.run();
- }
+ output.apply(TextIO.Write.to(resultPath));
+ p.run();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java
index 04e06b8..8c19f2c 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java
@@ -30,40 +30,40 @@ import java.util.List;
public class RemoveDuplicatesITCase extends JavaProgramTestBase {
- protected String resultPath;
+ protected String resultPath;
- public RemoveDuplicatesITCase(){
- }
+ public RemoveDuplicatesITCase(){
+ }
- static final String[] EXPECTED_RESULT = new String[] {
- "k1", "k5", "k2", "k3"};
+ static final String[] EXPECTED_RESULT = new String[] {
+ "k1", "k5", "k2", "k3"};
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
- @Override
- protected void testProgram() throws Exception {
+ @Override
+ protected void testProgram() throws Exception {
- List<String> strings = Arrays.asList("k1", "k5", "k5", "k2", "k1", "k2", "k3");
+ List<String> strings = Arrays.asList("k1", "k5", "k5", "k2", "k1", "k2", "k3");
- Pipeline p = FlinkTestPipeline.createForBatch();
+ Pipeline p = FlinkTestPipeline.createForBatch();
- PCollection<String> input =
- p.apply(Create.of(strings))
- .setCoder(StringUtf8Coder.of());
+ PCollection<String> input =
+ p.apply(Create.of(strings))
+ .setCoder(StringUtf8Coder.of());
- PCollection<String> output =
- input.apply(RemoveDuplicates.<String>create());
+ PCollection<String> output =
+ input.apply(RemoveDuplicates.<String>create());
- output.apply(TextIO.Write.to(resultPath));
- p.run();
- }
+ output.apply(TextIO.Write.to(resultPath));
+ p.run();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java
index ee8843c..7c3d6f9 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java
@@ -28,40 +28,40 @@ import java.io.Serializable;
public class SideInputITCase extends JavaProgramTestBase implements Serializable {
- private static final String expected = "Hello!";
+ private static final String expected = "Hello!";
- protected String resultPath;
+ protected String resultPath;
- @Override
- protected void testProgram() throws Exception {
+ @Override
+ protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.createForBatch();
+ Pipeline p = FlinkTestPipeline.createForBatch();
- final PCollectionView<String> sidesInput = p
- .apply(Create.of(expected))
- .apply(View.<String>asSingleton());
+ final PCollectionView<String> sidesInput = p
+ .apply(Create.of(expected))
+ .apply(View.<String>asSingleton());
- p.apply(Create.of("bli"))
- .apply(ParDo.of(new DoFn<String, String>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- String s = c.sideInput(sidesInput);
- c.output(s);
- }
- }).withSideInputs(sidesInput)).apply(TextIO.Write.to(resultPath));
+ p.apply(Create.of("bli"))
+ .apply(ParDo.of(new DoFn<String, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ String s = c.sideInput(sidesInput);
+ c.output(s);
+ }
+ }).withSideInputs(sidesInput)).apply(TextIO.Write.to(resultPath));
- p.run();
- }
+ p.run();
+ }
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(expected, resultPath);
- }
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(expected, resultPath);
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java
index 07c1294..715d0be 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java
@@ -32,45 +32,45 @@ import java.net.URI;
public class TfIdfITCase extends JavaProgramTestBase {
- protected String resultPath;
+ protected String resultPath;
- public TfIdfITCase(){
- }
+ public TfIdfITCase(){
+ }
- static final String[] EXPECTED_RESULT = new String[] {
- "a", "m", "n", "b", "c", "d"};
+ static final String[] EXPECTED_RESULT = new String[] {
+ "a", "m", "n", "b", "c", "d"};
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
- @Override
- protected void testProgram() throws Exception {
+ @Override
+ protected void testProgram() throws Exception {
- Pipeline pipeline = FlinkTestPipeline.createForBatch();
+ Pipeline pipeline = FlinkTestPipeline.createForBatch();
- pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
+ pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
- PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf = pipeline
- .apply(Create.of(
- KV.of(new URI("x"), "a b c d"),
- KV.of(new URI("y"), "a b c"),
- KV.of(new URI("z"), "a m n")))
- .apply(new TFIDF.ComputeTfIdf());
+ PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf = pipeline
+ .apply(Create.of(
+ KV.of(new URI("x"), "a b c d"),
+ KV.of(new URI("y"), "a b c"),
+ KV.of(new URI("z"), "a m n")))
+ .apply(new TFIDF.ComputeTfIdf());
- PCollection<String> words = wordToUriAndTfIdf
- .apply(Keys.<String>create())
- .apply(RemoveDuplicates.<String>create());
+ PCollection<String> words = wordToUriAndTfIdf
+ .apply(Keys.<String>create())
+ .apply(RemoveDuplicates.<String>create());
- words.apply(TextIO.Write.to(resultPath));
+ words.apply(TextIO.Write.to(resultPath));
- pipeline.run();
- }
+ pipeline.run();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java
index 9188097..f1a2454 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java
@@ -32,43 +32,43 @@ import java.util.List;
public class WordCountITCase extends JavaProgramTestBase {
- protected String resultPath;
+ protected String resultPath;
- public WordCountITCase(){
- }
+ public WordCountITCase(){
+ }
- static final String[] WORDS_ARRAY = new String[] {
- "hi there", "hi", "hi sue bob",
- "hi sue", "", "bob hi"};
+ static final String[] WORDS_ARRAY = new String[] {
+ "hi there", "hi", "hi sue bob",
+ "hi sue", "", "bob hi"};
- static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
+ static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
- static final String[] COUNTS_ARRAY = new String[] {
- "hi: 5", "there: 1", "sue: 2", "bob: 2"};
+ static final String[] COUNTS_ARRAY = new String[] {
+ "hi: 5", "there: 1", "sue: 2", "bob: 2"};
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(COUNTS_ARRAY), resultPath);
- }
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(COUNTS_ARRAY), resultPath);
+ }
- @Override
- protected void testProgram() throws Exception {
+ @Override
+ protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.createForBatch();
+ Pipeline p = FlinkTestPipeline.createForBatch();
- PCollection<String> input = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of());
+ PCollection<String> input = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of());
- input
- .apply(new WordCount.CountWords())
- .apply(MapElements.via(new WordCount.FormatAsTextFn()))
- .apply(TextIO.Write.to(resultPath));
+ input
+ .apply(new WordCount.CountWords())
+ .apply(MapElements.via(new WordCount.FormatAsTextFn()))
+ .apply(TextIO.Write.to(resultPath));
- p.run();
- }
+ p.run();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java
index ccc52c4..1cac036 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java
@@ -33,104 +33,104 @@ import org.apache.flink.test.util.JavaProgramTestBase;
public class WordCountJoin2ITCase extends JavaProgramTestBase {
- static final String[] WORDS_1 = new String[] {
- "hi there", "hi", "hi sue bob",
- "hi sue", "", "bob hi"};
-
- static final String[] WORDS_2 = new String[] {
- "hi tim", "beauty", "hooray sue bob",
- "hi there", "", "please say hi"};
-
- static final String[] RESULTS = new String[] {
- "beauty -> Tag1: Tag2: 1",
- "bob -> Tag1: 2 Tag2: 1",
- "hi -> Tag1: 5 Tag2: 3",
- "hooray -> Tag1: Tag2: 1",
- "please -> Tag1: Tag2: 1",
- "say -> Tag1: Tag2: 1",
- "sue -> Tag1: 2 Tag2: 1",
- "there -> Tag1: 1 Tag2: 1",
- "tim -> Tag1: Tag2: 1"
- };
-
- static final TupleTag<Long> tag1 = new TupleTag<>("Tag1");
- static final TupleTag<Long> tag2 = new TupleTag<>("Tag2");
-
- protected String resultPath;
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(RESULTS), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- /* Create two PCollections and join them */
- PCollection<KV<String,Long>> occurences1 = p.apply(Create.of(WORDS_1))
- .apply(ParDo.of(new ExtractWordsFn()))
- .apply(Count.<String>perElement());
-
- PCollection<KV<String,Long>> occurences2 = p.apply(Create.of(WORDS_2))
- .apply(ParDo.of(new ExtractWordsFn()))
- .apply(Count.<String>perElement());
-
- /* CoGroup the two collections */
- PCollection<KV<String, CoGbkResult>> mergedOccurences = KeyedPCollectionTuple
- .of(tag1, occurences1)
- .and(tag2, occurences2)
- .apply(CoGroupByKey.<String>create());
-
- /* Format output */
- mergedOccurences.apply(ParDo.of(new FormatCountsFn()))
- .apply(TextIO.Write.named("test").to(resultPath));
-
- p.run();
- }
-
-
- static class ExtractWordsFn extends DoFn<String, String> {
-
- @Override
- public void startBundle(Context c) {
- }
-
- @Override
- public void processElement(ProcessContext c) {
- // Split the line into words.
- String[] words = c.element().split("[^a-zA-Z']+");
-
- // Output each word encountered into the output PCollection.
- for (String word : words) {
- if (!word.isEmpty()) {
- c.output(word);
- }
- }
- }
- }
-
- static class FormatCountsFn extends DoFn<KV<String, CoGbkResult>, String> {
- @Override
- public void processElement(ProcessContext c) {
- CoGbkResult value = c.element().getValue();
- String key = c.element().getKey();
- String countTag1 = tag1.getId() + ": ";
- String countTag2 = tag2.getId() + ": ";
- for (Long count : value.getAll(tag1)) {
- countTag1 += count + " ";
- }
- for (Long count : value.getAll(tag2)) {
- countTag2 += count;
- }
- c.output(key + " -> " + countTag1 + countTag2);
- }
- }
+ static final String[] WORDS_1 = new String[] {
+ "hi there", "hi", "hi sue bob",
+ "hi sue", "", "bob hi"};
+
+ static final String[] WORDS_2 = new String[] {
+ "hi tim", "beauty", "hooray sue bob",
+ "hi there", "", "please say hi"};
+
+ static final String[] RESULTS = new String[] {
+ "beauty -> Tag1: Tag2: 1",
+ "bob -> Tag1: 2 Tag2: 1",
+ "hi -> Tag1: 5 Tag2: 3",
+ "hooray -> Tag1: Tag2: 1",
+ "please -> Tag1: Tag2: 1",
+ "say -> Tag1: Tag2: 1",
+ "sue -> Tag1: 2 Tag2: 1",
+ "there -> Tag1: 1 Tag2: 1",
+ "tim -> Tag1: Tag2: 1"
+ };
+
+ static final TupleTag<Long> tag1 = new TupleTag<>("Tag1");
+ static final TupleTag<Long> tag2 = new TupleTag<>("Tag2");
+
+ protected String resultPath;
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(RESULTS), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ /* Create two PCollections and join them */
+ PCollection<KV<String,Long>> occurences1 = p.apply(Create.of(WORDS_1))
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Count.<String>perElement());
+
+ PCollection<KV<String,Long>> occurences2 = p.apply(Create.of(WORDS_2))
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Count.<String>perElement());
+
+ /* CoGroup the two collections */
+ PCollection<KV<String, CoGbkResult>> mergedOccurences = KeyedPCollectionTuple
+ .of(tag1, occurences1)
+ .and(tag2, occurences2)
+ .apply(CoGroupByKey.<String>create());
+
+ /* Format output */
+ mergedOccurences.apply(ParDo.of(new FormatCountsFn()))
+ .apply(TextIO.Write.named("test").to(resultPath));
+
+ p.run();
+ }
+
+
+ static class ExtractWordsFn extends DoFn<String, String> {
+
+ @Override
+ public void startBundle(Context c) {
+ }
+
+ @Override
+ public void processElement(ProcessContext c) {
+ // Split the line into words.
+ String[] words = c.element().split("[^a-zA-Z']+");
+
+ // Output each word encountered into the output PCollection.
+ for (String word : words) {
+ if (!word.isEmpty()) {
+ c.output(word);
+ }
+ }
+ }
+ }
+
+ static class FormatCountsFn extends DoFn<KV<String, CoGbkResult>, String> {
+ @Override
+ public void processElement(ProcessContext c) {
+ CoGbkResult value = c.element().getValue();
+ String key = c.element().getKey();
+ String countTag1 = tag1.getId() + ": ";
+ String countTag2 = tag2.getId() + ": ";
+ for (Long count : value.getAll(tag1)) {
+ countTag1 += count + " ";
+ }
+ for (Long count : value.getAll(tag2)) {
+ countTag2 += count;
+ }
+ c.output(key + " -> " + countTag1 + countTag2);
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java
index e6eddc0..4c8b99b 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java
@@ -33,122 +33,122 @@ import org.apache.flink.test.util.JavaProgramTestBase;
public class WordCountJoin3ITCase extends JavaProgramTestBase {
- static final String[] WORDS_1 = new String[] {
- "hi there", "hi", "hi sue bob",
- "hi sue", "", "bob hi"};
-
- static final String[] WORDS_2 = new String[] {
- "hi tim", "beauty", "hooray sue bob",
- "hi there", "", "please say hi"};
-
- static final String[] WORDS_3 = new String[] {
- "hi stephan", "beauty", "hooray big fabian",
- "hi yo", "", "please say hi"};
-
- static final String[] RESULTS = new String[] {
- "beauty -> Tag1: Tag2: 1 Tag3: 1",
- "bob -> Tag1: 2 Tag2: 1 Tag3: ",
- "hi -> Tag1: 5 Tag2: 3 Tag3: 3",
- "hooray -> Tag1: Tag2: 1 Tag3: 1",
- "please -> Tag1: Tag2: 1 Tag3: 1",
- "say -> Tag1: Tag2: 1 Tag3: 1",
- "sue -> Tag1: 2 Tag2: 1 Tag3: ",
- "there -> Tag1: 1 Tag2: 1 Tag3: ",
- "tim -> Tag1: Tag2: 1 Tag3: ",
- "stephan -> Tag1: Tag2: Tag3: 1",
- "yo -> Tag1: Tag2: Tag3: 1",
- "fabian -> Tag1: Tag2: Tag3: 1",
- "big -> Tag1: Tag2: Tag3: 1"
- };
-
- static final TupleTag<Long> tag1 = new TupleTag<>("Tag1");
- static final TupleTag<Long> tag2 = new TupleTag<>("Tag2");
- static final TupleTag<Long> tag3 = new TupleTag<>("Tag3");
-
- protected String resultPath;
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(RESULTS), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
-
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- /* Create two PCollections and join them */
- PCollection<KV<String,Long>> occurences1 = p.apply(Create.of(WORDS_1))
- .apply(ParDo.of(new ExtractWordsFn()))
- .apply(Count.<String>perElement());
-
- PCollection<KV<String,Long>> occurences2 = p.apply(Create.of(WORDS_2))
- .apply(ParDo.of(new ExtractWordsFn()))
- .apply(Count.<String>perElement());
-
- PCollection<KV<String,Long>> occurences3 = p.apply(Create.of(WORDS_3))
- .apply(ParDo.of(new ExtractWordsFn()))
- .apply(Count.<String>perElement());
-
- /* CoGroup the two collections */
- PCollection<KV<String, CoGbkResult>> mergedOccurences = KeyedPCollectionTuple
- .of(tag1, occurences1)
- .and(tag2, occurences2)
- .and(tag3, occurences3)
- .apply(CoGroupByKey.<String>create());
-
- /* Format output */
- mergedOccurences.apply(ParDo.of(new FormatCountsFn()))
- .apply(TextIO.Write.named("test").to(resultPath));
-
- p.run();
- }
-
-
- static class ExtractWordsFn extends DoFn<String, String> {
-
- @Override
- public void startBundle(Context c) {
- }
-
- @Override
- public void processElement(ProcessContext c) {
- // Split the line into words.
- String[] words = c.element().split("[^a-zA-Z']+");
-
- // Output each word encountered into the output PCollection.
- for (String word : words) {
- if (!word.isEmpty()) {
- c.output(word);
- }
- }
- }
- }
-
- static class FormatCountsFn extends DoFn<KV<String, CoGbkResult>, String> {
- @Override
- public void processElement(ProcessContext c) {
- CoGbkResult value = c.element().getValue();
- String key = c.element().getKey();
- String countTag1 = tag1.getId() + ": ";
- String countTag2 = tag2.getId() + ": ";
- String countTag3 = tag3.getId() + ": ";
- for (Long count : value.getAll(tag1)) {
- countTag1 += count + " ";
- }
- for (Long count : value.getAll(tag2)) {
- countTag2 += count + " ";
- }
- for (Long count : value.getAll(tag3)) {
- countTag3 += count;
- }
- c.output(key + " -> " + countTag1 + countTag2 + countTag3);
- }
- }
+ static final String[] WORDS_1 = new String[] {
+ "hi there", "hi", "hi sue bob",
+ "hi sue", "", "bob hi"};
+
+ static final String[] WORDS_2 = new String[] {
+ "hi tim", "beauty", "hooray sue bob",
+ "hi there", "", "please say hi"};
+
+ static final String[] WORDS_3 = new String[] {
+ "hi stephan", "beauty", "hooray big fabian",
+ "hi yo", "", "please say hi"};
+
+ static final String[] RESULTS = new String[] {
+ "beauty -> Tag1: Tag2: 1 Tag3: 1",
+ "bob -> Tag1: 2 Tag2: 1 Tag3: ",
+ "hi -> Tag1: 5 Tag2: 3 Tag3: 3",
+ "hooray -> Tag1: Tag2: 1 Tag3: 1",
+ "please -> Tag1: Tag2: 1 Tag3: 1",
+ "say -> Tag1: Tag2: 1 Tag3: 1",
+ "sue -> Tag1: 2 Tag2: 1 Tag3: ",
+ "there -> Tag1: 1 Tag2: 1 Tag3: ",
+ "tim -> Tag1: Tag2: 1 Tag3: ",
+ "stephan -> Tag1: Tag2: Tag3: 1",
+ "yo -> Tag1: Tag2: Tag3: 1",
+ "fabian -> Tag1: Tag2: Tag3: 1",
+ "big -> Tag1: Tag2: Tag3: 1"
+ };
+
+ static final TupleTag<Long> tag1 = new TupleTag<>("Tag1");
+ static final TupleTag<Long> tag2 = new TupleTag<>("Tag2");
+ static final TupleTag<Long> tag3 = new TupleTag<>("Tag3");
+
+ protected String resultPath;
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(RESULTS), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ /* Create two PCollections and join them */
+ PCollection<KV<String,Long>> occurences1 = p.apply(Create.of(WORDS_1))
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Count.<String>perElement());
+
+ PCollection<KV<String,Long>> occurences2 = p.apply(Create.of(WORDS_2))
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Count.<String>perElement());
+
+ PCollection<KV<String,Long>> occurences3 = p.apply(Create.of(WORDS_3))
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Count.<String>perElement());
+
+ /* CoGroup the two collections */
+ PCollection<KV<String, CoGbkResult>> mergedOccurences = KeyedPCollectionTuple
+ .of(tag1, occurences1)
+ .and(tag2, occurences2)
+ .and(tag3, occurences3)
+ .apply(CoGroupByKey.<String>create());
+
+ /* Format output */
+ mergedOccurences.apply(ParDo.of(new FormatCountsFn()))
+ .apply(TextIO.Write.named("test").to(resultPath));
+
+ p.run();
+ }
+
+
+ static class ExtractWordsFn extends DoFn<String, String> {
+
+ @Override
+ public void startBundle(Context c) {
+ }
+
+ @Override
+ public void processElement(ProcessContext c) {
+ // Split the line into words.
+ String[] words = c.element().split("[^a-zA-Z']+");
+
+ // Output each word encountered into the output PCollection.
+ for (String word : words) {
+ if (!word.isEmpty()) {
+ c.output(word);
+ }
+ }
+ }
+ }
+
+ static class FormatCountsFn extends DoFn<KV<String, CoGbkResult>, String> {
+ @Override
+ public void processElement(ProcessContext c) {
+ CoGbkResult value = c.element().getValue();
+ String key = c.element().getKey();
+ String countTag1 = tag1.getId() + ": ";
+ String countTag2 = tag2.getId() + ": ";
+ String countTag3 = tag3.getId() + ": ";
+ for (Long count : value.getAll(tag1)) {
+ countTag1 += count + " ";
+ }
+ for (Long count : value.getAll(tag2)) {
+ countTag2 += count + " ";
+ }
+ for (Long count : value.getAll(tag3)) {
+ countTag3 += count;
+ }
+ c.output(key + " -> " + countTag1 + countTag2 + countTag3);
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java
index 865fc5f..a61bf52 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java
@@ -39,118 +39,118 @@ import static org.junit.Assert.*;
*/
public class WriteSinkITCase extends JavaProgramTestBase {
- protected String resultPath;
+ protected String resultPath;
- public WriteSinkITCase(){
- }
+ public WriteSinkITCase(){
+ }
- static final String[] EXPECTED_RESULT = new String[] {
- "Joe red 3", "Mary blue 4", "Max yellow 23"};
+ static final String[] EXPECTED_RESULT = new String[] {
+ "Joe red 3", "Mary blue 4", "Max yellow 23"};
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
- runProgram(resultPath);
- }
-
- private static void runProgram(String resultPath) {
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- p.apply(Create.of(EXPECTED_RESULT)).setCoder(StringUtf8Coder.of())
- .apply("CustomSink", Write.to(new MyCustomSink(resultPath)));
-
- p.run();
- }
-
- /**
- * Simple custom sink which writes to a file.
- */
- private static class MyCustomSink extends Sink<String> {
-
- private final String resultPath;
-
- public MyCustomSink(String resultPath) {
- this.resultPath = resultPath;
- }
-
- @Override
- public void validate(PipelineOptions options) {
- assertNotNull(options);
- }
-
- @Override
- public WriteOperation<String, ?> createWriteOperation(PipelineOptions options) {
- return new MyWriteOperation();
- }
-
- private class MyWriteOperation extends WriteOperation<String, String> {
-
- @Override
- public Coder<String> getWriterResultCoder() {
- return StringUtf8Coder.of();
- }
-
- @Override
- public void initialize(PipelineOptions options) throws Exception {
-
- }
-
- @Override
- public void finalize(Iterable<String> writerResults, PipelineOptions options) throws Exception {
-
- }
-
- @Override
- public Writer<String, String> createWriter(PipelineOptions options) throws Exception {
- return new MyWriter();
- }
-
- @Override
- public Sink<String> getSink() {
- return MyCustomSink.this;
- }
-
- /**
- * Simple Writer which writes to a file.
- */
- private class MyWriter extends Writer<String, String> {
-
- private PrintWriter internalWriter;
-
- @Override
- public void open(String uId) throws Exception {
- Path path = new Path(resultPath + "/" + uId);
- FileSystem.get(new URI("file:///")).create(path, false);
- internalWriter = new PrintWriter(new File(path.toUri()));
- }
-
- @Override
- public void write(String value) throws Exception {
- internalWriter.println(value);
- }
-
- @Override
- public String close() throws Exception {
- internalWriter.close();
- return resultPath;
- }
-
- @Override
- public WriteOperation<String, String> getWriteOperation() {
- return MyWriteOperation.this;
- }
- }
- }
- }
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+ runProgram(resultPath);
+ }
+
+ private static void runProgram(String resultPath) {
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ p.apply(Create.of(EXPECTED_RESULT)).setCoder(StringUtf8Coder.of())
+ .apply("CustomSink", Write.to(new MyCustomSink(resultPath)));
+
+ p.run();
+ }
+
+ /**
+ * Simple custom sink which writes to a file.
+ */
+ private static class MyCustomSink extends Sink<String> {
+
+ private final String resultPath;
+
+ public MyCustomSink(String resultPath) {
+ this.resultPath = resultPath;
+ }
+
+ @Override
+ public void validate(PipelineOptions options) {
+ assertNotNull(options);
+ }
+
+ @Override
+ public WriteOperation<String, ?> createWriteOperation(PipelineOptions options) {
+ return new MyWriteOperation();
+ }
+
+ private class MyWriteOperation extends WriteOperation<String, String> {
+
+ @Override
+ public Coder<String> getWriterResultCoder() {
+ return StringUtf8Coder.of();
+ }
+
+ @Override
+ public void initialize(PipelineOptions options) throws Exception {
+
+ }
+
+ @Override
+ public void finalize(Iterable<String> writerResults, PipelineOptions options) throws Exception {
+
+ }
+
+ @Override
+ public Writer<String, String> createWriter(PipelineOptions options) throws Exception {
+ return new MyWriter();
+ }
+
+ @Override
+ public Sink<String> getSink() {
+ return MyCustomSink.this;
+ }
+
+ /**
+ * Simple Writer which writes to a file.
+ */
+ private class MyWriter extends Writer<String, String> {
+
+ private PrintWriter internalWriter;
+
+ @Override
+ public void open(String uId) throws Exception {
+ Path path = new Path(resultPath + "/" + uId);
+ FileSystem.get(new URI("file:///")).create(path, false);
+ internalWriter = new PrintWriter(new File(path.toUri()));
+ }
+
+ @Override
+ public void write(String value) throws Exception {
+ internalWriter.println(value);
+ }
+
+ @Override
+ public String close() throws Exception {
+ internalWriter.close();
+ return resultPath;
+ }
+
+ @Override
+ public WriteOperation<String, String> getWriteOperation() {
+ return MyWriteOperation.this;
+ }
+ }
+ }
+ }
}
[48/50] [abbrv] incubator-beam git commit: [flink] update license
headers
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
index 7accf09..fc75948 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
index 84007af..6cf46e5 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java
index d73ac8c..5aadccd 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java
index 055a12a..b2dc33c 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java
index 738ce5f..18e118a 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateType.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateType.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateType.java
index 8b20600..5849773 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateType.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateType.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/resources/log4j.properties b/runners/flink/src/main/resources/log4j.properties
index c88022c..4daaad1 100644
--- a/runners/flink/src/main/resources/log4j.properties
+++ b/runners/flink/src/main/resources/log4j.properties
@@ -1,17 +1,19 @@
################################################################################
-# Copyright 2015 Stephan Ewen, dataArtisans
+# 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
#
-# 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://www.apache.org/licenses/LICENSE-2.0
#
-# 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.
+# 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.
################################################################################
log4j.rootLogger=INFO,console
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/AvroITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/AvroITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/AvroITCase.java
index eaa5979..5b32d54 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/AvroITCase.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/AvroITCase.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java
index 79eb163..5ae0e83 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
index 2dcebde..aadda24 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java
index 11b6ce4..f60056d 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java
index e39b81d..199602c 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/MaybeEmptyTestITCase.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java
index 08e5323..403de29 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/ParDoMultiOutputITCase.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java
index 7202417..323c41b 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java
index dc82d7d..524554a 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesEmptyITCase.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java
index 78b48b5..54e92aa 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/RemoveDuplicatesITCase.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java
index 5cd7d78..7f73b83 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/SideInputITCase.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java
index ceb0a3f..8722fee 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/TfIdfITCase.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java
index c2b6fdd..8ca978e 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountITCase.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java
index d78434b..e73c456 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin2ITCase.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java
index 0836279..6b57d77 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/WordCountJoin3ITCase.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
index 497a5bb..dfa15ce 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java
index 27ddc83..880da59 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupAlsoByWindowTest.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
index 80d78b9..63e0bcf 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java
index e6c5ae2..77a8de6 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
index f0b93a0..83c1661 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java
index 620dace..e850dd6 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
[47/50] [abbrv] incubator-beam git commit: [flink] convert tabs to 2
spaces
Posted by da...@apache.org.
[flink] convert tabs to 2 spaces
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/8852eb15
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/8852eb15
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/8852eb15
Branch: refs/heads/master
Commit: 8852eb15deec6a7354f29c894d7d2626748d5704
Parents: 39f08fa
Author: Maximilian Michels <mx...@apache.org>
Authored: Wed Mar 2 23:03:37 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
.../FlinkPipelineExecutionEnvironment.java | 394 +++---
.../flink/dataflow/FlinkPipelineOptions.java | 102 +-
.../flink/dataflow/FlinkPipelineRunner.java | 308 ++--
.../flink/dataflow/FlinkRunnerResult.java | 64 +-
.../flink/dataflow/examples/TFIDF.java | 720 +++++-----
.../flink/dataflow/examples/WordCount.java | 154 +-
.../examples/streaming/JoinExamples.java | 208 +--
.../KafkaWindowedWordCountExample.java | 204 +--
.../examples/streaming/WindowedWordCount.java | 160 +--
.../flink/dataflow/io/ConsoleIO.java | 82 +-
.../FlinkBatchPipelineTranslator.java | 230 +--
.../FlinkBatchTransformTranslators.java | 892 ++++++------
.../FlinkBatchTranslationContext.java | 154 +-
.../translation/FlinkPipelineTranslator.java | 6 +-
.../FlinkStreamingPipelineTranslator.java | 218 +--
.../FlinkStreamingTransformTranslators.java | 684 ++++-----
.../FlinkStreamingTranslationContext.java | 88 +-
.../FlinkCoGroupKeyedListAggregator.java | 48 +-
.../functions/FlinkCreateFunction.java | 50 +-
.../functions/FlinkDoFnFunction.java | 300 ++--
.../FlinkKeyedListAggregationFunction.java | 74 +-
.../functions/FlinkMultiOutputDoFnFunction.java | 242 ++--
.../FlinkMultiOutputPruningFunction.java | 22 +-
.../functions/FlinkPartialReduceFunction.java | 50 +-
.../functions/FlinkReduceFunction.java | 32 +-
.../translation/functions/UnionCoder.java | 218 +--
.../translation/types/CoderComparator.java | 362 ++---
.../translation/types/CoderTypeInformation.java | 162 +--
.../translation/types/CoderTypeSerializer.java | 228 +--
.../types/InspectableByteArrayOutputStream.java | 12 +-
.../translation/types/KvCoderComperator.java | 452 +++---
.../types/KvCoderTypeInformation.java | 298 ++--
.../types/VoidCoderTypeSerializer.java | 154 +-
.../wrappers/CombineFnAggregatorWrapper.java | 94 +-
.../wrappers/DataInputViewWrapper.java | 50 +-
.../wrappers/DataOutputViewWrapper.java | 32 +-
.../SerializableFnAggregatorWrapper.java | 92 +-
.../translation/wrappers/SinkOutputFormat.java | 160 +--
.../translation/wrappers/SourceInputFormat.java | 240 ++--
.../translation/wrappers/SourceInputSplit.java | 30 +-
.../streaming/FlinkAbstractParDoWrapper.java | 432 +++---
.../FlinkGroupAlsoByWindowWrapper.java | 1116 +++++++--------
.../streaming/FlinkGroupByKeyWrapper.java | 46 +-
.../streaming/FlinkParDoBoundMultiWrapper.java | 70 +-
.../streaming/FlinkParDoBoundWrapper.java | 102 +-
.../io/FlinkStreamingCreateFunction.java | 52 +-
.../streaming/io/UnboundedFlinkSource.java | 76 +-
.../streaming/io/UnboundedSocketSource.java | 372 ++---
.../streaming/io/UnboundedSourceWrapper.java | 182 +--
.../state/AbstractFlinkTimerInternals.java | 178 +--
.../streaming/state/FlinkStateInternals.java | 1338 +++++++++---------
.../streaming/state/StateCheckpointReader.java | 122 +-
.../streaming/state/StateCheckpointUtils.java | 232 +--
.../streaming/state/StateCheckpointWriter.java | 198 +--
.../wrappers/streaming/state/StateType.java | 68 +-
.../dataartisans/flink/dataflow/AvroITCase.java | 128 +-
.../flink/dataflow/FlattenizeITCase.java | 60 +-
.../flink/dataflow/FlinkTestPipeline.java | 74 +-
.../flink/dataflow/JoinExamplesITCase.java | 122 +-
.../flink/dataflow/MaybeEmptyTestITCase.java | 64 +-
.../flink/dataflow/ParDoMultiOutputITCase.java | 128 +-
.../flink/dataflow/ReadSourceITCase.java | 244 ++--
.../dataflow/RemoveDuplicatesEmptyITCase.java | 48 +-
.../flink/dataflow/RemoveDuplicatesITCase.java | 50 +-
.../flink/dataflow/SideInputITCase.java | 52 +-
.../flink/dataflow/TfIdfITCase.java | 58 +-
.../flink/dataflow/WordCountITCase.java | 54 +-
.../flink/dataflow/WordCountJoin2ITCase.java | 196 +--
.../flink/dataflow/WordCountJoin3ITCase.java | 234 +--
.../flink/dataflow/WriteSinkITCase.java | 218 +--
.../streaming/GroupAlsoByWindowTest.java | 920 ++++++------
.../dataflow/streaming/GroupByNullKeyTest.java | 138 +-
.../streaming/StateSerializationTest.java | 506 +++----
.../streaming/TopWikipediaSessionsITCase.java | 178 +--
.../flink/dataflow/util/JoinExamples.java | 232 +--
75 files changed, 8179 insertions(+), 8179 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java
index 796849d..c2139c6 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java
@@ -40,228 +40,228 @@ import java.util.List;
*/
public class FlinkPipelineExecutionEnvironment {
- private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class);
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class);
- private final FlinkPipelineOptions options;
+ private final FlinkPipelineOptions options;
- /**
- * The Flink Batch execution environment. This is instantiated to either a
- * {@link org.apache.flink.api.java.CollectionEnvironment},
- * a {@link org.apache.flink.api.java.LocalEnvironment} or
- * a {@link org.apache.flink.api.java.RemoteEnvironment}, depending on the configuration
- * options.
- */
- private ExecutionEnvironment flinkBatchEnv;
+ /**
+ * The Flink Batch execution environment. This is instantiated to either a
+ * {@link org.apache.flink.api.java.CollectionEnvironment},
+ * a {@link org.apache.flink.api.java.LocalEnvironment} or
+ * a {@link org.apache.flink.api.java.RemoteEnvironment}, depending on the configuration
+ * options.
+ */
+ private ExecutionEnvironment flinkBatchEnv;
- /**
- * The Flink Streaming execution environment. This is instantiated to either a
- * {@link org.apache.flink.streaming.api.environment.LocalStreamEnvironment} or
- * a {@link org.apache.flink.streaming.api.environment.RemoteStreamEnvironment}, depending
- * on the configuration options, and more specifically, the url of the master.
- */
- private StreamExecutionEnvironment flinkStreamEnv;
+ /**
+ * The Flink Streaming execution environment. This is instantiated to either a
+ * {@link org.apache.flink.streaming.api.environment.LocalStreamEnvironment} or
+ * a {@link org.apache.flink.streaming.api.environment.RemoteStreamEnvironment}, depending
+ * on the configuration options, and more specifically, the url of the master.
+ */
+ private StreamExecutionEnvironment flinkStreamEnv;
- /**
- * Translator for this FlinkPipelineRunner. Its role is to translate the Beam operators to
- * their Flink counterparts. Based on the options provided by the user, if we have a streaming job,
- * this is instantiated as a {@link FlinkStreamingPipelineTranslator}. In other case, i.e. a batch job,
- * a {@link FlinkBatchPipelineTranslator} is created.
- */
- private FlinkPipelineTranslator flinkPipelineTranslator;
+ /**
+ * Translator for this FlinkPipelineRunner. Its role is to translate the Beam operators to
+ * their Flink counterparts. Based on the options provided by the user, if we have a streaming job,
+ * this is instantiated as a {@link FlinkStreamingPipelineTranslator}. In other case, i.e. a batch job,
+ * a {@link FlinkBatchPipelineTranslator} is created.
+ */
+ private FlinkPipelineTranslator flinkPipelineTranslator;
- /**
- * Creates a {@link FlinkPipelineExecutionEnvironment} with the user-specified parameters in the
- * provided {@link FlinkPipelineOptions}.
- *
- * @param options the user-defined pipeline options.
- * */
- public FlinkPipelineExecutionEnvironment(FlinkPipelineOptions options) {
- this.options = Preconditions.checkNotNull(options);
- this.createPipelineExecutionEnvironment();
- this.createPipelineTranslator();
- }
+ /**
+ * Creates a {@link FlinkPipelineExecutionEnvironment} with the user-specified parameters in the
+ * provided {@link FlinkPipelineOptions}.
+ *
+ * @param options the user-defined pipeline options.
+ * */
+ public FlinkPipelineExecutionEnvironment(FlinkPipelineOptions options) {
+ this.options = Preconditions.checkNotNull(options);
+ this.createPipelineExecutionEnvironment();
+ this.createPipelineTranslator();
+ }
- /**
- * Depending on the type of job (Streaming or Batch) and the user-specified options,
- * this method creates the adequate ExecutionEnvironment.
- */
- private void createPipelineExecutionEnvironment() {
- if (options.isStreaming()) {
- createStreamExecutionEnvironment();
- } else {
- createBatchExecutionEnvironment();
- }
- }
+ /**
+ * Depending on the type of job (Streaming or Batch) and the user-specified options,
+ * this method creates the adequate ExecutionEnvironment.
+ */
+ private void createPipelineExecutionEnvironment() {
+ if (options.isStreaming()) {
+ createStreamExecutionEnvironment();
+ } else {
+ createBatchExecutionEnvironment();
+ }
+ }
- /**
- * Depending on the type of job (Streaming or Batch), this method creates the adequate job graph
- * translator. In the case of batch, it will work with {@link org.apache.flink.api.java.DataSet},
- * while for streaming, it will work with {@link org.apache.flink.streaming.api.datastream.DataStream}.
- */
- private void createPipelineTranslator() {
- checkInitializationState();
- if (this.flinkPipelineTranslator != null) {
- throw new IllegalStateException("FlinkPipelineTranslator already initialized.");
- }
+ /**
+ * Depending on the type of job (Streaming or Batch), this method creates the adequate job graph
+ * translator. In the case of batch, it will work with {@link org.apache.flink.api.java.DataSet},
+ * while for streaming, it will work with {@link org.apache.flink.streaming.api.datastream.DataStream}.
+ */
+ private void createPipelineTranslator() {
+ checkInitializationState();
+ if (this.flinkPipelineTranslator != null) {
+ throw new IllegalStateException("FlinkPipelineTranslator already initialized.");
+ }
- this.flinkPipelineTranslator = options.isStreaming() ?
- new FlinkStreamingPipelineTranslator(flinkStreamEnv, options) :
- new FlinkBatchPipelineTranslator(flinkBatchEnv, options);
- }
+ this.flinkPipelineTranslator = options.isStreaming() ?
+ new FlinkStreamingPipelineTranslator(flinkStreamEnv, options) :
+ new FlinkBatchPipelineTranslator(flinkBatchEnv, options);
+ }
- /**
- * Depending on if the job is a Streaming or a Batch one, this method creates
- * the necessary execution environment and pipeline translator, and translates
- * the {@link com.google.cloud.dataflow.sdk.values.PCollection} program into
- * a {@link org.apache.flink.api.java.DataSet} or {@link org.apache.flink.streaming.api.datastream.DataStream}
- * one.
- * */
- public void translate(Pipeline pipeline) {
- checkInitializationState();
- if(this.flinkBatchEnv == null && this.flinkStreamEnv == null) {
- createPipelineExecutionEnvironment();
- }
- if (this.flinkPipelineTranslator == null) {
- createPipelineTranslator();
- }
- this.flinkPipelineTranslator.translate(pipeline);
- }
+ /**
+ * Depending on if the job is a Streaming or a Batch one, this method creates
+ * the necessary execution environment and pipeline translator, and translates
+ * the {@link com.google.cloud.dataflow.sdk.values.PCollection} program into
+ * a {@link org.apache.flink.api.java.DataSet} or {@link org.apache.flink.streaming.api.datastream.DataStream}
+ * one.
+ * */
+ public void translate(Pipeline pipeline) {
+ checkInitializationState();
+ if(this.flinkBatchEnv == null && this.flinkStreamEnv == null) {
+ createPipelineExecutionEnvironment();
+ }
+ if (this.flinkPipelineTranslator == null) {
+ createPipelineTranslator();
+ }
+ this.flinkPipelineTranslator.translate(pipeline);
+ }
- /**
- * Launches the program execution.
- * */
- public JobExecutionResult executePipeline() throws Exception {
- if (options.isStreaming()) {
- if (this.flinkStreamEnv == null) {
- throw new RuntimeException("FlinkPipelineExecutionEnvironment not initialized.");
- }
- if (this.flinkPipelineTranslator == null) {
- throw new RuntimeException("FlinkPipelineTranslator not initialized.");
- }
- return this.flinkStreamEnv.execute();
- } else {
- if (this.flinkBatchEnv == null) {
- throw new RuntimeException("FlinkPipelineExecutionEnvironment not initialized.");
- }
- if (this.flinkPipelineTranslator == null) {
- throw new RuntimeException("FlinkPipelineTranslator not initialized.");
- }
- return this.flinkBatchEnv.execute();
- }
- }
+ /**
+ * Launches the program execution.
+ * */
+ public JobExecutionResult executePipeline() throws Exception {
+ if (options.isStreaming()) {
+ if (this.flinkStreamEnv == null) {
+ throw new RuntimeException("FlinkPipelineExecutionEnvironment not initialized.");
+ }
+ if (this.flinkPipelineTranslator == null) {
+ throw new RuntimeException("FlinkPipelineTranslator not initialized.");
+ }
+ return this.flinkStreamEnv.execute();
+ } else {
+ if (this.flinkBatchEnv == null) {
+ throw new RuntimeException("FlinkPipelineExecutionEnvironment not initialized.");
+ }
+ if (this.flinkPipelineTranslator == null) {
+ throw new RuntimeException("FlinkPipelineTranslator not initialized.");
+ }
+ return this.flinkBatchEnv.execute();
+ }
+ }
- /**
- * If the submitted job is a batch processing job, this method creates the adequate
- * Flink {@link org.apache.flink.api.java.ExecutionEnvironment} depending
- * on the user-specified options.
- */
- private void createBatchExecutionEnvironment() {
- if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) {
- throw new RuntimeException("FlinkPipelineExecutionEnvironment already initialized.");
- }
+ /**
+ * If the submitted job is a batch processing job, this method creates the adequate
+ * Flink {@link org.apache.flink.api.java.ExecutionEnvironment} depending
+ * on the user-specified options.
+ */
+ private void createBatchExecutionEnvironment() {
+ if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) {
+ throw new RuntimeException("FlinkPipelineExecutionEnvironment already initialized.");
+ }
- LOG.info("Creating the required Batch Execution Environment.");
+ LOG.info("Creating the required Batch Execution Environment.");
- String masterUrl = options.getFlinkMaster();
- this.flinkStreamEnv = null;
+ String masterUrl = options.getFlinkMaster();
+ this.flinkStreamEnv = null;
- // depending on the master, create the right environment.
- if (masterUrl.equals("[local]")) {
- this.flinkBatchEnv = ExecutionEnvironment.createLocalEnvironment();
- } else if (masterUrl.equals("[collection]")) {
- this.flinkBatchEnv = new CollectionEnvironment();
- } else if (masterUrl.equals("[auto]")) {
- this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
- } else if (masterUrl.matches(".*:\\d*")) {
- String[] parts = masterUrl.split(":");
- List<String> stagingFiles = options.getFilesToStage();
- this.flinkBatchEnv = ExecutionEnvironment.createRemoteEnvironment(parts[0],
- Integer.parseInt(parts[1]),
- stagingFiles.toArray(new String[stagingFiles.size()]));
- } else {
- LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
- this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
- }
+ // depending on the master, create the right environment.
+ if (masterUrl.equals("[local]")) {
+ this.flinkBatchEnv = ExecutionEnvironment.createLocalEnvironment();
+ } else if (masterUrl.equals("[collection]")) {
+ this.flinkBatchEnv = new CollectionEnvironment();
+ } else if (masterUrl.equals("[auto]")) {
+ this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
+ } else if (masterUrl.matches(".*:\\d*")) {
+ String[] parts = masterUrl.split(":");
+ List<String> stagingFiles = options.getFilesToStage();
+ this.flinkBatchEnv = ExecutionEnvironment.createRemoteEnvironment(parts[0],
+ Integer.parseInt(parts[1]),
+ stagingFiles.toArray(new String[stagingFiles.size()]));
+ } else {
+ LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
+ this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
+ }
- // set the correct parallelism.
- if (options.getParallelism() != -1 && !(this.flinkBatchEnv instanceof CollectionEnvironment)) {
- this.flinkBatchEnv.setParallelism(options.getParallelism());
- }
+ // set the correct parallelism.
+ if (options.getParallelism() != -1 && !(this.flinkBatchEnv instanceof CollectionEnvironment)) {
+ this.flinkBatchEnv.setParallelism(options.getParallelism());
+ }
- // set parallelism in the options (required by some execution code)
- options.setParallelism(flinkBatchEnv.getParallelism());
- }
+ // set parallelism in the options (required by some execution code)
+ options.setParallelism(flinkBatchEnv.getParallelism());
+ }
- /**
- * If the submitted job is a stream processing job, this method creates the adequate
- * Flink {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment} depending
- * on the user-specified options.
- */
- private void createStreamExecutionEnvironment() {
- if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) {
- throw new RuntimeException("FlinkPipelineExecutionEnvironment already initialized.");
- }
+ /**
+ * If the submitted job is a stream processing job, this method creates the adequate
+ * Flink {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment} depending
+ * on the user-specified options.
+ */
+ private void createStreamExecutionEnvironment() {
+ if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) {
+ throw new RuntimeException("FlinkPipelineExecutionEnvironment already initialized.");
+ }
- LOG.info("Creating the required Streaming Environment.");
+ LOG.info("Creating the required Streaming Environment.");
- String masterUrl = options.getFlinkMaster();
- this.flinkBatchEnv = null;
+ String masterUrl = options.getFlinkMaster();
+ this.flinkBatchEnv = null;
- // depending on the master, create the right environment.
- if (masterUrl.equals("[local]")) {
- this.flinkStreamEnv = StreamExecutionEnvironment.createLocalEnvironment();
- } else if (masterUrl.equals("[auto]")) {
- this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
- } else if (masterUrl.matches(".*:\\d*")) {
- String[] parts = masterUrl.split(":");
- List<String> stagingFiles = options.getFilesToStage();
- this.flinkStreamEnv = StreamExecutionEnvironment.createRemoteEnvironment(parts[0],
- Integer.parseInt(parts[1]), stagingFiles.toArray(new String[stagingFiles.size()]));
- } else {
- LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
- this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
- }
+ // depending on the master, create the right environment.
+ if (masterUrl.equals("[local]")) {
+ this.flinkStreamEnv = StreamExecutionEnvironment.createLocalEnvironment();
+ } else if (masterUrl.equals("[auto]")) {
+ this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+ } else if (masterUrl.matches(".*:\\d*")) {
+ String[] parts = masterUrl.split(":");
+ List<String> stagingFiles = options.getFilesToStage();
+ this.flinkStreamEnv = StreamExecutionEnvironment.createRemoteEnvironment(parts[0],
+ Integer.parseInt(parts[1]), stagingFiles.toArray(new String[stagingFiles.size()]));
+ } else {
+ LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
+ this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+ }
- // set the correct parallelism.
- if (options.getParallelism() != -1) {
- this.flinkStreamEnv.setParallelism(options.getParallelism());
- }
+ // set the correct parallelism.
+ if (options.getParallelism() != -1) {
+ this.flinkStreamEnv.setParallelism(options.getParallelism());
+ }
- // set parallelism in the options (required by some execution code)
- options.setParallelism(flinkStreamEnv.getParallelism());
+ // set parallelism in the options (required by some execution code)
+ options.setParallelism(flinkStreamEnv.getParallelism());
- // default to event time
- this.flinkStreamEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+ // default to event time
+ this.flinkStreamEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
- // for the following 2 parameters, a value of -1 means that Flink will use
- // the default values as specified in the configuration.
- int numRetries = options.getNumberOfExecutionRetries();
- if (numRetries != -1) {
- this.flinkStreamEnv.setNumberOfExecutionRetries(numRetries);
- }
- long retryDelay = options.getExecutionRetryDelay();
- if (retryDelay != -1) {
- this.flinkStreamEnv.getConfig().setExecutionRetryDelay(retryDelay);
- }
+ // for the following 2 parameters, a value of -1 means that Flink will use
+ // the default values as specified in the configuration.
+ int numRetries = options.getNumberOfExecutionRetries();
+ if (numRetries != -1) {
+ this.flinkStreamEnv.setNumberOfExecutionRetries(numRetries);
+ }
+ long retryDelay = options.getExecutionRetryDelay();
+ if (retryDelay != -1) {
+ this.flinkStreamEnv.getConfig().setExecutionRetryDelay(retryDelay);
+ }
- // A value of -1 corresponds to disabled checkpointing (see CheckpointConfig in Flink).
- // If the value is not -1, then the validity checks are applied.
- // By default, checkpointing is disabled.
- long checkpointInterval = options.getCheckpointingInterval();
- if(checkpointInterval != -1) {
- if (checkpointInterval < 1) {
- throw new IllegalArgumentException("The checkpoint interval must be positive");
- }
- this.flinkStreamEnv.enableCheckpointing(checkpointInterval);
- }
- }
+ // A value of -1 corresponds to disabled checkpointing (see CheckpointConfig in Flink).
+ // If the value is not -1, then the validity checks are applied.
+ // By default, checkpointing is disabled.
+ long checkpointInterval = options.getCheckpointingInterval();
+ if(checkpointInterval != -1) {
+ if (checkpointInterval < 1) {
+ throw new IllegalArgumentException("The checkpoint interval must be positive");
+ }
+ this.flinkStreamEnv.enableCheckpointing(checkpointInterval);
+ }
+ }
- private void checkInitializationState() {
- if (options.isStreaming() && this.flinkBatchEnv != null) {
- throw new IllegalStateException("Attempted to run a Streaming Job with a Batch Execution Environment.");
- } else if (!options.isStreaming() && this.flinkStreamEnv != null) {
- throw new IllegalStateException("Attempted to run a Batch Job with a Streaming Execution Environment.");
- }
- }
+ private void checkInitializationState() {
+ if (options.isStreaming() && this.flinkBatchEnv != null) {
+ throw new IllegalStateException("Attempted to run a Streaming Job with a Batch Execution Environment.");
+ } else if (!options.isStreaming() && this.flinkStreamEnv != null) {
+ throw new IllegalStateException("Attempted to run a Batch Job with a Streaming Execution Environment.");
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java
index 2429cac..fabbfad 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java
@@ -31,61 +31,61 @@ import java.util.List;
*/
public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOptions, StreamingOptions {
- /**
- * List of local files to make available to workers.
- * <p>
- * Jars are placed on the worker's classpath.
- * <p>
- * The default value is the list of jars from the main program's classpath.
- */
- @Description("Jar-Files to send to all workers and put on the classpath. " +
- "The default value is all files from the classpath.")
- @JsonIgnore
- List<String> getFilesToStage();
- void setFilesToStage(List<String> value);
+ /**
+ * List of local files to make available to workers.
+ * <p>
+ * Jars are placed on the worker's classpath.
+ * <p>
+ * The default value is the list of jars from the main program's classpath.
+ */
+ @Description("Jar-Files to send to all workers and put on the classpath. " +
+ "The default value is all files from the classpath.")
+ @JsonIgnore
+ List<String> getFilesToStage();
+ void setFilesToStage(List<String> value);
- /**
- * The job name is used to identify jobs running on a Flink cluster.
- */
- @Description("Dataflow job name, to uniquely identify active jobs. "
- + "Defaults to using the ApplicationName-UserName-Date.")
- @Default.InstanceFactory(DataflowPipelineOptions.JobNameFactory.class)
- String getJobName();
- void setJobName(String value);
+ /**
+ * The job name is used to identify jobs running on a Flink cluster.
+ */
+ @Description("Dataflow job name, to uniquely identify active jobs. "
+ + "Defaults to using the ApplicationName-UserName-Date.")
+ @Default.InstanceFactory(DataflowPipelineOptions.JobNameFactory.class)
+ String getJobName();
+ void setJobName(String value);
- /**
- * The url of the Flink JobManager on which to execute pipelines. This can either be
- * the the address of a cluster JobManager, in the form "host:port" or one of the special
- * Strings "[local]", "[collection]" or "[auto]". "[local]" will start a local Flink
- * Cluster in the JVM, "[collection]" will execute the pipeline on Java Collections while
- * "[auto]" will let the system decide where to execute the pipeline based on the environment.
- */
- @Description("Address of the Flink Master where the Pipeline should be executed. Can" +
- " either be of the form \"host:port\" or one of the special values [local], " +
- "[collection] or [auto].")
- String getFlinkMaster();
- void setFlinkMaster(String value);
+ /**
+ * The url of the Flink JobManager on which to execute pipelines. This can either be
+ * the the address of a cluster JobManager, in the form "host:port" or one of the special
+ * Strings "[local]", "[collection]" or "[auto]". "[local]" will start a local Flink
+ * Cluster in the JVM, "[collection]" will execute the pipeline on Java Collections while
+ * "[auto]" will let the system decide where to execute the pipeline based on the environment.
+ */
+ @Description("Address of the Flink Master where the Pipeline should be executed. Can" +
+ " either be of the form \"host:port\" or one of the special values [local], " +
+ "[collection] or [auto].")
+ String getFlinkMaster();
+ void setFlinkMaster(String value);
- @Description("The degree of parallelism to be used when distributing operations onto workers.")
- @Default.Integer(-1)
- Integer getParallelism();
- void setParallelism(Integer value);
+ @Description("The degree of parallelism to be used when distributing operations onto workers.")
+ @Default.Integer(-1)
+ Integer getParallelism();
+ void setParallelism(Integer value);
- @Description("The interval between consecutive checkpoints (i.e. snapshots of the current pipeline state used for " +
- "fault tolerance).")
- @Default.Long(-1L)
- Long getCheckpointingInterval();
- void setCheckpointingInterval(Long interval);
+ @Description("The interval between consecutive checkpoints (i.e. snapshots of the current pipeline state used for " +
+ "fault tolerance).")
+ @Default.Long(-1L)
+ Long getCheckpointingInterval();
+ void setCheckpointingInterval(Long interval);
- @Description("Sets the number of times that failed tasks are re-executed. " +
- "A value of zero effectively disables fault tolerance. A value of -1 indicates " +
- "that the system default value (as defined in the configuration) should be used.")
- @Default.Integer(-1)
- Integer getNumberOfExecutionRetries();
- void setNumberOfExecutionRetries(Integer retries);
+ @Description("Sets the number of times that failed tasks are re-executed. " +
+ "A value of zero effectively disables fault tolerance. A value of -1 indicates " +
+ "that the system default value (as defined in the configuration) should be used.")
+ @Default.Integer(-1)
+ Integer getNumberOfExecutionRetries();
+ void setNumberOfExecutionRetries(Integer retries);
- @Description("Sets the delay between executions. A value of {@code -1} indicates that the default value should be used.")
- @Default.Long(-1L)
- Long getExecutionRetryDelay();
- void setExecutionRetryDelay(Long delay);
+ @Description("Sets the delay between executions. A value of {@code -1} indicates that the default value should be used.")
+ @Default.Long(-1L)
+ Long getExecutionRetryDelay();
+ void setExecutionRetryDelay(Long delay);
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
index 7ea8370..742a316 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
@@ -47,158 +47,158 @@ import java.util.Map;
*/
public class FlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {
- private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineRunner.class);
-
- /**
- * Provided options.
- */
- private final FlinkPipelineOptions options;
-
- private final FlinkPipelineExecutionEnvironment flinkJobEnv;
-
- /**
- * Construct a runner from the provided options.
- *
- * @param options Properties which configure the runner.
- * @return The newly created runner.
- */
- public static FlinkPipelineRunner fromOptions(PipelineOptions options) {
- FlinkPipelineOptions flinkOptions =
- PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options);
- ArrayList<String> missing = new ArrayList<>();
-
- if (flinkOptions.getAppName() == null) {
- missing.add("appName");
- }
- if (missing.size() > 0) {
- throw new IllegalArgumentException(
- "Missing required values: " + Joiner.on(',').join(missing));
- }
-
- if (flinkOptions.getFilesToStage() == null) {
- flinkOptions.setFilesToStage(detectClassPathResourcesToStage(
- DataflowPipelineRunner.class.getClassLoader()));
- LOG.info("PipelineOptions.filesToStage was not specified. "
- + "Defaulting to files from the classpath: will stage {} files. "
- + "Enable logging at DEBUG level to see which files will be staged.",
- flinkOptions.getFilesToStage().size());
- LOG.debug("Classpath elements: {}", flinkOptions.getFilesToStage());
- }
-
- // Verify jobName according to service requirements.
- String jobName = flinkOptions.getJobName().toLowerCase();
- Preconditions.checkArgument(jobName.matches("[a-z]([-a-z0-9]*[a-z0-9])?"), "JobName invalid; " +
- "the name must consist of only the characters " + "[-a-z0-9], starting with a letter " +
- "and ending with a letter " + "or number");
- Preconditions.checkArgument(jobName.length() <= 40,
- "JobName too long; must be no more than 40 characters in length");
-
- // Set Flink Master to [auto] if no option was specified.
- if (flinkOptions.getFlinkMaster() == null) {
- flinkOptions.setFlinkMaster("[auto]");
- }
-
- return new FlinkPipelineRunner(flinkOptions);
- }
-
- private FlinkPipelineRunner(FlinkPipelineOptions options) {
- this.options = options;
- this.flinkJobEnv = new FlinkPipelineExecutionEnvironment(options);
- }
-
- @Override
- public FlinkRunnerResult run(Pipeline pipeline) {
- LOG.info("Executing pipeline using FlinkPipelineRunner.");
-
- LOG.info("Translating pipeline to Flink program.");
-
- this.flinkJobEnv.translate(pipeline);
-
- LOG.info("Starting execution of Flink program.");
-
- JobExecutionResult result;
- try {
- result = this.flinkJobEnv.executePipeline();
- } catch (Exception e) {
- LOG.error("Pipeline execution failed", e);
- throw new RuntimeException("Pipeline execution failed", e);
- }
-
- LOG.info("Execution finished in {} msecs", result.getNetRuntime());
-
- Map<String, Object> accumulators = result.getAllAccumulatorResults();
- if (accumulators != null && !accumulators.isEmpty()) {
- LOG.info("Final aggregator values:");
-
- for (Map.Entry<String, Object> entry : result.getAllAccumulatorResults().entrySet()) {
- LOG.info("{} : {}", entry.getKey(), entry.getValue());
- }
- }
-
- return new FlinkRunnerResult(accumulators, result.getNetRuntime());
- }
-
- /**
- * For testing.
- */
- public FlinkPipelineOptions getPipelineOptions() {
- return options;
- }
-
- /**
- * Constructs a runner with default properties for testing.
- *
- * @return The newly created runner.
- */
- public static FlinkPipelineRunner createForTest(boolean streaming) {
- FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
- // we use [auto] for testing since this will make it pick up the Testing
- // ExecutionEnvironment
- options.setFlinkMaster("[auto]");
- options.setStreaming(streaming);
- return new FlinkPipelineRunner(options);
- }
-
- @Override
- public <Output extends POutput, Input extends PInput> Output apply(
- PTransform<Input, Output> transform, Input input) {
- return super.apply(transform, input);
- }
-
- /////////////////////////////////////////////////////////////////////////////
-
- @Override
- public String toString() {
- return "DataflowPipelineRunner#" + hashCode();
- }
-
- /**
- * Attempts to detect all the resources the class loader has access to. This does not recurse
- * to class loader parents stopping it from pulling in resources from the system class loader.
- *
- * @param classLoader The URLClassLoader to use to detect resources to stage.
- * @return A list of absolute paths to the resources the class loader uses.
- * @throws IllegalArgumentException If either the class loader is not a URLClassLoader or one
- * of the resources the class loader exposes is not a file resource.
- */
- protected static List<String> detectClassPathResourcesToStage(ClassLoader classLoader) {
- if (!(classLoader instanceof URLClassLoader)) {
- String message = String.format("Unable to use ClassLoader to detect classpath elements. "
- + "Current ClassLoader is %s, only URLClassLoaders are supported.", classLoader);
- LOG.error(message);
- throw new IllegalArgumentException(message);
- }
-
- List<String> files = new ArrayList<>();
- for (URL url : ((URLClassLoader) classLoader).getURLs()) {
- try {
- files.add(new File(url.toURI()).getAbsolutePath());
- } catch (IllegalArgumentException | URISyntaxException e) {
- String message = String.format("Unable to convert url (%s) to file.", url);
- LOG.error(message);
- throw new IllegalArgumentException(message, e);
- }
- }
- return files;
- }
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineRunner.class);
+
+ /**
+ * Provided options.
+ */
+ private final FlinkPipelineOptions options;
+
+ private final FlinkPipelineExecutionEnvironment flinkJobEnv;
+
+ /**
+ * Construct a runner from the provided options.
+ *
+ * @param options Properties which configure the runner.
+ * @return The newly created runner.
+ */
+ public static FlinkPipelineRunner fromOptions(PipelineOptions options) {
+ FlinkPipelineOptions flinkOptions =
+ PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options);
+ ArrayList<String> missing = new ArrayList<>();
+
+ if (flinkOptions.getAppName() == null) {
+ missing.add("appName");
+ }
+ if (missing.size() > 0) {
+ throw new IllegalArgumentException(
+ "Missing required values: " + Joiner.on(',').join(missing));
+ }
+
+ if (flinkOptions.getFilesToStage() == null) {
+ flinkOptions.setFilesToStage(detectClassPathResourcesToStage(
+ DataflowPipelineRunner.class.getClassLoader()));
+ LOG.info("PipelineOptions.filesToStage was not specified. "
+ + "Defaulting to files from the classpath: will stage {} files. "
+ + "Enable logging at DEBUG level to see which files will be staged.",
+ flinkOptions.getFilesToStage().size());
+ LOG.debug("Classpath elements: {}", flinkOptions.getFilesToStage());
+ }
+
+ // Verify jobName according to service requirements.
+ String jobName = flinkOptions.getJobName().toLowerCase();
+ Preconditions.checkArgument(jobName.matches("[a-z]([-a-z0-9]*[a-z0-9])?"), "JobName invalid; " +
+ "the name must consist of only the characters " + "[-a-z0-9], starting with a letter " +
+ "and ending with a letter " + "or number");
+ Preconditions.checkArgument(jobName.length() <= 40,
+ "JobName too long; must be no more than 40 characters in length");
+
+ // Set Flink Master to [auto] if no option was specified.
+ if (flinkOptions.getFlinkMaster() == null) {
+ flinkOptions.setFlinkMaster("[auto]");
+ }
+
+ return new FlinkPipelineRunner(flinkOptions);
+ }
+
+ private FlinkPipelineRunner(FlinkPipelineOptions options) {
+ this.options = options;
+ this.flinkJobEnv = new FlinkPipelineExecutionEnvironment(options);
+ }
+
+ @Override
+ public FlinkRunnerResult run(Pipeline pipeline) {
+ LOG.info("Executing pipeline using FlinkPipelineRunner.");
+
+ LOG.info("Translating pipeline to Flink program.");
+
+ this.flinkJobEnv.translate(pipeline);
+
+ LOG.info("Starting execution of Flink program.");
+
+ JobExecutionResult result;
+ try {
+ result = this.flinkJobEnv.executePipeline();
+ } catch (Exception e) {
+ LOG.error("Pipeline execution failed", e);
+ throw new RuntimeException("Pipeline execution failed", e);
+ }
+
+ LOG.info("Execution finished in {} msecs", result.getNetRuntime());
+
+ Map<String, Object> accumulators = result.getAllAccumulatorResults();
+ if (accumulators != null && !accumulators.isEmpty()) {
+ LOG.info("Final aggregator values:");
+
+ for (Map.Entry<String, Object> entry : result.getAllAccumulatorResults().entrySet()) {
+ LOG.info("{} : {}", entry.getKey(), entry.getValue());
+ }
+ }
+
+ return new FlinkRunnerResult(accumulators, result.getNetRuntime());
+ }
+
+ /**
+ * For testing.
+ */
+ public FlinkPipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ /**
+ * Constructs a runner with default properties for testing.
+ *
+ * @return The newly created runner.
+ */
+ public static FlinkPipelineRunner createForTest(boolean streaming) {
+ FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
+ // we use [auto] for testing since this will make it pick up the Testing
+ // ExecutionEnvironment
+ options.setFlinkMaster("[auto]");
+ options.setStreaming(streaming);
+ return new FlinkPipelineRunner(options);
+ }
+
+ @Override
+ public <Output extends POutput, Input extends PInput> Output apply(
+ PTransform<Input, Output> transform, Input input) {
+ return super.apply(transform, input);
+ }
+
+ /////////////////////////////////////////////////////////////////////////////
+
+ @Override
+ public String toString() {
+ return "DataflowPipelineRunner#" + hashCode();
+ }
+
+ /**
+ * Attempts to detect all the resources the class loader has access to. This does not recurse
+ * to class loader parents stopping it from pulling in resources from the system class loader.
+ *
+ * @param classLoader The URLClassLoader to use to detect resources to stage.
+ * @return A list of absolute paths to the resources the class loader uses.
+ * @throws IllegalArgumentException If either the class loader is not a URLClassLoader or one
+ * of the resources the class loader exposes is not a file resource.
+ */
+ protected static List<String> detectClassPathResourcesToStage(ClassLoader classLoader) {
+ if (!(classLoader instanceof URLClassLoader)) {
+ String message = String.format("Unable to use ClassLoader to detect classpath elements. "
+ + "Current ClassLoader is %s, only URLClassLoaders are supported.", classLoader);
+ LOG.error(message);
+ throw new IllegalArgumentException(message);
+ }
+
+ List<String> files = new ArrayList<>();
+ for (URL url : ((URLClassLoader) classLoader).getURLs()) {
+ try {
+ files.add(new File(url.toURI()).getAbsolutePath());
+ } catch (IllegalArgumentException | URISyntaxException e) {
+ String message = String.format("Unable to convert url (%s) to file.", url);
+ LOG.error(message);
+ throw new IllegalArgumentException(message, e);
+ }
+ }
+ return files;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java
index 59b8b63..dfbaf66 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkRunnerResult.java
@@ -29,38 +29,38 @@ import java.util.Map;
* {@link com.google.cloud.dataflow.sdk.transforms.Aggregator}s.
*/
public class FlinkRunnerResult implements PipelineResult {
-
- private final Map<String, Object> aggregators;
-
- private final long runtime;
-
- public FlinkRunnerResult(Map<String, Object> aggregators, long runtime) {
- this.aggregators = (aggregators == null || aggregators.isEmpty()) ?
- Collections.<String, Object>emptyMap() :
- Collections.unmodifiableMap(aggregators);
-
- this.runtime = runtime;
- }
+
+ private final Map<String, Object> aggregators;
+
+ private final long runtime;
+
+ public FlinkRunnerResult(Map<String, Object> aggregators, long runtime) {
+ this.aggregators = (aggregators == null || aggregators.isEmpty()) ?
+ Collections.<String, Object>emptyMap() :
+ Collections.unmodifiableMap(aggregators);
+
+ this.runtime = runtime;
+ }
- @Override
- public State getState() {
- return null;
- }
+ @Override
+ public State getState() {
+ return null;
+ }
- @Override
- public <T> AggregatorValues<T> getAggregatorValues(final Aggregator<?, T> aggregator) throws AggregatorRetrievalException {
- // TODO provide a list of all accumulator step values
- Object value = aggregators.get(aggregator.getName());
- if (value != null) {
- return new AggregatorValues<T>() {
- @Override
- public Map<String, T> getValuesAtSteps() {
- return (Map<String, T>) aggregators;
- }
- };
- } else {
- throw new AggregatorRetrievalException("Accumulator results not found.",
- new RuntimeException("Accumulator does not exist."));
- }
- }
+ @Override
+ public <T> AggregatorValues<T> getAggregatorValues(final Aggregator<?, T> aggregator) throws AggregatorRetrievalException {
+ // TODO provide a list of all accumulator step values
+ Object value = aggregators.get(aggregator.getName());
+ if (value != null) {
+ return new AggregatorValues<T>() {
+ @Override
+ public Map<String, T> getValuesAtSteps() {
+ return (Map<String, T>) aggregators;
+ }
+ };
+ } else {
+ throw new AggregatorRetrievalException("Accumulator results not found.",
+ new RuntimeException("Accumulator does not exist."));
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java
index 86a2695..8accae7 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/TFIDF.java
@@ -89,364 +89,364 @@ import java.util.Set;
* {@code --input}.
*/
public class TFIDF {
- /**
- * Options supported by {@link TFIDF}.
- * <p>
- * Inherits standard configuration options.
- */
- private interface Options extends PipelineOptions, FlinkPipelineOptions {
- @Description("Path to the directory or GCS prefix containing files to read from")
- @Default.String("gs://dataflow-samples/shakespeare/")
- String getInput();
- void setInput(String value);
-
- @Description("Prefix of output URI to write to")
- @Validation.Required
- String getOutput();
- void setOutput(String value);
- }
-
- /**
- * Lists documents contained beneath the {@code options.input} prefix/directory.
- */
- public static Set<URI> listInputDocuments(Options options)
- throws URISyntaxException, IOException {
- URI baseUri = new URI(options.getInput());
-
- // List all documents in the directory or GCS prefix.
- URI absoluteUri;
- if (baseUri.getScheme() != null) {
- absoluteUri = baseUri;
- } else {
- absoluteUri = new URI(
- "file",
- baseUri.getAuthority(),
- baseUri.getPath(),
- baseUri.getQuery(),
- baseUri.getFragment());
- }
-
- Set<URI> uris = new HashSet<>();
- if (absoluteUri.getScheme().equals("file")) {
- File directory = new File(absoluteUri);
- for (String entry : directory.list()) {
- File path = new File(directory, entry);
- uris.add(path.toURI());
- }
- } else if (absoluteUri.getScheme().equals("gs")) {
- GcsUtil gcsUtil = options.as(GcsOptions.class).getGcsUtil();
- URI gcsUriGlob = new URI(
- absoluteUri.getScheme(),
- absoluteUri.getAuthority(),
- absoluteUri.getPath() + "*",
- absoluteUri.getQuery(),
- absoluteUri.getFragment());
- for (GcsPath entry : gcsUtil.expand(GcsPath.fromUri(gcsUriGlob))) {
- uris.add(entry.toUri());
- }
- }
-
- return uris;
- }
-
- /**
- * Reads the documents at the provided uris and returns all lines
- * from the documents tagged with which document they are from.
- */
- public static class ReadDocuments
- extends PTransform<PInput, PCollection<KV<URI, String>>> {
- private static final long serialVersionUID = 0;
-
- private Iterable<URI> uris;
-
- public ReadDocuments(Iterable<URI> uris) {
- this.uris = uris;
- }
-
- @Override
- public Coder<?> getDefaultOutputCoder() {
- return KvCoder.of(StringDelegateCoder.of(URI.class), StringUtf8Coder.of());
- }
-
- @Override
- public PCollection<KV<URI, String>> apply(PInput input) {
- Pipeline pipeline = input.getPipeline();
-
- // Create one TextIO.Read transform for each document
- // and add its output to a PCollectionList
- PCollectionList<KV<URI, String>> urisToLines =
- PCollectionList.empty(pipeline);
-
- // TextIO.Read supports:
- // - file: URIs and paths locally
- // - gs: URIs on the service
- for (final URI uri : uris) {
- String uriString;
- if (uri.getScheme().equals("file")) {
- uriString = new File(uri).getPath();
- } else {
- uriString = uri.toString();
- }
-
- PCollection<KV<URI, String>> oneUriToLines = pipeline
- .apply(TextIO.Read.from(uriString)
- .named("TextIO.Read(" + uriString + ")"))
- .apply("WithKeys(" + uriString + ")", WithKeys.<URI, String>of(uri));
-
- urisToLines = urisToLines.and(oneUriToLines);
- }
-
- return urisToLines.apply(Flatten.<KV<URI, String>>pCollections());
- }
- }
-
- /**
- * A transform containing a basic TF-IDF pipeline. The input consists of KV objects
- * where the key is the document's URI and the value is a piece
- * of the document's content. The output is mapping from terms to
- * scores for each document URI.
- */
- public static class ComputeTfIdf
- extends PTransform<PCollection<KV<URI, String>>, PCollection<KV<String, KV<URI, Double>>>> {
- private static final long serialVersionUID = 0;
-
- public ComputeTfIdf() { }
-
- @Override
- public PCollection<KV<String, KV<URI, Double>>> apply(
- PCollection<KV<URI, String>> uriToContent) {
-
- // Compute the total number of documents, and
- // prepare this singleton PCollectionView for
- // use as a side input.
- final PCollectionView<Long> totalDocuments =
- uriToContent
- .apply("GetURIs", Keys.<URI>create())
- .apply("RemoveDuplicateDocs", RemoveDuplicates.<URI>create())
- .apply(Count.<URI>globally())
- .apply(View.<Long>asSingleton());
-
- // Create a collection of pairs mapping a URI to each
- // of the words in the document associated with that that URI.
- PCollection<KV<URI, String>> uriToWords = uriToContent
- .apply(ParDo.named("SplitWords").of(
- new DoFn<KV<URI, String>, KV<URI, String>>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- URI uri = c.element().getKey();
- String line = c.element().getValue();
- for (String word : line.split("\\W+")) {
- // Log INFO messages when the word “love” is found.
- if (word.toLowerCase().equals("love")) {
- LOG.info("Found {}", word.toLowerCase());
- }
-
- if (!word.isEmpty()) {
- c.output(KV.of(uri, word.toLowerCase()));
- }
- }
- }
- }));
-
- // Compute a mapping from each word to the total
- // number of documents in which it appears.
- PCollection<KV<String, Long>> wordToDocCount = uriToWords
- .apply("RemoveDuplicateWords", RemoveDuplicates.<KV<URI, String>>create())
- .apply(Values.<String>create())
- .apply("CountDocs", Count.<String>perElement());
-
- // Compute a mapping from each URI to the total
- // number of words in the document associated with that URI.
- PCollection<KV<URI, Long>> uriToWordTotal = uriToWords
- .apply("GetURIs2", Keys.<URI>create())
- .apply("CountWords", Count.<URI>perElement());
-
- // Count, for each (URI, word) pair, the number of
- // occurrences of that word in the document associated
- // with the URI.
- PCollection<KV<KV<URI, String>, Long>> uriAndWordToCount = uriToWords
- .apply("CountWordDocPairs", Count.<KV<URI, String>>perElement());
-
- // Adjust the above collection to a mapping from
- // (URI, word) pairs to counts into an isomorphic mapping
- // from URI to (word, count) pairs, to prepare for a join
- // by the URI key.
- PCollection<KV<URI, KV<String, Long>>> uriToWordAndCount = uriAndWordToCount
- .apply(ParDo.named("ShiftKeys").of(
- new DoFn<KV<KV<URI, String>, Long>, KV<URI, KV<String, Long>>>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- URI uri = c.element().getKey().getKey();
- String word = c.element().getKey().getValue();
- Long occurrences = c.element().getValue();
- c.output(KV.of(uri, KV.of(word, occurrences)));
- }
- }));
-
- // Prepare to join the mapping of URI to (word, count) pairs with
- // the mapping of URI to total word counts, by associating
- // each of the input PCollection<KV<URI, ...>> with
- // a tuple tag. Each input must have the same key type, URI
- // in this case. The type parameter of the tuple tag matches
- // the types of the values for each collection.
- final TupleTag<Long> wordTotalsTag = new TupleTag<>();
- final TupleTag<KV<String, Long>> wordCountsTag = new TupleTag<>();
- KeyedPCollectionTuple<URI> coGbkInput = KeyedPCollectionTuple
- .of(wordTotalsTag, uriToWordTotal)
- .and(wordCountsTag, uriToWordAndCount);
-
- // Perform a CoGroupByKey (a sort of pre-join) on the prepared
- // inputs. This yields a mapping from URI to a CoGbkResult
- // (CoGroupByKey Result). The CoGbkResult is a mapping
- // from the above tuple tags to the values in each input
- // associated with a particular URI. In this case, each
- // KV<URI, CoGbkResult> group a URI with the total number of
- // words in that document as well as all the (word, count)
- // pairs for particular words.
- PCollection<KV<URI, CoGbkResult>> uriToWordAndCountAndTotal = coGbkInput
- .apply("CoGroupByUri", CoGroupByKey.<URI>create());
-
- // Compute a mapping from each word to a (URI, term frequency)
- // pair for each URI. A word's term frequency for a document
- // is simply the number of times that word occurs in the document
- // divided by the total number of words in the document.
- PCollection<KV<String, KV<URI, Double>>> wordToUriAndTf = uriToWordAndCountAndTotal
- .apply(ParDo.named("ComputeTermFrequencies").of(
- new DoFn<KV<URI, CoGbkResult>, KV<String, KV<URI, Double>>>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- URI uri = c.element().getKey();
- Long wordTotal = c.element().getValue().getOnly(wordTotalsTag);
-
- for (KV<String, Long> wordAndCount
- : c.element().getValue().getAll(wordCountsTag)) {
- String word = wordAndCount.getKey();
- Long wordCount = wordAndCount.getValue();
- Double termFrequency = wordCount.doubleValue() / wordTotal.doubleValue();
- c.output(KV.of(word, KV.of(uri, termFrequency)));
- }
- }
- }));
-
- // Compute a mapping from each word to its document frequency.
- // A word's document frequency in a corpus is the number of
- // documents in which the word appears divided by the total
- // number of documents in the corpus. Note how the total number of
- // documents is passed as a side input; the same value is
- // presented to each invocation of the DoFn.
- PCollection<KV<String, Double>> wordToDf = wordToDocCount
- .apply(ParDo
- .named("ComputeDocFrequencies")
- .withSideInputs(totalDocuments)
- .of(new DoFn<KV<String, Long>, KV<String, Double>>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- String word = c.element().getKey();
- Long documentCount = c.element().getValue();
- Long documentTotal = c.sideInput(totalDocuments);
- Double documentFrequency = documentCount.doubleValue()
- / documentTotal.doubleValue();
-
- c.output(KV.of(word, documentFrequency));
- }
- }));
-
- // Join the term frequency and document frequency
- // collections, each keyed on the word.
- final TupleTag<KV<URI, Double>> tfTag = new TupleTag<>();
- final TupleTag<Double> dfTag = new TupleTag<>();
- PCollection<KV<String, CoGbkResult>> wordToUriAndTfAndDf = KeyedPCollectionTuple
- .of(tfTag, wordToUriAndTf)
- .and(dfTag, wordToDf)
- .apply(CoGroupByKey.<String>create());
-
- // Compute a mapping from each word to a (URI, TF-IDF) score
- // for each URI. There are a variety of definitions of TF-IDF
- // ("term frequency - inverse document frequency") score;
- // here we use a basic version that is the term frequency
- // divided by the log of the document frequency.
-
- return wordToUriAndTfAndDf
- .apply(ParDo.named("ComputeTfIdf").of(
- new DoFn<KV<String, CoGbkResult>, KV<String, KV<URI, Double>>>() {
- private static final long serialVersionUID1 = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- String word = c.element().getKey();
- Double df = c.element().getValue().getOnly(dfTag);
-
- for (KV<URI, Double> uriAndTf : c.element().getValue().getAll(tfTag)) {
- URI uri = uriAndTf.getKey();
- Double tf = uriAndTf.getValue();
- Double tfIdf = tf * Math.log(1 / df);
- c.output(KV.of(word, KV.of(uri, tfIdf)));
- }
- }
- }));
- }
-
- // Instantiate Logger.
- // It is suggested that the user specify the class name of the containing class
- // (in this case ComputeTfIdf).
- private static final Logger LOG = LoggerFactory.getLogger(ComputeTfIdf.class);
- }
-
- /**
- * A {@link PTransform} to write, in CSV format, a mapping from term and URI
- * to score.
- */
- public static class WriteTfIdf
- extends PTransform<PCollection<KV<String, KV<URI, Double>>>, PDone> {
- private static final long serialVersionUID = 0;
-
- private String output;
-
- public WriteTfIdf(String output) {
- this.output = output;
- }
-
- @Override
- public PDone apply(PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf) {
- return wordToUriAndTfIdf
- .apply(ParDo.named("Format").of(new DoFn<KV<String, KV<URI, Double>>, String>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- c.output(String.format("%s,\t%s,\t%f",
- c.element().getKey(),
- c.element().getValue().getKey(),
- c.element().getValue().getValue()));
- }
- }))
- .apply(TextIO.Write
- .to(output)
- .withSuffix(".csv"));
- }
- }
-
- public static void main(String[] args) throws Exception {
- Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-
- options.setRunner(FlinkPipelineRunner.class);
-
- Pipeline pipeline = Pipeline.create(options);
- pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
-
- pipeline
- .apply(new ReadDocuments(listInputDocuments(options)))
- .apply(new ComputeTfIdf())
- .apply(new WriteTfIdf(options.getOutput()));
-
- pipeline.run();
- }
+ /**
+ * Options supported by {@link TFIDF}.
+ * <p>
+ * Inherits standard configuration options.
+ */
+ private interface Options extends PipelineOptions, FlinkPipelineOptions {
+ @Description("Path to the directory or GCS prefix containing files to read from")
+ @Default.String("gs://dataflow-samples/shakespeare/")
+ String getInput();
+ void setInput(String value);
+
+ @Description("Prefix of output URI to write to")
+ @Validation.Required
+ String getOutput();
+ void setOutput(String value);
+ }
+
+ /**
+ * Lists documents contained beneath the {@code options.input} prefix/directory.
+ */
+ public static Set<URI> listInputDocuments(Options options)
+ throws URISyntaxException, IOException {
+ URI baseUri = new URI(options.getInput());
+
+ // List all documents in the directory or GCS prefix.
+ URI absoluteUri;
+ if (baseUri.getScheme() != null) {
+ absoluteUri = baseUri;
+ } else {
+ absoluteUri = new URI(
+ "file",
+ baseUri.getAuthority(),
+ baseUri.getPath(),
+ baseUri.getQuery(),
+ baseUri.getFragment());
+ }
+
+ Set<URI> uris = new HashSet<>();
+ if (absoluteUri.getScheme().equals("file")) {
+ File directory = new File(absoluteUri);
+ for (String entry : directory.list()) {
+ File path = new File(directory, entry);
+ uris.add(path.toURI());
+ }
+ } else if (absoluteUri.getScheme().equals("gs")) {
+ GcsUtil gcsUtil = options.as(GcsOptions.class).getGcsUtil();
+ URI gcsUriGlob = new URI(
+ absoluteUri.getScheme(),
+ absoluteUri.getAuthority(),
+ absoluteUri.getPath() + "*",
+ absoluteUri.getQuery(),
+ absoluteUri.getFragment());
+ for (GcsPath entry : gcsUtil.expand(GcsPath.fromUri(gcsUriGlob))) {
+ uris.add(entry.toUri());
+ }
+ }
+
+ return uris;
+ }
+
+ /**
+ * Reads the documents at the provided uris and returns all lines
+ * from the documents tagged with which document they are from.
+ */
+ public static class ReadDocuments
+ extends PTransform<PInput, PCollection<KV<URI, String>>> {
+ private static final long serialVersionUID = 0;
+
+ private Iterable<URI> uris;
+
+ public ReadDocuments(Iterable<URI> uris) {
+ this.uris = uris;
+ }
+
+ @Override
+ public Coder<?> getDefaultOutputCoder() {
+ return KvCoder.of(StringDelegateCoder.of(URI.class), StringUtf8Coder.of());
+ }
+
+ @Override
+ public PCollection<KV<URI, String>> apply(PInput input) {
+ Pipeline pipeline = input.getPipeline();
+
+ // Create one TextIO.Read transform for each document
+ // and add its output to a PCollectionList
+ PCollectionList<KV<URI, String>> urisToLines =
+ PCollectionList.empty(pipeline);
+
+ // TextIO.Read supports:
+ // - file: URIs and paths locally
+ // - gs: URIs on the service
+ for (final URI uri : uris) {
+ String uriString;
+ if (uri.getScheme().equals("file")) {
+ uriString = new File(uri).getPath();
+ } else {
+ uriString = uri.toString();
+ }
+
+ PCollection<KV<URI, String>> oneUriToLines = pipeline
+ .apply(TextIO.Read.from(uriString)
+ .named("TextIO.Read(" + uriString + ")"))
+ .apply("WithKeys(" + uriString + ")", WithKeys.<URI, String>of(uri));
+
+ urisToLines = urisToLines.and(oneUriToLines);
+ }
+
+ return urisToLines.apply(Flatten.<KV<URI, String>>pCollections());
+ }
+ }
+
+ /**
+ * A transform containing a basic TF-IDF pipeline. The input consists of KV objects
+ * where the key is the document's URI and the value is a piece
+ * of the document's content. The output is mapping from terms to
+ * scores for each document URI.
+ */
+ public static class ComputeTfIdf
+ extends PTransform<PCollection<KV<URI, String>>, PCollection<KV<String, KV<URI, Double>>>> {
+ private static final long serialVersionUID = 0;
+
+ public ComputeTfIdf() { }
+
+ @Override
+ public PCollection<KV<String, KV<URI, Double>>> apply(
+ PCollection<KV<URI, String>> uriToContent) {
+
+ // Compute the total number of documents, and
+ // prepare this singleton PCollectionView for
+ // use as a side input.
+ final PCollectionView<Long> totalDocuments =
+ uriToContent
+ .apply("GetURIs", Keys.<URI>create())
+ .apply("RemoveDuplicateDocs", RemoveDuplicates.<URI>create())
+ .apply(Count.<URI>globally())
+ .apply(View.<Long>asSingleton());
+
+ // Create a collection of pairs mapping a URI to each
+ // of the words in the document associated with that that URI.
+ PCollection<KV<URI, String>> uriToWords = uriToContent
+ .apply(ParDo.named("SplitWords").of(
+ new DoFn<KV<URI, String>, KV<URI, String>>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ URI uri = c.element().getKey();
+ String line = c.element().getValue();
+ for (String word : line.split("\\W+")) {
+ // Log INFO messages when the word “love” is found.
+ if (word.toLowerCase().equals("love")) {
+ LOG.info("Found {}", word.toLowerCase());
+ }
+
+ if (!word.isEmpty()) {
+ c.output(KV.of(uri, word.toLowerCase()));
+ }
+ }
+ }
+ }));
+
+ // Compute a mapping from each word to the total
+ // number of documents in which it appears.
+ PCollection<KV<String, Long>> wordToDocCount = uriToWords
+ .apply("RemoveDuplicateWords", RemoveDuplicates.<KV<URI, String>>create())
+ .apply(Values.<String>create())
+ .apply("CountDocs", Count.<String>perElement());
+
+ // Compute a mapping from each URI to the total
+ // number of words in the document associated with that URI.
+ PCollection<KV<URI, Long>> uriToWordTotal = uriToWords
+ .apply("GetURIs2", Keys.<URI>create())
+ .apply("CountWords", Count.<URI>perElement());
+
+ // Count, for each (URI, word) pair, the number of
+ // occurrences of that word in the document associated
+ // with the URI.
+ PCollection<KV<KV<URI, String>, Long>> uriAndWordToCount = uriToWords
+ .apply("CountWordDocPairs", Count.<KV<URI, String>>perElement());
+
+ // Adjust the above collection to a mapping from
+ // (URI, word) pairs to counts into an isomorphic mapping
+ // from URI to (word, count) pairs, to prepare for a join
+ // by the URI key.
+ PCollection<KV<URI, KV<String, Long>>> uriToWordAndCount = uriAndWordToCount
+ .apply(ParDo.named("ShiftKeys").of(
+ new DoFn<KV<KV<URI, String>, Long>, KV<URI, KV<String, Long>>>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ URI uri = c.element().getKey().getKey();
+ String word = c.element().getKey().getValue();
+ Long occurrences = c.element().getValue();
+ c.output(KV.of(uri, KV.of(word, occurrences)));
+ }
+ }));
+
+ // Prepare to join the mapping of URI to (word, count) pairs with
+ // the mapping of URI to total word counts, by associating
+ // each of the input PCollection<KV<URI, ...>> with
+ // a tuple tag. Each input must have the same key type, URI
+ // in this case. The type parameter of the tuple tag matches
+ // the types of the values for each collection.
+ final TupleTag<Long> wordTotalsTag = new TupleTag<>();
+ final TupleTag<KV<String, Long>> wordCountsTag = new TupleTag<>();
+ KeyedPCollectionTuple<URI> coGbkInput = KeyedPCollectionTuple
+ .of(wordTotalsTag, uriToWordTotal)
+ .and(wordCountsTag, uriToWordAndCount);
+
+ // Perform a CoGroupByKey (a sort of pre-join) on the prepared
+ // inputs. This yields a mapping from URI to a CoGbkResult
+ // (CoGroupByKey Result). The CoGbkResult is a mapping
+ // from the above tuple tags to the values in each input
+ // associated with a particular URI. In this case, each
+ // KV<URI, CoGbkResult> group a URI with the total number of
+ // words in that document as well as all the (word, count)
+ // pairs for particular words.
+ PCollection<KV<URI, CoGbkResult>> uriToWordAndCountAndTotal = coGbkInput
+ .apply("CoGroupByUri", CoGroupByKey.<URI>create());
+
+ // Compute a mapping from each word to a (URI, term frequency)
+ // pair for each URI. A word's term frequency for a document
+ // is simply the number of times that word occurs in the document
+ // divided by the total number of words in the document.
+ PCollection<KV<String, KV<URI, Double>>> wordToUriAndTf = uriToWordAndCountAndTotal
+ .apply(ParDo.named("ComputeTermFrequencies").of(
+ new DoFn<KV<URI, CoGbkResult>, KV<String, KV<URI, Double>>>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ URI uri = c.element().getKey();
+ Long wordTotal = c.element().getValue().getOnly(wordTotalsTag);
+
+ for (KV<String, Long> wordAndCount
+ : c.element().getValue().getAll(wordCountsTag)) {
+ String word = wordAndCount.getKey();
+ Long wordCount = wordAndCount.getValue();
+ Double termFrequency = wordCount.doubleValue() / wordTotal.doubleValue();
+ c.output(KV.of(word, KV.of(uri, termFrequency)));
+ }
+ }
+ }));
+
+ // Compute a mapping from each word to its document frequency.
+ // A word's document frequency in a corpus is the number of
+ // documents in which the word appears divided by the total
+ // number of documents in the corpus. Note how the total number of
+ // documents is passed as a side input; the same value is
+ // presented to each invocation of the DoFn.
+ PCollection<KV<String, Double>> wordToDf = wordToDocCount
+ .apply(ParDo
+ .named("ComputeDocFrequencies")
+ .withSideInputs(totalDocuments)
+ .of(new DoFn<KV<String, Long>, KV<String, Double>>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ String word = c.element().getKey();
+ Long documentCount = c.element().getValue();
+ Long documentTotal = c.sideInput(totalDocuments);
+ Double documentFrequency = documentCount.doubleValue()
+ / documentTotal.doubleValue();
+
+ c.output(KV.of(word, documentFrequency));
+ }
+ }));
+
+ // Join the term frequency and document frequency
+ // collections, each keyed on the word.
+ final TupleTag<KV<URI, Double>> tfTag = new TupleTag<>();
+ final TupleTag<Double> dfTag = new TupleTag<>();
+ PCollection<KV<String, CoGbkResult>> wordToUriAndTfAndDf = KeyedPCollectionTuple
+ .of(tfTag, wordToUriAndTf)
+ .and(dfTag, wordToDf)
+ .apply(CoGroupByKey.<String>create());
+
+ // Compute a mapping from each word to a (URI, TF-IDF) score
+ // for each URI. There are a variety of definitions of TF-IDF
+ // ("term frequency - inverse document frequency") score;
+ // here we use a basic version that is the term frequency
+ // divided by the log of the document frequency.
+
+ return wordToUriAndTfAndDf
+ .apply(ParDo.named("ComputeTfIdf").of(
+ new DoFn<KV<String, CoGbkResult>, KV<String, KV<URI, Double>>>() {
+ private static final long serialVersionUID1 = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ String word = c.element().getKey();
+ Double df = c.element().getValue().getOnly(dfTag);
+
+ for (KV<URI, Double> uriAndTf : c.element().getValue().getAll(tfTag)) {
+ URI uri = uriAndTf.getKey();
+ Double tf = uriAndTf.getValue();
+ Double tfIdf = tf * Math.log(1 / df);
+ c.output(KV.of(word, KV.of(uri, tfIdf)));
+ }
+ }
+ }));
+ }
+
+ // Instantiate Logger.
+ // It is suggested that the user specify the class name of the containing class
+ // (in this case ComputeTfIdf).
+ private static final Logger LOG = LoggerFactory.getLogger(ComputeTfIdf.class);
+ }
+
+ /**
+ * A {@link PTransform} to write, in CSV format, a mapping from term and URI
+ * to score.
+ */
+ public static class WriteTfIdf
+ extends PTransform<PCollection<KV<String, KV<URI, Double>>>, PDone> {
+ private static final long serialVersionUID = 0;
+
+ private String output;
+
+ public WriteTfIdf(String output) {
+ this.output = output;
+ }
+
+ @Override
+ public PDone apply(PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf) {
+ return wordToUriAndTfIdf
+ .apply(ParDo.named("Format").of(new DoFn<KV<String, KV<URI, Double>>, String>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ c.output(String.format("%s,\t%s,\t%f",
+ c.element().getKey(),
+ c.element().getValue().getKey(),
+ c.element().getValue().getValue()));
+ }
+ }))
+ .apply(TextIO.Write
+ .to(output)
+ .withSuffix(".csv"));
+ }
+ }
+
+ public static void main(String[] args) throws Exception {
+ Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+
+ options.setRunner(FlinkPipelineRunner.class);
+
+ Pipeline pipeline = Pipeline.create(options);
+ pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
+
+ pipeline
+ .apply(new ReadDocuments(listInputDocuments(options)))
+ .apply(new ComputeTfIdf())
+ .apply(new WriteTfIdf(options.getOutput()));
+
+ pipeline.run();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java
index e737fe8..4f721b4 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/WordCount.java
@@ -29,83 +29,83 @@ import com.google.cloud.dataflow.sdk.values.PCollection;
public class WordCount {
- public static class ExtractWordsFn extends DoFn<String, String> {
- private final Aggregator<Long, Long> emptyLines =
- createAggregator("emptyLines", new Sum.SumLongFn());
-
- @Override
- public void processElement(ProcessContext c) {
- if (c.element().trim().isEmpty()) {
- emptyLines.addValue(1L);
- }
-
- // Split the line into words.
- String[] words = c.element().split("[^a-zA-Z']+");
-
- // Output each word encountered into the output PCollection.
- for (String word : words) {
- if (!word.isEmpty()) {
- c.output(word);
- }
- }
- }
- }
-
- public static class CountWords extends PTransform<PCollection<String>,
+ public static class ExtractWordsFn extends DoFn<String, String> {
+ private final Aggregator<Long, Long> emptyLines =
+ createAggregator("emptyLines", new Sum.SumLongFn());
+
+ @Override
+ public void processElement(ProcessContext c) {
+ if (c.element().trim().isEmpty()) {
+ emptyLines.addValue(1L);
+ }
+
+ // Split the line into words.
+ String[] words = c.element().split("[^a-zA-Z']+");
+
+ // Output each word encountered into the output PCollection.
+ for (String word : words) {
+ if (!word.isEmpty()) {
+ c.output(word);
+ }
+ }
+ }
+ }
+
+ public static class CountWords extends PTransform<PCollection<String>,
PCollection<KV<String, Long>>> {
- @Override
- public PCollection<KV<String, Long>> apply(PCollection<String> lines) {
-
- // Convert lines of text into individual words.
- PCollection<String> words = lines.apply(
- ParDo.of(new ExtractWordsFn()));
-
- // Count the number of times each word occurs.
- PCollection<KV<String, Long>> wordCounts =
- words.apply(Count.<String>perElement());
-
- return wordCounts;
- }
- }
-
- /** A SimpleFunction that converts a Word and Count into a printable string. */
- public static class FormatAsTextFn extends SimpleFunction<KV<String, Long>, String> {
- @Override
- public String apply(KV<String, Long> input) {
- return input.getKey() + ": " + input.getValue();
- }
- }
-
- /**
- * Options supported by {@link WordCount}.
- * <p>
- * Inherits standard configuration options.
- */
- public interface Options extends PipelineOptions, FlinkPipelineOptions {
- @Description("Path of the file to read from")
- @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt")
- String getInput();
- void setInput(String value);
-
- @Description("Path of the file to write to")
- String getOutput();
- void setOutput(String value);
- }
-
- public static void main(String[] args) {
-
- Options options = PipelineOptionsFactory.fromArgs(args).withValidation()
- .as(Options.class);
- options.setRunner(FlinkPipelineRunner.class);
-
- Pipeline p = Pipeline.create(options);
-
- p.apply(TextIO.Read.named("ReadLines").from(options.getInput()))
- .apply(new CountWords())
- .apply(MapElements.via(new FormatAsTextFn()))
- .apply(TextIO.Write.named("WriteCounts").to(options.getOutput()));
-
- p.run();
- }
+ @Override
+ public PCollection<KV<String, Long>> apply(PCollection<String> lines) {
+
+ // Convert lines of text into individual words.
+ PCollection<String> words = lines.apply(
+ ParDo.of(new ExtractWordsFn()));
+
+ // Count the number of times each word occurs.
+ PCollection<KV<String, Long>> wordCounts =
+ words.apply(Count.<String>perElement());
+
+ return wordCounts;
+ }
+ }
+
+ /** A SimpleFunction that converts a Word and Count into a printable string. */
+ public static class FormatAsTextFn extends SimpleFunction<KV<String, Long>, String> {
+ @Override
+ public String apply(KV<String, Long> input) {
+ return input.getKey() + ": " + input.getValue();
+ }
+ }
+
+ /**
+ * Options supported by {@link WordCount}.
+ * <p>
+ * Inherits standard configuration options.
+ */
+ public interface Options extends PipelineOptions, FlinkPipelineOptions {
+ @Description("Path of the file to read from")
+ @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt")
+ String getInput();
+ void setInput(String value);
+
+ @Description("Path of the file to write to")
+ String getOutput();
+ void setOutput(String value);
+ }
+
+ public static void main(String[] args) {
+
+ Options options = PipelineOptionsFactory.fromArgs(args).withValidation()
+ .as(Options.class);
+ options.setRunner(FlinkPipelineRunner.class);
+
+ Pipeline p = Pipeline.create(options);
+
+ p.apply(TextIO.Read.named("ReadLines").from(options.getInput()))
+ .apply(new CountWords())
+ .apply(MapElements.via(new FormatAsTextFn()))
+ .apply(TextIO.Write.named("WriteCounts").to(options.getOutput()));
+
+ p.run();
+ }
}
[13/50] [abbrv] incubator-beam git commit: [tests] add streaming mode
to TestPipeline
Posted by da...@apache.org.
[tests] add streaming mode to TestPipeline
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/0be42cbf
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/0be42cbf
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/0be42cbf
Branch: refs/heads/master
Commit: 0be42cbf39ca0740e7f8e5f8faf38aa9126e8cf6
Parents: f0cb5f0
Author: Max <ma...@posteo.de>
Authored: Mon Feb 22 16:31:23 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
.../flink/dataflow/FlinkPipelineRunner.java | 3 +-
.../flink/dataflow/FlinkTestPipeline.java | 33 ++++++++++++++++++--
2 files changed, 32 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0be42cbf/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
index f57fed2..ebd2691 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
@@ -150,11 +150,12 @@ public class FlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {
*
* @return The newly created runner.
*/
- public static FlinkPipelineRunner createForTest() {
+ public static FlinkPipelineRunner createForTest(boolean streaming) {
FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
// we use [auto] for testing since this will make it pick up the Testing
// ExecutionEnvironment
options.setFlinkMaster("[auto]");
+ options.setStreaming(streaming);
return new FlinkPipelineRunner(options);
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/0be42cbf/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
index a1f66c7..109b1ff 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
@@ -27,14 +27,41 @@ import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
public class FlinkTestPipeline extends Pipeline {
/**
- * Creates and returns a new test pipeline.
+ * Creates and returns a new test pipeline for batch execution.
*
* <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
* {@link Pipeline#run} to execute the pipeline and check the tests.
*/
public static FlinkTestPipeline create() {
- FlinkPipelineRunner flinkRunner = FlinkPipelineRunner.createForTest();
- return new FlinkTestPipeline(flinkRunner, flinkRunner.getPipelineOptions());
+ return create(false);
+ }
+
+ /**
+ * Creates and returns a new test pipeline for streaming execution.
+ *
+ * <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
+ * {@link Pipeline#run} to execute the pipeline and check the tests.
+ *
+ * @return The Test Pipeline
+ */
+ public static FlinkTestPipeline createStreaming() {
+ return create(true);
+ }
+
+ /**
+ * Creates and returns a new test pipeline for streaming or batch execution.
+ *
+ * <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
+ * {@link Pipeline#run} to execute the pipeline and check the tests.
+ *
+ * @param streaming True for streaming mode, False for batch
+ * @return The Test Pipeline
+ */
+ public static FlinkTestPipeline create(boolean streaming) {
+ FlinkPipelineRunner flinkRunner = FlinkPipelineRunner.createForTest(streaming);
+ FlinkPipelineOptions pipelineOptions = flinkRunner.getPipelineOptions();
+ pipelineOptions.setStreaming(streaming);
+ return new FlinkTestPipeline(flinkRunner, pipelineOptions);
}
private FlinkTestPipeline(PipelineRunner<? extends PipelineResult> runner, PipelineOptions
[11/50] [abbrv] incubator-beam git commit: [cleanup] various small
improvements
Posted by da...@apache.org.
[cleanup] various small improvements
- removed duplicate declarations in pom.xml
- removed reference to junit.framework.*
- removed 'static' from interface declarations
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/602d8fe9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/602d8fe9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/602d8fe9
Branch: refs/heads/master
Commit: 602d8fe9b69dba2300a573559e7b60e286f116d1
Parents: 70ae13c
Author: smarthi <sm...@apache.org>
Authored: Thu Feb 11 17:33:51 2016 -0500
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
runners/flink/pom.xml | 5 -----
.../flink/dataflow/FlinkJobExecutionEnvironment.java | 3 +--
.../flink/dataflow/examples/streaming/AutoComplete.java | 6 ++----
.../flink/dataflow/examples/streaming/JoinExamples.java | 6 +++---
.../examples/streaming/KafkaWindowedWordCountExample.java | 2 +-
.../flink/dataflow/examples/streaming/WindowedWordCount.java | 3 +--
.../dataflow/translation/FlinkBatchPipelineTranslator.java | 3 ---
.../dataflow/translation/FlinkBatchTransformTranslators.java | 2 +-
.../translation/FlinkStreamingPipelineTranslator.java | 4 ----
.../translation/FlinkStreamingTransformTranslators.java | 5 ++---
.../translation/FlinkStreamingTranslationContext.java | 6 ++----
.../functions/FlinkCoGroupKeyedListAggregator.java | 2 +-
.../flink/dataflow/translation/wrappers/SinkOutputFormat.java | 2 --
.../dataflow/translation/wrappers/SourceInputFormat.java | 7 +++----
.../java/com/dataartisans/flink/dataflow/WriteSinkITCase.java | 2 +-
.../com/dataartisans/flink/dataflow/util/JoinExamples.java | 2 +-
16 files changed, 19 insertions(+), 41 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/602d8fe9/runners/flink/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml
index 14693b8..cb784a0 100644
--- a/runners/flink/pom.xml
+++ b/runners/flink/pom.xml
@@ -73,11 +73,6 @@
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java</artifactId>
<version>${flink.version}</version>
- </dependency>
- <dependency>
- <groupId>org.apache.flink</groupId>
- <artifactId>flink-streaming-java</artifactId>
- <version>${flink.version}</version>
<scope>test</scope>
<type>test-jar</type>
</dependency>
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/602d8fe9/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java
index 66d60fa..91b2f64 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java
@@ -23,7 +23,6 @@ import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.java.CollectionEnvironment;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.environment.RemoteStreamEnvironment;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -224,7 +223,7 @@ public class FlinkJobExecutionEnvironment {
this.flinkStreamEnv.getConfig().setExecutionRetryDelay(3000);
}
- private final void checkInitializationState() {
+ private void checkInitializationState() {
if (this.options == null) {
throw new IllegalStateException("FlinkJobExecutionEnvironment is not initialized yet.");
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/602d8fe9/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java
index 0245a7b..711d9fb 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java
@@ -36,8 +36,6 @@ import org.joda.time.Duration;
import java.io.IOException;
import java.util.List;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
/**
* To run the example, first open a socket on a terminal by executing the command:
@@ -242,7 +240,7 @@ public class AutoComplete {
public void processElement(ProcessContext c) {
String word = c.element().value;
for (int i = minPrefix; i <= Math.min(word.length(), maxPrefix); i++) {
- KV kv = KV.of(word.substring(0, i), c.element());
+ KV<String, CompletionCandidate> kv = KV.of(word.substring(0, i), c.element());
c.output(kv);
}
}
@@ -349,7 +347,7 @@ public class AutoComplete {
*
* <p> Inherits standard Dataflow configuration options.
*/
- private static interface Options extends WindowedWordCount.StreamingWordCountOptions {
+ private interface Options extends WindowedWordCount.StreamingWordCountOptions {
@Description("Whether to use the recursive algorithm")
@Default.Boolean(true)
Boolean getRecursive();
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/602d8fe9/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
index b0cc4fa..9a5db64 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
@@ -53,8 +53,8 @@ public class JoinExamples {
static PCollection<String> joinEvents(PCollection<String> streamA,
PCollection<String> streamB) throws Exception {
- final TupleTag<String> firstInfoTag = new TupleTag<String>();
- final TupleTag<String> secondInfoTag = new TupleTag<String>();
+ final TupleTag<String> firstInfoTag = new TupleTag<>();
+ final TupleTag<String> secondInfoTag = new TupleTag<>();
// transform both input collections to tuple collections, where the keys are country
// codes in both cases.
@@ -118,7 +118,7 @@ public class JoinExamples {
}
}
- private static interface Options extends WindowedWordCount.StreamingWordCountOptions {
+ private interface Options extends WindowedWordCount.StreamingWordCountOptions {
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/602d8fe9/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
index 46c9bd6..42d3d88 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
@@ -72,7 +72,7 @@ public class KafkaWindowedWordCountExample {
}
}
- public static interface KafkaStreamingWordCountOptions extends WindowedWordCount.StreamingWordCountOptions {
+ public interface KafkaStreamingWordCountOptions extends WindowedWordCount.StreamingWordCountOptions {
@Description("The Kafka topic to read from")
@Default.String(KAFKA_TOPIC)
String getKafkaTopic();
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/602d8fe9/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
index 1d4a44b..b539245 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
@@ -28,7 +28,6 @@ import com.google.cloud.dataflow.sdk.values.KV;
import com.google.cloud.dataflow.sdk.values.PCollection;
import org.joda.time.Duration;
-import org.joda.time.Instant;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -81,7 +80,7 @@ public class WindowedWordCount {
}
}
- public static interface StreamingWordCountOptions extends com.dataartisans.flink.dataflow.examples.WordCount.Options {
+ public interface StreamingWordCountOptions extends com.dataartisans.flink.dataflow.examples.WordCount.Options {
@Description("Sliding window duration, in seconds")
@Default.Long(WINDOW_SIZE)
Long getWindowSize();
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/602d8fe9/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java
index 8c0183e..a1e4410 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java
@@ -115,9 +115,6 @@ public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
}
private <T extends PTransform<?, ?>> void applyBatchTransform(PTransform<?, ?> transform, TransformTreeNode node, BatchTransformTranslator<?> translator) {
- if (this.batchContext == null) {
- throw new IllegalStateException("The FlinkPipelineTranslator is not yet initialized.");
- }
@SuppressWarnings("unchecked")
T typedTransform = (T) transform;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/602d8fe9/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
index 8f64730..9a43d05 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
@@ -283,7 +283,7 @@ public class FlinkBatchTransformTranslators {
private static class ConsoleIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator<ConsoleIO.Write.Bound> {
@Override
public void translateNode(ConsoleIO.Write.Bound transform, FlinkBatchTranslationContext context) {
- PValue input = (PValue) context.getInput(transform);
+ PValue input = context.getInput(transform);
DataSet<?> inputDataSet = context.getInputDataSet(input);
inputDataSet.printOnTaskManager(transform.getName());
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/602d8fe9/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
index c8760c7..a8f4226 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
@@ -19,7 +19,6 @@ import com.google.cloud.dataflow.sdk.options.PipelineOptions;
import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
import com.google.cloud.dataflow.sdk.values.PValue;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -101,9 +100,6 @@ public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
}
private <T extends PTransform<?, ?>> void applyStreamingTransform(PTransform<?, ?> transform, TransformTreeNode node, StreamTransformTranslator<?> translator) {
- if (this.streamingContext == null) {
- throw new IllegalStateException("The FlinkPipelineTranslator is not yet initialized.");
- }
@SuppressWarnings("unchecked")
T typedTransform = (T) transform;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/602d8fe9/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
index 4c8cd4b..17583cd 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
@@ -76,8 +76,7 @@ public class FlinkStreamingTransformTranslators {
}
public static FlinkStreamingPipelineTranslator.StreamTransformTranslator<?> getTranslator(PTransform<?, ?> transform) {
- FlinkStreamingPipelineTranslator.StreamTransformTranslator<?> translator = TRANSLATORS.get(transform.getClass());
- return translator;
+ return TRANSLATORS.get(transform.getClass());
}
// --------------------------------------------------------------------------------------------
@@ -123,7 +122,7 @@ public class FlinkStreamingTransformTranslators {
public void translateNode(Read.Unbounded<T> transform, FlinkStreamingTranslationContext context) {
PCollection<T> output = context.getOutput(transform);
- DataStream<WindowedValue<T>> source = null;
+ DataStream<WindowedValue<T>> source;
if (transform.getSource().getClass().equals(UnboundedFlinkSource.class)) {
UnboundedFlinkSource flinkSource = (UnboundedFlinkSource) transform.getSource();
source = context.getExecutionEnvironment()
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/602d8fe9/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
index 83ea575..df68e50 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
@@ -74,13 +74,11 @@ public class FlinkStreamingTranslationContext {
@SuppressWarnings("unchecked")
public <I extends PInput> I getInput(PTransform<I, ?> transform) {
- I input = (I) currentTransform.getInput();
- return input;
+ return (I) currentTransform.getInput();
}
@SuppressWarnings("unchecked")
public <O extends POutput> O getOutput(PTransform<?, O> transform) {
- O output = (O) currentTransform.getOutput();
- return output;
+ return (O) currentTransform.getOutput();
}
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/602d8fe9/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java
index f859348..4c7fefd 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java
@@ -53,6 +53,6 @@ public class FlinkCoGroupKeyedListAggregator<K,V1,V2> implements CoGroupFunction
k = entry.getKey();
result.add(new RawUnionValue(index2, entry.getValue()));
}
- out.collect(KV.of(k, new CoGbkResult(schema, (List) result)));
+ out.collect(KV.of(k, new CoGbkResult(schema, result)));
}
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/602d8fe9/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java
index b10c86f..ec8c186 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java
@@ -23,7 +23,6 @@ import com.google.cloud.dataflow.sdk.io.Sink;
import com.google.cloud.dataflow.sdk.options.PipelineOptions;
import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions;
import com.google.cloud.dataflow.sdk.transforms.Write;
-import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.io.OutputFormat;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.util.AbstractID;
@@ -32,7 +31,6 @@ import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
import java.lang.reflect.Field;
-import java.util.UUID;
/**
* Wrapper class to use generic Write.Bound transforms as sinks.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/602d8fe9/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
index afb15da..b3eca96 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
@@ -45,7 +45,6 @@ public class SourceInputFormat<T> implements InputFormat<T, SourceInputSplit<T>>
private final BoundedSource<T> initialSource;
private transient PipelineOptions options;
- private final Coder<T> coder;
private BoundedSource.BoundedReader<T> reader = null;
private boolean reachedEnd = true;
@@ -53,7 +52,7 @@ public class SourceInputFormat<T> implements InputFormat<T, SourceInputSplit<T>>
public SourceInputFormat(BoundedSource<T> initialSource, PipelineOptions options, Coder<T> coder) {
this.initialSource = initialSource;
this.options = options;
- this.coder = coder;
+ Coder<T> coder1 = coder;
}
private void writeObject(ObjectOutputStream out)
@@ -111,12 +110,12 @@ public class SourceInputFormat<T> implements InputFormat<T, SourceInputSplit<T>>
@Override
@SuppressWarnings("unchecked")
public SourceInputSplit<T>[] createInputSplits(int numSplits) throws IOException {
- long desiredSizeBytes = 10000;
+ long desiredSizeBytes;
try {
desiredSizeBytes = initialSource.getEstimatedSizeBytes(options) / numSplits;
List<? extends Source<T>> shards = initialSource.splitIntoBundles(desiredSizeBytes,
options);
- List<SourceInputSplit<T>> splits = new ArrayList<SourceInputSplit<T>>();
+ List<SourceInputSplit<T>> splits = new ArrayList<>();
int splitCount = 0;
for (Source<T> shard: shards) {
splits.add(new SourceInputSplit<>(shard, splitCount++));
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/602d8fe9/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java
index c8302e8..205fe9b 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java
@@ -32,7 +32,7 @@ import java.io.File;
import java.io.PrintWriter;
import java.net.URI;
-import static junit.framework.Assert.*;
+import static org.junit.Assert.*;
/**
* Tests the translation of custom Write.Bound sinks.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/602d8fe9/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java
index 74f754b..aa5623d 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java
@@ -76,7 +76,7 @@ public class JoinExamples {
KV<String, CoGbkResult> e = c.element();
CoGbkResult val = e.getValue();
String countryCode = e.getKey();
- String countryName = "none";
+ String countryName;
countryName = e.getValue().getOnly(countryInfoTag, "Kostas");
for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) {
// Generate a string that combines information from both collection values
[03/50] [abbrv] incubator-beam git commit: [readme] add hint on how
to submit jar to cluster
Posted by da...@apache.org.
[readme] add hint on how to submit jar to cluster
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/517c1bd2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/517c1bd2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/517c1bd2
Branch: refs/heads/master
Commit: 517c1bd2656f2c1e6b93bae87081203396aaa281
Parents: d7bd7f4
Author: Max <ma...@posteo.de>
Authored: Tue Jan 19 14:33:55 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
runners/flink/README.md | 5 +++++
1 file changed, 5 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/517c1bd2/runners/flink/README.md
----------------------------------------------------------------------
diff --git a/runners/flink/README.md b/runners/flink/README.md
index e8c2170..8c06c1d 100644
--- a/runners/flink/README.md
+++ b/runners/flink/README.md
@@ -132,6 +132,11 @@ A fat jar is necessary if you want to submit your Dataflow code to a Flink clust
includes your program code but also Dataflow code which is necessary during runtime. Note that this
step is necessary because the Dataflow Runner is not part of Flink.
+You can then build the jar using `mvn clean package`. Please submit the fat jar in the `target`
+folder to the Flink cluster using the command-line utility like so:
+
+ ./bin/flink run /path/to/fat.jar
+
For more information, please visit the [Apache Flink Website](http://flink.apache.org) or contact
the [Mailinglists](http://flink.apache.org/community.html#mailing-lists).
[12/50] [abbrv] incubator-beam git commit: [cleanup] remove obsolete
code
Posted by da...@apache.org.
[cleanup] remove obsolete code
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/f0cb5f07
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/f0cb5f07
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/f0cb5f07
Branch: refs/heads/master
Commit: f0cb5f07361f6e6eca30fa66a1d80d205ee7d2b8
Parents: 602d8fe
Author: Max <ma...@posteo.de>
Authored: Wed Feb 17 13:23:26 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
.../translation/FlinkBatchTransformTranslators.java | 3 ++-
.../translation/wrappers/SourceInputFormat.java | 3 +--
.../wrappers/streaming/FlinkAbstractParDoWrapper.java | 14 --------------
.../flink/dataflow/JoinExamplesITCase.java | 7 -------
4 files changed, 3 insertions(+), 24 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f0cb5f07/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
index 9a43d05..d5c09b2 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
@@ -151,7 +151,8 @@ public class FlinkBatchTransformTranslators {
TypeInformation<T> typeInformation = context.getTypeInfo(output);
- DataSource<T> dataSource = new DataSource<>(context.getExecutionEnvironment(), new SourceInputFormat<>(source, context.getPipelineOptions(), coder), typeInformation, name);
+ DataSource<T> dataSource = new DataSource<>(context.getExecutionEnvironment(),
+ new SourceInputFormat<>(source, context.getPipelineOptions()), typeInformation, name);
context.setOutputDataSet(output, dataSource);
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f0cb5f07/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
index b3eca96..64dc072 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
@@ -49,10 +49,9 @@ public class SourceInputFormat<T> implements InputFormat<T, SourceInputSplit<T>>
private BoundedSource.BoundedReader<T> reader = null;
private boolean reachedEnd = true;
- public SourceInputFormat(BoundedSource<T> initialSource, PipelineOptions options, Coder<T> coder) {
+ public SourceInputFormat(BoundedSource<T> initialSource, PipelineOptions options) {
this.initialSource = initialSource;
this.options = options;
- Coder<T> coder1 = coder;
}
private void writeObject(ObjectOutputStream out)
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f0cb5f07/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
index 53bb177..71f9c7f 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
@@ -55,20 +55,6 @@ public abstract class FlinkAbstractParDoWrapper<IN, OUTDF, OUTFL> extends RichFl
this.windowingStrategy = windowingStrategy;
}
-// protected void writeObject(ObjectOutputStream out)
-// throws IOException, ClassNotFoundException {
-// out.defaultWriteObject();
-// ObjectMapper mapper = new ObjectMapper();
-// mapper.writeValue(out, options);
-// }
-//
-// protected void readObject(ObjectInputStream in)
-// throws IOException, ClassNotFoundException {
-// in.defaultReadObject();
-// ObjectMapper mapper = new ObjectMapper();
-// options = mapper.readValue(in, PipelineOptions.class);
-// }
-
private void initContext(DoFn<IN, OUTDF> function, Collector<WindowedValue<OUTFL>> outCollector) {
if (this.context == null) {
this.context = new DoFnProcessContext(function, outCollector);
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/f0cb5f07/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java
index dfcadc1..ed2ecf5 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java
@@ -53,13 +53,6 @@ public class JoinExamplesITCase extends JavaProgramTestBase {
};
static final List<TableRow> EVENT_ARRAY = Arrays.asList(EVENTS);
- private static final KV<String, String> kv1 = KV.of("VM",
- "Date: 20141212, Actor1: LAOS, url: http://www.chicagotribune.com");
- private static final KV<String, String> kv2 = KV.of("BE",
- "Date: 20141213, Actor1: AFGHANISTAN, url: http://cnn.com");
- private static final KV<String, String> kv3 = KV.of("BE", "Belgium");
- private static final KV<String, String> kv4 = KV.of("VM", "Vietnam");
-
private static final TableRow cc1 = new TableRow()
.set("FIPSCC", "VM").set("HumanName", "Vietnam");
private static final TableRow cc2 = new TableRow()
[33/50] [abbrv] incubator-beam git commit: Fixes the
GroupAlsoByWindowTest.
Posted by da...@apache.org.
Fixes the GroupAlsoByWindowTest.
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/52614ea3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/52614ea3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/52614ea3
Branch: refs/heads/master
Commit: 52614ea36a7431d83f907d99d3fb251c2f2b3551
Parents: 69f7623
Author: kl0u <kk...@gmail.com>
Authored: Wed Mar 2 16:09:53 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
.../streaming/GroupAlsoByWindowTest.java | 70 ++++++++++----------
1 file changed, 36 insertions(+), 34 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/52614ea3/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
index 434f827..01f9c32 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
@@ -116,7 +116,7 @@ public class GroupAlsoByWindowTest {
new Instant(initialTime + 1),
new IntervalWindow(new Instant(0), new Instant(2000)),
PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
- , initialTime));
+ , initialTime + 1));
expectedOutput.add(new Watermark(initialTime + 2000));
expectedOutput.add(new StreamRecord<>(
@@ -124,14 +124,15 @@ public class GroupAlsoByWindowTest {
new Instant(initialTime + 1999),
new IntervalWindow(new Instant(0), new Instant(2000)),
PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 1, 1))
- , initialTime));
+ , initialTime + 1999));
+
expectedOutput.add(new StreamRecord<>(
WindowedValue.of(KV.of("key1", 6),
new Instant(initialTime + 1999),
new IntervalWindow(new Instant(0), new Instant(2000)),
PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 2, 2))
- , initialTime));
+ , initialTime + 1999));
expectedOutput.add(new Watermark(initialTime + 4000));
TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
@@ -181,7 +182,7 @@ public class GroupAlsoByWindowTest {
new Instant(initialTime + 1),
new IntervalWindow(new Instant(1), new Instant(5700)),
PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
- , initialTime));
+ , initialTime + 1));
expectedOutput.add(new Watermark(initialTime + 6000));
expectedOutput.add(new StreamRecord<>(
@@ -189,7 +190,7 @@ public class GroupAlsoByWindowTest {
new Instant(initialTime + 6700),
new IntervalWindow(new Instant(1), new Instant(10900)),
PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
- , initialTime));
+ , initialTime + 6700));
expectedOutput.add(new Watermark(initialTime + 12000));
TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
@@ -210,13 +211,13 @@ public class GroupAlsoByWindowTest {
new Instant(initialTime + 5000),
new IntervalWindow(new Instant(0), new Instant(10000)),
PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime));
+ , initialTime + 5000));
expectedOutput.add(new StreamRecord<>(
WindowedValue.of(KV.of("key1", 6),
new Instant(initialTime + 1),
new IntervalWindow(new Instant(-5000), new Instant(5000)),
PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime));
+ , initialTime + 1));
expectedOutput.add(new Watermark(initialTime + 10000));
expectedOutput.add(new StreamRecord<>(
@@ -224,19 +225,19 @@ public class GroupAlsoByWindowTest {
new Instant(initialTime + 15000),
new IntervalWindow(new Instant(10000), new Instant(20000)),
PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime));
+ , initialTime + 15000));
expectedOutput.add(new StreamRecord<>(
WindowedValue.of(KV.of("key1", 3),
new Instant(initialTime + 10000),
new IntervalWindow(new Instant(5000), new Instant(15000)),
PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime));
+ , initialTime + 10000));
expectedOutput.add(new StreamRecord<>(
WindowedValue.of(KV.of("key2", 1),
new Instant(initialTime + 19500),
new IntervalWindow(new Instant(10000), new Instant(20000)),
PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime));
+ , initialTime + 19500));
expectedOutput.add(new Watermark(initialTime + 20000));
expectedOutput.add(new StreamRecord<>(
@@ -250,13 +251,13 @@ public class GroupAlsoByWindowTest {
*/
new IntervalWindow(new Instant(15000), new Instant(25000)),
PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime));
+ , initialTime + 20000));
expectedOutput.add(new StreamRecord<>(
WindowedValue.of(KV.of("key1", 8),
new Instant(initialTime + 20000),
new IntervalWindow(new Instant(15000), new Instant(25000)),
PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime));
+ , initialTime + 20000));
expectedOutput.add(new Watermark(initialTime + 25000));
TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
@@ -272,13 +273,13 @@ public class GroupAlsoByWindowTest {
ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6),
- new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime));
+ new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 1));
expectedOutput.add(new Watermark(initialTime + 10000));
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11),
- new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime));
+ new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 10000));
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime));
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500));
expectedOutput.add(new Watermark(initialTime + 20000));
TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
@@ -295,13 +296,13 @@ public class GroupAlsoByWindowTest {
ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime));
+ new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime + 1));
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime));
+ new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime + 10000));
expectedOutput.add(new Watermark(initialTime + 10000));
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME, 0, 0)), initialTime));
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME, 0, 0)), initialTime + 19500));
expectedOutput.add(new Watermark(initialTime + 20000));
TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
@@ -326,21 +327,21 @@ public class GroupAlsoByWindowTest {
* */
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime));
+ new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 1));
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime));
+ new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 10000));
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime));
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime + 19500));
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1),
- new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime));
+ new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime + 1200));
expectedOutput.add(new Watermark(initialTime + 10000));
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime));
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime + 19500));
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime));
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500));
expectedOutput.add(new Watermark(initialTime + 20000));
TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
@@ -348,7 +349,7 @@ public class GroupAlsoByWindowTest {
testHarness.close();
}
- // Disabled
+ @Test
public void testCompoundAccumulatingPanesProgram() throws Exception {
WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategyAcc;
long initialTime = 0L;
@@ -357,21 +358,21 @@ public class GroupAlsoByWindowTest {
ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime));
+ new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 1));
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime));
+ new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 10000));
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 10),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime));
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime + 19500));
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6),
- new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime));
+ new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime + 1200));
expectedOutput.add(new Watermark(initialTime + 10000));
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime));
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime + 19500));
expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime));
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500));
expectedOutput.add(new Watermark(initialTime + 20000));
TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
@@ -434,11 +435,12 @@ public class GroupAlsoByWindowTest {
StreamRecord<WindowedValue<KV<String, Integer>>> sr0 = (StreamRecord<WindowedValue<KV<String, Integer>>>) o1;
StreamRecord<WindowedValue<KV<String, Integer>>> sr1 = (StreamRecord<WindowedValue<KV<String, Integer>>>) o2;
- if (sr0.getTimestamp() != sr1.getTimestamp()) {
- return (int) (sr0.getTimestamp() - sr1.getTimestamp());
+ int comparison = (int) (sr0.getValue().getTimestamp().getMillis() - sr1.getValue().getTimestamp().getMillis());
+ if (comparison != 0) {
+ return comparison;
}
- int comparison = sr0.getValue().getValue().getKey().compareTo(sr1.getValue().getValue().getKey());
+ comparison = sr0.getValue().getValue().getKey().compareTo(sr1.getValue().getValue().getKey());
if(comparison == 0) {
comparison = Integer.compare(
sr0.getValue().getValue().getValue(),
[49/50] [abbrv] incubator-beam git commit: [flink] update license
headers
Posted by da...@apache.org.
[flink] update license headers
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/28fcfd7a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/28fcfd7a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/28fcfd7a
Branch: refs/heads/master
Commit: 28fcfd7a0b6afbb843126218c01475b960fbf061
Parents: 51bec31
Author: Maximilian Michels <mx...@apache.org>
Authored: Wed Mar 2 23:21:57 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
runners/flink/LICENSE | 202 -------------------
.../FlinkPipelineExecutionEnvironment.java | 12 +-
.../runners/flink/FlinkPipelineOptions.java | 12 +-
.../beam/runners/flink/FlinkPipelineRunner.java | 12 +-
.../beam/runners/flink/FlinkRunnerResult.java | 12 +-
.../beam/runners/flink/examples/WordCount.java | 12 +-
.../KafkaWindowedWordCountExample.java | 12 +-
.../examples/streaming/WindowedWordCount.java | 12 +-
.../apache/beam/runners/flink/io/ConsoleIO.java | 12 +-
.../FlinkBatchPipelineTranslator.java | 14 +-
.../FlinkBatchTransformTranslators.java | 14 +-
.../FlinkBatchTranslationContext.java | 14 +-
.../translation/FlinkPipelineTranslator.java | 12 +-
.../FlinkStreamingPipelineTranslator.java | 12 +-
.../FlinkStreamingTransformTranslators.java | 12 +-
.../FlinkStreamingTranslationContext.java | 12 +-
.../FlinkCoGroupKeyedListAggregator.java | 12 +-
.../functions/FlinkCreateFunction.java | 12 +-
.../functions/FlinkDoFnFunction.java | 12 +-
.../FlinkKeyedListAggregationFunction.java | 12 +-
.../functions/FlinkMultiOutputDoFnFunction.java | 12 +-
.../FlinkMultiOutputPruningFunction.java | 12 +-
.../functions/FlinkPartialReduceFunction.java | 14 +-
.../functions/FlinkReduceFunction.java | 14 +-
.../translation/types/CoderComparator.java | 14 +-
.../translation/types/CoderTypeInformation.java | 14 +-
.../translation/types/CoderTypeSerializer.java | 14 +-
.../types/InspectableByteArrayOutputStream.java | 14 +-
.../translation/types/KvCoderComperator.java | 14 +-
.../types/KvCoderTypeInformation.java | 14 +-
.../types/VoidCoderTypeSerializer.java | 14 +-
.../wrappers/CombineFnAggregatorWrapper.java | 14 +-
.../wrappers/DataInputViewWrapper.java | 14 +-
.../wrappers/DataOutputViewWrapper.java | 14 +-
.../SerializableFnAggregatorWrapper.java | 14 +-
.../translation/wrappers/SinkOutputFormat.java | 14 +-
.../translation/wrappers/SourceInputFormat.java | 14 +-
.../translation/wrappers/SourceInputSplit.java | 14 +-
.../streaming/FlinkAbstractParDoWrapper.java | 12 +-
.../FlinkGroupAlsoByWindowWrapper.java | 12 +-
.../streaming/FlinkGroupByKeyWrapper.java | 12 +-
.../streaming/FlinkParDoBoundMultiWrapper.java | 12 +-
.../streaming/FlinkParDoBoundWrapper.java | 12 +-
.../io/FlinkStreamingCreateFunction.java | 12 +-
.../streaming/io/UnboundedFlinkSource.java | 12 +-
.../streaming/io/UnboundedSocketSource.java | 12 +-
.../streaming/io/UnboundedSourceWrapper.java | 12 +-
.../state/AbstractFlinkTimerInternals.java | 12 +-
.../streaming/state/FlinkStateInternals.java | 12 +-
.../streaming/state/StateCheckpointReader.java | 12 +-
.../streaming/state/StateCheckpointUtils.java | 12 +-
.../streaming/state/StateCheckpointWriter.java | 12 +-
.../wrappers/streaming/state/StateType.java | 12 +-
.../flink/src/main/resources/log4j.properties | 24 ++-
.../apache/beam/runners/flink/AvroITCase.java | 12 +-
.../beam/runners/flink/FlattenizeITCase.java | 12 +-
.../beam/runners/flink/FlinkTestPipeline.java | 12 +-
.../beam/runners/flink/JoinExamplesITCase.java | 12 +-
.../runners/flink/MaybeEmptyTestITCase.java | 12 +-
.../runners/flink/ParDoMultiOutputITCase.java | 12 +-
.../beam/runners/flink/ReadSourceITCase.java | 12 +-
.../flink/RemoveDuplicatesEmptyITCase.java | 12 +-
.../runners/flink/RemoveDuplicatesITCase.java | 12 +-
.../beam/runners/flink/SideInputITCase.java | 12 +-
.../apache/beam/runners/flink/TfIdfITCase.java | 12 +-
.../beam/runners/flink/WordCountITCase.java | 12 +-
.../runners/flink/WordCountJoin2ITCase.java | 12 +-
.../runners/flink/WordCountJoin3ITCase.java | 12 +-
.../beam/runners/flink/WriteSinkITCase.java | 12 +-
.../flink/streaming/GroupAlsoByWindowTest.java | 12 +-
.../flink/streaming/GroupByNullKeyTest.java | 12 +-
.../flink/streaming/StateSerializationTest.java | 12 +-
.../streaming/TopWikipediaSessionsITCase.java | 12 +-
.../beam/runners/flink/util/JoinExamples.java | 12 +-
74 files changed, 517 insertions(+), 611 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/LICENSE
----------------------------------------------------------------------
diff --git a/runners/flink/LICENSE b/runners/flink/LICENSE
deleted file mode 100644
index e093461..0000000
--- a/runners/flink/LICENSE
+++ /dev/null
@@ -1,202 +0,0 @@
- Apache License
- Version 2.0, January 2004
- http://www.apache.org/licenses/
-
- TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
- 1. Definitions.
-
- "License" shall mean the terms and conditions for use, reproduction,
- and distribution as defined by Sections 1 through 9 of this document.
-
- "Licensor" shall mean the copyright owner or entity authorized by
- the copyright owner that is granting the License.
-
- "Legal Entity" shall mean the union of the acting entity and all
- other entities that control, are controlled by, or are under common
- control with that entity. For the purposes of this definition,
- "control" means (i) the power, direct or indirect, to cause the
- direction or management of such entity, whether by contract or
- otherwise, or (ii) ownership of fifty percent (50%) or more of the
- outstanding shares, or (iii) beneficial ownership of such entity.
-
- "You" (or "Your") shall mean an individual or Legal Entity
- exercising permissions granted by this License.
-
- "Source" form shall mean the preferred form for making modifications,
- including but not limited to software source code, documentation
- source, and configuration files.
-
- "Object" form shall mean any form resulting from mechanical
- transformation or translation of a Source form, including but
- not limited to compiled object code, generated documentation,
- and conversions to other media types.
-
- "Work" shall mean the work of authorship, whether in Source or
- Object form, made available under the License, as indicated by a
- copyright notice that is included in or attached to the work
- (an example is provided in the Appendix below).
-
- "Derivative Works" shall mean any work, whether in Source or Object
- form, that is based on (or derived from) the Work and for which the
- editorial revisions, annotations, elaborations, or other modifications
- represent, as a whole, an original work of authorship. For the purposes
- of this License, Derivative Works shall not include works that remain
- separable from, or merely link (or bind by name) to the interfaces of,
- the Work and Derivative Works thereof.
-
- "Contribution" shall mean any work of authorship, including
- the original version of the Work and any modifications or additions
- to that Work or Derivative Works thereof, that is intentionally
- submitted to Licensor for inclusion in the Work by the copyright owner
- or by an individual or Legal Entity authorized to submit on behalf of
- the copyright owner. For the purposes of this definition, "submitted"
- means any form of electronic, verbal, or written communication sent
- to the Licensor or its representatives, including but not limited to
- communication on electronic mailing lists, source code control systems,
- and issue tracking systems that are managed by, or on behalf of, the
- Licensor for the purpose of discussing and improving the Work, but
- excluding communication that is conspicuously marked or otherwise
- designated in writing by the copyright owner as "Not a Contribution."
-
- "Contributor" shall mean Licensor and any individual or Legal Entity
- on behalf of whom a Contribution has been received by Licensor and
- subsequently incorporated within the Work.
-
- 2. Grant of Copyright License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- copyright license to reproduce, prepare Derivative Works of,
- publicly display, publicly perform, sublicense, and distribute the
- Work and such Derivative Works in Source or Object form.
-
- 3. Grant of Patent License. Subject to the terms and conditions of
- this License, each Contributor hereby grants to You a perpetual,
- worldwide, non-exclusive, no-charge, royalty-free, irrevocable
- (except as stated in this section) patent license to make, have made,
- use, offer to sell, sell, import, and otherwise transfer the Work,
- where such license applies only to those patent claims licensable
- by such Contributor that are necessarily infringed by their
- Contribution(s) alone or by combination of their Contribution(s)
- with the Work to which such Contribution(s) was submitted. If You
- institute patent litigation against any entity (including a
- cross-claim or counterclaim in a lawsuit) alleging that the Work
- or a Contribution incorporated within the Work constitutes direct
- or contributory patent infringement, then any patent licenses
- granted to You under this License for that Work shall terminate
- as of the date such litigation is filed.
-
- 4. Redistribution. You may reproduce and distribute copies of the
- Work or Derivative Works thereof in any medium, with or without
- modifications, and in Source or Object form, provided that You
- meet the following conditions:
-
- (a) You must give any other recipients of the Work or
- Derivative Works a copy of this License; and
-
- (b) You must cause any modified files to carry prominent notices
- stating that You changed the files; and
-
- (c) You must retain, in the Source form of any Derivative Works
- that You distribute, all copyright, patent, trademark, and
- attribution notices from the Source form of the Work,
- excluding those notices that do not pertain to any part of
- the Derivative Works; and
-
- (d) If the Work includes a "NOTICE" text file as part of its
- distribution, then any Derivative Works that You distribute must
- include a readable copy of the attribution notices contained
- within such NOTICE file, excluding those notices that do not
- pertain to any part of the Derivative Works, in at least one
- of the following places: within a NOTICE text file distributed
- as part of the Derivative Works; within the Source form or
- documentation, if provided along with the Derivative Works; or,
- within a display generated by the Derivative Works, if and
- wherever such third-party notices normally appear. The contents
- of the NOTICE file are for informational purposes only and
- do not modify the License. You may add Your own attribution
- notices within Derivative Works that You distribute, alongside
- or as an addendum to the NOTICE text from the Work, provided
- that such additional attribution notices cannot be construed
- as modifying the License.
-
- You may add Your own copyright statement to Your modifications and
- may provide additional or different license terms and conditions
- for use, reproduction, or distribution of Your modifications, or
- for any such Derivative Works as a whole, provided Your use,
- reproduction, and distribution of the Work otherwise complies with
- the conditions stated in this License.
-
- 5. Submission of Contributions. Unless You explicitly state otherwise,
- any Contribution intentionally submitted for inclusion in the Work
- by You to the Licensor shall be under the terms and conditions of
- this License, without any additional terms or conditions.
- Notwithstanding the above, nothing herein shall supersede or modify
- the terms of any separate license agreement you may have executed
- with Licensor regarding such Contributions.
-
- 6. Trademarks. This License does not grant permission to use the trade
- names, trademarks, service marks, or product names of the Licensor,
- except as required for reasonable and customary use in describing the
- origin of the Work and reproducing the content of the NOTICE file.
-
- 7. Disclaimer of Warranty. Unless required by applicable law or
- agreed to in writing, Licensor provides the Work (and each
- Contributor provides its Contributions) on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
- implied, including, without limitation, any warranties or conditions
- of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
- PARTICULAR PURPOSE. You are solely responsible for determining the
- appropriateness of using or redistributing the Work and assume any
- risks associated with Your exercise of permissions under this License.
-
- 8. Limitation of Liability. In no event and under no legal theory,
- whether in tort (including negligence), contract, or otherwise,
- unless required by applicable law (such as deliberate and grossly
- negligent acts) or agreed to in writing, shall any Contributor be
- liable to You for damages, including any direct, indirect, special,
- incidental, or consequential damages of any character arising as a
- result of this License or out of the use or inability to use the
- Work (including but not limited to damages for loss of goodwill,
- work stoppage, computer failure or malfunction, or any and all
- other commercial damages or losses), even if such Contributor
- has been advised of the possibility of such damages.
-
- 9. Accepting Warranty or Additional Liability. While redistributing
- the Work or Derivative Works thereof, You may choose to offer,
- and charge a fee for, acceptance of support, warranty, indemnity,
- or other liability obligations and/or rights consistent with this
- License. However, in accepting such obligations, You may act only
- on Your own behalf and on Your sole responsibility, not on behalf
- of any other Contributor, and only if You agree to indemnify,
- defend, and hold each Contributor harmless for any liability
- incurred by, or claims asserted against, such Contributor by reason
- of your accepting any such warranty or additional liability.
-
- END OF TERMS AND CONDITIONS
-
- APPENDIX: How to apply the Apache License to your work.
-
- To apply the Apache License to your work, attach the following
- boilerplate notice, with the fields enclosed by brackets "{}"
- replaced with your own identifying information. (Don't include
- the brackets!) The text should be enclosed in the appropriate
- comment syntax for the file format. We also recommend that a
- file or class name and description of purpose be included on the
- same "printed page" as the copyright notice for easier
- identification within third-party archives.
-
- Copyright {yyyy} {name of copyright owner}
-
- 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://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.
-
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
index 02a49b9..8825ed3 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
index bf83353..2f4b3ea 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
index 3c33d20..fe773d9 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
index c2329a6..8fd08ec 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
index ba46301..7d12fed 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
index fa0c8e9..55cdc22 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
index 6af044d..7eb69ba 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java
index cd25ba3..71e3b54 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
index 5201423..9b47a08 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java
index f33e4f5..48c783d 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java
index fd99833..2294318 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java
index efe217f..9407bf5 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
index 21a8133..60fba0f 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
index 1083848..bdefeaf 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java
index 83b19ee..f6bdecd 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java
index 0e855ad..d5562b8 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java
index 32efe41..56af397 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
index 9e8b7cd..fe77e64 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java
index 56fa88d..f92f888 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java
index bd8a968..ca667ee 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java
index 3e1cb65..37de37e 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java
index 1ff06ba..2de681b 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java
index 580ac01..29193a2 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java
index ecfb95d..1249036 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
index 8880b48..f9d4dcd 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
index 481ee31..4e81054 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java
index 619fa55..36b5ba3 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java
index 4599c6a..3912295 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java
index 7a0d999..8862d48 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java
index c7b6ea2..8bc3620 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java
index 815765c..445d411 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java
index b56a90e..6a3cf50 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
index 513d7f8..6bd2240 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java
index 0d03f9f..4409586 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java
index d0423b9..4c2475d 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
index 2d62416..cd5cd40 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
index 1b45ad7..cde2b35 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
@@ -1,14 +1,14 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
index e2ceae6..10c8bbf 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
index 906a399..e115a15 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
index 61953a6..1a6a665 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
index cdf23f6..df7f953 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
index 3357cd5..2ed5620 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
index 2599e88..f6c243f 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java
index ddbc993..2857efd 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
index a24964a..1389e9d 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/28fcfd7a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
index 7c1ccdf..97084cf 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
@@ -1,9 +1,11 @@
/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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
+ * 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
*
[07/50] [abbrv] incubator-beam git commit: [runner] add streaming
support with checkpointing
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
new file mode 100644
index 0000000..8f64730
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
@@ -0,0 +1,593 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation;
+
+import com.dataartisans.flink.dataflow.io.ConsoleIO;
+import com.dataartisans.flink.dataflow.translation.functions.FlinkCoGroupKeyedListAggregator;
+import com.dataartisans.flink.dataflow.translation.functions.FlinkCreateFunction;
+import com.dataartisans.flink.dataflow.translation.functions.FlinkDoFnFunction;
+import com.dataartisans.flink.dataflow.translation.functions.FlinkKeyedListAggregationFunction;
+import com.dataartisans.flink.dataflow.translation.functions.FlinkMultiOutputDoFnFunction;
+import com.dataartisans.flink.dataflow.translation.functions.FlinkMultiOutputPruningFunction;
+import com.dataartisans.flink.dataflow.translation.functions.FlinkPartialReduceFunction;
+import com.dataartisans.flink.dataflow.translation.functions.FlinkReduceFunction;
+import com.dataartisans.flink.dataflow.translation.functions.UnionCoder;
+import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation;
+import com.dataartisans.flink.dataflow.translation.types.KvCoderTypeInformation;
+import com.dataartisans.flink.dataflow.translation.wrappers.SinkOutputFormat;
+import com.dataartisans.flink.dataflow.translation.wrappers.SourceInputFormat;
+import com.google.api.client.util.Maps;
+import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.io.AvroIO;
+import com.google.cloud.dataflow.sdk.io.BoundedSource;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.Flatten;
+import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.transforms.Write;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResultSchema;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
+import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.collect.Lists;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.io.AvroInputFormat;
+import org.apache.flink.api.java.io.AvroOutputFormat;
+import org.apache.flink.api.java.io.TextInputFormat;
+import org.apache.flink.api.java.operators.CoGroupOperator;
+import org.apache.flink.api.java.operators.DataSink;
+import org.apache.flink.api.java.operators.DataSource;
+import org.apache.flink.api.java.operators.FlatMapOperator;
+import org.apache.flink.api.java.operators.GroupCombineOperator;
+import org.apache.flink.api.java.operators.GroupReduceOperator;
+import org.apache.flink.api.java.operators.Grouping;
+import org.apache.flink.api.java.operators.Keys;
+import org.apache.flink.api.java.operators.MapPartitionOperator;
+import org.apache.flink.api.java.operators.UnsortedGrouping;
+import org.apache.flink.core.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Translators for transforming
+ * Dataflow {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s to
+ * Flink {@link org.apache.flink.api.java.DataSet}s
+ */
+public class FlinkBatchTransformTranslators {
+
+ // --------------------------------------------------------------------------------------------
+ // Transform Translator Registry
+ // --------------------------------------------------------------------------------------------
+
+ @SuppressWarnings("rawtypes")
+ private static final Map<Class<? extends PTransform>, FlinkBatchPipelineTranslator.BatchTransformTranslator> TRANSLATORS = new HashMap<>();
+
+ // register the known translators
+ static {
+ TRANSLATORS.put(View.CreatePCollectionView.class, new CreatePCollectionViewTranslatorBatch());
+
+ TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslatorBatch());
+ // we don't need this because we translate the Combine.PerKey directly
+ //TRANSLATORS.put(Combine.GroupedValues.class, new CombineGroupedValuesTranslator());
+
+ TRANSLATORS.put(Create.Values.class, new CreateTranslatorBatch());
+
+ TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslatorBatch());
+
+ TRANSLATORS.put(GroupByKey.GroupByKeyOnly.class, new GroupByKeyOnlyTranslatorBatch());
+ // TODO we're currently ignoring windows here but that has to change in the future
+ TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch());
+
+ TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslatorBatch());
+ TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslatorBatch());
+
+ TRANSLATORS.put(CoGroupByKey.class, new CoGroupByKeyTranslatorBatch());
+
+ TRANSLATORS.put(AvroIO.Read.Bound.class, new AvroIOReadTranslatorBatch());
+ TRANSLATORS.put(AvroIO.Write.Bound.class, new AvroIOWriteTranslatorBatch());
+
+ TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslatorBatch());
+ TRANSLATORS.put(Write.Bound.class, new WriteSinkTranslatorBatch());
+
+ TRANSLATORS.put(TextIO.Read.Bound.class, new TextIOReadTranslatorBatch());
+ TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteTranslatorBatch());
+
+ // Flink-specific
+ TRANSLATORS.put(ConsoleIO.Write.Bound.class, new ConsoleIOWriteTranslatorBatch());
+
+ }
+
+
+ public static FlinkBatchPipelineTranslator.BatchTransformTranslator<?> getTranslator(PTransform<?, ?> transform) {
+ return TRANSLATORS.get(transform.getClass());
+ }
+
+ private static class ReadSourceTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Read.Bounded<T>> {
+
+ @Override
+ public void translateNode(Read.Bounded<T> transform, FlinkBatchTranslationContext context) {
+ String name = transform.getName();
+ BoundedSource<T> source = transform.getSource();
+ PCollection<T> output = context.getOutput(transform);
+ Coder<T> coder = output.getCoder();
+
+ TypeInformation<T> typeInformation = context.getTypeInfo(output);
+
+ DataSource<T> dataSource = new DataSource<>(context.getExecutionEnvironment(), new SourceInputFormat<>(source, context.getPipelineOptions(), coder), typeInformation, name);
+
+ context.setOutputDataSet(output, dataSource);
+ }
+ }
+
+ private static class AvroIOReadTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<AvroIO.Read.Bound<T>> {
+ private static final Logger LOG = LoggerFactory.getLogger(AvroIOReadTranslatorBatch.class);
+
+ @Override
+ public void translateNode(AvroIO.Read.Bound<T> transform, FlinkBatchTranslationContext context) {
+ String path = transform.getFilepattern();
+ String name = transform.getName();
+// Schema schema = transform.getSchema();
+ PValue output = context.getOutput(transform);
+
+ TypeInformation<T> typeInformation = context.getTypeInfo(output);
+
+ // This is super hacky, but unfortunately we cannot get the type otherwise
+ Class<T> extractedAvroType;
+ try {
+ Field typeField = transform.getClass().getDeclaredField("type");
+ typeField.setAccessible(true);
+ @SuppressWarnings("unchecked")
+ Class<T> avroType = (Class<T>) typeField.get(transform);
+ extractedAvroType = avroType;
+ } catch (NoSuchFieldException | IllegalAccessException e) {
+ // we know that the field is there and it is accessible
+ throw new RuntimeException("Could not access type from AvroIO.Bound", e);
+ }
+
+ DataSource<T> source = new DataSource<>(context.getExecutionEnvironment(),
+ new AvroInputFormat<>(new Path(path), extractedAvroType),
+ typeInformation, name);
+
+ context.setOutputDataSet(output, source);
+ }
+ }
+
+ private static class AvroIOWriteTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<AvroIO.Write.Bound<T>> {
+ private static final Logger LOG = LoggerFactory.getLogger(AvroIOWriteTranslatorBatch.class);
+
+ @Override
+ public void translateNode(AvroIO.Write.Bound<T> transform, FlinkBatchTranslationContext context) {
+ DataSet<T> inputDataSet = context.getInputDataSet(context.getInput(transform));
+ String filenamePrefix = transform.getFilenamePrefix();
+ String filenameSuffix = transform.getFilenameSuffix();
+ int numShards = transform.getNumShards();
+ String shardNameTemplate = transform.getShardNameTemplate();
+
+ // TODO: Implement these. We need Flink support for this.
+ LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.",
+ filenameSuffix);
+ LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
+
+ // This is super hacky, but unfortunately we cannot get the type otherwise
+ Class<T> extractedAvroType;
+ try {
+ Field typeField = transform.getClass().getDeclaredField("type");
+ typeField.setAccessible(true);
+ @SuppressWarnings("unchecked")
+ Class<T> avroType = (Class<T>) typeField.get(transform);
+ extractedAvroType = avroType;
+ } catch (NoSuchFieldException | IllegalAccessException e) {
+ // we know that the field is there and it is accessible
+ throw new RuntimeException("Could not access type from AvroIO.Bound", e);
+ }
+
+ DataSink<T> dataSink = inputDataSet.output(new AvroOutputFormat<>(new Path
+ (filenamePrefix), extractedAvroType));
+
+ if (numShards > 0) {
+ dataSink.setParallelism(numShards);
+ }
+ }
+ }
+
+ private static class TextIOReadTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator<TextIO.Read.Bound<String>> {
+ private static final Logger LOG = LoggerFactory.getLogger(TextIOReadTranslatorBatch.class);
+
+ @Override
+ public void translateNode(TextIO.Read.Bound<String> transform, FlinkBatchTranslationContext context) {
+ String path = transform.getFilepattern();
+ String name = transform.getName();
+
+ TextIO.CompressionType compressionType = transform.getCompressionType();
+ boolean needsValidation = transform.needsValidation();
+
+ // TODO: Implement these. We need Flink support for this.
+ LOG.warn("Translation of TextIO.CompressionType not yet supported. Is: {}.", compressionType);
+ LOG.warn("Translation of TextIO.Read.needsValidation not yet supported. Is: {}.", needsValidation);
+
+ PValue output = context.getOutput(transform);
+
+ TypeInformation<String> typeInformation = context.getTypeInfo(output);
+ DataSource<String> source = new DataSource<>(context.getExecutionEnvironment(), new TextInputFormat(new Path(path)), typeInformation, name);
+
+ context.setOutputDataSet(output, source);
+ }
+ }
+
+ private static class TextIOWriteTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<TextIO.Write.Bound<T>> {
+ private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteTranslatorBatch.class);
+
+ @Override
+ public void translateNode(TextIO.Write.Bound<T> transform, FlinkBatchTranslationContext context) {
+ PValue input = context.getInput(transform);
+ DataSet<T> inputDataSet = context.getInputDataSet(input);
+
+ String filenamePrefix = transform.getFilenamePrefix();
+ String filenameSuffix = transform.getFilenameSuffix();
+ boolean needsValidation = transform.needsValidation();
+ int numShards = transform.getNumShards();
+ String shardNameTemplate = transform.getShardNameTemplate();
+
+ // TODO: Implement these. We need Flink support for this.
+ LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation);
+ LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix);
+ LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
+
+ //inputDataSet.print();
+ DataSink<T> dataSink = inputDataSet.writeAsText(filenamePrefix);
+
+ if (numShards > 0) {
+ dataSink.setParallelism(numShards);
+ }
+ }
+ }
+
+ private static class ConsoleIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator<ConsoleIO.Write.Bound> {
+ @Override
+ public void translateNode(ConsoleIO.Write.Bound transform, FlinkBatchTranslationContext context) {
+ PValue input = (PValue) context.getInput(transform);
+ DataSet<?> inputDataSet = context.getInputDataSet(input);
+ inputDataSet.printOnTaskManager(transform.getName());
+ }
+ }
+
+ private static class WriteSinkTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Write.Bound<T>> {
+
+ @Override
+ public void translateNode(Write.Bound<T> transform, FlinkBatchTranslationContext context) {
+ String name = transform.getName();
+ PValue input = context.getInput(transform);
+ DataSet<T> inputDataSet = context.getInputDataSet(input);
+
+ inputDataSet.output(new SinkOutputFormat<>(transform, context.getPipelineOptions())).name(name);
+ }
+ }
+
+ private static class GroupByKeyOnlyTranslatorBatch<K, V> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<GroupByKey.GroupByKeyOnly<K, V>> {
+
+ @Override
+ public void translateNode(GroupByKey.GroupByKeyOnly<K, V> transform, FlinkBatchTranslationContext context) {
+ DataSet<KV<K, V>> inputDataSet = context.getInputDataSet(context.getInput(transform));
+ GroupReduceFunction<KV<K, V>, KV<K, Iterable<V>>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>();
+
+ TypeInformation<KV<K, Iterable<V>>> typeInformation = context.getTypeInfo(context.getOutput(transform));
+
+ Grouping<KV<K, V>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType()));
+
+ GroupReduceOperator<KV<K, V>, KV<K, Iterable<V>>> outputDataSet =
+ new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
+ context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+ }
+ }
+
+ /**
+ * Translates a GroupByKey while ignoring window assignments. This is identical to the {@link GroupByKeyOnlyTranslatorBatch}
+ */
+ private static class GroupByKeyTranslatorBatch<K, V> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<GroupByKey<K, V>> {
+
+ @Override
+ public void translateNode(GroupByKey<K, V> transform, FlinkBatchTranslationContext context) {
+ DataSet<KV<K, V>> inputDataSet = context.getInputDataSet(context.getInput(transform));
+ GroupReduceFunction<KV<K, V>, KV<K, Iterable<V>>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>();
+
+ TypeInformation<KV<K, Iterable<V>>> typeInformation = context.getTypeInfo(context.getOutput(transform));
+
+ Grouping<KV<K, V>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType()));
+
+ GroupReduceOperator<KV<K, V>, KV<K, Iterable<V>>> outputDataSet =
+ new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
+
+ context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+ }
+ }
+
+ private static class CombinePerKeyTranslatorBatch<K, VI, VA, VO> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Combine.PerKey<K, VI, VO>> {
+
+ @Override
+ public void translateNode(Combine.PerKey<K, VI, VO> transform, FlinkBatchTranslationContext context) {
+ DataSet<KV<K, VI>> inputDataSet = context.getInputDataSet(context.getInput(transform));
+
+ @SuppressWarnings("unchecked")
+ Combine.KeyedCombineFn<K, VI, VA, VO> keyedCombineFn = (Combine.KeyedCombineFn<K, VI, VA, VO>) transform.getFn();
+
+ KvCoder<K, VI> inputCoder = (KvCoder<K, VI>) context.getInput(transform).getCoder();
+
+ Coder<VA> accumulatorCoder =
+ null;
+ try {
+ accumulatorCoder = keyedCombineFn.getAccumulatorCoder(context.getInput(transform).getPipeline().getCoderRegistry(), inputCoder.getKeyCoder(), inputCoder.getValueCoder());
+ } catch (CannotProvideCoderException e) {
+ e.printStackTrace();
+ // TODO
+ }
+
+ TypeInformation<KV<K, VI>> kvCoderTypeInformation = new KvCoderTypeInformation<>(inputCoder);
+ TypeInformation<KV<K, VA>> partialReduceTypeInfo = new KvCoderTypeInformation<>(KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder));
+
+ Grouping<KV<K, VI>> inputGrouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation));
+
+ FlinkPartialReduceFunction<K, VI, VA> partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn);
+
+ // Partially GroupReduce the values into the intermediate format VA (combine)
+ GroupCombineOperator<KV<K, VI>, KV<K, VA>> groupCombine =
+ new GroupCombineOperator<>(inputGrouping, partialReduceTypeInfo, partialReduceFunction,
+ "GroupCombine: " + transform.getName());
+
+ // Reduce fully to VO
+ GroupReduceFunction<KV<K, VA>, KV<K, VO>> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn);
+
+ TypeInformation<KV<K, VO>> reduceTypeInfo = context.getTypeInfo(context.getOutput(transform));
+
+ Grouping<KV<K, VA>> intermediateGrouping = new UnsortedGrouping<>(groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType()));
+
+ // Fully reduce the values and create output format VO
+ GroupReduceOperator<KV<K, VA>, KV<K, VO>> outputDataSet =
+ new GroupReduceOperator<>(intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName());
+
+ context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+ }
+ }
+
+// private static class CombineGroupedValuesTranslator<K, VI, VO> implements FlinkPipelineTranslator.TransformTranslator<Combine.GroupedValues<K, VI, VO>> {
+//
+// @Override
+// public void translateNode(Combine.GroupedValues<K, VI, VO> transform, TranslationContext context) {
+// DataSet<KV<K, VI>> inputDataSet = context.getInputDataSet(transform.getInput());
+//
+// Combine.KeyedCombineFn<? super K, ? super VI, ?, VO> keyedCombineFn = transform.getFn();
+//
+// GroupReduceFunction<KV<K, VI>, KV<K, VO>> groupReduceFunction = new FlinkCombineFunction<>(keyedCombineFn);
+//
+// TypeInformation<KV<K, VO>> typeInformation = context.getTypeInfo(transform.getOutput());
+//
+// Grouping<KV<K, VI>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{""}, inputDataSet.getType()));
+//
+// GroupReduceOperator<KV<K, VI>, KV<K, VO>> outputDataSet =
+// new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
+// context.setOutputDataSet(transform.getOutput(), outputDataSet);
+// }
+// }
+
+ private static class ParDoBoundTranslatorBatch<IN, OUT> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<ParDo.Bound<IN, OUT>> {
+ private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundTranslatorBatch.class);
+
+ @Override
+ public void translateNode(ParDo.Bound<IN, OUT> transform, FlinkBatchTranslationContext context) {
+ DataSet<IN> inputDataSet = context.getInputDataSet(context.getInput(transform));
+
+ final DoFn<IN, OUT> doFn = transform.getFn();
+
+ TypeInformation<OUT> typeInformation = context.getTypeInfo(context.getOutput(transform));
+
+ FlinkDoFnFunction<IN, OUT> doFnWrapper = new FlinkDoFnFunction<>(doFn, context.getPipelineOptions());
+ MapPartitionOperator<IN, OUT> outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName());
+
+ transformSideInputs(transform.getSideInputs(), outputDataSet, context);
+
+ context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+ }
+ }
+
+ private static class ParDoBoundMultiTranslatorBatch<IN, OUT> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<ParDo.BoundMulti<IN, OUT>> {
+ private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundMultiTranslatorBatch.class);
+
+ @Override
+ public void translateNode(ParDo.BoundMulti<IN, OUT> transform, FlinkBatchTranslationContext context) {
+ DataSet<IN> inputDataSet = context.getInputDataSet(context.getInput(transform));
+
+ final DoFn<IN, OUT> doFn = transform.getFn();
+
+ Map<TupleTag<?>, PCollection<?>> outputs = context.getOutput(transform).getAll();
+
+ Map<TupleTag<?>, Integer> outputMap = Maps.newHashMap();
+ // put the main output at index 0, FlinkMultiOutputDoFnFunction also expects this
+ outputMap.put(transform.getMainOutputTag(), 0);
+ int count = 1;
+ for (TupleTag<?> tag: outputs.keySet()) {
+ if (!outputMap.containsKey(tag)) {
+ outputMap.put(tag, count++);
+ }
+ }
+
+ // collect all output Coders and create a UnionCoder for our tagged outputs
+ List<Coder<?>> outputCoders = Lists.newArrayList();
+ for (PCollection<?> coll: outputs.values()) {
+ outputCoders.add(coll.getCoder());
+ }
+
+ UnionCoder unionCoder = UnionCoder.of(outputCoders);
+
+ @SuppressWarnings("unchecked")
+ TypeInformation<RawUnionValue> typeInformation = new CoderTypeInformation<>(unionCoder);
+
+ @SuppressWarnings("unchecked")
+ FlinkMultiOutputDoFnFunction<IN, OUT> doFnWrapper = new FlinkMultiOutputDoFnFunction(doFn, context.getPipelineOptions(), outputMap);
+ MapPartitionOperator<IN, RawUnionValue> outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName());
+
+ transformSideInputs(transform.getSideInputs(), outputDataSet, context);
+
+ for (Map.Entry<TupleTag<?>, PCollection<?>> output: outputs.entrySet()) {
+ TypeInformation<Object> outputType = context.getTypeInfo(output.getValue());
+ int outputTag = outputMap.get(output.getKey());
+ FlinkMultiOutputPruningFunction<Object> pruningFunction = new FlinkMultiOutputPruningFunction<>(outputTag);
+ FlatMapOperator<RawUnionValue, Object> pruningOperator = new
+ FlatMapOperator<>(outputDataSet, outputType,
+ pruningFunction, output.getValue().getName());
+ context.setOutputDataSet(output.getValue(), pruningOperator);
+
+ }
+ }
+ }
+
+ private static class FlattenPCollectionTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Flatten.FlattenPCollectionList<T>> {
+
+ @Override
+ public void translateNode(Flatten.FlattenPCollectionList<T> transform, FlinkBatchTranslationContext context) {
+ List<PCollection<T>> allInputs = context.getInput(transform).getAll();
+ DataSet<T> result = null;
+ for(PCollection<T> collection : allInputs) {
+ DataSet<T> current = context.getInputDataSet(collection);
+ if (result == null) {
+ result = current;
+ } else {
+ result = result.union(current);
+ }
+ }
+ context.setOutputDataSet(context.getOutput(transform), result);
+ }
+ }
+
+ private static class CreatePCollectionViewTranslatorBatch<R, T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<View.CreatePCollectionView<R, T>> {
+ @Override
+ public void translateNode(View.CreatePCollectionView<R, T> transform, FlinkBatchTranslationContext context) {
+ DataSet<T> inputDataSet = context.getInputDataSet(context.getInput(transform));
+ PCollectionView<T> input = transform.apply(null);
+ context.setSideInputDataSet(input, inputDataSet);
+ }
+ }
+
+ private static class CreateTranslatorBatch<OUT> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Create.Values<OUT>> {
+
+ @Override
+ public void translateNode(Create.Values<OUT> transform, FlinkBatchTranslationContext context) {
+ TypeInformation<OUT> typeInformation = context.getOutputTypeInfo();
+ Iterable<OUT> elements = transform.getElements();
+
+ // we need to serialize the elements to byte arrays, since they might contain
+ // elements that are not serializable by Java serialization. We deserialize them
+ // in the FlatMap function using the Coder.
+
+ List<byte[]> serializedElements = Lists.newArrayList();
+ Coder<OUT> coder = context.getOutput(transform).getCoder();
+ for (OUT element: elements) {
+ ByteArrayOutputStream bao = new ByteArrayOutputStream();
+ try {
+ coder.encode(element, bao, Coder.Context.OUTER);
+ serializedElements.add(bao.toByteArray());
+ } catch (IOException e) {
+ throw new RuntimeException("Could not serialize Create elements using Coder: " + e);
+ }
+ }
+
+ DataSet<Integer> initDataSet = context.getExecutionEnvironment().fromElements(1);
+ FlinkCreateFunction<Integer, OUT> flatMapFunction = new FlinkCreateFunction<>(serializedElements, coder);
+ FlatMapOperator<Integer, OUT> outputDataSet = new FlatMapOperator<>(initDataSet, typeInformation, flatMapFunction, transform.getName());
+
+ context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+ }
+ }
+
+ private static void transformSideInputs(List<PCollectionView<?>> sideInputs,
+ MapPartitionOperator<?, ?> outputDataSet,
+ FlinkBatchTranslationContext context) {
+ // get corresponding Flink broadcast DataSets
+ for(PCollectionView<?> input : sideInputs) {
+ DataSet<?> broadcastSet = context.getSideInputDataSet(input);
+ outputDataSet.withBroadcastSet(broadcastSet, input.getTagInternal().getId());
+ }
+ }
+
+// Disabled because it depends on a pending pull request to the DataFlowSDK
+ /**
+ * Special composite transform translator. Only called if the CoGroup is two dimensional.
+ * @param <K>
+ */
+ private static class CoGroupByKeyTranslatorBatch<K, V1, V2> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<CoGroupByKey<K>> {
+
+ @Override
+ public void translateNode(CoGroupByKey<K> transform, FlinkBatchTranslationContext context) {
+ KeyedPCollectionTuple<K> input = context.getInput(transform);
+
+ CoGbkResultSchema schema = input.getCoGbkResultSchema();
+ List<KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?>> keyedCollections = input.getKeyedCollections();
+
+ KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?> taggedCollection1 = keyedCollections.get(0);
+ KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?> taggedCollection2 = keyedCollections.get(1);
+
+ TupleTag<?> tupleTag1 = taggedCollection1.getTupleTag();
+ TupleTag<?> tupleTag2 = taggedCollection2.getTupleTag();
+
+ PCollection<? extends KV<K, ?>> collection1 = taggedCollection1.getCollection();
+ PCollection<? extends KV<K, ?>> collection2 = taggedCollection2.getCollection();
+
+ DataSet<KV<K,V1>> inputDataSet1 = context.getInputDataSet(collection1);
+ DataSet<KV<K,V2>> inputDataSet2 = context.getInputDataSet(collection2);
+
+ TypeInformation<KV<K,CoGbkResult>> typeInfo = context.getOutputTypeInfo();
+
+ FlinkCoGroupKeyedListAggregator<K,V1,V2> aggregator = new FlinkCoGroupKeyedListAggregator<>(schema, tupleTag1, tupleTag2);
+
+ Keys.ExpressionKeys<KV<K,V1>> keySelector1 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet1.getType());
+ Keys.ExpressionKeys<KV<K,V2>> keySelector2 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet2.getType());
+
+ DataSet<KV<K, CoGbkResult>> out = new CoGroupOperator<>(inputDataSet1, inputDataSet2,
+ keySelector1, keySelector2,
+ aggregator, typeInfo, null, transform.getName());
+ context.setOutputDataSet(context.getOutput(transform), out);
+ }
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Miscellaneous
+ // --------------------------------------------------------------------------------------------
+
+ private FlinkBatchTransformTranslators() {}
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java
new file mode 100644
index 0000000..1072fa3
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java
@@ -0,0 +1,129 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation;
+
+import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation;
+import com.dataartisans.flink.dataflow.translation.types.KvCoderTypeInformation;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import com.google.cloud.dataflow.sdk.values.TypedPValue;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class FlinkBatchTranslationContext {
+
+ private final Map<PValue, DataSet<?>> dataSets;
+ private final Map<PCollectionView<?>, DataSet<?>> broadcastDataSets;
+
+ private final ExecutionEnvironment env;
+ private final PipelineOptions options;
+
+ private AppliedPTransform<?, ?, ?> currentTransform;
+
+ // ------------------------------------------------------------------------
+
+ public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions options) {
+ this.env = env;
+ this.options = options;
+ this.dataSets = new HashMap<>();
+ this.broadcastDataSets = new HashMap<>();
+ }
+
+ // ------------------------------------------------------------------------
+
+ public ExecutionEnvironment getExecutionEnvironment() {
+ return env;
+ }
+
+ public PipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ @SuppressWarnings("unchecked")
+ public <T> DataSet<T> getInputDataSet(PValue value) {
+ return (DataSet<T>) dataSets.get(value);
+ }
+
+ public void setOutputDataSet(PValue value, DataSet<?> set) {
+ if (!dataSets.containsKey(value)) {
+ dataSets.put(value, set);
+ }
+ }
+
+ /**
+ * Sets the AppliedPTransform which carries input/output.
+ * @param currentTransform
+ */
+ public void setCurrentTransform(AppliedPTransform<?, ?, ?> currentTransform) {
+ this.currentTransform = currentTransform;
+ }
+
+ @SuppressWarnings("unchecked")
+ public <T> DataSet<T> getSideInputDataSet(PCollectionView<?> value) {
+ return (DataSet<T>) broadcastDataSets.get(value);
+ }
+
+ public void setSideInputDataSet(PCollectionView<?> value, DataSet<?> set) {
+ if (!broadcastDataSets.containsKey(value)) {
+ broadcastDataSets.put(value, set);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ public <T> TypeInformation<T> getTypeInfo(PInput output) {
+ if (output instanceof TypedPValue) {
+ Coder<?> outputCoder = ((TypedPValue) output).getCoder();
+ if (outputCoder instanceof KvCoder) {
+ return new KvCoderTypeInformation((KvCoder) outputCoder);
+ } else {
+ return new CoderTypeInformation(outputCoder);
+ }
+ }
+ return new GenericTypeInfo<>((Class<T>)Object.class);
+ }
+
+ public <T> TypeInformation<T> getInputTypeInfo() {
+ return getTypeInfo(currentTransform.getInput());
+ }
+
+ public <T> TypeInformation<T> getOutputTypeInfo() {
+ return getTypeInfo((PValue) currentTransform.getOutput());
+ }
+
+ @SuppressWarnings("unchecked")
+ <I extends PInput> I getInput(PTransform<I, ?> transform) {
+ return (I) currentTransform.getInput();
+ }
+
+ @SuppressWarnings("unchecked")
+ <O extends POutput> O getOutput(PTransform<?, O> transform) {
+ return (O) currentTransform.getOutput();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java
index 92b9135..e5c8545 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java
@@ -7,8 +7,6 @@
*
* 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.
@@ -18,151 +16,10 @@
package com.dataartisans.flink.dataflow.translation;
import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.Pipeline.PipelineVisitor;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-/**
- * FlinkPipelineTranslator knows how to translate Pipeline objects into Flink Jobs.
- *
- * This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator}
- */
-public class FlinkPipelineTranslator implements PipelineVisitor {
-
- private final TranslationContext context;
-
- private int depth = 0;
-
- /**
- * Composite transform that we want to translate before proceeding with other transforms
- */
- private PTransform<?, ?> currentCompositeTransform;
-
- public FlinkPipelineTranslator(ExecutionEnvironment env, PipelineOptions options) {
- this.context = new TranslationContext(env, options);
- }
+public abstract class FlinkPipelineTranslator implements Pipeline.PipelineVisitor {
public void translate(Pipeline pipeline) {
pipeline.traverseTopologically(this);
}
-
-
- // --------------------------------------------------------------------------------------------
- // Pipeline Visitor Methods
- // --------------------------------------------------------------------------------------------
-
- private static String genSpaces(int n) {
- String s = "";
- for(int i = 0; i < n; i++) {
- s += "| ";
- }
- return s;
- }
-
- private static String formatNodeName(TransformTreeNode node) {
- return node.toString().split("@")[1] + node.getTransform();
- }
-
- @Override
- public void enterCompositeTransform(TransformTreeNode node) {
- System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node));
- PTransform<?, ?> transform = node.getTransform();
-
- if (transform != null && currentCompositeTransform == null) {
- TransformTranslator<?> translator = FlinkTransformTranslators.getTranslator(transform);
-
- if (translator != null) {
- currentCompositeTransform = transform;
-
- if (transform instanceof CoGroupByKey && node.getInput().expand().size() != 2) {
- // we can only optimize CoGroupByKey for input size 2
- currentCompositeTransform = null;
- }
- }
- }
-
- this.depth++;
- }
-
- @Override
- public void leaveCompositeTransform(TransformTreeNode node) {
- PTransform<?, ?> transform = node.getTransform();
-
- if (transform != null) {
- TransformTranslator<?> translator = FlinkTransformTranslators.getTranslator(transform);
-
- if (currentCompositeTransform == transform) {
- if (translator != null) {
- System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node));
- applyTransform(transform, node, translator);
- currentCompositeTransform = null;
- } else {
- throw new IllegalStateException("Attempted to translate composite transform " +
- "but no translator was found: " + currentCompositeTransform);
- }
- }
- }
-
- this.depth--;
- System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node));
- }
-
- @Override
- public void visitTransform(TransformTreeNode node) {
- System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node));
- if (currentCompositeTransform != null) {
- // ignore it
- return;
- }
-
- // the transformation applied in this node
- PTransform<?, ?> transform = node.getTransform();
-
- // the translator to the Flink operation(s)
- TransformTranslator<?> translator = FlinkTransformTranslators.getTranslator(transform);
-
- if (translator == null) {
- System.out.println(node.getTransform().getClass());
- throw new UnsupportedOperationException("The transform " + transform + " is currently not supported.");
- }
-
- applyTransform(transform, node, translator);
- }
-
- @Override
- public void visitValue(PValue value, TransformTreeNode producer) {
- // do nothing here
- }
-
- /**
- * Utility method to define a generic variable to cast the translator and the transform to.
- */
- private <T extends PTransform<?, ?>> void applyTransform(PTransform<?, ?> transform, TransformTreeNode node, TransformTranslator<?> translator) {
-
- @SuppressWarnings("unchecked")
- T typedTransform = (T) transform;
-
- @SuppressWarnings("unchecked")
- TransformTranslator<T> typedTranslator = (TransformTranslator<T>) translator;
-
- // create the applied PTransform on the context
- context.setCurrentTransform(AppliedPTransform.of(
- node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform));
-
- typedTranslator.translateNode(typedTransform, context);
- }
-
- /**
- * A translator of a {@link PTransform}.
- */
- public interface TransformTranslator<Type extends PTransform> {
-
- void translateNode(Type transform, TranslationContext context);
- }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
new file mode 100644
index 0000000..c8760c7
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
@@ -0,0 +1,138 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation;
+
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
+
+ /** The necessary context in the case of a straming job. */
+ private final FlinkStreamingTranslationContext streamingContext;
+
+ private int depth = 0;
+
+ /** Composite transform that we want to translate before proceeding with other transforms. */
+ private PTransform<?, ?> currentCompositeTransform;
+
+ public FlinkStreamingPipelineTranslator(StreamExecutionEnvironment env, PipelineOptions options) {
+ this.streamingContext = new FlinkStreamingTranslationContext(env, options);
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Pipeline Visitor Methods
+ // --------------------------------------------------------------------------------------------
+
+ @Override
+ public void enterCompositeTransform(TransformTreeNode node) {
+ System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node));
+
+ PTransform<?, ?> transform = node.getTransform();
+ if (transform != null && currentCompositeTransform == null) {
+
+ StreamTransformTranslator<?> translator = FlinkStreamingTransformTranslators.getTranslator(transform);
+ if (translator != null) {
+ currentCompositeTransform = transform;
+ }
+ }
+ this.depth++;
+ }
+
+ @Override
+ public void leaveCompositeTransform(TransformTreeNode node) {
+ PTransform<?, ?> transform = node.getTransform();
+ if (transform != null && currentCompositeTransform == transform) {
+
+ StreamTransformTranslator<?> translator = FlinkStreamingTransformTranslators.getTranslator(transform);
+ if (translator != null) {
+ System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node));
+ applyStreamingTransform(transform, node, translator);
+ currentCompositeTransform = null;
+ } else {
+ throw new IllegalStateException("Attempted to translate composite transform " +
+ "but no translator was found: " + currentCompositeTransform);
+ }
+ }
+ this.depth--;
+ System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node));
+ }
+
+ @Override
+ public void visitTransform(TransformTreeNode node) {
+ System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node));
+ if (currentCompositeTransform != null) {
+ // ignore it
+ return;
+ }
+
+ // get the transformation corresponding to hte node we are
+ // currently visiting and translate it into its Flink alternative.
+
+ PTransform<?, ?> transform = node.getTransform();
+ StreamTransformTranslator<?> translator = FlinkStreamingTransformTranslators.getTranslator(transform);
+ if (translator == null) {
+ System.out.println(node.getTransform().getClass());
+ throw new UnsupportedOperationException("The transform " + transform + " is currently not supported.");
+ }
+ applyStreamingTransform(transform, node, translator);
+ }
+
+ @Override
+ public void visitValue(PValue value, TransformTreeNode producer) {
+ // do nothing here
+ }
+
+ private <T extends PTransform<?, ?>> void applyStreamingTransform(PTransform<?, ?> transform, TransformTreeNode node, StreamTransformTranslator<?> translator) {
+ if (this.streamingContext == null) {
+ throw new IllegalStateException("The FlinkPipelineTranslator is not yet initialized.");
+ }
+
+ @SuppressWarnings("unchecked")
+ T typedTransform = (T) transform;
+
+ @SuppressWarnings("unchecked")
+ StreamTransformTranslator<T> typedTranslator = (StreamTransformTranslator<T>) translator;
+
+ // create the applied PTransform on the batchContext
+ streamingContext.setCurrentTransform(AppliedPTransform.of(
+ node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform));
+ typedTranslator.translateNode(typedTransform, streamingContext);
+ }
+
+ /**
+ * A translator of a {@link PTransform}.
+ */
+ public interface StreamTransformTranslator<Type extends PTransform> {
+ void translateNode(Type transform, FlinkStreamingTranslationContext context);
+ }
+
+ private static String genSpaces(int n) {
+ String s = "";
+ for (int i = 0; i < n; i++) {
+ s += "| ";
+ }
+ return s;
+ }
+
+ private static String formatNodeName(TransformTreeNode node) {
+ return node.toString().split("@")[1] + node.getTransform();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
new file mode 100644
index 0000000..4c8cd4b
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
@@ -0,0 +1,356 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation;
+
+import com.dataartisans.flink.dataflow.translation.functions.UnionCoder;
+import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation;
+import com.dataartisans.flink.dataflow.translation.wrappers.streaming.*;
+import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedFlinkSource;
+import com.dataartisans.flink.dataflow.translation.wrappers.streaming.io.UnboundedSourceWrapper;
+import com.google.api.client.util.Maps;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.*;
+import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
+import com.google.cloud.dataflow.sdk.transforms.windowing.*;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.collect.Lists;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.streaming.api.datastream.*;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+
+/**
+ * <p>
+ * Coder<?> entryCoder = pCollection.getCoder();
+ * if (!(entryCoder instanceof KvCoder<?, ?>)) {
+ * throw new IllegalArgumentException("PCollection does not use a KvCoder");
+ * }
+ */
+public class FlinkStreamingTransformTranslators {
+
+ // --------------------------------------------------------------------------------------------
+ // Transform Translator Registry
+ // --------------------------------------------------------------------------------------------
+
+ @SuppressWarnings("rawtypes")
+ private static final Map<Class<? extends PTransform>, FlinkStreamingPipelineTranslator.StreamTransformTranslator> TRANSLATORS = new HashMap<>();
+
+ // here you can find all the available translators.
+ static {
+ TRANSLATORS.put(Read.Unbounded.class, new UnboundedReadSourceTranslator());
+ TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator());
+ TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteBoundStreamingTranslator());
+ TRANSLATORS.put(Window.Bound.class, new WindowBoundTranslator());
+ TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslator());
+ TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslator());
+ TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator());
+ TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiStreamingTranslator());
+
+ }
+
+ public static FlinkStreamingPipelineTranslator.StreamTransformTranslator<?> getTranslator(PTransform<?, ?> transform) {
+ FlinkStreamingPipelineTranslator.StreamTransformTranslator<?> translator = TRANSLATORS.get(transform.getClass());
+ return translator;
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Transformation Implementations
+ // --------------------------------------------------------------------------------------------
+
+ private static class TextIOWriteBoundStreamingTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<TextIO.Write.Bound<T>> {
+ private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteBoundStreamingTranslator.class);
+
+ @Override
+ public void translateNode(TextIO.Write.Bound<T> transform, FlinkStreamingTranslationContext context) {
+ PValue input = context.getInput(transform);
+ DataStream<WindowedValue<T>> inputDataStream = context.getInputDataStream(input);
+
+ String filenamePrefix = transform.getFilenamePrefix();
+ String filenameSuffix = transform.getFilenameSuffix();
+ boolean needsValidation = transform.needsValidation();
+ int numShards = transform.getNumShards();
+ String shardNameTemplate = transform.getShardNameTemplate();
+
+ // TODO: Implement these. We need Flink support for this.
+ LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation);
+ LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix);
+ LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
+
+ DataStream<String> dataSink = inputDataStream.flatMap(new FlatMapFunction<WindowedValue<T>, String>() {
+ @Override
+ public void flatMap(WindowedValue<T> value, Collector<String> out) throws Exception {
+ out.collect(value.getValue().toString());
+ }
+ });
+ DataStreamSink<String> output = dataSink.writeAsText(filenamePrefix, FileSystem.WriteMode.OVERWRITE);
+
+ if (numShards > 0) {
+ output.setParallelism(numShards);
+ }
+ }
+ }
+
+ private static class UnboundedReadSourceTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Read.Unbounded<T>> {
+
+ @Override
+ public void translateNode(Read.Unbounded<T> transform, FlinkStreamingTranslationContext context) {
+ PCollection<T> output = context.getOutput(transform);
+
+ DataStream<WindowedValue<T>> source = null;
+ if (transform.getSource().getClass().equals(UnboundedFlinkSource.class)) {
+ UnboundedFlinkSource flinkSource = (UnboundedFlinkSource) transform.getSource();
+ source = context.getExecutionEnvironment()
+ .addSource(flinkSource.getFlinkSource())
+ .flatMap(new FlatMapFunction<String, WindowedValue<String>>() {
+ @Override
+ public void flatMap(String s, Collector<WindowedValue<String>> collector) throws Exception {
+ collector.collect(WindowedValue.<String>of(s, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
+ }
+ });
+ } else {
+ source = context.getExecutionEnvironment()
+ .addSource(new UnboundedSourceWrapper<>(context.getPipelineOptions(), transform));
+ }
+ context.setOutputDataStream(output, source);
+ }
+ }
+
+ private static class ParDoBoundStreamingTranslator<IN, OUT> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<ParDo.Bound<IN, OUT>> {
+
+ @Override
+ public void translateNode(ParDo.Bound<IN, OUT> transform, FlinkStreamingTranslationContext context) {
+ PCollection<OUT> output = context.getOutput(transform);
+
+ final WindowingStrategy<OUT, ? extends BoundedWindow> windowingStrategy =
+ (WindowingStrategy<OUT, ? extends BoundedWindow>)
+ context.getOutput(transform).getWindowingStrategy();
+
+ WindowedValue.WindowedValueCoder<OUT> outputStreamCoder = WindowedValue.getFullCoder(output.getCoder(), windowingStrategy.getWindowFn().windowCoder());
+ CoderTypeInformation<WindowedValue<OUT>> outputWindowedValueCoder = new CoderTypeInformation<>(outputStreamCoder);
+
+ FlinkParDoBoundWrapper<IN, OUT> doFnWrapper = new FlinkParDoBoundWrapper<>(context.getPipelineOptions(), windowingStrategy, transform.getFn());
+ DataStream<WindowedValue<IN>> inputDataStream = context.getInputDataStream(context.getInput(transform));
+ SingleOutputStreamOperator<WindowedValue<OUT>, ?> outDataStream = inputDataStream.flatMap(doFnWrapper).returns(outputWindowedValueCoder);
+
+ context.setOutputDataStream(context.getOutput(transform), outDataStream);
+ }
+ }
+
+ public static class WindowBoundTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Window.Bound<T>> {
+
+ @Override
+ public void translateNode(Window.Bound<T> transform, FlinkStreamingTranslationContext context) {
+ PValue input = context.getInput(transform);
+ DataStream<WindowedValue<T>> inputDataStream = context.getInputDataStream(input);
+
+ final WindowingStrategy<T, ? extends BoundedWindow> windowingStrategy =
+ (WindowingStrategy<T, ? extends BoundedWindow>)
+ context.getOutput(transform).getWindowingStrategy();
+
+ final WindowFn<T, ? extends BoundedWindow> windowFn = windowingStrategy.getWindowFn();
+
+ WindowedValue.WindowedValueCoder<T> outputStreamCoder = WindowedValue.getFullCoder(
+ context.getInput(transform).getCoder(), windowingStrategy.getWindowFn().windowCoder());
+ CoderTypeInformation<WindowedValue<T>> outputWindowedValueCoder =
+ new CoderTypeInformation<>(outputStreamCoder);
+
+ final FlinkParDoBoundWrapper<T, T> windowDoFnAssigner = new FlinkParDoBoundWrapper<>(
+ context.getPipelineOptions(), windowingStrategy, createWindowAssigner(windowFn));
+
+ SingleOutputStreamOperator<WindowedValue<T>, ?> windowedStream =
+ inputDataStream.flatMap(windowDoFnAssigner).returns(outputWindowedValueCoder);
+ context.setOutputDataStream(context.getOutput(transform), windowedStream);
+ }
+
+ private static <T, W extends BoundedWindow> DoFn<T, T> createWindowAssigner(final WindowFn<T, W> windowFn) {
+ return new DoFn<T, T>() {
+
+ @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 Collection<? extends BoundedWindow> windows() {
+ return c.windowingInternals().windows();
+ }
+ });
+
+ c.windowingInternals().outputWindowedValue(
+ c.element(), c.timestamp(), windows, c.pane());
+ }
+ };
+ }
+ }
+
+ public static class GroupByKeyTranslator<K, V> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<GroupByKey<K, V>> {
+
+ @Override
+ public void translateNode(GroupByKey<K, V> transform, FlinkStreamingTranslationContext context) {
+ PValue input = context.getInput(transform);
+
+ DataStream<WindowedValue<KV<K, V>>> inputDataStream = context.getInputDataStream(input);
+ KvCoder<K, V> inputKvCoder = (KvCoder<K, V>) context.getInput(transform).getCoder();
+
+ KeyedStream<WindowedValue<KV<K, V>>, K> groupByKStream = FlinkGroupByKeyWrapper
+ .groupStreamByKey(inputDataStream, inputKvCoder);
+
+ DataStream<WindowedValue<KV<K, Iterable<V>>>> groupedByKNWstream =
+ FlinkGroupAlsoByWindowWrapper.createForIterable(context.getPipelineOptions(),
+ context.getInput(transform), groupByKStream);
+
+ context.setOutputDataStream(context.getOutput(transform), groupedByKNWstream);
+ }
+ }
+
+ public static class CombinePerKeyTranslator<K, VIN, VACC, VOUT> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Combine.PerKey<K, VIN, VOUT>> {
+
+ @Override
+ public void translateNode(Combine.PerKey<K, VIN, VOUT> transform, FlinkStreamingTranslationContext context) {
+ PValue input = context.getInput(transform);
+
+ DataStream<WindowedValue<KV<K, VIN>>> inputDataStream = context.getInputDataStream(input);
+ KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) context.getInput(transform).getCoder();
+ KvCoder<K, VOUT> outputKvCoder = (KvCoder<K, VOUT>) context.getOutput(transform).getCoder();
+
+ KeyedStream<WindowedValue<KV<K, VIN>>, K> groupByKStream = FlinkGroupByKeyWrapper
+ .groupStreamByKey(inputDataStream, inputKvCoder);
+
+ Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combineFn = (Combine.KeyedCombineFn<K, VIN, VACC, VOUT>) transform.getFn();
+ DataStream<WindowedValue<KV<K, VOUT>>> groupedByKNWstream =
+ FlinkGroupAlsoByWindowWrapper.create(context.getPipelineOptions(),
+ context.getInput(transform), groupByKStream, combineFn, outputKvCoder);
+
+ context.setOutputDataStream(context.getOutput(transform), groupedByKNWstream);
+ }
+ }
+
+ public static class FlattenPCollectionTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Flatten.FlattenPCollectionList<T>> {
+
+ @Override
+ public void translateNode(Flatten.FlattenPCollectionList<T> transform, FlinkStreamingTranslationContext context) {
+ List<PCollection<T>> allInputs = context.getInput(transform).getAll();
+ DataStream<T> result = null;
+ for (PCollection<T> collection : allInputs) {
+ DataStream<T> current = context.getInputDataStream(collection);
+ result = (result == null) ? current : result.union(current);
+ }
+ context.setOutputDataStream(context.getOutput(transform), result);
+ }
+ }
+
+ public static class ParDoBoundMultiStreamingTranslator<IN, OUT> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<ParDo.BoundMulti<IN, OUT>> {
+
+ private final int MAIN_TAG_INDEX = 0;
+
+ @Override
+ public void translateNode(ParDo.BoundMulti<IN, OUT> transform, FlinkStreamingTranslationContext context) {
+
+ // we assume that the transformation does not change the windowing strategy.
+ WindowingStrategy<?, ? extends BoundedWindow> windowingStrategy = context.getInput(transform).getWindowingStrategy();
+
+ Map<TupleTag<?>, PCollection<?>> outputs = context.getOutput(transform).getAll();
+ Map<TupleTag<?>, Integer> tagsToLabels = transformTupleTagsToLabels(
+ transform.getMainOutputTag(), outputs.keySet());
+
+ UnionCoder intermUnionCoder = getIntermUnionCoder(outputs.values());
+ WindowedValue.WindowedValueCoder<RawUnionValue> outputStreamCoder = WindowedValue.getFullCoder(
+ intermUnionCoder, windowingStrategy.getWindowFn().windowCoder());
+
+ CoderTypeInformation<WindowedValue<RawUnionValue>> intermWindowedValueCoder =
+ new CoderTypeInformation<>(outputStreamCoder);
+
+ FlinkParDoBoundMultiWrapper<IN, OUT> doFnWrapper = new FlinkParDoBoundMultiWrapper<>(
+ context.getPipelineOptions(), windowingStrategy, transform.getFn(),
+ transform.getMainOutputTag(), tagsToLabels);
+
+ DataStream<WindowedValue<IN>> inputDataStream = context.getInputDataStream(context.getInput(transform));
+ SingleOutputStreamOperator<WindowedValue<RawUnionValue>, ?> intermDataStream =
+ inputDataStream.flatMap(doFnWrapper).returns(intermWindowedValueCoder);
+
+ for (Map.Entry<TupleTag<?>, PCollection<?>> output : outputs.entrySet()) {
+ final int outputTag = tagsToLabels.get(output.getKey());
+
+ WindowedValue.WindowedValueCoder<?> coderForTag = WindowedValue.getFullCoder(
+ output.getValue().getCoder(),
+ windowingStrategy.getWindowFn().windowCoder());
+
+ CoderTypeInformation<WindowedValue<?>> windowedValueCoder =
+ new CoderTypeInformation(coderForTag);
+
+ context.setOutputDataStream(output.getValue(),
+ intermDataStream.filter(new FilterFunction<WindowedValue<RawUnionValue>>() {
+ @Override
+ public boolean filter(WindowedValue<RawUnionValue> value) throws Exception {
+ return value.getValue().getUnionTag() == outputTag;
+ }
+ }).flatMap(new FlatMapFunction<WindowedValue<RawUnionValue>, WindowedValue<?>>() {
+ @Override
+ public void flatMap(WindowedValue<RawUnionValue> value, Collector<WindowedValue<?>> collector) throws Exception {
+ collector.collect(WindowedValue.of(
+ value.getValue().getValue(),
+ value.getTimestamp(),
+ value.getWindows(),
+ value.getPane()));
+ }
+ }).returns(windowedValueCoder));
+ }
+ }
+
+ private Map<TupleTag<?>, Integer> transformTupleTagsToLabels(TupleTag<?> mainTag, Set<TupleTag<?>> secondaryTags) {
+ Map<TupleTag<?>, Integer> tagToLabelMap = Maps.newHashMap();
+ tagToLabelMap.put(mainTag, MAIN_TAG_INDEX);
+ int count = MAIN_TAG_INDEX + 1;
+ for (TupleTag<?> tag : secondaryTags) {
+ if (!tagToLabelMap.containsKey(tag)) {
+ tagToLabelMap.put(tag, count++);
+ }
+ }
+ return tagToLabelMap;
+ }
+
+ private UnionCoder getIntermUnionCoder(Collection<PCollection<?>> taggedCollections) {
+ List<Coder<?>> outputCoders = Lists.newArrayList();
+ for (PCollection<?> coll : taggedCollections) {
+ outputCoders.add(coll.getCoder());
+ }
+ return UnionCoder.of(outputCoders);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
new file mode 100644
index 0000000..83ea575
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
@@ -0,0 +1,86 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation;
+
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.values.*;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class FlinkStreamingTranslationContext {
+
+ private final StreamExecutionEnvironment env;
+ private final PipelineOptions options;
+
+ /**
+ * Keeps a mapping between the output value of the PTransform (in Dataflow) and the
+ * Flink Operator that produced it, after the translation of the correspondinf PTransform
+ * to its Flink equivalent.
+ * */
+ private final Map<PValue, DataStream<?>> dataStreams;
+
+ private AppliedPTransform<?, ?, ?> currentTransform;
+
+ public FlinkStreamingTranslationContext(StreamExecutionEnvironment env, PipelineOptions options) {
+ this.env = env;
+ this.options = options;
+ this.dataStreams = new HashMap<>();
+ }
+
+ public StreamExecutionEnvironment getExecutionEnvironment() {
+ return env;
+ }
+
+ public PipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ @SuppressWarnings("unchecked")
+ public <T> DataStream<T> getInputDataStream(PValue value) {
+ return (DataStream<T>) dataStreams.get(value);
+ }
+
+ public void setOutputDataStream(PValue value, DataStream<?> set) {
+ if (!dataStreams.containsKey(value)) {
+ dataStreams.put(value, set);
+ }
+ }
+
+ /**
+ * Sets the AppliedPTransform which carries input/output.
+ * @param currentTransform
+ */
+ public void setCurrentTransform(AppliedPTransform<?, ?, ?> currentTransform) {
+ this.currentTransform = currentTransform;
+ }
+
+ @SuppressWarnings("unchecked")
+ public <I extends PInput> I getInput(PTransform<I, ?> transform) {
+ I input = (I) currentTransform.getInput();
+ return input;
+ }
+
+ @SuppressWarnings("unchecked")
+ public <O extends POutput> O getOutput(PTransform<?, O> transform) {
+ O output = (O) currentTransform.getOutput();
+ return output;
+ }
+}
[21/50] [abbrv] incubator-beam git commit: [flink] adjust directories
according to package name
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java
deleted file mode 100644
index 0836279..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.base.Joiner;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-
-public class WordCountJoin3ITCase extends JavaProgramTestBase {
-
- static final String[] WORDS_1 = new String[] {
- "hi there", "hi", "hi sue bob",
- "hi sue", "", "bob hi"};
-
- static final String[] WORDS_2 = new String[] {
- "hi tim", "beauty", "hooray sue bob",
- "hi there", "", "please say hi"};
-
- static final String[] WORDS_3 = new String[] {
- "hi stephan", "beauty", "hooray big fabian",
- "hi yo", "", "please say hi"};
-
- static final String[] RESULTS = new String[] {
- "beauty -> Tag1: Tag2: 1 Tag3: 1",
- "bob -> Tag1: 2 Tag2: 1 Tag3: ",
- "hi -> Tag1: 5 Tag2: 3 Tag3: 3",
- "hooray -> Tag1: Tag2: 1 Tag3: 1",
- "please -> Tag1: Tag2: 1 Tag3: 1",
- "say -> Tag1: Tag2: 1 Tag3: 1",
- "sue -> Tag1: 2 Tag2: 1 Tag3: ",
- "there -> Tag1: 1 Tag2: 1 Tag3: ",
- "tim -> Tag1: Tag2: 1 Tag3: ",
- "stephan -> Tag1: Tag2: Tag3: 1",
- "yo -> Tag1: Tag2: Tag3: 1",
- "fabian -> Tag1: Tag2: Tag3: 1",
- "big -> Tag1: Tag2: Tag3: 1"
- };
-
- static final TupleTag<Long> tag1 = new TupleTag<>("Tag1");
- static final TupleTag<Long> tag2 = new TupleTag<>("Tag2");
- static final TupleTag<Long> tag3 = new TupleTag<>("Tag3");
-
- protected String resultPath;
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(RESULTS), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
-
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- /* Create two PCollections and join them */
- PCollection<KV<String,Long>> occurences1 = p.apply(Create.of(WORDS_1))
- .apply(ParDo.of(new ExtractWordsFn()))
- .apply(Count.<String>perElement());
-
- PCollection<KV<String,Long>> occurences2 = p.apply(Create.of(WORDS_2))
- .apply(ParDo.of(new ExtractWordsFn()))
- .apply(Count.<String>perElement());
-
- PCollection<KV<String,Long>> occurences3 = p.apply(Create.of(WORDS_3))
- .apply(ParDo.of(new ExtractWordsFn()))
- .apply(Count.<String>perElement());
-
- /* CoGroup the two collections */
- PCollection<KV<String, CoGbkResult>> mergedOccurences = KeyedPCollectionTuple
- .of(tag1, occurences1)
- .and(tag2, occurences2)
- .and(tag3, occurences3)
- .apply(CoGroupByKey.<String>create());
-
- /* Format output */
- mergedOccurences.apply(ParDo.of(new FormatCountsFn()))
- .apply(TextIO.Write.named("test").to(resultPath));
-
- p.run();
- }
-
-
- static class ExtractWordsFn extends DoFn<String, String> {
-
- @Override
- public void startBundle(Context c) {
- }
-
- @Override
- public void processElement(ProcessContext c) {
- // Split the line into words.
- String[] words = c.element().split("[^a-zA-Z']+");
-
- // Output each word encountered into the output PCollection.
- for (String word : words) {
- if (!word.isEmpty()) {
- c.output(word);
- }
- }
- }
- }
-
- static class FormatCountsFn extends DoFn<KV<String, CoGbkResult>, String> {
- @Override
- public void processElement(ProcessContext c) {
- CoGbkResult value = c.element().getValue();
- String key = c.element().getKey();
- String countTag1 = tag1.getId() + ": ";
- String countTag2 = tag2.getId() + ": ";
- String countTag3 = tag3.getId() + ": ";
- for (Long count : value.getAll(tag1)) {
- countTag1 += count + " ";
- }
- for (Long count : value.getAll(tag2)) {
- countTag2 += count + " ";
- }
- for (Long count : value.getAll(tag3)) {
- countTag3 += count;
- }
- c.output(key + " -> " + countTag1 + countTag2 + countTag3);
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java
deleted file mode 100644
index 497a5bb..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java
+++ /dev/null
@@ -1,156 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.Sink;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Write;
-import com.google.common.base.Joiner;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-import java.io.File;
-import java.io.PrintWriter;
-import java.net.URI;
-
-import static org.junit.Assert.*;
-
-/**
- * Tests the translation of custom Write.Bound sinks.
- */
-public class WriteSinkITCase extends JavaProgramTestBase {
-
- protected String resultPath;
-
- public WriteSinkITCase(){
- }
-
- static final String[] EXPECTED_RESULT = new String[] {
- "Joe red 3", "Mary blue 4", "Max yellow 23"};
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
- runProgram(resultPath);
- }
-
- private static void runProgram(String resultPath) {
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- p.apply(Create.of(EXPECTED_RESULT)).setCoder(StringUtf8Coder.of())
- .apply("CustomSink", Write.to(new MyCustomSink(resultPath)));
-
- p.run();
- }
-
- /**
- * Simple custom sink which writes to a file.
- */
- private static class MyCustomSink extends Sink<String> {
-
- private final String resultPath;
-
- public MyCustomSink(String resultPath) {
- this.resultPath = resultPath;
- }
-
- @Override
- public void validate(PipelineOptions options) {
- assertNotNull(options);
- }
-
- @Override
- public WriteOperation<String, ?> createWriteOperation(PipelineOptions options) {
- return new MyWriteOperation();
- }
-
- private class MyWriteOperation extends WriteOperation<String, String> {
-
- @Override
- public Coder<String> getWriterResultCoder() {
- return StringUtf8Coder.of();
- }
-
- @Override
- public void initialize(PipelineOptions options) throws Exception {
-
- }
-
- @Override
- public void finalize(Iterable<String> writerResults, PipelineOptions options) throws Exception {
-
- }
-
- @Override
- public Writer<String, String> createWriter(PipelineOptions options) throws Exception {
- return new MyWriter();
- }
-
- @Override
- public Sink<String> getSink() {
- return MyCustomSink.this;
- }
-
- /**
- * Simple Writer which writes to a file.
- */
- private class MyWriter extends Writer<String, String> {
-
- private PrintWriter internalWriter;
-
- @Override
- public void open(String uId) throws Exception {
- Path path = new Path(resultPath + "/" + uId);
- FileSystem.get(new URI("file:///")).create(path, false);
- internalWriter = new PrintWriter(new File(path.toUri()));
- }
-
- @Override
- public void write(String value) throws Exception {
- internalWriter.println(value);
- }
-
- @Override
- public String close() throws Exception {
- internalWriter.close();
- return resultPath;
- }
-
- @Override
- public WriteOperation<String, String> getWriteOperation() {
- return MyWriteOperation.this;
- }
- }
- }
- }
-
-}
-
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
deleted file mode 100644
index 27ddc83..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
+++ /dev/null
@@ -1,506 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.streaming;
-
-import org.apache.beam.runners.flink.FlinkTestPipeline;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkGroupAlsoByWindowWrapper;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.*;
-import com.google.cloud.dataflow.sdk.util.UserCodeException;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.common.base.Throwables;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-import org.junit.Test;
-
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.concurrent.ConcurrentLinkedQueue;
-
-public class GroupAlsoByWindowTest {
-
- private final Combine.CombineFn combiner = new Sum.SumIntegerFn();
-
- private final WindowingStrategy slidingWindowWithAfterWatermarkTriggerStrategy =
- WindowingStrategy.of(SlidingWindows.of(Duration.standardSeconds(10)).every(Duration.standardSeconds(5)))
- .withTrigger(AfterWatermark.pastEndOfWindow()).withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES);
-
- private final WindowingStrategy sessionWindowingStrategy =
- WindowingStrategy.of(Sessions.withGapDuration(Duration.standardSeconds(2)))
- .withTrigger(Repeatedly.forever(AfterWatermark.pastEndOfWindow()))
- .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES)
- .withAllowedLateness(Duration.standardSeconds(100));
-
- private final WindowingStrategy fixedWindowingStrategy =
- WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(10)));
-
- private final WindowingStrategy fixedWindowWithCountTriggerStrategy =
- fixedWindowingStrategy.withTrigger(AfterPane.elementCountAtLeast(5));
-
- private final WindowingStrategy fixedWindowWithAfterWatermarkTriggerStrategy =
- fixedWindowingStrategy.withTrigger(AfterWatermark.pastEndOfWindow());
-
- private final WindowingStrategy fixedWindowWithCompoundTriggerStrategy =
- fixedWindowingStrategy.withTrigger(
- AfterWatermark.pastEndOfWindow().withEarlyFirings(AfterPane.elementCountAtLeast(5))
- .withLateFirings(AfterPane.elementCountAtLeast(5)).buildTrigger());
-
- /**
- * The default accumulation mode is
- * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#DISCARDING_FIRED_PANES}.
- * This strategy changes it to
- * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#ACCUMULATING_FIRED_PANES}
- */
- private final WindowingStrategy fixedWindowWithCompoundTriggerStrategyAcc =
- fixedWindowWithCompoundTriggerStrategy
- .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES);
-
- @Test
- public void testWithLateness() throws Exception {
- WindowingStrategy strategy = WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(2)))
- .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES)
- .withAllowedLateness(Duration.millis(1000));
- long initialTime = 0L;
- Pipeline pipeline = FlinkTestPipeline.createForStreaming();
-
- KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
-
- FlinkGroupAlsoByWindowWrapper gbwOperaror =
- FlinkGroupAlsoByWindowWrapper.createForTesting(
- pipeline.getOptions(),
- pipeline.getCoderRegistry(),
- strategy,
- inputCoder,
- combiner.<String>asKeyedFn());
-
- OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
- new OneInputStreamOperatorTestHarness<>(gbwOperaror);
- testHarness.open();
-
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processWatermark(new Watermark(initialTime + 2000));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processWatermark(new Watermark(initialTime + 4000));
-
- ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 4),
- new Instant(initialTime + 1),
- new IntervalWindow(new Instant(0), new Instant(2000)),
- PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
- , initialTime + 1));
- expectedOutput.add(new Watermark(initialTime + 2000));
-
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 5),
- new Instant(initialTime + 1999),
- new IntervalWindow(new Instant(0), new Instant(2000)),
- PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 1, 1))
- , initialTime + 1999));
-
-
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 6),
- new Instant(initialTime + 1999),
- new IntervalWindow(new Instant(0), new Instant(2000)),
- PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 2, 2))
- , initialTime + 1999));
- expectedOutput.add(new Watermark(initialTime + 4000));
-
- TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
- testHarness.close();
- }
-
- @Test
- public void testSessionWindows() throws Exception {
- WindowingStrategy strategy = sessionWindowingStrategy;
-
- long initialTime = 0L;
- Pipeline pipeline = FlinkTestPipeline.createForStreaming();
-
- KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
-
- FlinkGroupAlsoByWindowWrapper gbwOperaror =
- FlinkGroupAlsoByWindowWrapper.createForTesting(
- pipeline.getOptions(),
- pipeline.getCoderRegistry(),
- strategy,
- inputCoder,
- combiner.<String>asKeyedFn());
-
- OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
- new OneInputStreamOperatorTestHarness<>(gbwOperaror);
- testHarness.open();
-
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 3500), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 3700), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 2700), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processWatermark(new Watermark(initialTime + 6000));
-
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 6700), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 6800), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 8900), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 7600), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 5600), null, PaneInfo.NO_FIRING), initialTime + 20));
-
- testHarness.processWatermark(new Watermark(initialTime + 12000));
-
- ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 6),
- new Instant(initialTime + 1),
- new IntervalWindow(new Instant(1), new Instant(5700)),
- PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
- , initialTime + 1));
- expectedOutput.add(new Watermark(initialTime + 6000));
-
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 11),
- new Instant(initialTime + 6700),
- new IntervalWindow(new Instant(1), new Instant(10900)),
- PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
- , initialTime + 6700));
- expectedOutput.add(new Watermark(initialTime + 12000));
-
- TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
- testHarness.close();
- }
-
- @Test
- public void testSlidingWindows() throws Exception {
- WindowingStrategy strategy = slidingWindowWithAfterWatermarkTriggerStrategy;
- long initialTime = 0L;
- OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
- createTestingOperatorAndState(strategy, initialTime);
- ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
- testHarness.processWatermark(new Watermark(initialTime + 25000));
-
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 6),
- new Instant(initialTime + 5000),
- new IntervalWindow(new Instant(0), new Instant(10000)),
- PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime + 5000));
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 6),
- new Instant(initialTime + 1),
- new IntervalWindow(new Instant(-5000), new Instant(5000)),
- PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime + 1));
- expectedOutput.add(new Watermark(initialTime + 10000));
-
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 11),
- new Instant(initialTime + 15000),
- new IntervalWindow(new Instant(10000), new Instant(20000)),
- PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime + 15000));
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 3),
- new Instant(initialTime + 10000),
- new IntervalWindow(new Instant(5000), new Instant(15000)),
- PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime + 10000));
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key2", 1),
- new Instant(initialTime + 19500),
- new IntervalWindow(new Instant(10000), new Instant(20000)),
- PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime + 19500));
- expectedOutput.add(new Watermark(initialTime + 20000));
-
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key2", 1),
- new Instant(initialTime + 20000),
- /**
- * this is 20000 and not 19500 because of a convention in dataflow where
- * timestamps of windowed values in a window cannot be smaller than the
- * end of a previous window. Checkout the documentation of the
- * {@link WindowFn#getOutputTime(Instant, BoundedWindow)}
- */
- new IntervalWindow(new Instant(15000), new Instant(25000)),
- PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime + 20000));
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 8),
- new Instant(initialTime + 20000),
- new IntervalWindow(new Instant(15000), new Instant(25000)),
- PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime + 20000));
- expectedOutput.add(new Watermark(initialTime + 25000));
-
- TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
- testHarness.close();
- }
-
- @Test
- public void testAfterWatermarkProgram() throws Exception {
- WindowingStrategy strategy = fixedWindowWithAfterWatermarkTriggerStrategy;
- long initialTime = 0L;
- OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
- createTestingOperatorAndState(strategy, initialTime);
- ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6),
- new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 1));
- expectedOutput.add(new Watermark(initialTime + 10000));
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11),
- new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 10000));
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500));
- expectedOutput.add(new Watermark(initialTime + 20000));
-
- TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
- testHarness.close();
- }
-
- @Test
- public void testAfterCountProgram() throws Exception {
- WindowingStrategy strategy = fixedWindowWithCountTriggerStrategy;
-
- long initialTime = 0L;
- OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
- createTestingOperatorAndState(strategy, initialTime);
- ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime + 1));
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime + 10000));
- expectedOutput.add(new Watermark(initialTime + 10000));
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME, 0, 0)), initialTime + 19500));
- expectedOutput.add(new Watermark(initialTime + 20000));
- TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
-
- testHarness.close();
- }
-
- @Test
- public void testCompoundProgram() throws Exception {
- WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategy;
-
- long initialTime = 0L;
- OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
- createTestingOperatorAndState(strategy, initialTime);
- ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
- /**
- * PaneInfo are:
- * isFirst (pane in window),
- * isLast, Timing (of triggering),
- * index (of pane in the window),
- * onTimeIndex (if it the 1st,2nd, ... pane that was fired on time)
- * */
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 1));
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 10000));
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime + 19500));
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1),
- new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime + 1200));
-
- expectedOutput.add(new Watermark(initialTime + 10000));
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime + 19500));
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500));
-
- expectedOutput.add(new Watermark(initialTime + 20000));
- TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
-
- testHarness.close();
- }
-
- @Test
- public void testCompoundAccumulatingPanesProgram() throws Exception {
- WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategyAcc;
- long initialTime = 0L;
- OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
- createTestingOperatorAndState(strategy, initialTime);
- ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 1));
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 10000));
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 10),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime + 19500));
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6),
- new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime + 1200));
-
- expectedOutput.add(new Watermark(initialTime + 10000));
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime + 19500));
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500));
-
- expectedOutput.add(new Watermark(initialTime + 20000));
- TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
-
- testHarness.close();
- }
-
- private OneInputStreamOperatorTestHarness createTestingOperatorAndState(WindowingStrategy strategy, long initialTime) throws Exception {
- Pipeline pipeline = FlinkTestPipeline.createForStreaming();
-
- KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
-
- FlinkGroupAlsoByWindowWrapper gbwOperaror =
- FlinkGroupAlsoByWindowWrapper.createForTesting(
- pipeline.getOptions(),
- pipeline.getCoderRegistry(),
- strategy,
- inputCoder,
- combiner.<String>asKeyedFn());
-
- OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
- new OneInputStreamOperatorTestHarness<>(gbwOperaror);
- testHarness.open();
-
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
-
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 10000), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 12100), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 14200), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 15300), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 16500), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
-
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
-
- testHarness.processWatermark(new Watermark(initialTime + 10000));
- testHarness.processWatermark(new Watermark(initialTime + 20000));
-
- return testHarness;
- }
-
- private static class ResultSortComparator implements Comparator<Object> {
- @Override
- public int compare(Object o1, Object o2) {
- if (o1 instanceof Watermark && o2 instanceof Watermark) {
- Watermark w1 = (Watermark) o1;
- Watermark w2 = (Watermark) o2;
- return (int) (w1.getTimestamp() - w2.getTimestamp());
- } else {
- StreamRecord<WindowedValue<KV<String, Integer>>> sr0 = (StreamRecord<WindowedValue<KV<String, Integer>>>) o1;
- StreamRecord<WindowedValue<KV<String, Integer>>> sr1 = (StreamRecord<WindowedValue<KV<String, Integer>>>) o2;
-
- int comparison = (int) (sr0.getValue().getTimestamp().getMillis() - sr1.getValue().getTimestamp().getMillis());
- if (comparison != 0) {
- return comparison;
- }
-
- comparison = sr0.getValue().getValue().getKey().compareTo(sr1.getValue().getValue().getKey());
- if(comparison == 0) {
- comparison = Integer.compare(
- sr0.getValue().getValue().getValue(),
- sr1.getValue().getValue().getValue());
- }
- if(comparison == 0) {
- Collection windowsA = sr0.getValue().getWindows();
- Collection windowsB = sr1.getValue().getWindows();
-
- if(windowsA.size() != 1 || windowsB.size() != 1) {
- throw new IllegalStateException("A value cannot belong to more than one windows after grouping.");
- }
-
- BoundedWindow windowA = (BoundedWindow) windowsA.iterator().next();
- BoundedWindow windowB = (BoundedWindow) windowsB.iterator().next();
- comparison = Long.compare(windowA.maxTimestamp().getMillis(), windowB.maxTimestamp().getMillis());
- }
- return comparison;
- }
- }
- }
-
- private <T> WindowedValue<T> makeWindowedValue(WindowingStrategy strategy,
- T output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
- final Instant inputTimestamp = timestamp;
- final WindowFn windowFn = strategy.getWindowFn();
-
- if (timestamp == null) {
- timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
- }
-
- if (windows == null) {
- try {
- windows = windowFn.assignWindows(windowFn.new AssignContext() {
- @Override
- public Object element() {
- throw new UnsupportedOperationException(
- "WindowFn attempted to access input element when none was available");
- }
-
- @Override
- public Instant timestamp() {
- if (inputTimestamp == null) {
- throw new UnsupportedOperationException(
- "WindowFn attempted to access input timestamp when none was available");
- }
- return inputTimestamp;
- }
-
- @Override
- public Collection<? extends BoundedWindow> windows() {
- throw new UnsupportedOperationException(
- "WindowFn attempted to access input windows when none were available");
- }
- });
- } catch (Exception e) {
- throw UserCodeException.wrap(e);
- }
- }
-
- return WindowedValue.of(output, timestamp, windows, pane);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java
deleted file mode 100644
index 80d78b9..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.streaming;
-
-import org.apache.beam.runners.flink.FlinkTestPipeline;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.AfterWatermark;
-import com.google.cloud.dataflow.sdk.transforms.windowing.FixedWindows;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Joiner;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-import java.io.Serializable;
-import java.util.Arrays;
-
-public class GroupByNullKeyTest extends StreamingProgramTestBase implements Serializable {
-
-
- protected String resultPath;
-
- static final String[] EXPECTED_RESULT = new String[] {
- "k: null v: user1 user1 user1 user2 user2 user2 user2 user3"
- };
-
- public GroupByNullKeyTest(){
- }
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
-
- public static class ExtractUserAndTimestamp extends DoFn<KV<Integer, String>, String> {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- KV<Integer, String> record = c.element();
- long now = System.currentTimeMillis();
- int timestamp = record.getKey();
- String userName = record.getValue();
- if (userName != null) {
- // Sets the implicit timestamp field to be used in windowing.
- c.outputWithTimestamp(userName, new Instant(timestamp + now));
- }
- }
- }
-
- @Override
- protected void testProgram() throws Exception {
-
- Pipeline p = FlinkTestPipeline.createForStreaming();
-
- PCollection<String> output =
- p.apply(Create.of(Arrays.asList(
- KV.<Integer, String>of(0, "user1"),
- KV.<Integer, String>of(1, "user1"),
- KV.<Integer, String>of(2, "user1"),
- KV.<Integer, String>of(10, "user2"),
- KV.<Integer, String>of(1, "user2"),
- KV.<Integer, String>of(15000, "user2"),
- KV.<Integer, String>of(12000, "user2"),
- KV.<Integer, String>of(25000, "user3"))))
- .apply(ParDo.of(new ExtractUserAndTimestamp()))
- .apply(Window.<String>into(FixedWindows.of(Duration.standardHours(1)))
- .triggering(AfterWatermark.pastEndOfWindow())
- .withAllowedLateness(Duration.ZERO)
- .discardingFiredPanes())
-
- .apply(ParDo.of(new DoFn<String, KV<Void, String>>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- String elem = c.element();
- c.output(KV.<Void, String>of((Void) null, elem));
- }
- }))
- .apply(GroupByKey.<Void, String>create())
- .apply(ParDo.of(new DoFn<KV<Void, Iterable<String>>, String>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- KV<Void, Iterable<String>> elem = c.element();
- StringBuilder str = new StringBuilder();
- str.append("k: " + elem.getKey() + " v:");
- for (String v : elem.getValue()) {
- str.append(" " + v);
- }
- c.output(str.toString());
- }
- }));
- output.apply(TextIO.Write.to(resultPath));
- p.run();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java
deleted file mode 100644
index e6c5ae2..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java
+++ /dev/null
@@ -1,303 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.streaming;
-
-import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.state.StateCheckpointReader;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.state.StateCheckpointUtils;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.state.StateCheckpointWriter;
-import com.google.cloud.dataflow.sdk.coders.*;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext;
-import com.google.cloud.dataflow.sdk.transforms.Sum;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-import com.google.cloud.dataflow.sdk.util.state.*;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.runtime.state.AbstractStateBackend;
-import org.apache.flink.runtime.state.memory.MemoryStateBackend;
-import org.apache.flink.runtime.util.DataInputDeserializer;
-import org.joda.time.Instant;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import static org.junit.Assert.assertEquals;
-
-public class StateSerializationTest {
-
- private static final StateNamespace NAMESPACE_1 = StateNamespaces.global();
- private static final String KEY_PREFIX = "TEST_";
-
- // TODO: This can be replaced with the standard Sum.SumIntererFn once the state no longer needs
- // to create a StateTag at the point of restoring state. Currently StateTags are compared strictly
- // by type and combiners always use KeyedCombineFnWithContext rather than KeyedCombineFn or CombineFn.
- private static CombineWithContext.KeyedCombineFnWithContext<Object, Integer, int[], Integer> SUM_COMBINER =
- new CombineWithContext.KeyedCombineFnWithContext<Object, Integer, int[], Integer>() {
- @Override
- public int[] createAccumulator(Object key, CombineWithContext.Context c) {
- return new int[1];
- }
-
- @Override
- public int[] addInput(Object key, int[] accumulator, Integer value, CombineWithContext.Context c) {
- accumulator[0] += value;
- return accumulator;
- }
-
- @Override
- public int[] mergeAccumulators(Object key, Iterable<int[]> accumulators, CombineWithContext.Context c) {
- int[] r = new int[1];
- for (int[] a : accumulators) {
- r[0] += a[0];
- }
- return r;
- }
-
- @Override
- public Integer extractOutput(Object key, int[] accumulator, CombineWithContext.Context c) {
- return accumulator[0];
- }
- };
-
- private static Coder<int[]> INT_ACCUM_CODER = DelegateCoder.of(
- VarIntCoder.of(),
- new DelegateCoder.CodingFunction<int[], Integer>() {
- @Override
- public Integer apply(int[] accumulator) {
- return accumulator[0];
- }
- },
- new DelegateCoder.CodingFunction<Integer, int[]>() {
- @Override
- public int[] apply(Integer value) {
- int[] a = new int[1];
- a[0] = value;
- return a;
- }
- });
-
- private static final StateTag<Object, ValueState<String>> STRING_VALUE_ADDR =
- StateTags.value("stringValue", StringUtf8Coder.of());
- private static final StateTag<Object, ValueState<Integer>> INT_VALUE_ADDR =
- StateTags.value("stringValue", VarIntCoder.of());
- private static final StateTag<Object, AccumulatorCombiningState<Integer, int[], Integer>> SUM_INTEGER_ADDR =
- StateTags.keyedCombiningValueWithContext("sumInteger", INT_ACCUM_CODER, SUM_COMBINER);
- private static final StateTag<Object, BagState<String>> STRING_BAG_ADDR =
- StateTags.bag("stringBag", StringUtf8Coder.of());
- private static final StateTag<Object, WatermarkHoldState<BoundedWindow>> WATERMARK_BAG_ADDR =
- StateTags.watermarkStateInternal("watermark", OutputTimeFns.outputAtEarliestInputTimestamp());
-
- private Map<String, FlinkStateInternals<String>> statePerKey = new HashMap<>();
-
- private Map<String, Set<TimerInternals.TimerData>> activeTimers = new HashMap<>();
-
- private void initializeStateAndTimers() throws CannotProvideCoderException {
- for (int i = 0; i < 10; i++) {
- String key = KEY_PREFIX + i;
-
- FlinkStateInternals state = initializeStateForKey(key);
- Set<TimerInternals.TimerData> timers = new HashSet<>();
- for (int j = 0; j < 5; j++) {
- TimerInternals.TimerData timer = TimerInternals
- .TimerData.of(NAMESPACE_1,
- new Instant(1000 + i + j), TimeDomain.values()[j % 3]);
- timers.add(timer);
- }
-
- statePerKey.put(key, state);
- activeTimers.put(key, timers);
- }
- }
-
- private FlinkStateInternals<String> initializeStateForKey(String key) throws CannotProvideCoderException {
- FlinkStateInternals<String> state = createState(key);
-
- ValueState<String> value = state.state(NAMESPACE_1, STRING_VALUE_ADDR);
- value.write("test");
-
- ValueState<Integer> value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR);
- value2.write(4);
- value2.write(5);
-
- AccumulatorCombiningState<Integer, int[], Integer> combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR);
- combiningValue.add(1);
- combiningValue.add(2);
-
- WatermarkHoldState<BoundedWindow> watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
- watermark.add(new Instant(1000));
-
- BagState<String> bag = state.state(NAMESPACE_1, STRING_BAG_ADDR);
- bag.add("v1");
- bag.add("v2");
- bag.add("v3");
- bag.add("v4");
- return state;
- }
-
- private boolean restoreAndTestState(DataInputView in) throws Exception {
- StateCheckpointReader reader = new StateCheckpointReader(in);
- final ClassLoader userClassloader = this.getClass().getClassLoader();
- Coder<? extends BoundedWindow> windowCoder = IntervalWindow.getCoder();
- Coder<String> keyCoder = StringUtf8Coder.of();
-
- boolean comparisonRes = true;
-
- for (String key : statePerKey.keySet()) {
- comparisonRes &= checkStateForKey(key);
- }
-
- // restore the timers
- Map<String, Set<TimerInternals.TimerData>> restoredTimersPerKey = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder);
- if (activeTimers.size() != restoredTimersPerKey.size()) {
- return false;
- }
-
- for (String key : statePerKey.keySet()) {
- Set<TimerInternals.TimerData> originalTimers = activeTimers.get(key);
- Set<TimerInternals.TimerData> restoredTimers = restoredTimersPerKey.get(key);
- comparisonRes &= checkTimersForKey(originalTimers, restoredTimers);
- }
-
- // restore the state
- Map<String, FlinkStateInternals<String>> restoredPerKeyState =
- StateCheckpointUtils.decodeState(reader, OutputTimeFns.outputAtEarliestInputTimestamp(), keyCoder, windowCoder, userClassloader);
- if (restoredPerKeyState.size() != statePerKey.size()) {
- return false;
- }
-
- for (String key : statePerKey.keySet()) {
- FlinkStateInternals<String> originalState = statePerKey.get(key);
- FlinkStateInternals<String> restoredState = restoredPerKeyState.get(key);
- comparisonRes &= checkStateForKey(originalState, restoredState);
- }
- return comparisonRes;
- }
-
- private boolean checkStateForKey(String key) throws CannotProvideCoderException {
- FlinkStateInternals<String> state = statePerKey.get(key);
-
- ValueState<String> value = state.state(NAMESPACE_1, STRING_VALUE_ADDR);
- boolean comp = value.read().equals("test");
-
- ValueState<Integer> value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR);
- comp &= value2.read().equals(5);
-
- AccumulatorCombiningState<Integer, int[], Integer> combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR);
- comp &= combiningValue.read().equals(3);
-
- WatermarkHoldState<BoundedWindow> watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
- comp &= watermark.read().equals(new Instant(1000));
-
- BagState<String> bag = state.state(NAMESPACE_1, STRING_BAG_ADDR);
- Iterator<String> it = bag.read().iterator();
- int i = 0;
- while (it.hasNext()) {
- comp &= it.next().equals("v" + (++i));
- }
- return comp;
- }
-
- private void storeState(AbstractStateBackend.CheckpointStateOutputView out) throws Exception {
- StateCheckpointWriter checkpointBuilder = StateCheckpointWriter.create(out);
- Coder<String> keyCoder = StringUtf8Coder.of();
-
- // checkpoint the timers
- StateCheckpointUtils.encodeTimers(activeTimers, checkpointBuilder, keyCoder);
-
- // checkpoint the state
- StateCheckpointUtils.encodeState(statePerKey, checkpointBuilder, keyCoder);
- }
-
- private boolean checkTimersForKey(Set<TimerInternals.TimerData> originalTimers, Set<TimerInternals.TimerData> restoredTimers) {
- boolean comp = true;
- if (restoredTimers == null) {
- return false;
- }
-
- if (originalTimers.size() != restoredTimers.size()) {
- return false;
- }
-
- for (TimerInternals.TimerData timer : originalTimers) {
- comp &= restoredTimers.contains(timer);
- }
- return comp;
- }
-
- private boolean checkStateForKey(FlinkStateInternals<String> originalState, FlinkStateInternals<String> restoredState) throws CannotProvideCoderException {
- if (restoredState == null) {
- return false;
- }
-
- ValueState<String> orValue = originalState.state(NAMESPACE_1, STRING_VALUE_ADDR);
- ValueState<String> resValue = restoredState.state(NAMESPACE_1, STRING_VALUE_ADDR);
- boolean comp = orValue.read().equals(resValue.read());
-
- ValueState<Integer> orIntValue = originalState.state(NAMESPACE_1, INT_VALUE_ADDR);
- ValueState<Integer> resIntValue = restoredState.state(NAMESPACE_1, INT_VALUE_ADDR);
- comp &= orIntValue.read().equals(resIntValue.read());
-
- AccumulatorCombiningState<Integer, int[], Integer> combOrValue = originalState.state(NAMESPACE_1, SUM_INTEGER_ADDR);
- AccumulatorCombiningState<Integer, int[], Integer> combResValue = restoredState.state(NAMESPACE_1, SUM_INTEGER_ADDR);
- comp &= combOrValue.read().equals(combResValue.read());
-
- WatermarkHoldState orWatermark = originalState.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
- WatermarkHoldState resWatermark = restoredState.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
- comp &= orWatermark.read().equals(resWatermark.read());
-
- BagState<String> orBag = originalState.state(NAMESPACE_1, STRING_BAG_ADDR);
- BagState<String> resBag = restoredState.state(NAMESPACE_1, STRING_BAG_ADDR);
-
- Iterator<String> orIt = orBag.read().iterator();
- Iterator<String> resIt = resBag.read().iterator();
-
- while (orIt.hasNext() && resIt.hasNext()) {
- comp &= orIt.next().equals(resIt.next());
- }
-
- return !((orIt.hasNext() && !resIt.hasNext()) || (!orIt.hasNext() && resIt.hasNext())) && comp;
- }
-
- private FlinkStateInternals<String> createState(String key) throws CannotProvideCoderException {
- return new FlinkStateInternals<>(
- key,
- StringUtf8Coder.of(),
- IntervalWindow.getCoder(),
- OutputTimeFns.outputAtEarliestInputTimestamp());
- }
-
- @Test
- public void test() throws Exception {
- StateSerializationTest test = new StateSerializationTest();
- test.initializeStateAndTimers();
-
- MemoryStateBackend.MemoryCheckpointOutputStream memBackend = new MemoryStateBackend.MemoryCheckpointOutputStream(32048);
- AbstractStateBackend.CheckpointStateOutputView out = new AbstractStateBackend.CheckpointStateOutputView(memBackend);
-
- test.storeState(out);
-
- byte[] contents = memBackend.closeAndGetBytes();
- DataInputView in = new DataInputDeserializer(contents, 0, contents.length);
-
- assertEquals(test.restoreAndTestState(in), true);
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java
deleted file mode 100644
index f0b93a0..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.streaming;
-
-import org.apache.beam.runners.flink.FlinkTestPipeline;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Joiner;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-import java.io.Serializable;
-import java.util.Arrays;
-
-
-/**
- * Session window test
- */
-public class TopWikipediaSessionsITCase extends StreamingProgramTestBase implements Serializable {
- protected String resultPath;
-
- public TopWikipediaSessionsITCase(){
- }
-
- static final String[] EXPECTED_RESULT = new String[] {
- "user: user1 value:3",
- "user: user1 value:1",
- "user: user2 value:4",
- "user: user2 value:6",
- "user: user3 value:7",
- "user: user3 value:2"
- };
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
-
- Pipeline p = FlinkTestPipeline.createForStreaming();
-
- Long now = (System.currentTimeMillis() + 10000) / 1000;
-
- PCollection<KV<String, Long>> output =
- p.apply(Create.of(Arrays.asList(new TableRow().set("timestamp", now).set
- ("contributor_username", "user1"), new TableRow().set("timestamp", now + 10).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now).set
- ("contributor_username", "user1"), new TableRow().set("timestamp", now + 2).set
- ("contributor_username", "user1"), new TableRow().set("timestamp", now).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 1).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 5).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 7).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 8).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 200).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 230).set
- ("contributor_username", "user1"), new TableRow().set("timestamp", now + 230).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 240).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 245).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 235).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 236).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 237).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 238).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 239).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 240).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 241).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now)
- .set("contributor_username", "user3"))))
-
-
-
- .apply(ParDo.of(new DoFn<TableRow, String>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- TableRow row = c.element();
- long timestamp = (Integer) row.get("timestamp");
- String userName = (String) row.get("contributor_username");
- if (userName != null) {
- // Sets the timestamp field to be used in windowing.
- c.outputWithTimestamp(userName, new Instant(timestamp * 1000L));
- }
- }
- }))
-
- .apply(Window.<String>into(Sessions.withGapDuration(Duration.standardMinutes(1))))
-
- .apply(Count.<String>perElement());
-
- PCollection<String> format = output.apply(ParDo.of(new DoFn<KV<String, Long>, String>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- KV<String, Long> el = c.element();
- String out = "user: " + el.getKey() + " value:" + el.getValue();
- c.output(out);
- }
- }));
-
- format.apply(TextIO.Write.to(resultPath));
-
- p.run();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java
deleted file mode 100644
index 620dace..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java
+++ /dev/null
@@ -1,158 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.util;
-
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.BigQueryIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.options.Validation;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-
-/**
- * Copied from {@link com.google.cloud.dataflow.examples.JoinExamples} because the code
- * is private there.
- */
-public class JoinExamples {
-
- // A 1000-row sample of the GDELT data here: gdelt-bq:full.events.
- private static final String GDELT_EVENTS_TABLE =
- "clouddataflow-readonly:samples.gdelt_sample";
- // A table that maps country codes to country names.
- private static final String COUNTRY_CODES =
- "gdelt-bq:full.crosswalk_geocountrycodetohuman";
-
- /**
- * Join two collections, using country code as the key.
- */
- public static PCollection<String> joinEvents(PCollection<TableRow> eventsTable,
- PCollection<TableRow> countryCodes) throws Exception {
-
- final TupleTag<String> eventInfoTag = new TupleTag<>();
- final TupleTag<String> countryInfoTag = new TupleTag<>();
-
- // transform both input collections to tuple collections, where the keys are country
- // codes in both cases.
- PCollection<KV<String, String>> eventInfo = eventsTable.apply(
- ParDo.of(new ExtractEventDataFn()));
- PCollection<KV<String, String>> countryInfo = countryCodes.apply(
- ParDo.of(new ExtractCountryInfoFn()));
-
- // country code 'key' -> CGBKR (<event info>, <country name>)
- PCollection<KV<String, CoGbkResult>> kvpCollection = KeyedPCollectionTuple
- .of(eventInfoTag, eventInfo)
- .and(countryInfoTag, countryInfo)
- .apply(CoGroupByKey.<String>create());
-
- // Process the CoGbkResult elements generated by the CoGroupByKey transform.
- // country code 'key' -> string of <event info>, <country name>
- PCollection<KV<String, String>> finalResultCollection =
- kvpCollection.apply(ParDo.of(new DoFn<KV<String, CoGbkResult>, KV<String, String>>() {
- @Override
- public void processElement(ProcessContext c) {
- KV<String, CoGbkResult> e = c.element();
- CoGbkResult val = e.getValue();
- String countryCode = e.getKey();
- String countryName;
- countryName = e.getValue().getOnly(countryInfoTag, "Kostas");
- for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) {
- // Generate a string that combines information from both collection values
- c.output(KV.of(countryCode, "Country name: " + countryName
- + ", Event info: " + eventInfo));
- }
- }
- }));
-
- // write to GCS
- return finalResultCollection
- .apply(ParDo.of(new DoFn<KV<String, String>, String>() {
- @Override
- public void processElement(ProcessContext c) {
- String outputstring = "Country code: " + c.element().getKey()
- + ", " + c.element().getValue();
- c.output(outputstring);
- }
- }));
- }
-
- /**
- * Examines each row (event) in the input table. Output a KV with the key the country
- * code of the event, and the value a string encoding event information.
- */
- static class ExtractEventDataFn extends DoFn<TableRow, KV<String, String>> {
- @Override
- public void processElement(ProcessContext c) {
- TableRow row = c.element();
- String countryCode = (String) row.get("ActionGeo_CountryCode");
- String sqlDate = (String) row.get("SQLDATE");
- String actor1Name = (String) row.get("Actor1Name");
- String sourceUrl = (String) row.get("SOURCEURL");
- String eventInfo = "Date: " + sqlDate + ", Actor1: " + actor1Name + ", url: " + sourceUrl;
- c.output(KV.of(countryCode, eventInfo));
- }
- }
-
-
- /**
- * Examines each row (country info) in the input table. Output a KV with the key the country
- * code, and the value the country name.
- */
- static class ExtractCountryInfoFn extends DoFn<TableRow, KV<String, String>> {
- @Override
- public void processElement(ProcessContext c) {
- TableRow row = c.element();
- String countryCode = (String) row.get("FIPSCC");
- String countryName = (String) row.get("HumanName");
- c.output(KV.of(countryCode, countryName));
- }
- }
-
-
- /**
- * Options supported by {@link JoinExamples}.
- * <p>
- * Inherits standard configuration options.
- */
- private interface Options extends PipelineOptions {
- @Description("Path of the file to write to")
- @Validation.Required
- String getOutput();
- void setOutput(String value);
- }
-
- public static void main(String[] args) throws Exception {
- Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
- Pipeline p = Pipeline.create(options);
- // the following two 'applys' create multiple inputs to our pipeline, one for each
- // of our two input sources.
- PCollection<TableRow> eventsTable = p.apply(BigQueryIO.Read.from(GDELT_EVENTS_TABLE));
- PCollection<TableRow> countryCodes = p.apply(BigQueryIO.Read.from(COUNTRY_CODES));
- PCollection<String> formattedResults = joinEvents(eventsTable, countryCodes);
- formattedResults.apply(TextIO.Write.to(options.getOutput()));
- p.run();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/AvroITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/AvroITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/AvroITCase.java
new file mode 100644
index 0000000..eaa5979
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/AvroITCase.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.AvroCoder;
+import com.google.cloud.dataflow.sdk.io.AvroIO;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.common.base.Joiner;
+import org.apache.flink.api.io.avro.example.User;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+
+public class AvroITCase extends JavaProgramTestBase {
+
+ protected String resultPath;
+ protected String tmpPath;
+
+ public AvroITCase(){
+ }
+
+ static final String[] EXPECTED_RESULT = new String[] {
+ "Joe red 3",
+ "Mary blue 4",
+ "Mark green 1",
+ "Julia purple 5"
+ };
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ tmpPath = getTempDirPath("tmp");
+
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+ runProgram(tmpPath, resultPath);
+ }
+
+ private static void runProgram(String tmpPath, String resultPath) {
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ p
+ .apply(Create.of(
+ new User("Joe", 3, "red"),
+ new User("Mary", 4, "blue"),
+ new User("Mark", 1, "green"),
+ new User("Julia", 5, "purple"))
+ .withCoder(AvroCoder.of(User.class)))
+
+ .apply(AvroIO.Write.to(tmpPath)
+ .withSchema(User.class));
+
+ p.run();
+
+ p = FlinkTestPipeline.createForBatch();
+
+ p
+ .apply(AvroIO.Read.from(tmpPath).withSchema(User.class).withoutValidation())
+
+ .apply(ParDo.of(new DoFn<User, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ User u = c.element();
+ String result = u.getName() + " " + u.getFavoriteColor() + " " + u.getFavoriteNumber();
+ c.output(result);
+ }
+ }))
+
+ .apply(TextIO.Write.to(resultPath));
+
+ p.run();
+ }
+
+}
+
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java
new file mode 100644
index 0000000..79eb163
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlattenizeITCase.java
@@ -0,0 +1,72 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.Flatten;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import com.google.common.base.Joiner;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+public class FlattenizeITCase extends JavaProgramTestBase {
+
+ private String resultPath;
+ private String resultPath2;
+
+ private static final String[] words = {"hello", "this", "is", "a", "DataSet!"};
+ private static final String[] words2 = {"hello", "this", "is", "another", "DataSet!"};
+ private static final String[] words3 = {"hello", "this", "is", "yet", "another", "DataSet!"};
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ resultPath2 = getTempDirPath("result2");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ String join = Joiner.on('\n').join(words);
+ String join2 = Joiner.on('\n').join(words2);
+ String join3 = Joiner.on('\n').join(words3);
+ compareResultsByLinesInMemory(join + "\n" + join2, resultPath);
+ compareResultsByLinesInMemory(join + "\n" + join2 + "\n" + join3, resultPath2);
+ }
+
+
+ @Override
+ protected void testProgram() throws Exception {
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ PCollection<String> p1 = p.apply(Create.of(words));
+ PCollection<String> p2 = p.apply(Create.of(words2));
+
+ PCollectionList<String> list = PCollectionList.of(p1).and(p2);
+
+ list.apply(Flatten.<String>pCollections()).apply(TextIO.Write.to(resultPath));
+
+ PCollection<String> p3 = p.apply(Create.of(words3));
+
+ PCollectionList<String> list2 = list.and(p3);
+
+ list2.apply(Flatten.<String>pCollections()).apply(TextIO.Write.to(resultPath2));
+
+ p.run();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
new file mode 100644
index 0000000..2dcebde
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTestPipeline.java
@@ -0,0 +1,70 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.PipelineResult;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
+
+/**
+ * {@link com.google.cloud.dataflow.sdk.Pipeline} for testing Dataflow programs on the
+ * {@link org.apache.beam.runners.flink.FlinkPipelineRunner}.
+ */
+public class FlinkTestPipeline extends Pipeline {
+
+ /**
+ * Creates and returns a new test pipeline for batch execution.
+ *
+ * <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
+ * {@link Pipeline#run} to execute the pipeline and check the tests.
+ */
+ public static FlinkTestPipeline createForBatch() {
+ return create(false);
+ }
+
+ /**
+ * Creates and returns a new test pipeline for streaming execution.
+ *
+ * <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
+ * {@link Pipeline#run} to execute the pipeline and check the tests.
+ *
+ * @return The Test Pipeline
+ */
+ public static FlinkTestPipeline createForStreaming() {
+ return create(true);
+ }
+
+ /**
+ * Creates and returns a new test pipeline for streaming or batch execution.
+ *
+ * <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
+ * {@link Pipeline#run} to execute the pipeline and check the tests.
+ *
+ * @param streaming <code>True</code> for streaming mode, <code>False</code> for batch.
+ * @return The Test Pipeline.
+ */
+ private static FlinkTestPipeline create(boolean streaming) {
+ FlinkPipelineRunner flinkRunner = FlinkPipelineRunner.createForTest(streaming);
+ return new FlinkTestPipeline(flinkRunner, flinkRunner.getPipelineOptions());
+ }
+
+ private FlinkTestPipeline(PipelineRunner<? extends PipelineResult> runner,
+ PipelineOptions options) {
+ super(runner, options);
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java
new file mode 100644
index 0000000..11b6ce4
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/JoinExamplesITCase.java
@@ -0,0 +1,99 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import org.apache.beam.runners.flink.util.JoinExamples;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Joiner;
+import org.apache.flink.test.util.JavaProgramTestBase;
+
+import java.util.Arrays;
+import java.util.List;
+
+
+/**
+ * Unfortunately we need to copy the code from the Dataflow SDK because it is not public there.
+ */
+public class JoinExamplesITCase extends JavaProgramTestBase {
+
+ protected String resultPath;
+
+ public JoinExamplesITCase(){
+ }
+
+ private static final TableRow row1 = new TableRow()
+ .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212")
+ .set("Actor1Name", "BANGKOK").set("SOURCEURL", "http://cnn.com");
+ private static final TableRow row2 = new TableRow()
+ .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212")
+ .set("Actor1Name", "LAOS").set("SOURCEURL", "http://www.chicagotribune.com");
+ private static final TableRow row3 = new TableRow()
+ .set("ActionGeo_CountryCode", "BE").set("SQLDATE", "20141213")
+ .set("Actor1Name", "AFGHANISTAN").set("SOURCEURL", "http://cnn.com");
+ static final TableRow[] EVENTS = new TableRow[] {
+ row1, row2, row3
+ };
+ static final List<TableRow> EVENT_ARRAY = Arrays.asList(EVENTS);
+
+ private static final TableRow cc1 = new TableRow()
+ .set("FIPSCC", "VM").set("HumanName", "Vietnam");
+ private static final TableRow cc2 = new TableRow()
+ .set("FIPSCC", "BE").set("HumanName", "Belgium");
+ static final TableRow[] CCS = new TableRow[] {
+ cc1, cc2
+ };
+ static final List<TableRow> CC_ARRAY = Arrays.asList(CCS);
+
+ static final String[] JOINED_EVENTS = new String[] {
+ "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: LAOS, "
+ + "url: http://www.chicagotribune.com",
+ "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: BANGKOK, "
+ + "url: http://cnn.com",
+ "Country code: BE, Country name: Belgium, Event info: Date: 20141213, Actor1: AFGHANISTAN, "
+ + "url: http://cnn.com"
+ };
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(JOINED_EVENTS), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+
+ Pipeline p = FlinkTestPipeline.createForBatch();
+
+ PCollection<TableRow> input1 = p.apply(Create.of(EVENT_ARRAY));
+ PCollection<TableRow> input2 = p.apply(Create.of(CC_ARRAY));
+
+ PCollection<String> output = JoinExamples.joinEvents(input1, input2);
+
+ output.apply(TextIO.Write.to(resultPath));
+
+ p.run();
+ }
+}
+
[17/50] [abbrv] incubator-beam git commit: Adds javadocs.
Posted by da...@apache.org.
Adds javadocs.
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/067837fa
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/067837fa
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/067837fa
Branch: refs/heads/master
Commit: 067837fa7d831d189174058718ffc94f5cea5822
Parents: bc4c60e
Author: kl0u <kk...@gmail.com>
Authored: Mon Feb 29 12:59:48 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
.../dataflow/FlinkPipelineExecutionEnvironment.java | 8 ++++++++
.../dataflow/translation/FlinkPipelineTranslator.java | 9 +++++++++
.../translation/FlinkStreamingPipelineTranslator.java | 13 +++++++++++--
.../FlinkStreamingTransformTranslators.java | 9 ++++-----
.../wrappers/streaming/FlinkAbstractParDoWrapper.java | 5 +++++
.../streaming/FlinkGroupAlsoByWindowWrapper.java | 6 +++---
.../wrappers/streaming/FlinkGroupByKeyWrapper.java | 4 ++++
.../streaming/FlinkParDoBoundMultiWrapper.java | 3 +++
.../wrappers/streaming/FlinkParDoBoundWrapper.java | 3 +++
.../wrappers/streaming/io/UnboundedFlinkSource.java | 4 ++++
.../wrappers/streaming/io/UnboundedSocketSource.java | 3 +++
.../wrappers/streaming/io/UnboundedSourceWrapper.java | 9 +++++++--
.../streaming/state/AbstractFlinkTimerInternals.java | 4 ++++
.../wrappers/streaming/state/FlinkStateInternals.java | 4 ++++
.../wrappers/streaming/state/StateType.java | 4 ++++
15 files changed, 76 insertions(+), 12 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067837fa/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java
index a1372bd..09ca184 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java
@@ -30,6 +30,14 @@ import org.slf4j.LoggerFactory;
import java.util.List;
+/**
+ * The class that instantiates and manages the execution of a given job.
+ * Depending on if the job is a Streaming or Batch processing one, it creates
+ * the adequate execution environment ({@link ExecutionEnvironment} or {@link StreamExecutionEnvironment}),
+ * the necessary {@link FlinkPipelineTranslator} ({@link FlinkBatchPipelineTranslator} or
+ * {@link FlinkStreamingPipelineTranslator})to transform the Beam job into a Flink one, and
+ * executes the (translated) job.
+ */
public class FlinkPipelineExecutionEnvironment {
private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class);
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067837fa/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java
index e5c8545..b56fe07 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java
@@ -17,6 +17,15 @@ package com.dataartisans.flink.dataflow.translation;
import com.google.cloud.dataflow.sdk.Pipeline;
+/**
+ * The role of this class is to translate the Beam operators to
+ * their Flink counterparts. If we have a streaming job, this is instantiated as a
+ * {@link FlinkStreamingPipelineTranslator}. In other case, i.e. for a batch job,
+ * a {@link FlinkBatchPipelineTranslator} is created. Correspondingly, the
+ * {@link com.google.cloud.dataflow.sdk.values.PCollection}-based user-provided job is translated into
+ * a {@link org.apache.flink.streaming.api.datastream.DataStream} (for streaming) or a
+ * {@link org.apache.flink.api.java.DataSet} (for batch) one.
+ */
public abstract class FlinkPipelineTranslator implements Pipeline.PipelineVisitor {
public void translate(Pipeline pipeline) {
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067837fa/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
index a8f4226..ea9ed14 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
@@ -22,6 +22,13 @@ import com.google.cloud.dataflow.sdk.transforms.PTransform;
import com.google.cloud.dataflow.sdk.values.PValue;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+/**
+ * This is a {@link FlinkPipelineTranslator} for streaming jobs. Its role is to translate the user-provided
+ * {@link com.google.cloud.dataflow.sdk.values.PCollection}-based job into a
+ * {@link org.apache.flink.streaming.api.datastream.DataStream} one.
+ *
+ * This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator}
+ * */
public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
/** The necessary context in the case of a straming job. */
@@ -107,14 +114,16 @@ public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
@SuppressWarnings("unchecked")
StreamTransformTranslator<T> typedTranslator = (StreamTransformTranslator<T>) translator;
- // create the applied PTransform on the batchContext
+ // create the applied PTransform on the streamingContext
streamingContext.setCurrentTransform(AppliedPTransform.of(
node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform));
typedTranslator.translateNode(typedTransform, streamingContext);
}
/**
- * A translator of a {@link PTransform}.
+ * The interface that every Flink translator of a Beam operator should implement.
+ * This interface is for <b>streaming</b> jobs. For examples of such translators see
+ * {@link FlinkStreamingTransformTranslators}.
*/
public interface StreamTransformTranslator<Type extends PTransform> {
void translateNode(Type transform, FlinkStreamingTranslationContext context);
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067837fa/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
index 27cc923..1be51ae 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
@@ -52,11 +52,10 @@ import java.io.IOException;
import java.util.*;
/**
- * <p>
- * Coder<?> entryCoder = pCollection.getCoder();
- * if (!(entryCoder instanceof KvCoder<?, ?>)) {
- * throw new IllegalArgumentException("PCollection does not use a KvCoder");
- * }
+ * This class contains all the mappings between Beam and Flink
+ * <b>streaming</b> transformations. The {@link FlinkStreamingPipelineTranslator}
+ * traverses the Beam job and comes here to translate the encountered Beam transformations
+ * into Flink one, based on the mapping available in this class.
*/
public class FlinkStreamingTransformTranslators {
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067837fa/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
index dfb2b7d..3605d3f 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
@@ -37,6 +37,11 @@ import org.joda.time.format.PeriodFormat;
import java.util.Collection;
+/**
+ * An abstract class that encapsulates the common code of the the {@link com.google.cloud.dataflow.sdk.transforms.ParDo.Bound}
+ * and {@link com.google.cloud.dataflow.sdk.transforms.ParDo.BoundMulti} wrappers. See the {@link FlinkParDoBoundWrapper} and
+ * {@link FlinkParDoBoundMultiWrapper} for the actual wrappers of the aforementioned transformations.
+ * */
public abstract class FlinkAbstractParDoWrapper<IN, OUTDF, OUTFL> extends RichFlatMapFunction<WindowedValue<IN>, WindowedValue<OUTFL>> {
private final DoFn<IN, OUTDF> doFn;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067837fa/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
index b78db65..75694cc 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
@@ -46,9 +46,9 @@ import java.io.IOException;
import java.util.*;
/**
- * This class is the key class implementing all the windowing/triggering logic of Google Dataflow.
- * To provide full compatibility and support all the windowing/triggering combinations offered by
- * Dataflow, we opted for a strategy that uses the SDK's code for doing these operations
+ * This class is the key class implementing all the windowing/triggering logic of Apache Beam.
+ * To provide full compatibility and support for all the windowing/triggering combinations offered by
+ * Beam, we opted for a strategy that uses the SDK's code for doing these operations. See the code in
* ({@link com.google.cloud.dataflow.sdk.util.StreamingGroupAlsoByWindowsDoFn}.
* <p>
* In a nutshell, when the execution arrives to this operator, we expect to have a stream <b>already
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067837fa/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
index 0a0e301..b0d9e48 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
@@ -26,6 +26,10 @@ import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.KeyedStream;
+/**
+ * This class groups the elements by key. It assumes that already the incoming stream
+ * is composed of <code>[Key,Value]</code> pairs.
+ * */
public class FlinkGroupByKeyWrapper {
/**
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067837fa/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
index 200c397..52ab19e 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
@@ -28,6 +28,9 @@ import org.joda.time.Instant;
import java.util.Map;
+/**
+ * A wrapper for the {@link com.google.cloud.dataflow.sdk.transforms.ParDo.BoundMulti} Beam transformation.
+ * */
public class FlinkParDoBoundMultiWrapper<IN, OUT> extends FlinkAbstractParDoWrapper<IN, OUT, RawUnionValue> {
private final TupleTag<?> mainTag;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067837fa/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
index 18d4249..4a5c854 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
@@ -29,6 +29,9 @@ import org.joda.time.Instant;
import java.io.IOException;
import java.util.*;
+/**
+ * A wrapper for the {@link com.google.cloud.dataflow.sdk.transforms.ParDo.Bound} Beam transformation.
+ * */
public class FlinkParDoBoundWrapper<IN, OUT> extends FlinkAbstractParDoWrapper<IN, OUT, OUT> {
public FlinkParDoBoundWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUT> doFn) {
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067837fa/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java
index 17e0746..7c8cd0b 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java
@@ -25,6 +25,10 @@ import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunctio
import javax.annotation.Nullable;
import java.util.List;
+/**
+ * A wrapper translating Flink Sources implementing the {@link RichParallelSourceFunction} interface, into
+ * unbounded Beam sources (see {@link UnboundedSource}).
+ * */
public class UnboundedFlinkSource<T, C extends UnboundedSource.CheckpointMark> extends UnboundedSource<T, C> {
private final PipelineOptions options;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067837fa/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java
index 2b0d6dc..dd14f68 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java
@@ -36,6 +36,9 @@ import java.util.NoSuchElementException;
import static com.google.common.base.Preconditions.checkArgument;
+/**
+ * An example unbounded Beam source that reads input from a socket. This is used mainly for testing and debugging.
+ * */
public class UnboundedSocketSource<C extends UnboundedSource.CheckpointMark> extends UnboundedSource<String, C> {
private static final Coder<String> DEFAULT_SOCKET_CODER = StringUtf8Coder.of();
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067837fa/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
index cdc2e95..c534079 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
@@ -30,8 +30,13 @@ import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.operators.Triggerable;
import org.joda.time.Instant;
-import java.util.Collection;
-
+/**
+ * A wrapper for Beam's unbounded sources. This class wraps around a source implementing the {@link com.google.cloud.dataflow.sdk.io.Read.Unbounded}
+ * interface.
+ *
+ *</p>
+ * For now we support non-parallel, not checkpointed sources.
+ * */
public class UnboundedSourceWrapper<T> extends RichSourceFunction<WindowedValue<T>> implements EventTimeSourceFunction<WindowedValue<T>>, Triggerable {
private final String name;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067837fa/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
index 4401eb3..7b68e9f 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
@@ -29,6 +29,10 @@ import org.joda.time.Instant;
import java.io.IOException;
import java.io.Serializable;
+/**
+ * An implementation of Beam's {@link TimerInternals}, that also provides serialization functionality.
+ * The latter is used when snapshots of the current state are taken, for fault-tolerance.
+ * */
public abstract class AbstractFlinkTimerInternals<K, VIN> implements TimerInternals, Serializable {
private TimerOrElement<WindowedValue<KV<K, VIN>>> element;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067837fa/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java
index 03b8bb5..f4ec6d5 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java
@@ -28,6 +28,10 @@ import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.util.*;
+/**
+ * An implementation of the Beam {@link MergingStateInternals}. This implementation simply keeps elements in memory.
+ * This state is periodically checkpointed by Flink, for fault-tolerance.
+ * */
public class FlinkStateInternals<K> extends MergingStateInternals {
private final K key;
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/067837fa/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java
index 11446ea..aa049ef 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java
@@ -17,6 +17,10 @@ package com.dataartisans.flink.dataflow.translation.wrappers.streaming.state;
import java.io.IOException;
+/**
+ * The available types of state, as provided by the Beam SDK. This class is used for serialization/deserialization
+ * purposes.
+ * */
public enum StateType {
VALUE(0),
[50/50] [abbrv] incubator-beam git commit: This closes #12
Posted by da...@apache.org.
This closes #12
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/4da935b6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/4da935b6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/4da935b6
Branch: refs/heads/master
Commit: 4da935b65b4220fd410255766dd00c851bfddeef
Parents: 0528570 4d4425c
Author: Davor Bonaci <da...@users.noreply.github.com>
Authored: Fri Mar 4 10:08:22 2016 -0800
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:08:22 2016 -0800
----------------------------------------------------------------------
.travis.yml | 2 +-
pom.xml | 1 +
runners/flink/README.md | 202 ++++++
runners/flink/pom.xml | 264 +++++++
.../FlinkPipelineExecutionEnvironment.java | 269 +++++++
.../runners/flink/FlinkPipelineOptions.java | 93 +++
.../beam/runners/flink/FlinkPipelineRunner.java | 206 ++++++
.../beam/runners/flink/FlinkRunnerResult.java | 68 ++
.../beam/runners/flink/examples/TFIDF.java | 452 ++++++++++++
.../beam/runners/flink/examples/WordCount.java | 113 +++
.../flink/examples/streaming/AutoComplete.java | 387 ++++++++++
.../flink/examples/streaming/JoinExamples.java | 158 ++++
.../KafkaWindowedWordCountExample.java | 143 ++++
.../examples/streaming/WindowedWordCount.java | 130 ++++
.../apache/beam/runners/flink/io/ConsoleIO.java | 82 +++
.../FlinkBatchPipelineTranslator.java | 149 ++++
.../FlinkBatchTransformTranslators.java | 594 +++++++++++++++
.../FlinkBatchTranslationContext.java | 129 ++++
.../translation/FlinkPipelineTranslator.java | 36 +
.../FlinkStreamingPipelineTranslator.java | 145 ++++
.../FlinkStreamingTransformTranslators.java | 406 +++++++++++
.../FlinkStreamingTranslationContext.java | 89 +++
.../FlinkCoGroupKeyedListAggregator.java | 60 ++
.../functions/FlinkCreateFunction.java | 62 ++
.../functions/FlinkDoFnFunction.java | 204 ++++++
.../FlinkKeyedListAggregationFunction.java | 77 ++
.../functions/FlinkMultiOutputDoFnFunction.java | 177 +++++
.../FlinkMultiOutputPruningFunction.java | 43 ++
.../functions/FlinkPartialReduceFunction.java | 60 ++
.../functions/FlinkReduceFunction.java | 57 ++
.../flink/translation/functions/UnionCoder.java | 150 ++++
.../translation/types/CoderComparator.java | 216 ++++++
.../translation/types/CoderTypeInformation.java | 116 +++
.../translation/types/CoderTypeSerializer.java | 152 ++++
.../types/InspectableByteArrayOutputStream.java | 34 +
.../translation/types/KvCoderComperator.java | 264 +++++++
.../types/KvCoderTypeInformation.java | 186 +++++
.../types/VoidCoderTypeSerializer.java | 112 +++
.../wrappers/CombineFnAggregatorWrapper.java | 92 +++
.../wrappers/DataInputViewWrapper.java | 59 ++
.../wrappers/DataOutputViewWrapper.java | 52 ++
.../SerializableFnAggregatorWrapper.java | 91 +++
.../translation/wrappers/SinkOutputFormat.java | 121 ++++
.../translation/wrappers/SourceInputFormat.java | 164 +++++
.../translation/wrappers/SourceInputSplit.java | 52 ++
.../streaming/FlinkAbstractParDoWrapper.java | 266 +++++++
.../FlinkGroupAlsoByWindowWrapper.java | 631 ++++++++++++++++
.../streaming/FlinkGroupByKeyWrapper.java | 66 ++
.../streaming/FlinkParDoBoundMultiWrapper.java | 77 ++
.../streaming/FlinkParDoBoundWrapper.java | 100 +++
.../io/FlinkStreamingCreateFunction.java | 65 ++
.../streaming/io/UnboundedFlinkSource.java | 82 +++
.../streaming/io/UnboundedSocketSource.java | 233 ++++++
.../streaming/io/UnboundedSourceWrapper.java | 134 ++++
.../state/AbstractFlinkTimerInternals.java | 128 ++++
.../streaming/state/FlinkStateInternals.java | 715 +++++++++++++++++++
.../streaming/state/StateCheckpointReader.java | 91 +++
.../streaming/state/StateCheckpointUtils.java | 155 ++++
.../streaming/state/StateCheckpointWriter.java | 129 ++++
.../wrappers/streaming/state/StateType.java | 73 ++
.../flink/src/main/resources/log4j.properties | 23 +
.../apache/beam/runners/flink/AvroITCase.java | 101 +++
.../beam/runners/flink/FlattenizeITCase.java | 74 ++
.../beam/runners/flink/FlinkTestPipeline.java | 72 ++
.../beam/runners/flink/JoinExamplesITCase.java | 101 +++
.../runners/flink/MaybeEmptyTestITCase.java | 65 ++
.../runners/flink/ParDoMultiOutputITCase.java | 100 +++
.../beam/runners/flink/ReadSourceITCase.java | 165 +++++
.../flink/RemoveDuplicatesEmptyITCase.java | 70 ++
.../runners/flink/RemoveDuplicatesITCase.java | 71 ++
.../beam/runners/flink/SideInputITCase.java | 69 ++
.../apache/beam/runners/flink/TfIdfITCase.java | 78 ++
.../beam/runners/flink/WordCountITCase.java | 76 ++
.../runners/flink/WordCountJoin2ITCase.java | 138 ++++
.../runners/flink/WordCountJoin3ITCase.java | 156 ++++
.../beam/runners/flink/WriteSinkITCase.java | 158 ++++
.../flink/streaming/GroupAlsoByWindowTest.java | 508 +++++++++++++
.../flink/streaming/GroupByNullKeyTest.java | 123 ++++
.../flink/streaming/StateSerializationTest.java | 305 ++++++++
.../streaming/TopWikipediaSessionsITCase.java | 134 ++++
.../beam/runners/flink/util/JoinExamples.java | 160 +++++
.../src/test/resources/log4j-test.properties | 27 +
runners/pom.xml | 43 ++
83 files changed, 12480 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
[38/50] [abbrv] incubator-beam git commit: [flink] update README
Posted by da...@apache.org.
[flink] update README
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/ceb2c87f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/ceb2c87f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/ceb2c87f
Branch: refs/heads/master
Commit: ceb2c87f8f749cb4db0582b9f1abc15c4da752fd
Parents: 28fcfd7
Author: Maximilian Michels <mx...@apache.org>
Authored: Wed Mar 2 23:51:38 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
runners/flink/README.md | 60 ++++++++++++++++++++++----------------------
1 file changed, 30 insertions(+), 30 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/ceb2c87f/runners/flink/README.md
----------------------------------------------------------------------
diff --git a/runners/flink/README.md b/runners/flink/README.md
index 499ed6d..0fee6f0 100644
--- a/runners/flink/README.md
+++ b/runners/flink/README.md
@@ -1,17 +1,17 @@
-Flink-Dataflow
---------------
+Flink Beam Runner (Flink-Runner)
+-------------------------------
-Flink-Dataflow is a Runner for Google Dataflow (aka Apache Beam) which enables you to
-run Dataflow programs with Flink. It integrates seamlessly with the Dataflow
-API, allowing you to execute Dataflow programs in streaming or batch mode.
+Flink-Runner is a Runner for Apache Beam which enables you to
+run Beam dataflows with Flink. It integrates seamlessly with the Beam
+API, allowing you to execute Apache Beam programs in streaming or batch mode.
## Streaming
-### Full Dataflow Windowing and Triggering Semantics
+### Full Beam Windowing and Triggering Semantics
-The Flink Dataflow Runner supports *Event Time* allowing you to analyze data with respect to its
+The Flink Beam Runner supports *Event Time* allowing you to analyze data with respect to its
associated timestamp. It handles out-or-order and late-arriving elements. You may leverage the full
-power of the Dataflow windowing semantics like *time-based*, *sliding*, *tumbling*, or *count*
+power of the Beam windowing semantics like *time-based*, *sliding*, *tumbling*, or *count*
windows. You may build *session* windows which allow you to keep track of events associated with
each other.
@@ -27,7 +27,7 @@ and sinks or use the provided support for Apache Kafka.
### Seamless integration
-To execute a Dataflow program in streaming mode, just enable streaming in the `PipelineOptions`:
+To execute a Beam program in streaming mode, just enable streaming in the `PipelineOptions`:
options.setStreaming(true);
@@ -52,7 +52,7 @@ and sinks.
## Features
-The Flink Dataflow Runner maintains as much compatibility with the Dataflow API as possible. We
+The Flink Beam Runner maintains as much compatibility with the Beam API as possible. We
support transformations on data like:
- Grouping
@@ -66,25 +66,25 @@ support transformations on data like:
# Getting Started
-To get started using Flink-Dataflow, we first need to install the latest version.
+To get started using the Flink Runner, we first need to install the latest version.
-## Install Flink-Dataflow ##
+## Install Flink-Runner ##
-To retrieve the latest version of Flink-Dataflow, run the following command
+To retrieve the latest version of Flink-Runner, run the following command
- git clone https://github.com/dataArtisans/flink-dataflow
+ git clone https://github.com/apache/incubator-beam
-Then switch to the newly created directory and run Maven to build the Dataflow runner:
+Then switch to the newly created directory and run Maven to build the Beam runner:
- cd flink-dataflow
+ cd incubator-beam
mvn clean install -DskipTests
-Flink-Dataflow is now installed in your local maven repository.
+Flink-Runner is now installed in your local maven repository.
## Executing an example
Next, let's run the classic WordCount example. It's semantically identically to
-the example provided with Google Dataflow. Only this time, we chose the
+the example provided with ApacheBeam. Only this time, we chose the
`FlinkPipelineRunner` to execute the WordCount on top of Flink.
Here's an excerpt from the WordCount class file:
@@ -113,15 +113,15 @@ Then let's run the included WordCount locally on your machine:
mvn exec:exec -Dinput=kinglear.txt -Doutput=wordcounts.txt
-Congratulations, you have run your first Google Dataflow program on top of Apache Flink!
+Congratulations, you have run your first ApacheBeam program on top of Apache Flink!
-# Running Dataflow programs on a Flink cluster
+# Running Beam programs on a Flink cluster
-You can run your Dataflow program on an Apache Flink cluster. Please start off by creating a new
+You can run your Beam program on an Apache Flink cluster. Please start off by creating a new
Maven project.
- mvn archetype:generate -DgroupId=com.mycompany.dataflow -DartifactId=dataflow-test \
+ mvn archetype:generate -DgroupId=com.mycompany.beam -DartifactId=beam-test \
-DarchetypeArtifactId=maven-archetype-quickstart -DinteractiveMode=false
The contents of the root `pom.xml` should be slightly changed aftewards (explanation below):
@@ -133,14 +133,14 @@ The contents of the root `pom.xml` should be slightly changed aftewards (explana
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
- <groupId>com.mycompany.dataflow</groupId>
- <artifactId>dataflow-test</artifactId>
+ <groupId>com.mycompany.beam</groupId>
+ <artifactId>beam-test</artifactId>
<version>1.0</version>
<dependencies>
<dependency>
- <groupId>com.dataartisans</groupId>
- <artifactId>flink-dataflow</artifactId>
+ <groupId>org.apache.beam</groupId>
+ <artifactId>flink-runner</artifactId>
<version>0.2</version>
</dependency>
</dependencies>
@@ -182,13 +182,13 @@ The contents of the root `pom.xml` should be slightly changed aftewards (explana
The following changes have been made:
-1. The Flink Dataflow Runner was added as a dependency.
+1. The Flink Beam Runner was added as a dependency.
2. The Maven Shade plugin was added to build a fat jar.
-A fat jar is necessary if you want to submit your Dataflow code to a Flink cluster. The fat jar
-includes your program code but also Dataflow code which is necessary during runtime. Note that this
-step is necessary because the Dataflow Runner is not part of Flink.
+A fat jar is necessary if you want to submit your Beam code to a Flink cluster. The fat jar
+includes your program code but also Beam code which is necessary during runtime. Note that this
+step is necessary because the Beam Runner is not part of Flink.
You can then build the jar using `mvn clean package`. Please submit the fat jar in the `target`
folder to the Flink cluster using the command-line utility like so:
[04/50] [abbrv] incubator-beam git commit: [runner] add streaming
support with checkpointing
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
new file mode 100644
index 0000000..b667187
--- /dev/null
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
@@ -0,0 +1,507 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.streaming;
+
+import com.dataartisans.flink.dataflow.FlinkTestPipeline;
+import com.dataartisans.flink.dataflow.translation.wrappers.streaming.FlinkGroupAlsoByWindowWrapper;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.Sum;
+import com.google.cloud.dataflow.sdk.transforms.windowing.*;
+import com.google.cloud.dataflow.sdk.util.UserCodeException;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.common.base.Throwables;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
+import org.apache.flink.streaming.util.TestHarnessUtil;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+public class GroupAlsoByWindowTest {
+
+ private final Combine.CombineFn combiner = new Sum.SumIntegerFn();
+
+ private final WindowingStrategy slidingWindowWithAfterWatermarkTriggerStrategy =
+ WindowingStrategy.of(SlidingWindows.of(Duration.standardSeconds(10)).every(Duration.standardSeconds(5)))
+ .withTrigger(AfterWatermark.pastEndOfWindow()).withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES);
+
+ private final WindowingStrategy sessionWindowingStrategy =
+ WindowingStrategy.of(Sessions.withGapDuration(Duration.standardSeconds(2)))
+ .withTrigger(Repeatedly.forever(AfterWatermark.pastEndOfWindow()))
+ .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES)
+ .withAllowedLateness(Duration.standardSeconds(100));
+
+ private final WindowingStrategy fixedWindowingStrategy =
+ WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(10)));
+
+ private final WindowingStrategy fixedWindowWithCountTriggerStrategy =
+ fixedWindowingStrategy.withTrigger(AfterPane.elementCountAtLeast(5));
+
+ private final WindowingStrategy fixedWindowWithAfterWatermarkTriggerStrategy =
+ fixedWindowingStrategy.withTrigger(AfterWatermark.pastEndOfWindow());
+
+ private final WindowingStrategy fixedWindowWithCompoundTriggerStrategy =
+ fixedWindowingStrategy.withTrigger(
+ Repeatedly.forever(AfterFirst.of(
+ AfterPane.elementCountAtLeast(5),
+ AfterWatermark.pastEndOfWindow())));
+
+ /**
+ * The default accumulation mode is
+ * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#DISCARDING_FIRED_PANES}.
+ * This strategy changes it to
+ * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#ACCUMULATING_FIRED_PANES}
+ */
+ private final WindowingStrategy fixedWindowWithCompoundTriggerStrategyAcc =
+ fixedWindowWithCompoundTriggerStrategy
+ .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES);
+
+ @Test
+ public void testWithLateness() throws Exception {
+ WindowingStrategy strategy = WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(2)))
+ .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES)
+ .withAllowedLateness(Duration.millis(1000));
+ long initialTime = 0L;
+ Pipeline pipeline = FlinkTestPipeline.create();
+
+ KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
+
+ FlinkGroupAlsoByWindowWrapper gbwOperaror =
+ FlinkGroupAlsoByWindowWrapper.createForTesting(
+ pipeline.getOptions(),
+ pipeline.getCoderRegistry(),
+ strategy,
+ inputCoder,
+ combiner.<String>asKeyedFn());
+
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ new OneInputStreamOperatorTestHarness<>(gbwOperaror);
+ testHarness.open();
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processWatermark(new Watermark(initialTime + 2000));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processWatermark(new Watermark(initialTime + 4000));
+
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 4),
+ new Instant(initialTime + 1),
+ new IntervalWindow(new Instant(0), new Instant(2000)),
+ PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
+ , initialTime));
+ expectedOutput.add(new Watermark(initialTime + 2000));
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 6),
+ new Instant(initialTime + 1999),
+ new IntervalWindow(new Instant(0), new Instant(2000)),
+ PaneInfo.createPane(false, true, PaneInfo.Timing.LATE, 1, 1))
+ , initialTime));
+
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 0),
+ new Instant(initialTime + 1999),
+ new IntervalWindow(new Instant(0), new Instant(2000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME, 0, 0))
+ , initialTime));
+ expectedOutput.add(new Watermark(initialTime + 4000));
+
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+ testHarness.close();
+ }
+
+ @Test
+ public void testSessionWindows() throws Exception {
+ WindowingStrategy strategy = sessionWindowingStrategy;
+
+ long initialTime = 0L;
+ Pipeline pipeline = FlinkTestPipeline.create();
+
+ KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
+
+ FlinkGroupAlsoByWindowWrapper gbwOperaror =
+ FlinkGroupAlsoByWindowWrapper.createForTesting(
+ pipeline.getOptions(),
+ pipeline.getCoderRegistry(),
+ strategy,
+ inputCoder,
+ combiner.<String>asKeyedFn());
+
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ new OneInputStreamOperatorTestHarness<>(gbwOperaror);
+ testHarness.open();
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 3500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 3700), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 2700), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processWatermark(new Watermark(initialTime + 6000));
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 6700), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 6800), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 8900), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 7600), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 5600), null, PaneInfo.NO_FIRING), initialTime + 20));
+
+ testHarness.processWatermark(new Watermark(initialTime + 12000));
+
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 6),
+ new Instant(initialTime + 1),
+ new IntervalWindow(new Instant(1), new Instant(5700)),
+ PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
+ , initialTime));
+ expectedOutput.add(new Watermark(initialTime + 6000));
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 11),
+ new Instant(initialTime + 6700),
+ new IntervalWindow(new Instant(1), new Instant(10900)),
+ PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
+ , initialTime));
+ expectedOutput.add(new Watermark(initialTime + 12000));
+
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+ testHarness.close();
+ }
+
+ @Test
+ public void testSlidingWindows() throws Exception {
+ WindowingStrategy strategy = slidingWindowWithAfterWatermarkTriggerStrategy;
+ long initialTime = 0L;
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ createTestingOperatorAndState(strategy, initialTime);
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+ testHarness.processWatermark(new Watermark(initialTime + 25000));
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 6),
+ new Instant(initialTime + 5000),
+ new IntervalWindow(new Instant(0), new Instant(10000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime));
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 6),
+ new Instant(initialTime + 1),
+ new IntervalWindow(new Instant(-5000), new Instant(5000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime));
+ expectedOutput.add(new Watermark(initialTime + 10000));
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 11),
+ new Instant(initialTime + 15000),
+ new IntervalWindow(new Instant(10000), new Instant(20000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime));
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 3),
+ new Instant(initialTime + 10000),
+ new IntervalWindow(new Instant(5000), new Instant(15000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime));
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key2", 1),
+ new Instant(initialTime + 19500),
+ new IntervalWindow(new Instant(10000), new Instant(20000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime));
+ expectedOutput.add(new Watermark(initialTime + 20000));
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key2", 1),
+ new Instant(initialTime + 20000),
+ /**
+ * this is 20000 and not 19500 because of a convention in dataflow where
+ * timestamps of windowed values in a window cannot be smaller than the
+ * end of a previous window. Checkout the documentation of the
+ * {@link WindowFn#getOutputTime(Instant, BoundedWindow)}
+ */
+ new IntervalWindow(new Instant(15000), new Instant(25000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime));
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 8),
+ new Instant(initialTime + 20000),
+ new IntervalWindow(new Instant(15000), new Instant(25000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime));
+ expectedOutput.add(new Watermark(initialTime + 25000));
+
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+ testHarness.close();
+ }
+
+ @Test
+ public void testAfterWatermarkProgram() throws Exception {
+ WindowingStrategy strategy = fixedWindowWithAfterWatermarkTriggerStrategy;
+ long initialTime = 0L;
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ createTestingOperatorAndState(strategy, initialTime);
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6),
+ new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime));
+ expectedOutput.add(new Watermark(initialTime + 10000));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11),
+ new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime));
+ expectedOutput.add(new Watermark(initialTime + 20000));
+
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+ testHarness.close();
+ }
+
+ @Test
+ public void testAfterCountProgram() throws Exception {
+ WindowingStrategy strategy = fixedWindowWithCountTriggerStrategy;
+
+ long initialTime = 0L;
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ createTestingOperatorAndState(strategy, initialTime);
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime));
+ expectedOutput.add(new Watermark(initialTime + 10000));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.LATE, 0, 0)), initialTime));
+ expectedOutput.add(new Watermark(initialTime + 20000));
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+ testHarness.close();
+ }
+
+ @Test
+ public void testCompoundProgram() throws Exception {
+ WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategy;
+
+ long initialTime = 0L;
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ createTestingOperatorAndState(strategy, initialTime);
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+ /**
+ * PaneInfo are:
+ * isFirst (pane in window),
+ * isLast, Timing (of triggering),
+ * index (of pane in the window),
+ * onTimeIndex (if it the 1st,2nd, ... pane that was fired on time)
+ * */
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1),
+ new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime));
+
+ expectedOutput.add(new Watermark(initialTime + 10000));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime));
+
+ expectedOutput.add(new Watermark(initialTime + 20000));
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+ testHarness.close();
+ }
+
+ @Test
+ public void testCompoundAccumulatingPanesProgram() throws Exception {
+ WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategyAcc;
+ long initialTime = 0L;
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ createTestingOperatorAndState(strategy, initialTime);
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 10),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6),
+ new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime));
+
+ expectedOutput.add(new Watermark(initialTime + 10000));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime));
+
+ expectedOutput.add(new Watermark(initialTime + 20000));
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+ testHarness.close();
+ }
+
+ private OneInputStreamOperatorTestHarness createTestingOperatorAndState(WindowingStrategy strategy, long initialTime) throws Exception {
+ Pipeline pipeline = FlinkTestPipeline.create();
+
+ KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
+
+ FlinkGroupAlsoByWindowWrapper gbwOperaror =
+ FlinkGroupAlsoByWindowWrapper.createForTesting(
+ pipeline.getOptions(),
+ pipeline.getCoderRegistry(),
+ strategy,
+ inputCoder,
+ combiner.<String>asKeyedFn());
+
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ new OneInputStreamOperatorTestHarness<>(gbwOperaror);
+ testHarness.open();
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 10000), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 12100), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 14200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 15300), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 16500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+
+ testHarness.processWatermark(new Watermark(initialTime + 10000));
+ testHarness.processWatermark(new Watermark(initialTime + 20000));
+
+ return testHarness;
+ }
+
+ private static class ResultSortComparator implements Comparator<Object> {
+ @Override
+ public int compare(Object o1, Object o2) {
+ if (o1 instanceof Watermark && o2 instanceof Watermark) {
+ Watermark w1 = (Watermark) o1;
+ Watermark w2 = (Watermark) o2;
+ return (int) (w1.getTimestamp() - w2.getTimestamp());
+ } else {
+ StreamRecord<WindowedValue<KV<String, Integer>>> sr0 = (StreamRecord<WindowedValue<KV<String, Integer>>>) o1;
+ StreamRecord<WindowedValue<KV<String, Integer>>> sr1 = (StreamRecord<WindowedValue<KV<String, Integer>>>) o2;
+
+ if (sr0.getTimestamp() != sr1.getTimestamp()) {
+ return (int) (sr0.getTimestamp() - sr1.getTimestamp());
+ }
+
+ int comparison = sr0.getValue().getValue().getKey().compareTo(sr1.getValue().getValue().getKey());
+ if(comparison == 0) {
+ comparison = Integer.compare(
+ sr0.getValue().getValue().getValue(),
+ sr1.getValue().getValue().getValue());
+ }
+ if(comparison == 0) {
+ Collection windowsA = sr0.getValue().getWindows();
+ Collection windowsB = sr1.getValue().getWindows();
+
+ if(windowsA.size() != 1 || windowsB.size() != 1) {
+ throw new IllegalStateException("A value cannot belong to more than one windows after grouping.");
+ }
+
+ BoundedWindow windowA = (BoundedWindow) windowsA.iterator().next();
+ BoundedWindow windowB = (BoundedWindow) windowsB.iterator().next();
+ comparison = Long.compare(windowA.maxTimestamp().getMillis(), windowB.maxTimestamp().getMillis());
+ }
+ return comparison;
+ }
+ }
+ }
+
+ private <T> WindowedValue<T> makeWindowedValue(WindowingStrategy strategy,
+ T output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+ final Instant inputTimestamp = timestamp;
+ final WindowFn windowFn = strategy.getWindowFn();
+
+ if (timestamp == null) {
+ timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+ }
+
+ if (windows == null) {
+ try {
+ windows = windowFn.assignWindows(windowFn.new AssignContext() {
+ @Override
+ public Object element() {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input element when none was available"); // TODO: 12/16/15 aljoscha's comment in slack
+ }
+
+ @Override
+ public Instant timestamp() {
+ if (inputTimestamp == null) {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input timestamp when none was available");
+ }
+ return inputTimestamp;
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input windows when none were available");
+ }
+ });
+ } catch (Exception e) {
+ Throwables.propagateIfInstanceOf(e, UserCodeException.class);
+ throw new UserCodeException(e);
+ }
+ }
+
+ return WindowedValue.of(output, timestamp, windows, pane);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java
new file mode 100644
index 0000000..084ada2
--- /dev/null
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java
@@ -0,0 +1,257 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.streaming;
+
+import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.FlinkStateInternals;
+import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.StateCheckpointReader;
+import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.StateCheckpointUtils;
+import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.StateCheckpointWriter;
+import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.coders.VarIntCoder;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.Sum;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
+import com.google.cloud.dataflow.sdk.util.TimeDomain;
+import com.google.cloud.dataflow.sdk.util.TimerInternals;
+import com.google.cloud.dataflow.sdk.util.state.*;
+import org.apache.flink.api.java.typeutils.runtime.ByteArrayInputView;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.joda.time.Instant;
+import org.junit.Test;
+
+import java.util.*;
+
+import static org.junit.Assert.assertEquals;
+
+public class StateSerializationTest {
+
+ private static final StateNamespace NAMESPACE_1 = StateNamespaces.global();
+ private static final String KEY_PREFIX = "TEST_";
+
+ private static final StateTag<ValueState<String>> STRING_VALUE_ADDR =
+ StateTags.value("stringValue", StringUtf8Coder.of());
+ private static final StateTag<ValueState<Integer>> INT_VALUE_ADDR =
+ StateTags.value("stringValue", VarIntCoder.of());
+ private static final StateTag<CombiningValueState<Integer, Integer>> SUM_INTEGER_ADDR =
+ StateTags.combiningValueFromInputInternal(
+ "sumInteger", VarIntCoder.of(), new Sum.SumIntegerFn());
+ private static final StateTag<BagState<String>> STRING_BAG_ADDR =
+ StateTags.bag("stringBag", StringUtf8Coder.of());
+ private static final StateTag<WatermarkStateInternal> WATERMARK_BAG_ADDR =
+ StateTags.watermarkStateInternal("watermark");
+
+ private Combine.CombineFn combiner = new Sum.SumIntegerFn();
+
+ private Map<String, FlinkStateInternals<String>> statePerKey = new HashMap<>();
+
+ private Map<String, Set<TimerInternals.TimerData>> activeTimers = new HashMap<>();
+
+ private void initializeStateAndTimers() throws CannotProvideCoderException {
+ for (int i = 0; i < 10; i++) {
+ String key = KEY_PREFIX + i;
+
+ FlinkStateInternals state = initializeStateForKey(key);
+ Set<TimerInternals.TimerData> timers = new HashSet<>();
+ for (int j = 0; j < 5; j++) {
+ TimerInternals.TimerData timer = TimerInternals
+ .TimerData.of(NAMESPACE_1,
+ new Instant(1000 + i + j), TimeDomain.values()[j % 3]);
+ timers.add(timer);
+ }
+
+ statePerKey.put(key, state);
+ activeTimers.put(key, timers);
+ }
+ }
+
+ private FlinkStateInternals<String> initializeStateForKey(String key) throws CannotProvideCoderException {
+ FlinkStateInternals<String> state = createState(key);
+
+ ValueState<String> value = state.state(NAMESPACE_1, STRING_VALUE_ADDR);
+ value.set("test");
+
+ ValueState<Integer> value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR);
+ value2.set(4);
+ value2.set(5);
+
+ CombiningValueState<Integer, Integer> combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+ combiningValue.add(1);
+ combiningValue.add(2);
+
+ WatermarkStateInternal watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
+ watermark.add(new Instant(1000));
+
+ BagState<String> bag = state.state(NAMESPACE_1, STRING_BAG_ADDR);
+ bag.add("v1");
+ bag.add("v2");
+ bag.add("v3");
+ bag.add("v4");
+ return state;
+ }
+
+ private boolean restoreAndTestState(DataInputView in) throws Exception {
+ StateCheckpointReader reader = new StateCheckpointReader(in);
+ final ClassLoader userClassloader = this.getClass().getClassLoader();
+ Coder<? extends BoundedWindow> windowCoder = IntervalWindow.getCoder();
+ Coder<String> keyCoder = StringUtf8Coder.of();
+
+ boolean comparisonRes = true;
+
+ for(String key: statePerKey.keySet()) {
+ comparisonRes &= checkStateForKey(key);
+ }
+
+ // restore the timers
+ Map<String, Set<TimerInternals.TimerData>> restoredTimersPerKey = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder);
+ if(activeTimers.size() != restoredTimersPerKey.size()) {
+ return false;
+ }
+
+ for(String key: statePerKey.keySet()) {
+ Set<TimerInternals.TimerData> originalTimers = activeTimers.get(key);
+ Set<TimerInternals.TimerData> restoredTimers = restoredTimersPerKey.get(key);
+ comparisonRes &= checkTimersForKey(originalTimers, restoredTimers);
+ }
+
+ // restore the state
+ Map<String, FlinkStateInternals<String>> restoredPerKeyState = StateCheckpointUtils.decodeState(reader, combiner.asKeyedFn(), keyCoder, windowCoder, userClassloader);
+ if(restoredPerKeyState.size() != statePerKey.size()) {
+ return false;
+ }
+
+ for(String key: statePerKey.keySet()) {
+ FlinkStateInternals<String> originalState = statePerKey.get(key);
+ FlinkStateInternals<String> restoredState = restoredPerKeyState.get(key);
+ comparisonRes &= checkStateForKey(originalState, restoredState);
+ }
+ return comparisonRes;
+ }
+
+ private boolean checkStateForKey(String key) throws CannotProvideCoderException {
+ FlinkStateInternals<String> state = statePerKey.get(key);
+
+ ValueState<String> value = state.state(NAMESPACE_1, STRING_VALUE_ADDR);
+ boolean comp = value.get().read().equals("test");
+
+ ValueState<Integer> value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR);
+ comp &= value2.get().read().equals(5);
+
+ CombiningValueState<Integer, Integer> combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+ comp &= combiningValue.get().read().equals(3);
+
+ WatermarkStateInternal watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
+ comp &= watermark.get().read().equals(new Instant(1000));
+
+ BagState<String> bag = state.state(NAMESPACE_1, STRING_BAG_ADDR);
+ Iterator<String> it = bag.get().read().iterator();
+ int i = 0;
+ while(it.hasNext()) {
+ comp &= it.next().equals("v"+ (++i));
+ }
+ return comp;
+ }
+
+ private void storeState(StateBackend.CheckpointStateOutputView out) throws Exception {
+ StateCheckpointWriter checkpointBuilder = StateCheckpointWriter.create(out);
+ Coder<String> keyCoder = StringUtf8Coder.of();
+
+ // checkpoint the timers
+ StateCheckpointUtils.encodeTimers(activeTimers, checkpointBuilder,keyCoder);
+
+ // checkpoint the state
+ StateCheckpointUtils.encodeState(statePerKey, checkpointBuilder, keyCoder);
+ }
+
+ private boolean checkTimersForKey(Set<TimerInternals.TimerData> originalTimers, Set<TimerInternals.TimerData> restoredTimers) {
+ boolean comp = true;
+ if(restoredTimers == null) {
+ return false;
+ }
+
+ if(originalTimers.size() != restoredTimers.size()) {
+ return false;
+ }
+
+ for(TimerInternals.TimerData timer: originalTimers) {
+ comp &= restoredTimers.contains(timer);
+ }
+ return comp;
+ }
+
+ private boolean checkStateForKey(FlinkStateInternals<String> originalState, FlinkStateInternals<String> restoredState) throws CannotProvideCoderException {
+ if(restoredState == null) {
+ return false;
+ }
+
+ ValueState<String> orValue = originalState.state(NAMESPACE_1, STRING_VALUE_ADDR);
+ ValueState<String> resValue = restoredState.state(NAMESPACE_1, STRING_VALUE_ADDR);
+ boolean comp = orValue.get().read().equals(resValue.get().read());
+
+ ValueState<Integer> orIntValue = originalState.state(NAMESPACE_1, INT_VALUE_ADDR);
+ ValueState<Integer> resIntValue = restoredState.state(NAMESPACE_1, INT_VALUE_ADDR);
+ comp &= orIntValue.get().read().equals(resIntValue.get().read());
+
+ CombiningValueState<Integer, Integer> combOrValue = originalState.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+ CombiningValueState<Integer, Integer> combResValue = restoredState.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+ comp &= combOrValue.get().read().equals(combResValue.get().read());
+
+ WatermarkStateInternal orWatermark = originalState.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
+ WatermarkStateInternal resWatermark = restoredState.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
+ comp &= orWatermark.get().read().equals(resWatermark.get().read());
+
+ BagState<String> orBag = originalState.state(NAMESPACE_1, STRING_BAG_ADDR);
+ BagState<String> resBag = restoredState.state(NAMESPACE_1, STRING_BAG_ADDR);
+
+ Iterator<String> orIt = orBag.get().read().iterator();
+ Iterator<String> resIt = resBag.get().read().iterator();
+
+ while (orIt.hasNext() && resIt.hasNext()) {
+ comp &= orIt.next().equals(resIt.next());
+ }
+
+ return !((orIt.hasNext() && !resIt.hasNext()) || (!orIt.hasNext() && resIt.hasNext())) && comp;
+ }
+
+ private FlinkStateInternals<String> createState(String key) throws CannotProvideCoderException {
+ return new FlinkStateInternals<>(
+ key,
+ StringUtf8Coder.of(),
+ IntervalWindow.getCoder(),
+ combiner.asKeyedFn());
+ }
+
+ @Test
+ public void test() throws Exception {
+ StateSerializationTest test = new StateSerializationTest();
+ test.initializeStateAndTimers();
+
+ MemoryStateBackend.MemoryCheckpointOutputStream memBackend = new MemoryStateBackend.MemoryCheckpointOutputStream(25728);
+ StateBackend.CheckpointStateOutputView out = new StateBackend.CheckpointStateOutputView(memBackend);
+
+ test.storeState(out);
+
+ byte[] contents = memBackend.closeAndGetBytes();
+ ByteArrayInputView in = new ByteArrayInputView(contents);
+
+ assertEquals(test.restoreAndTestState(in), true);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java
index cbf5d77..74f754b 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java
@@ -76,8 +76,8 @@ public class JoinExamples {
KV<String, CoGbkResult> e = c.element();
CoGbkResult val = e.getValue();
String countryCode = e.getKey();
- String countryName;
- countryName = e.getValue().getOnly(countryInfoTag);
+ String countryName = "none";
+ countryName = e.getValue().getOnly(countryInfoTag, "Kostas");
for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) {
// Generate a string that combines information from both collection values
c.output(KV.of(countryCode, "Country name: " + countryName
[15/50] [abbrv] incubator-beam git commit: [tests] integrate
Wikipedia session test
Posted by da...@apache.org.
[tests] integrate Wikipedia session test
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/37a9b292
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/37a9b292
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/37a9b292
Branch: refs/heads/master
Commit: 37a9b292d7895897225f7484bc18d0d2db55f547
Parents: 3227fcc
Author: Max <ma...@posteo.de>
Authored: Tue Feb 23 08:30:34 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
.../flink/dataflow/TopWikipediaSessions.java | 210 ------------------
.../dataflow/TopWikipediaSessionsITCase.java | 215 +++++++++++++------
2 files changed, 144 insertions(+), 281 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/37a9b292/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessions.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessions.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessions.java
deleted file mode 100644
index ab5565a..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessions.java
+++ /dev/null
@@ -1,210 +0,0 @@
-///*
-// * Copyright (C) 2015 Google Inc.
-// *
-// * 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://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 com.dataartisans.flink.dataflow;
-//
-//import com.google.api.services.bigquery.model.TableRow;
-//import com.google.cloud.dataflow.sdk.Pipeline;
-//import com.google.cloud.dataflow.sdk.coders.TableRowJsonCoder;
-//import com.google.cloud.dataflow.sdk.io.TextIO;
-//import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
-//import com.google.cloud.dataflow.sdk.options.Default;
-//import com.google.cloud.dataflow.sdk.options.Description;
-//import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-//import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-//import com.google.cloud.dataflow.sdk.options.Validation;
-//import com.google.cloud.dataflow.sdk.transforms.Count;
-//import com.google.cloud.dataflow.sdk.transforms.DoFn;
-//import com.google.cloud.dataflow.sdk.transforms.DoFn.RequiresWindowAccess;
-//import com.google.cloud.dataflow.sdk.transforms.PTransform;
-//import com.google.cloud.dataflow.sdk.transforms.ParDo;
-//import com.google.cloud.dataflow.sdk.transforms.SerializableComparator;
-//import com.google.cloud.dataflow.sdk.transforms.Top;
-//import com.google.cloud.dataflow.sdk.transforms.windowing.CalendarWindows;
-//import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
-//import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-//import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-//import com.google.cloud.dataflow.sdk.values.KV;
-//import com.google.cloud.dataflow.sdk.values.PCollection;
-//
-//import org.joda.time.Duration;
-//import org.joda.time.Instant;
-//
-//import java.util.List;
-//
-///**
-// * Copied from {@link com.google.cloud.dataflow.examples.complete.TopWikipediaSessions} because the code
-// * is private there.
-// */
-//public class TopWikipediaSessions {
-// private static final String EXPORTED_WIKI_TABLE = "gs://dataflow-samples/wikipedia_edits/*.json";
-//
-// /**
-// * Extracts user and timestamp from a TableRow representing a Wikipedia edit.
-// */
-// static class ExtractUserAndTimestamp extends DoFn<TableRow, String> {
-// private static final long serialVersionUID = 0;
-//
-// @Override
-// public void processElement(ProcessContext c) {
-// TableRow row = c.element();
-// int timestamp = (Integer) row.get("timestamp");
-// String userName = (String) row.get("contributor_username");
-// if (userName != null) {
-// // Sets the implicit timestamp field to be used in windowing.
-// c.outputWithTimestamp(userName, new Instant(timestamp * 1000L));
-// }
-// }
-// }
-//
-// /**
-// * Computes the number of edits in each user session. A session is defined as
-// * a string of edits where each is separated from the next by less than an hour.
-// */
-// static class ComputeSessions
-// extends PTransform<PCollection<String>, PCollection<KV<String, Long>>> {
-// private static final long serialVersionUID = 0;
-//
-// @Override
-// public PCollection<KV<String, Long>> apply(PCollection<String> actions) {
-// return actions
-// .apply(Window.<String>into(Sessions.withGapDuration(Duration.standardHours(1))))
-//
-// .apply(Count.<String>perElement());
-// }
-// }
-//
-// /**
-// * Computes the longest session ending in each month.
-// */
-// private static class TopPerMonth
-// extends PTransform<PCollection<KV<String, Long>>, PCollection<List<KV<String, Long>>>> {
-// private static final long serialVersionUID = 0;
-//
-// @Override
-// public PCollection<List<KV<String, Long>>> apply(PCollection<KV<String, Long>> sessions) {
-// return sessions
-// .apply(Window.<KV<String, Long>>into(CalendarWindows.months(1)))
-//
-// .apply(Top.of(1, new SerializableComparator<KV<String, Long>>() {
-// private static final long serialVersionUID = 0;
-//
-// @Override
-// public int compare(KV<String, Long> o1, KV<String, Long> o2) {
-// return Long.compare(o1.getValue(), o2.getValue());
-// }
-// }).withoutDefaults());
-// }
-// }
-//
-// static class SessionsToStringsDoFn extends DoFn<KV<String, Long>, KV<String, Long>>
-// implements RequiresWindowAccess {
-//
-// private static final long serialVersionUID = 0;
-//
-// @Override
-// public void processElement(ProcessContext c) {
-// c.output(KV.of(
-// c.element().getKey() + " : " + c.window(), c.element().getValue()));
-// }
-// }
-//
-// static class FormatOutputDoFn extends DoFn<List<KV<String, Long>>, String>
-// implements RequiresWindowAccess {
-// private static final long serialVersionUID = 0;
-//
-// @Override
-// public void processElement(ProcessContext c) {
-// for (KV<String, Long> item : c.element()) {
-// String session = item.getKey();
-// long count = item.getValue();
-// c.output(session + " : " + count + " : " + ((IntervalWindow) c.window()).start());
-// }
-// }
-// }
-//
-// static class ComputeTopSessions extends PTransform<PCollection<TableRow>, PCollection<String>> {
-//
-// private static final long serialVersionUID = 0;
-//
-// private final double samplingThreshold;
-//
-// public ComputeTopSessions(double samplingThreshold) {
-// this.samplingThreshold = samplingThreshold;
-// }
-//
-// @Override
-// public PCollection<String> apply(PCollection<TableRow> input) {
-// return input
-// .apply(ParDo.of(new ExtractUserAndTimestamp()))
-//
-// .apply(ParDo.named("SampleUsers").of(
-// new DoFn<String, String>() {
-// private static final long serialVersionUID = 0;
-//
-// @Override
-// public void processElement(ProcessContext c) {
-// if (Math.abs(c.element().hashCode()) <= Integer.MAX_VALUE * samplingThreshold) {
-// c.output(c.element());
-// }
-// }
-// }))
-//
-// .apply(new ComputeSessions())
-//
-// .apply(ParDo.named("SessionsToStrings").of(new SessionsToStringsDoFn()))
-// .apply(new TopPerMonth())
-// .apply(ParDo.named("FormatOutput").of(new FormatOutputDoFn()));
-// }
-// }
-//
-// /**
-// * Options supported by this class.
-// *
-// * <p> Inherits standard Dataflow configuration options.
-// */
-// private static interface Options extends PipelineOptions {
-// @Description(
-// "Input specified as a GCS path containing a BigQuery table exported as json")
-// @Default.String(EXPORTED_WIKI_TABLE)
-// String getInput();
-// void setInput(String value);
-//
-// @Description("File to output results to")
-// @Validation.Required
-// String getOutput();
-// void setOutput(String value);
-// }
-//
-// public static void main(String[] args) {
-// Options options = PipelineOptionsFactory.fromArgs(args)
-// .withValidation()
-// .as(Options.class);
-// DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class);
-//
-// Pipeline p = Pipeline.create(dataflowOptions);
-//
-// double samplingThreshold = 0.1;
-//
-// p.apply(TextIO.Read
-// .from(options.getInput())
-// .withCoder(TableRowJsonCoder.of()))
-// .apply(new ComputeTopSessions(samplingThreshold))
-// .apply(TextIO.Write.named("Write").withoutSharding().to(options.getOutput()));
-//
-// p.run();
-// }
-//}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/37a9b292/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java
index 9c8147b..eb020c5 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java
@@ -1,71 +1,144 @@
-///*
-// * Copyright 2015 Data Artisans GmbH
-// *
-// * 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://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 com.dataartisans.flink.dataflow;
-//
-//import com.google.api.services.bigquery.model.TableRow;
-//import com.google.cloud.dataflow.sdk.Pipeline;
-//import com.google.cloud.dataflow.sdk.io.TextIO;
-//import com.google.cloud.dataflow.sdk.transforms.Create;
-//import com.google.cloud.dataflow.sdk.values.PCollection;
-//import com.google.common.base.Joiner;
-//import org.apache.flink.test.util.JavaProgramTestBase;
-//
-//import java.util.Arrays;
-//
-//public class TopWikipediaSessionsITCase extends JavaProgramTestBase {
-// protected String resultPath;
-//
-// public TopWikipediaSessionsITCase(){
-// }
-//
-// static final String[] EXPECTED_RESULT = new String[] {
-// "user1 : [1970-01-01T00:00:00.000Z..1970-01-01T01:00:02.000Z)"
-// + " : 3 : 1970-01-01T00:00:00.000Z",
-// "user3 : [1970-02-05T00:00:00.000Z..1970-02-05T01:00:00.000Z)"
-// + " : 1 : 1970-02-01T00:00:00.000Z" };
-//
-// @Override
-// protected void preSubmit() throws Exception {
-// resultPath = getTempDirPath("result");
-// }
-//
-// @Override
-// protected void postSubmit() throws Exception {
-// compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
-// }
-//
-// @Override
-// protected void testProgram() throws Exception {
-//
-// Pipeline p = FlinkTestPipeline.create();
-//
-// PCollection<String> output =
-// p.apply(Create.of(Arrays.asList(new TableRow().set("timestamp", 0).set
-// ("contributor_username", "user1"), new TableRow().set("timestamp", 1).set
-// ("contributor_username", "user1"), new TableRow().set("timestamp", 2).set
-// ("contributor_username", "user1"), new TableRow().set("timestamp", 0).set
-// ("contributor_username", "user2"), new TableRow().set("timestamp", 1).set
-// ("contributor_username", "user2"), new TableRow().set("timestamp", 3601).set
-// ("contributor_username", "user2"), new TableRow().set("timestamp", 3602).set
-// ("contributor_username", "user2"), new TableRow().set("timestamp", 35 * 24 * 3600)
-// .set("contributor_username", "user3"))))
-// .apply(new TopWikipediaSessions.ComputeTopSessions(1.0));
-//
-// output.apply(TextIO.Write.to(resultPath));
-//
-// p.run();
-// }
-//}
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow;
+
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Count;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Joiner;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.io.Serializable;
+import java.util.Arrays;
+
+
+/**
+ * Session window test
+ */
+public class TopWikipediaSessionsITCase extends StreamingProgramTestBase implements Serializable {
+ protected String resultPath;
+
+ public TopWikipediaSessionsITCase(){
+ }
+
+ static final String[] EXPECTED_RESULT = new String[] {
+ "user: user1 value:3",
+ "user: user1 value:1",
+ "user: user2 value:4",
+ "user: user2 value:6",
+ "user: user3 value:7",
+ "user: user3 value:2"
+ };
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+
+ Pipeline p = FlinkTestPipeline.createStreaming();
+
+ long now = System.currentTimeMillis() + 10000;
+ System.out.println((now + 5000) / 1000);
+
+ PCollection<KV<String, Long>> output =
+ p.apply(Create.of(Arrays.asList(new TableRow().set("timestamp", now).set
+ ("contributor_username", "user1"), new TableRow().set("timestamp", now + 10).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now).set
+ ("contributor_username", "user1"), new TableRow().set("timestamp", now + 2).set
+ ("contributor_username", "user1"), new TableRow().set("timestamp", now).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 1).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 5).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 7).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 8).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 200).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 230).set
+ ("contributor_username", "user1"), new TableRow().set("timestamp", now + 230).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 240).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 245).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 235).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 236).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 237).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 238).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 239).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 240).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 241).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now)
+ .set("contributor_username", "user3"))))
+
+
+
+ .apply(ParDo.of(new DoFn<TableRow, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ TableRow row = c.element();
+ long timestamp = (Long) row.get("timestamp");
+ String userName = (String) row.get("contributor_username");
+ if (userName != null) {
+ // Sets the timestamp field to be used in windowing.
+ c.outputWithTimestamp(userName, new Instant(timestamp * 1000L));
+ }
+ }
+ }))
+
+ .apply(ParDo.named("SampleUsers").of(
+ new DoFn<String, String>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ if (Math.abs(c.element().hashCode()) <= Integer.MAX_VALUE * 1.0) {
+ c.output(c.element());
+ }
+ }
+ }))
+
+ .apply(Window.<String>into(Sessions.withGapDuration(Duration.standardMinutes(1))))
+ .apply(Count.<String>perElement());
+
+ PCollection<String> format = output.apply(ParDo.of(new DoFn<KV<String, Long>, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ KV<String, Long> el = c.element();
+ String out = "user: " + el.getKey() + " value:" + el.getValue();
+ System.out.println(out);
+ c.output(out);
+ }
+ }));
+
+ format.apply(TextIO.Write.to(resultPath));
+
+ p.run();
+ }
+}
[10/50] [abbrv] incubator-beam git commit: [readme] update to reflect
the current state
Posted by da...@apache.org.
[readme] update to reflect the current state
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/70ae13c7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/70ae13c7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/70ae13c7
Branch: refs/heads/master
Commit: 70ae13c7497907cd7ba81481dc7eafff1615adfb
Parents: 8434c3c
Author: Max <ma...@posteo.de>
Authored: Thu Feb 11 12:36:02 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
runners/flink/README.md | 82 ++++++++++++++++++++++++++++++++++++--------
1 file changed, 67 insertions(+), 15 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/70ae13c7/runners/flink/README.md
----------------------------------------------------------------------
diff --git a/runners/flink/README.md b/runners/flink/README.md
index 54d248c..499ed6d 100644
--- a/runners/flink/README.md
+++ b/runners/flink/README.md
@@ -1,13 +1,72 @@
Flink-Dataflow
--------------
-Flink-Dataflow is a Google Dataflow Runner for Apache Flink. It enables you to
-run Dataflow programs with Flink as an execution engine.
+Flink-Dataflow is a Runner for Google Dataflow (aka Apache Beam) which enables you to
+run Dataflow programs with Flink. It integrates seamlessly with the Dataflow
+API, allowing you to execute Dataflow programs in streaming or batch mode.
+
+## Streaming
+
+### Full Dataflow Windowing and Triggering Semantics
+
+The Flink Dataflow Runner supports *Event Time* allowing you to analyze data with respect to its
+associated timestamp. It handles out-or-order and late-arriving elements. You may leverage the full
+power of the Dataflow windowing semantics like *time-based*, *sliding*, *tumbling*, or *count*
+windows. You may build *session* windows which allow you to keep track of events associated with
+each other.
+
+### Fault-Tolerance
+
+The program's state is persisted by Apache Flink. You may re-run and resume your program upon
+failure or if you decide to continue computation at a later time.
+
+### Sources and Sinks
+
+Build your own data ingestion or digestion using the source/sink interface. Re-use Flink's sources
+and sinks or use the provided support for Apache Kafka.
+
+### Seamless integration
+
+To execute a Dataflow program in streaming mode, just enable streaming in the `PipelineOptions`:
+
+ options.setStreaming(true);
+
+That's it. If you prefer batched execution, simply disable streaming mode.
+
+## Batch
+
+### Batch optimization
+
+Flink gives you out-of-core algorithms which operate on its managed memory to perform sorting,
+caching, and hash table operations. We have optimized operations like CoGroup to use Flink's
+optimized out-of-core implementation.
+
+### Fault-Tolerance
+
+We guarantee job-level fault-tolerance which gracefully restarts failed batch jobs.
+
+### Sources and Sinks
+
+Build your own data ingestion or digestion using the source/sink interface or re-use Flink's sources
+and sinks.
+
+## Features
+
+The Flink Dataflow Runner maintains as much compatibility with the Dataflow API as possible. We
+support transformations on data like:
+
+- Grouping
+- Windowing
+- ParDo
+- CoGroup
+- Flatten
+- Combine
+- Side inputs/outputs
+- Encoding
# Getting Started
-To get started using Google Dataflow on top of Apache Flink, we need to install the
-latest version of Flink-Dataflow.
+To get started using Flink-Dataflow, we first need to install the latest version.
## Install Flink-Dataflow ##
@@ -46,7 +105,6 @@ p.apply(TextIO.Read.named("ReadLines").from(options.getInput()))
p.run();
```
-
To execute the example, let's first get some sample data:
curl http://www.gutenberg.org/cache/epub/1128/pg1128.txt > kinglear.txt
@@ -58,7 +116,7 @@ Then let's run the included WordCount locally on your machine:
Congratulations, you have run your first Google Dataflow program on top of Apache Flink!
-# Running Dataflow on Flink on a cluster
+# Running Dataflow programs on a Flink cluster
You can run your Dataflow program on an Apache Flink cluster. Please start off by creating a new
Maven project.
@@ -137,14 +195,8 @@ folder to the Flink cluster using the command-line utility like so:
./bin/flink run /path/to/fat.jar
-For more information, please visit the [Apache Flink Website](http://flink.apache.org) or contact
-the [Mailinglists](http://flink.apache.org/community.html#mailing-lists).
-
-# Streaming
-Streaming support has been added. It is currently in alpha stage. Please give it a try. To use
-streaming, just enable streaming mode in the `PipelineOptions`:
+# More
- options.setStreaming(true);
-
-That's all.
\ No newline at end of file
+For more information, please visit the [Apache Flink Website](http://flink.apache.org) or contact
+the [Mailinglists](http://flink.apache.org/community.html#mailing-lists).
\ No newline at end of file
[23/50] [abbrv] incubator-beam git commit: [flink] adjust directories
according to package name
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
new file mode 100644
index 0000000..e2ceae6
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
@@ -0,0 +1,264 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers.streaming;
+
+import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.common.base.Preconditions;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
+import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
+import com.google.cloud.dataflow.sdk.util.*;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.base.Throwables;
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.accumulators.AccumulatorHelper;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+import org.joda.time.format.PeriodFormat;
+
+import java.util.Collection;
+
+/**
+ * An abstract class that encapsulates the common code of the the {@link com.google.cloud.dataflow.sdk.transforms.ParDo.Bound}
+ * and {@link com.google.cloud.dataflow.sdk.transforms.ParDo.BoundMulti} wrappers. See the {@link FlinkParDoBoundWrapper} and
+ * {@link FlinkParDoBoundMultiWrapper} for the actual wrappers of the aforementioned transformations.
+ * */
+public abstract class FlinkAbstractParDoWrapper<IN, OUTDF, OUTFL> extends RichFlatMapFunction<WindowedValue<IN>, WindowedValue<OUTFL>> {
+
+ private final DoFn<IN, OUTDF> doFn;
+ private final WindowingStrategy<?, ?> windowingStrategy;
+ private transient PipelineOptions options;
+
+ private DoFnProcessContext context;
+
+ public FlinkAbstractParDoWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUTDF> doFn) {
+ Preconditions.checkNotNull(options);
+ Preconditions.checkNotNull(windowingStrategy);
+ Preconditions.checkNotNull(doFn);
+
+ this.doFn = doFn;
+ this.options = options;
+ this.windowingStrategy = windowingStrategy;
+ }
+
+ private void initContext(DoFn<IN, OUTDF> function, Collector<WindowedValue<OUTFL>> outCollector) {
+ if (this.context == null) {
+ this.context = new DoFnProcessContext(function, outCollector);
+ }
+ }
+
+ @Override
+ public void flatMap(WindowedValue<IN> value, Collector<WindowedValue<OUTFL>> out) throws Exception {
+ this.initContext(doFn, out);
+
+ // for each window the element belongs to, create a new copy here.
+ Collection<? extends BoundedWindow> windows = value.getWindows();
+ if (windows.size() <= 1) {
+ processElement(value);
+ } else {
+ for (BoundedWindow window : windows) {
+ processElement(WindowedValue.of(
+ value.getValue(), value.getTimestamp(), window, value.getPane()));
+ }
+ }
+ }
+
+ private void processElement(WindowedValue<IN> value) throws Exception {
+ this.context.setElement(value);
+ this.doFn.startBundle(context);
+ doFn.processElement(context);
+ this.doFn.finishBundle(context);
+ }
+
+ private class DoFnProcessContext extends DoFn<IN, OUTDF>.ProcessContext {
+
+ private final DoFn<IN, OUTDF> fn;
+
+ protected final Collector<WindowedValue<OUTFL>> collector;
+
+ private WindowedValue<IN> element;
+
+ private DoFnProcessContext(DoFn<IN, OUTDF> function, Collector<WindowedValue<OUTFL>> outCollector) {
+ function.super();
+ super.setupDelegateAggregators();
+
+ this.fn = function;
+ this.collector = outCollector;
+ }
+
+ public void setElement(WindowedValue<IN> value) {
+ this.element = value;
+ }
+
+ @Override
+ public IN element() {
+ return this.element.getValue();
+ }
+
+ @Override
+ public Instant timestamp() {
+ return this.element.getTimestamp();
+ }
+
+ @Override
+ public BoundedWindow window() {
+ if (!(fn instanceof DoFn.RequiresWindowAccess)) {
+ throw new UnsupportedOperationException(
+ "window() is only available in the context of a DoFn marked as RequiresWindow.");
+ }
+
+ Collection<? extends BoundedWindow> windows = this.element.getWindows();
+ if (windows.size() != 1) {
+ throw new IllegalArgumentException("Each element is expected to belong to 1 window. " +
+ "This belongs to " + windows.size() + ".");
+ }
+ return windows.iterator().next();
+ }
+
+ @Override
+ public PaneInfo pane() {
+ return this.element.getPane();
+ }
+
+ @Override
+ public WindowingInternals<IN, OUTDF> windowingInternals() {
+ return windowingInternalsHelper(element, collector);
+ }
+
+ @Override
+ public PipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view) {
+ throw new RuntimeException("sideInput() is not supported in Streaming mode.");
+ }
+
+ @Override
+ public void output(OUTDF output) {
+ outputWithTimestamp(output, this.element.getTimestamp());
+ }
+
+ @Override
+ public void outputWithTimestamp(OUTDF output, Instant timestamp) {
+ outputWithTimestampHelper(element, output, timestamp, collector);
+ }
+
+ @Override
+ public <T> void sideOutput(TupleTag<T> tag, T output) {
+ sideOutputWithTimestamp(tag, output, this.element.getTimestamp());
+ }
+
+ @Override
+ public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+ sideOutputWithTimestampHelper(element, output, timestamp, collector, tag);
+ }
+
+ @Override
+ protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
+ Accumulator acc = getRuntimeContext().getAccumulator(name);
+ if (acc != null) {
+ AccumulatorHelper.compareAccumulatorTypes(name,
+ SerializableFnAggregatorWrapper.class, acc.getClass());
+ return (Aggregator<AggInputT, AggOutputT>) acc;
+ }
+
+ SerializableFnAggregatorWrapper<AggInputT, AggOutputT> accumulator =
+ new SerializableFnAggregatorWrapper<>(combiner);
+ getRuntimeContext().addAccumulator(name, accumulator);
+ return accumulator;
+ }
+ }
+
+ protected void checkTimestamp(WindowedValue<IN> ref, Instant timestamp) {
+ if (timestamp.isBefore(ref.getTimestamp().minus(doFn.getAllowedTimestampSkew()))) {
+ throw new IllegalArgumentException(String.format(
+ "Cannot output with timestamp %s. Output timestamps must be no earlier than the "
+ + "timestamp of the current input (%s) minus the allowed skew (%s). See the "
+ + "DoFn#getAllowedTimestmapSkew() Javadoc for details on changing the allowed skew.",
+ timestamp, ref.getTimestamp(),
+ PeriodFormat.getDefault().print(doFn.getAllowedTimestampSkew().toPeriod())));
+ }
+ }
+
+ protected <T> WindowedValue<T> makeWindowedValue(
+ T output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+ final Instant inputTimestamp = timestamp;
+ final WindowFn windowFn = windowingStrategy.getWindowFn();
+
+ if (timestamp == null) {
+ timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+ }
+
+ if (windows == null) {
+ try {
+ windows = windowFn.assignWindows(windowFn.new AssignContext() {
+ @Override
+ public Object element() {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input element when none was available");
+ }
+
+ @Override
+ public Instant timestamp() {
+ if (inputTimestamp == null) {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input timestamp when none was available");
+ }
+ return inputTimestamp;
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input windows when none were available");
+ }
+ });
+ } catch (Exception e) {
+ throw UserCodeException.wrap(e);
+ }
+ }
+
+ return WindowedValue.of(output, timestamp, windows, pane);
+ }
+
+ /////////// ABSTRACT METHODS TO BE IMPLEMENTED BY SUBCLASSES /////////////////
+
+ public abstract void outputWithTimestampHelper(
+ WindowedValue<IN> inElement,
+ OUTDF output,
+ Instant timestamp,
+ Collector<WindowedValue<OUTFL>> outCollector);
+
+ public abstract <T> void sideOutputWithTimestampHelper(
+ WindowedValue<IN> inElement,
+ T output,
+ Instant timestamp,
+ Collector<WindowedValue<OUTFL>> outCollector,
+ TupleTag<T> tag);
+
+ public abstract WindowingInternals<IN, OUTDF> windowingInternalsHelper(
+ WindowedValue<IN> inElement,
+ Collector<WindowedValue<OUTFL>> outCollector);
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
new file mode 100644
index 0000000..906a399
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
@@ -0,0 +1,629 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers.streaming;
+
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.*;
+import com.google.cloud.dataflow.sdk.coders.*;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
+import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
+import com.google.cloud.dataflow.sdk.util.*;
+import com.google.cloud.dataflow.sdk.values.*;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimap;
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.accumulators.AccumulatorHelper;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.KeyedStream;
+import org.apache.flink.streaming.api.operators.*;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * This class is the key class implementing all the windowing/triggering logic of Apache Beam.
+ * To provide full compatibility and support for all the windowing/triggering combinations offered by
+ * Beam, we opted for a strategy that uses the SDK's code for doing these operations. See the code in
+ * ({@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn}.
+ * <p/>
+ * In a nutshell, when the execution arrives to this operator, we expect to have a stream <b>already
+ * grouped by key</b>. Each of the elements that enter here, registers a timer
+ * (see {@link TimerInternals#setTimer(TimerInternals.TimerData)} in the
+ * {@link FlinkGroupAlsoByWindowWrapper#activeTimers}.
+ * This is essentially a timestamp indicating when to trigger the computation over the window this
+ * element belongs to.
+ * <p/>
+ * When a watermark arrives, all the registered timers are checked to see which ones are ready to
+ * fire (see {@link FlinkGroupAlsoByWindowWrapper#processWatermark(Watermark)}). These are deregistered from
+ * the {@link FlinkGroupAlsoByWindowWrapper#activeTimers}
+ * list, and are fed into the {@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn}
+ * for furhter processing.
+ */
+public class FlinkGroupAlsoByWindowWrapper<K, VIN, VACC, VOUT>
+ extends AbstractStreamOperator<WindowedValue<KV<K, VOUT>>>
+ implements OneInputStreamOperator<WindowedValue<KV<K, VIN>>, WindowedValue<KV<K, VOUT>>> {
+
+ private static final long serialVersionUID = 1L;
+
+ private transient PipelineOptions options;
+
+ private transient CoderRegistry coderRegistry;
+
+ private DoFn<KeyedWorkItem<K, VIN>, KV<K, VOUT>> operator;
+
+ private ProcessContext context;
+
+ private final WindowingStrategy<KV<K, VIN>, BoundedWindow> windowingStrategy;
+
+ private final Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combineFn;
+
+ private final KvCoder<K, VIN> inputKvCoder;
+
+ /**
+ * State is kept <b>per-key</b>. This data structure keeps this mapping between an active key, i.e. a
+ * key whose elements are currently waiting to be processed, and its associated state.
+ */
+ private Map<K, FlinkStateInternals<K>> perKeyStateInternals = new HashMap<>();
+
+ /**
+ * Timers waiting to be processed.
+ */
+ private Map<K, Set<TimerInternals.TimerData>> activeTimers = new HashMap<>();
+
+ private FlinkTimerInternals timerInternals = new FlinkTimerInternals();
+
+ /**
+ * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy.
+ * This method assumes that <b>elements are already grouped by key</b>.
+ * <p/>
+ * The difference with {@link #createForIterable(PipelineOptions, PCollection, KeyedStream)}
+ * is that this method assumes that a combiner function is provided
+ * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}).
+ * A combiner helps at increasing the speed and, in most of the cases, reduce the per-window state.
+ *
+ * @param options the general job configuration options.
+ * @param input the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}.
+ * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key.
+ * @param combiner the combiner to be used.
+ * @param outputKvCoder the type of the output values.
+ */
+ public static <K, VIN, VACC, VOUT> DataStream<WindowedValue<KV<K, VOUT>>> create(
+ PipelineOptions options,
+ PCollection input,
+ KeyedStream<WindowedValue<KV<K, VIN>>, K> groupedStreamByKey,
+ Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner,
+ KvCoder<K, VOUT> outputKvCoder) {
+ Preconditions.checkNotNull(options);
+
+ KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) input.getCoder();
+ FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper<>(options,
+ input.getPipeline().getCoderRegistry(), input.getWindowingStrategy(), inputKvCoder, combiner);
+
+ Coder<WindowedValue<KV<K, VOUT>>> windowedOutputElemCoder = WindowedValue.FullWindowedValueCoder.of(
+ outputKvCoder,
+ input.getWindowingStrategy().getWindowFn().windowCoder());
+
+ CoderTypeInformation<WindowedValue<KV<K, VOUT>>> outputTypeInfo =
+ new CoderTypeInformation<>(windowedOutputElemCoder);
+
+ DataStream<WindowedValue<KV<K, VOUT>>> groupedByKeyAndWindow = groupedStreamByKey
+ .transform("GroupByWindowWithCombiner",
+ new CoderTypeInformation<>(outputKvCoder),
+ windower)
+ .returns(outputTypeInfo);
+
+ return groupedByKeyAndWindow;
+ }
+
+ /**
+ * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy.
+ * This method assumes that <b>elements are already grouped by key</b>.
+ * <p/>
+ * The difference with {@link #create(PipelineOptions, PCollection, KeyedStream, Combine.KeyedCombineFn, KvCoder)}
+ * is that this method assumes no combiner function
+ * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}).
+ *
+ * @param options the general job configuration options.
+ * @param input the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}.
+ * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key.
+ */
+ public static <K, VIN> DataStream<WindowedValue<KV<K, Iterable<VIN>>>> createForIterable(
+ PipelineOptions options,
+ PCollection input,
+ KeyedStream<WindowedValue<KV<K, VIN>>, K> groupedStreamByKey) {
+ Preconditions.checkNotNull(options);
+
+ KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) input.getCoder();
+ Coder<K> keyCoder = inputKvCoder.getKeyCoder();
+ Coder<VIN> inputValueCoder = inputKvCoder.getValueCoder();
+
+ FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper(options,
+ input.getPipeline().getCoderRegistry(), input.getWindowingStrategy(), inputKvCoder, null);
+
+ Coder<Iterable<VIN>> valueIterCoder = IterableCoder.of(inputValueCoder);
+ KvCoder<K, Iterable<VIN>> outputElemCoder = KvCoder.of(keyCoder, valueIterCoder);
+
+ Coder<WindowedValue<KV<K, Iterable<VIN>>>> windowedOutputElemCoder = WindowedValue.FullWindowedValueCoder.of(
+ outputElemCoder,
+ input.getWindowingStrategy().getWindowFn().windowCoder());
+
+ CoderTypeInformation<WindowedValue<KV<K, Iterable<VIN>>>> outputTypeInfo =
+ new CoderTypeInformation<>(windowedOutputElemCoder);
+
+ DataStream<WindowedValue<KV<K, Iterable<VIN>>>> groupedByKeyAndWindow = groupedStreamByKey
+ .transform("GroupByWindow",
+ new CoderTypeInformation<>(windowedOutputElemCoder),
+ windower)
+ .returns(outputTypeInfo);
+
+ return groupedByKeyAndWindow;
+ }
+
+ public static <K, VIN, VACC, VOUT> FlinkGroupAlsoByWindowWrapper
+ createForTesting(PipelineOptions options,
+ CoderRegistry registry,
+ WindowingStrategy<KV<K, VIN>, BoundedWindow> windowingStrategy,
+ KvCoder<K, VIN> inputCoder,
+ Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner) {
+ Preconditions.checkNotNull(options);
+
+ return new FlinkGroupAlsoByWindowWrapper(options, registry, windowingStrategy, inputCoder, combiner);
+ }
+
+ private FlinkGroupAlsoByWindowWrapper(PipelineOptions options,
+ CoderRegistry registry,
+ WindowingStrategy<KV<K, VIN>, BoundedWindow> windowingStrategy,
+ KvCoder<K, VIN> inputCoder,
+ Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner) {
+ Preconditions.checkNotNull(options);
+
+ this.options = Preconditions.checkNotNull(options);
+ this.coderRegistry = Preconditions.checkNotNull(registry);
+ this.inputKvCoder = Preconditions.checkNotNull(inputCoder);//(KvCoder<K, VIN>) input.getCoder();
+ this.windowingStrategy = Preconditions.checkNotNull(windowingStrategy);//input.getWindowingStrategy();
+ this.combineFn = combiner;
+ this.operator = createGroupAlsoByWindowOperator();
+ this.chainingStrategy = ChainingStrategy.ALWAYS;
+ }
+
+ @Override
+ public void open() throws Exception {
+ super.open();
+ this.context = new ProcessContext(operator, new TimestampedCollector<>(output), this.timerInternals);
+ }
+
+ /**
+ * Create the adequate {@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn},
+ * <b> if not already created</b>.
+ * If a {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn} was provided, then
+ * a function with that combiner is created, so that elements are combined as they arrive. This is
+ * done for speed and (in most of the cases) for reduction of the per-window state.
+ */
+ private <W extends BoundedWindow> DoFn<KeyedWorkItem<K, VIN>, KV<K, VOUT>> createGroupAlsoByWindowOperator() {
+ if (this.operator == null) {
+ if (this.combineFn == null) {
+ // Thus VOUT == Iterable<VIN>
+ Coder<VIN> inputValueCoder = inputKvCoder.getValueCoder();
+
+ this.operator = (DoFn) GroupAlsoByWindowViaWindowSetDoFn.create(
+ (WindowingStrategy<?, W>) this.windowingStrategy, SystemReduceFn.<K, VIN, W>buffering(inputValueCoder));
+ } else {
+ Coder<K> inputKeyCoder = inputKvCoder.getKeyCoder();
+
+ AppliedCombineFn<K, VIN, VACC, VOUT> appliedCombineFn = AppliedCombineFn
+ .withInputCoder(combineFn, coderRegistry, inputKvCoder);
+
+ this.operator = GroupAlsoByWindowViaWindowSetDoFn.create(
+ (WindowingStrategy<?, W>) this.windowingStrategy, SystemReduceFn.<K, VIN, VACC, VOUT, W>combining(inputKeyCoder, appliedCombineFn));
+ }
+ }
+ return this.operator;
+ }
+
+ private void processKeyedWorkItem(KeyedWorkItem<K, VIN> workItem) throws Exception {
+ context.setElement(workItem, getStateInternalsForKey(workItem.key()));
+
+ // TODO: Ideally startBundle/finishBundle would be called when the operator is first used / about to be discarded.
+ operator.startBundle(context);
+ operator.processElement(context);
+ operator.finishBundle(context);
+ }
+
+ @Override
+ public void processElement(StreamRecord<WindowedValue<KV<K, VIN>>> element) throws Exception {
+ ArrayList<WindowedValue<VIN>> elements = new ArrayList<>();
+ elements.add(WindowedValue.of(element.getValue().getValue().getValue(), element.getValue().getTimestamp(),
+ element.getValue().getWindows(), element.getValue().getPane()));
+ processKeyedWorkItem(KeyedWorkItems.elementsWorkItem(element.getValue().getValue().getKey(), elements));
+ }
+
+ @Override
+ public void processWatermark(Watermark mark) throws Exception {
+ context.setCurrentInputWatermark(new Instant(mark.getTimestamp()));
+
+ Multimap<K, TimerInternals.TimerData> timers = getTimersReadyToProcess(mark.getTimestamp());
+ if (!timers.isEmpty()) {
+ for (K key : timers.keySet()) {
+ processKeyedWorkItem(KeyedWorkItems.<K, VIN>timersWorkItem(key, timers.get(key)));
+ }
+ }
+
+ /**
+ * This is to take into account the different semantics of the Watermark in Flink and
+ * in Dataflow. To understand the reasoning behind the Dataflow semantics and its
+ * watermark holding logic, see the documentation of
+ * {@link WatermarkHold#addHold(ReduceFn.ProcessValueContext, boolean)}
+ * */
+ long millis = Long.MAX_VALUE;
+ for (FlinkStateInternals state : perKeyStateInternals.values()) {
+ Instant watermarkHold = state.getWatermarkHold();
+ if (watermarkHold != null && watermarkHold.getMillis() < millis) {
+ millis = watermarkHold.getMillis();
+ }
+ }
+
+ if (mark.getTimestamp() < millis) {
+ millis = mark.getTimestamp();
+ }
+
+ context.setCurrentOutputWatermark(new Instant(millis));
+
+ // Don't forget to re-emit the watermark for further operators down the line.
+ // This is critical for jobs with multiple aggregation steps.
+ // Imagine a job with a groupByKey() on key K1, followed by a map() that changes
+ // the key K1 to K2, and another groupByKey() on K2. In this case, if the watermark
+ // is not re-emitted, the second aggregation would never be triggered, and no result
+ // will be produced.
+ output.emitWatermark(new Watermark(millis));
+ }
+
+ @Override
+ public void close() throws Exception {
+ super.close();
+ }
+
+ private void registerActiveTimer(K key, TimerInternals.TimerData timer) {
+ Set<TimerInternals.TimerData> timersForKey = activeTimers.get(key);
+ if (timersForKey == null) {
+ timersForKey = new HashSet<>();
+ }
+ timersForKey.add(timer);
+ activeTimers.put(key, timersForKey);
+ }
+
+ private void unregisterActiveTimer(K key, TimerInternals.TimerData timer) {
+ Set<TimerInternals.TimerData> timersForKey = activeTimers.get(key);
+ if (timersForKey != null) {
+ timersForKey.remove(timer);
+ if (timersForKey.isEmpty()) {
+ activeTimers.remove(key);
+ } else {
+ activeTimers.put(key, timersForKey);
+ }
+ }
+ }
+
+ /**
+ * Returns the list of timers that are ready to fire. These are the timers
+ * that are registered to be triggered at a time before the current watermark.
+ * We keep these timers in a Set, so that they are deduplicated, as the same
+ * timer can be registered multiple times.
+ */
+ private Multimap<K, TimerInternals.TimerData> getTimersReadyToProcess(long currentWatermark) {
+
+ // we keep the timers to return in a different list and launch them later
+ // because we cannot prevent a trigger from registering another trigger,
+ // which would lead to concurrent modification exception.
+ Multimap<K, TimerInternals.TimerData> toFire = HashMultimap.create();
+
+ Iterator<Map.Entry<K, Set<TimerInternals.TimerData>>> it = activeTimers.entrySet().iterator();
+ while (it.hasNext()) {
+ Map.Entry<K, Set<TimerInternals.TimerData>> keyWithTimers = it.next();
+
+ Iterator<TimerInternals.TimerData> timerIt = keyWithTimers.getValue().iterator();
+ while (timerIt.hasNext()) {
+ TimerInternals.TimerData timerData = timerIt.next();
+ if (timerData.getTimestamp().isBefore(currentWatermark)) {
+ toFire.put(keyWithTimers.getKey(), timerData);
+ timerIt.remove();
+ }
+ }
+
+ if (keyWithTimers.getValue().isEmpty()) {
+ it.remove();
+ }
+ }
+ return toFire;
+ }
+
+ /**
+ * Gets the state associated with the specified key.
+ *
+ * @param key the key whose state we want.
+ * @return The {@link FlinkStateInternals}
+ * associated with that key.
+ */
+ private FlinkStateInternals<K> getStateInternalsForKey(K key) {
+ FlinkStateInternals<K> stateInternals = perKeyStateInternals.get(key);
+ if (stateInternals == null) {
+ Coder<? extends BoundedWindow> windowCoder = this.windowingStrategy.getWindowFn().windowCoder();
+ OutputTimeFn<? super BoundedWindow> outputTimeFn = this.windowingStrategy.getWindowFn().getOutputTimeFn();
+ stateInternals = new FlinkStateInternals<>(key, inputKvCoder.getKeyCoder(), windowCoder, outputTimeFn);
+ perKeyStateInternals.put(key, stateInternals);
+ }
+ return stateInternals;
+ }
+
+ private class FlinkTimerInternals extends AbstractFlinkTimerInternals<K, VIN> {
+ @Override
+ public void setTimer(TimerData timerKey) {
+ registerActiveTimer(context.element().key(), timerKey);
+ }
+
+ @Override
+ public void deleteTimer(TimerData timerKey) {
+ unregisterActiveTimer(context.element().key(), timerKey);
+ }
+ }
+
+ private class ProcessContext extends GroupAlsoByWindowViaWindowSetDoFn<K, VIN, VOUT, ?, KeyedWorkItem<K, VIN>>.ProcessContext {
+
+ private final FlinkTimerInternals timerInternals;
+
+ private final TimestampedCollector<WindowedValue<KV<K, VOUT>>> collector;
+
+ private FlinkStateInternals<K> stateInternals;
+
+ private KeyedWorkItem<K, VIN> element;
+
+ public ProcessContext(DoFn<KeyedWorkItem<K, VIN>, KV<K, VOUT>> function,
+ TimestampedCollector<WindowedValue<KV<K, VOUT>>> outCollector,
+ FlinkTimerInternals timerInternals) {
+ function.super();
+ super.setupDelegateAggregators();
+
+ this.collector = Preconditions.checkNotNull(outCollector);
+ this.timerInternals = Preconditions.checkNotNull(timerInternals);
+ }
+
+ public void setElement(KeyedWorkItem<K, VIN> element,
+ FlinkStateInternals<K> stateForKey) {
+ this.element = element;
+ this.stateInternals = stateForKey;
+ }
+
+ public void setCurrentInputWatermark(Instant watermark) {
+ this.timerInternals.setCurrentInputWatermark(watermark);
+ }
+
+ public void setCurrentOutputWatermark(Instant watermark) {
+ this.timerInternals.setCurrentOutputWatermark(watermark);
+ }
+
+ @Override
+ public KeyedWorkItem<K, VIN> element() {
+ return this.element;
+ }
+
+ @Override
+ public Instant timestamp() {
+ throw new UnsupportedOperationException("timestamp() is not available when processing KeyedWorkItems.");
+ }
+
+ @Override
+ public PipelineOptions getPipelineOptions() {
+ // TODO: PipelineOptions need to be available on the workers.
+ // Ideally they are captured as part of the pipeline.
+ // For now, construct empty options so that StateContexts.createFromComponents
+ // will yield a valid StateContext, which is needed to support the StateContext.window().
+ if (options == null) {
+ options = new PipelineOptions() {
+ @Override
+ public <T extends PipelineOptions> T as(Class<T> kls) {
+ return null;
+ }
+
+ @Override
+ public <T extends PipelineOptions> T cloneAs(Class<T> kls) {
+ return null;
+ }
+
+ @Override
+ public Class<? extends PipelineRunner<?>> getRunner() {
+ return null;
+ }
+
+ @Override
+ public void setRunner(Class<? extends PipelineRunner<?>> kls) {
+
+ }
+
+ @Override
+ public CheckEnabled getStableUniqueNames() {
+ return null;
+ }
+
+ @Override
+ public void setStableUniqueNames(CheckEnabled enabled) {
+ }
+ };
+ }
+ return options;
+ }
+
+ @Override
+ public void output(KV<K, VOUT> output) {
+ throw new UnsupportedOperationException(
+ "output() is not available when processing KeyedWorkItems.");
+ }
+
+ @Override
+ public void outputWithTimestamp(KV<K, VOUT> output, Instant timestamp) {
+ throw new UnsupportedOperationException(
+ "outputWithTimestamp() is not available when processing KeyedWorkItems.");
+ }
+
+ @Override
+ public PaneInfo pane() {
+ throw new UnsupportedOperationException("pane() is not available when processing KeyedWorkItems.");
+ }
+
+ @Override
+ public BoundedWindow window() {
+ throw new UnsupportedOperationException(
+ "window() is not available when processing KeyedWorkItems.");
+ }
+
+ @Override
+ public WindowingInternals<KeyedWorkItem<K, VIN>, KV<K, VOUT>> windowingInternals() {
+ return new WindowingInternals<KeyedWorkItem<K, VIN>, KV<K, VOUT>>() {
+
+ @Override
+ public com.google.cloud.dataflow.sdk.util.state.StateInternals stateInternals() {
+ return stateInternals;
+ }
+
+ @Override
+ public void outputWindowedValue(KV<K, VOUT> output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+ // TODO: No need to represent timestamp twice.
+ collector.setAbsoluteTimestamp(timestamp.getMillis());
+ collector.collect(WindowedValue.of(output, timestamp, windows, pane));
+
+ }
+
+ @Override
+ public TimerInternals timerInternals() {
+ return timerInternals;
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ throw new UnsupportedOperationException("windows() is not available in Streaming mode.");
+ }
+
+ @Override
+ public PaneInfo pane() {
+ throw new UnsupportedOperationException("pane() is not available in Streaming mode.");
+ }
+
+ @Override
+ public <T> void writePCollectionViewData(TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
+ throw new RuntimeException("writePCollectionViewData() not available in Streaming mode.");
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+ throw new RuntimeException("sideInput() is not available in Streaming mode.");
+ }
+ };
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view) {
+ throw new RuntimeException("sideInput() is not supported in Streaming mode.");
+ }
+
+ @Override
+ public <T> void sideOutput(TupleTag<T> tag, T output) {
+ // ignore the side output, this can happen when a user does not register
+ // side outputs but then outputs using a freshly created TupleTag.
+ throw new RuntimeException("sideOutput() is not available when grouping by window.");
+ }
+
+ @Override
+ public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+ sideOutput(tag, output);
+ }
+
+ @Override
+ protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
+ Accumulator acc = getRuntimeContext().getAccumulator(name);
+ if (acc != null) {
+ AccumulatorHelper.compareAccumulatorTypes(name,
+ SerializableFnAggregatorWrapper.class, acc.getClass());
+ return (Aggregator<AggInputT, AggOutputT>) acc;
+ }
+
+ SerializableFnAggregatorWrapper<AggInputT, AggOutputT> accumulator =
+ new SerializableFnAggregatorWrapper<>(combiner);
+ getRuntimeContext().addAccumulator(name, accumulator);
+ return accumulator;
+ }
+ }
+
+ ////////////// Checkpointing implementation ////////////////
+
+ @Override
+ public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception {
+ StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
+ AbstractStateBackend.CheckpointStateOutputView out = getStateBackend().createCheckpointStateOutputView(checkpointId, timestamp);
+ StateCheckpointWriter writer = StateCheckpointWriter.create(out);
+ Coder<K> keyCoder = inputKvCoder.getKeyCoder();
+
+ // checkpoint the timers
+ StateCheckpointUtils.encodeTimers(activeTimers, writer, keyCoder);
+
+ // checkpoint the state
+ StateCheckpointUtils.encodeState(perKeyStateInternals, writer, keyCoder);
+
+ // checkpoint the timerInternals
+ context.timerInternals.encodeTimerInternals(context, writer,
+ inputKvCoder, windowingStrategy.getWindowFn().windowCoder());
+
+ taskState.setOperatorState(out.closeAndGetHandle());
+ return taskState;
+ }
+
+ @Override
+ public void restoreState(StreamTaskState taskState, long recoveryTimestamp) throws Exception {
+ super.restoreState(taskState, recoveryTimestamp);
+
+ final ClassLoader userClassloader = getUserCodeClassloader();
+
+ Coder<? extends BoundedWindow> windowCoder = this.windowingStrategy.getWindowFn().windowCoder();
+ Coder<K> keyCoder = inputKvCoder.getKeyCoder();
+
+ @SuppressWarnings("unchecked")
+ StateHandle<DataInputView> inputState = (StateHandle<DataInputView>) taskState.getOperatorState();
+ DataInputView in = inputState.getState(userClassloader);
+ StateCheckpointReader reader = new StateCheckpointReader(in);
+
+ // restore the timers
+ this.activeTimers = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder);
+
+ // restore the state
+ this.perKeyStateInternals = StateCheckpointUtils.decodeState(
+ reader, windowingStrategy.getOutputTimeFn(), keyCoder, windowCoder, userClassloader);
+
+ // restore the timerInternals.
+ this.timerInternals.restoreTimerInternals(reader, inputKvCoder, windowCoder);
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
new file mode 100644
index 0000000..61953a6
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers.streaming;
+
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.coders.VoidCoder;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.KeyedStream;
+
+/**
+ * This class groups the elements by key. It assumes that already the incoming stream
+ * is composed of <code>[Key,Value]</code> pairs.
+ * */
+public class FlinkGroupByKeyWrapper {
+
+ /**
+ * Just an auxiliary interface to bypass the fact that java anonymous classes cannot implement
+ * multiple interfaces.
+ */
+ private interface KeySelectorWithQueryableResultType<K, V> extends KeySelector<WindowedValue<KV<K, V>>, K>, ResultTypeQueryable<K> {
+ }
+
+ public static <K, V> KeyedStream<WindowedValue<KV<K, V>>, K> groupStreamByKey(DataStream<WindowedValue<KV<K, V>>> inputDataStream, KvCoder<K, V> inputKvCoder) {
+ final Coder<K> keyCoder = inputKvCoder.getKeyCoder();
+ final TypeInformation<K> keyTypeInfo = new CoderTypeInformation<>(keyCoder);
+ final boolean isKeyVoid = keyCoder instanceof VoidCoder;
+
+ return inputDataStream.keyBy(
+ new KeySelectorWithQueryableResultType<K, V>() {
+
+ @Override
+ public K getKey(WindowedValue<KV<K, V>> value) throws Exception {
+ return isKeyVoid ? (K) VoidCoderTypeSerializer.VoidValue.INSTANCE :
+ value.getValue().getKey();
+ }
+
+ @Override
+ public TypeInformation<K> getProducedType() {
+ return keyTypeInfo;
+ }
+ });
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
new file mode 100644
index 0000000..cdf23f6
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers.streaming;
+
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.util.WindowingInternals;
+import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.base.Preconditions;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+
+import java.util.Map;
+
+/**
+ * A wrapper for the {@link com.google.cloud.dataflow.sdk.transforms.ParDo.BoundMulti} Beam transformation.
+ * */
+public class FlinkParDoBoundMultiWrapper<IN, OUT> extends FlinkAbstractParDoWrapper<IN, OUT, RawUnionValue> {
+
+ private final TupleTag<?> mainTag;
+ private final Map<TupleTag<?>, Integer> outputLabels;
+
+ public FlinkParDoBoundMultiWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUT> doFn, TupleTag<?> mainTag, Map<TupleTag<?>, Integer> tagsToLabels) {
+ super(options, windowingStrategy, doFn);
+ this.mainTag = Preconditions.checkNotNull(mainTag);
+ this.outputLabels = Preconditions.checkNotNull(tagsToLabels);
+ }
+
+ @Override
+ public void outputWithTimestampHelper(WindowedValue<IN> inElement, OUT output, Instant timestamp, Collector<WindowedValue<RawUnionValue>> collector) {
+ checkTimestamp(inElement, timestamp);
+ Integer index = outputLabels.get(mainTag);
+ collector.collect(makeWindowedValue(
+ new RawUnionValue(index, output),
+ timestamp,
+ inElement.getWindows(),
+ inElement.getPane()));
+ }
+
+ @Override
+ public <T> void sideOutputWithTimestampHelper(WindowedValue<IN> inElement, T output, Instant timestamp, Collector<WindowedValue<RawUnionValue>> collector, TupleTag<T> tag) {
+ checkTimestamp(inElement, timestamp);
+ Integer index = outputLabels.get(tag);
+ if (index != null) {
+ collector.collect(makeWindowedValue(
+ new RawUnionValue(index, output),
+ timestamp,
+ inElement.getWindows(),
+ inElement.getPane()));
+ }
+ }
+
+ @Override
+ public WindowingInternals<IN, OUT> windowingInternalsHelper(WindowedValue<IN> inElement, Collector<WindowedValue<RawUnionValue>> outCollector) {
+ throw new RuntimeException("FlinkParDoBoundMultiWrapper is just an internal operator serving as " +
+ "an intermediate transformation for the ParDo.BoundMulti translation. windowingInternals() " +
+ "is not available in this class.");
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
new file mode 100644
index 0000000..3357cd5
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
@@ -0,0 +1,98 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers.streaming;
+
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
+import com.google.cloud.dataflow.sdk.util.*;
+import com.google.cloud.dataflow.sdk.util.state.StateInternals;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * A wrapper for the {@link com.google.cloud.dataflow.sdk.transforms.ParDo.Bound} Beam transformation.
+ * */
+public class FlinkParDoBoundWrapper<IN, OUT> extends FlinkAbstractParDoWrapper<IN, OUT, OUT> {
+
+ public FlinkParDoBoundWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUT> doFn) {
+ super(options, windowingStrategy, doFn);
+ }
+
+ @Override
+ public void outputWithTimestampHelper(WindowedValue<IN> inElement, OUT output, Instant timestamp, Collector<WindowedValue<OUT>> collector) {
+ checkTimestamp(inElement, timestamp);
+ collector.collect(makeWindowedValue(
+ output,
+ timestamp,
+ inElement.getWindows(),
+ inElement.getPane()));
+ }
+
+ @Override
+ public <T> void sideOutputWithTimestampHelper(WindowedValue<IN> inElement, T output, Instant timestamp, Collector<WindowedValue<OUT>> outCollector, TupleTag<T> tag) {
+ // ignore the side output, this can happen when a user does not register
+ // side outputs but then outputs using a freshly created TupleTag.
+ throw new RuntimeException("sideOutput() not not available in ParDo.Bound().");
+ }
+
+ @Override
+ public WindowingInternals<IN, OUT> windowingInternalsHelper(final WindowedValue<IN> inElement, final Collector<WindowedValue<OUT>> collector) {
+ return new WindowingInternals<IN, OUT>() {
+ @Override
+ public StateInternals stateInternals() {
+ throw new NullPointerException("StateInternals are not available for ParDo.Bound().");
+ }
+
+ @Override
+ public void outputWindowedValue(OUT output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+ collector.collect(makeWindowedValue(output, timestamp, windows, pane));
+ }
+
+ @Override
+ public TimerInternals timerInternals() {
+ throw new NullPointerException("TimeInternals are not available for ParDo.Bound().");
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ return inElement.getWindows();
+ }
+
+ @Override
+ public PaneInfo pane() {
+ return inElement.getPane();
+ }
+
+ @Override
+ public <T> void writePCollectionViewData(TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
+ throw new RuntimeException("writePCollectionViewData() not supported in Streaming mode.");
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+ throw new RuntimeException("sideInput() not implemented.");
+ }
+ };
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
new file mode 100644
index 0000000..2599e88
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers.streaming.io;
+
+import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+
+import java.io.ByteArrayInputStream;
+import java.util.List;
+
+/**
+ * This flat map function bootstraps from collection elements and turns them into WindowedValues
+ * (as required by the Flink runner).
+ */
+public class FlinkStreamingCreateFunction<IN, OUT> implements FlatMapFunction<IN, WindowedValue<OUT>> {
+
+ private final List<byte[]> elements;
+ private final Coder<OUT> coder;
+
+ public FlinkStreamingCreateFunction(List<byte[]> elements, Coder<OUT> coder) {
+ this.elements = elements;
+ this.coder = coder;
+ }
+
+ @Override
+ public void flatMap(IN value, Collector<WindowedValue<OUT>> out) throws Exception {
+
+ @SuppressWarnings("unchecked")
+ OUT voidValue = (OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE;
+ for (byte[] element : elements) {
+ ByteArrayInputStream bai = new ByteArrayInputStream(element);
+ OUT outValue = coder.decode(bai, Coder.Context.OUTER);
+
+ if (outValue == null) {
+ out.collect(WindowedValue.of(voidValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
+ } else {
+ out.collect(WindowedValue.of(outValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
+ }
+ }
+
+ out.close();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java
new file mode 100644
index 0000000..ddbc993
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers.streaming.io;
+
+import org.apache.beam.runners.flink.FlinkPipelineRunner;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.io.UnboundedSource;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.common.base.Preconditions;
+import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+
+import javax.annotation.Nullable;
+import java.util.List;
+
+/**
+ * A wrapper translating Flink Sources implementing the {@link RichParallelSourceFunction} interface, into
+ * unbounded Beam sources (see {@link UnboundedSource}).
+ * */
+public class UnboundedFlinkSource<T, C extends UnboundedSource.CheckpointMark> extends UnboundedSource<T, C> {
+
+ private final PipelineOptions options;
+ private final RichParallelSourceFunction<T> flinkSource;
+
+ public UnboundedFlinkSource(PipelineOptions pipelineOptions, RichParallelSourceFunction<T> source) {
+ if(!pipelineOptions.getRunner().equals(FlinkPipelineRunner.class)) {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
+ options = Preconditions.checkNotNull(pipelineOptions);
+ flinkSource = Preconditions.checkNotNull(source);
+ validate();
+ }
+
+ public RichParallelSourceFunction<T> getFlinkSource() {
+ return this.flinkSource;
+ }
+
+ @Override
+ public List<? extends UnboundedSource<T, C>> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
+
+ @Override
+ public UnboundedReader<T> createReader(PipelineOptions options, @Nullable C checkpointMark) {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
+
+ @Nullable
+ @Override
+ public Coder<C> getCheckpointMarkCoder() {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
+
+
+ @Override
+ public void validate() {
+ Preconditions.checkNotNull(options);
+ Preconditions.checkNotNull(flinkSource);
+ if(!options.getRunner().equals(FlinkPipelineRunner.class)) {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
+ }
+
+ @Override
+ public Coder<T> getDefaultOutputCoder() {
+ throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
new file mode 100644
index 0000000..a24964a
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java
@@ -0,0 +1,231 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers.streaming.io;
+
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.io.UnboundedSource;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Serializable;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Collections;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+/**
+ * An example unbounded Beam source that reads input from a socket. This is used mainly for testing and debugging.
+ * */
+public class UnboundedSocketSource<C extends UnboundedSource.CheckpointMark> extends UnboundedSource<String, C> {
+
+ private static final Coder<String> DEFAULT_SOCKET_CODER = StringUtf8Coder.of();
+
+ private static final long serialVersionUID = 1L;
+
+ private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 500;
+
+ private static final int CONNECTION_TIMEOUT_TIME = 0;
+
+ private final String hostname;
+ private final int port;
+ private final char delimiter;
+ private final long maxNumRetries;
+ private final long delayBetweenRetries;
+
+ public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries) {
+ this(hostname, port, delimiter, maxNumRetries, DEFAULT_CONNECTION_RETRY_SLEEP);
+ }
+
+ public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries, long delayBetweenRetries) {
+ this.hostname = hostname;
+ this.port = port;
+ this.delimiter = delimiter;
+ this.maxNumRetries = maxNumRetries;
+ this.delayBetweenRetries = delayBetweenRetries;
+ }
+
+ public String getHostname() {
+ return this.hostname;
+ }
+
+ public int getPort() {
+ return this.port;
+ }
+
+ public char getDelimiter() {
+ return this.delimiter;
+ }
+
+ public long getMaxNumRetries() {
+ return this.maxNumRetries;
+ }
+
+ public long getDelayBetweenRetries() {
+ return this.delayBetweenRetries;
+ }
+
+ @Override
+ public List<? extends UnboundedSource<String, C>> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception {
+ return Collections.<UnboundedSource<String, C>>singletonList(this);
+ }
+
+ @Override
+ public UnboundedReader<String> createReader(PipelineOptions options, @Nullable C checkpointMark) {
+ return new UnboundedSocketReader(this);
+ }
+
+ @Nullable
+ @Override
+ public Coder getCheckpointMarkCoder() {
+ // Flink and Dataflow have different checkpointing mechanisms.
+ // In our case we do not need a coder.
+ return null;
+ }
+
+ @Override
+ public void validate() {
+ checkArgument(port > 0 && port < 65536, "port is out of range");
+ checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), or -1 (infinite retries)");
+ checkArgument(delayBetweenRetries >= 0, "delayBetweenRetries must be zero or positive");
+ }
+
+ @Override
+ public Coder getDefaultOutputCoder() {
+ return DEFAULT_SOCKET_CODER;
+ }
+
+ public static class UnboundedSocketReader extends UnboundedSource.UnboundedReader<String> implements Serializable {
+
+ private static final long serialVersionUID = 7526472295622776147L;
+ private static final Logger LOG = LoggerFactory.getLogger(UnboundedSocketReader.class);
+
+ private final UnboundedSocketSource source;
+
+ private Socket socket;
+ private BufferedReader reader;
+
+ private boolean isRunning;
+
+ private String currentRecord;
+
+ public UnboundedSocketReader(UnboundedSocketSource source) {
+ this.source = source;
+ }
+
+ private void openConnection() throws IOException {
+ this.socket = new Socket();
+ this.socket.connect(new InetSocketAddress(this.source.getHostname(), this.source.getPort()), CONNECTION_TIMEOUT_TIME);
+ this.reader = new BufferedReader(new InputStreamReader(this.socket.getInputStream()));
+ this.isRunning = true;
+ }
+
+ @Override
+ public boolean start() throws IOException {
+ int attempt = 0;
+ while (!isRunning) {
+ try {
+ openConnection();
+ LOG.info("Connected to server socket " + this.source.getHostname() + ':' + this.source.getPort());
+
+ return advance();
+ } catch (IOException e) {
+ LOG.info("Lost connection to server socket " + this.source.getHostname() + ':' + this.source.getPort() + ". Retrying in " + this.source.getDelayBetweenRetries() + " msecs...");
+
+ if (this.source.getMaxNumRetries() == -1 || attempt++ < this.source.getMaxNumRetries()) {
+ try {
+ Thread.sleep(this.source.getDelayBetweenRetries());
+ } catch (InterruptedException e1) {
+ e1.printStackTrace();
+ }
+ } else {
+ this.isRunning = false;
+ break;
+ }
+ }
+ }
+ LOG.error("Unable to connect to host " + this.source.getHostname() + " : " + this.source.getPort());
+ return false;
+ }
+
+ @Override
+ public boolean advance() throws IOException {
+ final StringBuilder buffer = new StringBuilder();
+ int data;
+ while (isRunning && (data = reader.read()) != -1) {
+ // check if the string is complete
+ if (data != this.source.getDelimiter()) {
+ buffer.append((char) data);
+ } else {
+ if (buffer.length() > 0 && buffer.charAt(buffer.length() - 1) == '\r') {
+ buffer.setLength(buffer.length() - 1);
+ }
+ this.currentRecord = buffer.toString();
+ buffer.setLength(0);
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public byte[] getCurrentRecordId() throws NoSuchElementException {
+ return new byte[0];
+ }
+
+ @Override
+ public String getCurrent() throws NoSuchElementException {
+ return this.currentRecord;
+ }
+
+ @Override
+ public Instant getCurrentTimestamp() throws NoSuchElementException {
+ return Instant.now();
+ }
+
+ @Override
+ public void close() throws IOException {
+ this.reader.close();
+ this.socket.close();
+ this.isRunning = false;
+ LOG.info("Closed connection to server socket at " + this.source.getHostname() + ":" + this.source.getPort() + ".");
+ }
+
+ @Override
+ public Instant getWatermark() {
+ return Instant.now();
+ }
+
+ @Override
+ public CheckpointMark getCheckpointMark() {
+ return null;
+ }
+
+ @Override
+ public UnboundedSource<String, ?> getCurrentSource() {
+ return this.source;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
new file mode 100644
index 0000000..7c1ccdf
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
@@ -0,0 +1,132 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers.streaming.io;
+
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.io.UnboundedSource;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.operators.Triggerable;
+import org.joda.time.Instant;
+
+/**
+ * A wrapper for Beam's unbounded sources. This class wraps around a source implementing the {@link com.google.cloud.dataflow.sdk.io.Read.Unbounded}
+ * interface.
+ *
+ *</p>
+ * For now we support non-parallel, not checkpointed sources.
+ * */
+public class UnboundedSourceWrapper<T> extends RichSourceFunction<WindowedValue<T>> implements Triggerable {
+
+ private final String name;
+ private final UnboundedSource.UnboundedReader<T> reader;
+
+ private StreamingRuntimeContext runtime = null;
+ private StreamSource.ManualWatermarkContext<WindowedValue<T>> context = null;
+
+ private volatile boolean isRunning = false;
+
+ public UnboundedSourceWrapper(PipelineOptions options, Read.Unbounded<T> transform) {
+ this.name = transform.getName();
+ this.reader = transform.getSource().createReader(options, null);
+ }
+
+ public String getName() {
+ return this.name;
+ }
+
+ WindowedValue<T> makeWindowedValue(T output, Instant timestamp) {
+ if (timestamp == null) {
+ timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+ }
+ return WindowedValue.of(output, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING);
+ }
+
+ @Override
+ public void run(SourceContext<WindowedValue<T>> ctx) throws Exception {
+ if (!(ctx instanceof StreamSource.ManualWatermarkContext)) {
+ throw new RuntimeException("We assume that all sources in Dataflow are EventTimeSourceFunction. " +
+ "Apparently " + this.name + " is not. Probably you should consider writing your own Wrapper for this source.");
+ }
+
+ context = (StreamSource.ManualWatermarkContext<WindowedValue<T>>) ctx;
+ runtime = (StreamingRuntimeContext) getRuntimeContext();
+
+ this.isRunning = true;
+ boolean inputAvailable = reader.start();
+
+ setNextWatermarkTimer(this.runtime);
+
+ while (isRunning) {
+
+ while (!inputAvailable && isRunning) {
+ // wait a bit until we retry to pull more records
+ Thread.sleep(50);
+ inputAvailable = reader.advance();
+ }
+
+ if (inputAvailable) {
+
+ // get it and its timestamp from the source
+ T item = reader.getCurrent();
+ Instant timestamp = reader.getCurrentTimestamp();
+
+ // write it to the output collector
+ synchronized (ctx.getCheckpointLock()) {
+ context.collectWithTimestamp(makeWindowedValue(item, timestamp), timestamp.getMillis());
+ }
+
+ inputAvailable = reader.advance();
+ }
+
+ }
+ }
+
+ @Override
+ public void cancel() {
+ isRunning = false;
+ }
+
+ @Override
+ public void trigger(long timestamp) throws Exception {
+ if (this.isRunning) {
+ synchronized (context.getCheckpointLock()) {
+ long watermarkMillis = this.reader.getWatermark().getMillis();
+ context.emitWatermark(new Watermark(watermarkMillis));
+ }
+ setNextWatermarkTimer(this.runtime);
+ }
+ }
+
+ private void setNextWatermarkTimer(StreamingRuntimeContext runtime) {
+ if (this.isRunning) {
+ long watermarkInterval = runtime.getExecutionConfig().getAutoWatermarkInterval();
+ long timeToNextWatermark = getTimeToNextWaternark(watermarkInterval);
+ runtime.registerTimer(timeToNextWatermark, this);
+ }
+ }
+
+ private long getTimeToNextWaternark(long watermarkInterval) {
+ return System.currentTimeMillis() + watermarkInterval;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
new file mode 100644
index 0000000..7accf09
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
@@ -0,0 +1,126 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers.streaming.state;
+
+import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.util.TimerInternals;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.values.KV;
+import org.joda.time.Instant;
+
+import javax.annotation.Nullable;
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * An implementation of Beam's {@link TimerInternals}, that also provides serialization functionality.
+ * The latter is used when snapshots of the current state are taken, for fault-tolerance.
+ * */
+public abstract class AbstractFlinkTimerInternals<K, VIN> implements TimerInternals, Serializable {
+ private Instant currentInputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE;
+ private Instant currentOutputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE;
+
+ public void setCurrentInputWatermark(Instant watermark) {
+ checkIfValidInputWatermark(watermark);
+ this.currentInputWatermark = watermark;
+ }
+
+ public void setCurrentOutputWatermark(Instant watermark) {
+ checkIfValidOutputWatermark(watermark);
+ this.currentOutputWatermark = watermark;
+ }
+
+ private void setCurrentInputWatermarkAfterRecovery(Instant watermark) {
+ if (!currentInputWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) {
+ throw new RuntimeException("Explicitly setting the input watermark is only allowed on " +
+ "initialization after recovery from a node failure. Apparently this is not " +
+ "the case here as the watermark is already set.");
+ }
+ this.currentInputWatermark = watermark;
+ }
+
+ private void setCurrentOutputWatermarkAfterRecovery(Instant watermark) {
+ if (!currentOutputWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) {
+ throw new RuntimeException("Explicitly setting the output watermark is only allowed on " +
+ "initialization after recovery from a node failure. Apparently this is not " +
+ "the case here as the watermark is already set.");
+ }
+ this.currentOutputWatermark = watermark;
+ }
+
+ @Override
+ public Instant currentProcessingTime() {
+ return Instant.now();
+ }
+
+ @Override
+ public Instant currentInputWatermarkTime() {
+ return currentInputWatermark;
+ }
+
+ @Nullable
+ @Override
+ public Instant currentSynchronizedProcessingTime() {
+ // TODO
+ return null;
+ }
+
+ @Override
+ public Instant currentOutputWatermarkTime() {
+ return currentOutputWatermark;
+ }
+
+ private void checkIfValidInputWatermark(Instant newWatermark) {
+ if (currentInputWatermark.isAfter(newWatermark)) {
+ throw new IllegalArgumentException(String.format(
+ "Cannot set current input watermark to %s. Newer watermarks " +
+ "must be no earlier than the current one (%s).",
+ newWatermark, currentInputWatermark));
+ }
+ }
+
+ private void checkIfValidOutputWatermark(Instant newWatermark) {
+ if (currentOutputWatermark.isAfter(newWatermark)) {
+ throw new IllegalArgumentException(String.format(
+ "Cannot set current output watermark to %s. Newer watermarks " +
+ "must be no earlier than the current one (%s).",
+ newWatermark, currentOutputWatermark));
+ }
+ }
+
+ public void encodeTimerInternals(DoFn.ProcessContext context,
+ StateCheckpointWriter writer,
+ KvCoder<K, VIN> kvCoder,
+ Coder<? extends BoundedWindow> windowCoder) throws IOException {
+ if (context == null) {
+ throw new RuntimeException("The Context has not been initialized.");
+ }
+
+ writer.setTimestamp(currentInputWatermark);
+ writer.setTimestamp(currentOutputWatermark);
+ }
+
+ public void restoreTimerInternals(StateCheckpointReader reader,
+ KvCoder<K, VIN> kvCoder,
+ Coder<? extends BoundedWindow> windowCoder) throws IOException {
+ setCurrentInputWatermarkAfterRecovery(reader.getTimestamp());
+ setCurrentOutputWatermarkAfterRecovery(reader.getTimestamp());
+ }
+}
[19/50] [abbrv] incubator-beam git commit: [flink] adjust directories
according to package name
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java
new file mode 100644
index 0000000..e6c5ae2
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/StateSerializationTest.java
@@ -0,0 +1,303 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.streaming;
+
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.StateCheckpointReader;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.StateCheckpointUtils;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.state.StateCheckpointWriter;
+import com.google.cloud.dataflow.sdk.coders.*;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.CombineWithContext;
+import com.google.cloud.dataflow.sdk.transforms.Sum;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.IntervalWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFns;
+import com.google.cloud.dataflow.sdk.util.TimeDomain;
+import com.google.cloud.dataflow.sdk.util.TimerInternals;
+import com.google.cloud.dataflow.sdk.util.state.*;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.apache.flink.runtime.state.memory.MemoryStateBackend;
+import org.apache.flink.runtime.util.DataInputDeserializer;
+import org.joda.time.Instant;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.*;
+
+import static org.junit.Assert.assertEquals;
+
+public class StateSerializationTest {
+
+ private static final StateNamespace NAMESPACE_1 = StateNamespaces.global();
+ private static final String KEY_PREFIX = "TEST_";
+
+ // TODO: This can be replaced with the standard Sum.SumIntererFn once the state no longer needs
+ // to create a StateTag at the point of restoring state. Currently StateTags are compared strictly
+ // by type and combiners always use KeyedCombineFnWithContext rather than KeyedCombineFn or CombineFn.
+ private static CombineWithContext.KeyedCombineFnWithContext<Object, Integer, int[], Integer> SUM_COMBINER =
+ new CombineWithContext.KeyedCombineFnWithContext<Object, Integer, int[], Integer>() {
+ @Override
+ public int[] createAccumulator(Object key, CombineWithContext.Context c) {
+ return new int[1];
+ }
+
+ @Override
+ public int[] addInput(Object key, int[] accumulator, Integer value, CombineWithContext.Context c) {
+ accumulator[0] += value;
+ return accumulator;
+ }
+
+ @Override
+ public int[] mergeAccumulators(Object key, Iterable<int[]> accumulators, CombineWithContext.Context c) {
+ int[] r = new int[1];
+ for (int[] a : accumulators) {
+ r[0] += a[0];
+ }
+ return r;
+ }
+
+ @Override
+ public Integer extractOutput(Object key, int[] accumulator, CombineWithContext.Context c) {
+ return accumulator[0];
+ }
+ };
+
+ private static Coder<int[]> INT_ACCUM_CODER = DelegateCoder.of(
+ VarIntCoder.of(),
+ new DelegateCoder.CodingFunction<int[], Integer>() {
+ @Override
+ public Integer apply(int[] accumulator) {
+ return accumulator[0];
+ }
+ },
+ new DelegateCoder.CodingFunction<Integer, int[]>() {
+ @Override
+ public int[] apply(Integer value) {
+ int[] a = new int[1];
+ a[0] = value;
+ return a;
+ }
+ });
+
+ private static final StateTag<Object, ValueState<String>> STRING_VALUE_ADDR =
+ StateTags.value("stringValue", StringUtf8Coder.of());
+ private static final StateTag<Object, ValueState<Integer>> INT_VALUE_ADDR =
+ StateTags.value("stringValue", VarIntCoder.of());
+ private static final StateTag<Object, AccumulatorCombiningState<Integer, int[], Integer>> SUM_INTEGER_ADDR =
+ StateTags.keyedCombiningValueWithContext("sumInteger", INT_ACCUM_CODER, SUM_COMBINER);
+ private static final StateTag<Object, BagState<String>> STRING_BAG_ADDR =
+ StateTags.bag("stringBag", StringUtf8Coder.of());
+ private static final StateTag<Object, WatermarkHoldState<BoundedWindow>> WATERMARK_BAG_ADDR =
+ StateTags.watermarkStateInternal("watermark", OutputTimeFns.outputAtEarliestInputTimestamp());
+
+ private Map<String, FlinkStateInternals<String>> statePerKey = new HashMap<>();
+
+ private Map<String, Set<TimerInternals.TimerData>> activeTimers = new HashMap<>();
+
+ private void initializeStateAndTimers() throws CannotProvideCoderException {
+ for (int i = 0; i < 10; i++) {
+ String key = KEY_PREFIX + i;
+
+ FlinkStateInternals state = initializeStateForKey(key);
+ Set<TimerInternals.TimerData> timers = new HashSet<>();
+ for (int j = 0; j < 5; j++) {
+ TimerInternals.TimerData timer = TimerInternals
+ .TimerData.of(NAMESPACE_1,
+ new Instant(1000 + i + j), TimeDomain.values()[j % 3]);
+ timers.add(timer);
+ }
+
+ statePerKey.put(key, state);
+ activeTimers.put(key, timers);
+ }
+ }
+
+ private FlinkStateInternals<String> initializeStateForKey(String key) throws CannotProvideCoderException {
+ FlinkStateInternals<String> state = createState(key);
+
+ ValueState<String> value = state.state(NAMESPACE_1, STRING_VALUE_ADDR);
+ value.write("test");
+
+ ValueState<Integer> value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR);
+ value2.write(4);
+ value2.write(5);
+
+ AccumulatorCombiningState<Integer, int[], Integer> combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+ combiningValue.add(1);
+ combiningValue.add(2);
+
+ WatermarkHoldState<BoundedWindow> watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
+ watermark.add(new Instant(1000));
+
+ BagState<String> bag = state.state(NAMESPACE_1, STRING_BAG_ADDR);
+ bag.add("v1");
+ bag.add("v2");
+ bag.add("v3");
+ bag.add("v4");
+ return state;
+ }
+
+ private boolean restoreAndTestState(DataInputView in) throws Exception {
+ StateCheckpointReader reader = new StateCheckpointReader(in);
+ final ClassLoader userClassloader = this.getClass().getClassLoader();
+ Coder<? extends BoundedWindow> windowCoder = IntervalWindow.getCoder();
+ Coder<String> keyCoder = StringUtf8Coder.of();
+
+ boolean comparisonRes = true;
+
+ for (String key : statePerKey.keySet()) {
+ comparisonRes &= checkStateForKey(key);
+ }
+
+ // restore the timers
+ Map<String, Set<TimerInternals.TimerData>> restoredTimersPerKey = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder);
+ if (activeTimers.size() != restoredTimersPerKey.size()) {
+ return false;
+ }
+
+ for (String key : statePerKey.keySet()) {
+ Set<TimerInternals.TimerData> originalTimers = activeTimers.get(key);
+ Set<TimerInternals.TimerData> restoredTimers = restoredTimersPerKey.get(key);
+ comparisonRes &= checkTimersForKey(originalTimers, restoredTimers);
+ }
+
+ // restore the state
+ Map<String, FlinkStateInternals<String>> restoredPerKeyState =
+ StateCheckpointUtils.decodeState(reader, OutputTimeFns.outputAtEarliestInputTimestamp(), keyCoder, windowCoder, userClassloader);
+ if (restoredPerKeyState.size() != statePerKey.size()) {
+ return false;
+ }
+
+ for (String key : statePerKey.keySet()) {
+ FlinkStateInternals<String> originalState = statePerKey.get(key);
+ FlinkStateInternals<String> restoredState = restoredPerKeyState.get(key);
+ comparisonRes &= checkStateForKey(originalState, restoredState);
+ }
+ return comparisonRes;
+ }
+
+ private boolean checkStateForKey(String key) throws CannotProvideCoderException {
+ FlinkStateInternals<String> state = statePerKey.get(key);
+
+ ValueState<String> value = state.state(NAMESPACE_1, STRING_VALUE_ADDR);
+ boolean comp = value.read().equals("test");
+
+ ValueState<Integer> value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR);
+ comp &= value2.read().equals(5);
+
+ AccumulatorCombiningState<Integer, int[], Integer> combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+ comp &= combiningValue.read().equals(3);
+
+ WatermarkHoldState<BoundedWindow> watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
+ comp &= watermark.read().equals(new Instant(1000));
+
+ BagState<String> bag = state.state(NAMESPACE_1, STRING_BAG_ADDR);
+ Iterator<String> it = bag.read().iterator();
+ int i = 0;
+ while (it.hasNext()) {
+ comp &= it.next().equals("v" + (++i));
+ }
+ return comp;
+ }
+
+ private void storeState(AbstractStateBackend.CheckpointStateOutputView out) throws Exception {
+ StateCheckpointWriter checkpointBuilder = StateCheckpointWriter.create(out);
+ Coder<String> keyCoder = StringUtf8Coder.of();
+
+ // checkpoint the timers
+ StateCheckpointUtils.encodeTimers(activeTimers, checkpointBuilder, keyCoder);
+
+ // checkpoint the state
+ StateCheckpointUtils.encodeState(statePerKey, checkpointBuilder, keyCoder);
+ }
+
+ private boolean checkTimersForKey(Set<TimerInternals.TimerData> originalTimers, Set<TimerInternals.TimerData> restoredTimers) {
+ boolean comp = true;
+ if (restoredTimers == null) {
+ return false;
+ }
+
+ if (originalTimers.size() != restoredTimers.size()) {
+ return false;
+ }
+
+ for (TimerInternals.TimerData timer : originalTimers) {
+ comp &= restoredTimers.contains(timer);
+ }
+ return comp;
+ }
+
+ private boolean checkStateForKey(FlinkStateInternals<String> originalState, FlinkStateInternals<String> restoredState) throws CannotProvideCoderException {
+ if (restoredState == null) {
+ return false;
+ }
+
+ ValueState<String> orValue = originalState.state(NAMESPACE_1, STRING_VALUE_ADDR);
+ ValueState<String> resValue = restoredState.state(NAMESPACE_1, STRING_VALUE_ADDR);
+ boolean comp = orValue.read().equals(resValue.read());
+
+ ValueState<Integer> orIntValue = originalState.state(NAMESPACE_1, INT_VALUE_ADDR);
+ ValueState<Integer> resIntValue = restoredState.state(NAMESPACE_1, INT_VALUE_ADDR);
+ comp &= orIntValue.read().equals(resIntValue.read());
+
+ AccumulatorCombiningState<Integer, int[], Integer> combOrValue = originalState.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+ AccumulatorCombiningState<Integer, int[], Integer> combResValue = restoredState.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+ comp &= combOrValue.read().equals(combResValue.read());
+
+ WatermarkHoldState orWatermark = originalState.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
+ WatermarkHoldState resWatermark = restoredState.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
+ comp &= orWatermark.read().equals(resWatermark.read());
+
+ BagState<String> orBag = originalState.state(NAMESPACE_1, STRING_BAG_ADDR);
+ BagState<String> resBag = restoredState.state(NAMESPACE_1, STRING_BAG_ADDR);
+
+ Iterator<String> orIt = orBag.read().iterator();
+ Iterator<String> resIt = resBag.read().iterator();
+
+ while (orIt.hasNext() && resIt.hasNext()) {
+ comp &= orIt.next().equals(resIt.next());
+ }
+
+ return !((orIt.hasNext() && !resIt.hasNext()) || (!orIt.hasNext() && resIt.hasNext())) && comp;
+ }
+
+ private FlinkStateInternals<String> createState(String key) throws CannotProvideCoderException {
+ return new FlinkStateInternals<>(
+ key,
+ StringUtf8Coder.of(),
+ IntervalWindow.getCoder(),
+ OutputTimeFns.outputAtEarliestInputTimestamp());
+ }
+
+ @Test
+ public void test() throws Exception {
+ StateSerializationTest test = new StateSerializationTest();
+ test.initializeStateAndTimers();
+
+ MemoryStateBackend.MemoryCheckpointOutputStream memBackend = new MemoryStateBackend.MemoryCheckpointOutputStream(32048);
+ AbstractStateBackend.CheckpointStateOutputView out = new AbstractStateBackend.CheckpointStateOutputView(memBackend);
+
+ test.storeState(out);
+
+ byte[] contents = memBackend.closeAndGetBytes();
+ DataInputView in = new DataInputDeserializer(contents, 0, contents.length);
+
+ assertEquals(test.restoreAndTestState(in), true);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
new file mode 100644
index 0000000..f0b93a0
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsITCase.java
@@ -0,0 +1,132 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.streaming;
+
+import org.apache.beam.runners.flink.FlinkTestPipeline;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Count;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Joiner;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.io.Serializable;
+import java.util.Arrays;
+
+
+/**
+ * Session window test
+ */
+public class TopWikipediaSessionsITCase extends StreamingProgramTestBase implements Serializable {
+ protected String resultPath;
+
+ public TopWikipediaSessionsITCase(){
+ }
+
+ static final String[] EXPECTED_RESULT = new String[] {
+ "user: user1 value:3",
+ "user: user1 value:1",
+ "user: user2 value:4",
+ "user: user2 value:6",
+ "user: user3 value:7",
+ "user: user3 value:2"
+ };
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+
+ Pipeline p = FlinkTestPipeline.createForStreaming();
+
+ Long now = (System.currentTimeMillis() + 10000) / 1000;
+
+ PCollection<KV<String, Long>> output =
+ p.apply(Create.of(Arrays.asList(new TableRow().set("timestamp", now).set
+ ("contributor_username", "user1"), new TableRow().set("timestamp", now + 10).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now).set
+ ("contributor_username", "user1"), new TableRow().set("timestamp", now + 2).set
+ ("contributor_username", "user1"), new TableRow().set("timestamp", now).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 1).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 5).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 7).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 8).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 200).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 230).set
+ ("contributor_username", "user1"), new TableRow().set("timestamp", now + 230).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 240).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 245).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 235).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 236).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 237).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 238).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 239).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 240).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 241).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now)
+ .set("contributor_username", "user3"))))
+
+
+
+ .apply(ParDo.of(new DoFn<TableRow, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ TableRow row = c.element();
+ long timestamp = (Integer) row.get("timestamp");
+ String userName = (String) row.get("contributor_username");
+ if (userName != null) {
+ // Sets the timestamp field to be used in windowing.
+ c.outputWithTimestamp(userName, new Instant(timestamp * 1000L));
+ }
+ }
+ }))
+
+ .apply(Window.<String>into(Sessions.withGapDuration(Duration.standardMinutes(1))))
+
+ .apply(Count.<String>perElement());
+
+ PCollection<String> format = output.apply(ParDo.of(new DoFn<KV<String, Long>, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ KV<String, Long> el = c.element();
+ String out = "user: " + el.getKey() + " value:" + el.getValue();
+ c.output(out);
+ }
+ }));
+
+ format.apply(TextIO.Write.to(resultPath));
+
+ p.run();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java
new file mode 100644
index 0000000..620dace
--- /dev/null
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/util/JoinExamples.java
@@ -0,0 +1,158 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.util;
+
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.BigQueryIO;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.options.Validation;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+
+/**
+ * Copied from {@link com.google.cloud.dataflow.examples.JoinExamples} because the code
+ * is private there.
+ */
+public class JoinExamples {
+
+ // A 1000-row sample of the GDELT data here: gdelt-bq:full.events.
+ private static final String GDELT_EVENTS_TABLE =
+ "clouddataflow-readonly:samples.gdelt_sample";
+ // A table that maps country codes to country names.
+ private static final String COUNTRY_CODES =
+ "gdelt-bq:full.crosswalk_geocountrycodetohuman";
+
+ /**
+ * Join two collections, using country code as the key.
+ */
+ public static PCollection<String> joinEvents(PCollection<TableRow> eventsTable,
+ PCollection<TableRow> countryCodes) throws Exception {
+
+ final TupleTag<String> eventInfoTag = new TupleTag<>();
+ final TupleTag<String> countryInfoTag = new TupleTag<>();
+
+ // transform both input collections to tuple collections, where the keys are country
+ // codes in both cases.
+ PCollection<KV<String, String>> eventInfo = eventsTable.apply(
+ ParDo.of(new ExtractEventDataFn()));
+ PCollection<KV<String, String>> countryInfo = countryCodes.apply(
+ ParDo.of(new ExtractCountryInfoFn()));
+
+ // country code 'key' -> CGBKR (<event info>, <country name>)
+ PCollection<KV<String, CoGbkResult>> kvpCollection = KeyedPCollectionTuple
+ .of(eventInfoTag, eventInfo)
+ .and(countryInfoTag, countryInfo)
+ .apply(CoGroupByKey.<String>create());
+
+ // Process the CoGbkResult elements generated by the CoGroupByKey transform.
+ // country code 'key' -> string of <event info>, <country name>
+ PCollection<KV<String, String>> finalResultCollection =
+ kvpCollection.apply(ParDo.of(new DoFn<KV<String, CoGbkResult>, KV<String, String>>() {
+ @Override
+ public void processElement(ProcessContext c) {
+ KV<String, CoGbkResult> e = c.element();
+ CoGbkResult val = e.getValue();
+ String countryCode = e.getKey();
+ String countryName;
+ countryName = e.getValue().getOnly(countryInfoTag, "Kostas");
+ for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) {
+ // Generate a string that combines information from both collection values
+ c.output(KV.of(countryCode, "Country name: " + countryName
+ + ", Event info: " + eventInfo));
+ }
+ }
+ }));
+
+ // write to GCS
+ return finalResultCollection
+ .apply(ParDo.of(new DoFn<KV<String, String>, String>() {
+ @Override
+ public void processElement(ProcessContext c) {
+ String outputstring = "Country code: " + c.element().getKey()
+ + ", " + c.element().getValue();
+ c.output(outputstring);
+ }
+ }));
+ }
+
+ /**
+ * Examines each row (event) in the input table. Output a KV with the key the country
+ * code of the event, and the value a string encoding event information.
+ */
+ static class ExtractEventDataFn extends DoFn<TableRow, KV<String, String>> {
+ @Override
+ public void processElement(ProcessContext c) {
+ TableRow row = c.element();
+ String countryCode = (String) row.get("ActionGeo_CountryCode");
+ String sqlDate = (String) row.get("SQLDATE");
+ String actor1Name = (String) row.get("Actor1Name");
+ String sourceUrl = (String) row.get("SOURCEURL");
+ String eventInfo = "Date: " + sqlDate + ", Actor1: " + actor1Name + ", url: " + sourceUrl;
+ c.output(KV.of(countryCode, eventInfo));
+ }
+ }
+
+
+ /**
+ * Examines each row (country info) in the input table. Output a KV with the key the country
+ * code, and the value the country name.
+ */
+ static class ExtractCountryInfoFn extends DoFn<TableRow, KV<String, String>> {
+ @Override
+ public void processElement(ProcessContext c) {
+ TableRow row = c.element();
+ String countryCode = (String) row.get("FIPSCC");
+ String countryName = (String) row.get("HumanName");
+ c.output(KV.of(countryCode, countryName));
+ }
+ }
+
+
+ /**
+ * Options supported by {@link JoinExamples}.
+ * <p>
+ * Inherits standard configuration options.
+ */
+ private interface Options extends PipelineOptions {
+ @Description("Path of the file to write to")
+ @Validation.Required
+ String getOutput();
+ void setOutput(String value);
+ }
+
+ public static void main(String[] args) throws Exception {
+ Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+ Pipeline p = Pipeline.create(options);
+ // the following two 'applys' create multiple inputs to our pipeline, one for each
+ // of our two input sources.
+ PCollection<TableRow> eventsTable = p.apply(BigQueryIO.Read.from(GDELT_EVENTS_TABLE));
+ PCollection<TableRow> countryCodes = p.apply(BigQueryIO.Read.from(COUNTRY_CODES));
+ PCollection<String> formattedResults = joinEvents(eventsTable, countryCodes);
+ formattedResults.apply(TextIO.Write.to(options.getOutput()));
+ p.run();
+ }
+
+}
[28/50] [abbrv] incubator-beam git commit: [flink] adjust directories
according to package name
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java
deleted file mode 100644
index d0423b9..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.cloud.dataflow.sdk.io.Sink;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.common.base.Preconditions;
-import com.google.cloud.dataflow.sdk.transforms.Write;
-import org.apache.flink.api.common.io.OutputFormat;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.util.AbstractID;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.lang.reflect.Field;
-
-/**
- * Wrapper class to use generic Write.Bound transforms as sinks.
- * @param <T> The type of the incoming records.
- */
-public class SinkOutputFormat<T> implements OutputFormat<T> {
-
- private final Sink<T> sink;
-
- private transient PipelineOptions pipelineOptions;
-
- private Sink.WriteOperation<T, ?> writeOperation;
- private Sink.Writer<T, ?> writer;
-
- private AbstractID uid = new AbstractID();
-
- public SinkOutputFormat(Write.Bound<T> transform, PipelineOptions pipelineOptions) {
- this.sink = extractSink(transform);
- this.pipelineOptions = Preconditions.checkNotNull(pipelineOptions);
- }
-
- private Sink<T> extractSink(Write.Bound<T> transform) {
- // TODO possibly add a getter in the upstream
- try {
- Field sinkField = transform.getClass().getDeclaredField("sink");
- sinkField.setAccessible(true);
- @SuppressWarnings("unchecked")
- Sink<T> extractedSink = (Sink<T>) sinkField.get(transform);
- return extractedSink;
- } catch (NoSuchFieldException | IllegalAccessException e) {
- throw new RuntimeException("Could not acquire custom sink field.", e);
- }
- }
-
- @Override
- public void configure(Configuration configuration) {
- writeOperation = sink.createWriteOperation(pipelineOptions);
- try {
- writeOperation.initialize(pipelineOptions);
- } catch (Exception e) {
- throw new RuntimeException("Failed to initialize the write operation.", e);
- }
- }
-
- @Override
- public void open(int taskNumber, int numTasks) throws IOException {
- try {
- writer = writeOperation.createWriter(pipelineOptions);
- } catch (Exception e) {
- throw new IOException("Couldn't create writer.", e);
- }
- try {
- writer.open(uid + "-" + String.valueOf(taskNumber));
- } catch (Exception e) {
- throw new IOException("Couldn't open writer.", e);
- }
- }
-
- @Override
- public void writeRecord(T record) throws IOException {
- try {
- writer.write(record);
- } catch (Exception e) {
- throw new IOException("Couldn't write record.", e);
- }
- }
-
- @Override
- public void close() throws IOException {
- try {
- writer.close();
- } catch (Exception e) {
- throw new IOException("Couldn't close writer.", e);
- }
- }
-
- private void writeObject(ObjectOutputStream out) throws IOException, ClassNotFoundException {
- out.defaultWriteObject();
- ObjectMapper mapper = new ObjectMapper();
- mapper.writeValue(out, pipelineOptions);
- }
-
- private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- ObjectMapper mapper = new ObjectMapper();
- pipelineOptions = mapper.readValue(in, PipelineOptions.class);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
deleted file mode 100644
index 2d62416..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Source;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.api.common.io.statistics.BaseStatistics;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.io.InputSplit;
-import org.apache.flink.core.io.InputSplitAssigner;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * A Flink {@link org.apache.flink.api.common.io.InputFormat} that wraps a
- * Dataflow {@link com.google.cloud.dataflow.sdk.io.Source}.
- */
-public class SourceInputFormat<T> implements InputFormat<T, SourceInputSplit<T>> {
- private static final Logger LOG = LoggerFactory.getLogger(SourceInputFormat.class);
-
- private final BoundedSource<T> initialSource;
- private transient PipelineOptions options;
-
- private BoundedSource.BoundedReader<T> reader = null;
- private boolean reachedEnd = true;
-
- public SourceInputFormat(BoundedSource<T> initialSource, PipelineOptions options) {
- this.initialSource = initialSource;
- this.options = options;
- }
-
- private void writeObject(ObjectOutputStream out)
- throws IOException, ClassNotFoundException {
- out.defaultWriteObject();
- ObjectMapper mapper = new ObjectMapper();
- mapper.writeValue(out, options);
- }
-
- private void readObject(ObjectInputStream in)
- throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- ObjectMapper mapper = new ObjectMapper();
- options = mapper.readValue(in, PipelineOptions.class);
- }
-
- @Override
- public void configure(Configuration configuration) {}
-
- @Override
- public void open(SourceInputSplit<T> sourceInputSplit) throws IOException {
- reader = ((BoundedSource<T>) sourceInputSplit.getSource()).createReader(options);
- reachedEnd = false;
- }
-
- @Override
- public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOException {
- try {
- final long estimatedSize = initialSource.getEstimatedSizeBytes(options);
-
- return new BaseStatistics() {
- @Override
- public long getTotalInputSize() {
- return estimatedSize;
-
- }
-
- @Override
- public long getNumberOfRecords() {
- return BaseStatistics.NUM_RECORDS_UNKNOWN;
- }
-
- @Override
- public float getAverageRecordWidth() {
- return BaseStatistics.AVG_RECORD_BYTES_UNKNOWN;
- }
- };
- } catch (Exception e) {
- LOG.warn("Could not read Source statistics: {}", e);
- }
-
- return null;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public SourceInputSplit<T>[] createInputSplits(int numSplits) throws IOException {
- long desiredSizeBytes;
- try {
- desiredSizeBytes = initialSource.getEstimatedSizeBytes(options) / numSplits;
- List<? extends Source<T>> shards = initialSource.splitIntoBundles(desiredSizeBytes,
- options);
- List<SourceInputSplit<T>> splits = new ArrayList<>();
- int splitCount = 0;
- for (Source<T> shard: shards) {
- splits.add(new SourceInputSplit<>(shard, splitCount++));
- }
- return splits.toArray(new SourceInputSplit[splits.size()]);
- } catch (Exception e) {
- throw new IOException("Could not create input splits from Source.", e);
- }
- }
-
- @Override
- public InputSplitAssigner getInputSplitAssigner(final SourceInputSplit[] sourceInputSplits) {
- return new InputSplitAssigner() {
- private int index = 0;
- private final SourceInputSplit[] splits = sourceInputSplits;
- @Override
- public InputSplit getNextInputSplit(String host, int taskId) {
- if (index < splits.length) {
- return splits[index++];
- } else {
- return null;
- }
- }
- };
- }
-
-
- @Override
- public boolean reachedEnd() throws IOException {
- return reachedEnd;
- }
-
- @Override
- public T nextRecord(T t) throws IOException {
-
- reachedEnd = !reader.advance();
- if (!reachedEnd) {
- return reader.getCurrent();
- }
- return null;
- }
-
- @Override
- public void close() throws IOException {
- reader.close();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java
deleted file mode 100644
index 1b45ad7..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers;
-
-import com.google.cloud.dataflow.sdk.io.Source;
-import org.apache.flink.core.io.InputSplit;
-
-/**
- * {@link org.apache.flink.core.io.InputSplit} for
- * {@link org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat}. We pass
- * the sharded Source around in the input split because Sources simply split up into several
- * Sources for sharding. This is different to how Flink creates a separate InputSplit from
- * an InputFormat.
- */
-public class SourceInputSplit<T> implements InputSplit {
-
- private Source<T> source;
- private int splitNumber;
-
- public SourceInputSplit() {
- }
-
- public SourceInputSplit(Source<T> source, int splitNumber) {
- this.source = source;
- this.splitNumber = splitNumber;
- }
-
- @Override
- public int getSplitNumber() {
- return splitNumber;
- }
-
- public Source<T> getSource() {
- return source;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
deleted file mode 100644
index e2ceae6..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
+++ /dev/null
@@ -1,264 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers.streaming;
-
-import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.common.base.Preconditions;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
-import com.google.cloud.dataflow.sdk.util.*;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.base.Throwables;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.accumulators.AccumulatorHelper;
-import org.apache.flink.api.common.functions.RichFlatMapFunction;
-import org.apache.flink.util.Collector;
-import org.joda.time.Instant;
-import org.joda.time.format.PeriodFormat;
-
-import java.util.Collection;
-
-/**
- * An abstract class that encapsulates the common code of the the {@link com.google.cloud.dataflow.sdk.transforms.ParDo.Bound}
- * and {@link com.google.cloud.dataflow.sdk.transforms.ParDo.BoundMulti} wrappers. See the {@link FlinkParDoBoundWrapper} and
- * {@link FlinkParDoBoundMultiWrapper} for the actual wrappers of the aforementioned transformations.
- * */
-public abstract class FlinkAbstractParDoWrapper<IN, OUTDF, OUTFL> extends RichFlatMapFunction<WindowedValue<IN>, WindowedValue<OUTFL>> {
-
- private final DoFn<IN, OUTDF> doFn;
- private final WindowingStrategy<?, ?> windowingStrategy;
- private transient PipelineOptions options;
-
- private DoFnProcessContext context;
-
- public FlinkAbstractParDoWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUTDF> doFn) {
- Preconditions.checkNotNull(options);
- Preconditions.checkNotNull(windowingStrategy);
- Preconditions.checkNotNull(doFn);
-
- this.doFn = doFn;
- this.options = options;
- this.windowingStrategy = windowingStrategy;
- }
-
- private void initContext(DoFn<IN, OUTDF> function, Collector<WindowedValue<OUTFL>> outCollector) {
- if (this.context == null) {
- this.context = new DoFnProcessContext(function, outCollector);
- }
- }
-
- @Override
- public void flatMap(WindowedValue<IN> value, Collector<WindowedValue<OUTFL>> out) throws Exception {
- this.initContext(doFn, out);
-
- // for each window the element belongs to, create a new copy here.
- Collection<? extends BoundedWindow> windows = value.getWindows();
- if (windows.size() <= 1) {
- processElement(value);
- } else {
- for (BoundedWindow window : windows) {
- processElement(WindowedValue.of(
- value.getValue(), value.getTimestamp(), window, value.getPane()));
- }
- }
- }
-
- private void processElement(WindowedValue<IN> value) throws Exception {
- this.context.setElement(value);
- this.doFn.startBundle(context);
- doFn.processElement(context);
- this.doFn.finishBundle(context);
- }
-
- private class DoFnProcessContext extends DoFn<IN, OUTDF>.ProcessContext {
-
- private final DoFn<IN, OUTDF> fn;
-
- protected final Collector<WindowedValue<OUTFL>> collector;
-
- private WindowedValue<IN> element;
-
- private DoFnProcessContext(DoFn<IN, OUTDF> function, Collector<WindowedValue<OUTFL>> outCollector) {
- function.super();
- super.setupDelegateAggregators();
-
- this.fn = function;
- this.collector = outCollector;
- }
-
- public void setElement(WindowedValue<IN> value) {
- this.element = value;
- }
-
- @Override
- public IN element() {
- return this.element.getValue();
- }
-
- @Override
- public Instant timestamp() {
- return this.element.getTimestamp();
- }
-
- @Override
- public BoundedWindow window() {
- if (!(fn instanceof DoFn.RequiresWindowAccess)) {
- throw new UnsupportedOperationException(
- "window() is only available in the context of a DoFn marked as RequiresWindow.");
- }
-
- Collection<? extends BoundedWindow> windows = this.element.getWindows();
- if (windows.size() != 1) {
- throw new IllegalArgumentException("Each element is expected to belong to 1 window. " +
- "This belongs to " + windows.size() + ".");
- }
- return windows.iterator().next();
- }
-
- @Override
- public PaneInfo pane() {
- return this.element.getPane();
- }
-
- @Override
- public WindowingInternals<IN, OUTDF> windowingInternals() {
- return windowingInternalsHelper(element, collector);
- }
-
- @Override
- public PipelineOptions getPipelineOptions() {
- return options;
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view) {
- throw new RuntimeException("sideInput() is not supported in Streaming mode.");
- }
-
- @Override
- public void output(OUTDF output) {
- outputWithTimestamp(output, this.element.getTimestamp());
- }
-
- @Override
- public void outputWithTimestamp(OUTDF output, Instant timestamp) {
- outputWithTimestampHelper(element, output, timestamp, collector);
- }
-
- @Override
- public <T> void sideOutput(TupleTag<T> tag, T output) {
- sideOutputWithTimestamp(tag, output, this.element.getTimestamp());
- }
-
- @Override
- public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
- sideOutputWithTimestampHelper(element, output, timestamp, collector, tag);
- }
-
- @Override
- protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
- Accumulator acc = getRuntimeContext().getAccumulator(name);
- if (acc != null) {
- AccumulatorHelper.compareAccumulatorTypes(name,
- SerializableFnAggregatorWrapper.class, acc.getClass());
- return (Aggregator<AggInputT, AggOutputT>) acc;
- }
-
- SerializableFnAggregatorWrapper<AggInputT, AggOutputT> accumulator =
- new SerializableFnAggregatorWrapper<>(combiner);
- getRuntimeContext().addAccumulator(name, accumulator);
- return accumulator;
- }
- }
-
- protected void checkTimestamp(WindowedValue<IN> ref, Instant timestamp) {
- if (timestamp.isBefore(ref.getTimestamp().minus(doFn.getAllowedTimestampSkew()))) {
- throw new IllegalArgumentException(String.format(
- "Cannot output with timestamp %s. Output timestamps must be no earlier than the "
- + "timestamp of the current input (%s) minus the allowed skew (%s). See the "
- + "DoFn#getAllowedTimestmapSkew() Javadoc for details on changing the allowed skew.",
- timestamp, ref.getTimestamp(),
- PeriodFormat.getDefault().print(doFn.getAllowedTimestampSkew().toPeriod())));
- }
- }
-
- protected <T> WindowedValue<T> makeWindowedValue(
- T output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
- final Instant inputTimestamp = timestamp;
- final WindowFn windowFn = windowingStrategy.getWindowFn();
-
- if (timestamp == null) {
- timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
- }
-
- if (windows == null) {
- try {
- windows = windowFn.assignWindows(windowFn.new AssignContext() {
- @Override
- public Object element() {
- throw new UnsupportedOperationException(
- "WindowFn attempted to access input element when none was available");
- }
-
- @Override
- public Instant timestamp() {
- if (inputTimestamp == null) {
- throw new UnsupportedOperationException(
- "WindowFn attempted to access input timestamp when none was available");
- }
- return inputTimestamp;
- }
-
- @Override
- public Collection<? extends BoundedWindow> windows() {
- throw new UnsupportedOperationException(
- "WindowFn attempted to access input windows when none were available");
- }
- });
- } catch (Exception e) {
- throw UserCodeException.wrap(e);
- }
- }
-
- return WindowedValue.of(output, timestamp, windows, pane);
- }
-
- /////////// ABSTRACT METHODS TO BE IMPLEMENTED BY SUBCLASSES /////////////////
-
- public abstract void outputWithTimestampHelper(
- WindowedValue<IN> inElement,
- OUTDF output,
- Instant timestamp,
- Collector<WindowedValue<OUTFL>> outCollector);
-
- public abstract <T> void sideOutputWithTimestampHelper(
- WindowedValue<IN> inElement,
- T output,
- Instant timestamp,
- Collector<WindowedValue<OUTFL>> outCollector,
- TupleTag<T> tag);
-
- public abstract WindowingInternals<IN, OUTDF> windowingInternalsHelper(
- WindowedValue<IN> inElement,
- Collector<WindowedValue<OUTFL>> outCollector);
-
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
deleted file mode 100644
index 906a399..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
+++ /dev/null
@@ -1,629 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers.streaming;
-
-import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
-import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.state.*;
-import com.google.cloud.dataflow.sdk.coders.*;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.*;
-import com.google.cloud.dataflow.sdk.values.*;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.HashMultimap;
-import com.google.common.collect.Multimap;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.accumulators.AccumulatorHelper;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.runtime.state.AbstractStateBackend;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.KeyedStream;
-import org.apache.flink.streaming.api.operators.*;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
-import org.joda.time.Instant;
-
-import java.io.IOException;
-import java.util.*;
-
-/**
- * This class is the key class implementing all the windowing/triggering logic of Apache Beam.
- * To provide full compatibility and support for all the windowing/triggering combinations offered by
- * Beam, we opted for a strategy that uses the SDK's code for doing these operations. See the code in
- * ({@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn}.
- * <p/>
- * In a nutshell, when the execution arrives to this operator, we expect to have a stream <b>already
- * grouped by key</b>. Each of the elements that enter here, registers a timer
- * (see {@link TimerInternals#setTimer(TimerInternals.TimerData)} in the
- * {@link FlinkGroupAlsoByWindowWrapper#activeTimers}.
- * This is essentially a timestamp indicating when to trigger the computation over the window this
- * element belongs to.
- * <p/>
- * When a watermark arrives, all the registered timers are checked to see which ones are ready to
- * fire (see {@link FlinkGroupAlsoByWindowWrapper#processWatermark(Watermark)}). These are deregistered from
- * the {@link FlinkGroupAlsoByWindowWrapper#activeTimers}
- * list, and are fed into the {@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn}
- * for furhter processing.
- */
-public class FlinkGroupAlsoByWindowWrapper<K, VIN, VACC, VOUT>
- extends AbstractStreamOperator<WindowedValue<KV<K, VOUT>>>
- implements OneInputStreamOperator<WindowedValue<KV<K, VIN>>, WindowedValue<KV<K, VOUT>>> {
-
- private static final long serialVersionUID = 1L;
-
- private transient PipelineOptions options;
-
- private transient CoderRegistry coderRegistry;
-
- private DoFn<KeyedWorkItem<K, VIN>, KV<K, VOUT>> operator;
-
- private ProcessContext context;
-
- private final WindowingStrategy<KV<K, VIN>, BoundedWindow> windowingStrategy;
-
- private final Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combineFn;
-
- private final KvCoder<K, VIN> inputKvCoder;
-
- /**
- * State is kept <b>per-key</b>. This data structure keeps this mapping between an active key, i.e. a
- * key whose elements are currently waiting to be processed, and its associated state.
- */
- private Map<K, FlinkStateInternals<K>> perKeyStateInternals = new HashMap<>();
-
- /**
- * Timers waiting to be processed.
- */
- private Map<K, Set<TimerInternals.TimerData>> activeTimers = new HashMap<>();
-
- private FlinkTimerInternals timerInternals = new FlinkTimerInternals();
-
- /**
- * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy.
- * This method assumes that <b>elements are already grouped by key</b>.
- * <p/>
- * The difference with {@link #createForIterable(PipelineOptions, PCollection, KeyedStream)}
- * is that this method assumes that a combiner function is provided
- * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}).
- * A combiner helps at increasing the speed and, in most of the cases, reduce the per-window state.
- *
- * @param options the general job configuration options.
- * @param input the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}.
- * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key.
- * @param combiner the combiner to be used.
- * @param outputKvCoder the type of the output values.
- */
- public static <K, VIN, VACC, VOUT> DataStream<WindowedValue<KV<K, VOUT>>> create(
- PipelineOptions options,
- PCollection input,
- KeyedStream<WindowedValue<KV<K, VIN>>, K> groupedStreamByKey,
- Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner,
- KvCoder<K, VOUT> outputKvCoder) {
- Preconditions.checkNotNull(options);
-
- KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) input.getCoder();
- FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper<>(options,
- input.getPipeline().getCoderRegistry(), input.getWindowingStrategy(), inputKvCoder, combiner);
-
- Coder<WindowedValue<KV<K, VOUT>>> windowedOutputElemCoder = WindowedValue.FullWindowedValueCoder.of(
- outputKvCoder,
- input.getWindowingStrategy().getWindowFn().windowCoder());
-
- CoderTypeInformation<WindowedValue<KV<K, VOUT>>> outputTypeInfo =
- new CoderTypeInformation<>(windowedOutputElemCoder);
-
- DataStream<WindowedValue<KV<K, VOUT>>> groupedByKeyAndWindow = groupedStreamByKey
- .transform("GroupByWindowWithCombiner",
- new CoderTypeInformation<>(outputKvCoder),
- windower)
- .returns(outputTypeInfo);
-
- return groupedByKeyAndWindow;
- }
-
- /**
- * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy.
- * This method assumes that <b>elements are already grouped by key</b>.
- * <p/>
- * The difference with {@link #create(PipelineOptions, PCollection, KeyedStream, Combine.KeyedCombineFn, KvCoder)}
- * is that this method assumes no combiner function
- * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}).
- *
- * @param options the general job configuration options.
- * @param input the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}.
- * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key.
- */
- public static <K, VIN> DataStream<WindowedValue<KV<K, Iterable<VIN>>>> createForIterable(
- PipelineOptions options,
- PCollection input,
- KeyedStream<WindowedValue<KV<K, VIN>>, K> groupedStreamByKey) {
- Preconditions.checkNotNull(options);
-
- KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) input.getCoder();
- Coder<K> keyCoder = inputKvCoder.getKeyCoder();
- Coder<VIN> inputValueCoder = inputKvCoder.getValueCoder();
-
- FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper(options,
- input.getPipeline().getCoderRegistry(), input.getWindowingStrategy(), inputKvCoder, null);
-
- Coder<Iterable<VIN>> valueIterCoder = IterableCoder.of(inputValueCoder);
- KvCoder<K, Iterable<VIN>> outputElemCoder = KvCoder.of(keyCoder, valueIterCoder);
-
- Coder<WindowedValue<KV<K, Iterable<VIN>>>> windowedOutputElemCoder = WindowedValue.FullWindowedValueCoder.of(
- outputElemCoder,
- input.getWindowingStrategy().getWindowFn().windowCoder());
-
- CoderTypeInformation<WindowedValue<KV<K, Iterable<VIN>>>> outputTypeInfo =
- new CoderTypeInformation<>(windowedOutputElemCoder);
-
- DataStream<WindowedValue<KV<K, Iterable<VIN>>>> groupedByKeyAndWindow = groupedStreamByKey
- .transform("GroupByWindow",
- new CoderTypeInformation<>(windowedOutputElemCoder),
- windower)
- .returns(outputTypeInfo);
-
- return groupedByKeyAndWindow;
- }
-
- public static <K, VIN, VACC, VOUT> FlinkGroupAlsoByWindowWrapper
- createForTesting(PipelineOptions options,
- CoderRegistry registry,
- WindowingStrategy<KV<K, VIN>, BoundedWindow> windowingStrategy,
- KvCoder<K, VIN> inputCoder,
- Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner) {
- Preconditions.checkNotNull(options);
-
- return new FlinkGroupAlsoByWindowWrapper(options, registry, windowingStrategy, inputCoder, combiner);
- }
-
- private FlinkGroupAlsoByWindowWrapper(PipelineOptions options,
- CoderRegistry registry,
- WindowingStrategy<KV<K, VIN>, BoundedWindow> windowingStrategy,
- KvCoder<K, VIN> inputCoder,
- Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner) {
- Preconditions.checkNotNull(options);
-
- this.options = Preconditions.checkNotNull(options);
- this.coderRegistry = Preconditions.checkNotNull(registry);
- this.inputKvCoder = Preconditions.checkNotNull(inputCoder);//(KvCoder<K, VIN>) input.getCoder();
- this.windowingStrategy = Preconditions.checkNotNull(windowingStrategy);//input.getWindowingStrategy();
- this.combineFn = combiner;
- this.operator = createGroupAlsoByWindowOperator();
- this.chainingStrategy = ChainingStrategy.ALWAYS;
- }
-
- @Override
- public void open() throws Exception {
- super.open();
- this.context = new ProcessContext(operator, new TimestampedCollector<>(output), this.timerInternals);
- }
-
- /**
- * Create the adequate {@link com.google.cloud.dataflow.sdk.util.GroupAlsoByWindowsDoFn},
- * <b> if not already created</b>.
- * If a {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn} was provided, then
- * a function with that combiner is created, so that elements are combined as they arrive. This is
- * done for speed and (in most of the cases) for reduction of the per-window state.
- */
- private <W extends BoundedWindow> DoFn<KeyedWorkItem<K, VIN>, KV<K, VOUT>> createGroupAlsoByWindowOperator() {
- if (this.operator == null) {
- if (this.combineFn == null) {
- // Thus VOUT == Iterable<VIN>
- Coder<VIN> inputValueCoder = inputKvCoder.getValueCoder();
-
- this.operator = (DoFn) GroupAlsoByWindowViaWindowSetDoFn.create(
- (WindowingStrategy<?, W>) this.windowingStrategy, SystemReduceFn.<K, VIN, W>buffering(inputValueCoder));
- } else {
- Coder<K> inputKeyCoder = inputKvCoder.getKeyCoder();
-
- AppliedCombineFn<K, VIN, VACC, VOUT> appliedCombineFn = AppliedCombineFn
- .withInputCoder(combineFn, coderRegistry, inputKvCoder);
-
- this.operator = GroupAlsoByWindowViaWindowSetDoFn.create(
- (WindowingStrategy<?, W>) this.windowingStrategy, SystemReduceFn.<K, VIN, VACC, VOUT, W>combining(inputKeyCoder, appliedCombineFn));
- }
- }
- return this.operator;
- }
-
- private void processKeyedWorkItem(KeyedWorkItem<K, VIN> workItem) throws Exception {
- context.setElement(workItem, getStateInternalsForKey(workItem.key()));
-
- // TODO: Ideally startBundle/finishBundle would be called when the operator is first used / about to be discarded.
- operator.startBundle(context);
- operator.processElement(context);
- operator.finishBundle(context);
- }
-
- @Override
- public void processElement(StreamRecord<WindowedValue<KV<K, VIN>>> element) throws Exception {
- ArrayList<WindowedValue<VIN>> elements = new ArrayList<>();
- elements.add(WindowedValue.of(element.getValue().getValue().getValue(), element.getValue().getTimestamp(),
- element.getValue().getWindows(), element.getValue().getPane()));
- processKeyedWorkItem(KeyedWorkItems.elementsWorkItem(element.getValue().getValue().getKey(), elements));
- }
-
- @Override
- public void processWatermark(Watermark mark) throws Exception {
- context.setCurrentInputWatermark(new Instant(mark.getTimestamp()));
-
- Multimap<K, TimerInternals.TimerData> timers = getTimersReadyToProcess(mark.getTimestamp());
- if (!timers.isEmpty()) {
- for (K key : timers.keySet()) {
- processKeyedWorkItem(KeyedWorkItems.<K, VIN>timersWorkItem(key, timers.get(key)));
- }
- }
-
- /**
- * This is to take into account the different semantics of the Watermark in Flink and
- * in Dataflow. To understand the reasoning behind the Dataflow semantics and its
- * watermark holding logic, see the documentation of
- * {@link WatermarkHold#addHold(ReduceFn.ProcessValueContext, boolean)}
- * */
- long millis = Long.MAX_VALUE;
- for (FlinkStateInternals state : perKeyStateInternals.values()) {
- Instant watermarkHold = state.getWatermarkHold();
- if (watermarkHold != null && watermarkHold.getMillis() < millis) {
- millis = watermarkHold.getMillis();
- }
- }
-
- if (mark.getTimestamp() < millis) {
- millis = mark.getTimestamp();
- }
-
- context.setCurrentOutputWatermark(new Instant(millis));
-
- // Don't forget to re-emit the watermark for further operators down the line.
- // This is critical for jobs with multiple aggregation steps.
- // Imagine a job with a groupByKey() on key K1, followed by a map() that changes
- // the key K1 to K2, and another groupByKey() on K2. In this case, if the watermark
- // is not re-emitted, the second aggregation would never be triggered, and no result
- // will be produced.
- output.emitWatermark(new Watermark(millis));
- }
-
- @Override
- public void close() throws Exception {
- super.close();
- }
-
- private void registerActiveTimer(K key, TimerInternals.TimerData timer) {
- Set<TimerInternals.TimerData> timersForKey = activeTimers.get(key);
- if (timersForKey == null) {
- timersForKey = new HashSet<>();
- }
- timersForKey.add(timer);
- activeTimers.put(key, timersForKey);
- }
-
- private void unregisterActiveTimer(K key, TimerInternals.TimerData timer) {
- Set<TimerInternals.TimerData> timersForKey = activeTimers.get(key);
- if (timersForKey != null) {
- timersForKey.remove(timer);
- if (timersForKey.isEmpty()) {
- activeTimers.remove(key);
- } else {
- activeTimers.put(key, timersForKey);
- }
- }
- }
-
- /**
- * Returns the list of timers that are ready to fire. These are the timers
- * that are registered to be triggered at a time before the current watermark.
- * We keep these timers in a Set, so that they are deduplicated, as the same
- * timer can be registered multiple times.
- */
- private Multimap<K, TimerInternals.TimerData> getTimersReadyToProcess(long currentWatermark) {
-
- // we keep the timers to return in a different list and launch them later
- // because we cannot prevent a trigger from registering another trigger,
- // which would lead to concurrent modification exception.
- Multimap<K, TimerInternals.TimerData> toFire = HashMultimap.create();
-
- Iterator<Map.Entry<K, Set<TimerInternals.TimerData>>> it = activeTimers.entrySet().iterator();
- while (it.hasNext()) {
- Map.Entry<K, Set<TimerInternals.TimerData>> keyWithTimers = it.next();
-
- Iterator<TimerInternals.TimerData> timerIt = keyWithTimers.getValue().iterator();
- while (timerIt.hasNext()) {
- TimerInternals.TimerData timerData = timerIt.next();
- if (timerData.getTimestamp().isBefore(currentWatermark)) {
- toFire.put(keyWithTimers.getKey(), timerData);
- timerIt.remove();
- }
- }
-
- if (keyWithTimers.getValue().isEmpty()) {
- it.remove();
- }
- }
- return toFire;
- }
-
- /**
- * Gets the state associated with the specified key.
- *
- * @param key the key whose state we want.
- * @return The {@link FlinkStateInternals}
- * associated with that key.
- */
- private FlinkStateInternals<K> getStateInternalsForKey(K key) {
- FlinkStateInternals<K> stateInternals = perKeyStateInternals.get(key);
- if (stateInternals == null) {
- Coder<? extends BoundedWindow> windowCoder = this.windowingStrategy.getWindowFn().windowCoder();
- OutputTimeFn<? super BoundedWindow> outputTimeFn = this.windowingStrategy.getWindowFn().getOutputTimeFn();
- stateInternals = new FlinkStateInternals<>(key, inputKvCoder.getKeyCoder(), windowCoder, outputTimeFn);
- perKeyStateInternals.put(key, stateInternals);
- }
- return stateInternals;
- }
-
- private class FlinkTimerInternals extends AbstractFlinkTimerInternals<K, VIN> {
- @Override
- public void setTimer(TimerData timerKey) {
- registerActiveTimer(context.element().key(), timerKey);
- }
-
- @Override
- public void deleteTimer(TimerData timerKey) {
- unregisterActiveTimer(context.element().key(), timerKey);
- }
- }
-
- private class ProcessContext extends GroupAlsoByWindowViaWindowSetDoFn<K, VIN, VOUT, ?, KeyedWorkItem<K, VIN>>.ProcessContext {
-
- private final FlinkTimerInternals timerInternals;
-
- private final TimestampedCollector<WindowedValue<KV<K, VOUT>>> collector;
-
- private FlinkStateInternals<K> stateInternals;
-
- private KeyedWorkItem<K, VIN> element;
-
- public ProcessContext(DoFn<KeyedWorkItem<K, VIN>, KV<K, VOUT>> function,
- TimestampedCollector<WindowedValue<KV<K, VOUT>>> outCollector,
- FlinkTimerInternals timerInternals) {
- function.super();
- super.setupDelegateAggregators();
-
- this.collector = Preconditions.checkNotNull(outCollector);
- this.timerInternals = Preconditions.checkNotNull(timerInternals);
- }
-
- public void setElement(KeyedWorkItem<K, VIN> element,
- FlinkStateInternals<K> stateForKey) {
- this.element = element;
- this.stateInternals = stateForKey;
- }
-
- public void setCurrentInputWatermark(Instant watermark) {
- this.timerInternals.setCurrentInputWatermark(watermark);
- }
-
- public void setCurrentOutputWatermark(Instant watermark) {
- this.timerInternals.setCurrentOutputWatermark(watermark);
- }
-
- @Override
- public KeyedWorkItem<K, VIN> element() {
- return this.element;
- }
-
- @Override
- public Instant timestamp() {
- throw new UnsupportedOperationException("timestamp() is not available when processing KeyedWorkItems.");
- }
-
- @Override
- public PipelineOptions getPipelineOptions() {
- // TODO: PipelineOptions need to be available on the workers.
- // Ideally they are captured as part of the pipeline.
- // For now, construct empty options so that StateContexts.createFromComponents
- // will yield a valid StateContext, which is needed to support the StateContext.window().
- if (options == null) {
- options = new PipelineOptions() {
- @Override
- public <T extends PipelineOptions> T as(Class<T> kls) {
- return null;
- }
-
- @Override
- public <T extends PipelineOptions> T cloneAs(Class<T> kls) {
- return null;
- }
-
- @Override
- public Class<? extends PipelineRunner<?>> getRunner() {
- return null;
- }
-
- @Override
- public void setRunner(Class<? extends PipelineRunner<?>> kls) {
-
- }
-
- @Override
- public CheckEnabled getStableUniqueNames() {
- return null;
- }
-
- @Override
- public void setStableUniqueNames(CheckEnabled enabled) {
- }
- };
- }
- return options;
- }
-
- @Override
- public void output(KV<K, VOUT> output) {
- throw new UnsupportedOperationException(
- "output() is not available when processing KeyedWorkItems.");
- }
-
- @Override
- public void outputWithTimestamp(KV<K, VOUT> output, Instant timestamp) {
- throw new UnsupportedOperationException(
- "outputWithTimestamp() is not available when processing KeyedWorkItems.");
- }
-
- @Override
- public PaneInfo pane() {
- throw new UnsupportedOperationException("pane() is not available when processing KeyedWorkItems.");
- }
-
- @Override
- public BoundedWindow window() {
- throw new UnsupportedOperationException(
- "window() is not available when processing KeyedWorkItems.");
- }
-
- @Override
- public WindowingInternals<KeyedWorkItem<K, VIN>, KV<K, VOUT>> windowingInternals() {
- return new WindowingInternals<KeyedWorkItem<K, VIN>, KV<K, VOUT>>() {
-
- @Override
- public com.google.cloud.dataflow.sdk.util.state.StateInternals stateInternals() {
- return stateInternals;
- }
-
- @Override
- public void outputWindowedValue(KV<K, VOUT> output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
- // TODO: No need to represent timestamp twice.
- collector.setAbsoluteTimestamp(timestamp.getMillis());
- collector.collect(WindowedValue.of(output, timestamp, windows, pane));
-
- }
-
- @Override
- public TimerInternals timerInternals() {
- return timerInternals;
- }
-
- @Override
- public Collection<? extends BoundedWindow> windows() {
- throw new UnsupportedOperationException("windows() is not available in Streaming mode.");
- }
-
- @Override
- public PaneInfo pane() {
- throw new UnsupportedOperationException("pane() is not available in Streaming mode.");
- }
-
- @Override
- public <T> void writePCollectionViewData(TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
- throw new RuntimeException("writePCollectionViewData() not available in Streaming mode.");
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
- throw new RuntimeException("sideInput() is not available in Streaming mode.");
- }
- };
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view) {
- throw new RuntimeException("sideInput() is not supported in Streaming mode.");
- }
-
- @Override
- public <T> void sideOutput(TupleTag<T> tag, T output) {
- // ignore the side output, this can happen when a user does not register
- // side outputs but then outputs using a freshly created TupleTag.
- throw new RuntimeException("sideOutput() is not available when grouping by window.");
- }
-
- @Override
- public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
- sideOutput(tag, output);
- }
-
- @Override
- protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
- Accumulator acc = getRuntimeContext().getAccumulator(name);
- if (acc != null) {
- AccumulatorHelper.compareAccumulatorTypes(name,
- SerializableFnAggregatorWrapper.class, acc.getClass());
- return (Aggregator<AggInputT, AggOutputT>) acc;
- }
-
- SerializableFnAggregatorWrapper<AggInputT, AggOutputT> accumulator =
- new SerializableFnAggregatorWrapper<>(combiner);
- getRuntimeContext().addAccumulator(name, accumulator);
- return accumulator;
- }
- }
-
- ////////////// Checkpointing implementation ////////////////
-
- @Override
- public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception {
- StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
- AbstractStateBackend.CheckpointStateOutputView out = getStateBackend().createCheckpointStateOutputView(checkpointId, timestamp);
- StateCheckpointWriter writer = StateCheckpointWriter.create(out);
- Coder<K> keyCoder = inputKvCoder.getKeyCoder();
-
- // checkpoint the timers
- StateCheckpointUtils.encodeTimers(activeTimers, writer, keyCoder);
-
- // checkpoint the state
- StateCheckpointUtils.encodeState(perKeyStateInternals, writer, keyCoder);
-
- // checkpoint the timerInternals
- context.timerInternals.encodeTimerInternals(context, writer,
- inputKvCoder, windowingStrategy.getWindowFn().windowCoder());
-
- taskState.setOperatorState(out.closeAndGetHandle());
- return taskState;
- }
-
- @Override
- public void restoreState(StreamTaskState taskState, long recoveryTimestamp) throws Exception {
- super.restoreState(taskState, recoveryTimestamp);
-
- final ClassLoader userClassloader = getUserCodeClassloader();
-
- Coder<? extends BoundedWindow> windowCoder = this.windowingStrategy.getWindowFn().windowCoder();
- Coder<K> keyCoder = inputKvCoder.getKeyCoder();
-
- @SuppressWarnings("unchecked")
- StateHandle<DataInputView> inputState = (StateHandle<DataInputView>) taskState.getOperatorState();
- DataInputView in = inputState.getState(userClassloader);
- StateCheckpointReader reader = new StateCheckpointReader(in);
-
- // restore the timers
- this.activeTimers = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder);
-
- // restore the state
- this.perKeyStateInternals = StateCheckpointUtils.decodeState(
- reader, windowingStrategy.getOutputTimeFn(), keyCoder, windowCoder, userClassloader);
-
- // restore the timerInternals.
- this.timerInternals.restoreTimerInternals(reader, inputKvCoder, windowCoder);
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
deleted file mode 100644
index 61953a6..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupByKeyWrapper.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers.streaming;
-
-import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
-import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.datastream.KeyedStream;
-
-/**
- * This class groups the elements by key. It assumes that already the incoming stream
- * is composed of <code>[Key,Value]</code> pairs.
- * */
-public class FlinkGroupByKeyWrapper {
-
- /**
- * Just an auxiliary interface to bypass the fact that java anonymous classes cannot implement
- * multiple interfaces.
- */
- private interface KeySelectorWithQueryableResultType<K, V> extends KeySelector<WindowedValue<KV<K, V>>, K>, ResultTypeQueryable<K> {
- }
-
- public static <K, V> KeyedStream<WindowedValue<KV<K, V>>, K> groupStreamByKey(DataStream<WindowedValue<KV<K, V>>> inputDataStream, KvCoder<K, V> inputKvCoder) {
- final Coder<K> keyCoder = inputKvCoder.getKeyCoder();
- final TypeInformation<K> keyTypeInfo = new CoderTypeInformation<>(keyCoder);
- final boolean isKeyVoid = keyCoder instanceof VoidCoder;
-
- return inputDataStream.keyBy(
- new KeySelectorWithQueryableResultType<K, V>() {
-
- @Override
- public K getKey(WindowedValue<KV<K, V>> value) throws Exception {
- return isKeyVoid ? (K) VoidCoderTypeSerializer.VoidValue.INSTANCE :
- value.getValue().getKey();
- }
-
- @Override
- public TypeInformation<K> getProducedType() {
- return keyTypeInfo;
- }
- });
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
deleted file mode 100644
index cdf23f6..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundMultiWrapper.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers.streaming;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingInternals;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.base.Preconditions;
-import org.apache.flink.util.Collector;
-import org.joda.time.Instant;
-
-import java.util.Map;
-
-/**
- * A wrapper for the {@link com.google.cloud.dataflow.sdk.transforms.ParDo.BoundMulti} Beam transformation.
- * */
-public class FlinkParDoBoundMultiWrapper<IN, OUT> extends FlinkAbstractParDoWrapper<IN, OUT, RawUnionValue> {
-
- private final TupleTag<?> mainTag;
- private final Map<TupleTag<?>, Integer> outputLabels;
-
- public FlinkParDoBoundMultiWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUT> doFn, TupleTag<?> mainTag, Map<TupleTag<?>, Integer> tagsToLabels) {
- super(options, windowingStrategy, doFn);
- this.mainTag = Preconditions.checkNotNull(mainTag);
- this.outputLabels = Preconditions.checkNotNull(tagsToLabels);
- }
-
- @Override
- public void outputWithTimestampHelper(WindowedValue<IN> inElement, OUT output, Instant timestamp, Collector<WindowedValue<RawUnionValue>> collector) {
- checkTimestamp(inElement, timestamp);
- Integer index = outputLabels.get(mainTag);
- collector.collect(makeWindowedValue(
- new RawUnionValue(index, output),
- timestamp,
- inElement.getWindows(),
- inElement.getPane()));
- }
-
- @Override
- public <T> void sideOutputWithTimestampHelper(WindowedValue<IN> inElement, T output, Instant timestamp, Collector<WindowedValue<RawUnionValue>> collector, TupleTag<T> tag) {
- checkTimestamp(inElement, timestamp);
- Integer index = outputLabels.get(tag);
- if (index != null) {
- collector.collect(makeWindowedValue(
- new RawUnionValue(index, output),
- timestamp,
- inElement.getWindows(),
- inElement.getPane()));
- }
- }
-
- @Override
- public WindowingInternals<IN, OUT> windowingInternalsHelper(WindowedValue<IN> inElement, Collector<WindowedValue<RawUnionValue>> outCollector) {
- throw new RuntimeException("FlinkParDoBoundMultiWrapper is just an internal operator serving as " +
- "an intermediate transformation for the ParDo.BoundMulti translation. windowingInternals() " +
- "is not available in this class.");
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
deleted file mode 100644
index 3357cd5..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkParDoBoundWrapper.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers.streaming;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.*;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import org.apache.flink.util.Collector;
-import org.joda.time.Instant;
-
-import java.io.IOException;
-import java.util.*;
-
-/**
- * A wrapper for the {@link com.google.cloud.dataflow.sdk.transforms.ParDo.Bound} Beam transformation.
- * */
-public class FlinkParDoBoundWrapper<IN, OUT> extends FlinkAbstractParDoWrapper<IN, OUT, OUT> {
-
- public FlinkParDoBoundWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUT> doFn) {
- super(options, windowingStrategy, doFn);
- }
-
- @Override
- public void outputWithTimestampHelper(WindowedValue<IN> inElement, OUT output, Instant timestamp, Collector<WindowedValue<OUT>> collector) {
- checkTimestamp(inElement, timestamp);
- collector.collect(makeWindowedValue(
- output,
- timestamp,
- inElement.getWindows(),
- inElement.getPane()));
- }
-
- @Override
- public <T> void sideOutputWithTimestampHelper(WindowedValue<IN> inElement, T output, Instant timestamp, Collector<WindowedValue<OUT>> outCollector, TupleTag<T> tag) {
- // ignore the side output, this can happen when a user does not register
- // side outputs but then outputs using a freshly created TupleTag.
- throw new RuntimeException("sideOutput() not not available in ParDo.Bound().");
- }
-
- @Override
- public WindowingInternals<IN, OUT> windowingInternalsHelper(final WindowedValue<IN> inElement, final Collector<WindowedValue<OUT>> collector) {
- return new WindowingInternals<IN, OUT>() {
- @Override
- public StateInternals stateInternals() {
- throw new NullPointerException("StateInternals are not available for ParDo.Bound().");
- }
-
- @Override
- public void outputWindowedValue(OUT output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
- collector.collect(makeWindowedValue(output, timestamp, windows, pane));
- }
-
- @Override
- public TimerInternals timerInternals() {
- throw new NullPointerException("TimeInternals are not available for ParDo.Bound().");
- }
-
- @Override
- public Collection<? extends BoundedWindow> windows() {
- return inElement.getWindows();
- }
-
- @Override
- public PaneInfo pane() {
- return inElement.getPane();
- }
-
- @Override
- public <T> void writePCollectionViewData(TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
- throw new RuntimeException("writePCollectionViewData() not supported in Streaming mode.");
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
- throw new RuntimeException("sideInput() not implemented.");
- }
- };
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
deleted file mode 100644
index 2599e88..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers.streaming.io;
-
-import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.streaming.api.operators.TimestampedCollector;
-import org.apache.flink.util.Collector;
-import org.joda.time.Instant;
-
-import java.io.ByteArrayInputStream;
-import java.util.List;
-
-/**
- * This flat map function bootstraps from collection elements and turns them into WindowedValues
- * (as required by the Flink runner).
- */
-public class FlinkStreamingCreateFunction<IN, OUT> implements FlatMapFunction<IN, WindowedValue<OUT>> {
-
- private final List<byte[]> elements;
- private final Coder<OUT> coder;
-
- public FlinkStreamingCreateFunction(List<byte[]> elements, Coder<OUT> coder) {
- this.elements = elements;
- this.coder = coder;
- }
-
- @Override
- public void flatMap(IN value, Collector<WindowedValue<OUT>> out) throws Exception {
-
- @SuppressWarnings("unchecked")
- OUT voidValue = (OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE;
- for (byte[] element : elements) {
- ByteArrayInputStream bai = new ByteArrayInputStream(element);
- OUT outValue = coder.decode(bai, Coder.Context.OUTER);
-
- if (outValue == null) {
- out.collect(WindowedValue.of(voidValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
- } else {
- out.collect(WindowedValue.of(outValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
- }
- }
-
- out.close();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java
deleted file mode 100644
index ddbc993..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedFlinkSource.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers.streaming.io;
-
-import org.apache.beam.runners.flink.FlinkPipelineRunner;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.common.base.Preconditions;
-import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-
-import javax.annotation.Nullable;
-import java.util.List;
-
-/**
- * A wrapper translating Flink Sources implementing the {@link RichParallelSourceFunction} interface, into
- * unbounded Beam sources (see {@link UnboundedSource}).
- * */
-public class UnboundedFlinkSource<T, C extends UnboundedSource.CheckpointMark> extends UnboundedSource<T, C> {
-
- private final PipelineOptions options;
- private final RichParallelSourceFunction<T> flinkSource;
-
- public UnboundedFlinkSource(PipelineOptions pipelineOptions, RichParallelSourceFunction<T> source) {
- if(!pipelineOptions.getRunner().equals(FlinkPipelineRunner.class)) {
- throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
- }
- options = Preconditions.checkNotNull(pipelineOptions);
- flinkSource = Preconditions.checkNotNull(source);
- validate();
- }
-
- public RichParallelSourceFunction<T> getFlinkSource() {
- return this.flinkSource;
- }
-
- @Override
- public List<? extends UnboundedSource<T, C>> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception {
- throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
- }
-
- @Override
- public UnboundedReader<T> createReader(PipelineOptions options, @Nullable C checkpointMark) {
- throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
- }
-
- @Nullable
- @Override
- public Coder<C> getCheckpointMarkCoder() {
- throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
- }
-
-
- @Override
- public void validate() {
- Preconditions.checkNotNull(options);
- Preconditions.checkNotNull(flinkSource);
- if(!options.getRunner().equals(FlinkPipelineRunner.class)) {
- throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
- }
- }
-
- @Override
- public Coder<T> getDefaultOutputCoder() {
- throw new RuntimeException("Flink Sources are supported only when running with the FlinkPipelineRunner.");
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java
deleted file mode 100644
index a24964a..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java
+++ /dev/null
@@ -1,231 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers.streaming.io;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.annotation.Nullable;
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.io.Serializable;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.util.Collections;
-import java.util.List;
-import java.util.NoSuchElementException;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-/**
- * An example unbounded Beam source that reads input from a socket. This is used mainly for testing and debugging.
- * */
-public class UnboundedSocketSource<C extends UnboundedSource.CheckpointMark> extends UnboundedSource<String, C> {
-
- private static final Coder<String> DEFAULT_SOCKET_CODER = StringUtf8Coder.of();
-
- private static final long serialVersionUID = 1L;
-
- private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 500;
-
- private static final int CONNECTION_TIMEOUT_TIME = 0;
-
- private final String hostname;
- private final int port;
- private final char delimiter;
- private final long maxNumRetries;
- private final long delayBetweenRetries;
-
- public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries) {
- this(hostname, port, delimiter, maxNumRetries, DEFAULT_CONNECTION_RETRY_SLEEP);
- }
-
- public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries, long delayBetweenRetries) {
- this.hostname = hostname;
- this.port = port;
- this.delimiter = delimiter;
- this.maxNumRetries = maxNumRetries;
- this.delayBetweenRetries = delayBetweenRetries;
- }
-
- public String getHostname() {
- return this.hostname;
- }
-
- public int getPort() {
- return this.port;
- }
-
- public char getDelimiter() {
- return this.delimiter;
- }
-
- public long getMaxNumRetries() {
- return this.maxNumRetries;
- }
-
- public long getDelayBetweenRetries() {
- return this.delayBetweenRetries;
- }
-
- @Override
- public List<? extends UnboundedSource<String, C>> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception {
- return Collections.<UnboundedSource<String, C>>singletonList(this);
- }
-
- @Override
- public UnboundedReader<String> createReader(PipelineOptions options, @Nullable C checkpointMark) {
- return new UnboundedSocketReader(this);
- }
-
- @Nullable
- @Override
- public Coder getCheckpointMarkCoder() {
- // Flink and Dataflow have different checkpointing mechanisms.
- // In our case we do not need a coder.
- return null;
- }
-
- @Override
- public void validate() {
- checkArgument(port > 0 && port < 65536, "port is out of range");
- checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), or -1 (infinite retries)");
- checkArgument(delayBetweenRetries >= 0, "delayBetweenRetries must be zero or positive");
- }
-
- @Override
- public Coder getDefaultOutputCoder() {
- return DEFAULT_SOCKET_CODER;
- }
-
- public static class UnboundedSocketReader extends UnboundedSource.UnboundedReader<String> implements Serializable {
-
- private static final long serialVersionUID = 7526472295622776147L;
- private static final Logger LOG = LoggerFactory.getLogger(UnboundedSocketReader.class);
-
- private final UnboundedSocketSource source;
-
- private Socket socket;
- private BufferedReader reader;
-
- private boolean isRunning;
-
- private String currentRecord;
-
- public UnboundedSocketReader(UnboundedSocketSource source) {
- this.source = source;
- }
-
- private void openConnection() throws IOException {
- this.socket = new Socket();
- this.socket.connect(new InetSocketAddress(this.source.getHostname(), this.source.getPort()), CONNECTION_TIMEOUT_TIME);
- this.reader = new BufferedReader(new InputStreamReader(this.socket.getInputStream()));
- this.isRunning = true;
- }
-
- @Override
- public boolean start() throws IOException {
- int attempt = 0;
- while (!isRunning) {
- try {
- openConnection();
- LOG.info("Connected to server socket " + this.source.getHostname() + ':' + this.source.getPort());
-
- return advance();
- } catch (IOException e) {
- LOG.info("Lost connection to server socket " + this.source.getHostname() + ':' + this.source.getPort() + ". Retrying in " + this.source.getDelayBetweenRetries() + " msecs...");
-
- if (this.source.getMaxNumRetries() == -1 || attempt++ < this.source.getMaxNumRetries()) {
- try {
- Thread.sleep(this.source.getDelayBetweenRetries());
- } catch (InterruptedException e1) {
- e1.printStackTrace();
- }
- } else {
- this.isRunning = false;
- break;
- }
- }
- }
- LOG.error("Unable to connect to host " + this.source.getHostname() + " : " + this.source.getPort());
- return false;
- }
-
- @Override
- public boolean advance() throws IOException {
- final StringBuilder buffer = new StringBuilder();
- int data;
- while (isRunning && (data = reader.read()) != -1) {
- // check if the string is complete
- if (data != this.source.getDelimiter()) {
- buffer.append((char) data);
- } else {
- if (buffer.length() > 0 && buffer.charAt(buffer.length() - 1) == '\r') {
- buffer.setLength(buffer.length() - 1);
- }
- this.currentRecord = buffer.toString();
- buffer.setLength(0);
- return true;
- }
- }
- return false;
- }
-
- @Override
- public byte[] getCurrentRecordId() throws NoSuchElementException {
- return new byte[0];
- }
-
- @Override
- public String getCurrent() throws NoSuchElementException {
- return this.currentRecord;
- }
-
- @Override
- public Instant getCurrentTimestamp() throws NoSuchElementException {
- return Instant.now();
- }
-
- @Override
- public void close() throws IOException {
- this.reader.close();
- this.socket.close();
- this.isRunning = false;
- LOG.info("Closed connection to server socket at " + this.source.getHostname() + ":" + this.source.getPort() + ".");
- }
-
- @Override
- public Instant getWatermark() {
- return Instant.now();
- }
-
- @Override
- public CheckpointMark getCheckpointMark() {
- return null;
- }
-
- @Override
- public UnboundedSource<String, ?> getCurrentSource() {
- return this.source;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
deleted file mode 100644
index 7c1ccdf..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers.streaming.io;
-
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
-import org.apache.flink.streaming.api.operators.StreamSource;
-import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.operators.Triggerable;
-import org.joda.time.Instant;
-
-/**
- * A wrapper for Beam's unbounded sources. This class wraps around a source implementing the {@link com.google.cloud.dataflow.sdk.io.Read.Unbounded}
- * interface.
- *
- *</p>
- * For now we support non-parallel, not checkpointed sources.
- * */
-public class UnboundedSourceWrapper<T> extends RichSourceFunction<WindowedValue<T>> implements Triggerable {
-
- private final String name;
- private final UnboundedSource.UnboundedReader<T> reader;
-
- private StreamingRuntimeContext runtime = null;
- private StreamSource.ManualWatermarkContext<WindowedValue<T>> context = null;
-
- private volatile boolean isRunning = false;
-
- public UnboundedSourceWrapper(PipelineOptions options, Read.Unbounded<T> transform) {
- this.name = transform.getName();
- this.reader = transform.getSource().createReader(options, null);
- }
-
- public String getName() {
- return this.name;
- }
-
- WindowedValue<T> makeWindowedValue(T output, Instant timestamp) {
- if (timestamp == null) {
- timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
- }
- return WindowedValue.of(output, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING);
- }
-
- @Override
- public void run(SourceContext<WindowedValue<T>> ctx) throws Exception {
- if (!(ctx instanceof StreamSource.ManualWatermarkContext)) {
- throw new RuntimeException("We assume that all sources in Dataflow are EventTimeSourceFunction. " +
- "Apparently " + this.name + " is not. Probably you should consider writing your own Wrapper for this source.");
- }
-
- context = (StreamSource.ManualWatermarkContext<WindowedValue<T>>) ctx;
- runtime = (StreamingRuntimeContext) getRuntimeContext();
-
- this.isRunning = true;
- boolean inputAvailable = reader.start();
-
- setNextWatermarkTimer(this.runtime);
-
- while (isRunning) {
-
- while (!inputAvailable && isRunning) {
- // wait a bit until we retry to pull more records
- Thread.sleep(50);
- inputAvailable = reader.advance();
- }
-
- if (inputAvailable) {
-
- // get it and its timestamp from the source
- T item = reader.getCurrent();
- Instant timestamp = reader.getCurrentTimestamp();
-
- // write it to the output collector
- synchronized (ctx.getCheckpointLock()) {
- context.collectWithTimestamp(makeWindowedValue(item, timestamp), timestamp.getMillis());
- }
-
- inputAvailable = reader.advance();
- }
-
- }
- }
-
- @Override
- public void cancel() {
- isRunning = false;
- }
-
- @Override
- public void trigger(long timestamp) throws Exception {
- if (this.isRunning) {
- synchronized (context.getCheckpointLock()) {
- long watermarkMillis = this.reader.getWatermark().getMillis();
- context.emitWatermark(new Watermark(watermarkMillis));
- }
- setNextWatermarkTimer(this.runtime);
- }
- }
-
- private void setNextWatermarkTimer(StreamingRuntimeContext runtime) {
- if (this.isRunning) {
- long watermarkInterval = runtime.getExecutionConfig().getAutoWatermarkInterval();
- long timeToNextWatermark = getTimeToNextWaternark(watermarkInterval);
- runtime.registerTimer(timeToNextWatermark, this);
- }
- }
-
- private long getTimeToNextWaternark(long watermarkInterval) {
- return System.currentTimeMillis() + watermarkInterval;
- }
-}
[42/50] [abbrv] incubator-beam git commit: [flink] convert tabs to 2
spaces
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java
index dd14f68..0b1a5da 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSocketSource.java
@@ -41,191 +41,191 @@ import static com.google.common.base.Preconditions.checkArgument;
* */
public class UnboundedSocketSource<C extends UnboundedSource.CheckpointMark> extends UnboundedSource<String, C> {
- private static final Coder<String> DEFAULT_SOCKET_CODER = StringUtf8Coder.of();
+ private static final Coder<String> DEFAULT_SOCKET_CODER = StringUtf8Coder.of();
- private static final long serialVersionUID = 1L;
-
- private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 500;
-
- private static final int CONNECTION_TIMEOUT_TIME = 0;
-
- private final String hostname;
- private final int port;
- private final char delimiter;
- private final long maxNumRetries;
- private final long delayBetweenRetries;
-
- public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries) {
- this(hostname, port, delimiter, maxNumRetries, DEFAULT_CONNECTION_RETRY_SLEEP);
- }
-
- public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries, long delayBetweenRetries) {
- this.hostname = hostname;
- this.port = port;
- this.delimiter = delimiter;
- this.maxNumRetries = maxNumRetries;
- this.delayBetweenRetries = delayBetweenRetries;
- }
-
- public String getHostname() {
- return this.hostname;
- }
-
- public int getPort() {
- return this.port;
- }
-
- public char getDelimiter() {
- return this.delimiter;
- }
-
- public long getMaxNumRetries() {
- return this.maxNumRetries;
- }
-
- public long getDelayBetweenRetries() {
- return this.delayBetweenRetries;
- }
-
- @Override
- public List<? extends UnboundedSource<String, C>> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception {
- return Collections.<UnboundedSource<String, C>>singletonList(this);
- }
-
- @Override
- public UnboundedReader<String> createReader(PipelineOptions options, @Nullable C checkpointMark) {
- return new UnboundedSocketReader(this);
- }
-
- @Nullable
- @Override
- public Coder getCheckpointMarkCoder() {
- // Flink and Dataflow have different checkpointing mechanisms.
- // In our case we do not need a coder.
- return null;
- }
-
- @Override
- public void validate() {
- checkArgument(port > 0 && port < 65536, "port is out of range");
- checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), or -1 (infinite retries)");
- checkArgument(delayBetweenRetries >= 0, "delayBetweenRetries must be zero or positive");
- }
-
- @Override
- public Coder getDefaultOutputCoder() {
- return DEFAULT_SOCKET_CODER;
- }
-
- public static class UnboundedSocketReader extends UnboundedSource.UnboundedReader<String> implements Serializable {
-
- private static final long serialVersionUID = 7526472295622776147L;
- private static final Logger LOG = LoggerFactory.getLogger(UnboundedSocketReader.class);
-
- private final UnboundedSocketSource source;
-
- private Socket socket;
- private BufferedReader reader;
-
- private boolean isRunning;
-
- private String currentRecord;
-
- public UnboundedSocketReader(UnboundedSocketSource source) {
- this.source = source;
- }
-
- private void openConnection() throws IOException {
- this.socket = new Socket();
- this.socket.connect(new InetSocketAddress(this.source.getHostname(), this.source.getPort()), CONNECTION_TIMEOUT_TIME);
- this.reader = new BufferedReader(new InputStreamReader(this.socket.getInputStream()));
- this.isRunning = true;
- }
-
- @Override
- public boolean start() throws IOException {
- int attempt = 0;
- while (!isRunning) {
- try {
- openConnection();
- LOG.info("Connected to server socket " + this.source.getHostname() + ':' + this.source.getPort());
-
- return advance();
- } catch (IOException e) {
- LOG.info("Lost connection to server socket " + this.source.getHostname() + ':' + this.source.getPort() + ". Retrying in " + this.source.getDelayBetweenRetries() + " msecs...");
-
- if (this.source.getMaxNumRetries() == -1 || attempt++ < this.source.getMaxNumRetries()) {
- try {
- Thread.sleep(this.source.getDelayBetweenRetries());
- } catch (InterruptedException e1) {
- e1.printStackTrace();
- }
- } else {
- this.isRunning = false;
- break;
- }
- }
- }
- LOG.error("Unable to connect to host " + this.source.getHostname() + " : " + this.source.getPort());
- return false;
- }
-
- @Override
- public boolean advance() throws IOException {
- final StringBuilder buffer = new StringBuilder();
- int data;
- while (isRunning && (data = reader.read()) != -1) {
- // check if the string is complete
- if (data != this.source.getDelimiter()) {
- buffer.append((char) data);
- } else {
- if (buffer.length() > 0 && buffer.charAt(buffer.length() - 1) == '\r') {
- buffer.setLength(buffer.length() - 1);
- }
- this.currentRecord = buffer.toString();
- buffer.setLength(0);
- return true;
- }
- }
- return false;
- }
-
- @Override
- public byte[] getCurrentRecordId() throws NoSuchElementException {
- return new byte[0];
- }
-
- @Override
- public String getCurrent() throws NoSuchElementException {
- return this.currentRecord;
- }
-
- @Override
- public Instant getCurrentTimestamp() throws NoSuchElementException {
- return Instant.now();
- }
-
- @Override
- public void close() throws IOException {
- this.reader.close();
- this.socket.close();
- this.isRunning = false;
- LOG.info("Closed connection to server socket at " + this.source.getHostname() + ":" + this.source.getPort() + ".");
- }
-
- @Override
- public Instant getWatermark() {
- return Instant.now();
- }
-
- @Override
- public CheckpointMark getCheckpointMark() {
- return null;
- }
-
- @Override
- public UnboundedSource<String, ?> getCurrentSource() {
- return this.source;
- }
- }
+ private static final long serialVersionUID = 1L;
+
+ private static final int DEFAULT_CONNECTION_RETRY_SLEEP = 500;
+
+ private static final int CONNECTION_TIMEOUT_TIME = 0;
+
+ private final String hostname;
+ private final int port;
+ private final char delimiter;
+ private final long maxNumRetries;
+ private final long delayBetweenRetries;
+
+ public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries) {
+ this(hostname, port, delimiter, maxNumRetries, DEFAULT_CONNECTION_RETRY_SLEEP);
+ }
+
+ public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries, long delayBetweenRetries) {
+ this.hostname = hostname;
+ this.port = port;
+ this.delimiter = delimiter;
+ this.maxNumRetries = maxNumRetries;
+ this.delayBetweenRetries = delayBetweenRetries;
+ }
+
+ public String getHostname() {
+ return this.hostname;
+ }
+
+ public int getPort() {
+ return this.port;
+ }
+
+ public char getDelimiter() {
+ return this.delimiter;
+ }
+
+ public long getMaxNumRetries() {
+ return this.maxNumRetries;
+ }
+
+ public long getDelayBetweenRetries() {
+ return this.delayBetweenRetries;
+ }
+
+ @Override
+ public List<? extends UnboundedSource<String, C>> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception {
+ return Collections.<UnboundedSource<String, C>>singletonList(this);
+ }
+
+ @Override
+ public UnboundedReader<String> createReader(PipelineOptions options, @Nullable C checkpointMark) {
+ return new UnboundedSocketReader(this);
+ }
+
+ @Nullable
+ @Override
+ public Coder getCheckpointMarkCoder() {
+ // Flink and Dataflow have different checkpointing mechanisms.
+ // In our case we do not need a coder.
+ return null;
+ }
+
+ @Override
+ public void validate() {
+ checkArgument(port > 0 && port < 65536, "port is out of range");
+ checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), or -1 (infinite retries)");
+ checkArgument(delayBetweenRetries >= 0, "delayBetweenRetries must be zero or positive");
+ }
+
+ @Override
+ public Coder getDefaultOutputCoder() {
+ return DEFAULT_SOCKET_CODER;
+ }
+
+ public static class UnboundedSocketReader extends UnboundedSource.UnboundedReader<String> implements Serializable {
+
+ private static final long serialVersionUID = 7526472295622776147L;
+ private static final Logger LOG = LoggerFactory.getLogger(UnboundedSocketReader.class);
+
+ private final UnboundedSocketSource source;
+
+ private Socket socket;
+ private BufferedReader reader;
+
+ private boolean isRunning;
+
+ private String currentRecord;
+
+ public UnboundedSocketReader(UnboundedSocketSource source) {
+ this.source = source;
+ }
+
+ private void openConnection() throws IOException {
+ this.socket = new Socket();
+ this.socket.connect(new InetSocketAddress(this.source.getHostname(), this.source.getPort()), CONNECTION_TIMEOUT_TIME);
+ this.reader = new BufferedReader(new InputStreamReader(this.socket.getInputStream()));
+ this.isRunning = true;
+ }
+
+ @Override
+ public boolean start() throws IOException {
+ int attempt = 0;
+ while (!isRunning) {
+ try {
+ openConnection();
+ LOG.info("Connected to server socket " + this.source.getHostname() + ':' + this.source.getPort());
+
+ return advance();
+ } catch (IOException e) {
+ LOG.info("Lost connection to server socket " + this.source.getHostname() + ':' + this.source.getPort() + ". Retrying in " + this.source.getDelayBetweenRetries() + " msecs...");
+
+ if (this.source.getMaxNumRetries() == -1 || attempt++ < this.source.getMaxNumRetries()) {
+ try {
+ Thread.sleep(this.source.getDelayBetweenRetries());
+ } catch (InterruptedException e1) {
+ e1.printStackTrace();
+ }
+ } else {
+ this.isRunning = false;
+ break;
+ }
+ }
+ }
+ LOG.error("Unable to connect to host " + this.source.getHostname() + " : " + this.source.getPort());
+ return false;
+ }
+
+ @Override
+ public boolean advance() throws IOException {
+ final StringBuilder buffer = new StringBuilder();
+ int data;
+ while (isRunning && (data = reader.read()) != -1) {
+ // check if the string is complete
+ if (data != this.source.getDelimiter()) {
+ buffer.append((char) data);
+ } else {
+ if (buffer.length() > 0 && buffer.charAt(buffer.length() - 1) == '\r') {
+ buffer.setLength(buffer.length() - 1);
+ }
+ this.currentRecord = buffer.toString();
+ buffer.setLength(0);
+ return true;
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public byte[] getCurrentRecordId() throws NoSuchElementException {
+ return new byte[0];
+ }
+
+ @Override
+ public String getCurrent() throws NoSuchElementException {
+ return this.currentRecord;
+ }
+
+ @Override
+ public Instant getCurrentTimestamp() throws NoSuchElementException {
+ return Instant.now();
+ }
+
+ @Override
+ public void close() throws IOException {
+ this.reader.close();
+ this.socket.close();
+ this.isRunning = false;
+ LOG.info("Closed connection to server socket at " + this.source.getHostname() + ":" + this.source.getPort() + ".");
+ }
+
+ @Override
+ public Instant getWatermark() {
+ return Instant.now();
+ }
+
+ @Override
+ public CheckpointMark getCheckpointMark() {
+ return null;
+ }
+
+ @Override
+ public UnboundedSource<String, ?> getCurrentSource() {
+ return this.source;
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
index e065f87..5a89894 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
@@ -38,95 +38,95 @@ import org.joda.time.Instant;
* */
public class UnboundedSourceWrapper<T> extends RichSourceFunction<WindowedValue<T>> implements Triggerable {
- private final String name;
- private final UnboundedSource.UnboundedReader<T> reader;
-
- private StreamingRuntimeContext runtime = null;
- private StreamSource.ManualWatermarkContext<WindowedValue<T>> context = null;
-
- private volatile boolean isRunning = false;
-
- public UnboundedSourceWrapper(PipelineOptions options, Read.Unbounded<T> transform) {
- this.name = transform.getName();
- this.reader = transform.getSource().createReader(options, null);
- }
-
- public String getName() {
- return this.name;
- }
-
- WindowedValue<T> makeWindowedValue(T output, Instant timestamp) {
- if (timestamp == null) {
- timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
- }
- return WindowedValue.of(output, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING);
- }
-
- @Override
- public void run(SourceContext<WindowedValue<T>> ctx) throws Exception {
- if (!(ctx instanceof StreamSource.ManualWatermarkContext)) {
- throw new RuntimeException("We assume that all sources in Dataflow are EventTimeSourceFunction. " +
- "Apparently " + this.name + " is not. Probably you should consider writing your own Wrapper for this source.");
- }
-
- context = (StreamSource.ManualWatermarkContext<WindowedValue<T>>) ctx;
- runtime = (StreamingRuntimeContext) getRuntimeContext();
-
- this.isRunning = true;
- boolean inputAvailable = reader.start();
-
- setNextWatermarkTimer(this.runtime);
-
- while (isRunning) {
-
- while (!inputAvailable && isRunning) {
- // wait a bit until we retry to pull more records
- Thread.sleep(50);
- inputAvailable = reader.advance();
- }
-
- if (inputAvailable) {
-
- // get it and its timestamp from the source
- T item = reader.getCurrent();
- Instant timestamp = reader.getCurrentTimestamp();
-
- // write it to the output collector
- synchronized (ctx.getCheckpointLock()) {
- context.collectWithTimestamp(makeWindowedValue(item, timestamp), timestamp.getMillis());
- }
-
- inputAvailable = reader.advance();
- }
-
- }
- }
-
- @Override
- public void cancel() {
- isRunning = false;
- }
-
- @Override
- public void trigger(long timestamp) throws Exception {
- if (this.isRunning) {
- synchronized (context.getCheckpointLock()) {
- long watermarkMillis = this.reader.getWatermark().getMillis();
- context.emitWatermark(new Watermark(watermarkMillis));
- }
- setNextWatermarkTimer(this.runtime);
- }
- }
-
- private void setNextWatermarkTimer(StreamingRuntimeContext runtime) {
- if (this.isRunning) {
- long watermarkInterval = runtime.getExecutionConfig().getAutoWatermarkInterval();
- long timeToNextWatermark = getTimeToNextWaternark(watermarkInterval);
- runtime.registerTimer(timeToNextWatermark, this);
- }
- }
-
- private long getTimeToNextWaternark(long watermarkInterval) {
- return System.currentTimeMillis() + watermarkInterval;
- }
+ private final String name;
+ private final UnboundedSource.UnboundedReader<T> reader;
+
+ private StreamingRuntimeContext runtime = null;
+ private StreamSource.ManualWatermarkContext<WindowedValue<T>> context = null;
+
+ private volatile boolean isRunning = false;
+
+ public UnboundedSourceWrapper(PipelineOptions options, Read.Unbounded<T> transform) {
+ this.name = transform.getName();
+ this.reader = transform.getSource().createReader(options, null);
+ }
+
+ public String getName() {
+ return this.name;
+ }
+
+ WindowedValue<T> makeWindowedValue(T output, Instant timestamp) {
+ if (timestamp == null) {
+ timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+ }
+ return WindowedValue.of(output, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING);
+ }
+
+ @Override
+ public void run(SourceContext<WindowedValue<T>> ctx) throws Exception {
+ if (!(ctx instanceof StreamSource.ManualWatermarkContext)) {
+ throw new RuntimeException("We assume that all sources in Dataflow are EventTimeSourceFunction. " +
+ "Apparently " + this.name + " is not. Probably you should consider writing your own Wrapper for this source.");
+ }
+
+ context = (StreamSource.ManualWatermarkContext<WindowedValue<T>>) ctx;
+ runtime = (StreamingRuntimeContext) getRuntimeContext();
+
+ this.isRunning = true;
+ boolean inputAvailable = reader.start();
+
+ setNextWatermarkTimer(this.runtime);
+
+ while (isRunning) {
+
+ while (!inputAvailable && isRunning) {
+ // wait a bit until we retry to pull more records
+ Thread.sleep(50);
+ inputAvailable = reader.advance();
+ }
+
+ if (inputAvailable) {
+
+ // get it and its timestamp from the source
+ T item = reader.getCurrent();
+ Instant timestamp = reader.getCurrentTimestamp();
+
+ // write it to the output collector
+ synchronized (ctx.getCheckpointLock()) {
+ context.collectWithTimestamp(makeWindowedValue(item, timestamp), timestamp.getMillis());
+ }
+
+ inputAvailable = reader.advance();
+ }
+
+ }
+ }
+
+ @Override
+ public void cancel() {
+ isRunning = false;
+ }
+
+ @Override
+ public void trigger(long timestamp) throws Exception {
+ if (this.isRunning) {
+ synchronized (context.getCheckpointLock()) {
+ long watermarkMillis = this.reader.getWatermark().getMillis();
+ context.emitWatermark(new Watermark(watermarkMillis));
+ }
+ setNextWatermarkTimer(this.runtime);
+ }
+ }
+
+ private void setNextWatermarkTimer(StreamingRuntimeContext runtime) {
+ if (this.isRunning) {
+ long watermarkInterval = runtime.getExecutionConfig().getAutoWatermarkInterval();
+ long timeToNextWatermark = getTimeToNextWaternark(watermarkInterval);
+ runtime.registerTimer(timeToNextWatermark, this);
+ }
+ }
+
+ private long getTimeToNextWaternark(long watermarkInterval) {
+ return System.currentTimeMillis() + watermarkInterval;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
index 84a322f..75c8ac6 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
@@ -34,93 +34,93 @@ import java.io.Serializable;
* The latter is used when snapshots of the current state are taken, for fault-tolerance.
* */
public abstract class AbstractFlinkTimerInternals<K, VIN> implements TimerInternals, Serializable {
- private Instant currentInputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE;
- private Instant currentOutputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE;
-
- public void setCurrentInputWatermark(Instant watermark) {
- checkIfValidInputWatermark(watermark);
- this.currentInputWatermark = watermark;
- }
-
- public void setCurrentOutputWatermark(Instant watermark) {
- checkIfValidOutputWatermark(watermark);
- this.currentOutputWatermark = watermark;
- }
-
- private void setCurrentInputWatermarkAfterRecovery(Instant watermark) {
- if (!currentInputWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) {
- throw new RuntimeException("Explicitly setting the input watermark is only allowed on " +
- "initialization after recovery from a node failure. Apparently this is not " +
- "the case here as the watermark is already set.");
- }
- this.currentInputWatermark = watermark;
- }
-
- private void setCurrentOutputWatermarkAfterRecovery(Instant watermark) {
- if (!currentOutputWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) {
- throw new RuntimeException("Explicitly setting the output watermark is only allowed on " +
- "initialization after recovery from a node failure. Apparently this is not " +
- "the case here as the watermark is already set.");
- }
- this.currentOutputWatermark = watermark;
- }
-
- @Override
- public Instant currentProcessingTime() {
- return Instant.now();
- }
-
- @Override
- public Instant currentInputWatermarkTime() {
- return currentInputWatermark;
- }
-
- @Nullable
- @Override
- public Instant currentSynchronizedProcessingTime() {
- // TODO
- return null;
- }
-
- @Override
- public Instant currentOutputWatermarkTime() {
- return currentOutputWatermark;
- }
-
- private void checkIfValidInputWatermark(Instant newWatermark) {
- if (currentInputWatermark.isAfter(newWatermark)) {
- throw new IllegalArgumentException(String.format(
- "Cannot set current input watermark to %s. Newer watermarks " +
- "must be no earlier than the current one (%s).",
- newWatermark, currentInputWatermark));
- }
- }
-
- private void checkIfValidOutputWatermark(Instant newWatermark) {
- if (currentOutputWatermark.isAfter(newWatermark)) {
- throw new IllegalArgumentException(String.format(
- "Cannot set current output watermark to %s. Newer watermarks " +
- "must be no earlier than the current one (%s).",
- newWatermark, currentOutputWatermark));
- }
- }
-
- public void encodeTimerInternals(DoFn.ProcessContext context,
- StateCheckpointWriter writer,
- KvCoder<K, VIN> kvCoder,
- Coder<? extends BoundedWindow> windowCoder) throws IOException {
- if (context == null) {
- throw new RuntimeException("The Context has not been initialized.");
- }
-
- writer.setTimestamp(currentInputWatermark);
- writer.setTimestamp(currentOutputWatermark);
- }
-
- public void restoreTimerInternals(StateCheckpointReader reader,
- KvCoder<K, VIN> kvCoder,
- Coder<? extends BoundedWindow> windowCoder) throws IOException {
- setCurrentInputWatermarkAfterRecovery(reader.getTimestamp());
- setCurrentOutputWatermarkAfterRecovery(reader.getTimestamp());
- }
+ private Instant currentInputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE;
+ private Instant currentOutputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE;
+
+ public void setCurrentInputWatermark(Instant watermark) {
+ checkIfValidInputWatermark(watermark);
+ this.currentInputWatermark = watermark;
+ }
+
+ public void setCurrentOutputWatermark(Instant watermark) {
+ checkIfValidOutputWatermark(watermark);
+ this.currentOutputWatermark = watermark;
+ }
+
+ private void setCurrentInputWatermarkAfterRecovery(Instant watermark) {
+ if (!currentInputWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) {
+ throw new RuntimeException("Explicitly setting the input watermark is only allowed on " +
+ "initialization after recovery from a node failure. Apparently this is not " +
+ "the case here as the watermark is already set.");
+ }
+ this.currentInputWatermark = watermark;
+ }
+
+ private void setCurrentOutputWatermarkAfterRecovery(Instant watermark) {
+ if (!currentOutputWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) {
+ throw new RuntimeException("Explicitly setting the output watermark is only allowed on " +
+ "initialization after recovery from a node failure. Apparently this is not " +
+ "the case here as the watermark is already set.");
+ }
+ this.currentOutputWatermark = watermark;
+ }
+
+ @Override
+ public Instant currentProcessingTime() {
+ return Instant.now();
+ }
+
+ @Override
+ public Instant currentInputWatermarkTime() {
+ return currentInputWatermark;
+ }
+
+ @Nullable
+ @Override
+ public Instant currentSynchronizedProcessingTime() {
+ // TODO
+ return null;
+ }
+
+ @Override
+ public Instant currentOutputWatermarkTime() {
+ return currentOutputWatermark;
+ }
+
+ private void checkIfValidInputWatermark(Instant newWatermark) {
+ if (currentInputWatermark.isAfter(newWatermark)) {
+ throw new IllegalArgumentException(String.format(
+ "Cannot set current input watermark to %s. Newer watermarks " +
+ "must be no earlier than the current one (%s).",
+ newWatermark, currentInputWatermark));
+ }
+ }
+
+ private void checkIfValidOutputWatermark(Instant newWatermark) {
+ if (currentOutputWatermark.isAfter(newWatermark)) {
+ throw new IllegalArgumentException(String.format(
+ "Cannot set current output watermark to %s. Newer watermarks " +
+ "must be no earlier than the current one (%s).",
+ newWatermark, currentOutputWatermark));
+ }
+ }
+
+ public void encodeTimerInternals(DoFn.ProcessContext context,
+ StateCheckpointWriter writer,
+ KvCoder<K, VIN> kvCoder,
+ Coder<? extends BoundedWindow> windowCoder) throws IOException {
+ if (context == null) {
+ throw new RuntimeException("The Context has not been initialized.");
+ }
+
+ writer.setTimestamp(currentInputWatermark);
+ writer.setTimestamp(currentOutputWatermark);
+ }
+
+ public void restoreTimerInternals(StateCheckpointReader reader,
+ KvCoder<K, VIN> kvCoder,
+ Coder<? extends BoundedWindow> windowCoder) throws IOException {
+ setCurrentInputWatermarkAfterRecovery(reader.getTimestamp());
+ setCurrentOutputWatermarkAfterRecovery(reader.getTimestamp());
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java
index 41ab5f0..39fec14 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java
@@ -41,673 +41,673 @@ import java.util.*;
*/
public class FlinkStateInternals<K> implements StateInternals<K> {
- private final K key;
-
- private final Coder<K> keyCoder;
-
- private final Coder<? extends BoundedWindow> windowCoder;
-
- private final OutputTimeFn<? super BoundedWindow> outputTimeFn;
-
- private Instant watermarkHoldAccessor;
-
- public FlinkStateInternals(K key,
- Coder<K> keyCoder,
- Coder<? extends BoundedWindow> windowCoder,
- OutputTimeFn<? super BoundedWindow> outputTimeFn) {
- this.key = key;
- this.keyCoder = keyCoder;
- this.windowCoder = windowCoder;
- this.outputTimeFn = outputTimeFn;
- }
-
- public Instant getWatermarkHold() {
- return watermarkHoldAccessor;
- }
-
- /**
- * This is the interface state has to implement in order for it to be fault tolerant when
- * executed by the FlinkPipelineRunner.
- */
- private interface CheckpointableIF {
-
- boolean shouldPersist();
-
- void persistState(StateCheckpointWriter checkpointBuilder) throws IOException;
- }
-
- protected final StateTable<K> inMemoryState = new StateTable<K>() {
- @Override
- protected StateTag.StateBinder binderForNamespace(final StateNamespace namespace, final StateContext<?> c) {
- return new StateTag.StateBinder<K>() {
-
- @Override
- public <T> ValueState<T> bindValue(StateTag<? super K, ValueState<T>> address, Coder<T> coder) {
- return new FlinkInMemoryValue<>(encodeKey(namespace, address), coder);
- }
-
- @Override
- public <T> BagState<T> bindBag(StateTag<? super K, BagState<T>> address, Coder<T> elemCoder) {
- return new FlinkInMemoryBag<>(encodeKey(namespace, address), elemCoder);
- }
-
- @Override
- public <InputT, AccumT, OutputT> AccumulatorCombiningState<InputT, AccumT, OutputT> bindCombiningValue(
- StateTag<? super K, AccumulatorCombiningState<InputT, AccumT, OutputT>> address,
- Coder<AccumT> accumCoder, Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
- return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c);
- }
-
- @Override
- public <InputT, AccumT, OutputT> AccumulatorCombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValue(
- StateTag<? super K, AccumulatorCombiningState<InputT, AccumT, OutputT>> address,
- Coder<AccumT> accumCoder,
- Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn) {
- return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c);
- }
-
- @Override
- public <InputT, AccumT, OutputT> AccumulatorCombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValueWithContext(
- StateTag<? super K, AccumulatorCombiningState<InputT, AccumT, OutputT>> address,
- Coder<AccumT> accumCoder,
- CombineWithContext.KeyedCombineFnWithContext<? super K, InputT, AccumT, OutputT> combineFn) {
- return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c);
- }
-
- @Override
- public <W extends BoundedWindow> WatermarkHoldState<W> bindWatermark(StateTag<? super K, WatermarkHoldState<W>> address, OutputTimeFn<? super W> outputTimeFn) {
- return new FlinkWatermarkHoldStateImpl<>(encodeKey(namespace, address), outputTimeFn);
- }
- };
- }
- };
-
- @Override
- public K getKey() {
- return key;
- }
-
- @Override
- public <StateT extends State> StateT state(StateNamespace namespace, StateTag<? super K, StateT> address) {
- return inMemoryState.get(namespace, address, null);
- }
-
- @Override
- public <T extends State> T state(StateNamespace namespace, StateTag<? super K, T> address, StateContext<?> c) {
- return inMemoryState.get(namespace, address, c);
- }
-
- public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
- checkpointBuilder.writeInt(getNoOfElements());
-
- for (State location : inMemoryState.values()) {
- if (!(location instanceof CheckpointableIF)) {
- throw new IllegalStateException(String.format(
- "%s wasn't created by %s -- unable to persist it",
- location.getClass().getSimpleName(),
- getClass().getSimpleName()));
- }
- ((CheckpointableIF) location).persistState(checkpointBuilder);
- }
- }
-
- public void restoreState(StateCheckpointReader checkpointReader, ClassLoader loader)
- throws IOException, ClassNotFoundException {
-
- // the number of elements to read.
- int noOfElements = checkpointReader.getInt();
- for (int i = 0; i < noOfElements; i++) {
- decodeState(checkpointReader, loader);
- }
- }
-
- /**
- * We remove the first character which encodes the type of the stateTag ('s' for system
- * and 'u' for user). For more details check out the source of
- * {@link StateTags.StateTagBase#getId()}.
- */
- private void decodeState(StateCheckpointReader reader, ClassLoader loader)
- throws IOException, ClassNotFoundException {
-
- StateType stateItemType = StateType.deserialize(reader);
- ByteString stateKey = reader.getTag();
-
- // first decode the namespace and the tagId...
- String[] namespaceAndTag = stateKey.toStringUtf8().split("\\+");
- if (namespaceAndTag.length != 2) {
- throw new IllegalArgumentException("Invalid stateKey " + stateKey.toString() + ".");
- }
- StateNamespace namespace = StateNamespaces.fromString(namespaceAndTag[0], windowCoder);
-
- // ... decide if it is a system or user stateTag...
- char ownerTag = namespaceAndTag[1].charAt(0);
- if (ownerTag != 's' && ownerTag != 'u') {
- throw new RuntimeException("Invalid StateTag name.");
- }
- boolean isSystemTag = ownerTag == 's';
- String tagId = namespaceAndTag[1].substring(1);
-
- // ...then decode the coder (if there is one)...
- Coder<?> coder = null;
- switch (stateItemType) {
- case VALUE:
- case LIST:
- case ACCUMULATOR:
- ByteString coderBytes = reader.getData();
- coder = InstantiationUtil.deserializeObject(coderBytes.toByteArray(), loader);
- break;
- case WATERMARK:
- break;
- }
-
- // ...then decode the combiner function (if there is one)...
- CombineWithContext.KeyedCombineFnWithContext<? super K, ?, ?, ?> combineFn = null;
- switch (stateItemType) {
- case ACCUMULATOR:
- ByteString combinerBytes = reader.getData();
- combineFn = InstantiationUtil.deserializeObject(combinerBytes.toByteArray(), loader);
- break;
- case VALUE:
- case LIST:
- case WATERMARK:
- break;
- }
-
- //... and finally, depending on the type of the state being decoded,
- // 1) create the adequate stateTag,
- // 2) create the state container,
- // 3) restore the actual content.
- switch (stateItemType) {
- case VALUE: {
- StateTag stateTag = StateTags.value(tagId, coder);
- stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
- @SuppressWarnings("unchecked")
- FlinkInMemoryValue<?> value = (FlinkInMemoryValue<?>) inMemoryState.get(namespace, stateTag, null);
- value.restoreState(reader);
- break;
- }
- case WATERMARK: {
- @SuppressWarnings("unchecked")
- StateTag<Object, WatermarkHoldState<BoundedWindow>> stateTag = StateTags.watermarkStateInternal(tagId, outputTimeFn);
- stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
- @SuppressWarnings("unchecked")
- FlinkWatermarkHoldStateImpl<?> watermark = (FlinkWatermarkHoldStateImpl<?>) inMemoryState.get(namespace, stateTag, null);
- watermark.restoreState(reader);
- break;
- }
- case LIST: {
- StateTag stateTag = StateTags.bag(tagId, coder);
- stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
- FlinkInMemoryBag<?> bag = (FlinkInMemoryBag<?>) inMemoryState.get(namespace, stateTag, null);
- bag.restoreState(reader);
- break;
- }
- case ACCUMULATOR: {
- @SuppressWarnings("unchecked")
- StateTag<K, AccumulatorCombiningState<?, ?, ?>> stateTag = StateTags.keyedCombiningValueWithContext(tagId, (Coder) coder, combineFn);
- stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
- @SuppressWarnings("unchecked")
- FlinkInMemoryKeyedCombiningValue<?, ?, ?> combiningValue =
- (FlinkInMemoryKeyedCombiningValue<?, ?, ?>) inMemoryState.get(namespace, stateTag, null);
- combiningValue.restoreState(reader);
- break;
- }
- default:
- throw new RuntimeException("Unknown State Type " + stateItemType + ".");
- }
- }
-
- private ByteString encodeKey(StateNamespace namespace, StateTag<? super K, ?> address) {
- StringBuilder sb = new StringBuilder();
- try {
- namespace.appendTo(sb);
- sb.append('+');
- address.appendTo(sb);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- return ByteString.copyFromUtf8(sb.toString());
- }
-
- private int getNoOfElements() {
- int noOfElements = 0;
- for (State state : inMemoryState.values()) {
- if (!(state instanceof CheckpointableIF)) {
- throw new RuntimeException("State Implementations used by the " +
- "Flink Dataflow Runner should implement the CheckpointableIF interface.");
- }
-
- if (((CheckpointableIF) state).shouldPersist()) {
- noOfElements++;
- }
- }
- return noOfElements;
- }
-
- private final class FlinkInMemoryValue<T> implements ValueState<T>, CheckpointableIF {
-
- private final ByteString stateKey;
- private final Coder<T> elemCoder;
-
- private T value = null;
-
- public FlinkInMemoryValue(ByteString stateKey, Coder<T> elemCoder) {
- this.stateKey = stateKey;
- this.elemCoder = elemCoder;
- }
-
- @Override
- public void clear() {
- value = null;
- }
-
- @Override
- public void write(T input) {
- this.value = input;
- }
-
- @Override
- public T read() {
- return value;
- }
-
- @Override
- public ValueState<T> readLater() {
- // Ignore
- return this;
- }
-
- @Override
- public boolean shouldPersist() {
- return value != null;
- }
-
- @Override
- public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
- if (value != null) {
- // serialize the coder.
- byte[] coder = InstantiationUtil.serializeObject(elemCoder);
-
- // encode the value into a ByteString
- ByteString.Output stream = ByteString.newOutput();
- elemCoder.encode(value, stream, Coder.Context.OUTER);
- ByteString data = stream.toByteString();
-
- checkpointBuilder.addValueBuilder()
- .setTag(stateKey)
- .setData(coder)
- .setData(data);
- }
- }
-
- public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
- ByteString valueContent = checkpointReader.getData();
- T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER);
- write(outValue);
- }
- }
-
- private final class FlinkWatermarkHoldStateImpl<W extends BoundedWindow>
- implements WatermarkHoldState<W>, CheckpointableIF {
-
- private final ByteString stateKey;
-
- private Instant minimumHold = null;
-
- private OutputTimeFn<? super W> outputTimeFn;
-
- public FlinkWatermarkHoldStateImpl(ByteString stateKey, OutputTimeFn<? super W> outputTimeFn) {
- this.stateKey = stateKey;
- this.outputTimeFn = outputTimeFn;
- }
-
- @Override
- public void clear() {
- // Even though we're clearing we can't remove this from the in-memory state map, since
- // other users may already have a handle on this WatermarkBagInternal.
- minimumHold = null;
- watermarkHoldAccessor = null;
- }
-
- @Override
- public void add(Instant watermarkHold) {
- if (minimumHold == null || minimumHold.isAfter(watermarkHold)) {
- watermarkHoldAccessor = watermarkHold;
- minimumHold = watermarkHold;
- }
- }
-
- @Override
- public ReadableState<Boolean> isEmpty() {
- return new ReadableState<Boolean>() {
- @Override
- public Boolean read() {
- return minimumHold == null;
- }
-
- @Override
- public ReadableState<Boolean> readLater() {
- // Ignore
- return this;
- }
- };
- }
-
- @Override
- public OutputTimeFn<? super W> getOutputTimeFn() {
- return outputTimeFn;
- }
-
- @Override
- public Instant read() {
- return minimumHold;
- }
-
- @Override
- public WatermarkHoldState<W> readLater() {
- // Ignore
- return this;
- }
-
- @Override
- public String toString() {
- return Objects.toString(minimumHold);
- }
-
- @Override
- public boolean shouldPersist() {
- return minimumHold != null;
- }
-
- @Override
- public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
- if (minimumHold != null) {
- checkpointBuilder.addWatermarkHoldsBuilder()
- .setTag(stateKey)
- .setTimestamp(minimumHold);
- }
- }
-
- public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
- Instant watermark = checkpointReader.getTimestamp();
- add(watermark);
- }
- }
-
-
- private static <K, InputT, AccumT, OutputT> CombineWithContext.KeyedCombineFnWithContext<K, InputT, AccumT, OutputT> withContext(
- final Combine.KeyedCombineFn<K, InputT, AccumT, OutputT> combineFn) {
- return new CombineWithContext.KeyedCombineFnWithContext<K, InputT, AccumT, OutputT>() {
- @Override
- public AccumT createAccumulator(K key, CombineWithContext.Context c) {
- return combineFn.createAccumulator(key);
- }
-
- @Override
- public AccumT addInput(K key, AccumT accumulator, InputT value, CombineWithContext.Context c) {
- return combineFn.addInput(key, accumulator, value);
- }
-
- @Override
- public AccumT mergeAccumulators(K key, Iterable<AccumT> accumulators, CombineWithContext.Context c) {
- return combineFn.mergeAccumulators(key, accumulators);
- }
-
- @Override
- public OutputT extractOutput(K key, AccumT accumulator, CombineWithContext.Context c) {
- return combineFn.extractOutput(key, accumulator);
- }
- };
- }
-
- private static <K, InputT, AccumT, OutputT> CombineWithContext.KeyedCombineFnWithContext<K, InputT, AccumT, OutputT> withKeyAndContext(
- final Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
- return new CombineWithContext.KeyedCombineFnWithContext<K, InputT, AccumT, OutputT>() {
- @Override
- public AccumT createAccumulator(K key, CombineWithContext.Context c) {
- return combineFn.createAccumulator();
- }
-
- @Override
- public AccumT addInput(K key, AccumT accumulator, InputT value, CombineWithContext.Context c) {
- return combineFn.addInput(accumulator, value);
- }
-
- @Override
- public AccumT mergeAccumulators(K key, Iterable<AccumT> accumulators, CombineWithContext.Context c) {
- return combineFn.mergeAccumulators(accumulators);
- }
-
- @Override
- public OutputT extractOutput(K key, AccumT accumulator, CombineWithContext.Context c) {
- return combineFn.extractOutput(accumulator);
- }
- };
- }
-
- private final class FlinkInMemoryKeyedCombiningValue<InputT, AccumT, OutputT>
- implements AccumulatorCombiningState<InputT, AccumT, OutputT>, CheckpointableIF {
-
- private final ByteString stateKey;
- private final CombineWithContext.KeyedCombineFnWithContext<? super K, InputT, AccumT, OutputT> combineFn;
- private final Coder<AccumT> accumCoder;
- private final CombineWithContext.Context context;
-
- private AccumT accum = null;
- private boolean isClear = true;
-
- private FlinkInMemoryKeyedCombiningValue(ByteString stateKey,
- Combine.CombineFn<InputT, AccumT, OutputT> combineFn,
- Coder<AccumT> accumCoder,
- final StateContext<?> stateContext) {
- this(stateKey, withKeyAndContext(combineFn), accumCoder, stateContext);
- }
-
-
- private FlinkInMemoryKeyedCombiningValue(ByteString stateKey,
- Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn,
- Coder<AccumT> accumCoder,
- final StateContext<?> stateContext) {
- this(stateKey, withContext(combineFn), accumCoder, stateContext);
- }
-
- private FlinkInMemoryKeyedCombiningValue(ByteString stateKey,
- CombineWithContext.KeyedCombineFnWithContext<? super K, InputT, AccumT, OutputT> combineFn,
- Coder<AccumT> accumCoder,
- final StateContext<?> stateContext) {
- Preconditions.checkNotNull(combineFn);
- Preconditions.checkNotNull(accumCoder);
-
- this.stateKey = stateKey;
- this.combineFn = combineFn;
- this.accumCoder = accumCoder;
- this.context = new CombineWithContext.Context() {
- @Override
- public PipelineOptions getPipelineOptions() {
- return stateContext.getPipelineOptions();
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view) {
- return stateContext.sideInput(view);
- }
- };
- accum = combineFn.createAccumulator(key, context);
- }
-
- @Override
- public void clear() {
- accum = combineFn.createAccumulator(key, context);
- isClear = true;
- }
-
- @Override
- public void add(InputT input) {
- isClear = false;
- accum = combineFn.addInput(key, accum, input, context);
- }
-
- @Override
- public AccumT getAccum() {
- return accum;
- }
-
- @Override
- public ReadableState<Boolean> isEmpty() {
- return new ReadableState<Boolean>() {
- @Override
- public ReadableState<Boolean> readLater() {
- // Ignore
- return this;
- }
-
- @Override
- public Boolean read() {
- return isClear;
- }
- };
- }
-
- @Override
- public void addAccum(AccumT accum) {
- isClear = false;
- this.accum = combineFn.mergeAccumulators(key, Arrays.asList(this.accum, accum), context);
- }
-
- @Override
- public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
- return combineFn.mergeAccumulators(key, accumulators, context);
- }
-
- @Override
- public OutputT read() {
- return combineFn.extractOutput(key, accum, context);
- }
-
- @Override
- public AccumulatorCombiningState<InputT, AccumT, OutputT> readLater() {
- // Ignore
- return this;
- }
-
- @Override
- public boolean shouldPersist() {
- return !isClear;
- }
-
- @Override
- public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
- if (!isClear) {
- // serialize the coder.
- byte[] coder = InstantiationUtil.serializeObject(accumCoder);
-
- // serialize the combiner.
- byte[] combiner = InstantiationUtil.serializeObject(combineFn);
-
- // encode the accumulator into a ByteString
- ByteString.Output stream = ByteString.newOutput();
- accumCoder.encode(accum, stream, Coder.Context.OUTER);
- ByteString data = stream.toByteString();
-
- // put the flag that the next serialized element is an accumulator
- checkpointBuilder.addAccumulatorBuilder()
- .setTag(stateKey)
- .setData(coder)
- .setData(combiner)
- .setData(data);
- }
- }
-
- public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
- ByteString valueContent = checkpointReader.getData();
- AccumT accum = this.accumCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER);
- addAccum(accum);
- }
- }
-
- private static final class FlinkInMemoryBag<T> implements BagState<T>, CheckpointableIF {
- private final List<T> contents = new ArrayList<>();
-
- private final ByteString stateKey;
- private final Coder<T> elemCoder;
-
- public FlinkInMemoryBag(ByteString stateKey, Coder<T> elemCoder) {
- this.stateKey = stateKey;
- this.elemCoder = elemCoder;
- }
-
- @Override
- public void clear() {
- contents.clear();
- }
-
- @Override
- public Iterable<T> read() {
- return contents;
- }
-
- @Override
- public BagState<T> readLater() {
- // Ignore
- return this;
- }
-
- @Override
- public void add(T input) {
- contents.add(input);
- }
-
- @Override
- public ReadableState<Boolean> isEmpty() {
- return new ReadableState<Boolean>() {
- @Override
- public ReadableState<Boolean> readLater() {
- // Ignore
- return this;
- }
-
- @Override
- public Boolean read() {
- return contents.isEmpty();
- }
- };
- }
-
- @Override
- public boolean shouldPersist() {
- return !contents.isEmpty();
- }
-
- @Override
- public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
- if (!contents.isEmpty()) {
- // serialize the coder.
- byte[] coder = InstantiationUtil.serializeObject(elemCoder);
-
- checkpointBuilder.addListUpdatesBuilder()
- .setTag(stateKey)
- .setData(coder)
- .writeInt(contents.size());
-
- for (T item : contents) {
- // encode the element
- ByteString.Output stream = ByteString.newOutput();
- elemCoder.encode(item, stream, Coder.Context.OUTER);
- ByteString data = stream.toByteString();
-
- // add the data to the checkpoint.
- checkpointBuilder.setData(data);
- }
- }
- }
-
- public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
- int noOfValues = checkpointReader.getInt();
- for (int j = 0; j < noOfValues; j++) {
- ByteString valueContent = checkpointReader.getData();
- T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER);
- add(outValue);
- }
- }
- }
+ private final K key;
+
+ private final Coder<K> keyCoder;
+
+ private final Coder<? extends BoundedWindow> windowCoder;
+
+ private final OutputTimeFn<? super BoundedWindow> outputTimeFn;
+
+ private Instant watermarkHoldAccessor;
+
+ public FlinkStateInternals(K key,
+ Coder<K> keyCoder,
+ Coder<? extends BoundedWindow> windowCoder,
+ OutputTimeFn<? super BoundedWindow> outputTimeFn) {
+ this.key = key;
+ this.keyCoder = keyCoder;
+ this.windowCoder = windowCoder;
+ this.outputTimeFn = outputTimeFn;
+ }
+
+ public Instant getWatermarkHold() {
+ return watermarkHoldAccessor;
+ }
+
+ /**
+ * This is the interface state has to implement in order for it to be fault tolerant when
+ * executed by the FlinkPipelineRunner.
+ */
+ private interface CheckpointableIF {
+
+ boolean shouldPersist();
+
+ void persistState(StateCheckpointWriter checkpointBuilder) throws IOException;
+ }
+
+ protected final StateTable<K> inMemoryState = new StateTable<K>() {
+ @Override
+ protected StateTag.StateBinder binderForNamespace(final StateNamespace namespace, final StateContext<?> c) {
+ return new StateTag.StateBinder<K>() {
+
+ @Override
+ public <T> ValueState<T> bindValue(StateTag<? super K, ValueState<T>> address, Coder<T> coder) {
+ return new FlinkInMemoryValue<>(encodeKey(namespace, address), coder);
+ }
+
+ @Override
+ public <T> BagState<T> bindBag(StateTag<? super K, BagState<T>> address, Coder<T> elemCoder) {
+ return new FlinkInMemoryBag<>(encodeKey(namespace, address), elemCoder);
+ }
+
+ @Override
+ public <InputT, AccumT, OutputT> AccumulatorCombiningState<InputT, AccumT, OutputT> bindCombiningValue(
+ StateTag<? super K, AccumulatorCombiningState<InputT, AccumT, OutputT>> address,
+ Coder<AccumT> accumCoder, Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
+ return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c);
+ }
+
+ @Override
+ public <InputT, AccumT, OutputT> AccumulatorCombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValue(
+ StateTag<? super K, AccumulatorCombiningState<InputT, AccumT, OutputT>> address,
+ Coder<AccumT> accumCoder,
+ Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn) {
+ return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c);
+ }
+
+ @Override
+ public <InputT, AccumT, OutputT> AccumulatorCombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValueWithContext(
+ StateTag<? super K, AccumulatorCombiningState<InputT, AccumT, OutputT>> address,
+ Coder<AccumT> accumCoder,
+ CombineWithContext.KeyedCombineFnWithContext<? super K, InputT, AccumT, OutputT> combineFn) {
+ return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c);
+ }
+
+ @Override
+ public <W extends BoundedWindow> WatermarkHoldState<W> bindWatermark(StateTag<? super K, WatermarkHoldState<W>> address, OutputTimeFn<? super W> outputTimeFn) {
+ return new FlinkWatermarkHoldStateImpl<>(encodeKey(namespace, address), outputTimeFn);
+ }
+ };
+ }
+ };
+
+ @Override
+ public K getKey() {
+ return key;
+ }
+
+ @Override
+ public <StateT extends State> StateT state(StateNamespace namespace, StateTag<? super K, StateT> address) {
+ return inMemoryState.get(namespace, address, null);
+ }
+
+ @Override
+ public <T extends State> T state(StateNamespace namespace, StateTag<? super K, T> address, StateContext<?> c) {
+ return inMemoryState.get(namespace, address, c);
+ }
+
+ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
+ checkpointBuilder.writeInt(getNoOfElements());
+
+ for (State location : inMemoryState.values()) {
+ if (!(location instanceof CheckpointableIF)) {
+ throw new IllegalStateException(String.format(
+ "%s wasn't created by %s -- unable to persist it",
+ location.getClass().getSimpleName(),
+ getClass().getSimpleName()));
+ }
+ ((CheckpointableIF) location).persistState(checkpointBuilder);
+ }
+ }
+
+ public void restoreState(StateCheckpointReader checkpointReader, ClassLoader loader)
+ throws IOException, ClassNotFoundException {
+
+ // the number of elements to read.
+ int noOfElements = checkpointReader.getInt();
+ for (int i = 0; i < noOfElements; i++) {
+ decodeState(checkpointReader, loader);
+ }
+ }
+
+ /**
+ * We remove the first character which encodes the type of the stateTag ('s' for system
+ * and 'u' for user). For more details check out the source of
+ * {@link StateTags.StateTagBase#getId()}.
+ */
+ private void decodeState(StateCheckpointReader reader, ClassLoader loader)
+ throws IOException, ClassNotFoundException {
+
+ StateType stateItemType = StateType.deserialize(reader);
+ ByteString stateKey = reader.getTag();
+
+ // first decode the namespace and the tagId...
+ String[] namespaceAndTag = stateKey.toStringUtf8().split("\\+");
+ if (namespaceAndTag.length != 2) {
+ throw new IllegalArgumentException("Invalid stateKey " + stateKey.toString() + ".");
+ }
+ StateNamespace namespace = StateNamespaces.fromString(namespaceAndTag[0], windowCoder);
+
+ // ... decide if it is a system or user stateTag...
+ char ownerTag = namespaceAndTag[1].charAt(0);
+ if (ownerTag != 's' && ownerTag != 'u') {
+ throw new RuntimeException("Invalid StateTag name.");
+ }
+ boolean isSystemTag = ownerTag == 's';
+ String tagId = namespaceAndTag[1].substring(1);
+
+ // ...then decode the coder (if there is one)...
+ Coder<?> coder = null;
+ switch (stateItemType) {
+ case VALUE:
+ case LIST:
+ case ACCUMULATOR:
+ ByteString coderBytes = reader.getData();
+ coder = InstantiationUtil.deserializeObject(coderBytes.toByteArray(), loader);
+ break;
+ case WATERMARK:
+ break;
+ }
+
+ // ...then decode the combiner function (if there is one)...
+ CombineWithContext.KeyedCombineFnWithContext<? super K, ?, ?, ?> combineFn = null;
+ switch (stateItemType) {
+ case ACCUMULATOR:
+ ByteString combinerBytes = reader.getData();
+ combineFn = InstantiationUtil.deserializeObject(combinerBytes.toByteArray(), loader);
+ break;
+ case VALUE:
+ case LIST:
+ case WATERMARK:
+ break;
+ }
+
+ //... and finally, depending on the type of the state being decoded,
+ // 1) create the adequate stateTag,
+ // 2) create the state container,
+ // 3) restore the actual content.
+ switch (stateItemType) {
+ case VALUE: {
+ StateTag stateTag = StateTags.value(tagId, coder);
+ stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
+ @SuppressWarnings("unchecked")
+ FlinkInMemoryValue<?> value = (FlinkInMemoryValue<?>) inMemoryState.get(namespace, stateTag, null);
+ value.restoreState(reader);
+ break;
+ }
+ case WATERMARK: {
+ @SuppressWarnings("unchecked")
+ StateTag<Object, WatermarkHoldState<BoundedWindow>> stateTag = StateTags.watermarkStateInternal(tagId, outputTimeFn);
+ stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
+ @SuppressWarnings("unchecked")
+ FlinkWatermarkHoldStateImpl<?> watermark = (FlinkWatermarkHoldStateImpl<?>) inMemoryState.get(namespace, stateTag, null);
+ watermark.restoreState(reader);
+ break;
+ }
+ case LIST: {
+ StateTag stateTag = StateTags.bag(tagId, coder);
+ stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
+ FlinkInMemoryBag<?> bag = (FlinkInMemoryBag<?>) inMemoryState.get(namespace, stateTag, null);
+ bag.restoreState(reader);
+ break;
+ }
+ case ACCUMULATOR: {
+ @SuppressWarnings("unchecked")
+ StateTag<K, AccumulatorCombiningState<?, ?, ?>> stateTag = StateTags.keyedCombiningValueWithContext(tagId, (Coder) coder, combineFn);
+ stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
+ @SuppressWarnings("unchecked")
+ FlinkInMemoryKeyedCombiningValue<?, ?, ?> combiningValue =
+ (FlinkInMemoryKeyedCombiningValue<?, ?, ?>) inMemoryState.get(namespace, stateTag, null);
+ combiningValue.restoreState(reader);
+ break;
+ }
+ default:
+ throw new RuntimeException("Unknown State Type " + stateItemType + ".");
+ }
+ }
+
+ private ByteString encodeKey(StateNamespace namespace, StateTag<? super K, ?> address) {
+ StringBuilder sb = new StringBuilder();
+ try {
+ namespace.appendTo(sb);
+ sb.append('+');
+ address.appendTo(sb);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ return ByteString.copyFromUtf8(sb.toString());
+ }
+
+ private int getNoOfElements() {
+ int noOfElements = 0;
+ for (State state : inMemoryState.values()) {
+ if (!(state instanceof CheckpointableIF)) {
+ throw new RuntimeException("State Implementations used by the " +
+ "Flink Dataflow Runner should implement the CheckpointableIF interface.");
+ }
+
+ if (((CheckpointableIF) state).shouldPersist()) {
+ noOfElements++;
+ }
+ }
+ return noOfElements;
+ }
+
+ private final class FlinkInMemoryValue<T> implements ValueState<T>, CheckpointableIF {
+
+ private final ByteString stateKey;
+ private final Coder<T> elemCoder;
+
+ private T value = null;
+
+ public FlinkInMemoryValue(ByteString stateKey, Coder<T> elemCoder) {
+ this.stateKey = stateKey;
+ this.elemCoder = elemCoder;
+ }
+
+ @Override
+ public void clear() {
+ value = null;
+ }
+
+ @Override
+ public void write(T input) {
+ this.value = input;
+ }
+
+ @Override
+ public T read() {
+ return value;
+ }
+
+ @Override
+ public ValueState<T> readLater() {
+ // Ignore
+ return this;
+ }
+
+ @Override
+ public boolean shouldPersist() {
+ return value != null;
+ }
+
+ @Override
+ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
+ if (value != null) {
+ // serialize the coder.
+ byte[] coder = InstantiationUtil.serializeObject(elemCoder);
+
+ // encode the value into a ByteString
+ ByteString.Output stream = ByteString.newOutput();
+ elemCoder.encode(value, stream, Coder.Context.OUTER);
+ ByteString data = stream.toByteString();
+
+ checkpointBuilder.addValueBuilder()
+ .setTag(stateKey)
+ .setData(coder)
+ .setData(data);
+ }
+ }
+
+ public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
+ ByteString valueContent = checkpointReader.getData();
+ T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER);
+ write(outValue);
+ }
+ }
+
+ private final class FlinkWatermarkHoldStateImpl<W extends BoundedWindow>
+ implements WatermarkHoldState<W>, CheckpointableIF {
+
+ private final ByteString stateKey;
+
+ private Instant minimumHold = null;
+
+ private OutputTimeFn<? super W> outputTimeFn;
+
+ public FlinkWatermarkHoldStateImpl(ByteString stateKey, OutputTimeFn<? super W> outputTimeFn) {
+ this.stateKey = stateKey;
+ this.outputTimeFn = outputTimeFn;
+ }
+
+ @Override
+ public void clear() {
+ // Even though we're clearing we can't remove this from the in-memory state map, since
+ // other users may already have a handle on this WatermarkBagInternal.
+ minimumHold = null;
+ watermarkHoldAccessor = null;
+ }
+
+ @Override
+ public void add(Instant watermarkHold) {
+ if (minimumHold == null || minimumHold.isAfter(watermarkHold)) {
+ watermarkHoldAccessor = watermarkHold;
+ minimumHold = watermarkHold;
+ }
+ }
+
+ @Override
+ public ReadableState<Boolean> isEmpty() {
+ return new ReadableState<Boolean>() {
+ @Override
+ public Boolean read() {
+ return minimumHold == null;
+ }
+
+ @Override
+ public ReadableState<Boolean> readLater() {
+ // Ignore
+ return this;
+ }
+ };
+ }
+
+ @Override
+ public OutputTimeFn<? super W> getOutputTimeFn() {
+ return outputTimeFn;
+ }
+
+ @Override
+ public Instant read() {
+ return minimumHold;
+ }
+
+ @Override
+ public WatermarkHoldState<W> readLater() {
+ // Ignore
+ return this;
+ }
+
+ @Override
+ public String toString() {
+ return Objects.toString(minimumHold);
+ }
+
+ @Override
+ public boolean shouldPersist() {
+ return minimumHold != null;
+ }
+
+ @Override
+ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
+ if (minimumHold != null) {
+ checkpointBuilder.addWatermarkHoldsBuilder()
+ .setTag(stateKey)
+ .setTimestamp(minimumHold);
+ }
+ }
+
+ public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
+ Instant watermark = checkpointReader.getTimestamp();
+ add(watermark);
+ }
+ }
+
+
+ private static <K, InputT, AccumT, OutputT> CombineWithContext.KeyedCombineFnWithContext<K, InputT, AccumT, OutputT> withContext(
+ final Combine.KeyedCombineFn<K, InputT, AccumT, OutputT> combineFn) {
+ return new CombineWithContext.KeyedCombineFnWithContext<K, InputT, AccumT, OutputT>() {
+ @Override
+ public AccumT createAccumulator(K key, CombineWithContext.Context c) {
+ return combineFn.createAccumulator(key);
+ }
+
+ @Override
+ public AccumT addInput(K key, AccumT accumulator, InputT value, CombineWithContext.Context c) {
+ return combineFn.addInput(key, accumulator, value);
+ }
+
+ @Override
+ public AccumT mergeAccumulators(K key, Iterable<AccumT> accumulators, CombineWithContext.Context c) {
+ return combineFn.mergeAccumulators(key, accumulators);
+ }
+
+ @Override
+ public OutputT extractOutput(K key, AccumT accumulator, CombineWithContext.Context c) {
+ return combineFn.extractOutput(key, accumulator);
+ }
+ };
+ }
+
+ private static <K, InputT, AccumT, OutputT> CombineWithContext.KeyedCombineFnWithContext<K, InputT, AccumT, OutputT> withKeyAndContext(
+ final Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
+ return new CombineWithContext.KeyedCombineFnWithContext<K, InputT, AccumT, OutputT>() {
+ @Override
+ public AccumT createAccumulator(K key, CombineWithContext.Context c) {
+ return combineFn.createAccumulator();
+ }
+
+ @Override
+ public AccumT addInput(K key, AccumT accumulator, InputT value, CombineWithContext.Context c) {
+ return combineFn.addInput(accumulator, value);
+ }
+
+ @Override
+ public AccumT mergeAccumulators(K key, Iterable<AccumT> accumulators, CombineWithContext.Context c) {
+ return combineFn.mergeAccumulators(accumulators);
+ }
+
+ @Override
+ public OutputT extractOutput(K key, AccumT accumulator, CombineWithContext.Context c) {
+ return combineFn.extractOutput(accumulator);
+ }
+ };
+ }
+
+ private final class FlinkInMemoryKeyedCombiningValue<InputT, AccumT, OutputT>
+ implements AccumulatorCombiningState<InputT, AccumT, OutputT>, CheckpointableIF {
+
+ private final ByteString stateKey;
+ private final CombineWithContext.KeyedCombineFnWithContext<? super K, InputT, AccumT, OutputT> combineFn;
+ private final Coder<AccumT> accumCoder;
+ private final CombineWithContext.Context context;
+
+ private AccumT accum = null;
+ private boolean isClear = true;
+
+ private FlinkInMemoryKeyedCombiningValue(ByteString stateKey,
+ Combine.CombineFn<InputT, AccumT, OutputT> combineFn,
+ Coder<AccumT> accumCoder,
+ final StateContext<?> stateContext) {
+ this(stateKey, withKeyAndContext(combineFn), accumCoder, stateContext);
+ }
+
+
+ private FlinkInMemoryKeyedCombiningValue(ByteString stateKey,
+ Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn,
+ Coder<AccumT> accumCoder,
+ final StateContext<?> stateContext) {
+ this(stateKey, withContext(combineFn), accumCoder, stateContext);
+ }
+
+ private FlinkInMemoryKeyedCombiningValue(ByteString stateKey,
+ CombineWithContext.KeyedCombineFnWithContext<? super K, InputT, AccumT, OutputT> combineFn,
+ Coder<AccumT> accumCoder,
+ final StateContext<?> stateContext) {
+ Preconditions.checkNotNull(combineFn);
+ Preconditions.checkNotNull(accumCoder);
+
+ this.stateKey = stateKey;
+ this.combineFn = combineFn;
+ this.accumCoder = accumCoder;
+ this.context = new CombineWithContext.Context() {
+ @Override
+ public PipelineOptions getPipelineOptions() {
+ return stateContext.getPipelineOptions();
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view) {
+ return stateContext.sideInput(view);
+ }
+ };
+ accum = combineFn.createAccumulator(key, context);
+ }
+
+ @Override
+ public void clear() {
+ accum = combineFn.createAccumulator(key, context);
+ isClear = true;
+ }
+
+ @Override
+ public void add(InputT input) {
+ isClear = false;
+ accum = combineFn.addInput(key, accum, input, context);
+ }
+
+ @Override
+ public AccumT getAccum() {
+ return accum;
+ }
+
+ @Override
+ public ReadableState<Boolean> isEmpty() {
+ return new ReadableState<Boolean>() {
+ @Override
+ public ReadableState<Boolean> readLater() {
+ // Ignore
+ return this;
+ }
+
+ @Override
+ public Boolean read() {
+ return isClear;
+ }
+ };
+ }
+
+ @Override
+ public void addAccum(AccumT accum) {
+ isClear = false;
+ this.accum = combineFn.mergeAccumulators(key, Arrays.asList(this.accum, accum), context);
+ }
+
+ @Override
+ public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
+ return combineFn.mergeAccumulators(key, accumulators, context);
+ }
+
+ @Override
+ public OutputT read() {
+ return combineFn.extractOutput(key, accum, context);
+ }
+
+ @Override
+ public AccumulatorCombiningState<InputT, AccumT, OutputT> readLater() {
+ // Ignore
+ return this;
+ }
+
+ @Override
+ public boolean shouldPersist() {
+ return !isClear;
+ }
+
+ @Override
+ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
+ if (!isClear) {
+ // serialize the coder.
+ byte[] coder = InstantiationUtil.serializeObject(accumCoder);
+
+ // serialize the combiner.
+ byte[] combiner = InstantiationUtil.serializeObject(combineFn);
+
+ // encode the accumulator into a ByteString
+ ByteString.Output stream = ByteString.newOutput();
+ accumCoder.encode(accum, stream, Coder.Context.OUTER);
+ ByteString data = stream.toByteString();
+
+ // put the flag that the next serialized element is an accumulator
+ checkpointBuilder.addAccumulatorBuilder()
+ .setTag(stateKey)
+ .setData(coder)
+ .setData(combiner)
+ .setData(data);
+ }
+ }
+
+ public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
+ ByteString valueContent = checkpointReader.getData();
+ AccumT accum = this.accumCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER);
+ addAccum(accum);
+ }
+ }
+
+ private static final class FlinkInMemoryBag<T> implements BagState<T>, CheckpointableIF {
+ private final List<T> contents = new ArrayList<>();
+
+ private final ByteString stateKey;
+ private final Coder<T> elemCoder;
+
+ public FlinkInMemoryBag(ByteString stateKey, Coder<T> elemCoder) {
+ this.stateKey = stateKey;
+ this.elemCoder = elemCoder;
+ }
+
+ @Override
+ public void clear() {
+ contents.clear();
+ }
+
+ @Override
+ public Iterable<T> read() {
+ return contents;
+ }
+
+ @Override
+ public BagState<T> readLater() {
+ // Ignore
+ return this;
+ }
+
+ @Override
+ public void add(T input) {
+ contents.add(input);
+ }
+
+ @Override
+ public ReadableState<Boolean> isEmpty() {
+ return new ReadableState<Boolean>() {
+ @Override
+ public ReadableState<Boolean> readLater() {
+ // Ignore
+ return this;
+ }
+
+ @Override
+ public Boolean read() {
+ return contents.isEmpty();
+ }
+ };
+ }
+
+ @Override
+ public boolean shouldPersist() {
+ return !contents.isEmpty();
+ }
+
+ @Override
+ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
+ if (!contents.isEmpty()) {
+ // serialize the coder.
+ byte[] coder = InstantiationUtil.serializeObject(elemCoder);
+
+ checkpointBuilder.addListUpdatesBuilder()
+ .setTag(stateKey)
+ .setData(coder)
+ .writeInt(contents.size());
+
+ for (T item : contents) {
+ // encode the element
+ ByteString.Output stream = ByteString.newOutput();
+ elemCoder.encode(item, stream, Coder.Context.OUTER);
+ ByteString data = stream.toByteString();
+
+ // add the data to the checkpoint.
+ checkpointBuilder.setData(data);
+ }
+ }
+ }
+
+ public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
+ int noOfValues = checkpointReader.getInt();
+ for (int j = 0; j < noOfValues; j++) {
+ ByteString valueContent = checkpointReader.getData();
+ T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER);
+ add(outValue);
+ }
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java
index ba8ef89..753309e 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java
@@ -25,65 +25,65 @@ import java.util.concurrent.TimeUnit;
public class StateCheckpointReader {
- private final DataInputView input;
-
- public StateCheckpointReader(DataInputView in) {
- this.input = in;
- }
-
- public ByteString getTag() throws IOException {
- return ByteString.copyFrom(readRawData());
- }
-
- public String getTagToString() throws IOException {
- return input.readUTF();
- }
-
- public ByteString getData() throws IOException {
- return ByteString.copyFrom(readRawData());
- }
-
- public int getInt() throws IOException {
- validate();
- return input.readInt();
- }
-
- public byte getByte() throws IOException {
- validate();
- return input.readByte();
- }
-
- public Instant getTimestamp() throws IOException {
- validate();
- Long watermarkMillis = input.readLong();
- return new Instant(TimeUnit.MICROSECONDS.toMillis(watermarkMillis));
- }
-
- public <K> K deserializeKey(CoderTypeSerializer<K> keySerializer) throws IOException {
- return deserializeObject(keySerializer);
- }
-
- public <T> T deserializeObject(CoderTypeSerializer<T> objectSerializer) throws IOException {
- return objectSerializer.deserialize(input);
- }
-
- ///////// Helper Methods ///////
-
- private byte[] readRawData() throws IOException {
- validate();
- int size = input.readInt();
-
- byte[] serData = new byte[size];
- int bytesRead = input.read(serData);
- if (bytesRead != size) {
- throw new RuntimeException("Error while deserializing checkpoint. Not enough bytes in the input stream.");
- }
- return serData;
- }
-
- private void validate() {
- if (this.input == null) {
- throw new RuntimeException("StateBackend not initialized yet.");
- }
- }
+ private final DataInputView input;
+
+ public StateCheckpointReader(DataInputView in) {
+ this.input = in;
+ }
+
+ public ByteString getTag() throws IOException {
+ return ByteString.copyFrom(readRawData());
+ }
+
+ public String getTagToString() throws IOException {
+ return input.readUTF();
+ }
+
+ public ByteString getData() throws IOException {
+ return ByteString.copyFrom(readRawData());
+ }
+
+ public int getInt() throws IOException {
+ validate();
+ return input.readInt();
+ }
+
+ public byte getByte() throws IOException {
+ validate();
+ return input.readByte();
+ }
+
+ public Instant getTimestamp() throws IOException {
+ validate();
+ Long watermarkMillis = input.readLong();
+ return new Instant(TimeUnit.MICROSECONDS.toMillis(watermarkMillis));
+ }
+
+ public <K> K deserializeKey(CoderTypeSerializer<K> keySerializer) throws IOException {
+ return deserializeObject(keySerializer);
+ }
+
+ public <T> T deserializeObject(CoderTypeSerializer<T> objectSerializer) throws IOException {
+ return objectSerializer.deserialize(input);
+ }
+
+ ///////// Helper Methods ///////
+
+ private byte[] readRawData() throws IOException {
+ validate();
+ int size = input.readInt();
+
+ byte[] serData = new byte[size];
+ int bytesRead = input.read(serData);
+ if (bytesRead != size) {
+ throw new RuntimeException("Error while deserializing checkpoint. Not enough bytes in the input stream.");
+ }
+ return serData;
+ }
+
+ private void validate() {
+ if (this.input == null) {
+ throw new RuntimeException("StateBackend not initialized yet.");
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java
index cd85163..1741829 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java
@@ -34,120 +34,120 @@ import java.util.Set;
public class StateCheckpointUtils {
- public static <K> void encodeState(Map<K, FlinkStateInternals<K>> perKeyStateInternals,
- StateCheckpointWriter writer, Coder<K> keyCoder) throws IOException {
- CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
-
- int noOfKeys = perKeyStateInternals.size();
- writer.writeInt(noOfKeys);
- for (Map.Entry<K, FlinkStateInternals<K>> keyStatePair : perKeyStateInternals.entrySet()) {
- K key = keyStatePair.getKey();
- FlinkStateInternals<K> state = keyStatePair.getValue();
-
- // encode the key
- writer.serializeKey(key, keySerializer);
-
- // write the associated state
- state.persistState(writer);
- }
- }
-
- public static <K> Map<K, FlinkStateInternals<K>> decodeState(
- StateCheckpointReader reader,
- OutputTimeFn<? super BoundedWindow> outputTimeFn,
- Coder<K> keyCoder,
- Coder<? extends BoundedWindow> windowCoder,
- ClassLoader classLoader) throws IOException, ClassNotFoundException {
-
- int noOfKeys = reader.getInt();
- Map<K, FlinkStateInternals<K>> perKeyStateInternals = new HashMap<>(noOfKeys);
- perKeyStateInternals.clear();
-
- CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
- for (int i = 0; i < noOfKeys; i++) {
-
- // decode the key.
- K key = reader.deserializeKey(keySerializer);
-
- //decode the state associated to the key.
- FlinkStateInternals<K> stateForKey =
- new FlinkStateInternals<>(key, keyCoder, windowCoder, outputTimeFn);
- stateForKey.restoreState(reader, classLoader);
- perKeyStateInternals.put(key, stateForKey);
- }
- return perKeyStateInternals;
- }
-
- ////////////// Encoding/Decoding the Timers ////////////////
-
-
- public static <K> void encodeTimers(Map<K, Set<TimerInternals.TimerData>> allTimers,
- StateCheckpointWriter writer,
- Coder<K> keyCoder) throws IOException {
- CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
-
- int noOfKeys = allTimers.size();
- writer.writeInt(noOfKeys);
- for (Map.Entry<K, Set<TimerInternals.TimerData>> timersPerKey : allTimers.entrySet()) {
- K key = timersPerKey.getKey();
-
- // encode the key
- writer.serializeKey(key, keySerializer);
-
- // write the associated timers
- Set<TimerInternals.TimerData> timers = timersPerKey.getValue();
- encodeTimerDataForKey(writer, timers);
- }
- }
-
- public static <K> Map<K, Set<TimerInternals.TimerData>> decodeTimers(
- StateCheckpointReader reader,
- Coder<? extends BoundedWindow> windowCoder,
- Coder<K> keyCoder) throws IOException {
-
- int noOfKeys = reader.getInt();
- Map<K, Set<TimerInternals.TimerData>> activeTimers = new HashMap<>(noOfKeys);
- activeTimers.clear();
-
- CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
- for (int i = 0; i < noOfKeys; i++) {
-
- // decode the key.
- K key = reader.deserializeKey(keySerializer);
-
- // decode the associated timers.
- Set<TimerInternals.TimerData> timers = decodeTimerDataForKey(reader, windowCoder);
- activeTimers.put(key, timers);
- }
- return activeTimers;
- }
-
- private static void encodeTimerDataForKey(StateCheckpointWriter writer, Set<TimerInternals.TimerData> timers) throws IOException {
- // encode timers
- writer.writeInt(timers.size());
- for (TimerInternals.TimerData timer : timers) {
- String stringKey = timer.getNamespace().stringKey();
-
- writer.setTag(stringKey);
- writer.setTimestamp(timer.getTimestamp());
- writer.writeInt(timer.getDomain().ordinal());
- }
- }
-
- private static Set<TimerInternals.TimerData> decodeTimerDataForKey(
- StateCheckpointReader reader, Coder<? extends BoundedWindow> windowCoder) throws IOException {
-
- // decode the timers: first their number and then the content itself.
- int noOfTimers = reader.getInt();
- Set<TimerInternals.TimerData> timers = new HashSet<>(noOfTimers);
- for (int i = 0; i < noOfTimers; i++) {
- String stringKey = reader.getTagToString();
- Instant instant = reader.getTimestamp();
- TimeDomain domain = TimeDomain.values()[reader.getInt()];
-
- StateNamespace namespace = StateNamespaces.fromString(stringKey, windowCoder);
- timers.add(TimerInternals.TimerData.of(namespace, instant, domain));
- }
- return timers;
- }
+ public static <K> void encodeState(Map<K, FlinkStateInternals<K>> perKeyStateInternals,
+ StateCheckpointWriter writer, Coder<K> keyCoder) throws IOException {
+ CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
+
+ int noOfKeys = perKeyStateInternals.size();
+ writer.writeInt(noOfKeys);
+ for (Map.Entry<K, FlinkStateInternals<K>> keyStatePair : perKeyStateInternals.entrySet()) {
+ K key = keyStatePair.getKey();
+ FlinkStateInternals<K> state = keyStatePair.getValue();
+
+ // encode the key
+ writer.serializeKey(key, keySerializer);
+
+ // write the associated state
+ state.persistState(writer);
+ }
+ }
+
+ public static <K> Map<K, FlinkStateInternals<K>> decodeState(
+ StateCheckpointReader reader,
+ OutputTimeFn<? super BoundedWindow> outputTimeFn,
+ Coder<K> keyCoder,
+ Coder<? extends BoundedWindow> windowCoder,
+ ClassLoader classLoader) throws IOException, ClassNotFoundException {
+
+ int noOfKeys = reader.getInt();
+ Map<K, FlinkStateInternals<K>> perKeyStateInternals = new HashMap<>(noOfKeys);
+ perKeyStateInternals.clear();
+
+ CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
+ for (int i = 0; i < noOfKeys; i++) {
+
+ // decode the key.
+ K key = reader.deserializeKey(keySerializer);
+
+ //decode the state associated to the key.
+ FlinkStateInternals<K> stateForKey =
+ new FlinkStateInternals<>(key, keyCoder, windowCoder, outputTimeFn);
+ stateForKey.restoreState(reader, classLoader);
+ perKeyStateInternals.put(key, stateForKey);
+ }
+ return perKeyStateInternals;
+ }
+
+ ////////////// Encoding/Decoding the Timers ////////////////
+
+
+ public static <K> void encodeTimers(Map<K, Set<TimerInternals.TimerData>> allTimers,
+ StateCheckpointWriter writer,
+ Coder<K> keyCoder) throws IOException {
+ CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
+
+ int noOfKeys = allTimers.size();
+ writer.writeInt(noOfKeys);
+ for (Map.Entry<K, Set<TimerInternals.TimerData>> timersPerKey : allTimers.entrySet()) {
+ K key = timersPerKey.getKey();
+
+ // encode the key
+ writer.serializeKey(key, keySerializer);
+
+ // write the associated timers
+ Set<TimerInternals.TimerData> timers = timersPerKey.getValue();
+ encodeTimerDataForKey(writer, timers);
+ }
+ }
+
+ public static <K> Map<K, Set<TimerInternals.TimerData>> decodeTimers(
+ StateCheckpointReader reader,
+ Coder<? extends BoundedWindow> windowCoder,
+ Coder<K> keyCoder) throws IOException {
+
+ int noOfKeys = reader.getInt();
+ Map<K, Set<TimerInternals.TimerData>> activeTimers = new HashMap<>(noOfKeys);
+ activeTimers.clear();
+
+ CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
+ for (int i = 0; i < noOfKeys; i++) {
+
+ // decode the key.
+ K key = reader.deserializeKey(keySerializer);
+
+ // decode the associated timers.
+ Set<TimerInternals.TimerData> timers = decodeTimerDataForKey(reader, windowCoder);
+ activeTimers.put(key, timers);
+ }
+ return activeTimers;
+ }
+
+ private static void encodeTimerDataForKey(StateCheckpointWriter writer, Set<TimerInternals.TimerData> timers) throws IOException {
+ // encode timers
+ writer.writeInt(timers.size());
+ for (TimerInternals.TimerData timer : timers) {
+ String stringKey = timer.getNamespace().stringKey();
+
+ writer.setTag(stringKey);
+ writer.setTimestamp(timer.getTimestamp());
+ writer.writeInt(timer.getDomain().ordinal());
+ }
+ }
+
+ private static Set<TimerInternals.TimerData> decodeTimerDataForKey(
+ StateCheckpointReader reader, Coder<? extends BoundedWindow> windowCoder) throws IOException {
+
+ // decode the timers: first their number and then the content itself.
+ int noOfTimers = reader.getInt();
+ Set<TimerInternals.TimerData> timers = new HashSet<>(noOfTimers);
+ for (int i = 0; i < noOfTimers; i++) {
+ String stringKey = reader.getTagToString();
+ Instant instant = reader.getTimestamp();
+ TimeDomain domain = TimeDomain.values()[reader.getInt()];
+
+ StateNamespace namespace = StateNamespaces.fromString(stringKey, windowCoder);
+ timers.add(TimerInternals.TimerData.of(namespace, instant, domain));
+ }
+ return timers;
+ }
}
[30/50] [abbrv] incubator-beam git commit: [flink] adjust directories
according to package name
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
deleted file mode 100644
index fa0c8e9..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
+++ /dev/null
@@ -1,141 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.examples.streaming;
-
-import org.apache.beam.runners.flink.FlinkPipelineRunner;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSource;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.io.UnboundedSource;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.transforms.windowing.*;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer082;
-import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
-import org.joda.time.Duration;
-
-import java.util.Properties;
-
-public class KafkaWindowedWordCountExample {
-
- static final String KAFKA_TOPIC = "test"; // Default kafka topic to read from
- static final String KAFKA_BROKER = "localhost:9092"; // Default kafka broker to contact
- static final String GROUP_ID = "myGroup"; // Default groupId
- static final String ZOOKEEPER = "localhost:2181"; // Default zookeeper to connect to for Kafka
-
- public static class ExtractWordsFn extends DoFn<String, String> {
- private final Aggregator<Long, Long> emptyLines =
- createAggregator("emptyLines", new Sum.SumLongFn());
-
- @Override
- public void processElement(ProcessContext c) {
- if (c.element().trim().isEmpty()) {
- emptyLines.addValue(1L);
- }
-
- // Split the line into words.
- String[] words = c.element().split("[^a-zA-Z']+");
-
- // Output each word encountered into the output PCollection.
- for (String word : words) {
- if (!word.isEmpty()) {
- c.output(word);
- }
- }
- }
- }
-
- public static class FormatAsStringFn extends DoFn<KV<String, Long>, String> {
- @Override
- public void processElement(ProcessContext c) {
- String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString();
- System.out.println(row);
- c.output(row);
- }
- }
-
- public interface KafkaStreamingWordCountOptions extends WindowedWordCount.StreamingWordCountOptions {
- @Description("The Kafka topic to read from")
- @Default.String(KAFKA_TOPIC)
- String getKafkaTopic();
-
- void setKafkaTopic(String value);
-
- @Description("The Kafka Broker to read from")
- @Default.String(KAFKA_BROKER)
- String getBroker();
-
- void setBroker(String value);
-
- @Description("The Zookeeper server to connect to")
- @Default.String(ZOOKEEPER)
- String getZookeeper();
-
- void setZookeeper(String value);
-
- @Description("The groupId")
- @Default.String(GROUP_ID)
- String getGroup();
-
- void setGroup(String value);
-
- }
-
- public static void main(String[] args) {
- PipelineOptionsFactory.register(KafkaStreamingWordCountOptions.class);
- KafkaStreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).as(KafkaStreamingWordCountOptions.class);
- options.setJobName("KafkaExample");
- options.setStreaming(true);
- options.setCheckpointingInterval(1000L);
- options.setNumberOfExecutionRetries(5);
- options.setExecutionRetryDelay(3000L);
- options.setRunner(FlinkPipelineRunner.class);
-
- System.out.println(options.getKafkaTopic() +" "+ options.getZookeeper() +" "+ options.getBroker() +" "+ options.getGroup() );
- Pipeline pipeline = Pipeline.create(options);
-
- Properties p = new Properties();
- p.setProperty("zookeeper.connect", options.getZookeeper());
- p.setProperty("bootstrap.servers", options.getBroker());
- p.setProperty("group.id", options.getGroup());
-
- // this is the Flink consumer that reads the input to
- // the program from a kafka topic.
- FlinkKafkaConsumer082 kafkaConsumer = new FlinkKafkaConsumer082<>(
- options.getKafkaTopic(),
- new SimpleStringSchema(), p);
-
- PCollection<String> words = pipeline
- .apply(Read.from(new UnboundedFlinkSource<String, UnboundedSource.CheckpointMark>(options, kafkaConsumer)).named("StreamingWordCount"))
- .apply(ParDo.of(new ExtractWordsFn()))
- .apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(options.getWindowSize())))
- .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
- .discardingFiredPanes());
-
- PCollection<KV<String, Long>> wordCounts =
- words.apply(Count.<String>perElement());
-
- wordCounts.apply(ParDo.of(new FormatAsStringFn()))
- .apply(TextIO.Write.to("./outputKafka.txt"));
-
- pipeline.run();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
deleted file mode 100644
index 6af044d..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.examples.streaming;
-
-import org.apache.beam.runners.flink.FlinkPipelineRunner;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.*;
-import com.google.cloud.dataflow.sdk.options.Default;
-import com.google.cloud.dataflow.sdk.options.Description;
-import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.transforms.windowing.*;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-
-import org.joda.time.Duration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-/**
- * To run the example, first open a socket on a terminal by executing the command:
- * <li>
- * <li>
- * <code>nc -lk 9999</code>
- * </li>
- * </li>
- * and then launch the example. Now whatever you type in the terminal is going to be
- * the input to the program.
- * */
-public class WindowedWordCount {
-
- private static final Logger LOG = LoggerFactory.getLogger(WindowedWordCount.class);
-
- static final long WINDOW_SIZE = 10; // Default window duration in seconds
- static final long SLIDE_SIZE = 5; // Default window slide in seconds
-
- static class FormatAsStringFn extends DoFn<KV<String, Long>, String> {
- @Override
- public void processElement(ProcessContext c) {
- String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString();
- c.output(row);
- }
- }
-
- static class ExtractWordsFn extends DoFn<String, String> {
- private final Aggregator<Long, Long> emptyLines =
- createAggregator("emptyLines", new Sum.SumLongFn());
-
- @Override
- public void processElement(ProcessContext c) {
- if (c.element().trim().isEmpty()) {
- emptyLines.addValue(1L);
- }
-
- // Split the line into words.
- String[] words = c.element().split("[^a-zA-Z']+");
-
- // Output each word encountered into the output PCollection.
- for (String word : words) {
- if (!word.isEmpty()) {
- c.output(word);
- }
- }
- }
- }
-
- public interface StreamingWordCountOptions extends org.apache.beam.runners.flink.examples.WordCount.Options {
- @Description("Sliding window duration, in seconds")
- @Default.Long(WINDOW_SIZE)
- Long getWindowSize();
-
- void setWindowSize(Long value);
-
- @Description("Window slide, in seconds")
- @Default.Long(SLIDE_SIZE)
- Long getSlide();
-
- void setSlide(Long value);
- }
-
- public static void main(String[] args) throws IOException {
- StreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(StreamingWordCountOptions.class);
- options.setStreaming(true);
- options.setWindowSize(10L);
- options.setSlide(5L);
- options.setCheckpointingInterval(1000L);
- options.setNumberOfExecutionRetries(5);
- options.setExecutionRetryDelay(3000L);
- options.setRunner(FlinkPipelineRunner.class);
-
- LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize() +
- " sec. and a slide of " + options.getSlide());
-
- Pipeline pipeline = Pipeline.create(options);
-
- PCollection<String> words = pipeline
- .apply(Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("StreamingWordCount"))
- .apply(ParDo.of(new ExtractWordsFn()))
- .apply(Window.<String>into(SlidingWindows.of(Duration.standardSeconds(options.getWindowSize()))
- .every(Duration.standardSeconds(options.getSlide())))
- .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
- .discardingFiredPanes());
-
- PCollection<KV<String, Long>> wordCounts =
- words.apply(Count.<String>perElement());
-
- wordCounts.apply(ParDo.of(new FormatAsStringFn()))
- .apply(TextIO.Write.to("./outputWordCount.txt"));
-
- pipeline.run();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java
deleted file mode 100644
index cd25ba3..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/io/ConsoleIO.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.io;
-
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PDone;
-
-/**
- * Transform for printing the contents of a {@link com.google.cloud.dataflow.sdk.values.PCollection}.
- * to standard output.
- *
- * This is Flink-specific and will only work when executed using the
- * {@link org.apache.beam.runners.flink.FlinkPipelineRunner}.
- */
-public class ConsoleIO {
-
- /**
- * A PTransform that writes a PCollection to a standard output.
- */
- public static class Write {
-
- /**
- * Returns a ConsoleIO.Write PTransform with a default step name.
- */
- public static Bound create() {
- return new Bound();
- }
-
- /**
- * Returns a ConsoleIO.Write PTransform with the given step name.
- */
- public static Bound named(String name) {
- return new Bound().named(name);
- }
-
- /**
- * A PTransform that writes a bounded PCollection to standard output.
- */
- public static class Bound extends PTransform<PCollection<?>, PDone> {
- private static final long serialVersionUID = 0;
-
- Bound() {
- super("ConsoleIO.Write");
- }
-
- Bound(String name) {
- super(name);
- }
-
- /**
- * Returns a new ConsoleIO.Write PTransform that's like this one but with the given
- * step
- * name. Does not modify this object.
- */
- public Bound named(String name) {
- return new Bound(name);
- }
-
- @Override
- public PDone apply(PCollection<?> input) {
- return PDone.in(input.getPipeline());
- }
- }
- }
-}
-
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java
deleted file mode 100644
index 5201423..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchPipelineTranslator.java
+++ /dev/null
@@ -1,149 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import org.apache.flink.api.java.ExecutionEnvironment;
-
-/**
- * FlinkBatchPipelineTranslator knows how to translate Pipeline objects into Flink Jobs.
- * This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator}
- */
-public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
-
- /**
- * The necessary context in the case of a batch job.
- */
- private final FlinkBatchTranslationContext batchContext;
-
- private int depth = 0;
-
- /**
- * Composite transform that we want to translate before proceeding with other transforms.
- */
- private PTransform<?, ?> currentCompositeTransform;
-
- public FlinkBatchPipelineTranslator(ExecutionEnvironment env, PipelineOptions options) {
- this.batchContext = new FlinkBatchTranslationContext(env, options);
- }
-
- // --------------------------------------------------------------------------------------------
- // Pipeline Visitor Methods
- // --------------------------------------------------------------------------------------------
-
- @Override
- public void enterCompositeTransform(TransformTreeNode node) {
- System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node));
-
- PTransform<?, ?> transform = node.getTransform();
- if (transform != null && currentCompositeTransform == null) {
-
- BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
- if (translator != null) {
- currentCompositeTransform = transform;
- if (transform instanceof CoGroupByKey && node.getInput().expand().size() != 2) {
- // we can only optimize CoGroupByKey for input size 2
- currentCompositeTransform = null;
- }
- }
- }
- this.depth++;
- }
-
- @Override
- public void leaveCompositeTransform(TransformTreeNode node) {
- PTransform<?, ?> transform = node.getTransform();
- if (transform != null && currentCompositeTransform == transform) {
-
- BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
- if (translator != null) {
- System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node));
- applyBatchTransform(transform, node, translator);
- currentCompositeTransform = null;
- } else {
- throw new IllegalStateException("Attempted to translate composite transform " +
- "but no translator was found: " + currentCompositeTransform);
- }
- }
- this.depth--;
- System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node));
- }
-
- @Override
- public void visitTransform(TransformTreeNode node) {
- System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node));
- if (currentCompositeTransform != null) {
- // ignore it
- return;
- }
-
- // get the transformation corresponding to hte node we are
- // currently visiting and translate it into its Flink alternative.
-
- PTransform<?, ?> transform = node.getTransform();
- BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
- if (translator == null) {
- System.out.println(node.getTransform().getClass());
- throw new UnsupportedOperationException("The transform " + transform + " is currently not supported.");
- }
- applyBatchTransform(transform, node, translator);
- }
-
- @Override
- public void visitValue(PValue value, TransformTreeNode producer) {
- // do nothing here
- }
-
- private <T extends PTransform<?, ?>> void applyBatchTransform(PTransform<?, ?> transform, TransformTreeNode node, BatchTransformTranslator<?> translator) {
-
- @SuppressWarnings("unchecked")
- T typedTransform = (T) transform;
-
- @SuppressWarnings("unchecked")
- BatchTransformTranslator<T> typedTranslator = (BatchTransformTranslator<T>) translator;
-
- // create the applied PTransform on the batchContext
- batchContext.setCurrentTransform(AppliedPTransform.of(
- node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform));
- typedTranslator.translateNode(typedTransform, batchContext);
- }
-
- /**
- * A translator of a {@link PTransform}.
- */
- public interface BatchTransformTranslator<Type extends PTransform> {
- void translateNode(Type transform, FlinkBatchTranslationContext context);
- }
-
- private static String genSpaces(int n) {
- String s = "";
- for (int i = 0; i < n; i++) {
- s += "| ";
- }
- return s;
- }
-
- private static String formatNodeName(TransformTreeNode node) {
- return node.toString().split("@")[1] + node.getTransform();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
deleted file mode 100644
index f33e4f5..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTransformTranslators.java
+++ /dev/null
@@ -1,594 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation;
-
-import org.apache.beam.runners.flink.io.ConsoleIO;
-import org.apache.beam.runners.flink.translation.functions.FlinkCoGroupKeyedListAggregator;
-import org.apache.beam.runners.flink.translation.functions.FlinkCreateFunction;
-import org.apache.beam.runners.flink.translation.functions.FlinkDoFnFunction;
-import org.apache.beam.runners.flink.translation.functions.FlinkKeyedListAggregationFunction;
-import org.apache.beam.runners.flink.translation.functions.FlinkMultiOutputDoFnFunction;
-import org.apache.beam.runners.flink.translation.functions.FlinkMultiOutputPruningFunction;
-import org.apache.beam.runners.flink.translation.functions.FlinkPartialReduceFunction;
-import org.apache.beam.runners.flink.translation.functions.FlinkReduceFunction;
-import org.apache.beam.runners.flink.translation.functions.UnionCoder;
-import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
-import org.apache.beam.runners.flink.translation.types.KvCoderTypeInformation;
-import org.apache.beam.runners.flink.translation.wrappers.SinkOutputFormat;
-import org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat;
-import com.google.api.client.util.Maps;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.Write;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResultSchema;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.collect.Lists;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.operators.Keys;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.io.AvroInputFormat;
-import org.apache.flink.api.java.io.AvroOutputFormat;
-import org.apache.flink.api.java.io.TextInputFormat;
-import org.apache.flink.api.java.operators.CoGroupOperator;
-import org.apache.flink.api.java.operators.DataSink;
-import org.apache.flink.api.java.operators.DataSource;
-import org.apache.flink.api.java.operators.FlatMapOperator;
-import org.apache.flink.api.java.operators.GroupCombineOperator;
-import org.apache.flink.api.java.operators.GroupReduceOperator;
-import org.apache.flink.api.java.operators.Grouping;
-import org.apache.flink.api.java.operators.MapPartitionOperator;
-import org.apache.flink.api.java.operators.UnsortedGrouping;
-import org.apache.flink.core.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Translators for transforming
- * Dataflow {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s to
- * Flink {@link org.apache.flink.api.java.DataSet}s
- */
-public class FlinkBatchTransformTranslators {
-
- // --------------------------------------------------------------------------------------------
- // Transform Translator Registry
- // --------------------------------------------------------------------------------------------
-
- @SuppressWarnings("rawtypes")
- private static final Map<Class<? extends PTransform>, FlinkBatchPipelineTranslator.BatchTransformTranslator> TRANSLATORS = new HashMap<>();
-
- // register the known translators
- static {
- TRANSLATORS.put(View.CreatePCollectionView.class, new CreatePCollectionViewTranslatorBatch());
-
- TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslatorBatch());
- // we don't need this because we translate the Combine.PerKey directly
- //TRANSLATORS.put(Combine.GroupedValues.class, new CombineGroupedValuesTranslator());
-
- TRANSLATORS.put(Create.Values.class, new CreateTranslatorBatch());
-
- TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslatorBatch());
-
- TRANSLATORS.put(GroupByKey.GroupByKeyOnly.class, new GroupByKeyOnlyTranslatorBatch());
- // TODO we're currently ignoring windows here but that has to change in the future
- TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch());
-
- TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslatorBatch());
- TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslatorBatch());
-
- TRANSLATORS.put(CoGroupByKey.class, new CoGroupByKeyTranslatorBatch());
-
- TRANSLATORS.put(AvroIO.Read.Bound.class, new AvroIOReadTranslatorBatch());
- TRANSLATORS.put(AvroIO.Write.Bound.class, new AvroIOWriteTranslatorBatch());
-
- TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslatorBatch());
- TRANSLATORS.put(Write.Bound.class, new WriteSinkTranslatorBatch());
-
- TRANSLATORS.put(TextIO.Read.Bound.class, new TextIOReadTranslatorBatch());
- TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteTranslatorBatch());
-
- // Flink-specific
- TRANSLATORS.put(ConsoleIO.Write.Bound.class, new ConsoleIOWriteTranslatorBatch());
-
- }
-
-
- public static FlinkBatchPipelineTranslator.BatchTransformTranslator<?> getTranslator(PTransform<?, ?> transform) {
- return TRANSLATORS.get(transform.getClass());
- }
-
- private static class ReadSourceTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Read.Bounded<T>> {
-
- @Override
- public void translateNode(Read.Bounded<T> transform, FlinkBatchTranslationContext context) {
- String name = transform.getName();
- BoundedSource<T> source = transform.getSource();
- PCollection<T> output = context.getOutput(transform);
- Coder<T> coder = output.getCoder();
-
- TypeInformation<T> typeInformation = context.getTypeInfo(output);
-
- DataSource<T> dataSource = new DataSource<>(context.getExecutionEnvironment(),
- new SourceInputFormat<>(source, context.getPipelineOptions()), typeInformation, name);
-
- context.setOutputDataSet(output, dataSource);
- }
- }
-
- private static class AvroIOReadTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<AvroIO.Read.Bound<T>> {
- private static final Logger LOG = LoggerFactory.getLogger(AvroIOReadTranslatorBatch.class);
-
- @Override
- public void translateNode(AvroIO.Read.Bound<T> transform, FlinkBatchTranslationContext context) {
- String path = transform.getFilepattern();
- String name = transform.getName();
-// Schema schema = transform.getSchema();
- PValue output = context.getOutput(transform);
-
- TypeInformation<T> typeInformation = context.getTypeInfo(output);
-
- // This is super hacky, but unfortunately we cannot get the type otherwise
- Class<T> extractedAvroType;
- try {
- Field typeField = transform.getClass().getDeclaredField("type");
- typeField.setAccessible(true);
- @SuppressWarnings("unchecked")
- Class<T> avroType = (Class<T>) typeField.get(transform);
- extractedAvroType = avroType;
- } catch (NoSuchFieldException | IllegalAccessException e) {
- // we know that the field is there and it is accessible
- throw new RuntimeException("Could not access type from AvroIO.Bound", e);
- }
-
- DataSource<T> source = new DataSource<>(context.getExecutionEnvironment(),
- new AvroInputFormat<>(new Path(path), extractedAvroType),
- typeInformation, name);
-
- context.setOutputDataSet(output, source);
- }
- }
-
- private static class AvroIOWriteTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<AvroIO.Write.Bound<T>> {
- private static final Logger LOG = LoggerFactory.getLogger(AvroIOWriteTranslatorBatch.class);
-
- @Override
- public void translateNode(AvroIO.Write.Bound<T> transform, FlinkBatchTranslationContext context) {
- DataSet<T> inputDataSet = context.getInputDataSet(context.getInput(transform));
- String filenamePrefix = transform.getFilenamePrefix();
- String filenameSuffix = transform.getFilenameSuffix();
- int numShards = transform.getNumShards();
- String shardNameTemplate = transform.getShardNameTemplate();
-
- // TODO: Implement these. We need Flink support for this.
- LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.",
- filenameSuffix);
- LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
-
- // This is super hacky, but unfortunately we cannot get the type otherwise
- Class<T> extractedAvroType;
- try {
- Field typeField = transform.getClass().getDeclaredField("type");
- typeField.setAccessible(true);
- @SuppressWarnings("unchecked")
- Class<T> avroType = (Class<T>) typeField.get(transform);
- extractedAvroType = avroType;
- } catch (NoSuchFieldException | IllegalAccessException e) {
- // we know that the field is there and it is accessible
- throw new RuntimeException("Could not access type from AvroIO.Bound", e);
- }
-
- DataSink<T> dataSink = inputDataSet.output(new AvroOutputFormat<>(new Path
- (filenamePrefix), extractedAvroType));
-
- if (numShards > 0) {
- dataSink.setParallelism(numShards);
- }
- }
- }
-
- private static class TextIOReadTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator<TextIO.Read.Bound<String>> {
- private static final Logger LOG = LoggerFactory.getLogger(TextIOReadTranslatorBatch.class);
-
- @Override
- public void translateNode(TextIO.Read.Bound<String> transform, FlinkBatchTranslationContext context) {
- String path = transform.getFilepattern();
- String name = transform.getName();
-
- TextIO.CompressionType compressionType = transform.getCompressionType();
- boolean needsValidation = transform.needsValidation();
-
- // TODO: Implement these. We need Flink support for this.
- LOG.warn("Translation of TextIO.CompressionType not yet supported. Is: {}.", compressionType);
- LOG.warn("Translation of TextIO.Read.needsValidation not yet supported. Is: {}.", needsValidation);
-
- PValue output = context.getOutput(transform);
-
- TypeInformation<String> typeInformation = context.getTypeInfo(output);
- DataSource<String> source = new DataSource<>(context.getExecutionEnvironment(), new TextInputFormat(new Path(path)), typeInformation, name);
-
- context.setOutputDataSet(output, source);
- }
- }
-
- private static class TextIOWriteTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<TextIO.Write.Bound<T>> {
- private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteTranslatorBatch.class);
-
- @Override
- public void translateNode(TextIO.Write.Bound<T> transform, FlinkBatchTranslationContext context) {
- PValue input = context.getInput(transform);
- DataSet<T> inputDataSet = context.getInputDataSet(input);
-
- String filenamePrefix = transform.getFilenamePrefix();
- String filenameSuffix = transform.getFilenameSuffix();
- boolean needsValidation = transform.needsValidation();
- int numShards = transform.getNumShards();
- String shardNameTemplate = transform.getShardNameTemplate();
-
- // TODO: Implement these. We need Flink support for this.
- LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation);
- LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix);
- LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
-
- //inputDataSet.print();
- DataSink<T> dataSink = inputDataSet.writeAsText(filenamePrefix);
-
- if (numShards > 0) {
- dataSink.setParallelism(numShards);
- }
- }
- }
-
- private static class ConsoleIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator<ConsoleIO.Write.Bound> {
- @Override
- public void translateNode(ConsoleIO.Write.Bound transform, FlinkBatchTranslationContext context) {
- PValue input = context.getInput(transform);
- DataSet<?> inputDataSet = context.getInputDataSet(input);
- inputDataSet.printOnTaskManager(transform.getName());
- }
- }
-
- private static class WriteSinkTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Write.Bound<T>> {
-
- @Override
- public void translateNode(Write.Bound<T> transform, FlinkBatchTranslationContext context) {
- String name = transform.getName();
- PValue input = context.getInput(transform);
- DataSet<T> inputDataSet = context.getInputDataSet(input);
-
- inputDataSet.output(new SinkOutputFormat<>(transform, context.getPipelineOptions())).name(name);
- }
- }
-
- private static class GroupByKeyOnlyTranslatorBatch<K, V> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<GroupByKey.GroupByKeyOnly<K, V>> {
-
- @Override
- public void translateNode(GroupByKey.GroupByKeyOnly<K, V> transform, FlinkBatchTranslationContext context) {
- DataSet<KV<K, V>> inputDataSet = context.getInputDataSet(context.getInput(transform));
- GroupReduceFunction<KV<K, V>, KV<K, Iterable<V>>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>();
-
- TypeInformation<KV<K, Iterable<V>>> typeInformation = context.getTypeInfo(context.getOutput(transform));
-
- Grouping<KV<K, V>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType()));
-
- GroupReduceOperator<KV<K, V>, KV<K, Iterable<V>>> outputDataSet =
- new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
- context.setOutputDataSet(context.getOutput(transform), outputDataSet);
- }
- }
-
- /**
- * Translates a GroupByKey while ignoring window assignments. This is identical to the {@link GroupByKeyOnlyTranslatorBatch}
- */
- private static class GroupByKeyTranslatorBatch<K, V> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<GroupByKey<K, V>> {
-
- @Override
- public void translateNode(GroupByKey<K, V> transform, FlinkBatchTranslationContext context) {
- DataSet<KV<K, V>> inputDataSet = context.getInputDataSet(context.getInput(transform));
- GroupReduceFunction<KV<K, V>, KV<K, Iterable<V>>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>();
-
- TypeInformation<KV<K, Iterable<V>>> typeInformation = context.getTypeInfo(context.getOutput(transform));
-
- Grouping<KV<K, V>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType()));
-
- GroupReduceOperator<KV<K, V>, KV<K, Iterable<V>>> outputDataSet =
- new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
-
- context.setOutputDataSet(context.getOutput(transform), outputDataSet);
- }
- }
-
- private static class CombinePerKeyTranslatorBatch<K, VI, VA, VO> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Combine.PerKey<K, VI, VO>> {
-
- @Override
- public void translateNode(Combine.PerKey<K, VI, VO> transform, FlinkBatchTranslationContext context) {
- DataSet<KV<K, VI>> inputDataSet = context.getInputDataSet(context.getInput(transform));
-
- @SuppressWarnings("unchecked")
- Combine.KeyedCombineFn<K, VI, VA, VO> keyedCombineFn = (Combine.KeyedCombineFn<K, VI, VA, VO>) transform.getFn();
-
- KvCoder<K, VI> inputCoder = (KvCoder<K, VI>) context.getInput(transform).getCoder();
-
- Coder<VA> accumulatorCoder =
- null;
- try {
- accumulatorCoder = keyedCombineFn.getAccumulatorCoder(context.getInput(transform).getPipeline().getCoderRegistry(), inputCoder.getKeyCoder(), inputCoder.getValueCoder());
- } catch (CannotProvideCoderException e) {
- e.printStackTrace();
- // TODO
- }
-
- TypeInformation<KV<K, VI>> kvCoderTypeInformation = new KvCoderTypeInformation<>(inputCoder);
- TypeInformation<KV<K, VA>> partialReduceTypeInfo = new KvCoderTypeInformation<>(KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder));
-
- Grouping<KV<K, VI>> inputGrouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation));
-
- FlinkPartialReduceFunction<K, VI, VA> partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn);
-
- // Partially GroupReduce the values into the intermediate format VA (combine)
- GroupCombineOperator<KV<K, VI>, KV<K, VA>> groupCombine =
- new GroupCombineOperator<>(inputGrouping, partialReduceTypeInfo, partialReduceFunction,
- "GroupCombine: " + transform.getName());
-
- // Reduce fully to VO
- GroupReduceFunction<KV<K, VA>, KV<K, VO>> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn);
-
- TypeInformation<KV<K, VO>> reduceTypeInfo = context.getTypeInfo(context.getOutput(transform));
-
- Grouping<KV<K, VA>> intermediateGrouping = new UnsortedGrouping<>(groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType()));
-
- // Fully reduce the values and create output format VO
- GroupReduceOperator<KV<K, VA>, KV<K, VO>> outputDataSet =
- new GroupReduceOperator<>(intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName());
-
- context.setOutputDataSet(context.getOutput(transform), outputDataSet);
- }
- }
-
-// private static class CombineGroupedValuesTranslator<K, VI, VO> implements FlinkPipelineTranslator.TransformTranslator<Combine.GroupedValues<K, VI, VO>> {
-//
-// @Override
-// public void translateNode(Combine.GroupedValues<K, VI, VO> transform, TranslationContext context) {
-// DataSet<KV<K, VI>> inputDataSet = context.getInputDataSet(transform.getInput());
-//
-// Combine.KeyedCombineFn<? super K, ? super VI, ?, VO> keyedCombineFn = transform.getFn();
-//
-// GroupReduceFunction<KV<K, VI>, KV<K, VO>> groupReduceFunction = new FlinkCombineFunction<>(keyedCombineFn);
-//
-// TypeInformation<KV<K, VO>> typeInformation = context.getTypeInfo(transform.getOutput());
-//
-// Grouping<KV<K, VI>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{""}, inputDataSet.getType()));
-//
-// GroupReduceOperator<KV<K, VI>, KV<K, VO>> outputDataSet =
-// new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
-// context.setOutputDataSet(transform.getOutput(), outputDataSet);
-// }
-// }
-
- private static class ParDoBoundTranslatorBatch<IN, OUT> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<ParDo.Bound<IN, OUT>> {
- private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundTranslatorBatch.class);
-
- @Override
- public void translateNode(ParDo.Bound<IN, OUT> transform, FlinkBatchTranslationContext context) {
- DataSet<IN> inputDataSet = context.getInputDataSet(context.getInput(transform));
-
- final DoFn<IN, OUT> doFn = transform.getFn();
-
- TypeInformation<OUT> typeInformation = context.getTypeInfo(context.getOutput(transform));
-
- FlinkDoFnFunction<IN, OUT> doFnWrapper = new FlinkDoFnFunction<>(doFn, context.getPipelineOptions());
- MapPartitionOperator<IN, OUT> outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName());
-
- transformSideInputs(transform.getSideInputs(), outputDataSet, context);
-
- context.setOutputDataSet(context.getOutput(transform), outputDataSet);
- }
- }
-
- private static class ParDoBoundMultiTranslatorBatch<IN, OUT> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<ParDo.BoundMulti<IN, OUT>> {
- private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundMultiTranslatorBatch.class);
-
- @Override
- public void translateNode(ParDo.BoundMulti<IN, OUT> transform, FlinkBatchTranslationContext context) {
- DataSet<IN> inputDataSet = context.getInputDataSet(context.getInput(transform));
-
- final DoFn<IN, OUT> doFn = transform.getFn();
-
- Map<TupleTag<?>, PCollection<?>> outputs = context.getOutput(transform).getAll();
-
- Map<TupleTag<?>, Integer> outputMap = Maps.newHashMap();
- // put the main output at index 0, FlinkMultiOutputDoFnFunction also expects this
- outputMap.put(transform.getMainOutputTag(), 0);
- int count = 1;
- for (TupleTag<?> tag: outputs.keySet()) {
- if (!outputMap.containsKey(tag)) {
- outputMap.put(tag, count++);
- }
- }
-
- // collect all output Coders and create a UnionCoder for our tagged outputs
- List<Coder<?>> outputCoders = Lists.newArrayList();
- for (PCollection<?> coll: outputs.values()) {
- outputCoders.add(coll.getCoder());
- }
-
- UnionCoder unionCoder = UnionCoder.of(outputCoders);
-
- @SuppressWarnings("unchecked")
- TypeInformation<RawUnionValue> typeInformation = new CoderTypeInformation<>(unionCoder);
-
- @SuppressWarnings("unchecked")
- FlinkMultiOutputDoFnFunction<IN, OUT> doFnWrapper = new FlinkMultiOutputDoFnFunction(doFn, context.getPipelineOptions(), outputMap);
- MapPartitionOperator<IN, RawUnionValue> outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName());
-
- transformSideInputs(transform.getSideInputs(), outputDataSet, context);
-
- for (Map.Entry<TupleTag<?>, PCollection<?>> output: outputs.entrySet()) {
- TypeInformation<Object> outputType = context.getTypeInfo(output.getValue());
- int outputTag = outputMap.get(output.getKey());
- FlinkMultiOutputPruningFunction<Object> pruningFunction = new FlinkMultiOutputPruningFunction<>(outputTag);
- FlatMapOperator<RawUnionValue, Object> pruningOperator = new
- FlatMapOperator<>(outputDataSet, outputType,
- pruningFunction, output.getValue().getName());
- context.setOutputDataSet(output.getValue(), pruningOperator);
-
- }
- }
- }
-
- private static class FlattenPCollectionTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Flatten.FlattenPCollectionList<T>> {
-
- @Override
- public void translateNode(Flatten.FlattenPCollectionList<T> transform, FlinkBatchTranslationContext context) {
- List<PCollection<T>> allInputs = context.getInput(transform).getAll();
- DataSet<T> result = null;
- for(PCollection<T> collection : allInputs) {
- DataSet<T> current = context.getInputDataSet(collection);
- if (result == null) {
- result = current;
- } else {
- result = result.union(current);
- }
- }
- context.setOutputDataSet(context.getOutput(transform), result);
- }
- }
-
- private static class CreatePCollectionViewTranslatorBatch<R, T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<View.CreatePCollectionView<R, T>> {
- @Override
- public void translateNode(View.CreatePCollectionView<R, T> transform, FlinkBatchTranslationContext context) {
- DataSet<T> inputDataSet = context.getInputDataSet(context.getInput(transform));
- PCollectionView<T> input = transform.apply(null);
- context.setSideInputDataSet(input, inputDataSet);
- }
- }
-
- private static class CreateTranslatorBatch<OUT> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Create.Values<OUT>> {
-
- @Override
- public void translateNode(Create.Values<OUT> transform, FlinkBatchTranslationContext context) {
- TypeInformation<OUT> typeInformation = context.getOutputTypeInfo();
- Iterable<OUT> elements = transform.getElements();
-
- // we need to serialize the elements to byte arrays, since they might contain
- // elements that are not serializable by Java serialization. We deserialize them
- // in the FlatMap function using the Coder.
-
- List<byte[]> serializedElements = Lists.newArrayList();
- Coder<OUT> coder = context.getOutput(transform).getCoder();
- for (OUT element: elements) {
- ByteArrayOutputStream bao = new ByteArrayOutputStream();
- try {
- coder.encode(element, bao, Coder.Context.OUTER);
- serializedElements.add(bao.toByteArray());
- } catch (IOException e) {
- throw new RuntimeException("Could not serialize Create elements using Coder: " + e);
- }
- }
-
- DataSet<Integer> initDataSet = context.getExecutionEnvironment().fromElements(1);
- FlinkCreateFunction<Integer, OUT> flatMapFunction = new FlinkCreateFunction<>(serializedElements, coder);
- FlatMapOperator<Integer, OUT> outputDataSet = new FlatMapOperator<>(initDataSet, typeInformation, flatMapFunction, transform.getName());
-
- context.setOutputDataSet(context.getOutput(transform), outputDataSet);
- }
- }
-
- private static void transformSideInputs(List<PCollectionView<?>> sideInputs,
- MapPartitionOperator<?, ?> outputDataSet,
- FlinkBatchTranslationContext context) {
- // get corresponding Flink broadcast DataSets
- for(PCollectionView<?> input : sideInputs) {
- DataSet<?> broadcastSet = context.getSideInputDataSet(input);
- outputDataSet.withBroadcastSet(broadcastSet, input.getTagInternal().getId());
- }
- }
-
-// Disabled because it depends on a pending pull request to the DataFlowSDK
- /**
- * Special composite transform translator. Only called if the CoGroup is two dimensional.
- * @param <K>
- */
- private static class CoGroupByKeyTranslatorBatch<K, V1, V2> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<CoGroupByKey<K>> {
-
- @Override
- public void translateNode(CoGroupByKey<K> transform, FlinkBatchTranslationContext context) {
- KeyedPCollectionTuple<K> input = context.getInput(transform);
-
- CoGbkResultSchema schema = input.getCoGbkResultSchema();
- List<KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?>> keyedCollections = input.getKeyedCollections();
-
- KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?> taggedCollection1 = keyedCollections.get(0);
- KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?> taggedCollection2 = keyedCollections.get(1);
-
- TupleTag<?> tupleTag1 = taggedCollection1.getTupleTag();
- TupleTag<?> tupleTag2 = taggedCollection2.getTupleTag();
-
- PCollection<? extends KV<K, ?>> collection1 = taggedCollection1.getCollection();
- PCollection<? extends KV<K, ?>> collection2 = taggedCollection2.getCollection();
-
- DataSet<KV<K,V1>> inputDataSet1 = context.getInputDataSet(collection1);
- DataSet<KV<K,V2>> inputDataSet2 = context.getInputDataSet(collection2);
-
- TypeInformation<KV<K,CoGbkResult>> typeInfo = context.getOutputTypeInfo();
-
- FlinkCoGroupKeyedListAggregator<K,V1,V2> aggregator = new FlinkCoGroupKeyedListAggregator<>(schema, tupleTag1, tupleTag2);
-
- Keys.ExpressionKeys<KV<K,V1>> keySelector1 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet1.getType());
- Keys.ExpressionKeys<KV<K,V2>> keySelector2 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet2.getType());
-
- DataSet<KV<K, CoGbkResult>> out = new CoGroupOperator<>(inputDataSet1, inputDataSet2,
- keySelector1, keySelector2,
- aggregator, typeInfo, null, transform.getName());
- context.setOutputDataSet(context.getOutput(transform), out);
- }
- }
-
- // --------------------------------------------------------------------------------------------
- // Miscellaneous
- // --------------------------------------------------------------------------------------------
-
- private FlinkBatchTransformTranslators() {}
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java
deleted file mode 100644
index fd99833..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation;
-
-import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
-import org.apache.beam.runners.flink.translation.types.KvCoderTypeInformation;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TypedPValue;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.typeutils.GenericTypeInfo;
-
-import java.util.HashMap;
-import java.util.Map;
-
-public class FlinkBatchTranslationContext {
-
- private final Map<PValue, DataSet<?>> dataSets;
- private final Map<PCollectionView<?>, DataSet<?>> broadcastDataSets;
-
- private final ExecutionEnvironment env;
- private final PipelineOptions options;
-
- private AppliedPTransform<?, ?, ?> currentTransform;
-
- // ------------------------------------------------------------------------
-
- public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions options) {
- this.env = env;
- this.options = options;
- this.dataSets = new HashMap<>();
- this.broadcastDataSets = new HashMap<>();
- }
-
- // ------------------------------------------------------------------------
-
- public ExecutionEnvironment getExecutionEnvironment() {
- return env;
- }
-
- public PipelineOptions getPipelineOptions() {
- return options;
- }
-
- @SuppressWarnings("unchecked")
- public <T> DataSet<T> getInputDataSet(PValue value) {
- return (DataSet<T>) dataSets.get(value);
- }
-
- public void setOutputDataSet(PValue value, DataSet<?> set) {
- if (!dataSets.containsKey(value)) {
- dataSets.put(value, set);
- }
- }
-
- /**
- * Sets the AppliedPTransform which carries input/output.
- * @param currentTransform
- */
- public void setCurrentTransform(AppliedPTransform<?, ?, ?> currentTransform) {
- this.currentTransform = currentTransform;
- }
-
- @SuppressWarnings("unchecked")
- public <T> DataSet<T> getSideInputDataSet(PCollectionView<?> value) {
- return (DataSet<T>) broadcastDataSets.get(value);
- }
-
- public void setSideInputDataSet(PCollectionView<?> value, DataSet<?> set) {
- if (!broadcastDataSets.containsKey(value)) {
- broadcastDataSets.put(value, set);
- }
- }
-
- @SuppressWarnings("unchecked")
- public <T> TypeInformation<T> getTypeInfo(PInput output) {
- if (output instanceof TypedPValue) {
- Coder<?> outputCoder = ((TypedPValue) output).getCoder();
- if (outputCoder instanceof KvCoder) {
- return new KvCoderTypeInformation((KvCoder) outputCoder);
- } else {
- return new CoderTypeInformation(outputCoder);
- }
- }
- return new GenericTypeInfo<>((Class<T>)Object.class);
- }
-
- public <T> TypeInformation<T> getInputTypeInfo() {
- return getTypeInfo(currentTransform.getInput());
- }
-
- public <T> TypeInformation<T> getOutputTypeInfo() {
- return getTypeInfo((PValue) currentTransform.getOutput());
- }
-
- @SuppressWarnings("unchecked")
- <I extends PInput> I getInput(PTransform<I, ?> transform) {
- return (I) currentTransform.getInput();
- }
-
- @SuppressWarnings("unchecked")
- <O extends POutput> O getOutput(PTransform<?, O> transform) {
- return (O) currentTransform.getOutput();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java
deleted file mode 100644
index efe217f..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-
-/**
- * The role of this class is to translate the Beam operators to
- * their Flink counterparts. If we have a streaming job, this is instantiated as a
- * {@link FlinkStreamingPipelineTranslator}. In other case, i.e. for a batch job,
- * a {@link FlinkBatchPipelineTranslator} is created. Correspondingly, the
- * {@link com.google.cloud.dataflow.sdk.values.PCollection}-based user-provided job is translated into
- * a {@link org.apache.flink.streaming.api.datastream.DataStream} (for streaming) or a
- * {@link org.apache.flink.api.java.DataSet} (for batch) one.
- */
-public abstract class FlinkPipelineTranslator implements Pipeline.PipelineVisitor {
-
- public void translate(Pipeline pipeline) {
- pipeline.traverseTopologically(this);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
deleted file mode 100644
index 21a8133..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-
-/**
- * This is a {@link FlinkPipelineTranslator} for streaming jobs. Its role is to translate the user-provided
- * {@link com.google.cloud.dataflow.sdk.values.PCollection}-based job into a
- * {@link org.apache.flink.streaming.api.datastream.DataStream} one.
- *
- * This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator}
- * */
-public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
-
- /** The necessary context in the case of a straming job. */
- private final FlinkStreamingTranslationContext streamingContext;
-
- private int depth = 0;
-
- /** Composite transform that we want to translate before proceeding with other transforms. */
- private PTransform<?, ?> currentCompositeTransform;
-
- public FlinkStreamingPipelineTranslator(StreamExecutionEnvironment env, PipelineOptions options) {
- this.streamingContext = new FlinkStreamingTranslationContext(env, options);
- }
-
- // --------------------------------------------------------------------------------------------
- // Pipeline Visitor Methods
- // --------------------------------------------------------------------------------------------
-
- @Override
- public void enterCompositeTransform(TransformTreeNode node) {
- System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node));
-
- PTransform<?, ?> transform = node.getTransform();
- if (transform != null && currentCompositeTransform == null) {
-
- StreamTransformTranslator<?> translator = FlinkStreamingTransformTranslators.getTranslator(transform);
- if (translator != null) {
- currentCompositeTransform = transform;
- }
- }
- this.depth++;
- }
-
- @Override
- public void leaveCompositeTransform(TransformTreeNode node) {
- PTransform<?, ?> transform = node.getTransform();
- if (transform != null && currentCompositeTransform == transform) {
-
- StreamTransformTranslator<?> translator = FlinkStreamingTransformTranslators.getTranslator(transform);
- if (translator != null) {
- System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node));
- applyStreamingTransform(transform, node, translator);
- currentCompositeTransform = null;
- } else {
- throw new IllegalStateException("Attempted to translate composite transform " +
- "but no translator was found: " + currentCompositeTransform);
- }
- }
- this.depth--;
- System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node));
- }
-
- @Override
- public void visitTransform(TransformTreeNode node) {
- System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node));
- if (currentCompositeTransform != null) {
- // ignore it
- return;
- }
-
- // get the transformation corresponding to hte node we are
- // currently visiting and translate it into its Flink alternative.
-
- PTransform<?, ?> transform = node.getTransform();
- StreamTransformTranslator<?> translator = FlinkStreamingTransformTranslators.getTranslator(transform);
- if (translator == null) {
- System.out.println(node.getTransform().getClass());
- throw new UnsupportedOperationException("The transform " + transform + " is currently not supported.");
- }
- applyStreamingTransform(transform, node, translator);
- }
-
- @Override
- public void visitValue(PValue value, TransformTreeNode producer) {
- // do nothing here
- }
-
- private <T extends PTransform<?, ?>> void applyStreamingTransform(PTransform<?, ?> transform, TransformTreeNode node, StreamTransformTranslator<?> translator) {
-
- @SuppressWarnings("unchecked")
- T typedTransform = (T) transform;
-
- @SuppressWarnings("unchecked")
- StreamTransformTranslator<T> typedTranslator = (StreamTransformTranslator<T>) translator;
-
- // create the applied PTransform on the streamingContext
- streamingContext.setCurrentTransform(AppliedPTransform.of(
- node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform));
- typedTranslator.translateNode(typedTransform, streamingContext);
- }
-
- /**
- * The interface that every Flink translator of a Beam operator should implement.
- * This interface is for <b>streaming</b> jobs. For examples of such translators see
- * {@link FlinkStreamingTransformTranslators}.
- */
- public interface StreamTransformTranslator<Type extends PTransform> {
- void translateNode(Type transform, FlinkStreamingTranslationContext context);
- }
-
- private static String genSpaces(int n) {
- String s = "";
- for (int i = 0; i < n; i++) {
- s += "| ";
- }
- return s;
- }
-
- private static String formatNodeName(TransformTreeNode node) {
- return node.toString().split("@")[1] + node.getTransform();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
deleted file mode 100644
index 1083848..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
+++ /dev/null
@@ -1,404 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation;
-
-import org.apache.beam.runners.flink.translation.functions.UnionCoder;
-import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.*;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.io.FlinkStreamingCreateFunction;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSource;
-import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper;
-import com.google.api.client.util.Maps;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.*;
-import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
-import com.google.cloud.dataflow.sdk.transforms.windowing.*;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.collect.Lists;
-import org.apache.flink.api.common.functions.FilterFunction;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.streaming.api.datastream.*;
-import org.apache.flink.util.Collector;
-import org.joda.time.Instant;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.util.*;
-
-/**
- * This class contains all the mappings between Beam and Flink
- * <b>streaming</b> transformations. The {@link FlinkStreamingPipelineTranslator}
- * traverses the Beam job and comes here to translate the encountered Beam transformations
- * into Flink one, based on the mapping available in this class.
- */
-public class FlinkStreamingTransformTranslators {
-
- // --------------------------------------------------------------------------------------------
- // Transform Translator Registry
- // --------------------------------------------------------------------------------------------
-
- @SuppressWarnings("rawtypes")
- private static final Map<Class<? extends PTransform>, FlinkStreamingPipelineTranslator.StreamTransformTranslator> TRANSLATORS = new HashMap<>();
-
- // here you can find all the available translators.
- static {
- TRANSLATORS.put(Create.Values.class, new CreateStreamingTranslator());
- TRANSLATORS.put(Read.Unbounded.class, new UnboundedReadSourceTranslator());
- TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator());
- TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteBoundStreamingTranslator());
- TRANSLATORS.put(Window.Bound.class, new WindowBoundTranslator());
- TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslator());
- TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslator());
- TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator());
- TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiStreamingTranslator());
- }
-
- public static FlinkStreamingPipelineTranslator.StreamTransformTranslator<?> getTranslator(PTransform<?, ?> transform) {
- return TRANSLATORS.get(transform.getClass());
- }
-
- // --------------------------------------------------------------------------------------------
- // Transformation Implementations
- // --------------------------------------------------------------------------------------------
-
- private static class CreateStreamingTranslator<OUT> implements
- FlinkStreamingPipelineTranslator.StreamTransformTranslator<Create.Values<OUT>> {
-
- @Override
- public void translateNode(Create.Values<OUT> transform, FlinkStreamingTranslationContext context) {
- PCollection<OUT> output = context.getOutput(transform);
- Iterable<OUT> elements = transform.getElements();
-
- // we need to serialize the elements to byte arrays, since they might contain
- // elements that are not serializable by Java serialization. We deserialize them
- // in the FlatMap function using the Coder.
-
- List<byte[]> serializedElements = Lists.newArrayList();
- Coder<OUT> elementCoder = context.getOutput(transform).getCoder();
- for (OUT element: elements) {
- ByteArrayOutputStream bao = new ByteArrayOutputStream();
- try {
- elementCoder.encode(element, bao, Coder.Context.OUTER);
- serializedElements.add(bao.toByteArray());
- } catch (IOException e) {
- throw new RuntimeException("Could not serialize Create elements using Coder: " + e);
- }
- }
-
-
- DataStream<Integer> initDataSet = context.getExecutionEnvironment().fromElements(1);
-
- FlinkStreamingCreateFunction<Integer, OUT> createFunction =
- new FlinkStreamingCreateFunction<>(serializedElements, elementCoder);
-
- WindowedValue.ValueOnlyWindowedValueCoder<OUT> windowCoder = WindowedValue.getValueOnlyCoder(elementCoder);
- TypeInformation<WindowedValue<OUT>> outputType = new CoderTypeInformation<>(windowCoder);
-
- DataStream<WindowedValue<OUT>> outputDataStream = initDataSet.flatMap(createFunction)
- .returns(outputType);
-
- context.setOutputDataStream(context.getOutput(transform), outputDataStream);
- }
- }
-
-
- private static class TextIOWriteBoundStreamingTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<TextIO.Write.Bound<T>> {
- private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteBoundStreamingTranslator.class);
-
- @Override
- public void translateNode(TextIO.Write.Bound<T> transform, FlinkStreamingTranslationContext context) {
- PValue input = context.getInput(transform);
- DataStream<WindowedValue<T>> inputDataStream = context.getInputDataStream(input);
-
- String filenamePrefix = transform.getFilenamePrefix();
- String filenameSuffix = transform.getFilenameSuffix();
- boolean needsValidation = transform.needsValidation();
- int numShards = transform.getNumShards();
- String shardNameTemplate = transform.getShardNameTemplate();
-
- // TODO: Implement these. We need Flink support for this.
- LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation);
- LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix);
- LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
-
- DataStream<String> dataSink = inputDataStream.flatMap(new FlatMapFunction<WindowedValue<T>, String>() {
- @Override
- public void flatMap(WindowedValue<T> value, Collector<String> out) throws Exception {
- out.collect(value.getValue().toString());
- }
- });
- DataStreamSink<String> output = dataSink.writeAsText(filenamePrefix, FileSystem.WriteMode.OVERWRITE);
-
- if (numShards > 0) {
- output.setParallelism(numShards);
- }
- }
- }
-
- private static class UnboundedReadSourceTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Read.Unbounded<T>> {
-
- @Override
- public void translateNode(Read.Unbounded<T> transform, FlinkStreamingTranslationContext context) {
- PCollection<T> output = context.getOutput(transform);
-
- DataStream<WindowedValue<T>> source;
- if (transform.getSource().getClass().equals(UnboundedFlinkSource.class)) {
- UnboundedFlinkSource flinkSource = (UnboundedFlinkSource) transform.getSource();
- source = context.getExecutionEnvironment()
- .addSource(flinkSource.getFlinkSource())
- .flatMap(new FlatMapFunction<String, WindowedValue<String>>() {
- @Override
- public void flatMap(String s, Collector<WindowedValue<String>> collector) throws Exception {
- collector.collect(WindowedValue.<String>of(s, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
- }
- });
- } else {
- source = context.getExecutionEnvironment()
- .addSource(new UnboundedSourceWrapper<>(context.getPipelineOptions(), transform));
- }
- context.setOutputDataStream(output, source);
- }
- }
-
- private static class ParDoBoundStreamingTranslator<IN, OUT> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<ParDo.Bound<IN, OUT>> {
-
- @Override
- public void translateNode(ParDo.Bound<IN, OUT> transform, FlinkStreamingTranslationContext context) {
- PCollection<OUT> output = context.getOutput(transform);
-
- final WindowingStrategy<OUT, ? extends BoundedWindow> windowingStrategy =
- (WindowingStrategy<OUT, ? extends BoundedWindow>)
- context.getOutput(transform).getWindowingStrategy();
-
- WindowedValue.WindowedValueCoder<OUT> outputStreamCoder = WindowedValue.getFullCoder(output.getCoder(),
- windowingStrategy.getWindowFn().windowCoder());
- CoderTypeInformation<WindowedValue<OUT>> outputWindowedValueCoder =
- new CoderTypeInformation<>(outputStreamCoder);
-
- FlinkParDoBoundWrapper<IN, OUT> doFnWrapper = new FlinkParDoBoundWrapper<>(
- context.getPipelineOptions(), windowingStrategy, transform.getFn());
- DataStream<WindowedValue<IN>> inputDataStream = context.getInputDataStream(context.getInput(transform));
- SingleOutputStreamOperator<WindowedValue<OUT>> outDataStream = inputDataStream.flatMap(doFnWrapper)
- .returns(outputWindowedValueCoder);
-
- context.setOutputDataStream(context.getOutput(transform), outDataStream);
- }
- }
-
- public static class WindowBoundTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Window.Bound<T>> {
-
- @Override
- public void translateNode(Window.Bound<T> transform, FlinkStreamingTranslationContext context) {
- PValue input = context.getInput(transform);
- DataStream<WindowedValue<T>> inputDataStream = context.getInputDataStream(input);
-
- final WindowingStrategy<T, ? extends BoundedWindow> windowingStrategy =
- (WindowingStrategy<T, ? extends BoundedWindow>)
- context.getOutput(transform).getWindowingStrategy();
-
- final WindowFn<T, ? extends BoundedWindow> windowFn = windowingStrategy.getWindowFn();
-
- WindowedValue.WindowedValueCoder<T> outputStreamCoder = WindowedValue.getFullCoder(
- context.getInput(transform).getCoder(), windowingStrategy.getWindowFn().windowCoder());
- CoderTypeInformation<WindowedValue<T>> outputWindowedValueCoder =
- new CoderTypeInformation<>(outputStreamCoder);
-
- final FlinkParDoBoundWrapper<T, T> windowDoFnAssigner = new FlinkParDoBoundWrapper<>(
- context.getPipelineOptions(), windowingStrategy, createWindowAssigner(windowFn));
-
- SingleOutputStreamOperator<WindowedValue<T>> windowedStream =
- inputDataStream.flatMap(windowDoFnAssigner).returns(outputWindowedValueCoder);
- context.setOutputDataStream(context.getOutput(transform), windowedStream);
- }
-
- private static <T, W extends BoundedWindow> DoFn<T, T> createWindowAssigner(final WindowFn<T, W> windowFn) {
- return new DoFn<T, T>() {
-
- @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 Collection<? extends BoundedWindow> windows() {
- return c.windowingInternals().windows();
- }
- });
-
- c.windowingInternals().outputWindowedValue(
- c.element(), c.timestamp(), windows, c.pane());
- }
- };
- }
- }
-
- public static class GroupByKeyTranslator<K, V> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<GroupByKey<K, V>> {
-
- @Override
- public void translateNode(GroupByKey<K, V> transform, FlinkStreamingTranslationContext context) {
- PValue input = context.getInput(transform);
-
- DataStream<WindowedValue<KV<K, V>>> inputDataStream = context.getInputDataStream(input);
- KvCoder<K, V> inputKvCoder = (KvCoder<K, V>) context.getInput(transform).getCoder();
-
- KeyedStream<WindowedValue<KV<K, V>>, K> groupByKStream = FlinkGroupByKeyWrapper
- .groupStreamByKey(inputDataStream, inputKvCoder);
-
- DataStream<WindowedValue<KV<K, Iterable<V>>>> groupedByKNWstream =
- FlinkGroupAlsoByWindowWrapper.createForIterable(context.getPipelineOptions(),
- context.getInput(transform), groupByKStream);
-
- context.setOutputDataStream(context.getOutput(transform), groupedByKNWstream);
- }
- }
-
- public static class CombinePerKeyTranslator<K, VIN, VACC, VOUT> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Combine.PerKey<K, VIN, VOUT>> {
-
- @Override
- public void translateNode(Combine.PerKey<K, VIN, VOUT> transform, FlinkStreamingTranslationContext context) {
- PValue input = context.getInput(transform);
-
- DataStream<WindowedValue<KV<K, VIN>>> inputDataStream = context.getInputDataStream(input);
- KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) context.getInput(transform).getCoder();
- KvCoder<K, VOUT> outputKvCoder = (KvCoder<K, VOUT>) context.getOutput(transform).getCoder();
-
- KeyedStream<WindowedValue<KV<K, VIN>>, K> groupByKStream = FlinkGroupByKeyWrapper
- .groupStreamByKey(inputDataStream, inputKvCoder);
-
- Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combineFn = (Combine.KeyedCombineFn<K, VIN, VACC, VOUT>) transform.getFn();
- DataStream<WindowedValue<KV<K, VOUT>>> groupedByKNWstream =
- FlinkGroupAlsoByWindowWrapper.create(context.getPipelineOptions(),
- context.getInput(transform), groupByKStream, combineFn, outputKvCoder);
-
- context.setOutputDataStream(context.getOutput(transform), groupedByKNWstream);
- }
- }
-
- public static class FlattenPCollectionTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Flatten.FlattenPCollectionList<T>> {
-
- @Override
- public void translateNode(Flatten.FlattenPCollectionList<T> transform, FlinkStreamingTranslationContext context) {
- List<PCollection<T>> allInputs = context.getInput(transform).getAll();
- DataStream<T> result = null;
- for (PCollection<T> collection : allInputs) {
- DataStream<T> current = context.getInputDataStream(collection);
- result = (result == null) ? current : result.union(current);
- }
- context.setOutputDataStream(context.getOutput(transform), result);
- }
- }
-
- public static class ParDoBoundMultiStreamingTranslator<IN, OUT> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<ParDo.BoundMulti<IN, OUT>> {
-
- private final int MAIN_TAG_INDEX = 0;
-
- @Override
- public void translateNode(ParDo.BoundMulti<IN, OUT> transform, FlinkStreamingTranslationContext context) {
-
- // we assume that the transformation does not change the windowing strategy.
- WindowingStrategy<?, ? extends BoundedWindow> windowingStrategy = context.getInput(transform).getWindowingStrategy();
-
- Map<TupleTag<?>, PCollection<?>> outputs = context.getOutput(transform).getAll();
- Map<TupleTag<?>, Integer> tagsToLabels = transformTupleTagsToLabels(
- transform.getMainOutputTag(), outputs.keySet());
-
- UnionCoder intermUnionCoder = getIntermUnionCoder(outputs.values());
- WindowedValue.WindowedValueCoder<RawUnionValue> outputStreamCoder = WindowedValue.getFullCoder(
- intermUnionCoder, windowingStrategy.getWindowFn().windowCoder());
-
- CoderTypeInformation<WindowedValue<RawUnionValue>> intermWindowedValueCoder =
- new CoderTypeInformation<>(outputStreamCoder);
-
- FlinkParDoBoundMultiWrapper<IN, OUT> doFnWrapper = new FlinkParDoBoundMultiWrapper<>(
- context.getPipelineOptions(), windowingStrategy, transform.getFn(),
- transform.getMainOutputTag(), tagsToLabels);
-
- DataStream<WindowedValue<IN>> inputDataStream = context.getInputDataStream(context.getInput(transform));
- SingleOutputStreamOperator<WindowedValue<RawUnionValue>> intermDataStream =
- inputDataStream.flatMap(doFnWrapper).returns(intermWindowedValueCoder);
-
- for (Map.Entry<TupleTag<?>, PCollection<?>> output : outputs.entrySet()) {
- final int outputTag = tagsToLabels.get(output.getKey());
-
- WindowedValue.WindowedValueCoder<?> coderForTag = WindowedValue.getFullCoder(
- output.getValue().getCoder(),
- windowingStrategy.getWindowFn().windowCoder());
-
- CoderTypeInformation<WindowedValue<?>> windowedValueCoder =
- new CoderTypeInformation(coderForTag);
-
- context.setOutputDataStream(output.getValue(),
- intermDataStream.filter(new FilterFunction<WindowedValue<RawUnionValue>>() {
- @Override
- public boolean filter(WindowedValue<RawUnionValue> value) throws Exception {
- return value.getValue().getUnionTag() == outputTag;
- }
- }).flatMap(new FlatMapFunction<WindowedValue<RawUnionValue>, WindowedValue<?>>() {
- @Override
- public void flatMap(WindowedValue<RawUnionValue> value, Collector<WindowedValue<?>> collector) throws Exception {
- collector.collect(WindowedValue.of(
- value.getValue().getValue(),
- value.getTimestamp(),
- value.getWindows(),
- value.getPane()));
- }
- }).returns(windowedValueCoder));
- }
- }
-
- private Map<TupleTag<?>, Integer> transformTupleTagsToLabels(TupleTag<?> mainTag, Set<TupleTag<?>> secondaryTags) {
- Map<TupleTag<?>, Integer> tagToLabelMap = Maps.newHashMap();
- tagToLabelMap.put(mainTag, MAIN_TAG_INDEX);
- int count = MAIN_TAG_INDEX + 1;
- for (TupleTag<?> tag : secondaryTags) {
- if (!tagToLabelMap.containsKey(tag)) {
- tagToLabelMap.put(tag, count++);
- }
- }
- return tagToLabelMap;
- }
-
- private UnionCoder getIntermUnionCoder(Collection<PCollection<?>> taggedCollections) {
- List<Coder<?>> outputCoders = Lists.newArrayList();
- for (PCollection<?> coll : taggedCollections) {
- outputCoders.add(coll.getCoder());
- }
- return UnionCoder.of(outputCoders);
- }
- }
-}
[29/50] [abbrv] incubator-beam git commit: [flink] adjust directories
according to package name
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
deleted file mode 100644
index 83b19ee..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation;
-
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.common.base.Preconditions;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-
-import java.util.HashMap;
-import java.util.Map;
-
-public class FlinkStreamingTranslationContext {
-
- private final StreamExecutionEnvironment env;
- private final PipelineOptions options;
-
- /**
- * Keeps a mapping between the output value of the PTransform (in Dataflow) and the
- * Flink Operator that produced it, after the translation of the correspondinf PTransform
- * to its Flink equivalent.
- * */
- private final Map<PValue, DataStream<?>> dataStreams;
-
- private AppliedPTransform<?, ?, ?> currentTransform;
-
- public FlinkStreamingTranslationContext(StreamExecutionEnvironment env, PipelineOptions options) {
- this.env = Preconditions.checkNotNull(env);
- this.options = Preconditions.checkNotNull(options);
- this.dataStreams = new HashMap<>();
- }
-
- public StreamExecutionEnvironment getExecutionEnvironment() {
- return env;
- }
-
- public PipelineOptions getPipelineOptions() {
- return options;
- }
-
- @SuppressWarnings("unchecked")
- public <T> DataStream<T> getInputDataStream(PValue value) {
- return (DataStream<T>) dataStreams.get(value);
- }
-
- public void setOutputDataStream(PValue value, DataStream<?> set) {
- if (!dataStreams.containsKey(value)) {
- dataStreams.put(value, set);
- }
- }
-
- /**
- * Sets the AppliedPTransform which carries input/output.
- * @param currentTransform
- */
- public void setCurrentTransform(AppliedPTransform<?, ?, ?> currentTransform) {
- this.currentTransform = currentTransform;
- }
-
- @SuppressWarnings("unchecked")
- public <I extends PInput> I getInput(PTransform<I, ?> transform) {
- return (I) currentTransform.getInput();
- }
-
- @SuppressWarnings("unchecked")
- public <O extends POutput> O getOutput(PTransform<?, O> transform) {
- return (O) currentTransform.getOutput();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java
deleted file mode 100644
index 0e855ad..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.functions;
-
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResultSchema;
-import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.util.Collector;
-
-import java.util.ArrayList;
-import java.util.List;
-
-
-public class FlinkCoGroupKeyedListAggregator<K,V1,V2> implements CoGroupFunction<KV<K,V1>, KV<K,V2>, KV<K, CoGbkResult>>{
-
- private CoGbkResultSchema schema;
- private TupleTag<?> tupleTag1;
- private TupleTag<?> tupleTag2;
-
- public FlinkCoGroupKeyedListAggregator(CoGbkResultSchema schema, TupleTag<?> tupleTag1, TupleTag<?> tupleTag2) {
- this.schema = schema;
- this.tupleTag1 = tupleTag1;
- this.tupleTag2 = tupleTag2;
- }
-
- @Override
- public void coGroup(Iterable<KV<K,V1>> first, Iterable<KV<K,V2>> second, Collector<KV<K, CoGbkResult>> out) throws Exception {
- K k = null;
- List<RawUnionValue> result = new ArrayList<>();
- int index1 = schema.getIndex(tupleTag1);
- for (KV<K,?> entry : first) {
- k = entry.getKey();
- result.add(new RawUnionValue(index1, entry.getValue()));
- }
- int index2 = schema.getIndex(tupleTag2);
- for (KV<K,?> entry : second) {
- k = entry.getKey();
- result.add(new RawUnionValue(index2, entry.getValue()));
- }
- out.collect(KV.of(k, new CoGbkResult(schema, result)));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java
deleted file mode 100644
index 32efe41..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.functions;
-
-import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.util.Collector;
-
-import java.io.ByteArrayInputStream;
-import java.util.List;
-
-/**
- * This is a hack for transforming a {@link com.google.cloud.dataflow.sdk.transforms.Create}
- * operation. Flink does not allow {@code null} in it's equivalent operation:
- * {@link org.apache.flink.api.java.ExecutionEnvironment#fromElements(Object[])}. Therefore
- * we use a DataSource with one dummy element and output the elements of the Create operation
- * inside this FlatMap.
- */
-public class FlinkCreateFunction<IN, OUT> implements FlatMapFunction<IN, OUT> {
-
- private final List<byte[]> elements;
- private final Coder<OUT> coder;
-
- public FlinkCreateFunction(List<byte[]> elements, Coder<OUT> coder) {
- this.elements = elements;
- this.coder = coder;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public void flatMap(IN value, Collector<OUT> out) throws Exception {
-
- for (byte[] element : elements) {
- ByteArrayInputStream bai = new ByteArrayInputStream(element);
- OUT outValue = coder.decode(bai, Coder.Context.OUTER);
- if (outValue == null) {
- // TODO Flink doesn't allow null values in records
- out.collect((OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE);
- } else {
- out.collect(outValue);
- }
- }
-
- out.close();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java
deleted file mode 100644
index 9e8b7cd..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.functions;
-
-import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingInternals;
-import com.google.cloud.dataflow.sdk.util.state.StateInternals;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.collect.ImmutableList;
-import org.apache.flink.api.common.functions.RichMapPartitionFunction;
-import org.apache.flink.util.Collector;
-import org.joda.time.Instant;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-/**
- * Encapsulates a {@link com.google.cloud.dataflow.sdk.transforms.DoFn}
- * inside a Flink {@link org.apache.flink.api.common.functions.RichMapPartitionFunction}.
- */
-public class FlinkDoFnFunction<IN, OUT> extends RichMapPartitionFunction<IN, OUT> {
-
- private final DoFn<IN, OUT> doFn;
- private transient PipelineOptions options;
-
- public FlinkDoFnFunction(DoFn<IN, OUT> doFn, PipelineOptions options) {
- this.doFn = doFn;
- this.options = options;
- }
-
- private void writeObject(ObjectOutputStream out)
- throws IOException, ClassNotFoundException {
- out.defaultWriteObject();
- ObjectMapper mapper = new ObjectMapper();
- mapper.writeValue(out, options);
- }
-
- private void readObject(ObjectInputStream in)
- throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- ObjectMapper mapper = new ObjectMapper();
- options = mapper.readValue(in, PipelineOptions.class);
- }
-
- @Override
- public void mapPartition(Iterable<IN> values, Collector<OUT> out) throws Exception {
- ProcessContext context = new ProcessContext(doFn, out);
- this.doFn.startBundle(context);
- for (IN value : values) {
- context.inValue = value;
- doFn.processElement(context);
- }
- this.doFn.finishBundle(context);
- }
-
- private class ProcessContext extends DoFn<IN, OUT>.ProcessContext {
-
- IN inValue;
- Collector<OUT> outCollector;
-
- public ProcessContext(DoFn<IN, OUT> fn, Collector<OUT> outCollector) {
- fn.super();
- super.setupDelegateAggregators();
- this.outCollector = outCollector;
- }
-
- @Override
- public IN element() {
- return this.inValue;
- }
-
-
- @Override
- public Instant timestamp() {
- return Instant.now();
- }
-
- @Override
- public BoundedWindow window() {
- return GlobalWindow.INSTANCE;
- }
-
- @Override
- public PaneInfo pane() {
- return PaneInfo.NO_FIRING;
- }
-
- @Override
- public WindowingInternals<IN, OUT> windowingInternals() {
- return new WindowingInternals<IN, OUT>() {
- @Override
- public StateInternals stateInternals() {
- return null;
- }
-
- @Override
- public void outputWindowedValue(OUT output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
-
- }
-
- @Override
- public TimerInternals timerInternals() {
- return null;
- }
-
- @Override
- public Collection<? extends BoundedWindow> windows() {
- return ImmutableList.of(GlobalWindow.INSTANCE);
- }
-
- @Override
- public PaneInfo pane() {
- return PaneInfo.NO_FIRING;
- }
-
- @Override
- public <T> void writePCollectionViewData(TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
- throw new RuntimeException("sideInput() not implemented.");
- }
- };
- }
-
- @Override
- public PipelineOptions getPipelineOptions() {
- return options;
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view) {
- List<T> sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal().getId());
- List<WindowedValue<?>> windowedValueList = new ArrayList<>(sideInput.size());
- for (T input : sideInput) {
- windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE), pane()));
- }
- return view.fromIterableInternal(windowedValueList);
- }
-
- @Override
- public void output(OUT output) {
- outCollector.collect(output);
- }
-
- @Override
- public void outputWithTimestamp(OUT output, Instant timestamp) {
- // not FLink's way, just output normally
- output(output);
- }
-
- @Override
- public <T> void sideOutput(TupleTag<T> tag, T output) {
- // ignore the side output, this can happen when a user does not register
- // side outputs but then outputs using a freshly created TupleTag.
- }
-
- @Override
- public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
- sideOutput(tag, output);
- }
-
- @Override
- protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
- SerializableFnAggregatorWrapper<AggInputT, AggOutputT> wrapper = new SerializableFnAggregatorWrapper<>(combiner);
- getRuntimeContext().addAccumulator(name, wrapper);
- return wrapper;
- }
-
-
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java
deleted file mode 100644
index 56fa88d..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.functions;
-
-import com.google.cloud.dataflow.sdk.values.KV;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.util.Collector;
-
-import java.util.Iterator;
-
-/**
- * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a
- * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey} operation. This reads the input
- * {@link com.google.cloud.dataflow.sdk.values.KV} elements, extracts the key and collects
- * the values in a {@code List}.
- */
-public class FlinkKeyedListAggregationFunction<K,V> implements GroupReduceFunction<KV<K, V>, KV<K, Iterable<V>>> {
-
- @Override
- public void reduce(Iterable<KV<K, V>> values, Collector<KV<K, Iterable<V>>> out) throws Exception {
- Iterator<KV<K, V>> it = values.iterator();
- KV<K, V> first = it.next();
- Iterable<V> passThrough = new PassThroughIterable<>(first, it);
- out.collect(KV.of(first.getKey(), passThrough));
- }
-
- private static class PassThroughIterable<K, V> implements Iterable<V>, Iterator<V> {
- private KV<K, V> first;
- private Iterator<KV<K, V>> iterator;
-
- public PassThroughIterable(KV<K, V> first, Iterator<KV<K, V>> iterator) {
- this.first = first;
- this.iterator = iterator;
- }
-
- @Override
- public Iterator<V> iterator() {
- return this;
- }
-
- @Override
- public boolean hasNext() {
- return first != null || iterator.hasNext();
- }
-
- @Override
- public V next() {
- if (first != null) {
- V result = first.getValue();
- first = null;
- return result;
- } else {
- return iterator.next().getValue();
- }
- }
-
- @Override
- public void remove() {
- throw new UnsupportedOperationException("Cannot remove elements from input.");
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java
deleted file mode 100644
index bd8a968..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java
+++ /dev/null
@@ -1,175 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.functions;
-
-import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.util.WindowingInternals;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.collect.ImmutableList;
-import org.apache.flink.api.common.functions.RichMapPartitionFunction;
-import org.apache.flink.util.Collector;
-import org.joda.time.Instant;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Encapsulates a {@link com.google.cloud.dataflow.sdk.transforms.DoFn} that uses side outputs
- * inside a Flink {@link org.apache.flink.api.common.functions.RichMapPartitionFunction}.
- *
- * We get a mapping from {@link com.google.cloud.dataflow.sdk.values.TupleTag} to output index
- * and must tag all outputs with the output number. Afterwards a filter will filter out
- * those elements that are not to be in a specific output.
- */
-public class FlinkMultiOutputDoFnFunction<IN, OUT> extends RichMapPartitionFunction<IN, RawUnionValue> {
-
- private final DoFn<IN, OUT> doFn;
- private transient PipelineOptions options;
- private final Map<TupleTag<?>, Integer> outputMap;
-
- public FlinkMultiOutputDoFnFunction(DoFn<IN, OUT> doFn, PipelineOptions options, Map<TupleTag<?>, Integer> outputMap) {
- this.doFn = doFn;
- this.options = options;
- this.outputMap = outputMap;
- }
-
- private void writeObject(ObjectOutputStream out)
- throws IOException, ClassNotFoundException {
- out.defaultWriteObject();
- ObjectMapper mapper = new ObjectMapper();
- mapper.writeValue(out, options);
- }
-
- private void readObject(ObjectInputStream in)
- throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- ObjectMapper mapper = new ObjectMapper();
- options = mapper.readValue(in, PipelineOptions.class);
-
- }
-
- @Override
- public void mapPartition(Iterable<IN> values, Collector<RawUnionValue> out) throws Exception {
- ProcessContext context = new ProcessContext(doFn, out);
- this.doFn.startBundle(context);
- for (IN value : values) {
- context.inValue = value;
- doFn.processElement(context);
- }
- this.doFn.finishBundle(context);
- }
-
- private class ProcessContext extends DoFn<IN, OUT>.ProcessContext {
-
- IN inValue;
- Collector<RawUnionValue> outCollector;
-
- public ProcessContext(DoFn<IN, OUT> fn, Collector<RawUnionValue> outCollector) {
- fn.super();
- this.outCollector = outCollector;
- }
-
- @Override
- public IN element() {
- return this.inValue;
- }
-
- @Override
- public Instant timestamp() {
- return Instant.now();
- }
-
- @Override
- public BoundedWindow window() {
- return GlobalWindow.INSTANCE;
- }
-
- @Override
- public PaneInfo pane() {
- return PaneInfo.NO_FIRING;
- }
-
- @Override
- public WindowingInternals<IN, OUT> windowingInternals() {
- return null;
- }
-
- @Override
- public PipelineOptions getPipelineOptions() {
- return options;
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view) {
- List<T> sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal()
- .getId());
- List<WindowedValue<?>> windowedValueList = new ArrayList<>(sideInput.size());
- for (T input : sideInput) {
- windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE), pane()));
- }
- return view.fromIterableInternal(windowedValueList);
- }
-
- @Override
- public void output(OUT value) {
- // assume that index 0 is the default output
- outCollector.collect(new RawUnionValue(0, value));
- }
-
- @Override
- public void outputWithTimestamp(OUT output, Instant timestamp) {
- // not FLink's way, just output normally
- output(output);
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public <T> void sideOutput(TupleTag<T> tag, T value) {
- Integer index = outputMap.get(tag);
- if (index != null) {
- outCollector.collect(new RawUnionValue(index, value));
- }
- }
-
- @Override
- public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
- sideOutput(tag, output);
- }
-
- @Override
- protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
- SerializableFnAggregatorWrapper<AggInputT, AggOutputT> wrapper = new SerializableFnAggregatorWrapper<>(combiner);
- getRuntimeContext().addAccumulator(name, wrapper);
- return null;
- }
-
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java
deleted file mode 100644
index 3e1cb65..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.functions;
-
-import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.util.Collector;
-
-/**
- * A FlatMap function that filters out those elements that don't belong in this output. We need
- * this to implement MultiOutput ParDo functions.
- */
-public class FlinkMultiOutputPruningFunction<T> implements FlatMapFunction<RawUnionValue, T> {
-
- private final int outputTag;
-
- public FlinkMultiOutputPruningFunction(int outputTag) {
- this.outputTag = outputTag;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public void flatMap(RawUnionValue rawUnionValue, Collector<T> collector) throws Exception {
- if (rawUnionValue.getUnionTag() == outputTag) {
- collector.collect((T) rawUnionValue.getValue());
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java
deleted file mode 100644
index 1ff06ba..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.functions;
-
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.values.KV;
-import org.apache.flink.api.common.functions.GroupCombineFunction;
-import org.apache.flink.util.Collector;
-
-import java.util.Iterator;
-
-/**
- * Flink {@link org.apache.flink.api.common.functions.GroupCombineFunction} for executing a
- * {@link com.google.cloud.dataflow.sdk.transforms.Combine.PerKey} operation. This reads the input
- * {@link com.google.cloud.dataflow.sdk.values.KV} elements VI, extracts the key and emits accumulated
- * values which have the intermediate format VA.
- */
-public class FlinkPartialReduceFunction<K, VI, VA> implements GroupCombineFunction<KV<K, VI>, KV<K, VA>> {
-
- private final Combine.KeyedCombineFn<K, VI, VA, ?> keyedCombineFn;
-
- public FlinkPartialReduceFunction(Combine.KeyedCombineFn<K, VI, VA, ?>
- keyedCombineFn) {
- this.keyedCombineFn = keyedCombineFn;
- }
-
- @Override
- public void combine(Iterable<KV<K, VI>> elements, Collector<KV<K, VA>> out) throws Exception {
-
- final Iterator<KV<K, VI>> iterator = elements.iterator();
- // create accumulator using the first elements key
- KV<K, VI> first = iterator.next();
- K key = first.getKey();
- VI value = first.getValue();
- VA accumulator = keyedCombineFn.createAccumulator(key);
- accumulator = keyedCombineFn.addInput(key, accumulator, value);
-
- while(iterator.hasNext()) {
- value = iterator.next().getValue();
- accumulator = keyedCombineFn.addInput(key, accumulator, value);
- }
-
- out.collect(KV.of(key, accumulator));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java
deleted file mode 100644
index 580ac01..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.functions;
-
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.common.collect.ImmutableList;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.util.Collector;
-
-import java.util.Iterator;
-
-/**
- * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a
- * {@link com.google.cloud.dataflow.sdk.transforms.Combine.PerKey} operation. This reads the input
- * {@link com.google.cloud.dataflow.sdk.values.KV} elements, extracts the key and merges the
- * accumulators resulting from the PartialReduce which produced the input VA.
- */
-public class FlinkReduceFunction<K, VA, VO> implements GroupReduceFunction<KV<K, VA>, KV<K, VO>> {
-
- private final Combine.KeyedCombineFn<K, ?, VA, VO> keyedCombineFn;
-
- public FlinkReduceFunction(Combine.KeyedCombineFn<K, ?, VA, VO> keyedCombineFn) {
- this.keyedCombineFn = keyedCombineFn;
- }
-
- @Override
- public void reduce(Iterable<KV<K, VA>> values, Collector<KV<K, VO>> out) throws Exception {
- Iterator<KV<K, VA>> it = values.iterator();
-
- KV<K, VA> current = it.next();
- K k = current.getKey();
- VA accumulator = current.getValue();
-
- while (it.hasNext()) {
- current = it.next();
- keyedCombineFn.mergeAccumulators(k, ImmutableList.of(accumulator, current.getValue()) );
- }
-
- out.collect(KV.of(k, keyedCombineFn.extractOutput(k, accumulator)));
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java
deleted file mode 100644
index 05f4415..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * Copyright (C) 2015 Google Inc.
- *
- * 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://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.beam.runners.flink.translation.functions;
-
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.StandardCoder;
-import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
-import com.google.cloud.dataflow.sdk.util.PropertyNames;
-import com.google.cloud.dataflow.sdk.util.VarInt;
-import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.List;
-
-/**
- * A UnionCoder encodes RawUnionValues.
- *
- * This file copied from {@link com.google.cloud.dataflow.sdk.transforms.join.UnionCoder}
- */
-@SuppressWarnings("serial")
-public class UnionCoder extends StandardCoder<RawUnionValue> {
- // TODO: Think about how to integrate this with a schema object (i.e.
- // a tuple of tuple tags).
- /**
- * Builds a union coder with the given list of element coders. This list
- * corresponds to a mapping of union tag to Coder. Union tags start at 0.
- */
- public static UnionCoder of(List<Coder<?>> elementCoders) {
- return new UnionCoder(elementCoders);
- }
-
- @JsonCreator
- public static UnionCoder jsonOf(
- @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
- List<Coder<?>> elements) {
- return UnionCoder.of(elements);
- }
-
- private int getIndexForEncoding(RawUnionValue union) {
- if (union == null) {
- throw new IllegalArgumentException("cannot encode a null tagged union");
- }
- int index = union.getUnionTag();
- if (index < 0 || index >= elementCoders.size()) {
- throw new IllegalArgumentException(
- "union value index " + index + " not in range [0.." +
- (elementCoders.size() - 1) + "]");
- }
- return index;
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public void encode(
- RawUnionValue union,
- OutputStream outStream,
- Context context)
- throws IOException {
- int index = getIndexForEncoding(union);
- // Write out the union tag.
- VarInt.encode(index, outStream);
-
- // Write out the actual value.
- Coder<Object> coder = (Coder<Object>) elementCoders.get(index);
- coder.encode(
- union.getValue(),
- outStream,
- context);
- }
-
- @Override
- public RawUnionValue decode(InputStream inStream, Context context)
- throws IOException {
- int index = VarInt.decodeInt(inStream);
- Object value = elementCoders.get(index).decode(inStream, context);
- return new RawUnionValue(index, value);
- }
-
- @Override
- public List<? extends Coder<?>> getCoderArguments() {
- return null;
- }
-
- @Override
- public List<? extends Coder<?>> getComponents() {
- return elementCoders;
- }
-
- /**
- * Since this coder uses elementCoders.get(index) and coders that are known to run in constant
- * time, we defer the return value to that coder.
- */
- @Override
- public boolean isRegisterByteSizeObserverCheap(RawUnionValue union, Context context) {
- int index = getIndexForEncoding(union);
- @SuppressWarnings("unchecked")
- Coder<Object> coder = (Coder<Object>) elementCoders.get(index);
- return coder.isRegisterByteSizeObserverCheap(union.getValue(), context);
- }
-
- /**
- * Notifies ElementByteSizeObserver about the byte size of the encoded value using this coder.
- */
- @Override
- public void registerByteSizeObserver(
- RawUnionValue union, ElementByteSizeObserver observer, Context context)
- throws Exception {
- int index = getIndexForEncoding(union);
- // Write out the union tag.
- observer.update(VarInt.getLength(index));
- // Write out the actual value.
- @SuppressWarnings("unchecked")
- Coder<Object> coder = (Coder<Object>) elementCoders.get(index);
- coder.registerByteSizeObserver(union.getValue(), observer, context);
- }
-
- /////////////////////////////////////////////////////////////////////////////
-
- private final List<Coder<?>> elementCoders;
-
- private UnionCoder(List<Coder<?>> elementCoders) {
- this.elementCoders = elementCoders;
- }
-
- @Override
- public void verifyDeterministic() throws NonDeterministicException {
- verifyDeterministic(
- "UnionCoder is only deterministic if all element coders are",
- elementCoders);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java
deleted file mode 100644
index ecfb95d..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java
+++ /dev/null
@@ -1,216 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.types;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.MemorySegment;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-
-/**
- * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for
- * {@link com.google.cloud.dataflow.sdk.coders.Coder}.
- */
-public class CoderComparator<T> extends TypeComparator<T> {
-
- private Coder<T> coder;
-
- // We use these for internal encoding/decoding for creating copies and comparing
- // serialized forms using a Coder
- private transient InspectableByteArrayOutputStream buffer1;
- private transient InspectableByteArrayOutputStream buffer2;
-
- // For storing the Reference in encoded form
- private transient InspectableByteArrayOutputStream referenceBuffer;
-
- public CoderComparator(Coder<T> coder) {
- this.coder = coder;
- buffer1 = new InspectableByteArrayOutputStream();
- buffer2 = new InspectableByteArrayOutputStream();
- referenceBuffer = new InspectableByteArrayOutputStream();
- }
-
- private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- buffer1 = new InspectableByteArrayOutputStream();
- buffer2 = new InspectableByteArrayOutputStream();
- referenceBuffer = new InspectableByteArrayOutputStream();
- }
-
- @Override
- public int hash(T record) {
- return record.hashCode();
- }
-
- @Override
- public void setReference(T toCompare) {
- referenceBuffer.reset();
- try {
- coder.encode(toCompare, referenceBuffer, Coder.Context.OUTER);
- } catch (IOException e) {
- throw new RuntimeException("Could not set reference " + toCompare + ": " + e);
- }
- }
-
- @Override
- public boolean equalToReference(T candidate) {
- try {
- buffer2.reset();
- coder.encode(candidate, buffer2, Coder.Context.OUTER);
- byte[] arr = referenceBuffer.getBuffer();
- byte[] arrOther = buffer2.getBuffer();
- if (referenceBuffer.size() != buffer2.size()) {
- return false;
- }
- int len = buffer2.size();
- for(int i = 0; i < len; i++ ) {
- if (arr[i] != arrOther[i]) {
- return false;
- }
- }
- return true;
- } catch (IOException e) {
- throw new RuntimeException("Could not compare reference.", e);
- }
- }
-
- @Override
- public int compareToReference(TypeComparator<T> other) {
- InspectableByteArrayOutputStream otherReferenceBuffer = ((CoderComparator<T>) other).referenceBuffer;
-
- byte[] arr = referenceBuffer.getBuffer();
- byte[] arrOther = otherReferenceBuffer.getBuffer();
- if (referenceBuffer.size() != otherReferenceBuffer.size()) {
- return referenceBuffer.size() - otherReferenceBuffer.size();
- }
- int len = referenceBuffer.size();
- for (int i = 0; i < len; i++) {
- if (arr[i] != arrOther[i]) {
- return arr[i] - arrOther[i];
- }
- }
- return 0;
- }
-
- @Override
- public int compare(T first, T second) {
- try {
- buffer1.reset();
- buffer2.reset();
- coder.encode(first, buffer1, Coder.Context.OUTER);
- coder.encode(second, buffer2, Coder.Context.OUTER);
- byte[] arr = buffer1.getBuffer();
- byte[] arrOther = buffer2.getBuffer();
- if (buffer1.size() != buffer2.size()) {
- return buffer1.size() - buffer2.size();
- }
- int len = buffer1.size();
- for(int i = 0; i < len; i++ ) {
- if (arr[i] != arrOther[i]) {
- return arr[i] - arrOther[i];
- }
- }
- return 0;
- } catch (IOException e) {
- throw new RuntimeException("Could not compare: ", e);
- }
- }
-
- @Override
- public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
- CoderTypeSerializer<T> serializer = new CoderTypeSerializer<>(coder);
- T first = serializer.deserialize(firstSource);
- T second = serializer.deserialize(secondSource);
- return compare(first, second);
- }
-
- @Override
- public boolean supportsNormalizedKey() {
- return true;
- }
-
- @Override
- public boolean supportsSerializationWithKeyNormalization() {
- return false;
- }
-
- @Override
- public int getNormalizeKeyLen() {
- return Integer.MAX_VALUE;
- }
-
- @Override
- public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
- return true;
- }
-
- @Override
- public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) {
- buffer1.reset();
- try {
- coder.encode(record, buffer1, Coder.Context.OUTER);
- } catch (IOException e) {
- throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e);
- }
- final byte[] data = buffer1.getBuffer();
- final int limit = offset + numBytes;
-
- target.put(offset, data, 0, Math.min(numBytes, buffer1.size()));
-
- offset += buffer1.size();
-
- while (offset < limit) {
- target.put(offset++, (byte) 0);
- }
- }
-
- @Override
- public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public boolean invertNormalizedKey() {
- return false;
- }
-
- @Override
- public TypeComparator<T> duplicate() {
- return new CoderComparator<>(coder);
- }
-
- @Override
- public int extractKeys(Object record, Object[] target, int index) {
- target[index] = record;
- return 1;
- }
-
- @Override
- public TypeComparator[] getFlatComparators() {
- return new TypeComparator[] { this.duplicate() };
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java
deleted file mode 100644
index 8880b48..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.types;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.AtomicType;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import com.google.common.base.Preconditions;
-
-/**
- * Flink {@link org.apache.flink.api.common.typeinfo.TypeInformation} for
- * Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s.
- */
-public class CoderTypeInformation<T> extends TypeInformation<T> implements AtomicType<T> {
-
- private final Coder<T> coder;
-
- public CoderTypeInformation(Coder<T> coder) {
- Preconditions.checkNotNull(coder);
- this.coder = coder;
- }
-
- @Override
- public boolean isBasicType() {
- return false;
- }
-
- @Override
- public boolean isTupleType() {
- return false;
- }
-
- @Override
- public int getArity() {
- return 1;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public Class<T> getTypeClass() {
- // We don't have the Class, so we have to pass null here. What a shame...
- return (Class<T>) Object.class;
- }
-
- @Override
- public boolean isKeyType() {
- return true;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public TypeSerializer<T> createSerializer(ExecutionConfig config) {
- if (coder instanceof VoidCoder) {
- return (TypeSerializer<T>) new VoidCoderTypeSerializer();
- }
- return new CoderTypeSerializer<>(coder);
- }
-
- @Override
- public int getTotalFields() {
- return 2;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
-
- CoderTypeInformation that = (CoderTypeInformation) o;
-
- return coder.equals(that.coder);
-
- }
-
- @Override
- public int hashCode() {
- return coder.hashCode();
- }
-
- @Override
- public boolean canEqual(Object obj) {
- return obj instanceof CoderTypeInformation;
- }
-
- @Override
- public String toString() {
- return "CoderTypeInformation{" +
- "coder=" + coder +
- '}';
- }
-
- @Override
- public TypeComparator<T> createComparator(boolean sortOrderAscending, ExecutionConfig
- executionConfig) {
- return new CoderComparator<>(coder);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java
deleted file mode 100644
index 481ee31..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java
+++ /dev/null
@@ -1,152 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.types;
-
-import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper;
-import org.apache.beam.runners.flink.translation.wrappers.DataOutputViewWrapper;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.CoderException;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-
-import java.io.ByteArrayInputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-
-/**
- * Flink {@link org.apache.flink.api.common.typeutils.TypeSerializer} for
- * Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s
- */
-public class CoderTypeSerializer<T> extends TypeSerializer<T> {
-
- private Coder<T> coder;
- private transient DataInputViewWrapper inputWrapper;
- private transient DataOutputViewWrapper outputWrapper;
-
- // We use this for internal encoding/decoding for creating copies using the Coder.
- private transient InspectableByteArrayOutputStream buffer;
-
- public CoderTypeSerializer(Coder<T> coder) {
- this.coder = coder;
- this.inputWrapper = new DataInputViewWrapper(null);
- this.outputWrapper = new DataOutputViewWrapper(null);
-
- buffer = new InspectableByteArrayOutputStream();
- }
-
- private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- this.inputWrapper = new DataInputViewWrapper(null);
- this.outputWrapper = new DataOutputViewWrapper(null);
-
- buffer = new InspectableByteArrayOutputStream();
- }
-
- @Override
- public boolean isImmutableType() {
- return false;
- }
-
- @Override
- public CoderTypeSerializer<T> duplicate() {
- return new CoderTypeSerializer<>(coder);
- }
-
- @Override
- public T createInstance() {
- return null;
- }
-
- @Override
- public T copy(T t) {
- buffer.reset();
- try {
- coder.encode(t, buffer, Coder.Context.OUTER);
- } catch (IOException e) {
- throw new RuntimeException("Could not copy.", e);
- }
- try {
- return coder.decode(new ByteArrayInputStream(buffer.getBuffer(), 0, buffer
- .size()), Coder.Context.OUTER);
- } catch (IOException e) {
- throw new RuntimeException("Could not copy.", e);
- }
- }
-
- @Override
- public T copy(T t, T reuse) {
- return copy(t);
- }
-
- @Override
- public int getLength() {
- return 0;
- }
-
- @Override
- public void serialize(T t, DataOutputView dataOutputView) throws IOException {
- outputWrapper.setOutputView(dataOutputView);
- coder.encode(t, outputWrapper, Coder.Context.NESTED);
- }
-
- @Override
- public T deserialize(DataInputView dataInputView) throws IOException {
- try {
- inputWrapper.setInputView(dataInputView);
- return coder.decode(inputWrapper, Coder.Context.NESTED);
- } catch (CoderException e) {
- Throwable cause = e.getCause();
- if (cause instanceof EOFException) {
- throw (EOFException) cause;
- } else {
- throw e;
- }
- }
- }
-
- @Override
- public T deserialize(T t, DataInputView dataInputView) throws IOException {
- return deserialize(dataInputView);
- }
-
- @Override
- public void copy(DataInputView dataInputView, DataOutputView dataOutputView) throws IOException {
- serialize(deserialize(dataInputView), dataOutputView);
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
-
- CoderTypeSerializer that = (CoderTypeSerializer) o;
- return coder.equals(that.coder);
- }
-
- @Override
- public boolean canEqual(Object obj) {
- return obj instanceof CoderTypeSerializer;
- }
-
- @Override
- public int hashCode() {
- return coder.hashCode();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java
deleted file mode 100644
index 619fa55..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.types;
-
-import java.io.ByteArrayOutputStream;
-
-/**
- * Version of {@link java.io.ByteArrayOutputStream} that allows to retrieve the internal
- * byte[] buffer without incurring an array copy.
- */
-public class InspectableByteArrayOutputStream extends ByteArrayOutputStream {
-
- /**
- * Get the underlying byte array.
- */
- public byte[] getBuffer() {
- return buf;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java
deleted file mode 100644
index 4599c6a..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java
+++ /dev/null
@@ -1,264 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.types;
-
-import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.values.KV;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.MemorySegment;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-
-/**
- * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for
- * {@link com.google.cloud.dataflow.sdk.coders.KvCoder}. We have a special comparator
- * for {@link KV} that always compares on the key only.
- */
-public class KvCoderComperator <K, V> extends TypeComparator<KV<K, V>> {
-
- private KvCoder<K, V> coder;
- private Coder<K> keyCoder;
-
- // We use these for internal encoding/decoding for creating copies and comparing
- // serialized forms using a Coder
- private transient InspectableByteArrayOutputStream buffer1;
- private transient InspectableByteArrayOutputStream buffer2;
-
- // For storing the Reference in encoded form
- private transient InspectableByteArrayOutputStream referenceBuffer;
-
-
- // For deserializing the key
- private transient DataInputViewWrapper inputWrapper;
-
- public KvCoderComperator(KvCoder<K, V> coder) {
- this.coder = coder;
- this.keyCoder = coder.getKeyCoder();
-
- buffer1 = new InspectableByteArrayOutputStream();
- buffer2 = new InspectableByteArrayOutputStream();
- referenceBuffer = new InspectableByteArrayOutputStream();
-
- inputWrapper = new DataInputViewWrapper(null);
- }
-
- private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
- in.defaultReadObject();
-
- buffer1 = new InspectableByteArrayOutputStream();
- buffer2 = new InspectableByteArrayOutputStream();
- referenceBuffer = new InspectableByteArrayOutputStream();
-
- inputWrapper = new DataInputViewWrapper(null);
- }
-
- @Override
- public int hash(KV<K, V> record) {
- K key = record.getKey();
- if (key != null) {
- return key.hashCode();
- } else {
- return 0;
- }
- }
-
- @Override
- public void setReference(KV<K, V> toCompare) {
- referenceBuffer.reset();
- try {
- keyCoder.encode(toCompare.getKey(), referenceBuffer, Coder.Context.OUTER);
- } catch (IOException e) {
- throw new RuntimeException("Could not set reference " + toCompare + ": " + e);
- }
- }
-
- @Override
- public boolean equalToReference(KV<K, V> candidate) {
- try {
- buffer2.reset();
- keyCoder.encode(candidate.getKey(), buffer2, Coder.Context.OUTER);
- byte[] arr = referenceBuffer.getBuffer();
- byte[] arrOther = buffer2.getBuffer();
- if (referenceBuffer.size() != buffer2.size()) {
- return false;
- }
- int len = buffer2.size();
- for(int i = 0; i < len; i++ ) {
- if (arr[i] != arrOther[i]) {
- return false;
- }
- }
- return true;
- } catch (IOException e) {
- throw new RuntimeException("Could not compare reference.", e);
- }
- }
-
- @Override
- public int compareToReference(TypeComparator<KV<K, V>> other) {
- InspectableByteArrayOutputStream otherReferenceBuffer = ((KvCoderComperator<K, V>) other).referenceBuffer;
-
- byte[] arr = referenceBuffer.getBuffer();
- byte[] arrOther = otherReferenceBuffer.getBuffer();
- if (referenceBuffer.size() != otherReferenceBuffer.size()) {
- return referenceBuffer.size() - otherReferenceBuffer.size();
- }
- int len = referenceBuffer.size();
- for (int i = 0; i < len; i++) {
- if (arr[i] != arrOther[i]) {
- return arr[i] - arrOther[i];
- }
- }
- return 0;
- }
-
-
- @Override
- public int compare(KV<K, V> first, KV<K, V> second) {
- try {
- buffer1.reset();
- buffer2.reset();
- keyCoder.encode(first.getKey(), buffer1, Coder.Context.OUTER);
- keyCoder.encode(second.getKey(), buffer2, Coder.Context.OUTER);
- byte[] arr = buffer1.getBuffer();
- byte[] arrOther = buffer2.getBuffer();
- if (buffer1.size() != buffer2.size()) {
- return buffer1.size() - buffer2.size();
- }
- int len = buffer1.size();
- for(int i = 0; i < len; i++ ) {
- if (arr[i] != arrOther[i]) {
- return arr[i] - arrOther[i];
- }
- }
- return 0;
- } catch (IOException e) {
- throw new RuntimeException("Could not compare reference.", e);
- }
- }
-
- @Override
- public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
-
- inputWrapper.setInputView(firstSource);
- K firstKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED);
- inputWrapper.setInputView(secondSource);
- K secondKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED);
-
- try {
- buffer1.reset();
- buffer2.reset();
- keyCoder.encode(firstKey, buffer1, Coder.Context.OUTER);
- keyCoder.encode(secondKey, buffer2, Coder.Context.OUTER);
- byte[] arr = buffer1.getBuffer();
- byte[] arrOther = buffer2.getBuffer();
- if (buffer1.size() != buffer2.size()) {
- return buffer1.size() - buffer2.size();
- }
- int len = buffer1.size();
- for(int i = 0; i < len; i++ ) {
- if (arr[i] != arrOther[i]) {
- return arr[i] - arrOther[i];
- }
- }
- return 0;
- } catch (IOException e) {
- throw new RuntimeException("Could not compare reference.", e);
- }
- }
-
- @Override
- public boolean supportsNormalizedKey() {
- return true;
- }
-
- @Override
- public boolean supportsSerializationWithKeyNormalization() {
- return false;
- }
-
- @Override
- public int getNormalizeKeyLen() {
- return Integer.MAX_VALUE;
- }
-
- @Override
- public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
- return true;
- }
-
- @Override
- public void putNormalizedKey(KV<K, V> record, MemorySegment target, int offset, int numBytes) {
- buffer1.reset();
- try {
- keyCoder.encode(record.getKey(), buffer1, Coder.Context.NESTED);
- } catch (IOException e) {
- throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e);
- }
- final byte[] data = buffer1.getBuffer();
- final int limit = offset + numBytes;
-
- int numBytesPut = Math.min(numBytes, buffer1.size());
-
- target.put(offset, data, 0, numBytesPut);
-
- offset += numBytesPut;
-
- while (offset < limit) {
- target.put(offset++, (byte) 0);
- }
- }
-
- @Override
- public void writeWithKeyNormalization(KV<K, V> record, DataOutputView target) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public KV<K, V> readWithKeyDenormalization(KV<K, V> reuse, DataInputView source) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public boolean invertNormalizedKey() {
- return false;
- }
-
- @Override
- public TypeComparator<KV<K, V>> duplicate() {
- return new KvCoderComperator<>(coder);
- }
-
- @Override
- public int extractKeys(Object record, Object[] target, int index) {
- KV<K, V> kv = (KV<K, V>) record;
- K k = kv.getKey();
- target[index] = k;
- return 1;
- }
-
- @Override
- public TypeComparator[] getFlatComparators() {
- return new TypeComparator[] {new CoderComparator<>(keyCoder)};
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java
deleted file mode 100644
index 7a0d999..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.types;
-
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.values.KV;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.CompositeType;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import com.google.common.base.Preconditions;
-
-import java.util.List;
-
-/**
- * Flink {@link org.apache.flink.api.common.typeinfo.TypeInformation} for
- * Dataflow {@link com.google.cloud.dataflow.sdk.coders.KvCoder}.
- */
-public class KvCoderTypeInformation<K, V> extends CompositeType<KV<K, V>> {
-
- private KvCoder<K, V> coder;
-
- // We don't have the Class, so we have to pass null here. What a shame...
- private static Object DUMMY = new Object();
-
- @SuppressWarnings("unchecked")
- public KvCoderTypeInformation(KvCoder<K, V> coder) {
- super(((Class<KV<K,V>>) DUMMY.getClass()));
- this.coder = coder;
- Preconditions.checkNotNull(coder);
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public TypeComparator<KV<K, V>> createComparator(int[] logicalKeyFields, boolean[] orders, int logicalFieldOffset, ExecutionConfig config) {
- return new KvCoderComperator((KvCoder) coder);
- }
-
- @Override
- public boolean isBasicType() {
- return false;
- }
-
- @Override
- public boolean isTupleType() {
- return false;
- }
-
- @Override
- public int getArity() {
- return 2;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public Class<KV<K, V>> getTypeClass() {
- return privateGetTypeClass();
- }
-
- @SuppressWarnings("unchecked")
- private static <X> Class<X> privateGetTypeClass() {
- return (Class<X>) Object.class;
- }
-
- @Override
- public boolean isKeyType() {
- return true;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public TypeSerializer<KV<K, V>> createSerializer(ExecutionConfig config) {
- return new CoderTypeSerializer<>(coder);
- }
-
- @Override
- public int getTotalFields() {
- return 2;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
-
- KvCoderTypeInformation that = (KvCoderTypeInformation) o;
-
- return coder.equals(that.coder);
-
- }
-
- @Override
- public int hashCode() {
- return coder.hashCode();
- }
-
- @Override
- public String toString() {
- return "CoderTypeInformation{" +
- "coder=" + coder +
- '}';
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public <X> TypeInformation<X> getTypeAt(int pos) {
- if (pos == 0) {
- return (TypeInformation<X>) new CoderTypeInformation<>(coder.getKeyCoder());
- } else if (pos == 1) {
- return (TypeInformation<X>) new CoderTypeInformation<>(coder.getValueCoder());
- } else {
- throw new RuntimeException("Invalid field position " + pos);
- }
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public <X> TypeInformation<X> getTypeAt(String fieldExpression) {
- switch (fieldExpression) {
- case "key":
- return (TypeInformation<X>) new CoderTypeInformation<>(coder.getKeyCoder());
- case "value":
- return (TypeInformation<X>) new CoderTypeInformation<>(coder.getValueCoder());
- default:
- throw new UnsupportedOperationException("Only KvCoder has fields.");
- }
- }
-
- @Override
- public String[] getFieldNames() {
- return new String[]{"key", "value"};
- }
-
- @Override
- public int getFieldIndex(String fieldName) {
- switch (fieldName) {
- case "key":
- return 0;
- case "value":
- return 1;
- default:
- return -1;
- }
- }
-
- @Override
- public void getFlatFields(String fieldExpression, int offset, List<FlatFieldDescriptor> result) {
- CoderTypeInformation keyTypeInfo = new CoderTypeInformation<>(coder.getKeyCoder());
- result.add(new FlatFieldDescriptor(0, keyTypeInfo));
- }
-
- @Override
- protected TypeComparatorBuilder<KV<K, V>> createTypeComparatorBuilder() {
- return new KvCoderTypeComparatorBuilder();
- }
-
- private class KvCoderTypeComparatorBuilder implements TypeComparatorBuilder<KV<K, V>> {
-
- @Override
- public void initializeTypeComparatorBuilder(int size) {}
-
- @Override
- public void addComparatorField(int fieldId, TypeComparator<?> comparator) {}
-
- @Override
- public TypeComparator<KV<K, V>> createTypeComparator(ExecutionConfig config) {
- return new KvCoderComperator<>(coder);
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java
deleted file mode 100644
index c7b6ea2..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.types;
-
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-
-import java.io.IOException;
-
-/**
- * Special Flink {@link org.apache.flink.api.common.typeutils.TypeSerializer} for
- * {@link com.google.cloud.dataflow.sdk.coders.VoidCoder}. We need this because Flink does not
- * allow returning {@code null} from an input reader. We return a {@link VoidValue} instead
- * that behaves like a {@code null}, hopefully.
- */
-public class VoidCoderTypeSerializer extends TypeSerializer<VoidCoderTypeSerializer.VoidValue> {
-
- @Override
- public boolean isImmutableType() {
- return false;
- }
-
- @Override
- public VoidCoderTypeSerializer duplicate() {
- return this;
- }
-
- @Override
- public VoidValue createInstance() {
- return VoidValue.INSTANCE;
- }
-
- @Override
- public VoidValue copy(VoidValue from) {
- return from;
- }
-
- @Override
- public VoidValue copy(VoidValue from, VoidValue reuse) {
- return from;
- }
-
- @Override
- public int getLength() {
- return 0;
- }
-
- @Override
- public void serialize(VoidValue record, DataOutputView target) throws IOException {
- target.writeByte(1);
- }
-
- @Override
- public VoidValue deserialize(DataInputView source) throws IOException {
- source.readByte();
- return VoidValue.INSTANCE;
- }
-
- @Override
- public VoidValue deserialize(VoidValue reuse, DataInputView source) throws IOException {
- return deserialize(source);
- }
-
- @Override
- public void copy(DataInputView source, DataOutputView target) throws IOException {
- source.readByte();
- target.writeByte(1);
- }
-
- @Override
- public boolean equals(Object obj) {
- if (obj instanceof VoidCoderTypeSerializer) {
- VoidCoderTypeSerializer other = (VoidCoderTypeSerializer) obj;
- return other.canEqual(this);
- } else {
- return false;
- }
- }
-
- @Override
- public boolean canEqual(Object obj) {
- return obj instanceof VoidCoderTypeSerializer;
- }
-
- @Override
- public int hashCode() {
- return 0;
- }
-
- public static class VoidValue {
- private VoidValue() {}
-
- public static VoidValue INSTANCE = new VoidValue();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java
deleted file mode 100644
index 815765c..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers;
-
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.common.collect.Lists;
-import org.apache.flink.api.common.accumulators.Accumulator;
-
-import java.io.Serializable;
-
-/**
- * Wrapper that wraps a {@link com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn}
- * in a Flink {@link org.apache.flink.api.common.accumulators.Accumulator} for using
- * the combine function as an aggregator in a {@link com.google.cloud.dataflow.sdk.transforms.ParDo}
- * operation.
- */
-public class CombineFnAggregatorWrapper<AI, AA, AR> implements Aggregator<AI, AR>, Accumulator<AI, Serializable> {
-
- private AA aa;
- private Combine.CombineFn<? super AI, AA, AR> combiner;
-
- public CombineFnAggregatorWrapper() {
- }
-
- public CombineFnAggregatorWrapper(Combine.CombineFn<? super AI, AA, AR> combiner) {
- this.combiner = combiner;
- this.aa = combiner.createAccumulator();
- }
-
- @Override
- public void add(AI value) {
- combiner.addInput(aa, value);
- }
-
- @Override
- public Serializable getLocalValue() {
- return (Serializable) combiner.extractOutput(aa);
- }
-
- @Override
- public void resetLocal() {
- aa = combiner.createAccumulator();
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public void merge(Accumulator<AI, Serializable> other) {
- aa = combiner.mergeAccumulators(Lists.newArrayList(aa, ((CombineFnAggregatorWrapper<AI, AA, AR>)other).aa));
- }
-
- @Override
- public Accumulator<AI, Serializable> clone() {
- // copy it by merging
- AA aaCopy = combiner.mergeAccumulators(Lists.newArrayList(aa));
- CombineFnAggregatorWrapper<AI, AA, AR> result = new
- CombineFnAggregatorWrapper<>(combiner);
- result.aa = aaCopy;
- return result;
- }
-
- @Override
- public void addValue(AI value) {
- add(value);
- }
-
- @Override
- public String getName() {
- return "CombineFn: " + combiner.toString();
- }
-
- @Override
- public Combine.CombineFn getCombineFn() {
- return combiner;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java
deleted file mode 100644
index b56a90e..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers;
-
-import org.apache.flink.core.memory.DataInputView;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * Wrapper for {@link DataInputView}. We need this because Flink reads data using a
- * {@link org.apache.flink.core.memory.DataInputView} while
- * Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s expect an
- * {@link java.io.InputStream}.
- */
-public class DataInputViewWrapper extends InputStream {
-
- private DataInputView inputView;
-
- public DataInputViewWrapper(DataInputView inputView) {
- this.inputView = inputView;
- }
-
- public void setInputView(DataInputView inputView) {
- this.inputView = inputView;
- }
-
- @Override
- public int read() throws IOException {
- try {
- return inputView.readUnsignedByte();
- } catch (EOFException e) {
- // translate between DataInput and InputStream,
- // DataInput signals EOF by exception, InputStream does it by returning -1
- return -1;
- }
- }
-
- @Override
- public int read(byte[] b, int off, int len) throws IOException {
- return inputView.read(b, off, len);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java
deleted file mode 100644
index 513d7f8..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers;
-
-import org.apache.flink.core.memory.DataOutputView;
-
-import java.io.IOException;
-import java.io.OutputStream;
-
-/**
- * Wrapper for {@link org.apache.flink.core.memory.DataOutputView}. We need this because
- * Flink writes data using a {@link org.apache.flink.core.memory.DataInputView} while
- * Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s expect an
- * {@link java.io.OutputStream}.
- */
-public class DataOutputViewWrapper extends OutputStream {
-
- private DataOutputView outputView;
-
- public DataOutputViewWrapper(DataOutputView outputView) {
- this.outputView = outputView;
- }
-
- public void setOutputView(DataOutputView outputView) {
- this.outputView = outputView;
- }
-
- @Override
- public void write(int b) throws IOException {
- outputView.write(b);
- }
-
- @Override
- public void write(byte[] b, int off, int len) throws IOException {
- outputView.write(b, off, len);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java
deleted file mode 100644
index 0d03f9f..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers;
-
-import com.google.cloud.dataflow.sdk.transforms.Aggregator;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
-import org.apache.flink.api.common.accumulators.Accumulator;
-
-import java.io.Serializable;
-
-/**
- * Wrapper that wraps a {@link com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn}
- * in a Flink {@link org.apache.flink.api.common.accumulators.Accumulator} for using
- * the function as an aggregator in a {@link com.google.cloud.dataflow.sdk.transforms.ParDo}
- * operation.
- */
-public class SerializableFnAggregatorWrapper<AI, AO> implements Aggregator<AI, AO>, Accumulator<AI, Serializable> {
-
- private AO aa;
- private Combine.CombineFn<AI, ?, AO> combiner;
-
- public SerializableFnAggregatorWrapper(Combine.CombineFn<AI, ?, AO> combiner) {
- this.combiner = combiner;
- resetLocal();
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public void add(AI value) {
- this.aa = combiner.apply(ImmutableList.of((AI) aa, value));
- }
-
- @Override
- public Serializable getLocalValue() {
- return (Serializable) aa;
- }
-
- @Override
- public void resetLocal() {
- this.aa = combiner.apply(ImmutableList.<AI>of());
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public void merge(Accumulator<AI, Serializable> other) {
- this.aa = combiner.apply(ImmutableList.of((AI) aa, (AI) other.getLocalValue()));
- }
-
- @Override
- public void addValue(AI value) {
- add(value);
- }
-
- @Override
- public String getName() {
- return "Aggregator :" + combiner.toString();
- }
-
- @Override
- public Combine.CombineFn<AI, ?, AO> getCombineFn() {
- return combiner;
- }
-
- @Override
- public Accumulator<AI, Serializable> clone() {
- // copy it by merging
- AO resultCopy = combiner.apply(Lists.newArrayList((AI) aa));
- SerializableFnAggregatorWrapper<AI, AO> result = new
- SerializableFnAggregatorWrapper<>(combiner);
-
- result.aa = resultCopy;
- return result;
- }
-}
[44/50] [abbrv] incubator-beam git commit: [flink] convert tabs to 2
spaces
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java
index 0befa88..3cc5c24 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/UnionCoder.java
@@ -38,113 +38,113 @@ import java.util.List;
*/
@SuppressWarnings("serial")
public class UnionCoder extends StandardCoder<RawUnionValue> {
- // TODO: Think about how to integrate this with a schema object (i.e.
- // a tuple of tuple tags).
- /**
- * Builds a union coder with the given list of element coders. This list
- * corresponds to a mapping of union tag to Coder. Union tags start at 0.
- */
- public static UnionCoder of(List<Coder<?>> elementCoders) {
- return new UnionCoder(elementCoders);
- }
-
- @JsonCreator
- public static UnionCoder jsonOf(
- @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
- List<Coder<?>> elements) {
- return UnionCoder.of(elements);
- }
-
- private int getIndexForEncoding(RawUnionValue union) {
- if (union == null) {
- throw new IllegalArgumentException("cannot encode a null tagged union");
- }
- int index = union.getUnionTag();
- if (index < 0 || index >= elementCoders.size()) {
- throw new IllegalArgumentException(
- "union value index " + index + " not in range [0.." +
- (elementCoders.size() - 1) + "]");
- }
- return index;
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public void encode(
- RawUnionValue union,
- OutputStream outStream,
- Context context)
- throws IOException {
- int index = getIndexForEncoding(union);
- // Write out the union tag.
- VarInt.encode(index, outStream);
-
- // Write out the actual value.
- Coder<Object> coder = (Coder<Object>) elementCoders.get(index);
- coder.encode(
- union.getValue(),
- outStream,
- context);
- }
-
- @Override
- public RawUnionValue decode(InputStream inStream, Context context)
- throws IOException {
- int index = VarInt.decodeInt(inStream);
- Object value = elementCoders.get(index).decode(inStream, context);
- return new RawUnionValue(index, value);
- }
-
- @Override
- public List<? extends Coder<?>> getCoderArguments() {
- return null;
- }
-
- @Override
- public List<? extends Coder<?>> getComponents() {
- return elementCoders;
- }
-
- /**
- * Since this coder uses elementCoders.get(index) and coders that are known to run in constant
- * time, we defer the return value to that coder.
- */
- @Override
- public boolean isRegisterByteSizeObserverCheap(RawUnionValue union, Context context) {
- int index = getIndexForEncoding(union);
- @SuppressWarnings("unchecked")
- Coder<Object> coder = (Coder<Object>) elementCoders.get(index);
- return coder.isRegisterByteSizeObserverCheap(union.getValue(), context);
- }
-
- /**
- * Notifies ElementByteSizeObserver about the byte size of the encoded value using this coder.
- */
- @Override
- public void registerByteSizeObserver(
- RawUnionValue union, ElementByteSizeObserver observer, Context context)
- throws Exception {
- int index = getIndexForEncoding(union);
- // Write out the union tag.
- observer.update(VarInt.getLength(index));
- // Write out the actual value.
- @SuppressWarnings("unchecked")
- Coder<Object> coder = (Coder<Object>) elementCoders.get(index);
- coder.registerByteSizeObserver(union.getValue(), observer, context);
- }
-
- /////////////////////////////////////////////////////////////////////////////
-
- private final List<Coder<?>> elementCoders;
-
- private UnionCoder(List<Coder<?>> elementCoders) {
- this.elementCoders = elementCoders;
- }
-
- @Override
- public void verifyDeterministic() throws NonDeterministicException {
- verifyDeterministic(
- "UnionCoder is only deterministic if all element coders are",
- elementCoders);
- }
+ // TODO: Think about how to integrate this with a schema object (i.e.
+ // a tuple of tuple tags).
+ /**
+ * Builds a union coder with the given list of element coders. This list
+ * corresponds to a mapping of union tag to Coder. Union tags start at 0.
+ */
+ public static UnionCoder of(List<Coder<?>> elementCoders) {
+ return new UnionCoder(elementCoders);
+ }
+
+ @JsonCreator
+ public static UnionCoder jsonOf(
+ @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
+ List<Coder<?>> elements) {
+ return UnionCoder.of(elements);
+ }
+
+ private int getIndexForEncoding(RawUnionValue union) {
+ if (union == null) {
+ throw new IllegalArgumentException("cannot encode a null tagged union");
+ }
+ int index = union.getUnionTag();
+ if (index < 0 || index >= elementCoders.size()) {
+ throw new IllegalArgumentException(
+ "union value index " + index + " not in range [0.." +
+ (elementCoders.size() - 1) + "]");
+ }
+ return index;
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void encode(
+ RawUnionValue union,
+ OutputStream outStream,
+ Context context)
+ throws IOException {
+ int index = getIndexForEncoding(union);
+ // Write out the union tag.
+ VarInt.encode(index, outStream);
+
+ // Write out the actual value.
+ Coder<Object> coder = (Coder<Object>) elementCoders.get(index);
+ coder.encode(
+ union.getValue(),
+ outStream,
+ context);
+ }
+
+ @Override
+ public RawUnionValue decode(InputStream inStream, Context context)
+ throws IOException {
+ int index = VarInt.decodeInt(inStream);
+ Object value = elementCoders.get(index).decode(inStream, context);
+ return new RawUnionValue(index, value);
+ }
+
+ @Override
+ public List<? extends Coder<?>> getCoderArguments() {
+ return null;
+ }
+
+ @Override
+ public List<? extends Coder<?>> getComponents() {
+ return elementCoders;
+ }
+
+ /**
+ * Since this coder uses elementCoders.get(index) and coders that are known to run in constant
+ * time, we defer the return value to that coder.
+ */
+ @Override
+ public boolean isRegisterByteSizeObserverCheap(RawUnionValue union, Context context) {
+ int index = getIndexForEncoding(union);
+ @SuppressWarnings("unchecked")
+ Coder<Object> coder = (Coder<Object>) elementCoders.get(index);
+ return coder.isRegisterByteSizeObserverCheap(union.getValue(), context);
+ }
+
+ /**
+ * Notifies ElementByteSizeObserver about the byte size of the encoded value using this coder.
+ */
+ @Override
+ public void registerByteSizeObserver(
+ RawUnionValue union, ElementByteSizeObserver observer, Context context)
+ throws Exception {
+ int index = getIndexForEncoding(union);
+ // Write out the union tag.
+ observer.update(VarInt.getLength(index));
+ // Write out the actual value.
+ @SuppressWarnings("unchecked")
+ Coder<Object> coder = (Coder<Object>) elementCoders.get(index);
+ coder.registerByteSizeObserver(union.getValue(), observer, context);
+ }
+
+ /////////////////////////////////////////////////////////////////////////////
+
+ private final List<Coder<?>> elementCoders;
+
+ private UnionCoder(List<Coder<?>> elementCoders) {
+ this.elementCoders = elementCoders;
+ }
+
+ @Override
+ public void verifyDeterministic() throws NonDeterministicException {
+ verifyDeterministic(
+ "UnionCoder is only deterministic if all element coders are",
+ elementCoders);
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java
index e433589..b402f7c 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java
@@ -32,185 +32,185 @@ import java.io.ObjectInputStream;
*/
public class CoderComparator<T> extends TypeComparator<T> {
- private Coder<T> coder;
-
- // We use these for internal encoding/decoding for creating copies and comparing
- // serialized forms using a Coder
- private transient InspectableByteArrayOutputStream buffer1;
- private transient InspectableByteArrayOutputStream buffer2;
-
- // For storing the Reference in encoded form
- private transient InspectableByteArrayOutputStream referenceBuffer;
-
- public CoderComparator(Coder<T> coder) {
- this.coder = coder;
- buffer1 = new InspectableByteArrayOutputStream();
- buffer2 = new InspectableByteArrayOutputStream();
- referenceBuffer = new InspectableByteArrayOutputStream();
- }
-
- private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- buffer1 = new InspectableByteArrayOutputStream();
- buffer2 = new InspectableByteArrayOutputStream();
- referenceBuffer = new InspectableByteArrayOutputStream();
- }
-
- @Override
- public int hash(T record) {
- return record.hashCode();
- }
-
- @Override
- public void setReference(T toCompare) {
- referenceBuffer.reset();
- try {
- coder.encode(toCompare, referenceBuffer, Coder.Context.OUTER);
- } catch (IOException e) {
- throw new RuntimeException("Could not set reference " + toCompare + ": " + e);
- }
- }
-
- @Override
- public boolean equalToReference(T candidate) {
- try {
- buffer2.reset();
- coder.encode(candidate, buffer2, Coder.Context.OUTER);
- byte[] arr = referenceBuffer.getBuffer();
- byte[] arrOther = buffer2.getBuffer();
- if (referenceBuffer.size() != buffer2.size()) {
- return false;
- }
- int len = buffer2.size();
- for(int i = 0; i < len; i++ ) {
- if (arr[i] != arrOther[i]) {
- return false;
- }
- }
- return true;
- } catch (IOException e) {
- throw new RuntimeException("Could not compare reference.", e);
- }
- }
-
- @Override
- public int compareToReference(TypeComparator<T> other) {
- InspectableByteArrayOutputStream otherReferenceBuffer = ((CoderComparator<T>) other).referenceBuffer;
-
- byte[] arr = referenceBuffer.getBuffer();
- byte[] arrOther = otherReferenceBuffer.getBuffer();
- if (referenceBuffer.size() != otherReferenceBuffer.size()) {
- return referenceBuffer.size() - otherReferenceBuffer.size();
- }
- int len = referenceBuffer.size();
- for (int i = 0; i < len; i++) {
- if (arr[i] != arrOther[i]) {
- return arr[i] - arrOther[i];
- }
- }
- return 0;
- }
-
- @Override
- public int compare(T first, T second) {
- try {
- buffer1.reset();
- buffer2.reset();
- coder.encode(first, buffer1, Coder.Context.OUTER);
- coder.encode(second, buffer2, Coder.Context.OUTER);
- byte[] arr = buffer1.getBuffer();
- byte[] arrOther = buffer2.getBuffer();
- if (buffer1.size() != buffer2.size()) {
- return buffer1.size() - buffer2.size();
- }
- int len = buffer1.size();
- for(int i = 0; i < len; i++ ) {
- if (arr[i] != arrOther[i]) {
- return arr[i] - arrOther[i];
- }
- }
- return 0;
- } catch (IOException e) {
- throw new RuntimeException("Could not compare: ", e);
- }
- }
-
- @Override
- public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
- CoderTypeSerializer<T> serializer = new CoderTypeSerializer<>(coder);
- T first = serializer.deserialize(firstSource);
- T second = serializer.deserialize(secondSource);
- return compare(first, second);
- }
-
- @Override
- public boolean supportsNormalizedKey() {
- return true;
- }
-
- @Override
- public boolean supportsSerializationWithKeyNormalization() {
- return false;
- }
-
- @Override
- public int getNormalizeKeyLen() {
- return Integer.MAX_VALUE;
- }
-
- @Override
- public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
- return true;
- }
-
- @Override
- public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) {
- buffer1.reset();
- try {
- coder.encode(record, buffer1, Coder.Context.OUTER);
- } catch (IOException e) {
- throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e);
- }
- final byte[] data = buffer1.getBuffer();
- final int limit = offset + numBytes;
-
- target.put(offset, data, 0, Math.min(numBytes, buffer1.size()));
-
- offset += buffer1.size();
-
- while (offset < limit) {
- target.put(offset++, (byte) 0);
- }
- }
-
- @Override
- public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public boolean invertNormalizedKey() {
- return false;
- }
-
- @Override
- public TypeComparator<T> duplicate() {
- return new CoderComparator<>(coder);
- }
-
- @Override
- public int extractKeys(Object record, Object[] target, int index) {
- target[index] = record;
- return 1;
- }
-
- @Override
- public TypeComparator[] getFlatComparators() {
- return new TypeComparator[] { this.duplicate() };
- }
+ private Coder<T> coder;
+
+ // We use these for internal encoding/decoding for creating copies and comparing
+ // serialized forms using a Coder
+ private transient InspectableByteArrayOutputStream buffer1;
+ private transient InspectableByteArrayOutputStream buffer2;
+
+ // For storing the Reference in encoded form
+ private transient InspectableByteArrayOutputStream referenceBuffer;
+
+ public CoderComparator(Coder<T> coder) {
+ this.coder = coder;
+ buffer1 = new InspectableByteArrayOutputStream();
+ buffer2 = new InspectableByteArrayOutputStream();
+ referenceBuffer = new InspectableByteArrayOutputStream();
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ buffer1 = new InspectableByteArrayOutputStream();
+ buffer2 = new InspectableByteArrayOutputStream();
+ referenceBuffer = new InspectableByteArrayOutputStream();
+ }
+
+ @Override
+ public int hash(T record) {
+ return record.hashCode();
+ }
+
+ @Override
+ public void setReference(T toCompare) {
+ referenceBuffer.reset();
+ try {
+ coder.encode(toCompare, referenceBuffer, Coder.Context.OUTER);
+ } catch (IOException e) {
+ throw new RuntimeException("Could not set reference " + toCompare + ": " + e);
+ }
+ }
+
+ @Override
+ public boolean equalToReference(T candidate) {
+ try {
+ buffer2.reset();
+ coder.encode(candidate, buffer2, Coder.Context.OUTER);
+ byte[] arr = referenceBuffer.getBuffer();
+ byte[] arrOther = buffer2.getBuffer();
+ if (referenceBuffer.size() != buffer2.size()) {
+ return false;
+ }
+ int len = buffer2.size();
+ for(int i = 0; i < len; i++ ) {
+ if (arr[i] != arrOther[i]) {
+ return false;
+ }
+ }
+ return true;
+ } catch (IOException e) {
+ throw new RuntimeException("Could not compare reference.", e);
+ }
+ }
+
+ @Override
+ public int compareToReference(TypeComparator<T> other) {
+ InspectableByteArrayOutputStream otherReferenceBuffer = ((CoderComparator<T>) other).referenceBuffer;
+
+ byte[] arr = referenceBuffer.getBuffer();
+ byte[] arrOther = otherReferenceBuffer.getBuffer();
+ if (referenceBuffer.size() != otherReferenceBuffer.size()) {
+ return referenceBuffer.size() - otherReferenceBuffer.size();
+ }
+ int len = referenceBuffer.size();
+ for (int i = 0; i < len; i++) {
+ if (arr[i] != arrOther[i]) {
+ return arr[i] - arrOther[i];
+ }
+ }
+ return 0;
+ }
+
+ @Override
+ public int compare(T first, T second) {
+ try {
+ buffer1.reset();
+ buffer2.reset();
+ coder.encode(first, buffer1, Coder.Context.OUTER);
+ coder.encode(second, buffer2, Coder.Context.OUTER);
+ byte[] arr = buffer1.getBuffer();
+ byte[] arrOther = buffer2.getBuffer();
+ if (buffer1.size() != buffer2.size()) {
+ return buffer1.size() - buffer2.size();
+ }
+ int len = buffer1.size();
+ for(int i = 0; i < len; i++ ) {
+ if (arr[i] != arrOther[i]) {
+ return arr[i] - arrOther[i];
+ }
+ }
+ return 0;
+ } catch (IOException e) {
+ throw new RuntimeException("Could not compare: ", e);
+ }
+ }
+
+ @Override
+ public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
+ CoderTypeSerializer<T> serializer = new CoderTypeSerializer<>(coder);
+ T first = serializer.deserialize(firstSource);
+ T second = serializer.deserialize(secondSource);
+ return compare(first, second);
+ }
+
+ @Override
+ public boolean supportsNormalizedKey() {
+ return true;
+ }
+
+ @Override
+ public boolean supportsSerializationWithKeyNormalization() {
+ return false;
+ }
+
+ @Override
+ public int getNormalizeKeyLen() {
+ return Integer.MAX_VALUE;
+ }
+
+ @Override
+ public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
+ return true;
+ }
+
+ @Override
+ public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) {
+ buffer1.reset();
+ try {
+ coder.encode(record, buffer1, Coder.Context.OUTER);
+ } catch (IOException e) {
+ throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e);
+ }
+ final byte[] data = buffer1.getBuffer();
+ final int limit = offset + numBytes;
+
+ target.put(offset, data, 0, Math.min(numBytes, buffer1.size()));
+
+ offset += buffer1.size();
+
+ while (offset < limit) {
+ target.put(offset++, (byte) 0);
+ }
+ }
+
+ @Override
+ public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean invertNormalizedKey() {
+ return false;
+ }
+
+ @Override
+ public TypeComparator<T> duplicate() {
+ return new CoderComparator<>(coder);
+ }
+
+ @Override
+ public int extractKeys(Object record, Object[] target, int index) {
+ target[index] = record;
+ return 1;
+ }
+
+ @Override
+ public TypeComparator[] getFlatComparators() {
+ return new TypeComparator[] { this.duplicate() };
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java
index dd9c5f6..ae4309e 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java
@@ -32,85 +32,85 @@ import com.google.common.base.Preconditions;
*/
public class CoderTypeInformation<T> extends TypeInformation<T> implements AtomicType<T> {
- private final Coder<T> coder;
-
- public CoderTypeInformation(Coder<T> coder) {
- Preconditions.checkNotNull(coder);
- this.coder = coder;
- }
-
- @Override
- public boolean isBasicType() {
- return false;
- }
-
- @Override
- public boolean isTupleType() {
- return false;
- }
-
- @Override
- public int getArity() {
- return 1;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public Class<T> getTypeClass() {
- // We don't have the Class, so we have to pass null here. What a shame...
- return (Class<T>) Object.class;
- }
-
- @Override
- public boolean isKeyType() {
- return true;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public TypeSerializer<T> createSerializer(ExecutionConfig config) {
- if (coder instanceof VoidCoder) {
- return (TypeSerializer<T>) new VoidCoderTypeSerializer();
- }
- return new CoderTypeSerializer<>(coder);
- }
-
- @Override
- public int getTotalFields() {
- return 2;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
-
- CoderTypeInformation that = (CoderTypeInformation) o;
-
- return coder.equals(that.coder);
-
- }
-
- @Override
- public int hashCode() {
- return coder.hashCode();
- }
-
- @Override
- public boolean canEqual(Object obj) {
- return obj instanceof CoderTypeInformation;
- }
-
- @Override
- public String toString() {
- return "CoderTypeInformation{" +
- "coder=" + coder +
- '}';
- }
-
- @Override
- public TypeComparator<T> createComparator(boolean sortOrderAscending, ExecutionConfig
- executionConfig) {
- return new CoderComparator<>(coder);
- }
+ private final Coder<T> coder;
+
+ public CoderTypeInformation(Coder<T> coder) {
+ Preconditions.checkNotNull(coder);
+ this.coder = coder;
+ }
+
+ @Override
+ public boolean isBasicType() {
+ return false;
+ }
+
+ @Override
+ public boolean isTupleType() {
+ return false;
+ }
+
+ @Override
+ public int getArity() {
+ return 1;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public Class<T> getTypeClass() {
+ // We don't have the Class, so we have to pass null here. What a shame...
+ return (Class<T>) Object.class;
+ }
+
+ @Override
+ public boolean isKeyType() {
+ return true;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public TypeSerializer<T> createSerializer(ExecutionConfig config) {
+ if (coder instanceof VoidCoder) {
+ return (TypeSerializer<T>) new VoidCoderTypeSerializer();
+ }
+ return new CoderTypeSerializer<>(coder);
+ }
+
+ @Override
+ public int getTotalFields() {
+ return 2;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ CoderTypeInformation that = (CoderTypeInformation) o;
+
+ return coder.equals(that.coder);
+
+ }
+
+ @Override
+ public int hashCode() {
+ return coder.hashCode();
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof CoderTypeInformation;
+ }
+
+ @Override
+ public String toString() {
+ return "CoderTypeInformation{" +
+ "coder=" + coder +
+ '}';
+ }
+
+ @Override
+ public TypeComparator<T> createComparator(boolean sortOrderAscending, ExecutionConfig
+ executionConfig) {
+ return new CoderComparator<>(coder);
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java
index f739397..6ed661c 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java
@@ -35,118 +35,118 @@ import java.io.ObjectInputStream;
* Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s
*/
public class CoderTypeSerializer<T> extends TypeSerializer<T> {
-
- private Coder<T> coder;
- private transient DataInputViewWrapper inputWrapper;
- private transient DataOutputViewWrapper outputWrapper;
-
- // We use this for internal encoding/decoding for creating copies using the Coder.
- private transient InspectableByteArrayOutputStream buffer;
-
- public CoderTypeSerializer(Coder<T> coder) {
- this.coder = coder;
- this.inputWrapper = new DataInputViewWrapper(null);
- this.outputWrapper = new DataOutputViewWrapper(null);
-
- buffer = new InspectableByteArrayOutputStream();
- }
-
- private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- this.inputWrapper = new DataInputViewWrapper(null);
- this.outputWrapper = new DataOutputViewWrapper(null);
-
- buffer = new InspectableByteArrayOutputStream();
- }
-
- @Override
- public boolean isImmutableType() {
- return false;
- }
-
- @Override
- public CoderTypeSerializer<T> duplicate() {
- return new CoderTypeSerializer<>(coder);
- }
-
- @Override
- public T createInstance() {
- return null;
- }
-
- @Override
- public T copy(T t) {
- buffer.reset();
- try {
- coder.encode(t, buffer, Coder.Context.OUTER);
- } catch (IOException e) {
- throw new RuntimeException("Could not copy.", e);
- }
- try {
- return coder.decode(new ByteArrayInputStream(buffer.getBuffer(), 0, buffer
- .size()), Coder.Context.OUTER);
- } catch (IOException e) {
- throw new RuntimeException("Could not copy.", e);
- }
- }
-
- @Override
- public T copy(T t, T reuse) {
- return copy(t);
- }
-
- @Override
- public int getLength() {
- return 0;
- }
-
- @Override
- public void serialize(T t, DataOutputView dataOutputView) throws IOException {
- outputWrapper.setOutputView(dataOutputView);
- coder.encode(t, outputWrapper, Coder.Context.NESTED);
- }
-
- @Override
- public T deserialize(DataInputView dataInputView) throws IOException {
- try {
- inputWrapper.setInputView(dataInputView);
- return coder.decode(inputWrapper, Coder.Context.NESTED);
- } catch (CoderException e) {
- Throwable cause = e.getCause();
- if (cause instanceof EOFException) {
- throw (EOFException) cause;
- } else {
- throw e;
- }
- }
- }
-
- @Override
- public T deserialize(T t, DataInputView dataInputView) throws IOException {
- return deserialize(dataInputView);
- }
-
- @Override
- public void copy(DataInputView dataInputView, DataOutputView dataOutputView) throws IOException {
- serialize(deserialize(dataInputView), dataOutputView);
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
-
- CoderTypeSerializer that = (CoderTypeSerializer) o;
- return coder.equals(that.coder);
- }
-
- @Override
- public boolean canEqual(Object obj) {
- return obj instanceof CoderTypeSerializer;
- }
-
- @Override
- public int hashCode() {
- return coder.hashCode();
- }
+
+ private Coder<T> coder;
+ private transient DataInputViewWrapper inputWrapper;
+ private transient DataOutputViewWrapper outputWrapper;
+
+ // We use this for internal encoding/decoding for creating copies using the Coder.
+ private transient InspectableByteArrayOutputStream buffer;
+
+ public CoderTypeSerializer(Coder<T> coder) {
+ this.coder = coder;
+ this.inputWrapper = new DataInputViewWrapper(null);
+ this.outputWrapper = new DataOutputViewWrapper(null);
+
+ buffer = new InspectableByteArrayOutputStream();
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ this.inputWrapper = new DataInputViewWrapper(null);
+ this.outputWrapper = new DataOutputViewWrapper(null);
+
+ buffer = new InspectableByteArrayOutputStream();
+ }
+
+ @Override
+ public boolean isImmutableType() {
+ return false;
+ }
+
+ @Override
+ public CoderTypeSerializer<T> duplicate() {
+ return new CoderTypeSerializer<>(coder);
+ }
+
+ @Override
+ public T createInstance() {
+ return null;
+ }
+
+ @Override
+ public T copy(T t) {
+ buffer.reset();
+ try {
+ coder.encode(t, buffer, Coder.Context.OUTER);
+ } catch (IOException e) {
+ throw new RuntimeException("Could not copy.", e);
+ }
+ try {
+ return coder.decode(new ByteArrayInputStream(buffer.getBuffer(), 0, buffer
+ .size()), Coder.Context.OUTER);
+ } catch (IOException e) {
+ throw new RuntimeException("Could not copy.", e);
+ }
+ }
+
+ @Override
+ public T copy(T t, T reuse) {
+ return copy(t);
+ }
+
+ @Override
+ public int getLength() {
+ return 0;
+ }
+
+ @Override
+ public void serialize(T t, DataOutputView dataOutputView) throws IOException {
+ outputWrapper.setOutputView(dataOutputView);
+ coder.encode(t, outputWrapper, Coder.Context.NESTED);
+ }
+
+ @Override
+ public T deserialize(DataInputView dataInputView) throws IOException {
+ try {
+ inputWrapper.setInputView(dataInputView);
+ return coder.decode(inputWrapper, Coder.Context.NESTED);
+ } catch (CoderException e) {
+ Throwable cause = e.getCause();
+ if (cause instanceof EOFException) {
+ throw (EOFException) cause;
+ } else {
+ throw e;
+ }
+ }
+ }
+
+ @Override
+ public T deserialize(T t, DataInputView dataInputView) throws IOException {
+ return deserialize(dataInputView);
+ }
+
+ @Override
+ public void copy(DataInputView dataInputView, DataOutputView dataOutputView) throws IOException {
+ serialize(deserialize(dataInputView), dataOutputView);
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ CoderTypeSerializer that = (CoderTypeSerializer) o;
+ return coder.equals(that.coder);
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof CoderTypeSerializer;
+ }
+
+ @Override
+ public int hashCode() {
+ return coder.hashCode();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java
index 5d918cc..be6eadd 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/InspectableByteArrayOutputStream.java
@@ -25,10 +25,10 @@ import java.io.ByteArrayOutputStream;
*/
public class InspectableByteArrayOutputStream extends ByteArrayOutputStream {
- /**
- * Get the underlying byte array.
- */
- public byte[] getBuffer() {
- return buf;
- }
+ /**
+ * Get the underlying byte array.
+ */
+ public byte[] getBuffer() {
+ return buf;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java
index 815569d..ba09ea9 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java
@@ -35,230 +35,230 @@ import java.io.ObjectInputStream;
* for {@link KV} that always compares on the key only.
*/
public class KvCoderComperator <K, V> extends TypeComparator<KV<K, V>> {
-
- private KvCoder<K, V> coder;
- private Coder<K> keyCoder;
-
- // We use these for internal encoding/decoding for creating copies and comparing
- // serialized forms using a Coder
- private transient InspectableByteArrayOutputStream buffer1;
- private transient InspectableByteArrayOutputStream buffer2;
-
- // For storing the Reference in encoded form
- private transient InspectableByteArrayOutputStream referenceBuffer;
-
-
- // For deserializing the key
- private transient DataInputViewWrapper inputWrapper;
-
- public KvCoderComperator(KvCoder<K, V> coder) {
- this.coder = coder;
- this.keyCoder = coder.getKeyCoder();
-
- buffer1 = new InspectableByteArrayOutputStream();
- buffer2 = new InspectableByteArrayOutputStream();
- referenceBuffer = new InspectableByteArrayOutputStream();
-
- inputWrapper = new DataInputViewWrapper(null);
- }
-
- private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
- in.defaultReadObject();
-
- buffer1 = new InspectableByteArrayOutputStream();
- buffer2 = new InspectableByteArrayOutputStream();
- referenceBuffer = new InspectableByteArrayOutputStream();
-
- inputWrapper = new DataInputViewWrapper(null);
- }
-
- @Override
- public int hash(KV<K, V> record) {
- K key = record.getKey();
- if (key != null) {
- return key.hashCode();
- } else {
- return 0;
- }
- }
-
- @Override
- public void setReference(KV<K, V> toCompare) {
- referenceBuffer.reset();
- try {
- keyCoder.encode(toCompare.getKey(), referenceBuffer, Coder.Context.OUTER);
- } catch (IOException e) {
- throw new RuntimeException("Could not set reference " + toCompare + ": " + e);
- }
- }
-
- @Override
- public boolean equalToReference(KV<K, V> candidate) {
- try {
- buffer2.reset();
- keyCoder.encode(candidate.getKey(), buffer2, Coder.Context.OUTER);
- byte[] arr = referenceBuffer.getBuffer();
- byte[] arrOther = buffer2.getBuffer();
- if (referenceBuffer.size() != buffer2.size()) {
- return false;
- }
- int len = buffer2.size();
- for(int i = 0; i < len; i++ ) {
- if (arr[i] != arrOther[i]) {
- return false;
- }
- }
- return true;
- } catch (IOException e) {
- throw new RuntimeException("Could not compare reference.", e);
- }
- }
-
- @Override
- public int compareToReference(TypeComparator<KV<K, V>> other) {
- InspectableByteArrayOutputStream otherReferenceBuffer = ((KvCoderComperator<K, V>) other).referenceBuffer;
-
- byte[] arr = referenceBuffer.getBuffer();
- byte[] arrOther = otherReferenceBuffer.getBuffer();
- if (referenceBuffer.size() != otherReferenceBuffer.size()) {
- return referenceBuffer.size() - otherReferenceBuffer.size();
- }
- int len = referenceBuffer.size();
- for (int i = 0; i < len; i++) {
- if (arr[i] != arrOther[i]) {
- return arr[i] - arrOther[i];
- }
- }
- return 0;
- }
-
-
- @Override
- public int compare(KV<K, V> first, KV<K, V> second) {
- try {
- buffer1.reset();
- buffer2.reset();
- keyCoder.encode(first.getKey(), buffer1, Coder.Context.OUTER);
- keyCoder.encode(second.getKey(), buffer2, Coder.Context.OUTER);
- byte[] arr = buffer1.getBuffer();
- byte[] arrOther = buffer2.getBuffer();
- if (buffer1.size() != buffer2.size()) {
- return buffer1.size() - buffer2.size();
- }
- int len = buffer1.size();
- for(int i = 0; i < len; i++ ) {
- if (arr[i] != arrOther[i]) {
- return arr[i] - arrOther[i];
- }
- }
- return 0;
- } catch (IOException e) {
- throw new RuntimeException("Could not compare reference.", e);
- }
- }
-
- @Override
- public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
-
- inputWrapper.setInputView(firstSource);
- K firstKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED);
- inputWrapper.setInputView(secondSource);
- K secondKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED);
-
- try {
- buffer1.reset();
- buffer2.reset();
- keyCoder.encode(firstKey, buffer1, Coder.Context.OUTER);
- keyCoder.encode(secondKey, buffer2, Coder.Context.OUTER);
- byte[] arr = buffer1.getBuffer();
- byte[] arrOther = buffer2.getBuffer();
- if (buffer1.size() != buffer2.size()) {
- return buffer1.size() - buffer2.size();
- }
- int len = buffer1.size();
- for(int i = 0; i < len; i++ ) {
- if (arr[i] != arrOther[i]) {
- return arr[i] - arrOther[i];
- }
- }
- return 0;
- } catch (IOException e) {
- throw new RuntimeException("Could not compare reference.", e);
- }
- }
-
- @Override
- public boolean supportsNormalizedKey() {
- return true;
- }
-
- @Override
- public boolean supportsSerializationWithKeyNormalization() {
- return false;
- }
-
- @Override
- public int getNormalizeKeyLen() {
- return Integer.MAX_VALUE;
- }
-
- @Override
- public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
- return true;
- }
-
- @Override
- public void putNormalizedKey(KV<K, V> record, MemorySegment target, int offset, int numBytes) {
- buffer1.reset();
- try {
- keyCoder.encode(record.getKey(), buffer1, Coder.Context.NESTED);
- } catch (IOException e) {
- throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e);
- }
- final byte[] data = buffer1.getBuffer();
- final int limit = offset + numBytes;
-
- int numBytesPut = Math.min(numBytes, buffer1.size());
-
- target.put(offset, data, 0, numBytesPut);
-
- offset += numBytesPut;
-
- while (offset < limit) {
- target.put(offset++, (byte) 0);
- }
- }
-
- @Override
- public void writeWithKeyNormalization(KV<K, V> record, DataOutputView target) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public KV<K, V> readWithKeyDenormalization(KV<K, V> reuse, DataInputView source) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public boolean invertNormalizedKey() {
- return false;
- }
-
- @Override
- public TypeComparator<KV<K, V>> duplicate() {
- return new KvCoderComperator<>(coder);
- }
-
- @Override
- public int extractKeys(Object record, Object[] target, int index) {
- KV<K, V> kv = (KV<K, V>) record;
- K k = kv.getKey();
- target[index] = k;
- return 1;
- }
-
- @Override
- public TypeComparator[] getFlatComparators() {
- return new TypeComparator[] {new CoderComparator<>(keyCoder)};
- }
+
+ private KvCoder<K, V> coder;
+ private Coder<K> keyCoder;
+
+ // We use these for internal encoding/decoding for creating copies and comparing
+ // serialized forms using a Coder
+ private transient InspectableByteArrayOutputStream buffer1;
+ private transient InspectableByteArrayOutputStream buffer2;
+
+ // For storing the Reference in encoded form
+ private transient InspectableByteArrayOutputStream referenceBuffer;
+
+
+ // For deserializing the key
+ private transient DataInputViewWrapper inputWrapper;
+
+ public KvCoderComperator(KvCoder<K, V> coder) {
+ this.coder = coder;
+ this.keyCoder = coder.getKeyCoder();
+
+ buffer1 = new InspectableByteArrayOutputStream();
+ buffer2 = new InspectableByteArrayOutputStream();
+ referenceBuffer = new InspectableByteArrayOutputStream();
+
+ inputWrapper = new DataInputViewWrapper(null);
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+
+ buffer1 = new InspectableByteArrayOutputStream();
+ buffer2 = new InspectableByteArrayOutputStream();
+ referenceBuffer = new InspectableByteArrayOutputStream();
+
+ inputWrapper = new DataInputViewWrapper(null);
+ }
+
+ @Override
+ public int hash(KV<K, V> record) {
+ K key = record.getKey();
+ if (key != null) {
+ return key.hashCode();
+ } else {
+ return 0;
+ }
+ }
+
+ @Override
+ public void setReference(KV<K, V> toCompare) {
+ referenceBuffer.reset();
+ try {
+ keyCoder.encode(toCompare.getKey(), referenceBuffer, Coder.Context.OUTER);
+ } catch (IOException e) {
+ throw new RuntimeException("Could not set reference " + toCompare + ": " + e);
+ }
+ }
+
+ @Override
+ public boolean equalToReference(KV<K, V> candidate) {
+ try {
+ buffer2.reset();
+ keyCoder.encode(candidate.getKey(), buffer2, Coder.Context.OUTER);
+ byte[] arr = referenceBuffer.getBuffer();
+ byte[] arrOther = buffer2.getBuffer();
+ if (referenceBuffer.size() != buffer2.size()) {
+ return false;
+ }
+ int len = buffer2.size();
+ for(int i = 0; i < len; i++ ) {
+ if (arr[i] != arrOther[i]) {
+ return false;
+ }
+ }
+ return true;
+ } catch (IOException e) {
+ throw new RuntimeException("Could not compare reference.", e);
+ }
+ }
+
+ @Override
+ public int compareToReference(TypeComparator<KV<K, V>> other) {
+ InspectableByteArrayOutputStream otherReferenceBuffer = ((KvCoderComperator<K, V>) other).referenceBuffer;
+
+ byte[] arr = referenceBuffer.getBuffer();
+ byte[] arrOther = otherReferenceBuffer.getBuffer();
+ if (referenceBuffer.size() != otherReferenceBuffer.size()) {
+ return referenceBuffer.size() - otherReferenceBuffer.size();
+ }
+ int len = referenceBuffer.size();
+ for (int i = 0; i < len; i++) {
+ if (arr[i] != arrOther[i]) {
+ return arr[i] - arrOther[i];
+ }
+ }
+ return 0;
+ }
+
+
+ @Override
+ public int compare(KV<K, V> first, KV<K, V> second) {
+ try {
+ buffer1.reset();
+ buffer2.reset();
+ keyCoder.encode(first.getKey(), buffer1, Coder.Context.OUTER);
+ keyCoder.encode(second.getKey(), buffer2, Coder.Context.OUTER);
+ byte[] arr = buffer1.getBuffer();
+ byte[] arrOther = buffer2.getBuffer();
+ if (buffer1.size() != buffer2.size()) {
+ return buffer1.size() - buffer2.size();
+ }
+ int len = buffer1.size();
+ for(int i = 0; i < len; i++ ) {
+ if (arr[i] != arrOther[i]) {
+ return arr[i] - arrOther[i];
+ }
+ }
+ return 0;
+ } catch (IOException e) {
+ throw new RuntimeException("Could not compare reference.", e);
+ }
+ }
+
+ @Override
+ public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
+
+ inputWrapper.setInputView(firstSource);
+ K firstKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED);
+ inputWrapper.setInputView(secondSource);
+ K secondKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED);
+
+ try {
+ buffer1.reset();
+ buffer2.reset();
+ keyCoder.encode(firstKey, buffer1, Coder.Context.OUTER);
+ keyCoder.encode(secondKey, buffer2, Coder.Context.OUTER);
+ byte[] arr = buffer1.getBuffer();
+ byte[] arrOther = buffer2.getBuffer();
+ if (buffer1.size() != buffer2.size()) {
+ return buffer1.size() - buffer2.size();
+ }
+ int len = buffer1.size();
+ for(int i = 0; i < len; i++ ) {
+ if (arr[i] != arrOther[i]) {
+ return arr[i] - arrOther[i];
+ }
+ }
+ return 0;
+ } catch (IOException e) {
+ throw new RuntimeException("Could not compare reference.", e);
+ }
+ }
+
+ @Override
+ public boolean supportsNormalizedKey() {
+ return true;
+ }
+
+ @Override
+ public boolean supportsSerializationWithKeyNormalization() {
+ return false;
+ }
+
+ @Override
+ public int getNormalizeKeyLen() {
+ return Integer.MAX_VALUE;
+ }
+
+ @Override
+ public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
+ return true;
+ }
+
+ @Override
+ public void putNormalizedKey(KV<K, V> record, MemorySegment target, int offset, int numBytes) {
+ buffer1.reset();
+ try {
+ keyCoder.encode(record.getKey(), buffer1, Coder.Context.NESTED);
+ } catch (IOException e) {
+ throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e);
+ }
+ final byte[] data = buffer1.getBuffer();
+ final int limit = offset + numBytes;
+
+ int numBytesPut = Math.min(numBytes, buffer1.size());
+
+ target.put(offset, data, 0, numBytesPut);
+
+ offset += numBytesPut;
+
+ while (offset < limit) {
+ target.put(offset++, (byte) 0);
+ }
+ }
+
+ @Override
+ public void writeWithKeyNormalization(KV<K, V> record, DataOutputView target) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public KV<K, V> readWithKeyDenormalization(KV<K, V> reuse, DataInputView source) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean invertNormalizedKey() {
+ return false;
+ }
+
+ @Override
+ public TypeComparator<KV<K, V>> duplicate() {
+ return new KvCoderComperator<>(coder);
+ }
+
+ @Override
+ public int extractKeys(Object record, Object[] target, int index) {
+ KV<K, V> kv = (KV<K, V>) record;
+ K k = kv.getKey();
+ target[index] = k;
+ return 1;
+ }
+
+ @Override
+ public TypeComparator[] getFlatComparators() {
+ return new TypeComparator[] {new CoderComparator<>(keyCoder)};
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java
index 090f79d..be11918 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderTypeInformation.java
@@ -34,153 +34,153 @@ import java.util.List;
*/
public class KvCoderTypeInformation<K, V> extends CompositeType<KV<K, V>> {
- private KvCoder<K, V> coder;
-
- // We don't have the Class, so we have to pass null here. What a shame...
- private static Object DUMMY = new Object();
-
- @SuppressWarnings("unchecked")
- public KvCoderTypeInformation(KvCoder<K, V> coder) {
- super(((Class<KV<K,V>>) DUMMY.getClass()));
- this.coder = coder;
- Preconditions.checkNotNull(coder);
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public TypeComparator<KV<K, V>> createComparator(int[] logicalKeyFields, boolean[] orders, int logicalFieldOffset, ExecutionConfig config) {
- return new KvCoderComperator((KvCoder) coder);
- }
-
- @Override
- public boolean isBasicType() {
- return false;
- }
-
- @Override
- public boolean isTupleType() {
- return false;
- }
-
- @Override
- public int getArity() {
- return 2;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public Class<KV<K, V>> getTypeClass() {
- return privateGetTypeClass();
- }
-
- @SuppressWarnings("unchecked")
- private static <X> Class<X> privateGetTypeClass() {
- return (Class<X>) Object.class;
- }
-
- @Override
- public boolean isKeyType() {
- return true;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public TypeSerializer<KV<K, V>> createSerializer(ExecutionConfig config) {
- return new CoderTypeSerializer<>(coder);
- }
-
- @Override
- public int getTotalFields() {
- return 2;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) return true;
- if (o == null || getClass() != o.getClass()) return false;
-
- KvCoderTypeInformation that = (KvCoderTypeInformation) o;
-
- return coder.equals(that.coder);
-
- }
-
- @Override
- public int hashCode() {
- return coder.hashCode();
- }
-
- @Override
- public String toString() {
- return "CoderTypeInformation{" +
- "coder=" + coder +
- '}';
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public <X> TypeInformation<X> getTypeAt(int pos) {
- if (pos == 0) {
- return (TypeInformation<X>) new CoderTypeInformation<>(coder.getKeyCoder());
- } else if (pos == 1) {
- return (TypeInformation<X>) new CoderTypeInformation<>(coder.getValueCoder());
- } else {
- throw new RuntimeException("Invalid field position " + pos);
- }
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public <X> TypeInformation<X> getTypeAt(String fieldExpression) {
- switch (fieldExpression) {
- case "key":
- return (TypeInformation<X>) new CoderTypeInformation<>(coder.getKeyCoder());
- case "value":
- return (TypeInformation<X>) new CoderTypeInformation<>(coder.getValueCoder());
- default:
- throw new UnsupportedOperationException("Only KvCoder has fields.");
- }
- }
-
- @Override
- public String[] getFieldNames() {
- return new String[]{"key", "value"};
- }
-
- @Override
- public int getFieldIndex(String fieldName) {
- switch (fieldName) {
- case "key":
- return 0;
- case "value":
- return 1;
- default:
- return -1;
- }
- }
-
- @Override
- public void getFlatFields(String fieldExpression, int offset, List<FlatFieldDescriptor> result) {
- CoderTypeInformation keyTypeInfo = new CoderTypeInformation<>(coder.getKeyCoder());
- result.add(new FlatFieldDescriptor(0, keyTypeInfo));
- }
-
- @Override
- protected TypeComparatorBuilder<KV<K, V>> createTypeComparatorBuilder() {
- return new KvCoderTypeComparatorBuilder();
- }
-
- private class KvCoderTypeComparatorBuilder implements TypeComparatorBuilder<KV<K, V>> {
-
- @Override
- public void initializeTypeComparatorBuilder(int size) {}
-
- @Override
- public void addComparatorField(int fieldId, TypeComparator<?> comparator) {}
-
- @Override
- public TypeComparator<KV<K, V>> createTypeComparator(ExecutionConfig config) {
- return new KvCoderComperator<>(coder);
- }
- }
+ private KvCoder<K, V> coder;
+
+ // We don't have the Class, so we have to pass null here. What a shame...
+ private static Object DUMMY = new Object();
+
+ @SuppressWarnings("unchecked")
+ public KvCoderTypeInformation(KvCoder<K, V> coder) {
+ super(((Class<KV<K,V>>) DUMMY.getClass()));
+ this.coder = coder;
+ Preconditions.checkNotNull(coder);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public TypeComparator<KV<K, V>> createComparator(int[] logicalKeyFields, boolean[] orders, int logicalFieldOffset, ExecutionConfig config) {
+ return new KvCoderComperator((KvCoder) coder);
+ }
+
+ @Override
+ public boolean isBasicType() {
+ return false;
+ }
+
+ @Override
+ public boolean isTupleType() {
+ return false;
+ }
+
+ @Override
+ public int getArity() {
+ return 2;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public Class<KV<K, V>> getTypeClass() {
+ return privateGetTypeClass();
+ }
+
+ @SuppressWarnings("unchecked")
+ private static <X> Class<X> privateGetTypeClass() {
+ return (Class<X>) Object.class;
+ }
+
+ @Override
+ public boolean isKeyType() {
+ return true;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public TypeSerializer<KV<K, V>> createSerializer(ExecutionConfig config) {
+ return new CoderTypeSerializer<>(coder);
+ }
+
+ @Override
+ public int getTotalFields() {
+ return 2;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ KvCoderTypeInformation that = (KvCoderTypeInformation) o;
+
+ return coder.equals(that.coder);
+
+ }
+
+ @Override
+ public int hashCode() {
+ return coder.hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return "CoderTypeInformation{" +
+ "coder=" + coder +
+ '}';
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public <X> TypeInformation<X> getTypeAt(int pos) {
+ if (pos == 0) {
+ return (TypeInformation<X>) new CoderTypeInformation<>(coder.getKeyCoder());
+ } else if (pos == 1) {
+ return (TypeInformation<X>) new CoderTypeInformation<>(coder.getValueCoder());
+ } else {
+ throw new RuntimeException("Invalid field position " + pos);
+ }
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public <X> TypeInformation<X> getTypeAt(String fieldExpression) {
+ switch (fieldExpression) {
+ case "key":
+ return (TypeInformation<X>) new CoderTypeInformation<>(coder.getKeyCoder());
+ case "value":
+ return (TypeInformation<X>) new CoderTypeInformation<>(coder.getValueCoder());
+ default:
+ throw new UnsupportedOperationException("Only KvCoder has fields.");
+ }
+ }
+
+ @Override
+ public String[] getFieldNames() {
+ return new String[]{"key", "value"};
+ }
+
+ @Override
+ public int getFieldIndex(String fieldName) {
+ switch (fieldName) {
+ case "key":
+ return 0;
+ case "value":
+ return 1;
+ default:
+ return -1;
+ }
+ }
+
+ @Override
+ public void getFlatFields(String fieldExpression, int offset, List<FlatFieldDescriptor> result) {
+ CoderTypeInformation keyTypeInfo = new CoderTypeInformation<>(coder.getKeyCoder());
+ result.add(new FlatFieldDescriptor(0, keyTypeInfo));
+ }
+
+ @Override
+ protected TypeComparatorBuilder<KV<K, V>> createTypeComparatorBuilder() {
+ return new KvCoderTypeComparatorBuilder();
+ }
+
+ private class KvCoderTypeComparatorBuilder implements TypeComparatorBuilder<KV<K, V>> {
+
+ @Override
+ public void initializeTypeComparatorBuilder(int size) {}
+
+ @Override
+ public void addComparatorField(int fieldId, TypeComparator<?> comparator) {}
+
+ @Override
+ public TypeComparator<KV<K, V>> createTypeComparator(ExecutionConfig config) {
+ return new KvCoderComperator<>(coder);
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java
index 7ce484a..190d898 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java
@@ -31,82 +31,82 @@ import java.io.IOException;
*/
public class VoidCoderTypeSerializer extends TypeSerializer<VoidCoderTypeSerializer.VoidValue> {
- @Override
- public boolean isImmutableType() {
- return false;
- }
-
- @Override
- public VoidCoderTypeSerializer duplicate() {
- return this;
- }
-
- @Override
- public VoidValue createInstance() {
- return VoidValue.INSTANCE;
- }
-
- @Override
- public VoidValue copy(VoidValue from) {
- return from;
- }
-
- @Override
- public VoidValue copy(VoidValue from, VoidValue reuse) {
- return from;
- }
-
- @Override
- public int getLength() {
- return 0;
- }
-
- @Override
- public void serialize(VoidValue record, DataOutputView target) throws IOException {
- target.writeByte(1);
- }
-
- @Override
- public VoidValue deserialize(DataInputView source) throws IOException {
- source.readByte();
- return VoidValue.INSTANCE;
- }
-
- @Override
- public VoidValue deserialize(VoidValue reuse, DataInputView source) throws IOException {
- return deserialize(source);
- }
-
- @Override
- public void copy(DataInputView source, DataOutputView target) throws IOException {
- source.readByte();
- target.writeByte(1);
- }
-
- @Override
- public boolean equals(Object obj) {
- if (obj instanceof VoidCoderTypeSerializer) {
- VoidCoderTypeSerializer other = (VoidCoderTypeSerializer) obj;
- return other.canEqual(this);
- } else {
- return false;
- }
- }
-
- @Override
- public boolean canEqual(Object obj) {
- return obj instanceof VoidCoderTypeSerializer;
- }
-
- @Override
- public int hashCode() {
- return 0;
- }
-
- public static class VoidValue {
- private VoidValue() {}
-
- public static VoidValue INSTANCE = new VoidValue();
- }
+ @Override
+ public boolean isImmutableType() {
+ return false;
+ }
+
+ @Override
+ public VoidCoderTypeSerializer duplicate() {
+ return this;
+ }
+
+ @Override
+ public VoidValue createInstance() {
+ return VoidValue.INSTANCE;
+ }
+
+ @Override
+ public VoidValue copy(VoidValue from) {
+ return from;
+ }
+
+ @Override
+ public VoidValue copy(VoidValue from, VoidValue reuse) {
+ return from;
+ }
+
+ @Override
+ public int getLength() {
+ return 0;
+ }
+
+ @Override
+ public void serialize(VoidValue record, DataOutputView target) throws IOException {
+ target.writeByte(1);
+ }
+
+ @Override
+ public VoidValue deserialize(DataInputView source) throws IOException {
+ source.readByte();
+ return VoidValue.INSTANCE;
+ }
+
+ @Override
+ public VoidValue deserialize(VoidValue reuse, DataInputView source) throws IOException {
+ return deserialize(source);
+ }
+
+ @Override
+ public void copy(DataInputView source, DataOutputView target) throws IOException {
+ source.readByte();
+ target.writeByte(1);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof VoidCoderTypeSerializer) {
+ VoidCoderTypeSerializer other = (VoidCoderTypeSerializer) obj;
+ return other.canEqual(this);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof VoidCoderTypeSerializer;
+ }
+
+ @Override
+ public int hashCode() {
+ return 0;
+ }
+
+ public static class VoidValue {
+ private VoidValue() {}
+
+ public static VoidValue INSTANCE = new VoidValue();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java
index 924b297..8f6d67c 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/CombineFnAggregatorWrapper.java
@@ -31,62 +31,62 @@ import java.io.Serializable;
* operation.
*/
public class CombineFnAggregatorWrapper<AI, AA, AR> implements Aggregator<AI, AR>, Accumulator<AI, Serializable> {
-
- private AA aa;
- private Combine.CombineFn<? super AI, AA, AR> combiner;
+
+ private AA aa;
+ private Combine.CombineFn<? super AI, AA, AR> combiner;
- public CombineFnAggregatorWrapper() {
- }
+ public CombineFnAggregatorWrapper() {
+ }
- public CombineFnAggregatorWrapper(Combine.CombineFn<? super AI, AA, AR> combiner) {
- this.combiner = combiner;
- this.aa = combiner.createAccumulator();
- }
+ public CombineFnAggregatorWrapper(Combine.CombineFn<? super AI, AA, AR> combiner) {
+ this.combiner = combiner;
+ this.aa = combiner.createAccumulator();
+ }
- @Override
- public void add(AI value) {
- combiner.addInput(aa, value);
- }
+ @Override
+ public void add(AI value) {
+ combiner.addInput(aa, value);
+ }
- @Override
- public Serializable getLocalValue() {
- return (Serializable) combiner.extractOutput(aa);
- }
+ @Override
+ public Serializable getLocalValue() {
+ return (Serializable) combiner.extractOutput(aa);
+ }
- @Override
- public void resetLocal() {
- aa = combiner.createAccumulator();
- }
+ @Override
+ public void resetLocal() {
+ aa = combiner.createAccumulator();
+ }
- @Override
- @SuppressWarnings("unchecked")
- public void merge(Accumulator<AI, Serializable> other) {
- aa = combiner.mergeAccumulators(Lists.newArrayList(aa, ((CombineFnAggregatorWrapper<AI, AA, AR>)other).aa));
- }
+ @Override
+ @SuppressWarnings("unchecked")
+ public void merge(Accumulator<AI, Serializable> other) {
+ aa = combiner.mergeAccumulators(Lists.newArrayList(aa, ((CombineFnAggregatorWrapper<AI, AA, AR>)other).aa));
+ }
- @Override
- public Accumulator<AI, Serializable> clone() {
- // copy it by merging
- AA aaCopy = combiner.mergeAccumulators(Lists.newArrayList(aa));
- CombineFnAggregatorWrapper<AI, AA, AR> result = new
- CombineFnAggregatorWrapper<>(combiner);
- result.aa = aaCopy;
- return result;
- }
+ @Override
+ public Accumulator<AI, Serializable> clone() {
+ // copy it by merging
+ AA aaCopy = combiner.mergeAccumulators(Lists.newArrayList(aa));
+ CombineFnAggregatorWrapper<AI, AA, AR> result = new
+ CombineFnAggregatorWrapper<>(combiner);
+ result.aa = aaCopy;
+ return result;
+ }
- @Override
- public void addValue(AI value) {
- add(value);
- }
+ @Override
+ public void addValue(AI value) {
+ add(value);
+ }
- @Override
- public String getName() {
- return "CombineFn: " + combiner.toString();
- }
+ @Override
+ public String getName() {
+ return "CombineFn: " + combiner.toString();
+ }
- @Override
- public Combine.CombineFn getCombineFn() {
- return combiner;
- }
+ @Override
+ public Combine.CombineFn getCombineFn() {
+ return combiner;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java
index 90582b0..3c96939 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataInputViewWrapper.java
@@ -31,29 +31,29 @@ import java.io.InputStream;
*/
public class DataInputViewWrapper extends InputStream {
- private DataInputView inputView;
-
- public DataInputViewWrapper(DataInputView inputView) {
- this.inputView = inputView;
- }
-
- public void setInputView(DataInputView inputView) {
- this.inputView = inputView;
- }
-
- @Override
- public int read() throws IOException {
- try {
- return inputView.readUnsignedByte();
- } catch (EOFException e) {
- // translate between DataInput and InputStream,
- // DataInput signals EOF by exception, InputStream does it by returning -1
- return -1;
- }
- }
-
- @Override
- public int read(byte[] b, int off, int len) throws IOException {
- return inputView.read(b, off, len);
- }
+ private DataInputView inputView;
+
+ public DataInputViewWrapper(DataInputView inputView) {
+ this.inputView = inputView;
+ }
+
+ public void setInputView(DataInputView inputView) {
+ this.inputView = inputView;
+ }
+
+ @Override
+ public int read() throws IOException {
+ try {
+ return inputView.readUnsignedByte();
+ } catch (EOFException e) {
+ // translate between DataInput and InputStream,
+ // DataInput signals EOF by exception, InputStream does it by returning -1
+ return -1;
+ }
+ }
+
+ @Override
+ public int read(byte[] b, int off, int len) throws IOException {
+ return inputView.read(b, off, len);
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java
index 46df8e5..a222cdd 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/DataOutputViewWrapper.java
@@ -29,24 +29,24 @@ import java.io.OutputStream;
* {@link java.io.OutputStream}.
*/
public class DataOutputViewWrapper extends OutputStream {
-
- private DataOutputView outputView;
+
+ private DataOutputView outputView;
- public DataOutputViewWrapper(DataOutputView outputView) {
- this.outputView = outputView;
- }
+ public DataOutputViewWrapper(DataOutputView outputView) {
+ this.outputView = outputView;
+ }
- public void setOutputView(DataOutputView outputView) {
- this.outputView = outputView;
- }
+ public void setOutputView(DataOutputView outputView) {
+ this.outputView = outputView;
+ }
- @Override
- public void write(int b) throws IOException {
- outputView.write(b);
- }
+ @Override
+ public void write(int b) throws IOException {
+ outputView.write(b);
+ }
- @Override
- public void write(byte[] b, int off, int len) throws IOException {
- outputView.write(b, off, len);
- }
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ outputView.write(b, off, len);
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java
index 1c0dae4..c193a4d 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SerializableFnAggregatorWrapper.java
@@ -33,59 +33,59 @@ import java.io.Serializable;
*/
public class SerializableFnAggregatorWrapper<AI, AO> implements Aggregator<AI, AO>, Accumulator<AI, Serializable> {
- private AO aa;
- private Combine.CombineFn<AI, ?, AO> combiner;
+ private AO aa;
+ private Combine.CombineFn<AI, ?, AO> combiner;
- public SerializableFnAggregatorWrapper(Combine.CombineFn<AI, ?, AO> combiner) {
- this.combiner = combiner;
- resetLocal();
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public void add(AI value) {
- this.aa = combiner.apply(ImmutableList.of((AI) aa, value));
- }
+ public SerializableFnAggregatorWrapper(Combine.CombineFn<AI, ?, AO> combiner) {
+ this.combiner = combiner;
+ resetLocal();
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void add(AI value) {
+ this.aa = combiner.apply(ImmutableList.of((AI) aa, value));
+ }
- @Override
- public Serializable getLocalValue() {
- return (Serializable) aa;
- }
+ @Override
+ public Serializable getLocalValue() {
+ return (Serializable) aa;
+ }
- @Override
- public void resetLocal() {
- this.aa = combiner.apply(ImmutableList.<AI>of());
- }
+ @Override
+ public void resetLocal() {
+ this.aa = combiner.apply(ImmutableList.<AI>of());
+ }
- @Override
- @SuppressWarnings("unchecked")
- public void merge(Accumulator<AI, Serializable> other) {
- this.aa = combiner.apply(ImmutableList.of((AI) aa, (AI) other.getLocalValue()));
- }
+ @Override
+ @SuppressWarnings("unchecked")
+ public void merge(Accumulator<AI, Serializable> other) {
+ this.aa = combiner.apply(ImmutableList.of((AI) aa, (AI) other.getLocalValue()));
+ }
- @Override
- public void addValue(AI value) {
- add(value);
- }
+ @Override
+ public void addValue(AI value) {
+ add(value);
+ }
- @Override
- public String getName() {
- return "Aggregator :" + combiner.toString();
- }
+ @Override
+ public String getName() {
+ return "Aggregator :" + combiner.toString();
+ }
- @Override
- public Combine.CombineFn<AI, ?, AO> getCombineFn() {
- return combiner;
- }
+ @Override
+ public Combine.CombineFn<AI, ?, AO> getCombineFn() {
+ return combiner;
+ }
- @Override
- public Accumulator<AI, Serializable> clone() {
- // copy it by merging
- AO resultCopy = combiner.apply(Lists.newArrayList((AI) aa));
- SerializableFnAggregatorWrapper<AI, AO> result = new
- SerializableFnAggregatorWrapper<>(combiner);
+ @Override
+ public Accumulator<AI, Serializable> clone() {
+ // copy it by merging
+ AO resultCopy = combiner.apply(Lists.newArrayList((AI) aa));
+ SerializableFnAggregatorWrapper<AI, AO> result = new
+ SerializableFnAggregatorWrapper<>(combiner);
- result.aa = resultCopy;
- return result;
- }
+ result.aa = resultCopy;
+ return result;
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java
index 8be9abf..3f28c16 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SinkOutputFormat.java
@@ -38,84 +38,84 @@ import java.lang.reflect.Field;
*/
public class SinkOutputFormat<T> implements OutputFormat<T> {
- private final Sink<T> sink;
-
- private transient PipelineOptions pipelineOptions;
-
- private Sink.WriteOperation<T, ?> writeOperation;
- private Sink.Writer<T, ?> writer;
-
- private AbstractID uid = new AbstractID();
-
- public SinkOutputFormat(Write.Bound<T> transform, PipelineOptions pipelineOptions) {
- this.sink = extractSink(transform);
- this.pipelineOptions = Preconditions.checkNotNull(pipelineOptions);
- }
-
- private Sink<T> extractSink(Write.Bound<T> transform) {
- // TODO possibly add a getter in the upstream
- try {
- Field sinkField = transform.getClass().getDeclaredField("sink");
- sinkField.setAccessible(true);
- @SuppressWarnings("unchecked")
- Sink<T> extractedSink = (Sink<T>) sinkField.get(transform);
- return extractedSink;
- } catch (NoSuchFieldException | IllegalAccessException e) {
- throw new RuntimeException("Could not acquire custom sink field.", e);
- }
- }
-
- @Override
- public void configure(Configuration configuration) {
- writeOperation = sink.createWriteOperation(pipelineOptions);
- try {
- writeOperation.initialize(pipelineOptions);
- } catch (Exception e) {
- throw new RuntimeException("Failed to initialize the write operation.", e);
- }
- }
-
- @Override
- public void open(int taskNumber, int numTasks) throws IOException {
- try {
- writer = writeOperation.createWriter(pipelineOptions);
- } catch (Exception e) {
- throw new IOException("Couldn't create writer.", e);
- }
- try {
- writer.open(uid + "-" + String.valueOf(taskNumber));
- } catch (Exception e) {
- throw new IOException("Couldn't open writer.", e);
- }
- }
-
- @Override
- public void writeRecord(T record) throws IOException {
- try {
- writer.write(record);
- } catch (Exception e) {
- throw new IOException("Couldn't write record.", e);
- }
- }
-
- @Override
- public void close() throws IOException {
- try {
- writer.close();
- } catch (Exception e) {
- throw new IOException("Couldn't close writer.", e);
- }
- }
-
- private void writeObject(ObjectOutputStream out) throws IOException, ClassNotFoundException {
- out.defaultWriteObject();
- ObjectMapper mapper = new ObjectMapper();
- mapper.writeValue(out, pipelineOptions);
- }
-
- private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- ObjectMapper mapper = new ObjectMapper();
- pipelineOptions = mapper.readValue(in, PipelineOptions.class);
- }
+ private final Sink<T> sink;
+
+ private transient PipelineOptions pipelineOptions;
+
+ private Sink.WriteOperation<T, ?> writeOperation;
+ private Sink.Writer<T, ?> writer;
+
+ private AbstractID uid = new AbstractID();
+
+ public SinkOutputFormat(Write.Bound<T> transform, PipelineOptions pipelineOptions) {
+ this.sink = extractSink(transform);
+ this.pipelineOptions = Preconditions.checkNotNull(pipelineOptions);
+ }
+
+ private Sink<T> extractSink(Write.Bound<T> transform) {
+ // TODO possibly add a getter in the upstream
+ try {
+ Field sinkField = transform.getClass().getDeclaredField("sink");
+ sinkField.setAccessible(true);
+ @SuppressWarnings("unchecked")
+ Sink<T> extractedSink = (Sink<T>) sinkField.get(transform);
+ return extractedSink;
+ } catch (NoSuchFieldException | IllegalAccessException e) {
+ throw new RuntimeException("Could not acquire custom sink field.", e);
+ }
+ }
+
+ @Override
+ public void configure(Configuration configuration) {
+ writeOperation = sink.createWriteOperation(pipelineOptions);
+ try {
+ writeOperation.initialize(pipelineOptions);
+ } catch (Exception e) {
+ throw new RuntimeException("Failed to initialize the write operation.", e);
+ }
+ }
+
+ @Override
+ public void open(int taskNumber, int numTasks) throws IOException {
+ try {
+ writer = writeOperation.createWriter(pipelineOptions);
+ } catch (Exception e) {
+ throw new IOException("Couldn't create writer.", e);
+ }
+ try {
+ writer.open(uid + "-" + String.valueOf(taskNumber));
+ } catch (Exception e) {
+ throw new IOException("Couldn't open writer.", e);
+ }
+ }
+
+ @Override
+ public void writeRecord(T record) throws IOException {
+ try {
+ writer.write(record);
+ } catch (Exception e) {
+ throw new IOException("Couldn't write record.", e);
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ try {
+ writer.close();
+ } catch (Exception e) {
+ throw new IOException("Couldn't close writer.", e);
+ }
+ }
+
+ private void writeObject(ObjectOutputStream out) throws IOException, ClassNotFoundException {
+ out.defaultWriteObject();
+ ObjectMapper mapper = new ObjectMapper();
+ mapper.writeValue(out, pipelineOptions);
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ ObjectMapper mapper = new ObjectMapper();
+ pipelineOptions = mapper.readValue(in, PipelineOptions.class);
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
index 64dc072..5981618 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
@@ -41,124 +41,124 @@ import java.util.List;
* Dataflow {@link com.google.cloud.dataflow.sdk.io.Source}.
*/
public class SourceInputFormat<T> implements InputFormat<T, SourceInputSplit<T>> {
- private static final Logger LOG = LoggerFactory.getLogger(SourceInputFormat.class);
-
- private final BoundedSource<T> initialSource;
- private transient PipelineOptions options;
-
- private BoundedSource.BoundedReader<T> reader = null;
- private boolean reachedEnd = true;
-
- public SourceInputFormat(BoundedSource<T> initialSource, PipelineOptions options) {
- this.initialSource = initialSource;
- this.options = options;
- }
-
- private void writeObject(ObjectOutputStream out)
- throws IOException, ClassNotFoundException {
- out.defaultWriteObject();
- ObjectMapper mapper = new ObjectMapper();
- mapper.writeValue(out, options);
- }
-
- private void readObject(ObjectInputStream in)
- throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- ObjectMapper mapper = new ObjectMapper();
- options = mapper.readValue(in, PipelineOptions.class);
- }
-
- @Override
- public void configure(Configuration configuration) {}
-
- @Override
- public void open(SourceInputSplit<T> sourceInputSplit) throws IOException {
- reader = ((BoundedSource<T>) sourceInputSplit.getSource()).createReader(options);
- reachedEnd = false;
- }
-
- @Override
- public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOException {
- try {
- final long estimatedSize = initialSource.getEstimatedSizeBytes(options);
-
- return new BaseStatistics() {
- @Override
- public long getTotalInputSize() {
- return estimatedSize;
-
- }
-
- @Override
- public long getNumberOfRecords() {
- return BaseStatistics.NUM_RECORDS_UNKNOWN;
- }
-
- @Override
- public float getAverageRecordWidth() {
- return BaseStatistics.AVG_RECORD_BYTES_UNKNOWN;
- }
- };
- } catch (Exception e) {
- LOG.warn("Could not read Source statistics: {}", e);
- }
-
- return null;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public SourceInputSplit<T>[] createInputSplits(int numSplits) throws IOException {
- long desiredSizeBytes;
- try {
- desiredSizeBytes = initialSource.getEstimatedSizeBytes(options) / numSplits;
- List<? extends Source<T>> shards = initialSource.splitIntoBundles(desiredSizeBytes,
- options);
- List<SourceInputSplit<T>> splits = new ArrayList<>();
- int splitCount = 0;
- for (Source<T> shard: shards) {
- splits.add(new SourceInputSplit<>(shard, splitCount++));
- }
- return splits.toArray(new SourceInputSplit[splits.size()]);
- } catch (Exception e) {
- throw new IOException("Could not create input splits from Source.", e);
- }
- }
-
- @Override
- public InputSplitAssigner getInputSplitAssigner(final SourceInputSplit[] sourceInputSplits) {
- return new InputSplitAssigner() {
- private int index = 0;
- private final SourceInputSplit[] splits = sourceInputSplits;
- @Override
- public InputSplit getNextInputSplit(String host, int taskId) {
- if (index < splits.length) {
- return splits[index++];
- } else {
- return null;
- }
- }
- };
- }
-
-
- @Override
- public boolean reachedEnd() throws IOException {
- return reachedEnd;
- }
-
- @Override
- public T nextRecord(T t) throws IOException {
-
- reachedEnd = !reader.advance();
- if (!reachedEnd) {
- return reader.getCurrent();
- }
- return null;
- }
-
- @Override
- public void close() throws IOException {
- reader.close();
- }
+ private static final Logger LOG = LoggerFactory.getLogger(SourceInputFormat.class);
+
+ private final BoundedSource<T> initialSource;
+ private transient PipelineOptions options;
+
+ private BoundedSource.BoundedReader<T> reader = null;
+ private boolean reachedEnd = true;
+
+ public SourceInputFormat(BoundedSource<T> initialSource, PipelineOptions options) {
+ this.initialSource = initialSource;
+ this.options = options;
+ }
+
+ private void writeObject(ObjectOutputStream out)
+ throws IOException, ClassNotFoundException {
+ out.defaultWriteObject();
+ ObjectMapper mapper = new ObjectMapper();
+ mapper.writeValue(out, options);
+ }
+
+ private void readObject(ObjectInputStream in)
+ throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ ObjectMapper mapper = new ObjectMapper();
+ options = mapper.readValue(in, PipelineOptions.class);
+ }
+
+ @Override
+ public void configure(Configuration configuration) {}
+
+ @Override
+ public void open(SourceInputSplit<T> sourceInputSplit) throws IOException {
+ reader = ((BoundedSource<T>) sourceInputSplit.getSource()).createReader(options);
+ reachedEnd = false;
+ }
+
+ @Override
+ public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOException {
+ try {
+ final long estimatedSize = initialSource.getEstimatedSizeBytes(options);
+
+ return new BaseStatistics() {
+ @Override
+ public long getTotalInputSize() {
+ return estimatedSize;
+
+ }
+
+ @Override
+ public long getNumberOfRecords() {
+ return BaseStatistics.NUM_RECORDS_UNKNOWN;
+ }
+
+ @Override
+ public float getAverageRecordWidth() {
+ return BaseStatistics.AVG_RECORD_BYTES_UNKNOWN;
+ }
+ };
+ } catch (Exception e) {
+ LOG.warn("Could not read Source statistics: {}", e);
+ }
+
+ return null;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public SourceInputSplit<T>[] createInputSplits(int numSplits) throws IOException {
+ long desiredSizeBytes;
+ try {
+ desiredSizeBytes = initialSource.getEstimatedSizeBytes(options) / numSplits;
+ List<? extends Source<T>> shards = initialSource.splitIntoBundles(desiredSizeBytes,
+ options);
+ List<SourceInputSplit<T>> splits = new ArrayList<>();
+ int splitCount = 0;
+ for (Source<T> shard: shards) {
+ splits.add(new SourceInputSplit<>(shard, splitCount++));
+ }
+ return splits.toArray(new SourceInputSplit[splits.size()]);
+ } catch (Exception e) {
+ throw new IOException("Could not create input splits from Source.", e);
+ }
+ }
+
+ @Override
+ public InputSplitAssigner getInputSplitAssigner(final SourceInputSplit[] sourceInputSplits) {
+ return new InputSplitAssigner() {
+ private int index = 0;
+ private final SourceInputSplit[] splits = sourceInputSplits;
+ @Override
+ public InputSplit getNextInputSplit(String host, int taskId) {
+ if (index < splits.length) {
+ return splits[index++];
+ } else {
+ return null;
+ }
+ }
+ };
+ }
+
+
+ @Override
+ public boolean reachedEnd() throws IOException {
+ return reachedEnd;
+ }
+
+ @Override
+ public T nextRecord(T t) throws IOException {
+
+ reachedEnd = !reader.advance();
+ if (!reachedEnd) {
+ return reader.getCurrent();
+ }
+ return null;
+ }
+
+ @Override
+ public void close() throws IOException {
+ reader.close();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java
index 2b93ab7..86fdada 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputSplit.java
@@ -29,24 +29,24 @@ import org.apache.flink.core.io.InputSplit;
*/
public class SourceInputSplit<T> implements InputSplit {
- private Source<T> source;
- private int splitNumber;
+ private Source<T> source;
+ private int splitNumber;
- public SourceInputSplit() {
- }
+ public SourceInputSplit() {
+ }
- public SourceInputSplit(Source<T> source, int splitNumber) {
- this.source = source;
- this.splitNumber = splitNumber;
- }
+ public SourceInputSplit(Source<T> source, int splitNumber) {
+ this.source = source;
+ this.splitNumber = splitNumber;
+ }
- @Override
- public int getSplitNumber() {
- return splitNumber;
- }
+ @Override
+ public int getSplitNumber() {
+ return splitNumber;
+ }
- public Source<T> getSource() {
- return source;
- }
+ public Source<T> getSource() {
+ return source;
+ }
}
[34/50] [abbrv] incubator-beam git commit: [tests] suppress
unnecessary log output
Posted by da...@apache.org.
[tests] suppress unnecessary log output
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/dc7786f3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/dc7786f3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/dc7786f3
Branch: refs/heads/master
Commit: dc7786f30c975df96e2992f803fe8a1964ecbb38
Parents: c6630b9
Author: Maximilian Michels <mx...@apache.org>
Authored: Thu Mar 3 14:57:06 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
runners/flink/pom.xml | 6 +++++
.../src/test/resources/log4j-test.properties | 27 ++++++++++++++++++++
2 files changed, 33 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/dc7786f3/runners/flink/pom.xml
----------------------------------------------------------------------
diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml
index 0234ddd..6b12f2d 100644
--- a/runners/flink/pom.xml
+++ b/runners/flink/pom.xml
@@ -174,6 +174,9 @@
</goals>
</execution>
</executions>
+ <configuration>
+ <argLine>-Dlog4j.configuration=log4j-test.properties -XX:-UseGCOverheadLimit</argLine>
+ </configuration>
</plugin>
<!-- Unit Tests -->
@@ -181,6 +184,9 @@
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<version>2.17</version><!--$NO-MVN-MAN-VER$-->
+ <configuration>
+ <argLine>-Dlog4j.configuration=log4j-test.properties -XX:-UseGCOverheadLimit</argLine>
+ </configuration>
</plugin>
<!-- Eclipse Integration -->
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/dc7786f3/runners/flink/src/test/resources/log4j-test.properties
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/resources/log4j-test.properties b/runners/flink/src/test/resources/log4j-test.properties
new file mode 100644
index 0000000..4c74d85
--- /dev/null
+++ b/runners/flink/src/test/resources/log4j-test.properties
@@ -0,0 +1,27 @@
+################################################################################
+# 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.
+################################################################################
+
+# Set root logger level to OFF to not flood build logs
+# set manually to INFO for debugging purposes
+log4j.rootLogger=OFF, testlogger
+
+# A1 is set to be a ConsoleAppender.
+log4j.appender.testlogger=org.apache.log4j.ConsoleAppender
+log4j.appender.testlogger.target = System.err
+log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout
+log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
[32/50] [abbrv] incubator-beam git commit: [travis] install snapshot
version of SDK before running CI
Posted by da...@apache.org.
[travis] install snapshot version of SDK before running CI
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/c6630b90
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/c6630b90
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/c6630b90
Branch: refs/heads/master
Commit: c6630b909cbb5c60d3cb5f966aa1327bb28c7f6c
Parents: 52614ea
Author: Maximilian Michels <mx...@apache.org>
Authored: Thu Mar 3 11:30:53 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
runners/flink/.travis.yml | 4 +++-
1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/c6630b90/runners/flink/.travis.yml
----------------------------------------------------------------------
diff --git a/runners/flink/.travis.yml b/runners/flink/.travis.yml
index d7fbf5b..4780b56 100644
--- a/runners/flink/.travis.yml
+++ b/runners/flink/.travis.yml
@@ -9,4 +9,6 @@ jdk:
# not supported yet :(
#- openjdk8
-script: mvn verify
+install: (git clone --depth 1 https://github.com/GoogleCloudPlatform/DataflowJavaSDK.git sdk && cd sdk && mvn clean install -DskipTests)
+
+script: mvn verify
\ No newline at end of file
[09/50] [abbrv] incubator-beam git commit: [readme] add hint that
streaming support is available
Posted by da...@apache.org.
[readme] add hint that streaming support is available
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/8434c3c0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/8434c3c0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/8434c3c0
Branch: refs/heads/master
Commit: 8434c3c0d7fcc01a96199bd9323b2e91914e68c1
Parents: edff078
Author: Max <ma...@posteo.de>
Authored: Wed Jan 20 17:25:13 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
runners/flink/README.md | 8 ++++++--
.../wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java | 2 +-
2 files changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8434c3c0/runners/flink/README.md
----------------------------------------------------------------------
diff --git a/runners/flink/README.md b/runners/flink/README.md
index 8c06c1d..54d248c 100644
--- a/runners/flink/README.md
+++ b/runners/flink/README.md
@@ -142,5 +142,9 @@ the [Mailinglists](http://flink.apache.org/community.html#mailing-lists).
# Streaming
-Streaming support is currently under development. See the `streaming_new` branch for the current
-work in progress version.
+Streaming support has been added. It is currently in alpha stage. Please give it a try. To use
+streaming, just enable streaming mode in the `PipelineOptions`:
+
+ options.setStreaming(true);
+
+That's all.
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8434c3c0/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
index c52fabe..0f0a9d0 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
@@ -48,7 +48,7 @@ import java.util.*;
/**
* This class is the key class implementing all the windowing/triggering logic of Google Dataflow.
* To provide full compatibility and support all the windowing/triggering combinations offered by
- * Datadlow, we opted for a strategy that uses the SDK's code for doing these operations
+ * Dataflow, we opted for a strategy that uses the SDK's code for doing these operations
* ({@link com.google.cloud.dataflow.sdk.util.StreamingGroupAlsoByWindowsDoFn}.
* <p>
* In a nutshell, when the execution arrives to this operator, we expect to have a stream <b>already
[25/50] [abbrv] incubator-beam git commit: [flink] adjust directories
according to package name
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java
new file mode 100644
index 0000000..f33e4f5
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java
@@ -0,0 +1,594 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation;
+
+import org.apache.beam.runners.flink.io.ConsoleIO;
+import org.apache.beam.runners.flink.translation.functions.FlinkCoGroupKeyedListAggregator;
+import org.apache.beam.runners.flink.translation.functions.FlinkCreateFunction;
+import org.apache.beam.runners.flink.translation.functions.FlinkDoFnFunction;
+import org.apache.beam.runners.flink.translation.functions.FlinkKeyedListAggregationFunction;
+import org.apache.beam.runners.flink.translation.functions.FlinkMultiOutputDoFnFunction;
+import org.apache.beam.runners.flink.translation.functions.FlinkMultiOutputPruningFunction;
+import org.apache.beam.runners.flink.translation.functions.FlinkPartialReduceFunction;
+import org.apache.beam.runners.flink.translation.functions.FlinkReduceFunction;
+import org.apache.beam.runners.flink.translation.functions.UnionCoder;
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.runners.flink.translation.types.KvCoderTypeInformation;
+import org.apache.beam.runners.flink.translation.wrappers.SinkOutputFormat;
+import org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat;
+import com.google.api.client.util.Maps;
+import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.io.AvroIO;
+import com.google.cloud.dataflow.sdk.io.BoundedSource;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.Flatten;
+import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.transforms.Write;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResultSchema;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
+import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.collect.Lists;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.operators.Keys;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.io.AvroInputFormat;
+import org.apache.flink.api.java.io.AvroOutputFormat;
+import org.apache.flink.api.java.io.TextInputFormat;
+import org.apache.flink.api.java.operators.CoGroupOperator;
+import org.apache.flink.api.java.operators.DataSink;
+import org.apache.flink.api.java.operators.DataSource;
+import org.apache.flink.api.java.operators.FlatMapOperator;
+import org.apache.flink.api.java.operators.GroupCombineOperator;
+import org.apache.flink.api.java.operators.GroupReduceOperator;
+import org.apache.flink.api.java.operators.Grouping;
+import org.apache.flink.api.java.operators.MapPartitionOperator;
+import org.apache.flink.api.java.operators.UnsortedGrouping;
+import org.apache.flink.core.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Translators for transforming
+ * Dataflow {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s to
+ * Flink {@link org.apache.flink.api.java.DataSet}s
+ */
+public class FlinkBatchTransformTranslators {
+
+ // --------------------------------------------------------------------------------------------
+ // Transform Translator Registry
+ // --------------------------------------------------------------------------------------------
+
+ @SuppressWarnings("rawtypes")
+ private static final Map<Class<? extends PTransform>, FlinkBatchPipelineTranslator.BatchTransformTranslator> TRANSLATORS = new HashMap<>();
+
+ // register the known translators
+ static {
+ TRANSLATORS.put(View.CreatePCollectionView.class, new CreatePCollectionViewTranslatorBatch());
+
+ TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslatorBatch());
+ // we don't need this because we translate the Combine.PerKey directly
+ //TRANSLATORS.put(Combine.GroupedValues.class, new CombineGroupedValuesTranslator());
+
+ TRANSLATORS.put(Create.Values.class, new CreateTranslatorBatch());
+
+ TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslatorBatch());
+
+ TRANSLATORS.put(GroupByKey.GroupByKeyOnly.class, new GroupByKeyOnlyTranslatorBatch());
+ // TODO we're currently ignoring windows here but that has to change in the future
+ TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslatorBatch());
+
+ TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslatorBatch());
+ TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslatorBatch());
+
+ TRANSLATORS.put(CoGroupByKey.class, new CoGroupByKeyTranslatorBatch());
+
+ TRANSLATORS.put(AvroIO.Read.Bound.class, new AvroIOReadTranslatorBatch());
+ TRANSLATORS.put(AvroIO.Write.Bound.class, new AvroIOWriteTranslatorBatch());
+
+ TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslatorBatch());
+ TRANSLATORS.put(Write.Bound.class, new WriteSinkTranslatorBatch());
+
+ TRANSLATORS.put(TextIO.Read.Bound.class, new TextIOReadTranslatorBatch());
+ TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteTranslatorBatch());
+
+ // Flink-specific
+ TRANSLATORS.put(ConsoleIO.Write.Bound.class, new ConsoleIOWriteTranslatorBatch());
+
+ }
+
+
+ public static FlinkBatchPipelineTranslator.BatchTransformTranslator<?> getTranslator(PTransform<?, ?> transform) {
+ return TRANSLATORS.get(transform.getClass());
+ }
+
+ private static class ReadSourceTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Read.Bounded<T>> {
+
+ @Override
+ public void translateNode(Read.Bounded<T> transform, FlinkBatchTranslationContext context) {
+ String name = transform.getName();
+ BoundedSource<T> source = transform.getSource();
+ PCollection<T> output = context.getOutput(transform);
+ Coder<T> coder = output.getCoder();
+
+ TypeInformation<T> typeInformation = context.getTypeInfo(output);
+
+ DataSource<T> dataSource = new DataSource<>(context.getExecutionEnvironment(),
+ new SourceInputFormat<>(source, context.getPipelineOptions()), typeInformation, name);
+
+ context.setOutputDataSet(output, dataSource);
+ }
+ }
+
+ private static class AvroIOReadTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<AvroIO.Read.Bound<T>> {
+ private static final Logger LOG = LoggerFactory.getLogger(AvroIOReadTranslatorBatch.class);
+
+ @Override
+ public void translateNode(AvroIO.Read.Bound<T> transform, FlinkBatchTranslationContext context) {
+ String path = transform.getFilepattern();
+ String name = transform.getName();
+// Schema schema = transform.getSchema();
+ PValue output = context.getOutput(transform);
+
+ TypeInformation<T> typeInformation = context.getTypeInfo(output);
+
+ // This is super hacky, but unfortunately we cannot get the type otherwise
+ Class<T> extractedAvroType;
+ try {
+ Field typeField = transform.getClass().getDeclaredField("type");
+ typeField.setAccessible(true);
+ @SuppressWarnings("unchecked")
+ Class<T> avroType = (Class<T>) typeField.get(transform);
+ extractedAvroType = avroType;
+ } catch (NoSuchFieldException | IllegalAccessException e) {
+ // we know that the field is there and it is accessible
+ throw new RuntimeException("Could not access type from AvroIO.Bound", e);
+ }
+
+ DataSource<T> source = new DataSource<>(context.getExecutionEnvironment(),
+ new AvroInputFormat<>(new Path(path), extractedAvroType),
+ typeInformation, name);
+
+ context.setOutputDataSet(output, source);
+ }
+ }
+
+ private static class AvroIOWriteTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<AvroIO.Write.Bound<T>> {
+ private static final Logger LOG = LoggerFactory.getLogger(AvroIOWriteTranslatorBatch.class);
+
+ @Override
+ public void translateNode(AvroIO.Write.Bound<T> transform, FlinkBatchTranslationContext context) {
+ DataSet<T> inputDataSet = context.getInputDataSet(context.getInput(transform));
+ String filenamePrefix = transform.getFilenamePrefix();
+ String filenameSuffix = transform.getFilenameSuffix();
+ int numShards = transform.getNumShards();
+ String shardNameTemplate = transform.getShardNameTemplate();
+
+ // TODO: Implement these. We need Flink support for this.
+ LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.",
+ filenameSuffix);
+ LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
+
+ // This is super hacky, but unfortunately we cannot get the type otherwise
+ Class<T> extractedAvroType;
+ try {
+ Field typeField = transform.getClass().getDeclaredField("type");
+ typeField.setAccessible(true);
+ @SuppressWarnings("unchecked")
+ Class<T> avroType = (Class<T>) typeField.get(transform);
+ extractedAvroType = avroType;
+ } catch (NoSuchFieldException | IllegalAccessException e) {
+ // we know that the field is there and it is accessible
+ throw new RuntimeException("Could not access type from AvroIO.Bound", e);
+ }
+
+ DataSink<T> dataSink = inputDataSet.output(new AvroOutputFormat<>(new Path
+ (filenamePrefix), extractedAvroType));
+
+ if (numShards > 0) {
+ dataSink.setParallelism(numShards);
+ }
+ }
+ }
+
+ private static class TextIOReadTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator<TextIO.Read.Bound<String>> {
+ private static final Logger LOG = LoggerFactory.getLogger(TextIOReadTranslatorBatch.class);
+
+ @Override
+ public void translateNode(TextIO.Read.Bound<String> transform, FlinkBatchTranslationContext context) {
+ String path = transform.getFilepattern();
+ String name = transform.getName();
+
+ TextIO.CompressionType compressionType = transform.getCompressionType();
+ boolean needsValidation = transform.needsValidation();
+
+ // TODO: Implement these. We need Flink support for this.
+ LOG.warn("Translation of TextIO.CompressionType not yet supported. Is: {}.", compressionType);
+ LOG.warn("Translation of TextIO.Read.needsValidation not yet supported. Is: {}.", needsValidation);
+
+ PValue output = context.getOutput(transform);
+
+ TypeInformation<String> typeInformation = context.getTypeInfo(output);
+ DataSource<String> source = new DataSource<>(context.getExecutionEnvironment(), new TextInputFormat(new Path(path)), typeInformation, name);
+
+ context.setOutputDataSet(output, source);
+ }
+ }
+
+ private static class TextIOWriteTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<TextIO.Write.Bound<T>> {
+ private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteTranslatorBatch.class);
+
+ @Override
+ public void translateNode(TextIO.Write.Bound<T> transform, FlinkBatchTranslationContext context) {
+ PValue input = context.getInput(transform);
+ DataSet<T> inputDataSet = context.getInputDataSet(input);
+
+ String filenamePrefix = transform.getFilenamePrefix();
+ String filenameSuffix = transform.getFilenameSuffix();
+ boolean needsValidation = transform.needsValidation();
+ int numShards = transform.getNumShards();
+ String shardNameTemplate = transform.getShardNameTemplate();
+
+ // TODO: Implement these. We need Flink support for this.
+ LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation);
+ LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix);
+ LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
+
+ //inputDataSet.print();
+ DataSink<T> dataSink = inputDataSet.writeAsText(filenamePrefix);
+
+ if (numShards > 0) {
+ dataSink.setParallelism(numShards);
+ }
+ }
+ }
+
+ private static class ConsoleIOWriteTranslatorBatch implements FlinkBatchPipelineTranslator.BatchTransformTranslator<ConsoleIO.Write.Bound> {
+ @Override
+ public void translateNode(ConsoleIO.Write.Bound transform, FlinkBatchTranslationContext context) {
+ PValue input = context.getInput(transform);
+ DataSet<?> inputDataSet = context.getInputDataSet(input);
+ inputDataSet.printOnTaskManager(transform.getName());
+ }
+ }
+
+ private static class WriteSinkTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Write.Bound<T>> {
+
+ @Override
+ public void translateNode(Write.Bound<T> transform, FlinkBatchTranslationContext context) {
+ String name = transform.getName();
+ PValue input = context.getInput(transform);
+ DataSet<T> inputDataSet = context.getInputDataSet(input);
+
+ inputDataSet.output(new SinkOutputFormat<>(transform, context.getPipelineOptions())).name(name);
+ }
+ }
+
+ private static class GroupByKeyOnlyTranslatorBatch<K, V> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<GroupByKey.GroupByKeyOnly<K, V>> {
+
+ @Override
+ public void translateNode(GroupByKey.GroupByKeyOnly<K, V> transform, FlinkBatchTranslationContext context) {
+ DataSet<KV<K, V>> inputDataSet = context.getInputDataSet(context.getInput(transform));
+ GroupReduceFunction<KV<K, V>, KV<K, Iterable<V>>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>();
+
+ TypeInformation<KV<K, Iterable<V>>> typeInformation = context.getTypeInfo(context.getOutput(transform));
+
+ Grouping<KV<K, V>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType()));
+
+ GroupReduceOperator<KV<K, V>, KV<K, Iterable<V>>> outputDataSet =
+ new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
+ context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+ }
+ }
+
+ /**
+ * Translates a GroupByKey while ignoring window assignments. This is identical to the {@link GroupByKeyOnlyTranslatorBatch}
+ */
+ private static class GroupByKeyTranslatorBatch<K, V> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<GroupByKey<K, V>> {
+
+ @Override
+ public void translateNode(GroupByKey<K, V> transform, FlinkBatchTranslationContext context) {
+ DataSet<KV<K, V>> inputDataSet = context.getInputDataSet(context.getInput(transform));
+ GroupReduceFunction<KV<K, V>, KV<K, Iterable<V>>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>();
+
+ TypeInformation<KV<K, Iterable<V>>> typeInformation = context.getTypeInfo(context.getOutput(transform));
+
+ Grouping<KV<K, V>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType()));
+
+ GroupReduceOperator<KV<K, V>, KV<K, Iterable<V>>> outputDataSet =
+ new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
+
+ context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+ }
+ }
+
+ private static class CombinePerKeyTranslatorBatch<K, VI, VA, VO> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Combine.PerKey<K, VI, VO>> {
+
+ @Override
+ public void translateNode(Combine.PerKey<K, VI, VO> transform, FlinkBatchTranslationContext context) {
+ DataSet<KV<K, VI>> inputDataSet = context.getInputDataSet(context.getInput(transform));
+
+ @SuppressWarnings("unchecked")
+ Combine.KeyedCombineFn<K, VI, VA, VO> keyedCombineFn = (Combine.KeyedCombineFn<K, VI, VA, VO>) transform.getFn();
+
+ KvCoder<K, VI> inputCoder = (KvCoder<K, VI>) context.getInput(transform).getCoder();
+
+ Coder<VA> accumulatorCoder =
+ null;
+ try {
+ accumulatorCoder = keyedCombineFn.getAccumulatorCoder(context.getInput(transform).getPipeline().getCoderRegistry(), inputCoder.getKeyCoder(), inputCoder.getValueCoder());
+ } catch (CannotProvideCoderException e) {
+ e.printStackTrace();
+ // TODO
+ }
+
+ TypeInformation<KV<K, VI>> kvCoderTypeInformation = new KvCoderTypeInformation<>(inputCoder);
+ TypeInformation<KV<K, VA>> partialReduceTypeInfo = new KvCoderTypeInformation<>(KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder));
+
+ Grouping<KV<K, VI>> inputGrouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation));
+
+ FlinkPartialReduceFunction<K, VI, VA> partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn);
+
+ // Partially GroupReduce the values into the intermediate format VA (combine)
+ GroupCombineOperator<KV<K, VI>, KV<K, VA>> groupCombine =
+ new GroupCombineOperator<>(inputGrouping, partialReduceTypeInfo, partialReduceFunction,
+ "GroupCombine: " + transform.getName());
+
+ // Reduce fully to VO
+ GroupReduceFunction<KV<K, VA>, KV<K, VO>> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn);
+
+ TypeInformation<KV<K, VO>> reduceTypeInfo = context.getTypeInfo(context.getOutput(transform));
+
+ Grouping<KV<K, VA>> intermediateGrouping = new UnsortedGrouping<>(groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType()));
+
+ // Fully reduce the values and create output format VO
+ GroupReduceOperator<KV<K, VA>, KV<K, VO>> outputDataSet =
+ new GroupReduceOperator<>(intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName());
+
+ context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+ }
+ }
+
+// private static class CombineGroupedValuesTranslator<K, VI, VO> implements FlinkPipelineTranslator.TransformTranslator<Combine.GroupedValues<K, VI, VO>> {
+//
+// @Override
+// public void translateNode(Combine.GroupedValues<K, VI, VO> transform, TranslationContext context) {
+// DataSet<KV<K, VI>> inputDataSet = context.getInputDataSet(transform.getInput());
+//
+// Combine.KeyedCombineFn<? super K, ? super VI, ?, VO> keyedCombineFn = transform.getFn();
+//
+// GroupReduceFunction<KV<K, VI>, KV<K, VO>> groupReduceFunction = new FlinkCombineFunction<>(keyedCombineFn);
+//
+// TypeInformation<KV<K, VO>> typeInformation = context.getTypeInfo(transform.getOutput());
+//
+// Grouping<KV<K, VI>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{""}, inputDataSet.getType()));
+//
+// GroupReduceOperator<KV<K, VI>, KV<K, VO>> outputDataSet =
+// new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
+// context.setOutputDataSet(transform.getOutput(), outputDataSet);
+// }
+// }
+
+ private static class ParDoBoundTranslatorBatch<IN, OUT> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<ParDo.Bound<IN, OUT>> {
+ private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundTranslatorBatch.class);
+
+ @Override
+ public void translateNode(ParDo.Bound<IN, OUT> transform, FlinkBatchTranslationContext context) {
+ DataSet<IN> inputDataSet = context.getInputDataSet(context.getInput(transform));
+
+ final DoFn<IN, OUT> doFn = transform.getFn();
+
+ TypeInformation<OUT> typeInformation = context.getTypeInfo(context.getOutput(transform));
+
+ FlinkDoFnFunction<IN, OUT> doFnWrapper = new FlinkDoFnFunction<>(doFn, context.getPipelineOptions());
+ MapPartitionOperator<IN, OUT> outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName());
+
+ transformSideInputs(transform.getSideInputs(), outputDataSet, context);
+
+ context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+ }
+ }
+
+ private static class ParDoBoundMultiTranslatorBatch<IN, OUT> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<ParDo.BoundMulti<IN, OUT>> {
+ private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundMultiTranslatorBatch.class);
+
+ @Override
+ public void translateNode(ParDo.BoundMulti<IN, OUT> transform, FlinkBatchTranslationContext context) {
+ DataSet<IN> inputDataSet = context.getInputDataSet(context.getInput(transform));
+
+ final DoFn<IN, OUT> doFn = transform.getFn();
+
+ Map<TupleTag<?>, PCollection<?>> outputs = context.getOutput(transform).getAll();
+
+ Map<TupleTag<?>, Integer> outputMap = Maps.newHashMap();
+ // put the main output at index 0, FlinkMultiOutputDoFnFunction also expects this
+ outputMap.put(transform.getMainOutputTag(), 0);
+ int count = 1;
+ for (TupleTag<?> tag: outputs.keySet()) {
+ if (!outputMap.containsKey(tag)) {
+ outputMap.put(tag, count++);
+ }
+ }
+
+ // collect all output Coders and create a UnionCoder for our tagged outputs
+ List<Coder<?>> outputCoders = Lists.newArrayList();
+ for (PCollection<?> coll: outputs.values()) {
+ outputCoders.add(coll.getCoder());
+ }
+
+ UnionCoder unionCoder = UnionCoder.of(outputCoders);
+
+ @SuppressWarnings("unchecked")
+ TypeInformation<RawUnionValue> typeInformation = new CoderTypeInformation<>(unionCoder);
+
+ @SuppressWarnings("unchecked")
+ FlinkMultiOutputDoFnFunction<IN, OUT> doFnWrapper = new FlinkMultiOutputDoFnFunction(doFn, context.getPipelineOptions(), outputMap);
+ MapPartitionOperator<IN, RawUnionValue> outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName());
+
+ transformSideInputs(transform.getSideInputs(), outputDataSet, context);
+
+ for (Map.Entry<TupleTag<?>, PCollection<?>> output: outputs.entrySet()) {
+ TypeInformation<Object> outputType = context.getTypeInfo(output.getValue());
+ int outputTag = outputMap.get(output.getKey());
+ FlinkMultiOutputPruningFunction<Object> pruningFunction = new FlinkMultiOutputPruningFunction<>(outputTag);
+ FlatMapOperator<RawUnionValue, Object> pruningOperator = new
+ FlatMapOperator<>(outputDataSet, outputType,
+ pruningFunction, output.getValue().getName());
+ context.setOutputDataSet(output.getValue(), pruningOperator);
+
+ }
+ }
+ }
+
+ private static class FlattenPCollectionTranslatorBatch<T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Flatten.FlattenPCollectionList<T>> {
+
+ @Override
+ public void translateNode(Flatten.FlattenPCollectionList<T> transform, FlinkBatchTranslationContext context) {
+ List<PCollection<T>> allInputs = context.getInput(transform).getAll();
+ DataSet<T> result = null;
+ for(PCollection<T> collection : allInputs) {
+ DataSet<T> current = context.getInputDataSet(collection);
+ if (result == null) {
+ result = current;
+ } else {
+ result = result.union(current);
+ }
+ }
+ context.setOutputDataSet(context.getOutput(transform), result);
+ }
+ }
+
+ private static class CreatePCollectionViewTranslatorBatch<R, T> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<View.CreatePCollectionView<R, T>> {
+ @Override
+ public void translateNode(View.CreatePCollectionView<R, T> transform, FlinkBatchTranslationContext context) {
+ DataSet<T> inputDataSet = context.getInputDataSet(context.getInput(transform));
+ PCollectionView<T> input = transform.apply(null);
+ context.setSideInputDataSet(input, inputDataSet);
+ }
+ }
+
+ private static class CreateTranslatorBatch<OUT> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<Create.Values<OUT>> {
+
+ @Override
+ public void translateNode(Create.Values<OUT> transform, FlinkBatchTranslationContext context) {
+ TypeInformation<OUT> typeInformation = context.getOutputTypeInfo();
+ Iterable<OUT> elements = transform.getElements();
+
+ // we need to serialize the elements to byte arrays, since they might contain
+ // elements that are not serializable by Java serialization. We deserialize them
+ // in the FlatMap function using the Coder.
+
+ List<byte[]> serializedElements = Lists.newArrayList();
+ Coder<OUT> coder = context.getOutput(transform).getCoder();
+ for (OUT element: elements) {
+ ByteArrayOutputStream bao = new ByteArrayOutputStream();
+ try {
+ coder.encode(element, bao, Coder.Context.OUTER);
+ serializedElements.add(bao.toByteArray());
+ } catch (IOException e) {
+ throw new RuntimeException("Could not serialize Create elements using Coder: " + e);
+ }
+ }
+
+ DataSet<Integer> initDataSet = context.getExecutionEnvironment().fromElements(1);
+ FlinkCreateFunction<Integer, OUT> flatMapFunction = new FlinkCreateFunction<>(serializedElements, coder);
+ FlatMapOperator<Integer, OUT> outputDataSet = new FlatMapOperator<>(initDataSet, typeInformation, flatMapFunction, transform.getName());
+
+ context.setOutputDataSet(context.getOutput(transform), outputDataSet);
+ }
+ }
+
+ private static void transformSideInputs(List<PCollectionView<?>> sideInputs,
+ MapPartitionOperator<?, ?> outputDataSet,
+ FlinkBatchTranslationContext context) {
+ // get corresponding Flink broadcast DataSets
+ for(PCollectionView<?> input : sideInputs) {
+ DataSet<?> broadcastSet = context.getSideInputDataSet(input);
+ outputDataSet.withBroadcastSet(broadcastSet, input.getTagInternal().getId());
+ }
+ }
+
+// Disabled because it depends on a pending pull request to the DataFlowSDK
+ /**
+ * Special composite transform translator. Only called if the CoGroup is two dimensional.
+ * @param <K>
+ */
+ private static class CoGroupByKeyTranslatorBatch<K, V1, V2> implements FlinkBatchPipelineTranslator.BatchTransformTranslator<CoGroupByKey<K>> {
+
+ @Override
+ public void translateNode(CoGroupByKey<K> transform, FlinkBatchTranslationContext context) {
+ KeyedPCollectionTuple<K> input = context.getInput(transform);
+
+ CoGbkResultSchema schema = input.getCoGbkResultSchema();
+ List<KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?>> keyedCollections = input.getKeyedCollections();
+
+ KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?> taggedCollection1 = keyedCollections.get(0);
+ KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?> taggedCollection2 = keyedCollections.get(1);
+
+ TupleTag<?> tupleTag1 = taggedCollection1.getTupleTag();
+ TupleTag<?> tupleTag2 = taggedCollection2.getTupleTag();
+
+ PCollection<? extends KV<K, ?>> collection1 = taggedCollection1.getCollection();
+ PCollection<? extends KV<K, ?>> collection2 = taggedCollection2.getCollection();
+
+ DataSet<KV<K,V1>> inputDataSet1 = context.getInputDataSet(collection1);
+ DataSet<KV<K,V2>> inputDataSet2 = context.getInputDataSet(collection2);
+
+ TypeInformation<KV<K,CoGbkResult>> typeInfo = context.getOutputTypeInfo();
+
+ FlinkCoGroupKeyedListAggregator<K,V1,V2> aggregator = new FlinkCoGroupKeyedListAggregator<>(schema, tupleTag1, tupleTag2);
+
+ Keys.ExpressionKeys<KV<K,V1>> keySelector1 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet1.getType());
+ Keys.ExpressionKeys<KV<K,V2>> keySelector2 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet2.getType());
+
+ DataSet<KV<K, CoGbkResult>> out = new CoGroupOperator<>(inputDataSet1, inputDataSet2,
+ keySelector1, keySelector2,
+ aggregator, typeInfo, null, transform.getName());
+ context.setOutputDataSet(context.getOutput(transform), out);
+ }
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Miscellaneous
+ // --------------------------------------------------------------------------------------------
+
+ private FlinkBatchTransformTranslators() {}
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java
new file mode 100644
index 0000000..fd99833
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java
@@ -0,0 +1,129 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation;
+
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.runners.flink.translation.types.KvCoderTypeInformation;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import com.google.cloud.dataflow.sdk.values.TypedPValue;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.DataSet;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.typeutils.GenericTypeInfo;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class FlinkBatchTranslationContext {
+
+ private final Map<PValue, DataSet<?>> dataSets;
+ private final Map<PCollectionView<?>, DataSet<?>> broadcastDataSets;
+
+ private final ExecutionEnvironment env;
+ private final PipelineOptions options;
+
+ private AppliedPTransform<?, ?, ?> currentTransform;
+
+ // ------------------------------------------------------------------------
+
+ public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions options) {
+ this.env = env;
+ this.options = options;
+ this.dataSets = new HashMap<>();
+ this.broadcastDataSets = new HashMap<>();
+ }
+
+ // ------------------------------------------------------------------------
+
+ public ExecutionEnvironment getExecutionEnvironment() {
+ return env;
+ }
+
+ public PipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ @SuppressWarnings("unchecked")
+ public <T> DataSet<T> getInputDataSet(PValue value) {
+ return (DataSet<T>) dataSets.get(value);
+ }
+
+ public void setOutputDataSet(PValue value, DataSet<?> set) {
+ if (!dataSets.containsKey(value)) {
+ dataSets.put(value, set);
+ }
+ }
+
+ /**
+ * Sets the AppliedPTransform which carries input/output.
+ * @param currentTransform
+ */
+ public void setCurrentTransform(AppliedPTransform<?, ?, ?> currentTransform) {
+ this.currentTransform = currentTransform;
+ }
+
+ @SuppressWarnings("unchecked")
+ public <T> DataSet<T> getSideInputDataSet(PCollectionView<?> value) {
+ return (DataSet<T>) broadcastDataSets.get(value);
+ }
+
+ public void setSideInputDataSet(PCollectionView<?> value, DataSet<?> set) {
+ if (!broadcastDataSets.containsKey(value)) {
+ broadcastDataSets.put(value, set);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ public <T> TypeInformation<T> getTypeInfo(PInput output) {
+ if (output instanceof TypedPValue) {
+ Coder<?> outputCoder = ((TypedPValue) output).getCoder();
+ if (outputCoder instanceof KvCoder) {
+ return new KvCoderTypeInformation((KvCoder) outputCoder);
+ } else {
+ return new CoderTypeInformation(outputCoder);
+ }
+ }
+ return new GenericTypeInfo<>((Class<T>)Object.class);
+ }
+
+ public <T> TypeInformation<T> getInputTypeInfo() {
+ return getTypeInfo(currentTransform.getInput());
+ }
+
+ public <T> TypeInformation<T> getOutputTypeInfo() {
+ return getTypeInfo((PValue) currentTransform.getOutput());
+ }
+
+ @SuppressWarnings("unchecked")
+ <I extends PInput> I getInput(PTransform<I, ?> transform) {
+ return (I) currentTransform.getInput();
+ }
+
+ @SuppressWarnings("unchecked")
+ <O extends POutput> O getOutput(PTransform<?, O> transform) {
+ return (O) currentTransform.getOutput();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java
new file mode 100644
index 0000000..efe217f
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+
+/**
+ * The role of this class is to translate the Beam operators to
+ * their Flink counterparts. If we have a streaming job, this is instantiated as a
+ * {@link FlinkStreamingPipelineTranslator}. In other case, i.e. for a batch job,
+ * a {@link FlinkBatchPipelineTranslator} is created. Correspondingly, the
+ * {@link com.google.cloud.dataflow.sdk.values.PCollection}-based user-provided job is translated into
+ * a {@link org.apache.flink.streaming.api.datastream.DataStream} (for streaming) or a
+ * {@link org.apache.flink.api.java.DataSet} (for batch) one.
+ */
+public abstract class FlinkPipelineTranslator implements Pipeline.PipelineVisitor {
+
+ public void translate(Pipeline pipeline) {
+ pipeline.traverseTopologically(this);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
new file mode 100644
index 0000000..21a8133
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingPipelineTranslator.java
@@ -0,0 +1,143 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation;
+
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+/**
+ * This is a {@link FlinkPipelineTranslator} for streaming jobs. Its role is to translate the user-provided
+ * {@link com.google.cloud.dataflow.sdk.values.PCollection}-based job into a
+ * {@link org.apache.flink.streaming.api.datastream.DataStream} one.
+ *
+ * This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator}
+ * */
+public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
+
+ /** The necessary context in the case of a straming job. */
+ private final FlinkStreamingTranslationContext streamingContext;
+
+ private int depth = 0;
+
+ /** Composite transform that we want to translate before proceeding with other transforms. */
+ private PTransform<?, ?> currentCompositeTransform;
+
+ public FlinkStreamingPipelineTranslator(StreamExecutionEnvironment env, PipelineOptions options) {
+ this.streamingContext = new FlinkStreamingTranslationContext(env, options);
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Pipeline Visitor Methods
+ // --------------------------------------------------------------------------------------------
+
+ @Override
+ public void enterCompositeTransform(TransformTreeNode node) {
+ System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node));
+
+ PTransform<?, ?> transform = node.getTransform();
+ if (transform != null && currentCompositeTransform == null) {
+
+ StreamTransformTranslator<?> translator = FlinkStreamingTransformTranslators.getTranslator(transform);
+ if (translator != null) {
+ currentCompositeTransform = transform;
+ }
+ }
+ this.depth++;
+ }
+
+ @Override
+ public void leaveCompositeTransform(TransformTreeNode node) {
+ PTransform<?, ?> transform = node.getTransform();
+ if (transform != null && currentCompositeTransform == transform) {
+
+ StreamTransformTranslator<?> translator = FlinkStreamingTransformTranslators.getTranslator(transform);
+ if (translator != null) {
+ System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node));
+ applyStreamingTransform(transform, node, translator);
+ currentCompositeTransform = null;
+ } else {
+ throw new IllegalStateException("Attempted to translate composite transform " +
+ "but no translator was found: " + currentCompositeTransform);
+ }
+ }
+ this.depth--;
+ System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node));
+ }
+
+ @Override
+ public void visitTransform(TransformTreeNode node) {
+ System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node));
+ if (currentCompositeTransform != null) {
+ // ignore it
+ return;
+ }
+
+ // get the transformation corresponding to hte node we are
+ // currently visiting and translate it into its Flink alternative.
+
+ PTransform<?, ?> transform = node.getTransform();
+ StreamTransformTranslator<?> translator = FlinkStreamingTransformTranslators.getTranslator(transform);
+ if (translator == null) {
+ System.out.println(node.getTransform().getClass());
+ throw new UnsupportedOperationException("The transform " + transform + " is currently not supported.");
+ }
+ applyStreamingTransform(transform, node, translator);
+ }
+
+ @Override
+ public void visitValue(PValue value, TransformTreeNode producer) {
+ // do nothing here
+ }
+
+ private <T extends PTransform<?, ?>> void applyStreamingTransform(PTransform<?, ?> transform, TransformTreeNode node, StreamTransformTranslator<?> translator) {
+
+ @SuppressWarnings("unchecked")
+ T typedTransform = (T) transform;
+
+ @SuppressWarnings("unchecked")
+ StreamTransformTranslator<T> typedTranslator = (StreamTransformTranslator<T>) translator;
+
+ // create the applied PTransform on the streamingContext
+ streamingContext.setCurrentTransform(AppliedPTransform.of(
+ node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform));
+ typedTranslator.translateNode(typedTransform, streamingContext);
+ }
+
+ /**
+ * The interface that every Flink translator of a Beam operator should implement.
+ * This interface is for <b>streaming</b> jobs. For examples of such translators see
+ * {@link FlinkStreamingTransformTranslators}.
+ */
+ public interface StreamTransformTranslator<Type extends PTransform> {
+ void translateNode(Type transform, FlinkStreamingTranslationContext context);
+ }
+
+ private static String genSpaces(int n) {
+ String s = "";
+ for (int i = 0; i < n; i++) {
+ s += "| ";
+ }
+ return s;
+ }
+
+ private static String formatNodeName(TransformTreeNode node) {
+ return node.toString().split("@")[1] + node.getTransform();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
new file mode 100644
index 0000000..1083848
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTransformTranslators.java
@@ -0,0 +1,404 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation;
+
+import org.apache.beam.runners.flink.translation.functions.UnionCoder;
+import org.apache.beam.runners.flink.translation.types.CoderTypeInformation;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.*;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.io.FlinkStreamingCreateFunction;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSource;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper;
+import com.google.api.client.util.Maps;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.*;
+import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
+import com.google.cloud.dataflow.sdk.transforms.windowing.*;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.util.WindowingStrategy;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.collect.Lists;
+import org.apache.flink.api.common.functions.FilterFunction;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.streaming.api.datastream.*;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * This class contains all the mappings between Beam and Flink
+ * <b>streaming</b> transformations. The {@link FlinkStreamingPipelineTranslator}
+ * traverses the Beam job and comes here to translate the encountered Beam transformations
+ * into Flink one, based on the mapping available in this class.
+ */
+public class FlinkStreamingTransformTranslators {
+
+ // --------------------------------------------------------------------------------------------
+ // Transform Translator Registry
+ // --------------------------------------------------------------------------------------------
+
+ @SuppressWarnings("rawtypes")
+ private static final Map<Class<? extends PTransform>, FlinkStreamingPipelineTranslator.StreamTransformTranslator> TRANSLATORS = new HashMap<>();
+
+ // here you can find all the available translators.
+ static {
+ TRANSLATORS.put(Create.Values.class, new CreateStreamingTranslator());
+ TRANSLATORS.put(Read.Unbounded.class, new UnboundedReadSourceTranslator());
+ TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator());
+ TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteBoundStreamingTranslator());
+ TRANSLATORS.put(Window.Bound.class, new WindowBoundTranslator());
+ TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslator());
+ TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslator());
+ TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator());
+ TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiStreamingTranslator());
+ }
+
+ public static FlinkStreamingPipelineTranslator.StreamTransformTranslator<?> getTranslator(PTransform<?, ?> transform) {
+ return TRANSLATORS.get(transform.getClass());
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Transformation Implementations
+ // --------------------------------------------------------------------------------------------
+
+ private static class CreateStreamingTranslator<OUT> implements
+ FlinkStreamingPipelineTranslator.StreamTransformTranslator<Create.Values<OUT>> {
+
+ @Override
+ public void translateNode(Create.Values<OUT> transform, FlinkStreamingTranslationContext context) {
+ PCollection<OUT> output = context.getOutput(transform);
+ Iterable<OUT> elements = transform.getElements();
+
+ // we need to serialize the elements to byte arrays, since they might contain
+ // elements that are not serializable by Java serialization. We deserialize them
+ // in the FlatMap function using the Coder.
+
+ List<byte[]> serializedElements = Lists.newArrayList();
+ Coder<OUT> elementCoder = context.getOutput(transform).getCoder();
+ for (OUT element: elements) {
+ ByteArrayOutputStream bao = new ByteArrayOutputStream();
+ try {
+ elementCoder.encode(element, bao, Coder.Context.OUTER);
+ serializedElements.add(bao.toByteArray());
+ } catch (IOException e) {
+ throw new RuntimeException("Could not serialize Create elements using Coder: " + e);
+ }
+ }
+
+
+ DataStream<Integer> initDataSet = context.getExecutionEnvironment().fromElements(1);
+
+ FlinkStreamingCreateFunction<Integer, OUT> createFunction =
+ new FlinkStreamingCreateFunction<>(serializedElements, elementCoder);
+
+ WindowedValue.ValueOnlyWindowedValueCoder<OUT> windowCoder = WindowedValue.getValueOnlyCoder(elementCoder);
+ TypeInformation<WindowedValue<OUT>> outputType = new CoderTypeInformation<>(windowCoder);
+
+ DataStream<WindowedValue<OUT>> outputDataStream = initDataSet.flatMap(createFunction)
+ .returns(outputType);
+
+ context.setOutputDataStream(context.getOutput(transform), outputDataStream);
+ }
+ }
+
+
+ private static class TextIOWriteBoundStreamingTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<TextIO.Write.Bound<T>> {
+ private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteBoundStreamingTranslator.class);
+
+ @Override
+ public void translateNode(TextIO.Write.Bound<T> transform, FlinkStreamingTranslationContext context) {
+ PValue input = context.getInput(transform);
+ DataStream<WindowedValue<T>> inputDataStream = context.getInputDataStream(input);
+
+ String filenamePrefix = transform.getFilenamePrefix();
+ String filenameSuffix = transform.getFilenameSuffix();
+ boolean needsValidation = transform.needsValidation();
+ int numShards = transform.getNumShards();
+ String shardNameTemplate = transform.getShardNameTemplate();
+
+ // TODO: Implement these. We need Flink support for this.
+ LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation);
+ LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix);
+ LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
+
+ DataStream<String> dataSink = inputDataStream.flatMap(new FlatMapFunction<WindowedValue<T>, String>() {
+ @Override
+ public void flatMap(WindowedValue<T> value, Collector<String> out) throws Exception {
+ out.collect(value.getValue().toString());
+ }
+ });
+ DataStreamSink<String> output = dataSink.writeAsText(filenamePrefix, FileSystem.WriteMode.OVERWRITE);
+
+ if (numShards > 0) {
+ output.setParallelism(numShards);
+ }
+ }
+ }
+
+ private static class UnboundedReadSourceTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Read.Unbounded<T>> {
+
+ @Override
+ public void translateNode(Read.Unbounded<T> transform, FlinkStreamingTranslationContext context) {
+ PCollection<T> output = context.getOutput(transform);
+
+ DataStream<WindowedValue<T>> source;
+ if (transform.getSource().getClass().equals(UnboundedFlinkSource.class)) {
+ UnboundedFlinkSource flinkSource = (UnboundedFlinkSource) transform.getSource();
+ source = context.getExecutionEnvironment()
+ .addSource(flinkSource.getFlinkSource())
+ .flatMap(new FlatMapFunction<String, WindowedValue<String>>() {
+ @Override
+ public void flatMap(String s, Collector<WindowedValue<String>> collector) throws Exception {
+ collector.collect(WindowedValue.<String>of(s, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
+ }
+ });
+ } else {
+ source = context.getExecutionEnvironment()
+ .addSource(new UnboundedSourceWrapper<>(context.getPipelineOptions(), transform));
+ }
+ context.setOutputDataStream(output, source);
+ }
+ }
+
+ private static class ParDoBoundStreamingTranslator<IN, OUT> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<ParDo.Bound<IN, OUT>> {
+
+ @Override
+ public void translateNode(ParDo.Bound<IN, OUT> transform, FlinkStreamingTranslationContext context) {
+ PCollection<OUT> output = context.getOutput(transform);
+
+ final WindowingStrategy<OUT, ? extends BoundedWindow> windowingStrategy =
+ (WindowingStrategy<OUT, ? extends BoundedWindow>)
+ context.getOutput(transform).getWindowingStrategy();
+
+ WindowedValue.WindowedValueCoder<OUT> outputStreamCoder = WindowedValue.getFullCoder(output.getCoder(),
+ windowingStrategy.getWindowFn().windowCoder());
+ CoderTypeInformation<WindowedValue<OUT>> outputWindowedValueCoder =
+ new CoderTypeInformation<>(outputStreamCoder);
+
+ FlinkParDoBoundWrapper<IN, OUT> doFnWrapper = new FlinkParDoBoundWrapper<>(
+ context.getPipelineOptions(), windowingStrategy, transform.getFn());
+ DataStream<WindowedValue<IN>> inputDataStream = context.getInputDataStream(context.getInput(transform));
+ SingleOutputStreamOperator<WindowedValue<OUT>> outDataStream = inputDataStream.flatMap(doFnWrapper)
+ .returns(outputWindowedValueCoder);
+
+ context.setOutputDataStream(context.getOutput(transform), outDataStream);
+ }
+ }
+
+ public static class WindowBoundTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Window.Bound<T>> {
+
+ @Override
+ public void translateNode(Window.Bound<T> transform, FlinkStreamingTranslationContext context) {
+ PValue input = context.getInput(transform);
+ DataStream<WindowedValue<T>> inputDataStream = context.getInputDataStream(input);
+
+ final WindowingStrategy<T, ? extends BoundedWindow> windowingStrategy =
+ (WindowingStrategy<T, ? extends BoundedWindow>)
+ context.getOutput(transform).getWindowingStrategy();
+
+ final WindowFn<T, ? extends BoundedWindow> windowFn = windowingStrategy.getWindowFn();
+
+ WindowedValue.WindowedValueCoder<T> outputStreamCoder = WindowedValue.getFullCoder(
+ context.getInput(transform).getCoder(), windowingStrategy.getWindowFn().windowCoder());
+ CoderTypeInformation<WindowedValue<T>> outputWindowedValueCoder =
+ new CoderTypeInformation<>(outputStreamCoder);
+
+ final FlinkParDoBoundWrapper<T, T> windowDoFnAssigner = new FlinkParDoBoundWrapper<>(
+ context.getPipelineOptions(), windowingStrategy, createWindowAssigner(windowFn));
+
+ SingleOutputStreamOperator<WindowedValue<T>> windowedStream =
+ inputDataStream.flatMap(windowDoFnAssigner).returns(outputWindowedValueCoder);
+ context.setOutputDataStream(context.getOutput(transform), windowedStream);
+ }
+
+ private static <T, W extends BoundedWindow> DoFn<T, T> createWindowAssigner(final WindowFn<T, W> windowFn) {
+ return new DoFn<T, T>() {
+
+ @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 Collection<? extends BoundedWindow> windows() {
+ return c.windowingInternals().windows();
+ }
+ });
+
+ c.windowingInternals().outputWindowedValue(
+ c.element(), c.timestamp(), windows, c.pane());
+ }
+ };
+ }
+ }
+
+ public static class GroupByKeyTranslator<K, V> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<GroupByKey<K, V>> {
+
+ @Override
+ public void translateNode(GroupByKey<K, V> transform, FlinkStreamingTranslationContext context) {
+ PValue input = context.getInput(transform);
+
+ DataStream<WindowedValue<KV<K, V>>> inputDataStream = context.getInputDataStream(input);
+ KvCoder<K, V> inputKvCoder = (KvCoder<K, V>) context.getInput(transform).getCoder();
+
+ KeyedStream<WindowedValue<KV<K, V>>, K> groupByKStream = FlinkGroupByKeyWrapper
+ .groupStreamByKey(inputDataStream, inputKvCoder);
+
+ DataStream<WindowedValue<KV<K, Iterable<V>>>> groupedByKNWstream =
+ FlinkGroupAlsoByWindowWrapper.createForIterable(context.getPipelineOptions(),
+ context.getInput(transform), groupByKStream);
+
+ context.setOutputDataStream(context.getOutput(transform), groupedByKNWstream);
+ }
+ }
+
+ public static class CombinePerKeyTranslator<K, VIN, VACC, VOUT> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Combine.PerKey<K, VIN, VOUT>> {
+
+ @Override
+ public void translateNode(Combine.PerKey<K, VIN, VOUT> transform, FlinkStreamingTranslationContext context) {
+ PValue input = context.getInput(transform);
+
+ DataStream<WindowedValue<KV<K, VIN>>> inputDataStream = context.getInputDataStream(input);
+ KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) context.getInput(transform).getCoder();
+ KvCoder<K, VOUT> outputKvCoder = (KvCoder<K, VOUT>) context.getOutput(transform).getCoder();
+
+ KeyedStream<WindowedValue<KV<K, VIN>>, K> groupByKStream = FlinkGroupByKeyWrapper
+ .groupStreamByKey(inputDataStream, inputKvCoder);
+
+ Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combineFn = (Combine.KeyedCombineFn<K, VIN, VACC, VOUT>) transform.getFn();
+ DataStream<WindowedValue<KV<K, VOUT>>> groupedByKNWstream =
+ FlinkGroupAlsoByWindowWrapper.create(context.getPipelineOptions(),
+ context.getInput(transform), groupByKStream, combineFn, outputKvCoder);
+
+ context.setOutputDataStream(context.getOutput(transform), groupedByKNWstream);
+ }
+ }
+
+ public static class FlattenPCollectionTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Flatten.FlattenPCollectionList<T>> {
+
+ @Override
+ public void translateNode(Flatten.FlattenPCollectionList<T> transform, FlinkStreamingTranslationContext context) {
+ List<PCollection<T>> allInputs = context.getInput(transform).getAll();
+ DataStream<T> result = null;
+ for (PCollection<T> collection : allInputs) {
+ DataStream<T> current = context.getInputDataStream(collection);
+ result = (result == null) ? current : result.union(current);
+ }
+ context.setOutputDataStream(context.getOutput(transform), result);
+ }
+ }
+
+ public static class ParDoBoundMultiStreamingTranslator<IN, OUT> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<ParDo.BoundMulti<IN, OUT>> {
+
+ private final int MAIN_TAG_INDEX = 0;
+
+ @Override
+ public void translateNode(ParDo.BoundMulti<IN, OUT> transform, FlinkStreamingTranslationContext context) {
+
+ // we assume that the transformation does not change the windowing strategy.
+ WindowingStrategy<?, ? extends BoundedWindow> windowingStrategy = context.getInput(transform).getWindowingStrategy();
+
+ Map<TupleTag<?>, PCollection<?>> outputs = context.getOutput(transform).getAll();
+ Map<TupleTag<?>, Integer> tagsToLabels = transformTupleTagsToLabels(
+ transform.getMainOutputTag(), outputs.keySet());
+
+ UnionCoder intermUnionCoder = getIntermUnionCoder(outputs.values());
+ WindowedValue.WindowedValueCoder<RawUnionValue> outputStreamCoder = WindowedValue.getFullCoder(
+ intermUnionCoder, windowingStrategy.getWindowFn().windowCoder());
+
+ CoderTypeInformation<WindowedValue<RawUnionValue>> intermWindowedValueCoder =
+ new CoderTypeInformation<>(outputStreamCoder);
+
+ FlinkParDoBoundMultiWrapper<IN, OUT> doFnWrapper = new FlinkParDoBoundMultiWrapper<>(
+ context.getPipelineOptions(), windowingStrategy, transform.getFn(),
+ transform.getMainOutputTag(), tagsToLabels);
+
+ DataStream<WindowedValue<IN>> inputDataStream = context.getInputDataStream(context.getInput(transform));
+ SingleOutputStreamOperator<WindowedValue<RawUnionValue>> intermDataStream =
+ inputDataStream.flatMap(doFnWrapper).returns(intermWindowedValueCoder);
+
+ for (Map.Entry<TupleTag<?>, PCollection<?>> output : outputs.entrySet()) {
+ final int outputTag = tagsToLabels.get(output.getKey());
+
+ WindowedValue.WindowedValueCoder<?> coderForTag = WindowedValue.getFullCoder(
+ output.getValue().getCoder(),
+ windowingStrategy.getWindowFn().windowCoder());
+
+ CoderTypeInformation<WindowedValue<?>> windowedValueCoder =
+ new CoderTypeInformation(coderForTag);
+
+ context.setOutputDataStream(output.getValue(),
+ intermDataStream.filter(new FilterFunction<WindowedValue<RawUnionValue>>() {
+ @Override
+ public boolean filter(WindowedValue<RawUnionValue> value) throws Exception {
+ return value.getValue().getUnionTag() == outputTag;
+ }
+ }).flatMap(new FlatMapFunction<WindowedValue<RawUnionValue>, WindowedValue<?>>() {
+ @Override
+ public void flatMap(WindowedValue<RawUnionValue> value, Collector<WindowedValue<?>> collector) throws Exception {
+ collector.collect(WindowedValue.of(
+ value.getValue().getValue(),
+ value.getTimestamp(),
+ value.getWindows(),
+ value.getPane()));
+ }
+ }).returns(windowedValueCoder));
+ }
+ }
+
+ private Map<TupleTag<?>, Integer> transformTupleTagsToLabels(TupleTag<?> mainTag, Set<TupleTag<?>> secondaryTags) {
+ Map<TupleTag<?>, Integer> tagToLabelMap = Maps.newHashMap();
+ tagToLabelMap.put(mainTag, MAIN_TAG_INDEX);
+ int count = MAIN_TAG_INDEX + 1;
+ for (TupleTag<?> tag : secondaryTags) {
+ if (!tagToLabelMap.containsKey(tag)) {
+ tagToLabelMap.put(tag, count++);
+ }
+ }
+ return tagToLabelMap;
+ }
+
+ private UnionCoder getIntermUnionCoder(Collection<PCollection<?>> taggedCollections) {
+ List<Coder<?>> outputCoders = Lists.newArrayList();
+ for (PCollection<?> coll : taggedCollections) {
+ outputCoders.add(coll.getCoder());
+ }
+ return UnionCoder.of(outputCoders);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java
new file mode 100644
index 0000000..83b19ee
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java
@@ -0,0 +1,87 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation;
+
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import com.google.common.base.Preconditions;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class FlinkStreamingTranslationContext {
+
+ private final StreamExecutionEnvironment env;
+ private final PipelineOptions options;
+
+ /**
+ * Keeps a mapping between the output value of the PTransform (in Dataflow) and the
+ * Flink Operator that produced it, after the translation of the correspondinf PTransform
+ * to its Flink equivalent.
+ * */
+ private final Map<PValue, DataStream<?>> dataStreams;
+
+ private AppliedPTransform<?, ?, ?> currentTransform;
+
+ public FlinkStreamingTranslationContext(StreamExecutionEnvironment env, PipelineOptions options) {
+ this.env = Preconditions.checkNotNull(env);
+ this.options = Preconditions.checkNotNull(options);
+ this.dataStreams = new HashMap<>();
+ }
+
+ public StreamExecutionEnvironment getExecutionEnvironment() {
+ return env;
+ }
+
+ public PipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ @SuppressWarnings("unchecked")
+ public <T> DataStream<T> getInputDataStream(PValue value) {
+ return (DataStream<T>) dataStreams.get(value);
+ }
+
+ public void setOutputDataStream(PValue value, DataStream<?> set) {
+ if (!dataStreams.containsKey(value)) {
+ dataStreams.put(value, set);
+ }
+ }
+
+ /**
+ * Sets the AppliedPTransform which carries input/output.
+ * @param currentTransform
+ */
+ public void setCurrentTransform(AppliedPTransform<?, ?, ?> currentTransform) {
+ this.currentTransform = currentTransform;
+ }
+
+ @SuppressWarnings("unchecked")
+ public <I extends PInput> I getInput(PTransform<I, ?> transform) {
+ return (I) currentTransform.getInput();
+ }
+
+ @SuppressWarnings("unchecked")
+ public <O extends POutput> O getOutput(PTransform<?, O> transform) {
+ return (O) currentTransform.getOutput();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java
new file mode 100644
index 0000000..0e855ad
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCoGroupKeyedListAggregator.java
@@ -0,0 +1,58 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.functions;
+
+import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResultSchema;
+import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.apache.flink.api.common.functions.CoGroupFunction;
+import org.apache.flink.util.Collector;
+
+import java.util.ArrayList;
+import java.util.List;
+
+
+public class FlinkCoGroupKeyedListAggregator<K,V1,V2> implements CoGroupFunction<KV<K,V1>, KV<K,V2>, KV<K, CoGbkResult>>{
+
+ private CoGbkResultSchema schema;
+ private TupleTag<?> tupleTag1;
+ private TupleTag<?> tupleTag2;
+
+ public FlinkCoGroupKeyedListAggregator(CoGbkResultSchema schema, TupleTag<?> tupleTag1, TupleTag<?> tupleTag2) {
+ this.schema = schema;
+ this.tupleTag1 = tupleTag1;
+ this.tupleTag2 = tupleTag2;
+ }
+
+ @Override
+ public void coGroup(Iterable<KV<K,V1>> first, Iterable<KV<K,V2>> second, Collector<KV<K, CoGbkResult>> out) throws Exception {
+ K k = null;
+ List<RawUnionValue> result = new ArrayList<>();
+ int index1 = schema.getIndex(tupleTag1);
+ for (KV<K,?> entry : first) {
+ k = entry.getKey();
+ result.add(new RawUnionValue(index1, entry.getValue()));
+ }
+ int index2 = schema.getIndex(tupleTag2);
+ for (KV<K,?> entry : second) {
+ k = entry.getKey();
+ result.add(new RawUnionValue(index2, entry.getValue()));
+ }
+ out.collect(KV.of(k, new CoGbkResult(schema, result)));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java
new file mode 100644
index 0000000..32efe41
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkCreateFunction.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.functions;
+
+import org.apache.beam.runners.flink.translation.types.VoidCoderTypeSerializer;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.util.Collector;
+
+import java.io.ByteArrayInputStream;
+import java.util.List;
+
+/**
+ * This is a hack for transforming a {@link com.google.cloud.dataflow.sdk.transforms.Create}
+ * operation. Flink does not allow {@code null} in it's equivalent operation:
+ * {@link org.apache.flink.api.java.ExecutionEnvironment#fromElements(Object[])}. Therefore
+ * we use a DataSource with one dummy element and output the elements of the Create operation
+ * inside this FlatMap.
+ */
+public class FlinkCreateFunction<IN, OUT> implements FlatMapFunction<IN, OUT> {
+
+ private final List<byte[]> elements;
+ private final Coder<OUT> coder;
+
+ public FlinkCreateFunction(List<byte[]> elements, Coder<OUT> coder) {
+ this.elements = elements;
+ this.coder = coder;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void flatMap(IN value, Collector<OUT> out) throws Exception {
+
+ for (byte[] element : elements) {
+ ByteArrayInputStream bai = new ByteArrayInputStream(element);
+ OUT outValue = coder.decode(bai, Coder.Context.OUTER);
+ if (outValue == null) {
+ // TODO Flink doesn't allow null values in records
+ out.collect((OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE);
+ } else {
+ out.collect(outValue);
+ }
+ }
+
+ out.close();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
new file mode 100644
index 0000000..9e8b7cd
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
@@ -0,0 +1,202 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.functions;
+
+import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
+import com.google.cloud.dataflow.sdk.util.TimerInternals;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.util.WindowingInternals;
+import com.google.cloud.dataflow.sdk.util.state.StateInternals;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.collect.ImmutableList;
+import org.apache.flink.api.common.functions.RichMapPartitionFunction;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Encapsulates a {@link com.google.cloud.dataflow.sdk.transforms.DoFn}
+ * inside a Flink {@link org.apache.flink.api.common.functions.RichMapPartitionFunction}.
+ */
+public class FlinkDoFnFunction<IN, OUT> extends RichMapPartitionFunction<IN, OUT> {
+
+ private final DoFn<IN, OUT> doFn;
+ private transient PipelineOptions options;
+
+ public FlinkDoFnFunction(DoFn<IN, OUT> doFn, PipelineOptions options) {
+ this.doFn = doFn;
+ this.options = options;
+ }
+
+ private void writeObject(ObjectOutputStream out)
+ throws IOException, ClassNotFoundException {
+ out.defaultWriteObject();
+ ObjectMapper mapper = new ObjectMapper();
+ mapper.writeValue(out, options);
+ }
+
+ private void readObject(ObjectInputStream in)
+ throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ ObjectMapper mapper = new ObjectMapper();
+ options = mapper.readValue(in, PipelineOptions.class);
+ }
+
+ @Override
+ public void mapPartition(Iterable<IN> values, Collector<OUT> out) throws Exception {
+ ProcessContext context = new ProcessContext(doFn, out);
+ this.doFn.startBundle(context);
+ for (IN value : values) {
+ context.inValue = value;
+ doFn.processElement(context);
+ }
+ this.doFn.finishBundle(context);
+ }
+
+ private class ProcessContext extends DoFn<IN, OUT>.ProcessContext {
+
+ IN inValue;
+ Collector<OUT> outCollector;
+
+ public ProcessContext(DoFn<IN, OUT> fn, Collector<OUT> outCollector) {
+ fn.super();
+ super.setupDelegateAggregators();
+ this.outCollector = outCollector;
+ }
+
+ @Override
+ public IN element() {
+ return this.inValue;
+ }
+
+
+ @Override
+ public Instant timestamp() {
+ return Instant.now();
+ }
+
+ @Override
+ public BoundedWindow window() {
+ return GlobalWindow.INSTANCE;
+ }
+
+ @Override
+ public PaneInfo pane() {
+ return PaneInfo.NO_FIRING;
+ }
+
+ @Override
+ public WindowingInternals<IN, OUT> windowingInternals() {
+ return new WindowingInternals<IN, OUT>() {
+ @Override
+ public StateInternals stateInternals() {
+ return null;
+ }
+
+ @Override
+ public void outputWindowedValue(OUT output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+
+ }
+
+ @Override
+ public TimerInternals timerInternals() {
+ return null;
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ return ImmutableList.of(GlobalWindow.INSTANCE);
+ }
+
+ @Override
+ public PaneInfo pane() {
+ return PaneInfo.NO_FIRING;
+ }
+
+ @Override
+ public <T> void writePCollectionViewData(TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+ throw new RuntimeException("sideInput() not implemented.");
+ }
+ };
+ }
+
+ @Override
+ public PipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view) {
+ List<T> sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal().getId());
+ List<WindowedValue<?>> windowedValueList = new ArrayList<>(sideInput.size());
+ for (T input : sideInput) {
+ windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE), pane()));
+ }
+ return view.fromIterableInternal(windowedValueList);
+ }
+
+ @Override
+ public void output(OUT output) {
+ outCollector.collect(output);
+ }
+
+ @Override
+ public void outputWithTimestamp(OUT output, Instant timestamp) {
+ // not FLink's way, just output normally
+ output(output);
+ }
+
+ @Override
+ public <T> void sideOutput(TupleTag<T> tag, T output) {
+ // ignore the side output, this can happen when a user does not register
+ // side outputs but then outputs using a freshly created TupleTag.
+ }
+
+ @Override
+ public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+ sideOutput(tag, output);
+ }
+
+ @Override
+ protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
+ SerializableFnAggregatorWrapper<AggInputT, AggOutputT> wrapper = new SerializableFnAggregatorWrapper<>(combiner);
+ getRuntimeContext().addAccumulator(name, wrapper);
+ return wrapper;
+ }
+
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java
new file mode 100644
index 0000000..56fa88d
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkKeyedListAggregationFunction.java
@@ -0,0 +1,75 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.functions;
+
+import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.util.Collector;
+
+import java.util.Iterator;
+
+/**
+ * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a
+ * {@link com.google.cloud.dataflow.sdk.transforms.GroupByKey} operation. This reads the input
+ * {@link com.google.cloud.dataflow.sdk.values.KV} elements, extracts the key and collects
+ * the values in a {@code List}.
+ */
+public class FlinkKeyedListAggregationFunction<K,V> implements GroupReduceFunction<KV<K, V>, KV<K, Iterable<V>>> {
+
+ @Override
+ public void reduce(Iterable<KV<K, V>> values, Collector<KV<K, Iterable<V>>> out) throws Exception {
+ Iterator<KV<K, V>> it = values.iterator();
+ KV<K, V> first = it.next();
+ Iterable<V> passThrough = new PassThroughIterable<>(first, it);
+ out.collect(KV.of(first.getKey(), passThrough));
+ }
+
+ private static class PassThroughIterable<K, V> implements Iterable<V>, Iterator<V> {
+ private KV<K, V> first;
+ private Iterator<KV<K, V>> iterator;
+
+ public PassThroughIterable(KV<K, V> first, Iterator<KV<K, V>> iterator) {
+ this.first = first;
+ this.iterator = iterator;
+ }
+
+ @Override
+ public Iterator<V> iterator() {
+ return this;
+ }
+
+ @Override
+ public boolean hasNext() {
+ return first != null || iterator.hasNext();
+ }
+
+ @Override
+ public V next() {
+ if (first != null) {
+ V result = first.getValue();
+ first = null;
+ return result;
+ } else {
+ return iterator.next().getValue();
+ }
+ }
+
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException("Cannot remove elements from input.");
+ }
+ }
+}
[45/50] [abbrv] incubator-beam git commit: [flink] convert tabs to 2
spaces
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java
index 1072fa3..ec1d6c8 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkBatchTranslationContext.java
@@ -38,92 +38,92 @@ import java.util.HashMap;
import java.util.Map;
public class FlinkBatchTranslationContext {
-
- private final Map<PValue, DataSet<?>> dataSets;
- private final Map<PCollectionView<?>, DataSet<?>> broadcastDataSets;
+
+ private final Map<PValue, DataSet<?>> dataSets;
+ private final Map<PCollectionView<?>, DataSet<?>> broadcastDataSets;
- private final ExecutionEnvironment env;
- private final PipelineOptions options;
+ private final ExecutionEnvironment env;
+ private final PipelineOptions options;
- private AppliedPTransform<?, ?, ?> currentTransform;
-
- // ------------------------------------------------------------------------
-
- public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions options) {
- this.env = env;
- this.options = options;
- this.dataSets = new HashMap<>();
- this.broadcastDataSets = new HashMap<>();
- }
-
- // ------------------------------------------------------------------------
-
- public ExecutionEnvironment getExecutionEnvironment() {
- return env;
- }
+ private AppliedPTransform<?, ?, ?> currentTransform;
+
+ // ------------------------------------------------------------------------
+
+ public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions options) {
+ this.env = env;
+ this.options = options;
+ this.dataSets = new HashMap<>();
+ this.broadcastDataSets = new HashMap<>();
+ }
+
+ // ------------------------------------------------------------------------
+
+ public ExecutionEnvironment getExecutionEnvironment() {
+ return env;
+ }
- public PipelineOptions getPipelineOptions() {
- return options;
- }
-
- @SuppressWarnings("unchecked")
- public <T> DataSet<T> getInputDataSet(PValue value) {
- return (DataSet<T>) dataSets.get(value);
- }
+ public PipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ @SuppressWarnings("unchecked")
+ public <T> DataSet<T> getInputDataSet(PValue value) {
+ return (DataSet<T>) dataSets.get(value);
+ }
- public void setOutputDataSet(PValue value, DataSet<?> set) {
- if (!dataSets.containsKey(value)) {
- dataSets.put(value, set);
- }
- }
+ public void setOutputDataSet(PValue value, DataSet<?> set) {
+ if (!dataSets.containsKey(value)) {
+ dataSets.put(value, set);
+ }
+ }
- /**
- * Sets the AppliedPTransform which carries input/output.
- * @param currentTransform
- */
- public void setCurrentTransform(AppliedPTransform<?, ?, ?> currentTransform) {
- this.currentTransform = currentTransform;
- }
+ /**
+ * Sets the AppliedPTransform which carries input/output.
+ * @param currentTransform
+ */
+ public void setCurrentTransform(AppliedPTransform<?, ?, ?> currentTransform) {
+ this.currentTransform = currentTransform;
+ }
- @SuppressWarnings("unchecked")
- public <T> DataSet<T> getSideInputDataSet(PCollectionView<?> value) {
- return (DataSet<T>) broadcastDataSets.get(value);
- }
+ @SuppressWarnings("unchecked")
+ public <T> DataSet<T> getSideInputDataSet(PCollectionView<?> value) {
+ return (DataSet<T>) broadcastDataSets.get(value);
+ }
- public void setSideInputDataSet(PCollectionView<?> value, DataSet<?> set) {
- if (!broadcastDataSets.containsKey(value)) {
- broadcastDataSets.put(value, set);
- }
- }
-
- @SuppressWarnings("unchecked")
- public <T> TypeInformation<T> getTypeInfo(PInput output) {
- if (output instanceof TypedPValue) {
- Coder<?> outputCoder = ((TypedPValue) output).getCoder();
- if (outputCoder instanceof KvCoder) {
- return new KvCoderTypeInformation((KvCoder) outputCoder);
- } else {
- return new CoderTypeInformation(outputCoder);
- }
- }
- return new GenericTypeInfo<>((Class<T>)Object.class);
- }
+ public void setSideInputDataSet(PCollectionView<?> value, DataSet<?> set) {
+ if (!broadcastDataSets.containsKey(value)) {
+ broadcastDataSets.put(value, set);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ public <T> TypeInformation<T> getTypeInfo(PInput output) {
+ if (output instanceof TypedPValue) {
+ Coder<?> outputCoder = ((TypedPValue) output).getCoder();
+ if (outputCoder instanceof KvCoder) {
+ return new KvCoderTypeInformation((KvCoder) outputCoder);
+ } else {
+ return new CoderTypeInformation(outputCoder);
+ }
+ }
+ return new GenericTypeInfo<>((Class<T>)Object.class);
+ }
- public <T> TypeInformation<T> getInputTypeInfo() {
- return getTypeInfo(currentTransform.getInput());
- }
+ public <T> TypeInformation<T> getInputTypeInfo() {
+ return getTypeInfo(currentTransform.getInput());
+ }
- public <T> TypeInformation<T> getOutputTypeInfo() {
- return getTypeInfo((PValue) currentTransform.getOutput());
- }
+ public <T> TypeInformation<T> getOutputTypeInfo() {
+ return getTypeInfo((PValue) currentTransform.getOutput());
+ }
- @SuppressWarnings("unchecked")
- <I extends PInput> I getInput(PTransform<I, ?> transform) {
- return (I) currentTransform.getInput();
- }
+ @SuppressWarnings("unchecked")
+ <I extends PInput> I getInput(PTransform<I, ?> transform) {
+ return (I) currentTransform.getInput();
+ }
- @SuppressWarnings("unchecked")
- <O extends POutput> O getOutput(PTransform<?, O> transform) {
- return (O) currentTransform.getOutput();
- }
+ @SuppressWarnings("unchecked")
+ <O extends POutput> O getOutput(PTransform<?, O> transform) {
+ return (O) currentTransform.getOutput();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java
index b56fe07..a6a333b 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkPipelineTranslator.java
@@ -28,7 +28,7 @@ import com.google.cloud.dataflow.sdk.Pipeline;
*/
public abstract class FlinkPipelineTranslator implements Pipeline.PipelineVisitor {
- public void translate(Pipeline pipeline) {
- pipeline.traverseTopologically(this);
- }
+ public void translate(Pipeline pipeline) {
+ pipeline.traverseTopologically(this);
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
index ea9ed14..897303d 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingPipelineTranslator.java
@@ -31,113 +31,113 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
* */
public class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
- /** The necessary context in the case of a straming job. */
- private final FlinkStreamingTranslationContext streamingContext;
-
- private int depth = 0;
-
- /** Composite transform that we want to translate before proceeding with other transforms. */
- private PTransform<?, ?> currentCompositeTransform;
-
- public FlinkStreamingPipelineTranslator(StreamExecutionEnvironment env, PipelineOptions options) {
- this.streamingContext = new FlinkStreamingTranslationContext(env, options);
- }
-
- // --------------------------------------------------------------------------------------------
- // Pipeline Visitor Methods
- // --------------------------------------------------------------------------------------------
-
- @Override
- public void enterCompositeTransform(TransformTreeNode node) {
- System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node));
-
- PTransform<?, ?> transform = node.getTransform();
- if (transform != null && currentCompositeTransform == null) {
-
- StreamTransformTranslator<?> translator = FlinkStreamingTransformTranslators.getTranslator(transform);
- if (translator != null) {
- currentCompositeTransform = transform;
- }
- }
- this.depth++;
- }
-
- @Override
- public void leaveCompositeTransform(TransformTreeNode node) {
- PTransform<?, ?> transform = node.getTransform();
- if (transform != null && currentCompositeTransform == transform) {
-
- StreamTransformTranslator<?> translator = FlinkStreamingTransformTranslators.getTranslator(transform);
- if (translator != null) {
- System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node));
- applyStreamingTransform(transform, node, translator);
- currentCompositeTransform = null;
- } else {
- throw new IllegalStateException("Attempted to translate composite transform " +
- "but no translator was found: " + currentCompositeTransform);
- }
- }
- this.depth--;
- System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node));
- }
-
- @Override
- public void visitTransform(TransformTreeNode node) {
- System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node));
- if (currentCompositeTransform != null) {
- // ignore it
- return;
- }
-
- // get the transformation corresponding to hte node we are
- // currently visiting and translate it into its Flink alternative.
-
- PTransform<?, ?> transform = node.getTransform();
- StreamTransformTranslator<?> translator = FlinkStreamingTransformTranslators.getTranslator(transform);
- if (translator == null) {
- System.out.println(node.getTransform().getClass());
- throw new UnsupportedOperationException("The transform " + transform + " is currently not supported.");
- }
- applyStreamingTransform(transform, node, translator);
- }
-
- @Override
- public void visitValue(PValue value, TransformTreeNode producer) {
- // do nothing here
- }
-
- private <T extends PTransform<?, ?>> void applyStreamingTransform(PTransform<?, ?> transform, TransformTreeNode node, StreamTransformTranslator<?> translator) {
-
- @SuppressWarnings("unchecked")
- T typedTransform = (T) transform;
-
- @SuppressWarnings("unchecked")
- StreamTransformTranslator<T> typedTranslator = (StreamTransformTranslator<T>) translator;
-
- // create the applied PTransform on the streamingContext
- streamingContext.setCurrentTransform(AppliedPTransform.of(
- node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform));
- typedTranslator.translateNode(typedTransform, streamingContext);
- }
-
- /**
- * The interface that every Flink translator of a Beam operator should implement.
- * This interface is for <b>streaming</b> jobs. For examples of such translators see
- * {@link FlinkStreamingTransformTranslators}.
- */
- public interface StreamTransformTranslator<Type extends PTransform> {
- void translateNode(Type transform, FlinkStreamingTranslationContext context);
- }
-
- private static String genSpaces(int n) {
- String s = "";
- for (int i = 0; i < n; i++) {
- s += "| ";
- }
- return s;
- }
-
- private static String formatNodeName(TransformTreeNode node) {
- return node.toString().split("@")[1] + node.getTransform();
- }
+ /** The necessary context in the case of a straming job. */
+ private final FlinkStreamingTranslationContext streamingContext;
+
+ private int depth = 0;
+
+ /** Composite transform that we want to translate before proceeding with other transforms. */
+ private PTransform<?, ?> currentCompositeTransform;
+
+ public FlinkStreamingPipelineTranslator(StreamExecutionEnvironment env, PipelineOptions options) {
+ this.streamingContext = new FlinkStreamingTranslationContext(env, options);
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Pipeline Visitor Methods
+ // --------------------------------------------------------------------------------------------
+
+ @Override
+ public void enterCompositeTransform(TransformTreeNode node) {
+ System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node));
+
+ PTransform<?, ?> transform = node.getTransform();
+ if (transform != null && currentCompositeTransform == null) {
+
+ StreamTransformTranslator<?> translator = FlinkStreamingTransformTranslators.getTranslator(transform);
+ if (translator != null) {
+ currentCompositeTransform = transform;
+ }
+ }
+ this.depth++;
+ }
+
+ @Override
+ public void leaveCompositeTransform(TransformTreeNode node) {
+ PTransform<?, ?> transform = node.getTransform();
+ if (transform != null && currentCompositeTransform == transform) {
+
+ StreamTransformTranslator<?> translator = FlinkStreamingTransformTranslators.getTranslator(transform);
+ if (translator != null) {
+ System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node));
+ applyStreamingTransform(transform, node, translator);
+ currentCompositeTransform = null;
+ } else {
+ throw new IllegalStateException("Attempted to translate composite transform " +
+ "but no translator was found: " + currentCompositeTransform);
+ }
+ }
+ this.depth--;
+ System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node));
+ }
+
+ @Override
+ public void visitTransform(TransformTreeNode node) {
+ System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node));
+ if (currentCompositeTransform != null) {
+ // ignore it
+ return;
+ }
+
+ // get the transformation corresponding to hte node we are
+ // currently visiting and translate it into its Flink alternative.
+
+ PTransform<?, ?> transform = node.getTransform();
+ StreamTransformTranslator<?> translator = FlinkStreamingTransformTranslators.getTranslator(transform);
+ if (translator == null) {
+ System.out.println(node.getTransform().getClass());
+ throw new UnsupportedOperationException("The transform " + transform + " is currently not supported.");
+ }
+ applyStreamingTransform(transform, node, translator);
+ }
+
+ @Override
+ public void visitValue(PValue value, TransformTreeNode producer) {
+ // do nothing here
+ }
+
+ private <T extends PTransform<?, ?>> void applyStreamingTransform(PTransform<?, ?> transform, TransformTreeNode node, StreamTransformTranslator<?> translator) {
+
+ @SuppressWarnings("unchecked")
+ T typedTransform = (T) transform;
+
+ @SuppressWarnings("unchecked")
+ StreamTransformTranslator<T> typedTranslator = (StreamTransformTranslator<T>) translator;
+
+ // create the applied PTransform on the streamingContext
+ streamingContext.setCurrentTransform(AppliedPTransform.of(
+ node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform));
+ typedTranslator.translateNode(typedTransform, streamingContext);
+ }
+
+ /**
+ * The interface that every Flink translator of a Beam operator should implement.
+ * This interface is for <b>streaming</b> jobs. For examples of such translators see
+ * {@link FlinkStreamingTransformTranslators}.
+ */
+ public interface StreamTransformTranslator<Type extends PTransform> {
+ void translateNode(Type transform, FlinkStreamingTranslationContext context);
+ }
+
+ private static String genSpaces(int n) {
+ String s = "";
+ for (int i = 0; i < n; i++) {
+ s += "| ";
+ }
+ return s;
+ }
+
+ private static String formatNodeName(TransformTreeNode node) {
+ return node.toString().split("@")[1] + node.getTransform();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
index 99dbedb..9fd33be 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
@@ -59,346 +59,346 @@ import java.util.*;
*/
public class FlinkStreamingTransformTranslators {
- // --------------------------------------------------------------------------------------------
- // Transform Translator Registry
- // --------------------------------------------------------------------------------------------
-
- @SuppressWarnings("rawtypes")
- private static final Map<Class<? extends PTransform>, FlinkStreamingPipelineTranslator.StreamTransformTranslator> TRANSLATORS = new HashMap<>();
-
- // here you can find all the available translators.
- static {
- TRANSLATORS.put(Create.Values.class, new CreateStreamingTranslator());
- TRANSLATORS.put(Read.Unbounded.class, new UnboundedReadSourceTranslator());
- TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator());
- TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteBoundStreamingTranslator());
- TRANSLATORS.put(Window.Bound.class, new WindowBoundTranslator());
- TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslator());
- TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslator());
- TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator());
- TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiStreamingTranslator());
- }
-
- public static FlinkStreamingPipelineTranslator.StreamTransformTranslator<?> getTranslator(PTransform<?, ?> transform) {
- return TRANSLATORS.get(transform.getClass());
- }
-
- // --------------------------------------------------------------------------------------------
- // Transformation Implementations
- // --------------------------------------------------------------------------------------------
-
- private static class CreateStreamingTranslator<OUT> implements
- FlinkStreamingPipelineTranslator.StreamTransformTranslator<Create.Values<OUT>> {
-
- @Override
- public void translateNode(Create.Values<OUT> transform, FlinkStreamingTranslationContext context) {
- PCollection<OUT> output = context.getOutput(transform);
- Iterable<OUT> elements = transform.getElements();
-
- // we need to serialize the elements to byte arrays, since they might contain
- // elements that are not serializable by Java serialization. We deserialize them
- // in the FlatMap function using the Coder.
-
- List<byte[]> serializedElements = Lists.newArrayList();
- Coder<OUT> elementCoder = context.getOutput(transform).getCoder();
- for (OUT element: elements) {
- ByteArrayOutputStream bao = new ByteArrayOutputStream();
- try {
- elementCoder.encode(element, bao, Coder.Context.OUTER);
- serializedElements.add(bao.toByteArray());
- } catch (IOException e) {
- throw new RuntimeException("Could not serialize Create elements using Coder: " + e);
- }
- }
-
-
- DataStream<Integer> initDataSet = context.getExecutionEnvironment().fromElements(1);
-
- FlinkStreamingCreateFunction<Integer, OUT> createFunction =
- new FlinkStreamingCreateFunction<>(serializedElements, elementCoder);
-
- WindowedValue.ValueOnlyWindowedValueCoder<OUT> windowCoder = WindowedValue.getValueOnlyCoder(elementCoder);
- TypeInformation<WindowedValue<OUT>> outputType = new CoderTypeInformation<>(windowCoder);
-
- DataStream<WindowedValue<OUT>> outputDataStream = initDataSet.flatMap(createFunction)
- .returns(outputType);
-
- context.setOutputDataStream(context.getOutput(transform), outputDataStream);
- }
- }
-
-
- private static class TextIOWriteBoundStreamingTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<TextIO.Write.Bound<T>> {
- private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteBoundStreamingTranslator.class);
-
- @Override
- public void translateNode(TextIO.Write.Bound<T> transform, FlinkStreamingTranslationContext context) {
- PValue input = context.getInput(transform);
- DataStream<WindowedValue<T>> inputDataStream = context.getInputDataStream(input);
-
- String filenamePrefix = transform.getFilenamePrefix();
- String filenameSuffix = transform.getFilenameSuffix();
- boolean needsValidation = transform.needsValidation();
- int numShards = transform.getNumShards();
- String shardNameTemplate = transform.getShardNameTemplate();
-
- // TODO: Implement these. We need Flink support for this.
- LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation);
- LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix);
- LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
-
- DataStream<String> dataSink = inputDataStream.flatMap(new FlatMapFunction<WindowedValue<T>, String>() {
- @Override
- public void flatMap(WindowedValue<T> value, Collector<String> out) throws Exception {
- out.collect(value.getValue().toString());
- }
- });
- DataStreamSink<String> output = dataSink.writeAsText(filenamePrefix, FileSystem.WriteMode.OVERWRITE);
-
- if (numShards > 0) {
- output.setParallelism(numShards);
- }
- }
- }
-
- private static class UnboundedReadSourceTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Read.Unbounded<T>> {
-
- @Override
- public void translateNode(Read.Unbounded<T> transform, FlinkStreamingTranslationContext context) {
- PCollection<T> output = context.getOutput(transform);
-
- DataStream<WindowedValue<T>> source;
- if (transform.getSource().getClass().equals(UnboundedFlinkSource.class)) {
- UnboundedFlinkSource flinkSource = (UnboundedFlinkSource) transform.getSource();
- source = context.getExecutionEnvironment()
- .addSource(flinkSource.getFlinkSource())
- .flatMap(new FlatMapFunction<String, WindowedValue<String>>() {
- @Override
- public void flatMap(String s, Collector<WindowedValue<String>> collector) throws Exception {
- collector.collect(WindowedValue.<String>of(s, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
- }
- });
- } else {
- source = context.getExecutionEnvironment()
- .addSource(new UnboundedSourceWrapper<>(context.getPipelineOptions(), transform));
- }
- context.setOutputDataStream(output, source);
- }
- }
-
- private static class ParDoBoundStreamingTranslator<IN, OUT> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<ParDo.Bound<IN, OUT>> {
-
- @Override
- public void translateNode(ParDo.Bound<IN, OUT> transform, FlinkStreamingTranslationContext context) {
- PCollection<OUT> output = context.getOutput(transform);
-
- final WindowingStrategy<OUT, ? extends BoundedWindow> windowingStrategy =
- (WindowingStrategy<OUT, ? extends BoundedWindow>)
- context.getOutput(transform).getWindowingStrategy();
-
- WindowedValue.WindowedValueCoder<OUT> outputStreamCoder = WindowedValue.getFullCoder(output.getCoder(),
- windowingStrategy.getWindowFn().windowCoder());
- CoderTypeInformation<WindowedValue<OUT>> outputWindowedValueCoder =
- new CoderTypeInformation<>(outputStreamCoder);
-
- FlinkParDoBoundWrapper<IN, OUT> doFnWrapper = new FlinkParDoBoundWrapper<>(
- context.getPipelineOptions(), windowingStrategy, transform.getFn());
- DataStream<WindowedValue<IN>> inputDataStream = context.getInputDataStream(context.getInput(transform));
- SingleOutputStreamOperator<WindowedValue<OUT>> outDataStream = inputDataStream.flatMap(doFnWrapper)
- .returns(outputWindowedValueCoder);
-
- context.setOutputDataStream(context.getOutput(transform), outDataStream);
- }
- }
-
- public static class WindowBoundTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Window.Bound<T>> {
-
- @Override
- public void translateNode(Window.Bound<T> transform, FlinkStreamingTranslationContext context) {
- PValue input = context.getInput(transform);
- DataStream<WindowedValue<T>> inputDataStream = context.getInputDataStream(input);
-
- final WindowingStrategy<T, ? extends BoundedWindow> windowingStrategy =
- (WindowingStrategy<T, ? extends BoundedWindow>)
- context.getOutput(transform).getWindowingStrategy();
-
- final WindowFn<T, ? extends BoundedWindow> windowFn = windowingStrategy.getWindowFn();
-
- WindowedValue.WindowedValueCoder<T> outputStreamCoder = WindowedValue.getFullCoder(
- context.getInput(transform).getCoder(), windowingStrategy.getWindowFn().windowCoder());
- CoderTypeInformation<WindowedValue<T>> outputWindowedValueCoder =
- new CoderTypeInformation<>(outputStreamCoder);
-
- final FlinkParDoBoundWrapper<T, T> windowDoFnAssigner = new FlinkParDoBoundWrapper<>(
- context.getPipelineOptions(), windowingStrategy, createWindowAssigner(windowFn));
-
- SingleOutputStreamOperator<WindowedValue<T>> windowedStream =
- inputDataStream.flatMap(windowDoFnAssigner).returns(outputWindowedValueCoder);
- context.setOutputDataStream(context.getOutput(transform), windowedStream);
- }
-
- private static <T, W extends BoundedWindow> DoFn<T, T> createWindowAssigner(final WindowFn<T, W> windowFn) {
- return new DoFn<T, T>() {
-
- @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 Collection<? extends BoundedWindow> windows() {
- return c.windowingInternals().windows();
- }
- });
-
- c.windowingInternals().outputWindowedValue(
- c.element(), c.timestamp(), windows, c.pane());
- }
- };
- }
- }
-
- public static class GroupByKeyTranslator<K, V> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<GroupByKey<K, V>> {
-
- @Override
- public void translateNode(GroupByKey<K, V> transform, FlinkStreamingTranslationContext context) {
- PValue input = context.getInput(transform);
-
- DataStream<WindowedValue<KV<K, V>>> inputDataStream = context.getInputDataStream(input);
- KvCoder<K, V> inputKvCoder = (KvCoder<K, V>) context.getInput(transform).getCoder();
-
- KeyedStream<WindowedValue<KV<K, V>>, K> groupByKStream = FlinkGroupByKeyWrapper
- .groupStreamByKey(inputDataStream, inputKvCoder);
-
- DataStream<WindowedValue<KV<K, Iterable<V>>>> groupedByKNWstream =
- FlinkGroupAlsoByWindowWrapper.createForIterable(context.getPipelineOptions(),
- context.getInput(transform), groupByKStream);
-
- context.setOutputDataStream(context.getOutput(transform), groupedByKNWstream);
- }
- }
-
- public static class CombinePerKeyTranslator<K, VIN, VACC, VOUT> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Combine.PerKey<K, VIN, VOUT>> {
-
- @Override
- public void translateNode(Combine.PerKey<K, VIN, VOUT> transform, FlinkStreamingTranslationContext context) {
- PValue input = context.getInput(transform);
-
- DataStream<WindowedValue<KV<K, VIN>>> inputDataStream = context.getInputDataStream(input);
- KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) context.getInput(transform).getCoder();
- KvCoder<K, VOUT> outputKvCoder = (KvCoder<K, VOUT>) context.getOutput(transform).getCoder();
-
- KeyedStream<WindowedValue<KV<K, VIN>>, K> groupByKStream = FlinkGroupByKeyWrapper
- .groupStreamByKey(inputDataStream, inputKvCoder);
-
- Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combineFn = (Combine.KeyedCombineFn<K, VIN, VACC, VOUT>) transform.getFn();
- DataStream<WindowedValue<KV<K, VOUT>>> groupedByKNWstream =
- FlinkGroupAlsoByWindowWrapper.create(context.getPipelineOptions(),
- context.getInput(transform), groupByKStream, combineFn, outputKvCoder);
-
- context.setOutputDataStream(context.getOutput(transform), groupedByKNWstream);
- }
- }
-
- public static class FlattenPCollectionTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Flatten.FlattenPCollectionList<T>> {
-
- @Override
- public void translateNode(Flatten.FlattenPCollectionList<T> transform, FlinkStreamingTranslationContext context) {
- List<PCollection<T>> allInputs = context.getInput(transform).getAll();
- DataStream<T> result = null;
- for (PCollection<T> collection : allInputs) {
- DataStream<T> current = context.getInputDataStream(collection);
- result = (result == null) ? current : result.union(current);
- }
- context.setOutputDataStream(context.getOutput(transform), result);
- }
- }
-
- public static class ParDoBoundMultiStreamingTranslator<IN, OUT> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<ParDo.BoundMulti<IN, OUT>> {
-
- private final int MAIN_TAG_INDEX = 0;
-
- @Override
- public void translateNode(ParDo.BoundMulti<IN, OUT> transform, FlinkStreamingTranslationContext context) {
-
- // we assume that the transformation does not change the windowing strategy.
- WindowingStrategy<?, ? extends BoundedWindow> windowingStrategy = context.getInput(transform).getWindowingStrategy();
-
- Map<TupleTag<?>, PCollection<?>> outputs = context.getOutput(transform).getAll();
- Map<TupleTag<?>, Integer> tagsToLabels = transformTupleTagsToLabels(
- transform.getMainOutputTag(), outputs.keySet());
-
- UnionCoder intermUnionCoder = getIntermUnionCoder(outputs.values());
- WindowedValue.WindowedValueCoder<RawUnionValue> outputStreamCoder = WindowedValue.getFullCoder(
- intermUnionCoder, windowingStrategy.getWindowFn().windowCoder());
-
- CoderTypeInformation<WindowedValue<RawUnionValue>> intermWindowedValueCoder =
- new CoderTypeInformation<>(outputStreamCoder);
-
- FlinkParDoBoundMultiWrapper<IN, OUT> doFnWrapper = new FlinkParDoBoundMultiWrapper<>(
- context.getPipelineOptions(), windowingStrategy, transform.getFn(),
- transform.getMainOutputTag(), tagsToLabels);
-
- DataStream<WindowedValue<IN>> inputDataStream = context.getInputDataStream(context.getInput(transform));
- SingleOutputStreamOperator<WindowedValue<RawUnionValue>> intermDataStream =
- inputDataStream.flatMap(doFnWrapper).returns(intermWindowedValueCoder);
-
- for (Map.Entry<TupleTag<?>, PCollection<?>> output : outputs.entrySet()) {
- final int outputTag = tagsToLabels.get(output.getKey());
-
- WindowedValue.WindowedValueCoder<?> coderForTag = WindowedValue.getFullCoder(
- output.getValue().getCoder(),
- windowingStrategy.getWindowFn().windowCoder());
-
- CoderTypeInformation<WindowedValue<?>> windowedValueCoder =
- new CoderTypeInformation(coderForTag);
-
- context.setOutputDataStream(output.getValue(),
- intermDataStream.filter(new FilterFunction<WindowedValue<RawUnionValue>>() {
- @Override
- public boolean filter(WindowedValue<RawUnionValue> value) throws Exception {
- return value.getValue().getUnionTag() == outputTag;
- }
- }).flatMap(new FlatMapFunction<WindowedValue<RawUnionValue>, WindowedValue<?>>() {
- @Override
- public void flatMap(WindowedValue<RawUnionValue> value, Collector<WindowedValue<?>> collector) throws Exception {
- collector.collect(WindowedValue.of(
- value.getValue().getValue(),
- value.getTimestamp(),
- value.getWindows(),
- value.getPane()));
- }
- }).returns(windowedValueCoder));
- }
- }
-
- private Map<TupleTag<?>, Integer> transformTupleTagsToLabels(TupleTag<?> mainTag, Set<TupleTag<?>> secondaryTags) {
- Map<TupleTag<?>, Integer> tagToLabelMap = Maps.newHashMap();
- tagToLabelMap.put(mainTag, MAIN_TAG_INDEX);
- int count = MAIN_TAG_INDEX + 1;
- for (TupleTag<?> tag : secondaryTags) {
- if (!tagToLabelMap.containsKey(tag)) {
- tagToLabelMap.put(tag, count++);
- }
- }
- return tagToLabelMap;
- }
-
- private UnionCoder getIntermUnionCoder(Collection<PCollection<?>> taggedCollections) {
- List<Coder<?>> outputCoders = Lists.newArrayList();
- for (PCollection<?> coll : taggedCollections) {
- outputCoders.add(coll.getCoder());
- }
- return UnionCoder.of(outputCoders);
- }
- }
+ // --------------------------------------------------------------------------------------------
+ // Transform Translator Registry
+ // --------------------------------------------------------------------------------------------
+
+ @SuppressWarnings("rawtypes")
+ private static final Map<Class<? extends PTransform>, FlinkStreamingPipelineTranslator.StreamTransformTranslator> TRANSLATORS = new HashMap<>();
+
+ // here you can find all the available translators.
+ static {
+ TRANSLATORS.put(Create.Values.class, new CreateStreamingTranslator());
+ TRANSLATORS.put(Read.Unbounded.class, new UnboundedReadSourceTranslator());
+ TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator());
+ TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteBoundStreamingTranslator());
+ TRANSLATORS.put(Window.Bound.class, new WindowBoundTranslator());
+ TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslator());
+ TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslator());
+ TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator());
+ TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiStreamingTranslator());
+ }
+
+ public static FlinkStreamingPipelineTranslator.StreamTransformTranslator<?> getTranslator(PTransform<?, ?> transform) {
+ return TRANSLATORS.get(transform.getClass());
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Transformation Implementations
+ // --------------------------------------------------------------------------------------------
+
+ private static class CreateStreamingTranslator<OUT> implements
+ FlinkStreamingPipelineTranslator.StreamTransformTranslator<Create.Values<OUT>> {
+
+ @Override
+ public void translateNode(Create.Values<OUT> transform, FlinkStreamingTranslationContext context) {
+ PCollection<OUT> output = context.getOutput(transform);
+ Iterable<OUT> elements = transform.getElements();
+
+ // we need to serialize the elements to byte arrays, since they might contain
+ // elements that are not serializable by Java serialization. We deserialize them
+ // in the FlatMap function using the Coder.
+
+ List<byte[]> serializedElements = Lists.newArrayList();
+ Coder<OUT> elementCoder = context.getOutput(transform).getCoder();
+ for (OUT element: elements) {
+ ByteArrayOutputStream bao = new ByteArrayOutputStream();
+ try {
+ elementCoder.encode(element, bao, Coder.Context.OUTER);
+ serializedElements.add(bao.toByteArray());
+ } catch (IOException e) {
+ throw new RuntimeException("Could not serialize Create elements using Coder: " + e);
+ }
+ }
+
+
+ DataStream<Integer> initDataSet = context.getExecutionEnvironment().fromElements(1);
+
+ FlinkStreamingCreateFunction<Integer, OUT> createFunction =
+ new FlinkStreamingCreateFunction<>(serializedElements, elementCoder);
+
+ WindowedValue.ValueOnlyWindowedValueCoder<OUT> windowCoder = WindowedValue.getValueOnlyCoder(elementCoder);
+ TypeInformation<WindowedValue<OUT>> outputType = new CoderTypeInformation<>(windowCoder);
+
+ DataStream<WindowedValue<OUT>> outputDataStream = initDataSet.flatMap(createFunction)
+ .returns(outputType);
+
+ context.setOutputDataStream(context.getOutput(transform), outputDataStream);
+ }
+ }
+
+
+ private static class TextIOWriteBoundStreamingTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<TextIO.Write.Bound<T>> {
+ private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteBoundStreamingTranslator.class);
+
+ @Override
+ public void translateNode(TextIO.Write.Bound<T> transform, FlinkStreamingTranslationContext context) {
+ PValue input = context.getInput(transform);
+ DataStream<WindowedValue<T>> inputDataStream = context.getInputDataStream(input);
+
+ String filenamePrefix = transform.getFilenamePrefix();
+ String filenameSuffix = transform.getFilenameSuffix();
+ boolean needsValidation = transform.needsValidation();
+ int numShards = transform.getNumShards();
+ String shardNameTemplate = transform.getShardNameTemplate();
+
+ // TODO: Implement these. We need Flink support for this.
+ LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation);
+ LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix);
+ LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
+
+ DataStream<String> dataSink = inputDataStream.flatMap(new FlatMapFunction<WindowedValue<T>, String>() {
+ @Override
+ public void flatMap(WindowedValue<T> value, Collector<String> out) throws Exception {
+ out.collect(value.getValue().toString());
+ }
+ });
+ DataStreamSink<String> output = dataSink.writeAsText(filenamePrefix, FileSystem.WriteMode.OVERWRITE);
+
+ if (numShards > 0) {
+ output.setParallelism(numShards);
+ }
+ }
+ }
+
+ private static class UnboundedReadSourceTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Read.Unbounded<T>> {
+
+ @Override
+ public void translateNode(Read.Unbounded<T> transform, FlinkStreamingTranslationContext context) {
+ PCollection<T> output = context.getOutput(transform);
+
+ DataStream<WindowedValue<T>> source;
+ if (transform.getSource().getClass().equals(UnboundedFlinkSource.class)) {
+ UnboundedFlinkSource flinkSource = (UnboundedFlinkSource) transform.getSource();
+ source = context.getExecutionEnvironment()
+ .addSource(flinkSource.getFlinkSource())
+ .flatMap(new FlatMapFunction<String, WindowedValue<String>>() {
+ @Override
+ public void flatMap(String s, Collector<WindowedValue<String>> collector) throws Exception {
+ collector.collect(WindowedValue.<String>of(s, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
+ }
+ });
+ } else {
+ source = context.getExecutionEnvironment()
+ .addSource(new UnboundedSourceWrapper<>(context.getPipelineOptions(), transform));
+ }
+ context.setOutputDataStream(output, source);
+ }
+ }
+
+ private static class ParDoBoundStreamingTranslator<IN, OUT> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<ParDo.Bound<IN, OUT>> {
+
+ @Override
+ public void translateNode(ParDo.Bound<IN, OUT> transform, FlinkStreamingTranslationContext context) {
+ PCollection<OUT> output = context.getOutput(transform);
+
+ final WindowingStrategy<OUT, ? extends BoundedWindow> windowingStrategy =
+ (WindowingStrategy<OUT, ? extends BoundedWindow>)
+ context.getOutput(transform).getWindowingStrategy();
+
+ WindowedValue.WindowedValueCoder<OUT> outputStreamCoder = WindowedValue.getFullCoder(output.getCoder(),
+ windowingStrategy.getWindowFn().windowCoder());
+ CoderTypeInformation<WindowedValue<OUT>> outputWindowedValueCoder =
+ new CoderTypeInformation<>(outputStreamCoder);
+
+ FlinkParDoBoundWrapper<IN, OUT> doFnWrapper = new FlinkParDoBoundWrapper<>(
+ context.getPipelineOptions(), windowingStrategy, transform.getFn());
+ DataStream<WindowedValue<IN>> inputDataStream = context.getInputDataStream(context.getInput(transform));
+ SingleOutputStreamOperator<WindowedValue<OUT>> outDataStream = inputDataStream.flatMap(doFnWrapper)
+ .returns(outputWindowedValueCoder);
+
+ context.setOutputDataStream(context.getOutput(transform), outDataStream);
+ }
+ }
+
+ public static class WindowBoundTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Window.Bound<T>> {
+
+ @Override
+ public void translateNode(Window.Bound<T> transform, FlinkStreamingTranslationContext context) {
+ PValue input = context.getInput(transform);
+ DataStream<WindowedValue<T>> inputDataStream = context.getInputDataStream(input);
+
+ final WindowingStrategy<T, ? extends BoundedWindow> windowingStrategy =
+ (WindowingStrategy<T, ? extends BoundedWindow>)
+ context.getOutput(transform).getWindowingStrategy();
+
+ final WindowFn<T, ? extends BoundedWindow> windowFn = windowingStrategy.getWindowFn();
+
+ WindowedValue.WindowedValueCoder<T> outputStreamCoder = WindowedValue.getFullCoder(
+ context.getInput(transform).getCoder(), windowingStrategy.getWindowFn().windowCoder());
+ CoderTypeInformation<WindowedValue<T>> outputWindowedValueCoder =
+ new CoderTypeInformation<>(outputStreamCoder);
+
+ final FlinkParDoBoundWrapper<T, T> windowDoFnAssigner = new FlinkParDoBoundWrapper<>(
+ context.getPipelineOptions(), windowingStrategy, createWindowAssigner(windowFn));
+
+ SingleOutputStreamOperator<WindowedValue<T>> windowedStream =
+ inputDataStream.flatMap(windowDoFnAssigner).returns(outputWindowedValueCoder);
+ context.setOutputDataStream(context.getOutput(transform), windowedStream);
+ }
+
+ private static <T, W extends BoundedWindow> DoFn<T, T> createWindowAssigner(final WindowFn<T, W> windowFn) {
+ return new DoFn<T, T>() {
+
+ @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 Collection<? extends BoundedWindow> windows() {
+ return c.windowingInternals().windows();
+ }
+ });
+
+ c.windowingInternals().outputWindowedValue(
+ c.element(), c.timestamp(), windows, c.pane());
+ }
+ };
+ }
+ }
+
+ public static class GroupByKeyTranslator<K, V> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<GroupByKey<K, V>> {
+
+ @Override
+ public void translateNode(GroupByKey<K, V> transform, FlinkStreamingTranslationContext context) {
+ PValue input = context.getInput(transform);
+
+ DataStream<WindowedValue<KV<K, V>>> inputDataStream = context.getInputDataStream(input);
+ KvCoder<K, V> inputKvCoder = (KvCoder<K, V>) context.getInput(transform).getCoder();
+
+ KeyedStream<WindowedValue<KV<K, V>>, K> groupByKStream = FlinkGroupByKeyWrapper
+ .groupStreamByKey(inputDataStream, inputKvCoder);
+
+ DataStream<WindowedValue<KV<K, Iterable<V>>>> groupedByKNWstream =
+ FlinkGroupAlsoByWindowWrapper.createForIterable(context.getPipelineOptions(),
+ context.getInput(transform), groupByKStream);
+
+ context.setOutputDataStream(context.getOutput(transform), groupedByKNWstream);
+ }
+ }
+
+ public static class CombinePerKeyTranslator<K, VIN, VACC, VOUT> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Combine.PerKey<K, VIN, VOUT>> {
+
+ @Override
+ public void translateNode(Combine.PerKey<K, VIN, VOUT> transform, FlinkStreamingTranslationContext context) {
+ PValue input = context.getInput(transform);
+
+ DataStream<WindowedValue<KV<K, VIN>>> inputDataStream = context.getInputDataStream(input);
+ KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) context.getInput(transform).getCoder();
+ KvCoder<K, VOUT> outputKvCoder = (KvCoder<K, VOUT>) context.getOutput(transform).getCoder();
+
+ KeyedStream<WindowedValue<KV<K, VIN>>, K> groupByKStream = FlinkGroupByKeyWrapper
+ .groupStreamByKey(inputDataStream, inputKvCoder);
+
+ Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combineFn = (Combine.KeyedCombineFn<K, VIN, VACC, VOUT>) transform.getFn();
+ DataStream<WindowedValue<KV<K, VOUT>>> groupedByKNWstream =
+ FlinkGroupAlsoByWindowWrapper.create(context.getPipelineOptions(),
+ context.getInput(transform), groupByKStream, combineFn, outputKvCoder);
+
+ context.setOutputDataStream(context.getOutput(transform), groupedByKNWstream);
+ }
+ }
+
+ public static class FlattenPCollectionTranslator<T> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<Flatten.FlattenPCollectionList<T>> {
+
+ @Override
+ public void translateNode(Flatten.FlattenPCollectionList<T> transform, FlinkStreamingTranslationContext context) {
+ List<PCollection<T>> allInputs = context.getInput(transform).getAll();
+ DataStream<T> result = null;
+ for (PCollection<T> collection : allInputs) {
+ DataStream<T> current = context.getInputDataStream(collection);
+ result = (result == null) ? current : result.union(current);
+ }
+ context.setOutputDataStream(context.getOutput(transform), result);
+ }
+ }
+
+ public static class ParDoBoundMultiStreamingTranslator<IN, OUT> implements FlinkStreamingPipelineTranslator.StreamTransformTranslator<ParDo.BoundMulti<IN, OUT>> {
+
+ private final int MAIN_TAG_INDEX = 0;
+
+ @Override
+ public void translateNode(ParDo.BoundMulti<IN, OUT> transform, FlinkStreamingTranslationContext context) {
+
+ // we assume that the transformation does not change the windowing strategy.
+ WindowingStrategy<?, ? extends BoundedWindow> windowingStrategy = context.getInput(transform).getWindowingStrategy();
+
+ Map<TupleTag<?>, PCollection<?>> outputs = context.getOutput(transform).getAll();
+ Map<TupleTag<?>, Integer> tagsToLabels = transformTupleTagsToLabels(
+ transform.getMainOutputTag(), outputs.keySet());
+
+ UnionCoder intermUnionCoder = getIntermUnionCoder(outputs.values());
+ WindowedValue.WindowedValueCoder<RawUnionValue> outputStreamCoder = WindowedValue.getFullCoder(
+ intermUnionCoder, windowingStrategy.getWindowFn().windowCoder());
+
+ CoderTypeInformation<WindowedValue<RawUnionValue>> intermWindowedValueCoder =
+ new CoderTypeInformation<>(outputStreamCoder);
+
+ FlinkParDoBoundMultiWrapper<IN, OUT> doFnWrapper = new FlinkParDoBoundMultiWrapper<>(
+ context.getPipelineOptions(), windowingStrategy, transform.getFn(),
+ transform.getMainOutputTag(), tagsToLabels);
+
+ DataStream<WindowedValue<IN>> inputDataStream = context.getInputDataStream(context.getInput(transform));
+ SingleOutputStreamOperator<WindowedValue<RawUnionValue>> intermDataStream =
+ inputDataStream.flatMap(doFnWrapper).returns(intermWindowedValueCoder);
+
+ for (Map.Entry<TupleTag<?>, PCollection<?>> output : outputs.entrySet()) {
+ final int outputTag = tagsToLabels.get(output.getKey());
+
+ WindowedValue.WindowedValueCoder<?> coderForTag = WindowedValue.getFullCoder(
+ output.getValue().getCoder(),
+ windowingStrategy.getWindowFn().windowCoder());
+
+ CoderTypeInformation<WindowedValue<?>> windowedValueCoder =
+ new CoderTypeInformation(coderForTag);
+
+ context.setOutputDataStream(output.getValue(),
+ intermDataStream.filter(new FilterFunction<WindowedValue<RawUnionValue>>() {
+ @Override
+ public boolean filter(WindowedValue<RawUnionValue> value) throws Exception {
+ return value.getValue().getUnionTag() == outputTag;
+ }
+ }).flatMap(new FlatMapFunction<WindowedValue<RawUnionValue>, WindowedValue<?>>() {
+ @Override
+ public void flatMap(WindowedValue<RawUnionValue> value, Collector<WindowedValue<?>> collector) throws Exception {
+ collector.collect(WindowedValue.of(
+ value.getValue().getValue(),
+ value.getTimestamp(),
+ value.getWindows(),
+ value.getPane()));
+ }
+ }).returns(windowedValueCoder));
+ }
+ }
+
+ private Map<TupleTag<?>, Integer> transformTupleTagsToLabels(TupleTag<?> mainTag, Set<TupleTag<?>> secondaryTags) {
+ Map<TupleTag<?>, Integer> tagToLabelMap = Maps.newHashMap();
+ tagToLabelMap.put(mainTag, MAIN_TAG_INDEX);
+ int count = MAIN_TAG_INDEX + 1;
+ for (TupleTag<?> tag : secondaryTags) {
+ if (!tagToLabelMap.containsKey(tag)) {
+ tagToLabelMap.put(tag, count++);
+ }
+ }
+ return tagToLabelMap;
+ }
+
+ private UnionCoder getIntermUnionCoder(Collection<PCollection<?>> taggedCollections) {
+ List<Coder<?>> outputCoders = Lists.newArrayList();
+ for (PCollection<?> coll : taggedCollections) {
+ outputCoders.add(coll.getCoder());
+ }
+ return UnionCoder.of(outputCoders);
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
index 7c4ab93..3586d0c 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
@@ -30,58 +30,58 @@ import java.util.Map;
public class FlinkStreamingTranslationContext {
- private final StreamExecutionEnvironment env;
- private final PipelineOptions options;
+ private final StreamExecutionEnvironment env;
+ private final PipelineOptions options;
- /**
- * Keeps a mapping between the output value of the PTransform (in Dataflow) and the
- * Flink Operator that produced it, after the translation of the correspondinf PTransform
- * to its Flink equivalent.
- * */
- private final Map<PValue, DataStream<?>> dataStreams;
+ /**
+ * Keeps a mapping between the output value of the PTransform (in Dataflow) and the
+ * Flink Operator that produced it, after the translation of the correspondinf PTransform
+ * to its Flink equivalent.
+ * */
+ private final Map<PValue, DataStream<?>> dataStreams;
- private AppliedPTransform<?, ?, ?> currentTransform;
+ private AppliedPTransform<?, ?, ?> currentTransform;
- public FlinkStreamingTranslationContext(StreamExecutionEnvironment env, PipelineOptions options) {
- this.env = Preconditions.checkNotNull(env);
- this.options = Preconditions.checkNotNull(options);
- this.dataStreams = new HashMap<>();
- }
+ public FlinkStreamingTranslationContext(StreamExecutionEnvironment env, PipelineOptions options) {
+ this.env = Preconditions.checkNotNull(env);
+ this.options = Preconditions.checkNotNull(options);
+ this.dataStreams = new HashMap<>();
+ }
- public StreamExecutionEnvironment getExecutionEnvironment() {
- return env;
- }
+ public StreamExecutionEnvironment getExecutionEnvironment() {
+ return env;
+ }
- public PipelineOptions getPipelineOptions() {
- return options;
- }
+ public PipelineOptions getPipelineOptions() {
+ return options;
+ }
- @SuppressWarnings("unchecked")
- public <T> DataStream<T> getInputDataStream(PValue value) {
- return (DataStream<T>) dataStreams.get(value);
- }
+ @SuppressWarnings("unchecked")
+ public <T> DataStream<T> getInputDataStream(PValue value) {
+ return (DataStream<T>) dataStreams.get(value);
+ }
- public void setOutputDataStream(PValue value, DataStream<?> set) {
- if (!dataStreams.containsKey(value)) {
- dataStreams.put(value, set);
- }
- }
+ public void setOutputDataStream(PValue value, DataStream<?> set) {
+ if (!dataStreams.containsKey(value)) {
+ dataStreams.put(value, set);
+ }
+ }
- /**
- * Sets the AppliedPTransform which carries input/output.
- * @param currentTransform
- */
- public void setCurrentTransform(AppliedPTransform<?, ?, ?> currentTransform) {
- this.currentTransform = currentTransform;
- }
+ /**
+ * Sets the AppliedPTransform which carries input/output.
+ * @param currentTransform
+ */
+ public void setCurrentTransform(AppliedPTransform<?, ?, ?> currentTransform) {
+ this.currentTransform = currentTransform;
+ }
- @SuppressWarnings("unchecked")
- public <I extends PInput> I getInput(PTransform<I, ?> transform) {
- return (I) currentTransform.getInput();
- }
+ @SuppressWarnings("unchecked")
+ public <I extends PInput> I getInput(PTransform<I, ?> transform) {
+ return (I) currentTransform.getInput();
+ }
- @SuppressWarnings("unchecked")
- public <O extends POutput> O getOutput(PTransform<?, O> transform) {
- return (O) currentTransform.getOutput();
- }
+ @SuppressWarnings("unchecked")
+ public <O extends POutput> O getOutput(PTransform<?, O> transform) {
+ return (O) currentTransform.getOutput();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java
index 4c7fefd..5897473 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCoGroupKeyedListAggregator.java
@@ -29,30 +29,30 @@ import java.util.List;
public class FlinkCoGroupKeyedListAggregator<K,V1,V2> implements CoGroupFunction<KV<K,V1>, KV<K,V2>, KV<K, CoGbkResult>>{
- private CoGbkResultSchema schema;
- private TupleTag<?> tupleTag1;
- private TupleTag<?> tupleTag2;
+ private CoGbkResultSchema schema;
+ private TupleTag<?> tupleTag1;
+ private TupleTag<?> tupleTag2;
- public FlinkCoGroupKeyedListAggregator(CoGbkResultSchema schema, TupleTag<?> tupleTag1, TupleTag<?> tupleTag2) {
- this.schema = schema;
- this.tupleTag1 = tupleTag1;
- this.tupleTag2 = tupleTag2;
- }
+ public FlinkCoGroupKeyedListAggregator(CoGbkResultSchema schema, TupleTag<?> tupleTag1, TupleTag<?> tupleTag2) {
+ this.schema = schema;
+ this.tupleTag1 = tupleTag1;
+ this.tupleTag2 = tupleTag2;
+ }
- @Override
- public void coGroup(Iterable<KV<K,V1>> first, Iterable<KV<K,V2>> second, Collector<KV<K, CoGbkResult>> out) throws Exception {
- K k = null;
- List<RawUnionValue> result = new ArrayList<>();
- int index1 = schema.getIndex(tupleTag1);
- for (KV<K,?> entry : first) {
- k = entry.getKey();
- result.add(new RawUnionValue(index1, entry.getValue()));
- }
- int index2 = schema.getIndex(tupleTag2);
- for (KV<K,?> entry : second) {
- k = entry.getKey();
- result.add(new RawUnionValue(index2, entry.getValue()));
- }
- out.collect(KV.of(k, new CoGbkResult(schema, result)));
- }
+ @Override
+ public void coGroup(Iterable<KV<K,V1>> first, Iterable<KV<K,V2>> second, Collector<KV<K, CoGbkResult>> out) throws Exception {
+ K k = null;
+ List<RawUnionValue> result = new ArrayList<>();
+ int index1 = schema.getIndex(tupleTag1);
+ for (KV<K,?> entry : first) {
+ k = entry.getKey();
+ result.add(new RawUnionValue(index1, entry.getValue()));
+ }
+ int index2 = schema.getIndex(tupleTag2);
+ for (KV<K,?> entry : second) {
+ k = entry.getKey();
+ result.add(new RawUnionValue(index2, entry.getValue()));
+ }
+ out.collect(KV.of(k, new CoGbkResult(schema, result)));
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java
index 21ecaf0..03f2b06 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkCreateFunction.java
@@ -32,29 +32,29 @@ import java.util.List;
*/
public class FlinkCreateFunction<IN, OUT> implements FlatMapFunction<IN, OUT> {
- private final List<byte[]> elements;
- private final Coder<OUT> coder;
-
- public FlinkCreateFunction(List<byte[]> elements, Coder<OUT> coder) {
- this.elements = elements;
- this.coder = coder;
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public void flatMap(IN value, Collector<OUT> out) throws Exception {
-
- for (byte[] element : elements) {
- ByteArrayInputStream bai = new ByteArrayInputStream(element);
- OUT outValue = coder.decode(bai, Coder.Context.OUTER);
- if (outValue == null) {
- // TODO Flink doesn't allow null values in records
- out.collect((OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE);
- } else {
- out.collect(outValue);
- }
- }
-
- out.close();
- }
+ private final List<byte[]> elements;
+ private final Coder<OUT> coder;
+
+ public FlinkCreateFunction(List<byte[]> elements, Coder<OUT> coder) {
+ this.elements = elements;
+ this.coder = coder;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void flatMap(IN value, Collector<OUT> out) throws Exception {
+
+ for (byte[] element : elements) {
+ ByteArrayInputStream bai = new ByteArrayInputStream(element);
+ OUT outValue = coder.decode(bai, Coder.Context.OUTER);
+ if (outValue == null) {
+ // TODO Flink doesn't allow null values in records
+ out.collect((OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE);
+ } else {
+ out.collect(outValue);
+ }
+ }
+
+ out.close();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java
index 9c57d4e..53ff1cf 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkDoFnFunction.java
@@ -49,154 +49,154 @@ import java.util.List;
*/
public class FlinkDoFnFunction<IN, OUT> extends RichMapPartitionFunction<IN, OUT> {
- private final DoFn<IN, OUT> doFn;
- private transient PipelineOptions options;
-
- public FlinkDoFnFunction(DoFn<IN, OUT> doFn, PipelineOptions options) {
- this.doFn = doFn;
- this.options = options;
- }
-
- private void writeObject(ObjectOutputStream out)
- throws IOException, ClassNotFoundException {
- out.defaultWriteObject();
- ObjectMapper mapper = new ObjectMapper();
- mapper.writeValue(out, options);
- }
-
- private void readObject(ObjectInputStream in)
- throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- ObjectMapper mapper = new ObjectMapper();
- options = mapper.readValue(in, PipelineOptions.class);
- }
-
- @Override
- public void mapPartition(Iterable<IN> values, Collector<OUT> out) throws Exception {
- ProcessContext context = new ProcessContext(doFn, out);
- this.doFn.startBundle(context);
- for (IN value : values) {
- context.inValue = value;
- doFn.processElement(context);
- }
- this.doFn.finishBundle(context);
- }
-
- private class ProcessContext extends DoFn<IN, OUT>.ProcessContext {
-
- IN inValue;
- Collector<OUT> outCollector;
-
- public ProcessContext(DoFn<IN, OUT> fn, Collector<OUT> outCollector) {
- fn.super();
- super.setupDelegateAggregators();
- this.outCollector = outCollector;
- }
-
- @Override
- public IN element() {
- return this.inValue;
- }
-
-
- @Override
- public Instant timestamp() {
- return Instant.now();
- }
-
- @Override
- public BoundedWindow window() {
- return GlobalWindow.INSTANCE;
- }
-
- @Override
- public PaneInfo pane() {
- return PaneInfo.NO_FIRING;
- }
-
- @Override
- public WindowingInternals<IN, OUT> windowingInternals() {
- return new WindowingInternals<IN, OUT>() {
- @Override
- public StateInternals stateInternals() {
- return null;
- }
-
- @Override
- public void outputWindowedValue(OUT output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
-
- }
-
- @Override
- public TimerInternals timerInternals() {
- return null;
- }
-
- @Override
- public Collection<? extends BoundedWindow> windows() {
- return ImmutableList.of(GlobalWindow.INSTANCE);
- }
-
- @Override
- public PaneInfo pane() {
- return PaneInfo.NO_FIRING;
- }
-
- @Override
- public <T> void writePCollectionViewData(TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
- throw new RuntimeException("sideInput() not implemented.");
- }
- };
- }
-
- @Override
- public PipelineOptions getPipelineOptions() {
- return options;
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view) {
- List<T> sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal().getId());
- List<WindowedValue<?>> windowedValueList = new ArrayList<>(sideInput.size());
- for (T input : sideInput) {
- windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE), pane()));
- }
- return view.fromIterableInternal(windowedValueList);
- }
-
- @Override
- public void output(OUT output) {
- outCollector.collect(output);
- }
-
- @Override
- public void outputWithTimestamp(OUT output, Instant timestamp) {
- // not FLink's way, just output normally
- output(output);
- }
-
- @Override
- public <T> void sideOutput(TupleTag<T> tag, T output) {
- // ignore the side output, this can happen when a user does not register
- // side outputs but then outputs using a freshly created TupleTag.
- }
-
- @Override
- public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
- sideOutput(tag, output);
- }
-
- @Override
- protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
- SerializableFnAggregatorWrapper<AggInputT, AggOutputT> wrapper = new SerializableFnAggregatorWrapper<>(combiner);
- getRuntimeContext().addAccumulator(name, wrapper);
- return wrapper;
- }
-
-
- }
+ private final DoFn<IN, OUT> doFn;
+ private transient PipelineOptions options;
+
+ public FlinkDoFnFunction(DoFn<IN, OUT> doFn, PipelineOptions options) {
+ this.doFn = doFn;
+ this.options = options;
+ }
+
+ private void writeObject(ObjectOutputStream out)
+ throws IOException, ClassNotFoundException {
+ out.defaultWriteObject();
+ ObjectMapper mapper = new ObjectMapper();
+ mapper.writeValue(out, options);
+ }
+
+ private void readObject(ObjectInputStream in)
+ throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ ObjectMapper mapper = new ObjectMapper();
+ options = mapper.readValue(in, PipelineOptions.class);
+ }
+
+ @Override
+ public void mapPartition(Iterable<IN> values, Collector<OUT> out) throws Exception {
+ ProcessContext context = new ProcessContext(doFn, out);
+ this.doFn.startBundle(context);
+ for (IN value : values) {
+ context.inValue = value;
+ doFn.processElement(context);
+ }
+ this.doFn.finishBundle(context);
+ }
+
+ private class ProcessContext extends DoFn<IN, OUT>.ProcessContext {
+
+ IN inValue;
+ Collector<OUT> outCollector;
+
+ public ProcessContext(DoFn<IN, OUT> fn, Collector<OUT> outCollector) {
+ fn.super();
+ super.setupDelegateAggregators();
+ this.outCollector = outCollector;
+ }
+
+ @Override
+ public IN element() {
+ return this.inValue;
+ }
+
+
+ @Override
+ public Instant timestamp() {
+ return Instant.now();
+ }
+
+ @Override
+ public BoundedWindow window() {
+ return GlobalWindow.INSTANCE;
+ }
+
+ @Override
+ public PaneInfo pane() {
+ return PaneInfo.NO_FIRING;
+ }
+
+ @Override
+ public WindowingInternals<IN, OUT> windowingInternals() {
+ return new WindowingInternals<IN, OUT>() {
+ @Override
+ public StateInternals stateInternals() {
+ return null;
+ }
+
+ @Override
+ public void outputWindowedValue(OUT output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+
+ }
+
+ @Override
+ public TimerInternals timerInternals() {
+ return null;
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ return ImmutableList.of(GlobalWindow.INSTANCE);
+ }
+
+ @Override
+ public PaneInfo pane() {
+ return PaneInfo.NO_FIRING;
+ }
+
+ @Override
+ public <T> void writePCollectionViewData(TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view, BoundedWindow mainInputWindow) {
+ throw new RuntimeException("sideInput() not implemented.");
+ }
+ };
+ }
+
+ @Override
+ public PipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view) {
+ List<T> sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal().getId());
+ List<WindowedValue<?>> windowedValueList = new ArrayList<>(sideInput.size());
+ for (T input : sideInput) {
+ windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE), pane()));
+ }
+ return view.fromIterableInternal(windowedValueList);
+ }
+
+ @Override
+ public void output(OUT output) {
+ outCollector.collect(output);
+ }
+
+ @Override
+ public void outputWithTimestamp(OUT output, Instant timestamp) {
+ // not FLink's way, just output normally
+ output(output);
+ }
+
+ @Override
+ public <T> void sideOutput(TupleTag<T> tag, T output) {
+ // ignore the side output, this can happen when a user does not register
+ // side outputs but then outputs using a freshly created TupleTag.
+ }
+
+ @Override
+ public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+ sideOutput(tag, output);
+ }
+
+ @Override
+ protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
+ SerializableFnAggregatorWrapper<AggInputT, AggOutputT> wrapper = new SerializableFnAggregatorWrapper<>(combiner);
+ getRuntimeContext().addAccumulator(name, wrapper);
+ return wrapper;
+ }
+
+
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java
index 5d3702a..0116972 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkKeyedListAggregationFunction.java
@@ -29,47 +29,47 @@ import java.util.Iterator;
*/
public class FlinkKeyedListAggregationFunction<K,V> implements GroupReduceFunction<KV<K, V>, KV<K, Iterable<V>>> {
- @Override
- public void reduce(Iterable<KV<K, V>> values, Collector<KV<K, Iterable<V>>> out) throws Exception {
- Iterator<KV<K, V>> it = values.iterator();
- KV<K, V> first = it.next();
- Iterable<V> passThrough = new PassThroughIterable<>(first, it);
- out.collect(KV.of(first.getKey(), passThrough));
- }
+ @Override
+ public void reduce(Iterable<KV<K, V>> values, Collector<KV<K, Iterable<V>>> out) throws Exception {
+ Iterator<KV<K, V>> it = values.iterator();
+ KV<K, V> first = it.next();
+ Iterable<V> passThrough = new PassThroughIterable<>(first, it);
+ out.collect(KV.of(first.getKey(), passThrough));
+ }
- private static class PassThroughIterable<K, V> implements Iterable<V>, Iterator<V> {
- private KV<K, V> first;
- private Iterator<KV<K, V>> iterator;
+ private static class PassThroughIterable<K, V> implements Iterable<V>, Iterator<V> {
+ private KV<K, V> first;
+ private Iterator<KV<K, V>> iterator;
- public PassThroughIterable(KV<K, V> first, Iterator<KV<K, V>> iterator) {
- this.first = first;
- this.iterator = iterator;
- }
+ public PassThroughIterable(KV<K, V> first, Iterator<KV<K, V>> iterator) {
+ this.first = first;
+ this.iterator = iterator;
+ }
- @Override
- public Iterator<V> iterator() {
- return this;
- }
+ @Override
+ public Iterator<V> iterator() {
+ return this;
+ }
- @Override
- public boolean hasNext() {
- return first != null || iterator.hasNext();
- }
+ @Override
+ public boolean hasNext() {
+ return first != null || iterator.hasNext();
+ }
- @Override
- public V next() {
- if (first != null) {
- V result = first.getValue();
- first = null;
- return result;
- } else {
- return iterator.next().getValue();
- }
- }
+ @Override
+ public V next() {
+ if (first != null) {
+ V result = first.getValue();
+ first = null;
+ return result;
+ } else {
+ return iterator.next().getValue();
+ }
+ }
- @Override
- public void remove() {
- throw new UnsupportedOperationException("Cannot remove elements from input.");
- }
- }
+ @Override
+ public void remove() {
+ throw new UnsupportedOperationException("Cannot remove elements from input.");
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java
index 6187182..9e51638 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputDoFnFunction.java
@@ -51,125 +51,125 @@ import java.util.Map;
*/
public class FlinkMultiOutputDoFnFunction<IN, OUT> extends RichMapPartitionFunction<IN, RawUnionValue> {
- private final DoFn<IN, OUT> doFn;
- private transient PipelineOptions options;
- private final Map<TupleTag<?>, Integer> outputMap;
-
- public FlinkMultiOutputDoFnFunction(DoFn<IN, OUT> doFn, PipelineOptions options, Map<TupleTag<?>, Integer> outputMap) {
- this.doFn = doFn;
- this.options = options;
- this.outputMap = outputMap;
- }
-
- private void writeObject(ObjectOutputStream out)
- throws IOException, ClassNotFoundException {
- out.defaultWriteObject();
- ObjectMapper mapper = new ObjectMapper();
- mapper.writeValue(out, options);
- }
-
- private void readObject(ObjectInputStream in)
- throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- ObjectMapper mapper = new ObjectMapper();
- options = mapper.readValue(in, PipelineOptions.class);
-
- }
-
- @Override
- public void mapPartition(Iterable<IN> values, Collector<RawUnionValue> out) throws Exception {
- ProcessContext context = new ProcessContext(doFn, out);
- this.doFn.startBundle(context);
- for (IN value : values) {
- context.inValue = value;
- doFn.processElement(context);
- }
- this.doFn.finishBundle(context);
- }
-
- private class ProcessContext extends DoFn<IN, OUT>.ProcessContext {
-
- IN inValue;
- Collector<RawUnionValue> outCollector;
-
- public ProcessContext(DoFn<IN, OUT> fn, Collector<RawUnionValue> outCollector) {
- fn.super();
- this.outCollector = outCollector;
- }
-
- @Override
- public IN element() {
- return this.inValue;
- }
-
- @Override
- public Instant timestamp() {
- return Instant.now();
- }
-
- @Override
- public BoundedWindow window() {
- return GlobalWindow.INSTANCE;
- }
-
- @Override
- public PaneInfo pane() {
- return PaneInfo.NO_FIRING;
- }
-
- @Override
- public WindowingInternals<IN, OUT> windowingInternals() {
- return null;
- }
-
- @Override
- public PipelineOptions getPipelineOptions() {
- return options;
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view) {
- List<T> sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal()
- .getId());
- List<WindowedValue<?>> windowedValueList = new ArrayList<>(sideInput.size());
- for (T input : sideInput) {
- windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE), pane()));
- }
- return view.fromIterableInternal(windowedValueList);
- }
-
- @Override
- public void output(OUT value) {
- // assume that index 0 is the default output
- outCollector.collect(new RawUnionValue(0, value));
- }
-
- @Override
- public void outputWithTimestamp(OUT output, Instant timestamp) {
- // not FLink's way, just output normally
- output(output);
- }
-
- @Override
- @SuppressWarnings("unchecked")
- public <T> void sideOutput(TupleTag<T> tag, T value) {
- Integer index = outputMap.get(tag);
- if (index != null) {
- outCollector.collect(new RawUnionValue(index, value));
- }
- }
-
- @Override
- public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
- sideOutput(tag, output);
- }
-
- @Override
- protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
- SerializableFnAggregatorWrapper<AggInputT, AggOutputT> wrapper = new SerializableFnAggregatorWrapper<>(combiner);
- getRuntimeContext().addAccumulator(name, wrapper);
- return null;
- }
-
- }
+ private final DoFn<IN, OUT> doFn;
+ private transient PipelineOptions options;
+ private final Map<TupleTag<?>, Integer> outputMap;
+
+ public FlinkMultiOutputDoFnFunction(DoFn<IN, OUT> doFn, PipelineOptions options, Map<TupleTag<?>, Integer> outputMap) {
+ this.doFn = doFn;
+ this.options = options;
+ this.outputMap = outputMap;
+ }
+
+ private void writeObject(ObjectOutputStream out)
+ throws IOException, ClassNotFoundException {
+ out.defaultWriteObject();
+ ObjectMapper mapper = new ObjectMapper();
+ mapper.writeValue(out, options);
+ }
+
+ private void readObject(ObjectInputStream in)
+ throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ ObjectMapper mapper = new ObjectMapper();
+ options = mapper.readValue(in, PipelineOptions.class);
+
+ }
+
+ @Override
+ public void mapPartition(Iterable<IN> values, Collector<RawUnionValue> out) throws Exception {
+ ProcessContext context = new ProcessContext(doFn, out);
+ this.doFn.startBundle(context);
+ for (IN value : values) {
+ context.inValue = value;
+ doFn.processElement(context);
+ }
+ this.doFn.finishBundle(context);
+ }
+
+ private class ProcessContext extends DoFn<IN, OUT>.ProcessContext {
+
+ IN inValue;
+ Collector<RawUnionValue> outCollector;
+
+ public ProcessContext(DoFn<IN, OUT> fn, Collector<RawUnionValue> outCollector) {
+ fn.super();
+ this.outCollector = outCollector;
+ }
+
+ @Override
+ public IN element() {
+ return this.inValue;
+ }
+
+ @Override
+ public Instant timestamp() {
+ return Instant.now();
+ }
+
+ @Override
+ public BoundedWindow window() {
+ return GlobalWindow.INSTANCE;
+ }
+
+ @Override
+ public PaneInfo pane() {
+ return PaneInfo.NO_FIRING;
+ }
+
+ @Override
+ public WindowingInternals<IN, OUT> windowingInternals() {
+ return null;
+ }
+
+ @Override
+ public PipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view) {
+ List<T> sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal()
+ .getId());
+ List<WindowedValue<?>> windowedValueList = new ArrayList<>(sideInput.size());
+ for (T input : sideInput) {
+ windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE), pane()));
+ }
+ return view.fromIterableInternal(windowedValueList);
+ }
+
+ @Override
+ public void output(OUT value) {
+ // assume that index 0 is the default output
+ outCollector.collect(new RawUnionValue(0, value));
+ }
+
+ @Override
+ public void outputWithTimestamp(OUT output, Instant timestamp) {
+ // not FLink's way, just output normally
+ output(output);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public <T> void sideOutput(TupleTag<T> tag, T value) {
+ Integer index = outputMap.get(tag);
+ if (index != null) {
+ outCollector.collect(new RawUnionValue(index, value));
+ }
+ }
+
+ @Override
+ public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+ sideOutput(tag, output);
+ }
+
+ @Override
+ protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
+ SerializableFnAggregatorWrapper<AggInputT, AggOutputT> wrapper = new SerializableFnAggregatorWrapper<>(combiner);
+ getRuntimeContext().addAccumulator(name, wrapper);
+ return null;
+ }
+
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java
index 6792b23..e883d42 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkMultiOutputPruningFunction.java
@@ -25,17 +25,17 @@ import org.apache.flink.util.Collector;
*/
public class FlinkMultiOutputPruningFunction<T> implements FlatMapFunction<RawUnionValue, T> {
- private final int outputTag;
+ private final int outputTag;
- public FlinkMultiOutputPruningFunction(int outputTag) {
- this.outputTag = outputTag;
- }
+ public FlinkMultiOutputPruningFunction(int outputTag) {
+ this.outputTag = outputTag;
+ }
- @Override
- @SuppressWarnings("unchecked")
- public void flatMap(RawUnionValue rawUnionValue, Collector<T> collector) throws Exception {
- if (rawUnionValue.getUnionTag() == outputTag) {
- collector.collect((T) rawUnionValue.getValue());
- }
- }
+ @Override
+ @SuppressWarnings("unchecked")
+ public void flatMap(RawUnionValue rawUnionValue, Collector<T> collector) throws Exception {
+ if (rawUnionValue.getUnionTag() == outputTag) {
+ collector.collect((T) rawUnionValue.getValue());
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java
index ef47b72..1ff9aff 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkPartialReduceFunction.java
@@ -32,29 +32,29 @@ import java.util.Iterator;
*/
public class FlinkPartialReduceFunction<K, VI, VA> implements GroupCombineFunction<KV<K, VI>, KV<K, VA>> {
- private final Combine.KeyedCombineFn<K, VI, VA, ?> keyedCombineFn;
-
- public FlinkPartialReduceFunction(Combine.KeyedCombineFn<K, VI, VA, ?>
- keyedCombineFn) {
- this.keyedCombineFn = keyedCombineFn;
- }
-
- @Override
- public void combine(Iterable<KV<K, VI>> elements, Collector<KV<K, VA>> out) throws Exception {
-
- final Iterator<KV<K, VI>> iterator = elements.iterator();
- // create accumulator using the first elements key
- KV<K, VI> first = iterator.next();
- K key = first.getKey();
- VI value = first.getValue();
- VA accumulator = keyedCombineFn.createAccumulator(key);
- accumulator = keyedCombineFn.addInput(key, accumulator, value);
-
- while(iterator.hasNext()) {
- value = iterator.next().getValue();
- accumulator = keyedCombineFn.addInput(key, accumulator, value);
- }
-
- out.collect(KV.of(key, accumulator));
- }
+ private final Combine.KeyedCombineFn<K, VI, VA, ?> keyedCombineFn;
+
+ public FlinkPartialReduceFunction(Combine.KeyedCombineFn<K, VI, VA, ?>
+ keyedCombineFn) {
+ this.keyedCombineFn = keyedCombineFn;
+ }
+
+ @Override
+ public void combine(Iterable<KV<K, VI>> elements, Collector<KV<K, VA>> out) throws Exception {
+
+ final Iterator<KV<K, VI>> iterator = elements.iterator();
+ // create accumulator using the first elements key
+ KV<K, VI> first = iterator.next();
+ K key = first.getKey();
+ VI value = first.getValue();
+ VA accumulator = keyedCombineFn.createAccumulator(key);
+ accumulator = keyedCombineFn.addInput(key, accumulator, value);
+
+ while(iterator.hasNext()) {
+ value = iterator.next().getValue();
+ accumulator = keyedCombineFn.addInput(key, accumulator, value);
+ }
+
+ out.collect(KV.of(key, accumulator));
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java
index cd0b38c..94676a2 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/functions/FlinkReduceFunction.java
@@ -33,25 +33,25 @@ import java.util.Iterator;
*/
public class FlinkReduceFunction<K, VA, VO> implements GroupReduceFunction<KV<K, VA>, KV<K, VO>> {
- private final Combine.KeyedCombineFn<K, ?, VA, VO> keyedCombineFn;
+ private final Combine.KeyedCombineFn<K, ?, VA, VO> keyedCombineFn;
- public FlinkReduceFunction(Combine.KeyedCombineFn<K, ?, VA, VO> keyedCombineFn) {
- this.keyedCombineFn = keyedCombineFn;
- }
+ public FlinkReduceFunction(Combine.KeyedCombineFn<K, ?, VA, VO> keyedCombineFn) {
+ this.keyedCombineFn = keyedCombineFn;
+ }
- @Override
- public void reduce(Iterable<KV<K, VA>> values, Collector<KV<K, VO>> out) throws Exception {
- Iterator<KV<K, VA>> it = values.iterator();
+ @Override
+ public void reduce(Iterable<KV<K, VA>> values, Collector<KV<K, VO>> out) throws Exception {
+ Iterator<KV<K, VA>> it = values.iterator();
- KV<K, VA> current = it.next();
- K k = current.getKey();
- VA accumulator = current.getValue();
+ KV<K, VA> current = it.next();
+ K k = current.getKey();
+ VA accumulator = current.getValue();
- while (it.hasNext()) {
- current = it.next();
- keyedCombineFn.mergeAccumulators(k, ImmutableList.of(accumulator, current.getValue()) );
- }
+ while (it.hasNext()) {
+ current = it.next();
+ keyedCombineFn.mergeAccumulators(k, ImmutableList.of(accumulator, current.getValue()) );
+ }
- out.collect(KV.of(k, keyedCombineFn.extractOutput(k, accumulator)));
- }
+ out.collect(KV.of(k, keyedCombineFn.extractOutput(k, accumulator)));
+ }
}
[16/50] [abbrv] incubator-beam git commit: Rearranging the code and
renaming certain classes.
Posted by da...@apache.org.
Rearranging the code and renaming certain classes.
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/bc4c60eb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/bc4c60eb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/bc4c60eb
Branch: refs/heads/master
Commit: bc4c60ebfa49ad050367533809c265375e8c0b01
Parents: 37a9b29
Author: kl0u <kk...@gmail.com>
Authored: Mon Feb 29 12:38:56 2016 +0100
Committer: Davor Bonaci <da...@users.noreply.github.com>
Committed: Fri Mar 4 10:04:23 2016 -0800
----------------------------------------------------------------------
.../dataflow/FlinkJobExecutionEnvironment.java | 237 -----------------
.../FlinkPipelineExecutionEnvironment.java | 255 +++++++++++++++++++
.../flink/dataflow/FlinkPipelineOptions.java | 23 +-
.../flink/dataflow/FlinkPipelineRunner.java | 6 +-
.../examples/streaming/AutoComplete.java | 7 +-
.../examples/streaming/JoinExamples.java | 5 +-
.../KafkaWindowedWordCountExample.java | 3 +
.../examples/streaming/WindowedWordCount.java | 3 +
.../FlinkStreamingTransformTranslators.java | 2 -
.../FlinkStreamingTranslationContext.java | 9 +-
.../streaming/FlinkAbstractParDoWrapper.java | 10 +-
.../FlinkGroupAlsoByWindowWrapper.java | 2 -
.../io/FlinkStreamingCreateFunction.java | 7 +-
.../streaming/io/UnboundedSourceWrapper.java | 11 +-
.../dataartisans/flink/dataflow/AvroITCase.java | 4 +-
.../flink/dataflow/FlattenizeITCase.java | 2 +-
.../flink/dataflow/FlinkTestPipeline.java | 14 +-
.../flink/dataflow/JoinExamplesITCase.java | 2 +-
.../flink/dataflow/MaybeEmptyTestITCase.java | 2 +-
.../flink/dataflow/ParDoMultiOutputITCase.java | 2 +-
.../flink/dataflow/ReadSourceITCase.java | 2 +-
.../dataflow/RemoveDuplicatesEmptyITCase.java | 2 +-
.../flink/dataflow/RemoveDuplicatesITCase.java | 2 +-
.../flink/dataflow/SideInputITCase.java | 2 +-
.../flink/dataflow/TfIdfITCase.java | 2 +-
.../dataflow/TopWikipediaSessionsITCase.java | 144 -----------
.../flink/dataflow/WordCountITCase.java | 2 +-
.../flink/dataflow/WordCountJoin2ITCase.java | 2 +-
.../flink/dataflow/WordCountJoin3ITCase.java | 2 +-
.../flink/dataflow/WriteSinkITCase.java | 2 +-
.../streaming/GroupAlsoByWindowTest.java | 8 +-
.../streaming/TopWikipediaSessionsITCase.java | 145 +++++++++++
32 files changed, 481 insertions(+), 440 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java
deleted file mode 100644
index 91b2f64..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkJobExecutionEnvironment.java
+++ /dev/null
@@ -1,237 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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 com.dataartisans.flink.dataflow;
-
-import com.dataartisans.flink.dataflow.translation.FlinkPipelineTranslator;
-import com.dataartisans.flink.dataflow.translation.FlinkBatchPipelineTranslator;
-import com.dataartisans.flink.dataflow.translation.FlinkStreamingPipelineTranslator;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.java.CollectionEnvironment;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.streaming.api.TimeCharacteristic;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.List;
-
-public class FlinkJobExecutionEnvironment {
-
- private static final Logger LOG = LoggerFactory.getLogger(FlinkJobExecutionEnvironment.class);
-
- private final FlinkPipelineOptions options;
-
- /**
- * The Flink Batch execution environment. This is instantiated to either a
- * {@link org.apache.flink.api.java.CollectionEnvironment},
- * a {@link org.apache.flink.api.java.LocalEnvironment} or
- * a {@link org.apache.flink.api.java.RemoteEnvironment}, depending on the configuration
- * options.
- */
- private ExecutionEnvironment flinkBatchEnv;
-
-
- /**
- * The Flink Streaming execution environment. This is instantiated to either a
- * {@link org.apache.flink.streaming.api.environment.LocalStreamEnvironment} or
- * a {@link org.apache.flink.streaming.api.environment.RemoteStreamEnvironment}, depending
- * on the configuration options, and more specifically, the url of the master url.
- */
- private StreamExecutionEnvironment flinkStreamEnv;
-
- /**
- * Translator for this FlinkPipelineRunner. Its role is to translate the Dataflow operators to
- * their Flink based counterparts. Based on the options provided by the user, if we have a streaming job,
- * this is instantiated to a FlinkStreamingPipelineTranslator. In other case, i.e. a batch job,
- * a FlinkBatchPipelineTranslator is created.
- */
- private FlinkPipelineTranslator flinkPipelineTranslator;
-
- public FlinkJobExecutionEnvironment(FlinkPipelineOptions options) {
- if (options == null) {
- throw new IllegalArgumentException("Options in the FlinkJobExecutionEnvironment cannot be NULL.");
- }
- this.options = options;
- this.createJobEnvironment();
- this.createJobGraphTranslator();
- }
-
- /**
- * Depending on the type of job (Streaming or Batch) and the user-specified options,
- * this method creates the adequate ExecutionEnvironment.
- */
- private void createJobEnvironment() {
- if (options.isStreaming()) {
- LOG.info("Creating the required STREAMING Environment.");
- createStreamExecutionEnvironment();
- } else {
- LOG.info("Creating the required BATCH Environment.");
- createBatchExecutionEnvironment();
- }
- }
-
- /**
- * Depending on the type of job (Streaming or Batch), this method creates the adequate job graph
- * translator. In the case of batch, it will work with DataSets, while for streaming, it will work
- * with DataStreams.
- */
- private void createJobGraphTranslator() {
- checkInitializationState();
- if (this.flinkPipelineTranslator != null) {
- throw new IllegalStateException("JobGraphTranslator already initialized.");
- }
-
- this.flinkPipelineTranslator = options.isStreaming() ?
- new FlinkStreamingPipelineTranslator(flinkStreamEnv, options) :
- new FlinkBatchPipelineTranslator(flinkBatchEnv, options);
- }
-
- public void translate(Pipeline pipeline) {
- checkInitializationState();
- if(this.flinkBatchEnv == null && this.flinkStreamEnv == null) {
- createJobEnvironment();
- }
- if (this.flinkPipelineTranslator == null) {
- createJobGraphTranslator();
- }
- this.flinkPipelineTranslator.translate(pipeline);
- }
-
- public JobExecutionResult executeJob() throws Exception {
- if (options.isStreaming()) {
-
- System.out.println("Plan: " + this.flinkStreamEnv.getExecutionPlan());
-
- if (this.flinkStreamEnv == null) {
- throw new RuntimeException("JobExecutionEnvironment not initialized.");
- }
- if (this.flinkPipelineTranslator == null) {
- throw new RuntimeException("JobGraphTranslator not initialized.");
- }
- return this.flinkStreamEnv.execute();
- } else {
- if (this.flinkBatchEnv == null) {
- throw new RuntimeException("JobExecutionEnvironment not initialized.");
- }
- if (this.flinkPipelineTranslator == null) {
- throw new RuntimeException("JobGraphTranslator not initialized.");
- }
- return this.flinkBatchEnv.execute();
- }
- }
-
- /**
- * If the submitted job is a batch processing job, this method creates the adequate
- * Flink {@link org.apache.flink.api.java.ExecutionEnvironment} depending
- * on the user-specified options.
- */
- private void createBatchExecutionEnvironment() {
- if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) {
- throw new RuntimeException("JobExecutionEnvironment already initialized.");
- }
-
- String masterUrl = options.getFlinkMaster();
- this.flinkStreamEnv = null;
-
- // depending on the master, create the right environment.
- if (masterUrl.equals("[local]")) {
- this.flinkBatchEnv = ExecutionEnvironment.createLocalEnvironment();
- } else if (masterUrl.equals("[collection]")) {
- this.flinkBatchEnv = new CollectionEnvironment();
- } else if (masterUrl.equals("[auto]")) {
- this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
- } else if (masterUrl.matches(".*:\\d*")) {
- String[] parts = masterUrl.split(":");
- List<String> stagingFiles = options.getFilesToStage();
- this.flinkBatchEnv = ExecutionEnvironment.createRemoteEnvironment(parts[0],
- Integer.parseInt(parts[1]),
- stagingFiles.toArray(new String[stagingFiles.size()]));
- } else {
- LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
- this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
- }
-
- // set the correct parallelism.
- if (options.getParallelism() != -1 && !(this.flinkBatchEnv instanceof CollectionEnvironment)) {
- this.flinkBatchEnv.setParallelism(options.getParallelism());
- }
-
- // set parallelism in the options (required by some execution code)
- options.setParallelism(flinkBatchEnv.getParallelism());
- }
-
- /**
- * If the submitted job is a stream processing job, this method creates the adequate
- * Flink {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment} depending
- * on the user-specified options.
- */
- private void createStreamExecutionEnvironment() {
- if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) {
- throw new RuntimeException("JobExecutionEnvironment already initialized.");
- }
-
- String masterUrl = options.getFlinkMaster();
- this.flinkBatchEnv = null;
-
- // depending on the master, create the right environment.
- if (masterUrl.equals("[local]")) {
- this.flinkStreamEnv = StreamExecutionEnvironment.createLocalEnvironment();
- } else if (masterUrl.equals("[auto]")) {
- this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
- } else if (masterUrl.matches(".*:\\d*")) {
- String[] parts = masterUrl.split(":");
- List<String> stagingFiles = options.getFilesToStage();
- this.flinkStreamEnv = StreamExecutionEnvironment.createRemoteEnvironment(parts[0],
- Integer.parseInt(parts[1]), stagingFiles.toArray(new String[stagingFiles.size()]));
- } else {
- LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
- this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
- }
-
- // set the correct parallelism.
- if (options.getParallelism() != -1) {
- this.flinkStreamEnv.setParallelism(options.getParallelism());
- }
-
- // set parallelism in the options (required by some execution code)
- options.setParallelism(flinkStreamEnv.getParallelism());
-
- // although we do not use the generated timestamps,
- // enabling timestamps is needed for the watermarks.
- this.flinkStreamEnv.getConfig().enableTimestamps();
-
- this.flinkStreamEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
-
- this.flinkStreamEnv.enableCheckpointing(1000);
- this.flinkStreamEnv.setNumberOfExecutionRetries(5);
-
- LOG.info("Setting execution retry delay to 3 sec");
- this.flinkStreamEnv.getConfig().setExecutionRetryDelay(3000);
- }
-
- private void checkInitializationState() {
- if (this.options == null) {
- throw new IllegalStateException("FlinkJobExecutionEnvironment is not initialized yet.");
- }
-
- if (options.isStreaming() && this.flinkBatchEnv != null) {
- throw new IllegalStateException("Attempted to run a Streaming Job with a Batch Execution Environment.");
- } else if (!options.isStreaming() && this.flinkStreamEnv != null) {
- throw new IllegalStateException("Attempted to run a Batch Job with a Streaming Execution Environment.");
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java
new file mode 100644
index 0000000..a1372bd
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineExecutionEnvironment.java
@@ -0,0 +1,255 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow;
+
+import com.dataartisans.flink.dataflow.translation.FlinkPipelineTranslator;
+import com.dataartisans.flink.dataflow.translation.FlinkBatchPipelineTranslator;
+import com.dataartisans.flink.dataflow.translation.FlinkStreamingPipelineTranslator;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.common.base.Preconditions;
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.java.CollectionEnvironment;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class FlinkPipelineExecutionEnvironment {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class);
+
+ private final FlinkPipelineOptions options;
+
+ /**
+ * The Flink Batch execution environment. This is instantiated to either a
+ * {@link org.apache.flink.api.java.CollectionEnvironment},
+ * a {@link org.apache.flink.api.java.LocalEnvironment} or
+ * a {@link org.apache.flink.api.java.RemoteEnvironment}, depending on the configuration
+ * options.
+ */
+ private ExecutionEnvironment flinkBatchEnv;
+
+
+ /**
+ * The Flink Streaming execution environment. This is instantiated to either a
+ * {@link org.apache.flink.streaming.api.environment.LocalStreamEnvironment} or
+ * a {@link org.apache.flink.streaming.api.environment.RemoteStreamEnvironment}, depending
+ * on the configuration options, and more specifically, the url of the master.
+ */
+ private StreamExecutionEnvironment flinkStreamEnv;
+
+ /**
+ * Translator for this FlinkPipelineRunner. Its role is to translate the Beam operators to
+ * their Flink counterparts. Based on the options provided by the user, if we have a streaming job,
+ * this is instantiated as a {@link FlinkStreamingPipelineTranslator}. In other case, i.e. a batch job,
+ * a {@link FlinkBatchPipelineTranslator} is created.
+ */
+ private FlinkPipelineTranslator flinkPipelineTranslator;
+
+ /**
+ * Creates a {@link FlinkPipelineExecutionEnvironment} with the user-specified parameters in the
+ * provided {@link FlinkPipelineOptions}.
+ *
+ * @param options the user-defined pipeline options.
+ * */
+ public FlinkPipelineExecutionEnvironment(FlinkPipelineOptions options) {
+ this.options = Preconditions.checkNotNull(options);
+ this.createPipelineExecutionEnvironment();
+ this.createPipelineTranslator();
+ }
+
+ /**
+ * Depending on the type of job (Streaming or Batch) and the user-specified options,
+ * this method creates the adequate ExecutionEnvironment.
+ */
+ private void createPipelineExecutionEnvironment() {
+ if (options.isStreaming()) {
+ createStreamExecutionEnvironment();
+ } else {
+ createBatchExecutionEnvironment();
+ }
+ }
+
+ /**
+ * Depending on the type of job (Streaming or Batch), this method creates the adequate job graph
+ * translator. In the case of batch, it will work with {@link org.apache.flink.api.java.DataSet},
+ * while for streaming, it will work with {@link org.apache.flink.streaming.api.datastream.DataStream}.
+ */
+ private void createPipelineTranslator() {
+ checkInitializationState();
+ if (this.flinkPipelineTranslator != null) {
+ throw new IllegalStateException("FlinkPipelineTranslator already initialized.");
+ }
+
+ this.flinkPipelineTranslator = options.isStreaming() ?
+ new FlinkStreamingPipelineTranslator(flinkStreamEnv, options) :
+ new FlinkBatchPipelineTranslator(flinkBatchEnv, options);
+ }
+
+ /**
+ * Depending on if the job is a Streaming or a Batch one, this method creates
+ * the necessary execution environment and pipeline translator, and translates
+ * the {@link com.google.cloud.dataflow.sdk.values.PCollection} program into
+ * a {@link org.apache.flink.api.java.DataSet} or {@link org.apache.flink.streaming.api.datastream.DataStream}
+ * one.
+ * */
+ public void translate(Pipeline pipeline) {
+ checkInitializationState();
+ if(this.flinkBatchEnv == null && this.flinkStreamEnv == null) {
+ createPipelineExecutionEnvironment();
+ }
+ if (this.flinkPipelineTranslator == null) {
+ createPipelineTranslator();
+ }
+ this.flinkPipelineTranslator.translate(pipeline);
+ }
+
+ /**
+ * Launches the program execution.
+ * */
+ public JobExecutionResult executePipeline() throws Exception {
+ if (options.isStreaming()) {
+ if (this.flinkStreamEnv == null) {
+ throw new RuntimeException("FlinkPipelineExecutionEnvironment not initialized.");
+ }
+ if (this.flinkPipelineTranslator == null) {
+ throw new RuntimeException("FlinkPipelineTranslator not initialized.");
+ }
+ return this.flinkStreamEnv.execute();
+ } else {
+ if (this.flinkBatchEnv == null) {
+ throw new RuntimeException("FlinkPipelineExecutionEnvironment not initialized.");
+ }
+ if (this.flinkPipelineTranslator == null) {
+ throw new RuntimeException("FlinkPipelineTranslator not initialized.");
+ }
+ return this.flinkBatchEnv.execute();
+ }
+ }
+
+ /**
+ * If the submitted job is a batch processing job, this method creates the adequate
+ * Flink {@link org.apache.flink.api.java.ExecutionEnvironment} depending
+ * on the user-specified options.
+ */
+ private void createBatchExecutionEnvironment() {
+ if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) {
+ throw new RuntimeException("FlinkPipelineExecutionEnvironment already initialized.");
+ }
+
+ LOG.info("Creating the required Batch Execution Environment.");
+
+ String masterUrl = options.getFlinkMaster();
+ this.flinkStreamEnv = null;
+
+ // depending on the master, create the right environment.
+ if (masterUrl.equals("[local]")) {
+ this.flinkBatchEnv = ExecutionEnvironment.createLocalEnvironment();
+ } else if (masterUrl.equals("[collection]")) {
+ this.flinkBatchEnv = new CollectionEnvironment();
+ } else if (masterUrl.equals("[auto]")) {
+ this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
+ } else if (masterUrl.matches(".*:\\d*")) {
+ String[] parts = masterUrl.split(":");
+ List<String> stagingFiles = options.getFilesToStage();
+ this.flinkBatchEnv = ExecutionEnvironment.createRemoteEnvironment(parts[0],
+ Integer.parseInt(parts[1]),
+ stagingFiles.toArray(new String[stagingFiles.size()]));
+ } else {
+ LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
+ this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
+ }
+
+ // set the correct parallelism.
+ if (options.getParallelism() != -1 && !(this.flinkBatchEnv instanceof CollectionEnvironment)) {
+ this.flinkBatchEnv.setParallelism(options.getParallelism());
+ }
+
+ // set parallelism in the options (required by some execution code)
+ options.setParallelism(flinkBatchEnv.getParallelism());
+ }
+
+ /**
+ * If the submitted job is a stream processing job, this method creates the adequate
+ * Flink {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment} depending
+ * on the user-specified options.
+ */
+ private void createStreamExecutionEnvironment() {
+ if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) {
+ throw new RuntimeException("FlinkPipelineExecutionEnvironment already initialized.");
+ }
+
+ LOG.info("Creating the required Streaming Environment.");
+
+ String masterUrl = options.getFlinkMaster();
+ this.flinkBatchEnv = null;
+
+ // depending on the master, create the right environment.
+ if (masterUrl.equals("[local]")) {
+ this.flinkStreamEnv = StreamExecutionEnvironment.createLocalEnvironment();
+ } else if (masterUrl.equals("[auto]")) {
+ this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+ } else if (masterUrl.matches(".*:\\d*")) {
+ String[] parts = masterUrl.split(":");
+ List<String> stagingFiles = options.getFilesToStage();
+ this.flinkStreamEnv = StreamExecutionEnvironment.createRemoteEnvironment(parts[0],
+ Integer.parseInt(parts[1]), stagingFiles.toArray(new String[stagingFiles.size()]));
+ } else {
+ LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
+ this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+ }
+
+ // set the correct parallelism.
+ if (options.getParallelism() != -1) {
+ this.flinkStreamEnv.setParallelism(options.getParallelism());
+ }
+
+ // set parallelism in the options (required by some execution code)
+ options.setParallelism(flinkStreamEnv.getParallelism());
+
+ // although we do not use the generated timestamps,
+ // enabling timestamps is needed for the watermarks.
+ this.flinkStreamEnv.getConfig().enableTimestamps();
+ this.flinkStreamEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+
+ // for the following 2 parameters, a value of -1 means that Flink will use
+ // the default values as specified in the configuration.
+ this.flinkStreamEnv.setNumberOfExecutionRetries(options.getNumberOfExecutionRetries());
+ this.flinkStreamEnv.getConfig().setExecutionRetryDelay(options.getExecutionRetryDelay());
+
+ // A value of -1 corresponds to disabled checkpointing (see CheckpointConfig in Flink).
+ // If the value is not -1, then the validity checks are applied.
+ // By default, checkpointing is disabled.
+ long checkpointInterval = options.getCheckpointingInterval();
+ if(checkpointInterval != -1) {
+ if (checkpointInterval < 1) {
+ throw new IllegalArgumentException("The checkpoint interval must be positive");
+ }
+ this.flinkStreamEnv.enableCheckpointing(checkpointInterval);
+ }
+ }
+
+ private void checkInitializationState() {
+ if (options.isStreaming() && this.flinkBatchEnv != null) {
+ throw new IllegalStateException("Attempted to run a Streaming Job with a Batch Execution Environment.");
+ } else if (!options.isStreaming() && this.flinkStreamEnv != null) {
+ throw new IllegalStateException("Attempted to run a Batch Job with a Streaming Execution Environment.");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java
index e746f41..2429cac 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineOptions.java
@@ -66,11 +66,26 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp
String getFlinkMaster();
void setFlinkMaster(String value);
- /**
- * The degree of parallelism to be used when parallelizing operations onto workers.
- */
- @Description("The degree of parallelism to be used when parallelizing operations onto workers.")
+ @Description("The degree of parallelism to be used when distributing operations onto workers.")
@Default.Integer(-1)
Integer getParallelism();
void setParallelism(Integer value);
+
+ @Description("The interval between consecutive checkpoints (i.e. snapshots of the current pipeline state used for " +
+ "fault tolerance).")
+ @Default.Long(-1L)
+ Long getCheckpointingInterval();
+ void setCheckpointingInterval(Long interval);
+
+ @Description("Sets the number of times that failed tasks are re-executed. " +
+ "A value of zero effectively disables fault tolerance. A value of -1 indicates " +
+ "that the system default value (as defined in the configuration) should be used.")
+ @Default.Integer(-1)
+ Integer getNumberOfExecutionRetries();
+ void setNumberOfExecutionRetries(Integer retries);
+
+ @Description("Sets the delay between executions. A value of {@code -1} indicates that the default value should be used.")
+ @Default.Long(-1L)
+ Long getExecutionRetryDelay();
+ void setExecutionRetryDelay(Long delay);
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
index ebd2691..7ea8370 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/FlinkPipelineRunner.java
@@ -54,7 +54,7 @@ public class FlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {
*/
private final FlinkPipelineOptions options;
- private final FlinkJobExecutionEnvironment flinkJobEnv;
+ private final FlinkPipelineExecutionEnvironment flinkJobEnv;
/**
* Construct a runner from the provided options.
@@ -103,7 +103,7 @@ public class FlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {
private FlinkPipelineRunner(FlinkPipelineOptions options) {
this.options = options;
- this.flinkJobEnv = new FlinkJobExecutionEnvironment(options);
+ this.flinkJobEnv = new FlinkPipelineExecutionEnvironment(options);
}
@Override
@@ -118,7 +118,7 @@ public class FlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {
JobExecutionResult result;
try {
- result = this.flinkJobEnv.executeJob();
+ result = this.flinkJobEnv.executePipeline();
} catch (Exception e) {
LOG.error("Pipeline execution failed", e);
throw new RuntimeException("Pipeline execution failed", e);
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java
index 711d9fb..493fb25 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/AutoComplete.java
@@ -325,7 +325,9 @@ public class AutoComplete {
* Takes as input a the top candidates per prefix, and emits an entity
* suitable for writing to Datastore.
*/
- static class FormatForPerTaskLocalFile extends DoFn<KV<String, List<CompletionCandidate>>, String> {
+ static class FormatForPerTaskLocalFile extends DoFn<KV<String, List<CompletionCandidate>>, String>
+ implements DoFn.RequiresWindowAccess{
+
private static final long serialVersionUID = 0;
@Override
@@ -357,6 +359,9 @@ public class AutoComplete {
public static void main(String[] args) throws IOException {
Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
options.setStreaming(true);
+ options.setCheckpointingInterval(1000L);
+ options.setNumberOfExecutionRetries(5);
+ options.setExecutionRetryDelay(3000L);
options.setRunner(FlinkPipelineRunner.class);
PTransform<? super PBegin, PCollection<String>> readSource =
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
index 9a5db64..60f6788 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/JoinExamples.java
@@ -124,9 +124,10 @@ public class JoinExamples {
public static void main(String[] args) throws Exception {
Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-
- // make it a streaming example.
options.setStreaming(true);
+ options.setCheckpointingInterval(1000L);
+ options.setNumberOfExecutionRetries(5);
+ options.setExecutionRetryDelay(3000L);
options.setRunner(FlinkPipelineRunner.class);
PTransform<? super PBegin, PCollection<String>> readSourceA =
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
index 42d3d88..dba2721 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/KafkaWindowedWordCountExample.java
@@ -104,6 +104,9 @@ public class KafkaWindowedWordCountExample {
KafkaStreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).as(KafkaStreamingWordCountOptions.class);
options.setJobName("KafkaExample");
options.setStreaming(true);
+ options.setCheckpointingInterval(1000L);
+ options.setNumberOfExecutionRetries(5);
+ options.setExecutionRetryDelay(3000L);
options.setRunner(FlinkPipelineRunner.class);
System.out.println(options.getKafkaTopic() +" "+ options.getZookeeper() +" "+ options.getBroker() +" "+ options.getGroup() );
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
index b539245..37dc39a 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/examples/streaming/WindowedWordCount.java
@@ -99,6 +99,9 @@ public class WindowedWordCount {
options.setStreaming(true);
options.setWindowSize(10L);
options.setSlide(5L);
+ options.setCheckpointingInterval(1000L);
+ options.setNumberOfExecutionRetries(5);
+ options.setExecutionRetryDelay(3000L);
options.setRunner(FlinkPipelineRunner.class);
LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize() +
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
index 46d3e36..27cc923 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTransformTranslators.java
@@ -69,7 +69,6 @@ public class FlinkStreamingTransformTranslators {
// here you can find all the available translators.
static {
-
TRANSLATORS.put(Create.Values.class, new CreateStreamingTranslator());
TRANSLATORS.put(Read.Unbounded.class, new UnboundedReadSourceTranslator());
TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundStreamingTranslator());
@@ -79,7 +78,6 @@ public class FlinkStreamingTransformTranslators {
TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslator());
TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator());
TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiStreamingTranslator());
-
}
public static FlinkStreamingPipelineTranslator.StreamTransformTranslator<?> getTranslator(PTransform<?, ?> transform) {
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
index df68e50..7c4ab93 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkStreamingTranslationContext.java
@@ -18,7 +18,10 @@ package com.dataartisans.flink.dataflow.translation;
import com.google.cloud.dataflow.sdk.options.PipelineOptions;
import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.*;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import com.google.common.base.Preconditions;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -40,8 +43,8 @@ public class FlinkStreamingTranslationContext {
private AppliedPTransform<?, ?, ?> currentTransform;
public FlinkStreamingTranslationContext(StreamExecutionEnvironment env, PipelineOptions options) {
- this.env = env;
- this.options = options;
+ this.env = Preconditions.checkNotNull(env);
+ this.options = Preconditions.checkNotNull(options);
this.dataStreams = new HashMap<>();
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
index 71f9c7f..dfb2b7d 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
@@ -116,10 +116,10 @@ public abstract class FlinkAbstractParDoWrapper<IN, OUTDF, OUTFL> extends RichFl
@Override
public BoundedWindow window() {
-// if (!(fn instanceof DoFn.RequiresWindowAccess)) {
-// throw new UnsupportedOperationException(
-// "window() is only available in the context of a DoFn marked as RequiresWindow.");
-// }
+ if (!(fn instanceof DoFn.RequiresWindowAccess)) {
+ throw new UnsupportedOperationException(
+ "window() is only available in the context of a DoFn marked as RequiresWindow.");
+ }
Collection<? extends BoundedWindow> windows = this.element.getWindows();
if (windows.size() != 1) {
@@ -211,7 +211,7 @@ public abstract class FlinkAbstractParDoWrapper<IN, OUTDF, OUTFL> extends RichFl
@Override
public Object element() {
throw new UnsupportedOperationException(
- "WindowFn attempted to access input element when none was available"); // TODO: 12/16/15 aljoscha's comment in slack
+ "WindowFn attempted to access input element when none was available");
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
index 0f0a9d0..b78db65 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
@@ -238,9 +238,7 @@ public class FlinkGroupAlsoByWindowWrapper<K, VIN, VACC, VOUT>
this.operator = StreamingGroupAlsoByWindowsDoFn.createForIterable(
this.windowingStrategy, inputValueCoder);
} else {
-
Coder<K> inputKeyCoder = inputKvCoder.getKeyCoder();
- //CoderRegistry dataflowRegistry = input.getPipeline().getCoderRegistry();
AppliedCombineFn<K, VIN, VACC, VOUT> appliedCombineFn = AppliedCombineFn
.withInputCoder(combineFn, coderRegistry, inputKvCoder);
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
index b8824f5..c952d6f 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/FlinkStreamingCreateFunction.java
@@ -22,6 +22,7 @@ import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
import com.google.cloud.dataflow.sdk.util.WindowedValue;
import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
import java.io.ByteArrayInputStream;
import java.util.List;
@@ -44,17 +45,15 @@ public class FlinkStreamingCreateFunction<IN, OUT> implements FlatMapFunction<IN
public void flatMap(IN value, Collector<WindowedValue<OUT>> out) throws Exception {
@SuppressWarnings("unchecked")
- // TODO Flink doesn't allow null values in records
OUT voidValue = (OUT) VoidCoderTypeSerializer.VoidValue.INSTANCE;
-
for (byte[] element : elements) {
ByteArrayInputStream bai = new ByteArrayInputStream(element);
OUT outValue = coder.decode(bai, Coder.Context.OUTER);
if (outValue == null) {
- out.collect(WindowedValue.of(voidValue, GlobalWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
+ out.collect(WindowedValue.of(voidValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
} else {
- out.collect(WindowedValue.of(outValue, GlobalWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
+ out.collect(WindowedValue.of(outValue, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING));
}
}
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
index 3e248a6..cdc2e95 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
@@ -38,7 +38,7 @@ public class UnboundedSourceWrapper<T> extends RichSourceFunction<WindowedValue<
private final UnboundedSource.UnboundedReader<T> reader;
private StreamingRuntimeContext runtime = null;
- private StreamSource.ManualWatermarkContext<T> context = null;
+ private StreamSource.ManualWatermarkContext<WindowedValue<T>> context = null;
private volatile boolean isRunning = false;
@@ -51,8 +51,7 @@ public class UnboundedSourceWrapper<T> extends RichSourceFunction<WindowedValue<
return this.name;
}
- WindowedValue<T> makeWindowedValue(
- T output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+ WindowedValue<T> makeWindowedValue(T output, Instant timestamp) {
if (timestamp == null) {
timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
}
@@ -66,7 +65,7 @@ public class UnboundedSourceWrapper<T> extends RichSourceFunction<WindowedValue<
"Apparently " + this.name + " is not. Probably you should consider writing your own Wrapper for this source.");
}
- context = (StreamSource.ManualWatermarkContext<T>) ctx;
+ context = (StreamSource.ManualWatermarkContext<WindowedValue<T>>) ctx;
runtime = (StreamingRuntimeContext) getRuntimeContext();
this.isRunning = reader.start();
@@ -78,11 +77,9 @@ public class UnboundedSourceWrapper<T> extends RichSourceFunction<WindowedValue<
T item = reader.getCurrent();
Instant timestamp = reader.getCurrentTimestamp();
- long milliseconds = timestamp.getMillis();
-
// write it to the output collector
synchronized (ctx.getCheckpointLock()) {
- ctx.collectWithTimestamp(makeWindowedValue(item, timestamp, null, PaneInfo.NO_FIRING), milliseconds);
+ context.collectWithTimestamp(makeWindowedValue(item, timestamp), timestamp.getMillis());
}
// try to go to the next record
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java
index c6e3e99..2b1f091 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java
@@ -56,14 +56,14 @@ public class AvroITCase extends JavaProgramTestBase {
}
private static void runProgram(String tmpPath, String resultPath) {
- Pipeline p = FlinkTestPipeline.create();
+ Pipeline p = FlinkTestPipeline.createForBatch();
p.apply(Create.of(new User("Joe", 3, "red"), new User("Mary", 4, "blue")).withCoder(AvroCoder.of(User.class)))
.apply(AvroIO.Write.to(tmpPath).withSchema(User.class));
p.run();
- p = FlinkTestPipeline.create();
+ p = FlinkTestPipeline.createForBatch();
p.apply(AvroIO.Read.from(tmpPath).withSchema(User.class))
.apply(ParDo.of(new DoFn<User, String>() {
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java
index bc24514..928388c 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java
@@ -51,7 +51,7 @@ public class FlattenizeITCase extends JavaProgramTestBase {
@Override
protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.create();
+ Pipeline p = FlinkTestPipeline.createForBatch();
PCollection<String> p1 = p.apply(Create.of(words));
PCollection<String> p2 = p.apply(Create.of(words2));
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
index 109b1ff..56af3f1 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
@@ -32,7 +32,7 @@ public class FlinkTestPipeline extends Pipeline {
* <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
* {@link Pipeline#run} to execute the pipeline and check the tests.
*/
- public static FlinkTestPipeline create() {
+ public static FlinkTestPipeline createForBatch() {
return create(false);
}
@@ -44,7 +44,7 @@ public class FlinkTestPipeline extends Pipeline {
*
* @return The Test Pipeline
*/
- public static FlinkTestPipeline createStreaming() {
+ public static FlinkTestPipeline createForStreaming() {
return create(true);
}
@@ -54,18 +54,18 @@ public class FlinkTestPipeline extends Pipeline {
* <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
* {@link Pipeline#run} to execute the pipeline and check the tests.
*
- * @param streaming True for streaming mode, False for batch
- * @return The Test Pipeline
+ * @param streaming <code>True</code> for streaming mode, <code>False</code> for batch.
+ * @return The Test Pipeline.
*/
- public static FlinkTestPipeline create(boolean streaming) {
+ private static FlinkTestPipeline create(boolean streaming) {
FlinkPipelineRunner flinkRunner = FlinkPipelineRunner.createForTest(streaming);
FlinkPipelineOptions pipelineOptions = flinkRunner.getPipelineOptions();
pipelineOptions.setStreaming(streaming);
return new FlinkTestPipeline(flinkRunner, pipelineOptions);
}
- private FlinkTestPipeline(PipelineRunner<? extends PipelineResult> runner, PipelineOptions
- options) {
+ private FlinkTestPipeline(PipelineRunner<? extends PipelineResult> runner,
+ PipelineOptions options) {
super(runner, options);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java
index ed2ecf5..af0f217 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java
@@ -84,7 +84,7 @@ public class JoinExamplesITCase extends JavaProgramTestBase {
@Override
protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.create();
+ Pipeline p = FlinkTestPipeline.createForBatch();
PCollection<TableRow> input1 = p.apply(Create.of(EVENT_ARRAY));
PCollection<TableRow> input2 = p.apply(Create.of(CC_ARRAY));
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java
index 29c34d4..35f2eaf 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java
@@ -47,7 +47,7 @@ public class MaybeEmptyTestITCase extends JavaProgramTestBase implements Seriali
@Override
protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.create();
+ Pipeline p = FlinkTestPipeline.createForBatch();
p.apply(Create.of((Void) null)).setCoder(VoidCoder.of())
.apply(ParDo.of(
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java
index dbe88d2..ccdbbf9 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java
@@ -47,7 +47,7 @@ public class ParDoMultiOutputITCase extends JavaProgramTestBase implements Seria
@Override
protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.create();
+ Pipeline p = FlinkTestPipeline.createForBatch();
PCollection<String> words = p.apply(Create.of("Hello", "Whatupmyman", "hey", "SPECIALthere", "MAAA", "MAAFOOO"));
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java
index ba675b1..39f54e4 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java
@@ -61,7 +61,7 @@ public class ReadSourceITCase extends JavaProgramTestBase {
private static void runProgram(String resultPath) {
- Pipeline p = FlinkTestPipeline.create();
+ Pipeline p = FlinkTestPipeline.createForBatch();
PCollection<String> result = p
.apply(Read.from(new ReadSource(1, 10)))
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java
index ff59db7..db794f7 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java
@@ -52,7 +52,7 @@ public class RemoveDuplicatesEmptyITCase extends JavaProgramTestBase {
List<String> strings = Collections.emptyList();
- Pipeline p = FlinkTestPipeline.create();
+ Pipeline p = FlinkTestPipeline.createForBatch();
PCollection<String> input =
p.apply(Create.of(strings))
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java
index a8200aa..04e06b8 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java
@@ -53,7 +53,7 @@ public class RemoveDuplicatesITCase extends JavaProgramTestBase {
List<String> strings = Arrays.asList("k1", "k5", "k5", "k2", "k1", "k2", "k3");
- Pipeline p = FlinkTestPipeline.create();
+ Pipeline p = FlinkTestPipeline.createForBatch();
PCollection<String> input =
p.apply(Create.of(strings))
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java
index d932c80..ee8843c 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java
@@ -36,7 +36,7 @@ public class SideInputITCase extends JavaProgramTestBase implements Serializable
protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.create();
+ Pipeline p = FlinkTestPipeline.createForBatch();
final PCollectionView<String> sidesInput = p
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java
index e801ac4..1b4afb3 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java
@@ -53,7 +53,7 @@ public class TfIdfITCase extends JavaProgramTestBase {
@Override
protected void testProgram() throws Exception {
- Pipeline pipeline = FlinkTestPipeline.create();
+ Pipeline pipeline = FlinkTestPipeline.createForBatch();
pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java
deleted file mode 100644
index eb020c5..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TopWikipediaSessionsITCase.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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 com.dataartisans.flink.dataflow;
-
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
-import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Joiner;
-import org.apache.flink.streaming.util.StreamingProgramTestBase;
-import org.joda.time.Duration;
-import org.joda.time.Instant;
-
-import java.io.Serializable;
-import java.util.Arrays;
-
-
-/**
- * Session window test
- */
-public class TopWikipediaSessionsITCase extends StreamingProgramTestBase implements Serializable {
- protected String resultPath;
-
- public TopWikipediaSessionsITCase(){
- }
-
- static final String[] EXPECTED_RESULT = new String[] {
- "user: user1 value:3",
- "user: user1 value:1",
- "user: user2 value:4",
- "user: user2 value:6",
- "user: user3 value:7",
- "user: user3 value:2"
- };
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
-
- Pipeline p = FlinkTestPipeline.createStreaming();
-
- long now = System.currentTimeMillis() + 10000;
- System.out.println((now + 5000) / 1000);
-
- PCollection<KV<String, Long>> output =
- p.apply(Create.of(Arrays.asList(new TableRow().set("timestamp", now).set
- ("contributor_username", "user1"), new TableRow().set("timestamp", now + 10).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now).set
- ("contributor_username", "user1"), new TableRow().set("timestamp", now + 2).set
- ("contributor_username", "user1"), new TableRow().set("timestamp", now).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 1).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 5).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 7).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 8).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 200).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 230).set
- ("contributor_username", "user1"), new TableRow().set("timestamp", now + 230).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 240).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 245).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 235).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 236).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 237).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 238).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 239).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 240).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 241).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now)
- .set("contributor_username", "user3"))))
-
-
-
- .apply(ParDo.of(new DoFn<TableRow, String>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- TableRow row = c.element();
- long timestamp = (Long) row.get("timestamp");
- String userName = (String) row.get("contributor_username");
- if (userName != null) {
- // Sets the timestamp field to be used in windowing.
- c.outputWithTimestamp(userName, new Instant(timestamp * 1000L));
- }
- }
- }))
-
- .apply(ParDo.named("SampleUsers").of(
- new DoFn<String, String>() {
- private static final long serialVersionUID = 0;
-
- @Override
- public void processElement(ProcessContext c) {
- if (Math.abs(c.element().hashCode()) <= Integer.MAX_VALUE * 1.0) {
- c.output(c.element());
- }
- }
- }))
-
- .apply(Window.<String>into(Sessions.withGapDuration(Duration.standardMinutes(1))))
- .apply(Count.<String>perElement());
-
- PCollection<String> format = output.apply(ParDo.of(new DoFn<KV<String, Long>, String>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- KV<String, Long> el = c.element();
- String out = "user: " + el.getKey() + " value:" + el.getValue();
- System.out.println(out);
- c.output(out);
- }
- }));
-
- format.apply(TextIO.Write.to(resultPath));
-
- p.run();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java
index 9427ab6..5ddd379 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java
@@ -58,7 +58,7 @@ public class WordCountITCase extends JavaProgramTestBase {
@Override
protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.create();
+ Pipeline p = FlinkTestPipeline.createForBatch();
PCollection<String> input = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of());
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java
index c3eed61..ccc52c4 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java
@@ -70,7 +70,7 @@ public class WordCountJoin2ITCase extends JavaProgramTestBase {
@Override
protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.create();
+ Pipeline p = FlinkTestPipeline.createForBatch();
/* Create two PCollections and join them */
PCollection<KV<String,Long>> occurences1 = p.apply(Create.of(WORDS_1))
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java
index 33e67cc..e6eddc0 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin3ITCase.java
@@ -80,7 +80,7 @@ public class WordCountJoin3ITCase extends JavaProgramTestBase {
@Override
protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.create();
+ Pipeline p = FlinkTestPipeline.createForBatch();
/* Create two PCollections and join them */
PCollection<KV<String,Long>> occurences1 = p.apply(Create.of(WORDS_1))
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java
index 205fe9b..865fc5f 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WriteSinkITCase.java
@@ -63,7 +63,7 @@ public class WriteSinkITCase extends JavaProgramTestBase {
}
private static void runProgram(String resultPath) {
- Pipeline p = FlinkTestPipeline.create();
+ Pipeline p = FlinkTestPipeline.createForBatch();
p.apply(Create.of(EXPECTED_RESULT)).setCoder(StringUtf8Coder.of())
.apply("CustomSink", Write.to(new MyCustomSink(resultPath)));
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
index b667187..1f36ee7 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
@@ -86,7 +86,7 @@ public class GroupAlsoByWindowTest {
.withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES)
.withAllowedLateness(Duration.millis(1000));
long initialTime = 0L;
- Pipeline pipeline = FlinkTestPipeline.create();
+ Pipeline pipeline = FlinkTestPipeline.createForStreaming();
KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
@@ -145,7 +145,7 @@ public class GroupAlsoByWindowTest {
WindowingStrategy strategy = sessionWindowingStrategy;
long initialTime = 0L;
- Pipeline pipeline = FlinkTestPipeline.create();
+ Pipeline pipeline = FlinkTestPipeline.createForStreaming();
KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
@@ -382,7 +382,7 @@ public class GroupAlsoByWindowTest {
}
private OneInputStreamOperatorTestHarness createTestingOperatorAndState(WindowingStrategy strategy, long initialTime) throws Exception {
- Pipeline pipeline = FlinkTestPipeline.create();
+ Pipeline pipeline = FlinkTestPipeline.createForStreaming();
KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
@@ -478,7 +478,7 @@ public class GroupAlsoByWindowTest {
@Override
public Object element() {
throw new UnsupportedOperationException(
- "WindowFn attempted to access input element when none was available"); // TODO: 12/16/15 aljoscha's comment in slack
+ "WindowFn attempted to access input element when none was available");
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/bc4c60eb/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java
new file mode 100644
index 0000000..1c800fa
--- /dev/null
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java
@@ -0,0 +1,145 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.streaming;
+
+import com.dataartisans.flink.dataflow.FlinkTestPipeline;
+import com.google.api.services.bigquery.model.TableRow;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.transforms.Count;
+import com.google.cloud.dataflow.sdk.transforms.Create;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Sessions;
+import com.google.cloud.dataflow.sdk.transforms.windowing.Window;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.common.base.Joiner;
+import org.apache.flink.streaming.util.StreamingProgramTestBase;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+
+import java.io.Serializable;
+import java.util.Arrays;
+
+
+/**
+ * Session window test
+ */
+public class TopWikipediaSessionsITCase extends StreamingProgramTestBase implements Serializable {
+ protected String resultPath;
+
+ public TopWikipediaSessionsITCase(){
+ }
+
+ static final String[] EXPECTED_RESULT = new String[] {
+ "user: user1 value:3",
+ "user: user1 value:1",
+ "user: user2 value:4",
+ "user: user2 value:6",
+ "user: user3 value:7",
+ "user: user3 value:2"
+ };
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+
+ Pipeline p = FlinkTestPipeline.createForStreaming();
+
+ long now = System.currentTimeMillis() + 10000;
+ System.out.println((now + 5000) / 1000);
+
+ PCollection<KV<String, Long>> output =
+ p.apply(Create.of(Arrays.asList(new TableRow().set("timestamp", now).set
+ ("contributor_username", "user1"), new TableRow().set("timestamp", now + 10).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now).set
+ ("contributor_username", "user1"), new TableRow().set("timestamp", now + 2).set
+ ("contributor_username", "user1"), new TableRow().set("timestamp", now).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 1).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 5).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 7).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 8).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 200).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 230).set
+ ("contributor_username", "user1"), new TableRow().set("timestamp", now + 230).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 240).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 245).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 235).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 236).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 237).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 238).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 239).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 240).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 241).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now)
+ .set("contributor_username", "user3"))))
+
+
+
+ .apply(ParDo.of(new DoFn<TableRow, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ TableRow row = c.element();
+ long timestamp = (Long) row.get("timestamp");
+ String userName = (String) row.get("contributor_username");
+ if (userName != null) {
+ // Sets the timestamp field to be used in windowing.
+ c.outputWithTimestamp(userName, new Instant(timestamp * 1000L));
+ }
+ }
+ }))
+
+ .apply(ParDo.named("SampleUsers").of(
+ new DoFn<String, String>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ if (Math.abs(c.element().hashCode()) <= Integer.MAX_VALUE * 1.0) {
+ c.output(c.element());
+ }
+ }
+ }))
+
+ .apply(Window.<String>into(Sessions.withGapDuration(Duration.standardMinutes(1))))
+ .apply(Count.<String>perElement());
+
+ PCollection<String> format = output.apply(ParDo.of(new DoFn<KV<String, Long>, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ KV<String, Long> el = c.element();
+ String out = "user: " + el.getKey() + " value:" + el.getValue();
+ System.out.println(out);
+ c.output(out);
+ }
+ }));
+
+ format.apply(TextIO.Write.to(resultPath));
+
+ p.run();
+ }
+}
[39/50] [abbrv] incubator-beam git commit: [flink] convert tabs to 2
spaces
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java
index ad5b53a..90073c1 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/TopWikipediaSessionsITCase.java
@@ -40,93 +40,93 @@ import java.util.Arrays;
* Session window test
*/
public class TopWikipediaSessionsITCase extends StreamingProgramTestBase implements Serializable {
- protected String resultPath;
-
- public TopWikipediaSessionsITCase(){
- }
-
- static final String[] EXPECTED_RESULT = new String[] {
- "user: user1 value:3",
- "user: user1 value:1",
- "user: user2 value:4",
- "user: user2 value:6",
- "user: user3 value:7",
- "user: user3 value:2"
- };
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
-
- Pipeline p = FlinkTestPipeline.createForStreaming();
-
- Long now = (System.currentTimeMillis() + 10000) / 1000;
-
- PCollection<KV<String, Long>> output =
- p.apply(Create.of(Arrays.asList(new TableRow().set("timestamp", now).set
- ("contributor_username", "user1"), new TableRow().set("timestamp", now + 10).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now).set
- ("contributor_username", "user1"), new TableRow().set("timestamp", now + 2).set
- ("contributor_username", "user1"), new TableRow().set("timestamp", now).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 1).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 5).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 7).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 8).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 200).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 230).set
- ("contributor_username", "user1"), new TableRow().set("timestamp", now + 230).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 240).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now + 245).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 235).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 236).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 237).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 238).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 239).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 240).set
- ("contributor_username", "user3"), new TableRow().set("timestamp", now + 241).set
- ("contributor_username", "user2"), new TableRow().set("timestamp", now)
- .set("contributor_username", "user3"))))
-
-
-
- .apply(ParDo.of(new DoFn<TableRow, String>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- TableRow row = c.element();
- long timestamp = (Integer) row.get("timestamp");
- String userName = (String) row.get("contributor_username");
- if (userName != null) {
- // Sets the timestamp field to be used in windowing.
- c.outputWithTimestamp(userName, new Instant(timestamp * 1000L));
- }
- }
- }))
-
- .apply(Window.<String>into(Sessions.withGapDuration(Duration.standardMinutes(1))))
-
- .apply(Count.<String>perElement());
-
- PCollection<String> format = output.apply(ParDo.of(new DoFn<KV<String, Long>, String>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- KV<String, Long> el = c.element();
- String out = "user: " + el.getKey() + " value:" + el.getValue();
- c.output(out);
- }
- }));
-
- format.apply(TextIO.Write.to(resultPath));
-
- p.run();
- }
+ protected String resultPath;
+
+ public TopWikipediaSessionsITCase(){
+ }
+
+ static final String[] EXPECTED_RESULT = new String[] {
+ "user: user1 value:3",
+ "user: user1 value:1",
+ "user: user2 value:4",
+ "user: user2 value:6",
+ "user: user3 value:7",
+ "user: user3 value:2"
+ };
+
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
+
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
+
+ @Override
+ protected void testProgram() throws Exception {
+
+ Pipeline p = FlinkTestPipeline.createForStreaming();
+
+ Long now = (System.currentTimeMillis() + 10000) / 1000;
+
+ PCollection<KV<String, Long>> output =
+ p.apply(Create.of(Arrays.asList(new TableRow().set("timestamp", now).set
+ ("contributor_username", "user1"), new TableRow().set("timestamp", now + 10).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now).set
+ ("contributor_username", "user1"), new TableRow().set("timestamp", now + 2).set
+ ("contributor_username", "user1"), new TableRow().set("timestamp", now).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 1).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 5).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 7).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 8).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 200).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 230).set
+ ("contributor_username", "user1"), new TableRow().set("timestamp", now + 230).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 240).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now + 245).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 235).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 236).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 237).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 238).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 239).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 240).set
+ ("contributor_username", "user3"), new TableRow().set("timestamp", now + 241).set
+ ("contributor_username", "user2"), new TableRow().set("timestamp", now)
+ .set("contributor_username", "user3"))))
+
+
+
+ .apply(ParDo.of(new DoFn<TableRow, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ TableRow row = c.element();
+ long timestamp = (Integer) row.get("timestamp");
+ String userName = (String) row.get("contributor_username");
+ if (userName != null) {
+ // Sets the timestamp field to be used in windowing.
+ c.outputWithTimestamp(userName, new Instant(timestamp * 1000L));
+ }
+ }
+ }))
+
+ .apply(Window.<String>into(Sessions.withGapDuration(Duration.standardMinutes(1))))
+
+ .apply(Count.<String>perElement());
+
+ PCollection<String> format = output.apply(ParDo.of(new DoFn<KV<String, Long>, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ KV<String, Long> el = c.element();
+ String out = "user: " + el.getKey() + " value:" + el.getValue();
+ c.output(out);
+ }
+ }));
+
+ format.apply(TextIO.Write.to(resultPath));
+
+ p.run();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java
index aa5623d..b1ccee4 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/util/JoinExamples.java
@@ -38,121 +38,121 @@ import com.google.cloud.dataflow.sdk.values.TupleTag;
*/
public class JoinExamples {
- // A 1000-row sample of the GDELT data here: gdelt-bq:full.events.
- private static final String GDELT_EVENTS_TABLE =
- "clouddataflow-readonly:samples.gdelt_sample";
- // A table that maps country codes to country names.
- private static final String COUNTRY_CODES =
- "gdelt-bq:full.crosswalk_geocountrycodetohuman";
-
- /**
- * Join two collections, using country code as the key.
- */
- public static PCollection<String> joinEvents(PCollection<TableRow> eventsTable,
- PCollection<TableRow> countryCodes) throws Exception {
-
- final TupleTag<String> eventInfoTag = new TupleTag<>();
- final TupleTag<String> countryInfoTag = new TupleTag<>();
-
- // transform both input collections to tuple collections, where the keys are country
- // codes in both cases.
- PCollection<KV<String, String>> eventInfo = eventsTable.apply(
- ParDo.of(new ExtractEventDataFn()));
- PCollection<KV<String, String>> countryInfo = countryCodes.apply(
- ParDo.of(new ExtractCountryInfoFn()));
-
- // country code 'key' -> CGBKR (<event info>, <country name>)
- PCollection<KV<String, CoGbkResult>> kvpCollection = KeyedPCollectionTuple
- .of(eventInfoTag, eventInfo)
- .and(countryInfoTag, countryInfo)
- .apply(CoGroupByKey.<String>create());
-
- // Process the CoGbkResult elements generated by the CoGroupByKey transform.
- // country code 'key' -> string of <event info>, <country name>
- PCollection<KV<String, String>> finalResultCollection =
- kvpCollection.apply(ParDo.of(new DoFn<KV<String, CoGbkResult>, KV<String, String>>() {
- @Override
- public void processElement(ProcessContext c) {
- KV<String, CoGbkResult> e = c.element();
- CoGbkResult val = e.getValue();
- String countryCode = e.getKey();
- String countryName;
- countryName = e.getValue().getOnly(countryInfoTag, "Kostas");
- for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) {
- // Generate a string that combines information from both collection values
- c.output(KV.of(countryCode, "Country name: " + countryName
- + ", Event info: " + eventInfo));
- }
- }
- }));
-
- // write to GCS
- return finalResultCollection
- .apply(ParDo.of(new DoFn<KV<String, String>, String>() {
- @Override
- public void processElement(ProcessContext c) {
- String outputstring = "Country code: " + c.element().getKey()
- + ", " + c.element().getValue();
- c.output(outputstring);
- }
- }));
- }
-
- /**
- * Examines each row (event) in the input table. Output a KV with the key the country
- * code of the event, and the value a string encoding event information.
- */
- static class ExtractEventDataFn extends DoFn<TableRow, KV<String, String>> {
- @Override
- public void processElement(ProcessContext c) {
- TableRow row = c.element();
- String countryCode = (String) row.get("ActionGeo_CountryCode");
- String sqlDate = (String) row.get("SQLDATE");
- String actor1Name = (String) row.get("Actor1Name");
- String sourceUrl = (String) row.get("SOURCEURL");
- String eventInfo = "Date: " + sqlDate + ", Actor1: " + actor1Name + ", url: " + sourceUrl;
- c.output(KV.of(countryCode, eventInfo));
- }
- }
-
-
- /**
- * Examines each row (country info) in the input table. Output a KV with the key the country
- * code, and the value the country name.
- */
- static class ExtractCountryInfoFn extends DoFn<TableRow, KV<String, String>> {
- @Override
- public void processElement(ProcessContext c) {
- TableRow row = c.element();
- String countryCode = (String) row.get("FIPSCC");
- String countryName = (String) row.get("HumanName");
- c.output(KV.of(countryCode, countryName));
- }
- }
-
-
- /**
- * Options supported by {@link JoinExamples}.
- * <p>
- * Inherits standard configuration options.
- */
- private interface Options extends PipelineOptions {
- @Description("Path of the file to write to")
- @Validation.Required
- String getOutput();
- void setOutput(String value);
- }
-
- public static void main(String[] args) throws Exception {
- Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
- Pipeline p = Pipeline.create(options);
- // the following two 'applys' create multiple inputs to our pipeline, one for each
- // of our two input sources.
- PCollection<TableRow> eventsTable = p.apply(BigQueryIO.Read.from(GDELT_EVENTS_TABLE));
- PCollection<TableRow> countryCodes = p.apply(BigQueryIO.Read.from(COUNTRY_CODES));
- PCollection<String> formattedResults = joinEvents(eventsTable, countryCodes);
- formattedResults.apply(TextIO.Write.to(options.getOutput()));
- p.run();
- }
+ // A 1000-row sample of the GDELT data here: gdelt-bq:full.events.
+ private static final String GDELT_EVENTS_TABLE =
+ "clouddataflow-readonly:samples.gdelt_sample";
+ // A table that maps country codes to country names.
+ private static final String COUNTRY_CODES =
+ "gdelt-bq:full.crosswalk_geocountrycodetohuman";
+
+ /**
+ * Join two collections, using country code as the key.
+ */
+ public static PCollection<String> joinEvents(PCollection<TableRow> eventsTable,
+ PCollection<TableRow> countryCodes) throws Exception {
+
+ final TupleTag<String> eventInfoTag = new TupleTag<>();
+ final TupleTag<String> countryInfoTag = new TupleTag<>();
+
+ // transform both input collections to tuple collections, where the keys are country
+ // codes in both cases.
+ PCollection<KV<String, String>> eventInfo = eventsTable.apply(
+ ParDo.of(new ExtractEventDataFn()));
+ PCollection<KV<String, String>> countryInfo = countryCodes.apply(
+ ParDo.of(new ExtractCountryInfoFn()));
+
+ // country code 'key' -> CGBKR (<event info>, <country name>)
+ PCollection<KV<String, CoGbkResult>> kvpCollection = KeyedPCollectionTuple
+ .of(eventInfoTag, eventInfo)
+ .and(countryInfoTag, countryInfo)
+ .apply(CoGroupByKey.<String>create());
+
+ // Process the CoGbkResult elements generated by the CoGroupByKey transform.
+ // country code 'key' -> string of <event info>, <country name>
+ PCollection<KV<String, String>> finalResultCollection =
+ kvpCollection.apply(ParDo.of(new DoFn<KV<String, CoGbkResult>, KV<String, String>>() {
+ @Override
+ public void processElement(ProcessContext c) {
+ KV<String, CoGbkResult> e = c.element();
+ CoGbkResult val = e.getValue();
+ String countryCode = e.getKey();
+ String countryName;
+ countryName = e.getValue().getOnly(countryInfoTag, "Kostas");
+ for (String eventInfo : c.element().getValue().getAll(eventInfoTag)) {
+ // Generate a string that combines information from both collection values
+ c.output(KV.of(countryCode, "Country name: " + countryName
+ + ", Event info: " + eventInfo));
+ }
+ }
+ }));
+
+ // write to GCS
+ return finalResultCollection
+ .apply(ParDo.of(new DoFn<KV<String, String>, String>() {
+ @Override
+ public void processElement(ProcessContext c) {
+ String outputstring = "Country code: " + c.element().getKey()
+ + ", " + c.element().getValue();
+ c.output(outputstring);
+ }
+ }));
+ }
+
+ /**
+ * Examines each row (event) in the input table. Output a KV with the key the country
+ * code of the event, and the value a string encoding event information.
+ */
+ static class ExtractEventDataFn extends DoFn<TableRow, KV<String, String>> {
+ @Override
+ public void processElement(ProcessContext c) {
+ TableRow row = c.element();
+ String countryCode = (String) row.get("ActionGeo_CountryCode");
+ String sqlDate = (String) row.get("SQLDATE");
+ String actor1Name = (String) row.get("Actor1Name");
+ String sourceUrl = (String) row.get("SOURCEURL");
+ String eventInfo = "Date: " + sqlDate + ", Actor1: " + actor1Name + ", url: " + sourceUrl;
+ c.output(KV.of(countryCode, eventInfo));
+ }
+ }
+
+
+ /**
+ * Examines each row (country info) in the input table. Output a KV with the key the country
+ * code, and the value the country name.
+ */
+ static class ExtractCountryInfoFn extends DoFn<TableRow, KV<String, String>> {
+ @Override
+ public void processElement(ProcessContext c) {
+ TableRow row = c.element();
+ String countryCode = (String) row.get("FIPSCC");
+ String countryName = (String) row.get("HumanName");
+ c.output(KV.of(countryCode, countryName));
+ }
+ }
+
+
+ /**
+ * Options supported by {@link JoinExamples}.
+ * <p>
+ * Inherits standard configuration options.
+ */
+ private interface Options extends PipelineOptions {
+ @Description("Path of the file to write to")
+ @Validation.Required
+ String getOutput();
+ void setOutput(String value);
+ }
+
+ public static void main(String[] args) throws Exception {
+ Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+ Pipeline p = Pipeline.create(options);
+ // the following two 'applys' create multiple inputs to our pipeline, one for each
+ // of our two input sources.
+ PCollection<TableRow> eventsTable = p.apply(BigQueryIO.Read.from(GDELT_EVENTS_TABLE));
+ PCollection<TableRow> countryCodes = p.apply(BigQueryIO.Read.from(COUNTRY_CODES));
+ PCollection<String> formattedResults = joinEvents(eventsTable, countryCodes);
+ formattedResults.apply(TextIO.Write.to(options.getOutput()));
+ p.run();
+ }
}
[27/50] [abbrv] incubator-beam git commit: [flink] adjust directories
according to package name
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
deleted file mode 100644
index 7accf09..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/AbstractFlinkTimerInternals.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers.streaming.state;
-
-import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-import com.google.cloud.dataflow.sdk.util.WindowedValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import org.joda.time.Instant;
-
-import javax.annotation.Nullable;
-import java.io.IOException;
-import java.io.Serializable;
-
-/**
- * An implementation of Beam's {@link TimerInternals}, that also provides serialization functionality.
- * The latter is used when snapshots of the current state are taken, for fault-tolerance.
- * */
-public abstract class AbstractFlinkTimerInternals<K, VIN> implements TimerInternals, Serializable {
- private Instant currentInputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE;
- private Instant currentOutputWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE;
-
- public void setCurrentInputWatermark(Instant watermark) {
- checkIfValidInputWatermark(watermark);
- this.currentInputWatermark = watermark;
- }
-
- public void setCurrentOutputWatermark(Instant watermark) {
- checkIfValidOutputWatermark(watermark);
- this.currentOutputWatermark = watermark;
- }
-
- private void setCurrentInputWatermarkAfterRecovery(Instant watermark) {
- if (!currentInputWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) {
- throw new RuntimeException("Explicitly setting the input watermark is only allowed on " +
- "initialization after recovery from a node failure. Apparently this is not " +
- "the case here as the watermark is already set.");
- }
- this.currentInputWatermark = watermark;
- }
-
- private void setCurrentOutputWatermarkAfterRecovery(Instant watermark) {
- if (!currentOutputWatermark.isEqual(BoundedWindow.TIMESTAMP_MIN_VALUE)) {
- throw new RuntimeException("Explicitly setting the output watermark is only allowed on " +
- "initialization after recovery from a node failure. Apparently this is not " +
- "the case here as the watermark is already set.");
- }
- this.currentOutputWatermark = watermark;
- }
-
- @Override
- public Instant currentProcessingTime() {
- return Instant.now();
- }
-
- @Override
- public Instant currentInputWatermarkTime() {
- return currentInputWatermark;
- }
-
- @Nullable
- @Override
- public Instant currentSynchronizedProcessingTime() {
- // TODO
- return null;
- }
-
- @Override
- public Instant currentOutputWatermarkTime() {
- return currentOutputWatermark;
- }
-
- private void checkIfValidInputWatermark(Instant newWatermark) {
- if (currentInputWatermark.isAfter(newWatermark)) {
- throw new IllegalArgumentException(String.format(
- "Cannot set current input watermark to %s. Newer watermarks " +
- "must be no earlier than the current one (%s).",
- newWatermark, currentInputWatermark));
- }
- }
-
- private void checkIfValidOutputWatermark(Instant newWatermark) {
- if (currentOutputWatermark.isAfter(newWatermark)) {
- throw new IllegalArgumentException(String.format(
- "Cannot set current output watermark to %s. Newer watermarks " +
- "must be no earlier than the current one (%s).",
- newWatermark, currentOutputWatermark));
- }
- }
-
- public void encodeTimerInternals(DoFn.ProcessContext context,
- StateCheckpointWriter writer,
- KvCoder<K, VIN> kvCoder,
- Coder<? extends BoundedWindow> windowCoder) throws IOException {
- if (context == null) {
- throw new RuntimeException("The Context has not been initialized.");
- }
-
- writer.setTimestamp(currentInputWatermark);
- writer.setTimestamp(currentOutputWatermark);
- }
-
- public void restoreTimerInternals(StateCheckpointReader reader,
- KvCoder<K, VIN> kvCoder,
- Coder<? extends BoundedWindow> windowCoder) throws IOException {
- setCurrentInputWatermarkAfterRecovery(reader.getTimestamp());
- setCurrentOutputWatermarkAfterRecovery(reader.getTimestamp());
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java
deleted file mode 100644
index 84007af..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/FlinkStateInternals.java
+++ /dev/null
@@ -1,713 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers.streaming.state;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.CombineWithContext;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
-import com.google.cloud.dataflow.sdk.util.state.*;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.common.base.Preconditions;
-import com.google.protobuf.ByteString;
-import org.apache.flink.util.InstantiationUtil;
-import org.joda.time.Instant;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.util.*;
-
-/**
- * An implementation of the Beam {@link StateInternals}. This implementation simply keeps elements in memory.
- * This state is periodically checkpointed by Flink, for fault-tolerance.
- *
- * TODO: State should be rewritten to redirect to Flink per-key state so that coders and combiners don't need
- * to be serialized along with encoded values when snapshotting.
- */
-public class FlinkStateInternals<K> implements StateInternals<K> {
-
- private final K key;
-
- private final Coder<K> keyCoder;
-
- private final Coder<? extends BoundedWindow> windowCoder;
-
- private final OutputTimeFn<? super BoundedWindow> outputTimeFn;
-
- private Instant watermarkHoldAccessor;
-
- public FlinkStateInternals(K key,
- Coder<K> keyCoder,
- Coder<? extends BoundedWindow> windowCoder,
- OutputTimeFn<? super BoundedWindow> outputTimeFn) {
- this.key = key;
- this.keyCoder = keyCoder;
- this.windowCoder = windowCoder;
- this.outputTimeFn = outputTimeFn;
- }
-
- public Instant getWatermarkHold() {
- return watermarkHoldAccessor;
- }
-
- /**
- * This is the interface state has to implement in order for it to be fault tolerant when
- * executed by the FlinkPipelineRunner.
- */
- private interface CheckpointableIF {
-
- boolean shouldPersist();
-
- void persistState(StateCheckpointWriter checkpointBuilder) throws IOException;
- }
-
- protected final StateTable<K> inMemoryState = new StateTable<K>() {
- @Override
- protected StateTag.StateBinder binderForNamespace(final StateNamespace namespace, final StateContext<?> c) {
- return new StateTag.StateBinder<K>() {
-
- @Override
- public <T> ValueState<T> bindValue(StateTag<? super K, ValueState<T>> address, Coder<T> coder) {
- return new FlinkInMemoryValue<>(encodeKey(namespace, address), coder);
- }
-
- @Override
- public <T> BagState<T> bindBag(StateTag<? super K, BagState<T>> address, Coder<T> elemCoder) {
- return new FlinkInMemoryBag<>(encodeKey(namespace, address), elemCoder);
- }
-
- @Override
- public <InputT, AccumT, OutputT> AccumulatorCombiningState<InputT, AccumT, OutputT> bindCombiningValue(
- StateTag<? super K, AccumulatorCombiningState<InputT, AccumT, OutputT>> address,
- Coder<AccumT> accumCoder, Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
- return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c);
- }
-
- @Override
- public <InputT, AccumT, OutputT> AccumulatorCombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValue(
- StateTag<? super K, AccumulatorCombiningState<InputT, AccumT, OutputT>> address,
- Coder<AccumT> accumCoder,
- Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn) {
- return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c);
- }
-
- @Override
- public <InputT, AccumT, OutputT> AccumulatorCombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValueWithContext(
- StateTag<? super K, AccumulatorCombiningState<InputT, AccumT, OutputT>> address,
- Coder<AccumT> accumCoder,
- CombineWithContext.KeyedCombineFnWithContext<? super K, InputT, AccumT, OutputT> combineFn) {
- return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c);
- }
-
- @Override
- public <W extends BoundedWindow> WatermarkHoldState<W> bindWatermark(StateTag<? super K, WatermarkHoldState<W>> address, OutputTimeFn<? super W> outputTimeFn) {
- return new FlinkWatermarkHoldStateImpl<>(encodeKey(namespace, address), outputTimeFn);
- }
- };
- }
- };
-
- @Override
- public K getKey() {
- return key;
- }
-
- @Override
- public <StateT extends State> StateT state(StateNamespace namespace, StateTag<? super K, StateT> address) {
- return inMemoryState.get(namespace, address, null);
- }
-
- @Override
- public <T extends State> T state(StateNamespace namespace, StateTag<? super K, T> address, StateContext<?> c) {
- return inMemoryState.get(namespace, address, c);
- }
-
- public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
- checkpointBuilder.writeInt(getNoOfElements());
-
- for (State location : inMemoryState.values()) {
- if (!(location instanceof CheckpointableIF)) {
- throw new IllegalStateException(String.format(
- "%s wasn't created by %s -- unable to persist it",
- location.getClass().getSimpleName(),
- getClass().getSimpleName()));
- }
- ((CheckpointableIF) location).persistState(checkpointBuilder);
- }
- }
-
- public void restoreState(StateCheckpointReader checkpointReader, ClassLoader loader)
- throws IOException, ClassNotFoundException {
-
- // the number of elements to read.
- int noOfElements = checkpointReader.getInt();
- for (int i = 0; i < noOfElements; i++) {
- decodeState(checkpointReader, loader);
- }
- }
-
- /**
- * We remove the first character which encodes the type of the stateTag ('s' for system
- * and 'u' for user). For more details check out the source of
- * {@link StateTags.StateTagBase#getId()}.
- */
- private void decodeState(StateCheckpointReader reader, ClassLoader loader)
- throws IOException, ClassNotFoundException {
-
- StateType stateItemType = StateType.deserialize(reader);
- ByteString stateKey = reader.getTag();
-
- // first decode the namespace and the tagId...
- String[] namespaceAndTag = stateKey.toStringUtf8().split("\\+");
- if (namespaceAndTag.length != 2) {
- throw new IllegalArgumentException("Invalid stateKey " + stateKey.toString() + ".");
- }
- StateNamespace namespace = StateNamespaces.fromString(namespaceAndTag[0], windowCoder);
-
- // ... decide if it is a system or user stateTag...
- char ownerTag = namespaceAndTag[1].charAt(0);
- if (ownerTag != 's' && ownerTag != 'u') {
- throw new RuntimeException("Invalid StateTag name.");
- }
- boolean isSystemTag = ownerTag == 's';
- String tagId = namespaceAndTag[1].substring(1);
-
- // ...then decode the coder (if there is one)...
- Coder<?> coder = null;
- switch (stateItemType) {
- case VALUE:
- case LIST:
- case ACCUMULATOR:
- ByteString coderBytes = reader.getData();
- coder = InstantiationUtil.deserializeObject(coderBytes.toByteArray(), loader);
- break;
- case WATERMARK:
- break;
- }
-
- // ...then decode the combiner function (if there is one)...
- CombineWithContext.KeyedCombineFnWithContext<? super K, ?, ?, ?> combineFn = null;
- switch (stateItemType) {
- case ACCUMULATOR:
- ByteString combinerBytes = reader.getData();
- combineFn = InstantiationUtil.deserializeObject(combinerBytes.toByteArray(), loader);
- break;
- case VALUE:
- case LIST:
- case WATERMARK:
- break;
- }
-
- //... and finally, depending on the type of the state being decoded,
- // 1) create the adequate stateTag,
- // 2) create the state container,
- // 3) restore the actual content.
- switch (stateItemType) {
- case VALUE: {
- StateTag stateTag = StateTags.value(tagId, coder);
- stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
- @SuppressWarnings("unchecked")
- FlinkInMemoryValue<?> value = (FlinkInMemoryValue<?>) inMemoryState.get(namespace, stateTag, null);
- value.restoreState(reader);
- break;
- }
- case WATERMARK: {
- @SuppressWarnings("unchecked")
- StateTag<Object, WatermarkHoldState<BoundedWindow>> stateTag = StateTags.watermarkStateInternal(tagId, outputTimeFn);
- stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
- @SuppressWarnings("unchecked")
- FlinkWatermarkHoldStateImpl<?> watermark = (FlinkWatermarkHoldStateImpl<?>) inMemoryState.get(namespace, stateTag, null);
- watermark.restoreState(reader);
- break;
- }
- case LIST: {
- StateTag stateTag = StateTags.bag(tagId, coder);
- stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
- FlinkInMemoryBag<?> bag = (FlinkInMemoryBag<?>) inMemoryState.get(namespace, stateTag, null);
- bag.restoreState(reader);
- break;
- }
- case ACCUMULATOR: {
- @SuppressWarnings("unchecked")
- StateTag<K, AccumulatorCombiningState<?, ?, ?>> stateTag = StateTags.keyedCombiningValueWithContext(tagId, (Coder) coder, combineFn);
- stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
- @SuppressWarnings("unchecked")
- FlinkInMemoryKeyedCombiningValue<?, ?, ?> combiningValue =
- (FlinkInMemoryKeyedCombiningValue<?, ?, ?>) inMemoryState.get(namespace, stateTag, null);
- combiningValue.restoreState(reader);
- break;
- }
- default:
- throw new RuntimeException("Unknown State Type " + stateItemType + ".");
- }
- }
-
- private ByteString encodeKey(StateNamespace namespace, StateTag<? super K, ?> address) {
- StringBuilder sb = new StringBuilder();
- try {
- namespace.appendTo(sb);
- sb.append('+');
- address.appendTo(sb);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- return ByteString.copyFromUtf8(sb.toString());
- }
-
- private int getNoOfElements() {
- int noOfElements = 0;
- for (State state : inMemoryState.values()) {
- if (!(state instanceof CheckpointableIF)) {
- throw new RuntimeException("State Implementations used by the " +
- "Flink Dataflow Runner should implement the CheckpointableIF interface.");
- }
-
- if (((CheckpointableIF) state).shouldPersist()) {
- noOfElements++;
- }
- }
- return noOfElements;
- }
-
- private final class FlinkInMemoryValue<T> implements ValueState<T>, CheckpointableIF {
-
- private final ByteString stateKey;
- private final Coder<T> elemCoder;
-
- private T value = null;
-
- public FlinkInMemoryValue(ByteString stateKey, Coder<T> elemCoder) {
- this.stateKey = stateKey;
- this.elemCoder = elemCoder;
- }
-
- @Override
- public void clear() {
- value = null;
- }
-
- @Override
- public void write(T input) {
- this.value = input;
- }
-
- @Override
- public T read() {
- return value;
- }
-
- @Override
- public ValueState<T> readLater() {
- // Ignore
- return this;
- }
-
- @Override
- public boolean shouldPersist() {
- return value != null;
- }
-
- @Override
- public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
- if (value != null) {
- // serialize the coder.
- byte[] coder = InstantiationUtil.serializeObject(elemCoder);
-
- // encode the value into a ByteString
- ByteString.Output stream = ByteString.newOutput();
- elemCoder.encode(value, stream, Coder.Context.OUTER);
- ByteString data = stream.toByteString();
-
- checkpointBuilder.addValueBuilder()
- .setTag(stateKey)
- .setData(coder)
- .setData(data);
- }
- }
-
- public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
- ByteString valueContent = checkpointReader.getData();
- T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER);
- write(outValue);
- }
- }
-
- private final class FlinkWatermarkHoldStateImpl<W extends BoundedWindow>
- implements WatermarkHoldState<W>, CheckpointableIF {
-
- private final ByteString stateKey;
-
- private Instant minimumHold = null;
-
- private OutputTimeFn<? super W> outputTimeFn;
-
- public FlinkWatermarkHoldStateImpl(ByteString stateKey, OutputTimeFn<? super W> outputTimeFn) {
- this.stateKey = stateKey;
- this.outputTimeFn = outputTimeFn;
- }
-
- @Override
- public void clear() {
- // Even though we're clearing we can't remove this from the in-memory state map, since
- // other users may already have a handle on this WatermarkBagInternal.
- minimumHold = null;
- watermarkHoldAccessor = null;
- }
-
- @Override
- public void add(Instant watermarkHold) {
- if (minimumHold == null || minimumHold.isAfter(watermarkHold)) {
- watermarkHoldAccessor = watermarkHold;
- minimumHold = watermarkHold;
- }
- }
-
- @Override
- public ReadableState<Boolean> isEmpty() {
- return new ReadableState<Boolean>() {
- @Override
- public Boolean read() {
- return minimumHold == null;
- }
-
- @Override
- public ReadableState<Boolean> readLater() {
- // Ignore
- return this;
- }
- };
- }
-
- @Override
- public OutputTimeFn<? super W> getOutputTimeFn() {
- return outputTimeFn;
- }
-
- @Override
- public Instant read() {
- return minimumHold;
- }
-
- @Override
- public WatermarkHoldState<W> readLater() {
- // Ignore
- return this;
- }
-
- @Override
- public String toString() {
- return Objects.toString(minimumHold);
- }
-
- @Override
- public boolean shouldPersist() {
- return minimumHold != null;
- }
-
- @Override
- public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
- if (minimumHold != null) {
- checkpointBuilder.addWatermarkHoldsBuilder()
- .setTag(stateKey)
- .setTimestamp(minimumHold);
- }
- }
-
- public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
- Instant watermark = checkpointReader.getTimestamp();
- add(watermark);
- }
- }
-
-
- private static <K, InputT, AccumT, OutputT> CombineWithContext.KeyedCombineFnWithContext<K, InputT, AccumT, OutputT> withContext(
- final Combine.KeyedCombineFn<K, InputT, AccumT, OutputT> combineFn) {
- return new CombineWithContext.KeyedCombineFnWithContext<K, InputT, AccumT, OutputT>() {
- @Override
- public AccumT createAccumulator(K key, CombineWithContext.Context c) {
- return combineFn.createAccumulator(key);
- }
-
- @Override
- public AccumT addInput(K key, AccumT accumulator, InputT value, CombineWithContext.Context c) {
- return combineFn.addInput(key, accumulator, value);
- }
-
- @Override
- public AccumT mergeAccumulators(K key, Iterable<AccumT> accumulators, CombineWithContext.Context c) {
- return combineFn.mergeAccumulators(key, accumulators);
- }
-
- @Override
- public OutputT extractOutput(K key, AccumT accumulator, CombineWithContext.Context c) {
- return combineFn.extractOutput(key, accumulator);
- }
- };
- }
-
- private static <K, InputT, AccumT, OutputT> CombineWithContext.KeyedCombineFnWithContext<K, InputT, AccumT, OutputT> withKeyAndContext(
- final Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
- return new CombineWithContext.KeyedCombineFnWithContext<K, InputT, AccumT, OutputT>() {
- @Override
- public AccumT createAccumulator(K key, CombineWithContext.Context c) {
- return combineFn.createAccumulator();
- }
-
- @Override
- public AccumT addInput(K key, AccumT accumulator, InputT value, CombineWithContext.Context c) {
- return combineFn.addInput(accumulator, value);
- }
-
- @Override
- public AccumT mergeAccumulators(K key, Iterable<AccumT> accumulators, CombineWithContext.Context c) {
- return combineFn.mergeAccumulators(accumulators);
- }
-
- @Override
- public OutputT extractOutput(K key, AccumT accumulator, CombineWithContext.Context c) {
- return combineFn.extractOutput(accumulator);
- }
- };
- }
-
- private final class FlinkInMemoryKeyedCombiningValue<InputT, AccumT, OutputT>
- implements AccumulatorCombiningState<InputT, AccumT, OutputT>, CheckpointableIF {
-
- private final ByteString stateKey;
- private final CombineWithContext.KeyedCombineFnWithContext<? super K, InputT, AccumT, OutputT> combineFn;
- private final Coder<AccumT> accumCoder;
- private final CombineWithContext.Context context;
-
- private AccumT accum = null;
- private boolean isClear = true;
-
- private FlinkInMemoryKeyedCombiningValue(ByteString stateKey,
- Combine.CombineFn<InputT, AccumT, OutputT> combineFn,
- Coder<AccumT> accumCoder,
- final StateContext<?> stateContext) {
- this(stateKey, withKeyAndContext(combineFn), accumCoder, stateContext);
- }
-
-
- private FlinkInMemoryKeyedCombiningValue(ByteString stateKey,
- Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn,
- Coder<AccumT> accumCoder,
- final StateContext<?> stateContext) {
- this(stateKey, withContext(combineFn), accumCoder, stateContext);
- }
-
- private FlinkInMemoryKeyedCombiningValue(ByteString stateKey,
- CombineWithContext.KeyedCombineFnWithContext<? super K, InputT, AccumT, OutputT> combineFn,
- Coder<AccumT> accumCoder,
- final StateContext<?> stateContext) {
- Preconditions.checkNotNull(combineFn);
- Preconditions.checkNotNull(accumCoder);
-
- this.stateKey = stateKey;
- this.combineFn = combineFn;
- this.accumCoder = accumCoder;
- this.context = new CombineWithContext.Context() {
- @Override
- public PipelineOptions getPipelineOptions() {
- return stateContext.getPipelineOptions();
- }
-
- @Override
- public <T> T sideInput(PCollectionView<T> view) {
- return stateContext.sideInput(view);
- }
- };
- accum = combineFn.createAccumulator(key, context);
- }
-
- @Override
- public void clear() {
- accum = combineFn.createAccumulator(key, context);
- isClear = true;
- }
-
- @Override
- public void add(InputT input) {
- isClear = false;
- accum = combineFn.addInput(key, accum, input, context);
- }
-
- @Override
- public AccumT getAccum() {
- return accum;
- }
-
- @Override
- public ReadableState<Boolean> isEmpty() {
- return new ReadableState<Boolean>() {
- @Override
- public ReadableState<Boolean> readLater() {
- // Ignore
- return this;
- }
-
- @Override
- public Boolean read() {
- return isClear;
- }
- };
- }
-
- @Override
- public void addAccum(AccumT accum) {
- isClear = false;
- this.accum = combineFn.mergeAccumulators(key, Arrays.asList(this.accum, accum), context);
- }
-
- @Override
- public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
- return combineFn.mergeAccumulators(key, accumulators, context);
- }
-
- @Override
- public OutputT read() {
- return combineFn.extractOutput(key, accum, context);
- }
-
- @Override
- public AccumulatorCombiningState<InputT, AccumT, OutputT> readLater() {
- // Ignore
- return this;
- }
-
- @Override
- public boolean shouldPersist() {
- return !isClear;
- }
-
- @Override
- public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
- if (!isClear) {
- // serialize the coder.
- byte[] coder = InstantiationUtil.serializeObject(accumCoder);
-
- // serialize the combiner.
- byte[] combiner = InstantiationUtil.serializeObject(combineFn);
-
- // encode the accumulator into a ByteString
- ByteString.Output stream = ByteString.newOutput();
- accumCoder.encode(accum, stream, Coder.Context.OUTER);
- ByteString data = stream.toByteString();
-
- // put the flag that the next serialized element is an accumulator
- checkpointBuilder.addAccumulatorBuilder()
- .setTag(stateKey)
- .setData(coder)
- .setData(combiner)
- .setData(data);
- }
- }
-
- public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
- ByteString valueContent = checkpointReader.getData();
- AccumT accum = this.accumCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER);
- addAccum(accum);
- }
- }
-
- private static final class FlinkInMemoryBag<T> implements BagState<T>, CheckpointableIF {
- private final List<T> contents = new ArrayList<>();
-
- private final ByteString stateKey;
- private final Coder<T> elemCoder;
-
- public FlinkInMemoryBag(ByteString stateKey, Coder<T> elemCoder) {
- this.stateKey = stateKey;
- this.elemCoder = elemCoder;
- }
-
- @Override
- public void clear() {
- contents.clear();
- }
-
- @Override
- public Iterable<T> read() {
- return contents;
- }
-
- @Override
- public BagState<T> readLater() {
- // Ignore
- return this;
- }
-
- @Override
- public void add(T input) {
- contents.add(input);
- }
-
- @Override
- public ReadableState<Boolean> isEmpty() {
- return new ReadableState<Boolean>() {
- @Override
- public ReadableState<Boolean> readLater() {
- // Ignore
- return this;
- }
-
- @Override
- public Boolean read() {
- return contents.isEmpty();
- }
- };
- }
-
- @Override
- public boolean shouldPersist() {
- return !contents.isEmpty();
- }
-
- @Override
- public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
- if (!contents.isEmpty()) {
- // serialize the coder.
- byte[] coder = InstantiationUtil.serializeObject(elemCoder);
-
- checkpointBuilder.addListUpdatesBuilder()
- .setTag(stateKey)
- .setData(coder)
- .writeInt(contents.size());
-
- for (T item : contents) {
- // encode the element
- ByteString.Output stream = ByteString.newOutput();
- elemCoder.encode(item, stream, Coder.Context.OUTER);
- ByteString data = stream.toByteString();
-
- // add the data to the checkpoint.
- checkpointBuilder.setData(data);
- }
- }
- }
-
- public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
- int noOfValues = checkpointReader.getInt();
- for (int j = 0; j < noOfValues; j++) {
- ByteString valueContent = checkpointReader.getData();
- T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER);
- add(outValue);
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java
deleted file mode 100644
index d73ac8c..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointReader.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers.streaming.state;
-
-import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
-import com.google.protobuf.ByteString;
-import org.apache.flink.core.memory.DataInputView;
-import org.joda.time.Instant;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
-public class StateCheckpointReader {
-
- private final DataInputView input;
-
- public StateCheckpointReader(DataInputView in) {
- this.input = in;
- }
-
- public ByteString getTag() throws IOException {
- return ByteString.copyFrom(readRawData());
- }
-
- public String getTagToString() throws IOException {
- return input.readUTF();
- }
-
- public ByteString getData() throws IOException {
- return ByteString.copyFrom(readRawData());
- }
-
- public int getInt() throws IOException {
- validate();
- return input.readInt();
- }
-
- public byte getByte() throws IOException {
- validate();
- return input.readByte();
- }
-
- public Instant getTimestamp() throws IOException {
- validate();
- Long watermarkMillis = input.readLong();
- return new Instant(TimeUnit.MICROSECONDS.toMillis(watermarkMillis));
- }
-
- public <K> K deserializeKey(CoderTypeSerializer<K> keySerializer) throws IOException {
- return deserializeObject(keySerializer);
- }
-
- public <T> T deserializeObject(CoderTypeSerializer<T> objectSerializer) throws IOException {
- return objectSerializer.deserialize(input);
- }
-
- ///////// Helper Methods ///////
-
- private byte[] readRawData() throws IOException {
- validate();
- int size = input.readInt();
-
- byte[] serData = new byte[size];
- int bytesRead = input.read(serData);
- if (bytesRead != size) {
- throw new RuntimeException("Error while deserializing checkpoint. Not enough bytes in the input stream.");
- }
- return serData;
- }
-
- private void validate() {
- if (this.input == null) {
- throw new RuntimeException("StateBackend not initialized yet.");
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java
deleted file mode 100644
index 055a12a..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointUtils.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers.streaming.state;
-
-import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
-import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
-import com.google.cloud.dataflow.sdk.util.TimeDomain;
-import com.google.cloud.dataflow.sdk.util.TimerInternals;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespace;
-import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
-import org.joda.time.Instant;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-public class StateCheckpointUtils {
-
- public static <K> void encodeState(Map<K, FlinkStateInternals<K>> perKeyStateInternals,
- StateCheckpointWriter writer, Coder<K> keyCoder) throws IOException {
- CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
-
- int noOfKeys = perKeyStateInternals.size();
- writer.writeInt(noOfKeys);
- for (Map.Entry<K, FlinkStateInternals<K>> keyStatePair : perKeyStateInternals.entrySet()) {
- K key = keyStatePair.getKey();
- FlinkStateInternals<K> state = keyStatePair.getValue();
-
- // encode the key
- writer.serializeKey(key, keySerializer);
-
- // write the associated state
- state.persistState(writer);
- }
- }
-
- public static <K> Map<K, FlinkStateInternals<K>> decodeState(
- StateCheckpointReader reader,
- OutputTimeFn<? super BoundedWindow> outputTimeFn,
- Coder<K> keyCoder,
- Coder<? extends BoundedWindow> windowCoder,
- ClassLoader classLoader) throws IOException, ClassNotFoundException {
-
- int noOfKeys = reader.getInt();
- Map<K, FlinkStateInternals<K>> perKeyStateInternals = new HashMap<>(noOfKeys);
- perKeyStateInternals.clear();
-
- CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
- for (int i = 0; i < noOfKeys; i++) {
-
- // decode the key.
- K key = reader.deserializeKey(keySerializer);
-
- //decode the state associated to the key.
- FlinkStateInternals<K> stateForKey =
- new FlinkStateInternals<>(key, keyCoder, windowCoder, outputTimeFn);
- stateForKey.restoreState(reader, classLoader);
- perKeyStateInternals.put(key, stateForKey);
- }
- return perKeyStateInternals;
- }
-
- ////////////// Encoding/Decoding the Timers ////////////////
-
-
- public static <K> void encodeTimers(Map<K, Set<TimerInternals.TimerData>> allTimers,
- StateCheckpointWriter writer,
- Coder<K> keyCoder) throws IOException {
- CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
-
- int noOfKeys = allTimers.size();
- writer.writeInt(noOfKeys);
- for (Map.Entry<K, Set<TimerInternals.TimerData>> timersPerKey : allTimers.entrySet()) {
- K key = timersPerKey.getKey();
-
- // encode the key
- writer.serializeKey(key, keySerializer);
-
- // write the associated timers
- Set<TimerInternals.TimerData> timers = timersPerKey.getValue();
- encodeTimerDataForKey(writer, timers);
- }
- }
-
- public static <K> Map<K, Set<TimerInternals.TimerData>> decodeTimers(
- StateCheckpointReader reader,
- Coder<? extends BoundedWindow> windowCoder,
- Coder<K> keyCoder) throws IOException {
-
- int noOfKeys = reader.getInt();
- Map<K, Set<TimerInternals.TimerData>> activeTimers = new HashMap<>(noOfKeys);
- activeTimers.clear();
-
- CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
- for (int i = 0; i < noOfKeys; i++) {
-
- // decode the key.
- K key = reader.deserializeKey(keySerializer);
-
- // decode the associated timers.
- Set<TimerInternals.TimerData> timers = decodeTimerDataForKey(reader, windowCoder);
- activeTimers.put(key, timers);
- }
- return activeTimers;
- }
-
- private static void encodeTimerDataForKey(StateCheckpointWriter writer, Set<TimerInternals.TimerData> timers) throws IOException {
- // encode timers
- writer.writeInt(timers.size());
- for (TimerInternals.TimerData timer : timers) {
- String stringKey = timer.getNamespace().stringKey();
-
- writer.setTag(stringKey);
- writer.setTimestamp(timer.getTimestamp());
- writer.writeInt(timer.getDomain().ordinal());
- }
- }
-
- private static Set<TimerInternals.TimerData> decodeTimerDataForKey(
- StateCheckpointReader reader, Coder<? extends BoundedWindow> windowCoder) throws IOException {
-
- // decode the timers: first their number and then the content itself.
- int noOfTimers = reader.getInt();
- Set<TimerInternals.TimerData> timers = new HashSet<>(noOfTimers);
- for (int i = 0; i < noOfTimers; i++) {
- String stringKey = reader.getTagToString();
- Instant instant = reader.getTimestamp();
- TimeDomain domain = TimeDomain.values()[reader.getInt()];
-
- StateNamespace namespace = StateNamespaces.fromString(stringKey, windowCoder);
- timers.add(TimerInternals.TimerData.of(namespace, instant, domain));
- }
- return timers;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java
deleted file mode 100644
index 738ce5f..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateCheckpointWriter.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers.streaming.state;
-
-import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
-import com.google.protobuf.ByteString;
-import org.apache.flink.runtime.state.AbstractStateBackend;
-import org.joda.time.Instant;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
-public class StateCheckpointWriter {
-
- private final AbstractStateBackend.CheckpointStateOutputView output;
-
- public static StateCheckpointWriter create(AbstractStateBackend.CheckpointStateOutputView output) {
- return new StateCheckpointWriter(output);
- }
-
- private StateCheckpointWriter(AbstractStateBackend.CheckpointStateOutputView output) {
- this.output = output;
- }
-
- ///////// Creating the serialized versions of the different types of state held by dataflow ///////
-
- public StateCheckpointWriter addValueBuilder() throws IOException {
- validate();
- StateType.serialize(StateType.VALUE, this);
- return this;
- }
-
- public StateCheckpointWriter addWatermarkHoldsBuilder() throws IOException {
- validate();
- StateType.serialize(StateType.WATERMARK, this);
- return this;
- }
-
- public StateCheckpointWriter addListUpdatesBuilder() throws IOException {
- validate();
- StateType.serialize(StateType.LIST, this);
- return this;
- }
-
- public StateCheckpointWriter addAccumulatorBuilder() throws IOException {
- validate();
- StateType.serialize(StateType.ACCUMULATOR, this);
- return this;
- }
-
- ///////// Setting the tag for a given state element ///////
-
- public StateCheckpointWriter setTag(ByteString stateKey) throws IOException {
- return writeData(stateKey.toByteArray());
- }
-
- public StateCheckpointWriter setTag(String stateKey) throws IOException {
- output.writeUTF(stateKey);
- return this;
- }
-
-
- public <K> StateCheckpointWriter serializeKey(K key, CoderTypeSerializer<K> keySerializer) throws IOException {
- return serializeObject(key, keySerializer);
- }
-
- public <T> StateCheckpointWriter serializeObject(T object, CoderTypeSerializer<T> objectSerializer) throws IOException {
- objectSerializer.serialize(object, output);
- return this;
- }
-
- ///////// Write the actual serialized data //////////
-
- public StateCheckpointWriter setData(ByteString data) throws IOException {
- return writeData(data.toByteArray());
- }
-
- public StateCheckpointWriter setData(byte[] data) throws IOException {
- return writeData(data);
- }
-
- public StateCheckpointWriter setTimestamp(Instant timestamp) throws IOException {
- validate();
- output.writeLong(TimeUnit.MILLISECONDS.toMicros(timestamp.getMillis()));
- return this;
- }
-
- public StateCheckpointWriter writeInt(int number) throws IOException {
- validate();
- output.writeInt(number);
- return this;
- }
-
- public StateCheckpointWriter writeByte(byte b) throws IOException {
- validate();
- output.writeByte(b);
- return this;
- }
-
- ///////// Helper Methods ///////
-
- private StateCheckpointWriter writeData(byte[] data) throws IOException {
- validate();
- output.writeInt(data.length);
- output.write(data);
- return this;
- }
-
- private void validate() {
- if (this.output == null) {
- throw new RuntimeException("StateBackend not initialized yet.");
- }
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java
deleted file mode 100644
index 8b20600..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/state/StateType.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink.translation.wrappers.streaming.state;
-
-import java.io.IOException;
-
-/**
- * The available types of state, as provided by the Beam SDK. This class is used for serialization/deserialization
- * purposes.
- * */
-public enum StateType {
-
- VALUE(0),
-
- WATERMARK(1),
-
- LIST(2),
-
- ACCUMULATOR(3);
-
- private final int numVal;
-
- StateType(int value) {
- this.numVal = value;
- }
-
- public static void serialize(StateType type, StateCheckpointWriter output) throws IOException {
- if (output == null) {
- throw new IllegalArgumentException("Cannot write to a null output.");
- }
-
- if(type.numVal < 0 || type.numVal > 3) {
- throw new RuntimeException("Unknown State Type " + type + ".");
- }
-
- output.writeByte((byte) type.numVal);
- }
-
- public static StateType deserialize(StateCheckpointReader input) throws IOException {
- if (input == null) {
- throw new IllegalArgumentException("Cannot read from a null input.");
- }
-
- int typeInt = (int) input.getByte();
- if(typeInt < 0 || typeInt > 3) {
- throw new RuntimeException("Unknown State Type " + typeInt + ".");
- }
-
- StateType resultType = null;
- for(StateType st: values()) {
- if(st.numVal == typeInt) {
- resultType = st;
- break;
- }
- }
- return resultType;
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
new file mode 100644
index 0000000..02a49b9
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
@@ -0,0 +1,267 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import org.apache.beam.runners.flink.translation.FlinkPipelineTranslator;
+import org.apache.beam.runners.flink.translation.FlinkBatchPipelineTranslator;
+import org.apache.beam.runners.flink.translation.FlinkStreamingPipelineTranslator;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.common.base.Preconditions;
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.java.CollectionEnvironment;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+/**
+ * The class that instantiates and manages the execution of a given job.
+ * Depending on if the job is a Streaming or Batch processing one, it creates
+ * the adequate execution environment ({@link ExecutionEnvironment} or {@link StreamExecutionEnvironment}),
+ * the necessary {@link FlinkPipelineTranslator} ({@link FlinkBatchPipelineTranslator} or
+ * {@link FlinkStreamingPipelineTranslator})to transform the Beam job into a Flink one, and
+ * executes the (translated) job.
+ */
+public class FlinkPipelineExecutionEnvironment {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class);
+
+ private final FlinkPipelineOptions options;
+
+ /**
+ * The Flink Batch execution environment. This is instantiated to either a
+ * {@link org.apache.flink.api.java.CollectionEnvironment},
+ * a {@link org.apache.flink.api.java.LocalEnvironment} or
+ * a {@link org.apache.flink.api.java.RemoteEnvironment}, depending on the configuration
+ * options.
+ */
+ private ExecutionEnvironment flinkBatchEnv;
+
+
+ /**
+ * The Flink Streaming execution environment. This is instantiated to either a
+ * {@link org.apache.flink.streaming.api.environment.LocalStreamEnvironment} or
+ * a {@link org.apache.flink.streaming.api.environment.RemoteStreamEnvironment}, depending
+ * on the configuration options, and more specifically, the url of the master.
+ */
+ private StreamExecutionEnvironment flinkStreamEnv;
+
+ /**
+ * Translator for this FlinkPipelineRunner. Its role is to translate the Beam operators to
+ * their Flink counterparts. Based on the options provided by the user, if we have a streaming job,
+ * this is instantiated as a {@link FlinkStreamingPipelineTranslator}. In other case, i.e. a batch job,
+ * a {@link FlinkBatchPipelineTranslator} is created.
+ */
+ private FlinkPipelineTranslator flinkPipelineTranslator;
+
+ /**
+ * Creates a {@link FlinkPipelineExecutionEnvironment} with the user-specified parameters in the
+ * provided {@link FlinkPipelineOptions}.
+ *
+ * @param options the user-defined pipeline options.
+ * */
+ public FlinkPipelineExecutionEnvironment(FlinkPipelineOptions options) {
+ this.options = Preconditions.checkNotNull(options);
+ this.createPipelineExecutionEnvironment();
+ this.createPipelineTranslator();
+ }
+
+ /**
+ * Depending on the type of job (Streaming or Batch) and the user-specified options,
+ * this method creates the adequate ExecutionEnvironment.
+ */
+ private void createPipelineExecutionEnvironment() {
+ if (options.isStreaming()) {
+ createStreamExecutionEnvironment();
+ } else {
+ createBatchExecutionEnvironment();
+ }
+ }
+
+ /**
+ * Depending on the type of job (Streaming or Batch), this method creates the adequate job graph
+ * translator. In the case of batch, it will work with {@link org.apache.flink.api.java.DataSet},
+ * while for streaming, it will work with {@link org.apache.flink.streaming.api.datastream.DataStream}.
+ */
+ private void createPipelineTranslator() {
+ checkInitializationState();
+ if (this.flinkPipelineTranslator != null) {
+ throw new IllegalStateException("FlinkPipelineTranslator already initialized.");
+ }
+
+ this.flinkPipelineTranslator = options.isStreaming() ?
+ new FlinkStreamingPipelineTranslator(flinkStreamEnv, options) :
+ new FlinkBatchPipelineTranslator(flinkBatchEnv, options);
+ }
+
+ /**
+ * Depending on if the job is a Streaming or a Batch one, this method creates
+ * the necessary execution environment and pipeline translator, and translates
+ * the {@link com.google.cloud.dataflow.sdk.values.PCollection} program into
+ * a {@link org.apache.flink.api.java.DataSet} or {@link org.apache.flink.streaming.api.datastream.DataStream}
+ * one.
+ * */
+ public void translate(Pipeline pipeline) {
+ checkInitializationState();
+ if(this.flinkBatchEnv == null && this.flinkStreamEnv == null) {
+ createPipelineExecutionEnvironment();
+ }
+ if (this.flinkPipelineTranslator == null) {
+ createPipelineTranslator();
+ }
+ this.flinkPipelineTranslator.translate(pipeline);
+ }
+
+ /**
+ * Launches the program execution.
+ * */
+ public JobExecutionResult executePipeline() throws Exception {
+ if (options.isStreaming()) {
+ if (this.flinkStreamEnv == null) {
+ throw new RuntimeException("FlinkPipelineExecutionEnvironment not initialized.");
+ }
+ if (this.flinkPipelineTranslator == null) {
+ throw new RuntimeException("FlinkPipelineTranslator not initialized.");
+ }
+ return this.flinkStreamEnv.execute();
+ } else {
+ if (this.flinkBatchEnv == null) {
+ throw new RuntimeException("FlinkPipelineExecutionEnvironment not initialized.");
+ }
+ if (this.flinkPipelineTranslator == null) {
+ throw new RuntimeException("FlinkPipelineTranslator not initialized.");
+ }
+ return this.flinkBatchEnv.execute();
+ }
+ }
+
+ /**
+ * If the submitted job is a batch processing job, this method creates the adequate
+ * Flink {@link org.apache.flink.api.java.ExecutionEnvironment} depending
+ * on the user-specified options.
+ */
+ private void createBatchExecutionEnvironment() {
+ if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) {
+ throw new RuntimeException("FlinkPipelineExecutionEnvironment already initialized.");
+ }
+
+ LOG.info("Creating the required Batch Execution Environment.");
+
+ String masterUrl = options.getFlinkMaster();
+ this.flinkStreamEnv = null;
+
+ // depending on the master, create the right environment.
+ if (masterUrl.equals("[local]")) {
+ this.flinkBatchEnv = ExecutionEnvironment.createLocalEnvironment();
+ } else if (masterUrl.equals("[collection]")) {
+ this.flinkBatchEnv = new CollectionEnvironment();
+ } else if (masterUrl.equals("[auto]")) {
+ this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
+ } else if (masterUrl.matches(".*:\\d*")) {
+ String[] parts = masterUrl.split(":");
+ List<String> stagingFiles = options.getFilesToStage();
+ this.flinkBatchEnv = ExecutionEnvironment.createRemoteEnvironment(parts[0],
+ Integer.parseInt(parts[1]),
+ stagingFiles.toArray(new String[stagingFiles.size()]));
+ } else {
+ LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
+ this.flinkBatchEnv = ExecutionEnvironment.getExecutionEnvironment();
+ }
+
+ // set the correct parallelism.
+ if (options.getParallelism() != -1 && !(this.flinkBatchEnv instanceof CollectionEnvironment)) {
+ this.flinkBatchEnv.setParallelism(options.getParallelism());
+ }
+
+ // set parallelism in the options (required by some execution code)
+ options.setParallelism(flinkBatchEnv.getParallelism());
+ }
+
+ /**
+ * If the submitted job is a stream processing job, this method creates the adequate
+ * Flink {@link org.apache.flink.streaming.api.environment.StreamExecutionEnvironment} depending
+ * on the user-specified options.
+ */
+ private void createStreamExecutionEnvironment() {
+ if (this.flinkStreamEnv != null || this.flinkBatchEnv != null) {
+ throw new RuntimeException("FlinkPipelineExecutionEnvironment already initialized.");
+ }
+
+ LOG.info("Creating the required Streaming Environment.");
+
+ String masterUrl = options.getFlinkMaster();
+ this.flinkBatchEnv = null;
+
+ // depending on the master, create the right environment.
+ if (masterUrl.equals("[local]")) {
+ this.flinkStreamEnv = StreamExecutionEnvironment.createLocalEnvironment();
+ } else if (masterUrl.equals("[auto]")) {
+ this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+ } else if (masterUrl.matches(".*:\\d*")) {
+ String[] parts = masterUrl.split(":");
+ List<String> stagingFiles = options.getFilesToStage();
+ this.flinkStreamEnv = StreamExecutionEnvironment.createRemoteEnvironment(parts[0],
+ Integer.parseInt(parts[1]), stagingFiles.toArray(new String[stagingFiles.size()]));
+ } else {
+ LOG.warn("Unrecognized Flink Master URL {}. Defaulting to [auto].", masterUrl);
+ this.flinkStreamEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+ }
+
+ // set the correct parallelism.
+ if (options.getParallelism() != -1) {
+ this.flinkStreamEnv.setParallelism(options.getParallelism());
+ }
+
+ // set parallelism in the options (required by some execution code)
+ options.setParallelism(flinkStreamEnv.getParallelism());
+
+ // default to event time
+ this.flinkStreamEnv.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+
+ // for the following 2 parameters, a value of -1 means that Flink will use
+ // the default values as specified in the configuration.
+ int numRetries = options.getNumberOfExecutionRetries();
+ if (numRetries != -1) {
+ this.flinkStreamEnv.setNumberOfExecutionRetries(numRetries);
+ }
+ long retryDelay = options.getExecutionRetryDelay();
+ if (retryDelay != -1) {
+ this.flinkStreamEnv.getConfig().setExecutionRetryDelay(retryDelay);
+ }
+
+ // A value of -1 corresponds to disabled checkpointing (see CheckpointConfig in Flink).
+ // If the value is not -1, then the validity checks are applied.
+ // By default, checkpointing is disabled.
+ long checkpointInterval = options.getCheckpointingInterval();
+ if(checkpointInterval != -1) {
+ if (checkpointInterval < 1) {
+ throw new IllegalArgumentException("The checkpoint interval must be positive");
+ }
+ this.flinkStreamEnv.enableCheckpointing(checkpointInterval);
+ }
+ }
+
+ private void checkInitializationState() {
+ if (options.isStreaming() && this.flinkBatchEnv != null) {
+ throw new IllegalStateException("Attempted to run a Streaming Job with a Batch Execution Environment.");
+ } else if (!options.isStreaming() && this.flinkStreamEnv != null) {
+ throw new IllegalStateException("Attempted to run a Batch Job with a Streaming Execution Environment.");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
new file mode 100644
index 0000000..bf83353
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java
@@ -0,0 +1,91 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.google.cloud.dataflow.sdk.options.ApplicationNameOptions;
+import com.google.cloud.dataflow.sdk.options.DataflowPipelineOptions;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.StreamingOptions;
+
+import java.util.List;
+
+/**
+ * Options which can be used to configure a Flink PipelineRunner.
+ */
+public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOptions, StreamingOptions {
+
+ /**
+ * List of local files to make available to workers.
+ * <p>
+ * Jars are placed on the worker's classpath.
+ * <p>
+ * The default value is the list of jars from the main program's classpath.
+ */
+ @Description("Jar-Files to send to all workers and put on the classpath. " +
+ "The default value is all files from the classpath.")
+ @JsonIgnore
+ List<String> getFilesToStage();
+ void setFilesToStage(List<String> value);
+
+ /**
+ * The job name is used to identify jobs running on a Flink cluster.
+ */
+ @Description("Dataflow job name, to uniquely identify active jobs. "
+ + "Defaults to using the ApplicationName-UserName-Date.")
+ @Default.InstanceFactory(DataflowPipelineOptions.JobNameFactory.class)
+ String getJobName();
+ void setJobName(String value);
+
+ /**
+ * The url of the Flink JobManager on which to execute pipelines. This can either be
+ * the the address of a cluster JobManager, in the form "host:port" or one of the special
+ * Strings "[local]", "[collection]" or "[auto]". "[local]" will start a local Flink
+ * Cluster in the JVM, "[collection]" will execute the pipeline on Java Collections while
+ * "[auto]" will let the system decide where to execute the pipeline based on the environment.
+ */
+ @Description("Address of the Flink Master where the Pipeline should be executed. Can" +
+ " either be of the form \"host:port\" or one of the special values [local], " +
+ "[collection] or [auto].")
+ String getFlinkMaster();
+ void setFlinkMaster(String value);
+
+ @Description("The degree of parallelism to be used when distributing operations onto workers.")
+ @Default.Integer(-1)
+ Integer getParallelism();
+ void setParallelism(Integer value);
+
+ @Description("The interval between consecutive checkpoints (i.e. snapshots of the current pipeline state used for " +
+ "fault tolerance).")
+ @Default.Long(-1L)
+ Long getCheckpointingInterval();
+ void setCheckpointingInterval(Long interval);
+
+ @Description("Sets the number of times that failed tasks are re-executed. " +
+ "A value of zero effectively disables fault tolerance. A value of -1 indicates " +
+ "that the system default value (as defined in the configuration) should be used.")
+ @Default.Integer(-1)
+ Integer getNumberOfExecutionRetries();
+ void setNumberOfExecutionRetries(Integer retries);
+
+ @Description("Sets the delay between executions. A value of {@code -1} indicates that the default value should be used.")
+ @Default.Long(-1L)
+ Long getExecutionRetryDelay();
+ void setExecutionRetryDelay(Long delay);
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
new file mode 100644
index 0000000..3c33d20
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
@@ -0,0 +1,204 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsValidator;
+import com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner;
+import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.POutput;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import org.apache.flink.api.common.JobExecutionResult;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * A {@link PipelineRunner} that executes the operations in the
+ * pipeline by first translating them to a Flink Plan and then executing them either locally
+ * or on a Flink cluster, depending on the configuration.
+ * <p>
+ * This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineRunner}.
+ */
+public class FlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {
+
+ private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineRunner.class);
+
+ /**
+ * Provided options.
+ */
+ private final FlinkPipelineOptions options;
+
+ private final FlinkPipelineExecutionEnvironment flinkJobEnv;
+
+ /**
+ * Construct a runner from the provided options.
+ *
+ * @param options Properties which configure the runner.
+ * @return The newly created runner.
+ */
+ public static FlinkPipelineRunner fromOptions(PipelineOptions options) {
+ FlinkPipelineOptions flinkOptions =
+ PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options);
+ ArrayList<String> missing = new ArrayList<>();
+
+ if (flinkOptions.getAppName() == null) {
+ missing.add("appName");
+ }
+ if (missing.size() > 0) {
+ throw new IllegalArgumentException(
+ "Missing required values: " + Joiner.on(',').join(missing));
+ }
+
+ if (flinkOptions.getFilesToStage() == null) {
+ flinkOptions.setFilesToStage(detectClassPathResourcesToStage(
+ DataflowPipelineRunner.class.getClassLoader()));
+ LOG.info("PipelineOptions.filesToStage was not specified. "
+ + "Defaulting to files from the classpath: will stage {} files. "
+ + "Enable logging at DEBUG level to see which files will be staged.",
+ flinkOptions.getFilesToStage().size());
+ LOG.debug("Classpath elements: {}", flinkOptions.getFilesToStage());
+ }
+
+ // Verify jobName according to service requirements.
+ String jobName = flinkOptions.getJobName().toLowerCase();
+ Preconditions.checkArgument(jobName.matches("[a-z]([-a-z0-9]*[a-z0-9])?"), "JobName invalid; " +
+ "the name must consist of only the characters " + "[-a-z0-9], starting with a letter " +
+ "and ending with a letter " + "or number");
+ Preconditions.checkArgument(jobName.length() <= 40,
+ "JobName too long; must be no more than 40 characters in length");
+
+ // Set Flink Master to [auto] if no option was specified.
+ if (flinkOptions.getFlinkMaster() == null) {
+ flinkOptions.setFlinkMaster("[auto]");
+ }
+
+ return new FlinkPipelineRunner(flinkOptions);
+ }
+
+ private FlinkPipelineRunner(FlinkPipelineOptions options) {
+ this.options = options;
+ this.flinkJobEnv = new FlinkPipelineExecutionEnvironment(options);
+ }
+
+ @Override
+ public FlinkRunnerResult run(Pipeline pipeline) {
+ LOG.info("Executing pipeline using FlinkPipelineRunner.");
+
+ LOG.info("Translating pipeline to Flink program.");
+
+ this.flinkJobEnv.translate(pipeline);
+
+ LOG.info("Starting execution of Flink program.");
+
+ JobExecutionResult result;
+ try {
+ result = this.flinkJobEnv.executePipeline();
+ } catch (Exception e) {
+ LOG.error("Pipeline execution failed", e);
+ throw new RuntimeException("Pipeline execution failed", e);
+ }
+
+ LOG.info("Execution finished in {} msecs", result.getNetRuntime());
+
+ Map<String, Object> accumulators = result.getAllAccumulatorResults();
+ if (accumulators != null && !accumulators.isEmpty()) {
+ LOG.info("Final aggregator values:");
+
+ for (Map.Entry<String, Object> entry : result.getAllAccumulatorResults().entrySet()) {
+ LOG.info("{} : {}", entry.getKey(), entry.getValue());
+ }
+ }
+
+ return new FlinkRunnerResult(accumulators, result.getNetRuntime());
+ }
+
+ /**
+ * For testing.
+ */
+ public FlinkPipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ /**
+ * Constructs a runner with default properties for testing.
+ *
+ * @return The newly created runner.
+ */
+ public static FlinkPipelineRunner createForTest(boolean streaming) {
+ FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
+ // we use [auto] for testing since this will make it pick up the Testing
+ // ExecutionEnvironment
+ options.setFlinkMaster("[auto]");
+ options.setStreaming(streaming);
+ return new FlinkPipelineRunner(options);
+ }
+
+ @Override
+ public <Output extends POutput, Input extends PInput> Output apply(
+ PTransform<Input, Output> transform, Input input) {
+ return super.apply(transform, input);
+ }
+
+ /////////////////////////////////////////////////////////////////////////////
+
+ @Override
+ public String toString() {
+ return "DataflowPipelineRunner#" + hashCode();
+ }
+
+ /**
+ * Attempts to detect all the resources the class loader has access to. This does not recurse
+ * to class loader parents stopping it from pulling in resources from the system class loader.
+ *
+ * @param classLoader The URLClassLoader to use to detect resources to stage.
+ * @return A list of absolute paths to the resources the class loader uses.
+ * @throws IllegalArgumentException If either the class loader is not a URLClassLoader or one
+ * of the resources the class loader exposes is not a file resource.
+ */
+ protected static List<String> detectClassPathResourcesToStage(ClassLoader classLoader) {
+ if (!(classLoader instanceof URLClassLoader)) {
+ String message = String.format("Unable to use ClassLoader to detect classpath elements. "
+ + "Current ClassLoader is %s, only URLClassLoaders are supported.", classLoader);
+ LOG.error(message);
+ throw new IllegalArgumentException(message);
+ }
+
+ List<String> files = new ArrayList<>();
+ for (URL url : ((URLClassLoader) classLoader).getURLs()) {
+ try {
+ files.add(new File(url.toURI()).getAbsolutePath());
+ } catch (IllegalArgumentException | URISyntaxException e) {
+ String message = String.format("Unable to convert url (%s) to file.", url);
+ LOG.error(message);
+ throw new IllegalArgumentException(message, e);
+ }
+ }
+ return files;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
new file mode 100644
index 0000000..c2329a6
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
@@ -0,0 +1,66 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink;
+
+import com.google.cloud.dataflow.sdk.PipelineResult;
+import com.google.cloud.dataflow.sdk.runners.AggregatorRetrievalException;
+import com.google.cloud.dataflow.sdk.runners.AggregatorValues;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ * Result of executing a {@link com.google.cloud.dataflow.sdk.Pipeline} with Flink. This
+ * has methods to query to job runtime and the final values of
+ * {@link com.google.cloud.dataflow.sdk.transforms.Aggregator}s.
+ */
+public class FlinkRunnerResult implements PipelineResult {
+
+ private final Map<String, Object> aggregators;
+
+ private final long runtime;
+
+ public FlinkRunnerResult(Map<String, Object> aggregators, long runtime) {
+ this.aggregators = (aggregators == null || aggregators.isEmpty()) ?
+ Collections.<String, Object>emptyMap() :
+ Collections.unmodifiableMap(aggregators);
+
+ this.runtime = runtime;
+ }
+
+ @Override
+ public State getState() {
+ return null;
+ }
+
+ @Override
+ public <T> AggregatorValues<T> getAggregatorValues(final Aggregator<?, T> aggregator) throws AggregatorRetrievalException {
+ // TODO provide a list of all accumulator step values
+ Object value = aggregators.get(aggregator.getName());
+ if (value != null) {
+ return new AggregatorValues<T>() {
+ @Override
+ public Map<String, T> getValuesAtSteps() {
+ return (Map<String, T>) aggregators;
+ }
+ };
+ } else {
+ throw new AggregatorRetrievalException("Accumulator results not found.",
+ new RuntimeException("Accumulator does not exist."));
+ }
+ }
+}
[22/50] [abbrv] incubator-beam git commit: [flink] adjust directories
according to package name
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
new file mode 100644
index 0000000..84007af
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
@@ -0,0 +1,713 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers.streaming.state;
+
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.CombineWithContext;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
+import com.google.cloud.dataflow.sdk.util.state.*;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.ByteString;
+import org.apache.flink.util.InstantiationUtil;
+import org.joda.time.Instant;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * An implementation of the Beam {@link StateInternals}. This implementation simply keeps elements in memory.
+ * This state is periodically checkpointed by Flink, for fault-tolerance.
+ *
+ * TODO: State should be rewritten to redirect to Flink per-key state so that coders and combiners don't need
+ * to be serialized along with encoded values when snapshotting.
+ */
+public class FlinkStateInternals<K> implements StateInternals<K> {
+
+ private final K key;
+
+ private final Coder<K> keyCoder;
+
+ private final Coder<? extends BoundedWindow> windowCoder;
+
+ private final OutputTimeFn<? super BoundedWindow> outputTimeFn;
+
+ private Instant watermarkHoldAccessor;
+
+ public FlinkStateInternals(K key,
+ Coder<K> keyCoder,
+ Coder<? extends BoundedWindow> windowCoder,
+ OutputTimeFn<? super BoundedWindow> outputTimeFn) {
+ this.key = key;
+ this.keyCoder = keyCoder;
+ this.windowCoder = windowCoder;
+ this.outputTimeFn = outputTimeFn;
+ }
+
+ public Instant getWatermarkHold() {
+ return watermarkHoldAccessor;
+ }
+
+ /**
+ * This is the interface state has to implement in order for it to be fault tolerant when
+ * executed by the FlinkPipelineRunner.
+ */
+ private interface CheckpointableIF {
+
+ boolean shouldPersist();
+
+ void persistState(StateCheckpointWriter checkpointBuilder) throws IOException;
+ }
+
+ protected final StateTable<K> inMemoryState = new StateTable<K>() {
+ @Override
+ protected StateTag.StateBinder binderForNamespace(final StateNamespace namespace, final StateContext<?> c) {
+ return new StateTag.StateBinder<K>() {
+
+ @Override
+ public <T> ValueState<T> bindValue(StateTag<? super K, ValueState<T>> address, Coder<T> coder) {
+ return new FlinkInMemoryValue<>(encodeKey(namespace, address), coder);
+ }
+
+ @Override
+ public <T> BagState<T> bindBag(StateTag<? super K, BagState<T>> address, Coder<T> elemCoder) {
+ return new FlinkInMemoryBag<>(encodeKey(namespace, address), elemCoder);
+ }
+
+ @Override
+ public <InputT, AccumT, OutputT> AccumulatorCombiningState<InputT, AccumT, OutputT> bindCombiningValue(
+ StateTag<? super K, AccumulatorCombiningState<InputT, AccumT, OutputT>> address,
+ Coder<AccumT> accumCoder, Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
+ return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c);
+ }
+
+ @Override
+ public <InputT, AccumT, OutputT> AccumulatorCombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValue(
+ StateTag<? super K, AccumulatorCombiningState<InputT, AccumT, OutputT>> address,
+ Coder<AccumT> accumCoder,
+ Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn) {
+ return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c);
+ }
+
+ @Override
+ public <InputT, AccumT, OutputT> AccumulatorCombiningState<InputT, AccumT, OutputT> bindKeyedCombiningValueWithContext(
+ StateTag<? super K, AccumulatorCombiningState<InputT, AccumT, OutputT>> address,
+ Coder<AccumT> accumCoder,
+ CombineWithContext.KeyedCombineFnWithContext<? super K, InputT, AccumT, OutputT> combineFn) {
+ return new FlinkInMemoryKeyedCombiningValue<>(encodeKey(namespace, address), combineFn, accumCoder, c);
+ }
+
+ @Override
+ public <W extends BoundedWindow> WatermarkHoldState<W> bindWatermark(StateTag<? super K, WatermarkHoldState<W>> address, OutputTimeFn<? super W> outputTimeFn) {
+ return new FlinkWatermarkHoldStateImpl<>(encodeKey(namespace, address), outputTimeFn);
+ }
+ };
+ }
+ };
+
+ @Override
+ public K getKey() {
+ return key;
+ }
+
+ @Override
+ public <StateT extends State> StateT state(StateNamespace namespace, StateTag<? super K, StateT> address) {
+ return inMemoryState.get(namespace, address, null);
+ }
+
+ @Override
+ public <T extends State> T state(StateNamespace namespace, StateTag<? super K, T> address, StateContext<?> c) {
+ return inMemoryState.get(namespace, address, c);
+ }
+
+ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
+ checkpointBuilder.writeInt(getNoOfElements());
+
+ for (State location : inMemoryState.values()) {
+ if (!(location instanceof CheckpointableIF)) {
+ throw new IllegalStateException(String.format(
+ "%s wasn't created by %s -- unable to persist it",
+ location.getClass().getSimpleName(),
+ getClass().getSimpleName()));
+ }
+ ((CheckpointableIF) location).persistState(checkpointBuilder);
+ }
+ }
+
+ public void restoreState(StateCheckpointReader checkpointReader, ClassLoader loader)
+ throws IOException, ClassNotFoundException {
+
+ // the number of elements to read.
+ int noOfElements = checkpointReader.getInt();
+ for (int i = 0; i < noOfElements; i++) {
+ decodeState(checkpointReader, loader);
+ }
+ }
+
+ /**
+ * We remove the first character which encodes the type of the stateTag ('s' for system
+ * and 'u' for user). For more details check out the source of
+ * {@link StateTags.StateTagBase#getId()}.
+ */
+ private void decodeState(StateCheckpointReader reader, ClassLoader loader)
+ throws IOException, ClassNotFoundException {
+
+ StateType stateItemType = StateType.deserialize(reader);
+ ByteString stateKey = reader.getTag();
+
+ // first decode the namespace and the tagId...
+ String[] namespaceAndTag = stateKey.toStringUtf8().split("\\+");
+ if (namespaceAndTag.length != 2) {
+ throw new IllegalArgumentException("Invalid stateKey " + stateKey.toString() + ".");
+ }
+ StateNamespace namespace = StateNamespaces.fromString(namespaceAndTag[0], windowCoder);
+
+ // ... decide if it is a system or user stateTag...
+ char ownerTag = namespaceAndTag[1].charAt(0);
+ if (ownerTag != 's' && ownerTag != 'u') {
+ throw new RuntimeException("Invalid StateTag name.");
+ }
+ boolean isSystemTag = ownerTag == 's';
+ String tagId = namespaceAndTag[1].substring(1);
+
+ // ...then decode the coder (if there is one)...
+ Coder<?> coder = null;
+ switch (stateItemType) {
+ case VALUE:
+ case LIST:
+ case ACCUMULATOR:
+ ByteString coderBytes = reader.getData();
+ coder = InstantiationUtil.deserializeObject(coderBytes.toByteArray(), loader);
+ break;
+ case WATERMARK:
+ break;
+ }
+
+ // ...then decode the combiner function (if there is one)...
+ CombineWithContext.KeyedCombineFnWithContext<? super K, ?, ?, ?> combineFn = null;
+ switch (stateItemType) {
+ case ACCUMULATOR:
+ ByteString combinerBytes = reader.getData();
+ combineFn = InstantiationUtil.deserializeObject(combinerBytes.toByteArray(), loader);
+ break;
+ case VALUE:
+ case LIST:
+ case WATERMARK:
+ break;
+ }
+
+ //... and finally, depending on the type of the state being decoded,
+ // 1) create the adequate stateTag,
+ // 2) create the state container,
+ // 3) restore the actual content.
+ switch (stateItemType) {
+ case VALUE: {
+ StateTag stateTag = StateTags.value(tagId, coder);
+ stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
+ @SuppressWarnings("unchecked")
+ FlinkInMemoryValue<?> value = (FlinkInMemoryValue<?>) inMemoryState.get(namespace, stateTag, null);
+ value.restoreState(reader);
+ break;
+ }
+ case WATERMARK: {
+ @SuppressWarnings("unchecked")
+ StateTag<Object, WatermarkHoldState<BoundedWindow>> stateTag = StateTags.watermarkStateInternal(tagId, outputTimeFn);
+ stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
+ @SuppressWarnings("unchecked")
+ FlinkWatermarkHoldStateImpl<?> watermark = (FlinkWatermarkHoldStateImpl<?>) inMemoryState.get(namespace, stateTag, null);
+ watermark.restoreState(reader);
+ break;
+ }
+ case LIST: {
+ StateTag stateTag = StateTags.bag(tagId, coder);
+ stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
+ FlinkInMemoryBag<?> bag = (FlinkInMemoryBag<?>) inMemoryState.get(namespace, stateTag, null);
+ bag.restoreState(reader);
+ break;
+ }
+ case ACCUMULATOR: {
+ @SuppressWarnings("unchecked")
+ StateTag<K, AccumulatorCombiningState<?, ?, ?>> stateTag = StateTags.keyedCombiningValueWithContext(tagId, (Coder) coder, combineFn);
+ stateTag = isSystemTag ? StateTags.makeSystemTagInternal(stateTag) : stateTag;
+ @SuppressWarnings("unchecked")
+ FlinkInMemoryKeyedCombiningValue<?, ?, ?> combiningValue =
+ (FlinkInMemoryKeyedCombiningValue<?, ?, ?>) inMemoryState.get(namespace, stateTag, null);
+ combiningValue.restoreState(reader);
+ break;
+ }
+ default:
+ throw new RuntimeException("Unknown State Type " + stateItemType + ".");
+ }
+ }
+
+ private ByteString encodeKey(StateNamespace namespace, StateTag<? super K, ?> address) {
+ StringBuilder sb = new StringBuilder();
+ try {
+ namespace.appendTo(sb);
+ sb.append('+');
+ address.appendTo(sb);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ return ByteString.copyFromUtf8(sb.toString());
+ }
+
+ private int getNoOfElements() {
+ int noOfElements = 0;
+ for (State state : inMemoryState.values()) {
+ if (!(state instanceof CheckpointableIF)) {
+ throw new RuntimeException("State Implementations used by the " +
+ "Flink Dataflow Runner should implement the CheckpointableIF interface.");
+ }
+
+ if (((CheckpointableIF) state).shouldPersist()) {
+ noOfElements++;
+ }
+ }
+ return noOfElements;
+ }
+
+ private final class FlinkInMemoryValue<T> implements ValueState<T>, CheckpointableIF {
+
+ private final ByteString stateKey;
+ private final Coder<T> elemCoder;
+
+ private T value = null;
+
+ public FlinkInMemoryValue(ByteString stateKey, Coder<T> elemCoder) {
+ this.stateKey = stateKey;
+ this.elemCoder = elemCoder;
+ }
+
+ @Override
+ public void clear() {
+ value = null;
+ }
+
+ @Override
+ public void write(T input) {
+ this.value = input;
+ }
+
+ @Override
+ public T read() {
+ return value;
+ }
+
+ @Override
+ public ValueState<T> readLater() {
+ // Ignore
+ return this;
+ }
+
+ @Override
+ public boolean shouldPersist() {
+ return value != null;
+ }
+
+ @Override
+ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
+ if (value != null) {
+ // serialize the coder.
+ byte[] coder = InstantiationUtil.serializeObject(elemCoder);
+
+ // encode the value into a ByteString
+ ByteString.Output stream = ByteString.newOutput();
+ elemCoder.encode(value, stream, Coder.Context.OUTER);
+ ByteString data = stream.toByteString();
+
+ checkpointBuilder.addValueBuilder()
+ .setTag(stateKey)
+ .setData(coder)
+ .setData(data);
+ }
+ }
+
+ public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
+ ByteString valueContent = checkpointReader.getData();
+ T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER);
+ write(outValue);
+ }
+ }
+
+ private final class FlinkWatermarkHoldStateImpl<W extends BoundedWindow>
+ implements WatermarkHoldState<W>, CheckpointableIF {
+
+ private final ByteString stateKey;
+
+ private Instant minimumHold = null;
+
+ private OutputTimeFn<? super W> outputTimeFn;
+
+ public FlinkWatermarkHoldStateImpl(ByteString stateKey, OutputTimeFn<? super W> outputTimeFn) {
+ this.stateKey = stateKey;
+ this.outputTimeFn = outputTimeFn;
+ }
+
+ @Override
+ public void clear() {
+ // Even though we're clearing we can't remove this from the in-memory state map, since
+ // other users may already have a handle on this WatermarkBagInternal.
+ minimumHold = null;
+ watermarkHoldAccessor = null;
+ }
+
+ @Override
+ public void add(Instant watermarkHold) {
+ if (minimumHold == null || minimumHold.isAfter(watermarkHold)) {
+ watermarkHoldAccessor = watermarkHold;
+ minimumHold = watermarkHold;
+ }
+ }
+
+ @Override
+ public ReadableState<Boolean> isEmpty() {
+ return new ReadableState<Boolean>() {
+ @Override
+ public Boolean read() {
+ return minimumHold == null;
+ }
+
+ @Override
+ public ReadableState<Boolean> readLater() {
+ // Ignore
+ return this;
+ }
+ };
+ }
+
+ @Override
+ public OutputTimeFn<? super W> getOutputTimeFn() {
+ return outputTimeFn;
+ }
+
+ @Override
+ public Instant read() {
+ return minimumHold;
+ }
+
+ @Override
+ public WatermarkHoldState<W> readLater() {
+ // Ignore
+ return this;
+ }
+
+ @Override
+ public String toString() {
+ return Objects.toString(minimumHold);
+ }
+
+ @Override
+ public boolean shouldPersist() {
+ return minimumHold != null;
+ }
+
+ @Override
+ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
+ if (minimumHold != null) {
+ checkpointBuilder.addWatermarkHoldsBuilder()
+ .setTag(stateKey)
+ .setTimestamp(minimumHold);
+ }
+ }
+
+ public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
+ Instant watermark = checkpointReader.getTimestamp();
+ add(watermark);
+ }
+ }
+
+
+ private static <K, InputT, AccumT, OutputT> CombineWithContext.KeyedCombineFnWithContext<K, InputT, AccumT, OutputT> withContext(
+ final Combine.KeyedCombineFn<K, InputT, AccumT, OutputT> combineFn) {
+ return new CombineWithContext.KeyedCombineFnWithContext<K, InputT, AccumT, OutputT>() {
+ @Override
+ public AccumT createAccumulator(K key, CombineWithContext.Context c) {
+ return combineFn.createAccumulator(key);
+ }
+
+ @Override
+ public AccumT addInput(K key, AccumT accumulator, InputT value, CombineWithContext.Context c) {
+ return combineFn.addInput(key, accumulator, value);
+ }
+
+ @Override
+ public AccumT mergeAccumulators(K key, Iterable<AccumT> accumulators, CombineWithContext.Context c) {
+ return combineFn.mergeAccumulators(key, accumulators);
+ }
+
+ @Override
+ public OutputT extractOutput(K key, AccumT accumulator, CombineWithContext.Context c) {
+ return combineFn.extractOutput(key, accumulator);
+ }
+ };
+ }
+
+ private static <K, InputT, AccumT, OutputT> CombineWithContext.KeyedCombineFnWithContext<K, InputT, AccumT, OutputT> withKeyAndContext(
+ final Combine.CombineFn<InputT, AccumT, OutputT> combineFn) {
+ return new CombineWithContext.KeyedCombineFnWithContext<K, InputT, AccumT, OutputT>() {
+ @Override
+ public AccumT createAccumulator(K key, CombineWithContext.Context c) {
+ return combineFn.createAccumulator();
+ }
+
+ @Override
+ public AccumT addInput(K key, AccumT accumulator, InputT value, CombineWithContext.Context c) {
+ return combineFn.addInput(accumulator, value);
+ }
+
+ @Override
+ public AccumT mergeAccumulators(K key, Iterable<AccumT> accumulators, CombineWithContext.Context c) {
+ return combineFn.mergeAccumulators(accumulators);
+ }
+
+ @Override
+ public OutputT extractOutput(K key, AccumT accumulator, CombineWithContext.Context c) {
+ return combineFn.extractOutput(accumulator);
+ }
+ };
+ }
+
+ private final class FlinkInMemoryKeyedCombiningValue<InputT, AccumT, OutputT>
+ implements AccumulatorCombiningState<InputT, AccumT, OutputT>, CheckpointableIF {
+
+ private final ByteString stateKey;
+ private final CombineWithContext.KeyedCombineFnWithContext<? super K, InputT, AccumT, OutputT> combineFn;
+ private final Coder<AccumT> accumCoder;
+ private final CombineWithContext.Context context;
+
+ private AccumT accum = null;
+ private boolean isClear = true;
+
+ private FlinkInMemoryKeyedCombiningValue(ByteString stateKey,
+ Combine.CombineFn<InputT, AccumT, OutputT> combineFn,
+ Coder<AccumT> accumCoder,
+ final StateContext<?> stateContext) {
+ this(stateKey, withKeyAndContext(combineFn), accumCoder, stateContext);
+ }
+
+
+ private FlinkInMemoryKeyedCombiningValue(ByteString stateKey,
+ Combine.KeyedCombineFn<? super K, InputT, AccumT, OutputT> combineFn,
+ Coder<AccumT> accumCoder,
+ final StateContext<?> stateContext) {
+ this(stateKey, withContext(combineFn), accumCoder, stateContext);
+ }
+
+ private FlinkInMemoryKeyedCombiningValue(ByteString stateKey,
+ CombineWithContext.KeyedCombineFnWithContext<? super K, InputT, AccumT, OutputT> combineFn,
+ Coder<AccumT> accumCoder,
+ final StateContext<?> stateContext) {
+ Preconditions.checkNotNull(combineFn);
+ Preconditions.checkNotNull(accumCoder);
+
+ this.stateKey = stateKey;
+ this.combineFn = combineFn;
+ this.accumCoder = accumCoder;
+ this.context = new CombineWithContext.Context() {
+ @Override
+ public PipelineOptions getPipelineOptions() {
+ return stateContext.getPipelineOptions();
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view) {
+ return stateContext.sideInput(view);
+ }
+ };
+ accum = combineFn.createAccumulator(key, context);
+ }
+
+ @Override
+ public void clear() {
+ accum = combineFn.createAccumulator(key, context);
+ isClear = true;
+ }
+
+ @Override
+ public void add(InputT input) {
+ isClear = false;
+ accum = combineFn.addInput(key, accum, input, context);
+ }
+
+ @Override
+ public AccumT getAccum() {
+ return accum;
+ }
+
+ @Override
+ public ReadableState<Boolean> isEmpty() {
+ return new ReadableState<Boolean>() {
+ @Override
+ public ReadableState<Boolean> readLater() {
+ // Ignore
+ return this;
+ }
+
+ @Override
+ public Boolean read() {
+ return isClear;
+ }
+ };
+ }
+
+ @Override
+ public void addAccum(AccumT accum) {
+ isClear = false;
+ this.accum = combineFn.mergeAccumulators(key, Arrays.asList(this.accum, accum), context);
+ }
+
+ @Override
+ public AccumT mergeAccumulators(Iterable<AccumT> accumulators) {
+ return combineFn.mergeAccumulators(key, accumulators, context);
+ }
+
+ @Override
+ public OutputT read() {
+ return combineFn.extractOutput(key, accum, context);
+ }
+
+ @Override
+ public AccumulatorCombiningState<InputT, AccumT, OutputT> readLater() {
+ // Ignore
+ return this;
+ }
+
+ @Override
+ public boolean shouldPersist() {
+ return !isClear;
+ }
+
+ @Override
+ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
+ if (!isClear) {
+ // serialize the coder.
+ byte[] coder = InstantiationUtil.serializeObject(accumCoder);
+
+ // serialize the combiner.
+ byte[] combiner = InstantiationUtil.serializeObject(combineFn);
+
+ // encode the accumulator into a ByteString
+ ByteString.Output stream = ByteString.newOutput();
+ accumCoder.encode(accum, stream, Coder.Context.OUTER);
+ ByteString data = stream.toByteString();
+
+ // put the flag that the next serialized element is an accumulator
+ checkpointBuilder.addAccumulatorBuilder()
+ .setTag(stateKey)
+ .setData(coder)
+ .setData(combiner)
+ .setData(data);
+ }
+ }
+
+ public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
+ ByteString valueContent = checkpointReader.getData();
+ AccumT accum = this.accumCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER);
+ addAccum(accum);
+ }
+ }
+
+ private static final class FlinkInMemoryBag<T> implements BagState<T>, CheckpointableIF {
+ private final List<T> contents = new ArrayList<>();
+
+ private final ByteString stateKey;
+ private final Coder<T> elemCoder;
+
+ public FlinkInMemoryBag(ByteString stateKey, Coder<T> elemCoder) {
+ this.stateKey = stateKey;
+ this.elemCoder = elemCoder;
+ }
+
+ @Override
+ public void clear() {
+ contents.clear();
+ }
+
+ @Override
+ public Iterable<T> read() {
+ return contents;
+ }
+
+ @Override
+ public BagState<T> readLater() {
+ // Ignore
+ return this;
+ }
+
+ @Override
+ public void add(T input) {
+ contents.add(input);
+ }
+
+ @Override
+ public ReadableState<Boolean> isEmpty() {
+ return new ReadableState<Boolean>() {
+ @Override
+ public ReadableState<Boolean> readLater() {
+ // Ignore
+ return this;
+ }
+
+ @Override
+ public Boolean read() {
+ return contents.isEmpty();
+ }
+ };
+ }
+
+ @Override
+ public boolean shouldPersist() {
+ return !contents.isEmpty();
+ }
+
+ @Override
+ public void persistState(StateCheckpointWriter checkpointBuilder) throws IOException {
+ if (!contents.isEmpty()) {
+ // serialize the coder.
+ byte[] coder = InstantiationUtil.serializeObject(elemCoder);
+
+ checkpointBuilder.addListUpdatesBuilder()
+ .setTag(stateKey)
+ .setData(coder)
+ .writeInt(contents.size());
+
+ for (T item : contents) {
+ // encode the element
+ ByteString.Output stream = ByteString.newOutput();
+ elemCoder.encode(item, stream, Coder.Context.OUTER);
+ ByteString data = stream.toByteString();
+
+ // add the data to the checkpoint.
+ checkpointBuilder.setData(data);
+ }
+ }
+ }
+
+ public void restoreState(StateCheckpointReader checkpointReader) throws IOException {
+ int noOfValues = checkpointReader.getInt();
+ for (int j = 0; j < noOfValues; j++) {
+ ByteString valueContent = checkpointReader.getData();
+ T outValue = elemCoder.decode(new ByteArrayInputStream(valueContent.toByteArray()), Coder.Context.OUTER);
+ add(outValue);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java
new file mode 100644
index 0000000..d73ac8c
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointReader.java
@@ -0,0 +1,89 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers.streaming.state;
+
+import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
+import com.google.protobuf.ByteString;
+import org.apache.flink.core.memory.DataInputView;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+public class StateCheckpointReader {
+
+ private final DataInputView input;
+
+ public StateCheckpointReader(DataInputView in) {
+ this.input = in;
+ }
+
+ public ByteString getTag() throws IOException {
+ return ByteString.copyFrom(readRawData());
+ }
+
+ public String getTagToString() throws IOException {
+ return input.readUTF();
+ }
+
+ public ByteString getData() throws IOException {
+ return ByteString.copyFrom(readRawData());
+ }
+
+ public int getInt() throws IOException {
+ validate();
+ return input.readInt();
+ }
+
+ public byte getByte() throws IOException {
+ validate();
+ return input.readByte();
+ }
+
+ public Instant getTimestamp() throws IOException {
+ validate();
+ Long watermarkMillis = input.readLong();
+ return new Instant(TimeUnit.MICROSECONDS.toMillis(watermarkMillis));
+ }
+
+ public <K> K deserializeKey(CoderTypeSerializer<K> keySerializer) throws IOException {
+ return deserializeObject(keySerializer);
+ }
+
+ public <T> T deserializeObject(CoderTypeSerializer<T> objectSerializer) throws IOException {
+ return objectSerializer.deserialize(input);
+ }
+
+ ///////// Helper Methods ///////
+
+ private byte[] readRawData() throws IOException {
+ validate();
+ int size = input.readInt();
+
+ byte[] serData = new byte[size];
+ int bytesRead = input.read(serData);
+ if (bytesRead != size) {
+ throw new RuntimeException("Error while deserializing checkpoint. Not enough bytes in the input stream.");
+ }
+ return serData;
+ }
+
+ private void validate() {
+ if (this.input == null) {
+ throw new RuntimeException("StateBackend not initialized yet.");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java
new file mode 100644
index 0000000..055a12a
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointUtils.java
@@ -0,0 +1,153 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers.streaming.state;
+
+import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.OutputTimeFn;
+import com.google.cloud.dataflow.sdk.util.TimeDomain;
+import com.google.cloud.dataflow.sdk.util.TimerInternals;
+import com.google.cloud.dataflow.sdk.util.state.StateNamespace;
+import com.google.cloud.dataflow.sdk.util.state.StateNamespaces;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public class StateCheckpointUtils {
+
+ public static <K> void encodeState(Map<K, FlinkStateInternals<K>> perKeyStateInternals,
+ StateCheckpointWriter writer, Coder<K> keyCoder) throws IOException {
+ CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
+
+ int noOfKeys = perKeyStateInternals.size();
+ writer.writeInt(noOfKeys);
+ for (Map.Entry<K, FlinkStateInternals<K>> keyStatePair : perKeyStateInternals.entrySet()) {
+ K key = keyStatePair.getKey();
+ FlinkStateInternals<K> state = keyStatePair.getValue();
+
+ // encode the key
+ writer.serializeKey(key, keySerializer);
+
+ // write the associated state
+ state.persistState(writer);
+ }
+ }
+
+ public static <K> Map<K, FlinkStateInternals<K>> decodeState(
+ StateCheckpointReader reader,
+ OutputTimeFn<? super BoundedWindow> outputTimeFn,
+ Coder<K> keyCoder,
+ Coder<? extends BoundedWindow> windowCoder,
+ ClassLoader classLoader) throws IOException, ClassNotFoundException {
+
+ int noOfKeys = reader.getInt();
+ Map<K, FlinkStateInternals<K>> perKeyStateInternals = new HashMap<>(noOfKeys);
+ perKeyStateInternals.clear();
+
+ CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
+ for (int i = 0; i < noOfKeys; i++) {
+
+ // decode the key.
+ K key = reader.deserializeKey(keySerializer);
+
+ //decode the state associated to the key.
+ FlinkStateInternals<K> stateForKey =
+ new FlinkStateInternals<>(key, keyCoder, windowCoder, outputTimeFn);
+ stateForKey.restoreState(reader, classLoader);
+ perKeyStateInternals.put(key, stateForKey);
+ }
+ return perKeyStateInternals;
+ }
+
+ ////////////// Encoding/Decoding the Timers ////////////////
+
+
+ public static <K> void encodeTimers(Map<K, Set<TimerInternals.TimerData>> allTimers,
+ StateCheckpointWriter writer,
+ Coder<K> keyCoder) throws IOException {
+ CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
+
+ int noOfKeys = allTimers.size();
+ writer.writeInt(noOfKeys);
+ for (Map.Entry<K, Set<TimerInternals.TimerData>> timersPerKey : allTimers.entrySet()) {
+ K key = timersPerKey.getKey();
+
+ // encode the key
+ writer.serializeKey(key, keySerializer);
+
+ // write the associated timers
+ Set<TimerInternals.TimerData> timers = timersPerKey.getValue();
+ encodeTimerDataForKey(writer, timers);
+ }
+ }
+
+ public static <K> Map<K, Set<TimerInternals.TimerData>> decodeTimers(
+ StateCheckpointReader reader,
+ Coder<? extends BoundedWindow> windowCoder,
+ Coder<K> keyCoder) throws IOException {
+
+ int noOfKeys = reader.getInt();
+ Map<K, Set<TimerInternals.TimerData>> activeTimers = new HashMap<>(noOfKeys);
+ activeTimers.clear();
+
+ CoderTypeSerializer<K> keySerializer = new CoderTypeSerializer<>(keyCoder);
+ for (int i = 0; i < noOfKeys; i++) {
+
+ // decode the key.
+ K key = reader.deserializeKey(keySerializer);
+
+ // decode the associated timers.
+ Set<TimerInternals.TimerData> timers = decodeTimerDataForKey(reader, windowCoder);
+ activeTimers.put(key, timers);
+ }
+ return activeTimers;
+ }
+
+ private static void encodeTimerDataForKey(StateCheckpointWriter writer, Set<TimerInternals.TimerData> timers) throws IOException {
+ // encode timers
+ writer.writeInt(timers.size());
+ for (TimerInternals.TimerData timer : timers) {
+ String stringKey = timer.getNamespace().stringKey();
+
+ writer.setTag(stringKey);
+ writer.setTimestamp(timer.getTimestamp());
+ writer.writeInt(timer.getDomain().ordinal());
+ }
+ }
+
+ private static Set<TimerInternals.TimerData> decodeTimerDataForKey(
+ StateCheckpointReader reader, Coder<? extends BoundedWindow> windowCoder) throws IOException {
+
+ // decode the timers: first their number and then the content itself.
+ int noOfTimers = reader.getInt();
+ Set<TimerInternals.TimerData> timers = new HashSet<>(noOfTimers);
+ for (int i = 0; i < noOfTimers; i++) {
+ String stringKey = reader.getTagToString();
+ Instant instant = reader.getTimestamp();
+ TimeDomain domain = TimeDomain.values()[reader.getInt()];
+
+ StateNamespace namespace = StateNamespaces.fromString(stringKey, windowCoder);
+ timers.add(TimerInternals.TimerData.of(namespace, instant, domain));
+ }
+ return timers;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java
new file mode 100644
index 0000000..738ce5f
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateCheckpointWriter.java
@@ -0,0 +1,127 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers.streaming.state;
+
+import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer;
+import com.google.protobuf.ByteString;
+import org.apache.flink.runtime.state.AbstractStateBackend;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+public class StateCheckpointWriter {
+
+ private final AbstractStateBackend.CheckpointStateOutputView output;
+
+ public static StateCheckpointWriter create(AbstractStateBackend.CheckpointStateOutputView output) {
+ return new StateCheckpointWriter(output);
+ }
+
+ private StateCheckpointWriter(AbstractStateBackend.CheckpointStateOutputView output) {
+ this.output = output;
+ }
+
+ ///////// Creating the serialized versions of the different types of state held by dataflow ///////
+
+ public StateCheckpointWriter addValueBuilder() throws IOException {
+ validate();
+ StateType.serialize(StateType.VALUE, this);
+ return this;
+ }
+
+ public StateCheckpointWriter addWatermarkHoldsBuilder() throws IOException {
+ validate();
+ StateType.serialize(StateType.WATERMARK, this);
+ return this;
+ }
+
+ public StateCheckpointWriter addListUpdatesBuilder() throws IOException {
+ validate();
+ StateType.serialize(StateType.LIST, this);
+ return this;
+ }
+
+ public StateCheckpointWriter addAccumulatorBuilder() throws IOException {
+ validate();
+ StateType.serialize(StateType.ACCUMULATOR, this);
+ return this;
+ }
+
+ ///////// Setting the tag for a given state element ///////
+
+ public StateCheckpointWriter setTag(ByteString stateKey) throws IOException {
+ return writeData(stateKey.toByteArray());
+ }
+
+ public StateCheckpointWriter setTag(String stateKey) throws IOException {
+ output.writeUTF(stateKey);
+ return this;
+ }
+
+
+ public <K> StateCheckpointWriter serializeKey(K key, CoderTypeSerializer<K> keySerializer) throws IOException {
+ return serializeObject(key, keySerializer);
+ }
+
+ public <T> StateCheckpointWriter serializeObject(T object, CoderTypeSerializer<T> objectSerializer) throws IOException {
+ objectSerializer.serialize(object, output);
+ return this;
+ }
+
+ ///////// Write the actual serialized data //////////
+
+ public StateCheckpointWriter setData(ByteString data) throws IOException {
+ return writeData(data.toByteArray());
+ }
+
+ public StateCheckpointWriter setData(byte[] data) throws IOException {
+ return writeData(data);
+ }
+
+ public StateCheckpointWriter setTimestamp(Instant timestamp) throws IOException {
+ validate();
+ output.writeLong(TimeUnit.MILLISECONDS.toMicros(timestamp.getMillis()));
+ return this;
+ }
+
+ public StateCheckpointWriter writeInt(int number) throws IOException {
+ validate();
+ output.writeInt(number);
+ return this;
+ }
+
+ public StateCheckpointWriter writeByte(byte b) throws IOException {
+ validate();
+ output.writeByte(b);
+ return this;
+ }
+
+ ///////// Helper Methods ///////
+
+ private StateCheckpointWriter writeData(byte[] data) throws IOException {
+ validate();
+ output.writeInt(data.length);
+ output.write(data);
+ return this;
+ }
+
+ private void validate() {
+ if (this.output == null) {
+ throw new RuntimeException("StateBackend not initialized yet.");
+ }
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateType.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateType.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateType.java
new file mode 100644
index 0000000..8b20600
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/StateType.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers.streaming.state;
+
+import java.io.IOException;
+
+/**
+ * The available types of state, as provided by the Beam SDK. This class is used for serialization/deserialization
+ * purposes.
+ * */
+public enum StateType {
+
+ VALUE(0),
+
+ WATERMARK(1),
+
+ LIST(2),
+
+ ACCUMULATOR(3);
+
+ private final int numVal;
+
+ StateType(int value) {
+ this.numVal = value;
+ }
+
+ public static void serialize(StateType type, StateCheckpointWriter output) throws IOException {
+ if (output == null) {
+ throw new IllegalArgumentException("Cannot write to a null output.");
+ }
+
+ if(type.numVal < 0 || type.numVal > 3) {
+ throw new RuntimeException("Unknown State Type " + type + ".");
+ }
+
+ output.writeByte((byte) type.numVal);
+ }
+
+ public static StateType deserialize(StateCheckpointReader input) throws IOException {
+ if (input == null) {
+ throw new IllegalArgumentException("Cannot read from a null input.");
+ }
+
+ int typeInt = (int) input.getByte();
+ if(typeInt < 0 || typeInt > 3) {
+ throw new RuntimeException("Unknown State Type " + typeInt + ".");
+ }
+
+ StateType resultType = null;
+ for(StateType st: values()) {
+ if(st.numVal == typeInt) {
+ resultType = st;
+ break;
+ }
+ }
+ return resultType;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java
deleted file mode 100644
index eaa5979..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/AvroITCase.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.AvroCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.common.base.Joiner;
-import org.apache.flink.api.io.avro.example.User;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-
-public class AvroITCase extends JavaProgramTestBase {
-
- protected String resultPath;
- protected String tmpPath;
-
- public AvroITCase(){
- }
-
- static final String[] EXPECTED_RESULT = new String[] {
- "Joe red 3",
- "Mary blue 4",
- "Mark green 1",
- "Julia purple 5"
- };
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- tmpPath = getTempDirPath("tmp");
-
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
- runProgram(tmpPath, resultPath);
- }
-
- private static void runProgram(String tmpPath, String resultPath) {
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- p
- .apply(Create.of(
- new User("Joe", 3, "red"),
- new User("Mary", 4, "blue"),
- new User("Mark", 1, "green"),
- new User("Julia", 5, "purple"))
- .withCoder(AvroCoder.of(User.class)))
-
- .apply(AvroIO.Write.to(tmpPath)
- .withSchema(User.class));
-
- p.run();
-
- p = FlinkTestPipeline.createForBatch();
-
- p
- .apply(AvroIO.Read.from(tmpPath).withSchema(User.class).withoutValidation())
-
- .apply(ParDo.of(new DoFn<User, String>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- User u = c.element();
- String result = u.getName() + " " + u.getFavoriteColor() + " " + u.getFavoriteNumber();
- c.output(result);
- }
- }))
-
- .apply(TextIO.Write.to(resultPath));
-
- p.run();
- }
-
-}
-
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java
deleted file mode 100644
index 79eb163..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlattenizeITCase.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionList;
-import com.google.common.base.Joiner;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-public class FlattenizeITCase extends JavaProgramTestBase {
-
- private String resultPath;
- private String resultPath2;
-
- private static final String[] words = {"hello", "this", "is", "a", "DataSet!"};
- private static final String[] words2 = {"hello", "this", "is", "another", "DataSet!"};
- private static final String[] words3 = {"hello", "this", "is", "yet", "another", "DataSet!"};
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- resultPath2 = getTempDirPath("result2");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- String join = Joiner.on('\n').join(words);
- String join2 = Joiner.on('\n').join(words2);
- String join3 = Joiner.on('\n').join(words3);
- compareResultsByLinesInMemory(join + "\n" + join2, resultPath);
- compareResultsByLinesInMemory(join + "\n" + join2 + "\n" + join3, resultPath2);
- }
-
-
- @Override
- protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- PCollection<String> p1 = p.apply(Create.of(words));
- PCollection<String> p2 = p.apply(Create.of(words2));
-
- PCollectionList<String> list = PCollectionList.of(p1).and(p2);
-
- list.apply(Flatten.<String>pCollections()).apply(TextIO.Write.to(resultPath));
-
- PCollection<String> p3 = p.apply(Create.of(words3));
-
- PCollectionList<String> list2 = list.and(p3);
-
- list2.apply(Flatten.<String>pCollections()).apply(TextIO.Write.to(resultPath2));
-
- p.run();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
deleted file mode 100644
index 2dcebde..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/FlinkTestPipeline.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.PipelineResult;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.runners.PipelineRunner;
-
-/**
- * {@link com.google.cloud.dataflow.sdk.Pipeline} for testing Dataflow programs on the
- * {@link org.apache.beam.runners.flink.FlinkPipelineRunner}.
- */
-public class FlinkTestPipeline extends Pipeline {
-
- /**
- * Creates and returns a new test pipeline for batch execution.
- *
- * <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
- * {@link Pipeline#run} to execute the pipeline and check the tests.
- */
- public static FlinkTestPipeline createForBatch() {
- return create(false);
- }
-
- /**
- * Creates and returns a new test pipeline for streaming execution.
- *
- * <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
- * {@link Pipeline#run} to execute the pipeline and check the tests.
- *
- * @return The Test Pipeline
- */
- public static FlinkTestPipeline createForStreaming() {
- return create(true);
- }
-
- /**
- * Creates and returns a new test pipeline for streaming or batch execution.
- *
- * <p> Use {@link com.google.cloud.dataflow.sdk.testing.DataflowAssert} to add tests, then call
- * {@link Pipeline#run} to execute the pipeline and check the tests.
- *
- * @param streaming <code>True</code> for streaming mode, <code>False</code> for batch.
- * @return The Test Pipeline.
- */
- private static FlinkTestPipeline create(boolean streaming) {
- FlinkPipelineRunner flinkRunner = FlinkPipelineRunner.createForTest(streaming);
- return new FlinkTestPipeline(flinkRunner, flinkRunner.getPipelineOptions());
- }
-
- private FlinkTestPipeline(PipelineRunner<? extends PipelineResult> runner,
- PipelineOptions options) {
- super(runner, options);
- }
-}
-
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java
deleted file mode 100644
index 11b6ce4..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/JoinExamplesITCase.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import org.apache.beam.runners.flink.util.JoinExamples;
-import com.google.api.services.bigquery.model.TableRow;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Joiner;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-import java.util.Arrays;
-import java.util.List;
-
-
-/**
- * Unfortunately we need to copy the code from the Dataflow SDK because it is not public there.
- */
-public class JoinExamplesITCase extends JavaProgramTestBase {
-
- protected String resultPath;
-
- public JoinExamplesITCase(){
- }
-
- private static final TableRow row1 = new TableRow()
- .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212")
- .set("Actor1Name", "BANGKOK").set("SOURCEURL", "http://cnn.com");
- private static final TableRow row2 = new TableRow()
- .set("ActionGeo_CountryCode", "VM").set("SQLDATE", "20141212")
- .set("Actor1Name", "LAOS").set("SOURCEURL", "http://www.chicagotribune.com");
- private static final TableRow row3 = new TableRow()
- .set("ActionGeo_CountryCode", "BE").set("SQLDATE", "20141213")
- .set("Actor1Name", "AFGHANISTAN").set("SOURCEURL", "http://cnn.com");
- static final TableRow[] EVENTS = new TableRow[] {
- row1, row2, row3
- };
- static final List<TableRow> EVENT_ARRAY = Arrays.asList(EVENTS);
-
- private static final TableRow cc1 = new TableRow()
- .set("FIPSCC", "VM").set("HumanName", "Vietnam");
- private static final TableRow cc2 = new TableRow()
- .set("FIPSCC", "BE").set("HumanName", "Belgium");
- static final TableRow[] CCS = new TableRow[] {
- cc1, cc2
- };
- static final List<TableRow> CC_ARRAY = Arrays.asList(CCS);
-
- static final String[] JOINED_EVENTS = new String[] {
- "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: LAOS, "
- + "url: http://www.chicagotribune.com",
- "Country code: VM, Country name: Vietnam, Event info: Date: 20141212, Actor1: BANGKOK, "
- + "url: http://cnn.com",
- "Country code: BE, Country name: Belgium, Event info: Date: 20141213, Actor1: AFGHANISTAN, "
- + "url: http://cnn.com"
- };
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(JOINED_EVENTS), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
-
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- PCollection<TableRow> input1 = p.apply(Create.of(EVENT_ARRAY));
- PCollection<TableRow> input2 = p.apply(Create.of(CC_ARRAY));
-
- PCollection<String> output = JoinExamples.joinEvents(input1, input2);
-
- output.apply(TextIO.Write.to(resultPath));
-
- p.run();
- }
-}
-
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java
deleted file mode 100644
index e39b81d..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/MaybeEmptyTestITCase.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.VoidCoder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-import java.io.Serializable;
-
-public class MaybeEmptyTestITCase extends JavaProgramTestBase implements Serializable {
-
- protected String resultPath;
-
- protected final String expected = "test";
-
- public MaybeEmptyTestITCase() {
- }
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(expected, resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
-
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- p.apply(Create.of((Void) null)).setCoder(VoidCoder.of())
- .apply(ParDo.of(
- new DoFn<Void, String>() {
- @Override
- public void processElement(DoFn<Void, String>.ProcessContext c) {
- c.output(expected);
- }
- })).apply(TextIO.Write.to(resultPath));
- p.run();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java
deleted file mode 100644
index 08e5323..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ParDoMultiOutputITCase.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.cloud.dataflow.sdk.values.TupleTagList;
-import com.google.common.base.Joiner;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-import java.io.Serializable;
-
-public class ParDoMultiOutputITCase extends JavaProgramTestBase implements Serializable {
-
- private String resultPath;
-
- private static String[] expectedWords = {"MAAA", "MAAFOOO"};
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on("\n").join(expectedWords), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- PCollection<String> words = p.apply(Create.of("Hello", "Whatupmyman", "hey", "SPECIALthere", "MAAA", "MAAFOOO"));
-
- // Select words whose length is below a cut off,
- // plus the lengths of words that are above the cut off.
- // Also select words starting with "MARKER".
- final int wordLengthCutOff = 3;
- // Create tags to use for the main and side outputs.
- final TupleTag<String> wordsBelowCutOffTag = new TupleTag<String>(){};
- final TupleTag<Integer> wordLengthsAboveCutOffTag = new TupleTag<Integer>(){};
- final TupleTag<String> markedWordsTag = new TupleTag<String>(){};
-
- PCollectionTuple results =
- words.apply(ParDo
- .withOutputTags(wordsBelowCutOffTag, TupleTagList.of(wordLengthsAboveCutOffTag)
- .and(markedWordsTag))
- .of(new DoFn<String, String>() {
- final TupleTag<String> specialWordsTag = new TupleTag<String>() {
- };
-
- public void processElement(ProcessContext c) {
- String word = c.element();
- if (word.length() <= wordLengthCutOff) {
- c.output(word);
- } else {
- c.sideOutput(wordLengthsAboveCutOffTag, word.length());
- }
- if (word.startsWith("MAA")) {
- c.sideOutput(markedWordsTag, word);
- }
-
- if (word.startsWith("SPECIAL")) {
- c.sideOutput(specialWordsTag, word);
- }
- }
- }));
-
- // Extract the PCollection results, by tag.
- PCollection<String> wordsBelowCutOff = results.get(wordsBelowCutOffTag);
- PCollection<Integer> wordLengthsAboveCutOff = results.get
- (wordLengthsAboveCutOffTag);
- PCollection<String> markedWords = results.get(markedWordsTag);
-
- markedWords.apply(TextIO.Write.to(resultPath));
-
- p.run();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java
deleted file mode 100644
index 7202417..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/ReadSourceITCase.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.BigEndianIntegerCoder;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-
-public class ReadSourceITCase extends JavaProgramTestBase {
-
- protected String resultPath;
-
- public ReadSourceITCase(){
- }
-
- static final String[] EXPECTED_RESULT = new String[] {
- "1", "2", "3", "4", "5", "6", "7", "8", "9"};
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
- runProgram(resultPath);
- }
-
- private static void runProgram(String resultPath) {
-
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- PCollection<String> result = p
- .apply(Read.from(new ReadSource(1, 10)))
- .apply(ParDo.of(new DoFn<Integer, String>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- c.output(c.element().toString());
- }
- }));
-
- result.apply(TextIO.Write.to(resultPath));
- p.run();
- }
-
-
- private static class ReadSource extends BoundedSource<Integer> {
- final int from;
- final int to;
-
- ReadSource(int from, int to) {
- this.from = from;
- this.to = to;
- }
-
- @Override
- public List<ReadSource> splitIntoBundles(long desiredShardSizeBytes, PipelineOptions options)
- throws Exception {
- List<ReadSource> res = new ArrayList<>();
- FlinkPipelineOptions flinkOptions = options.as(FlinkPipelineOptions.class);
- int numWorkers = flinkOptions.getParallelism();
- Preconditions.checkArgument(numWorkers > 0, "Number of workers should be larger than 0.");
-
- float step = 1.0f * (to - from) / numWorkers;
- for (int i = 0; i < numWorkers; ++i) {
- res.add(new ReadSource(Math.round(from + i * step), Math.round(from + (i + 1) * step)));
- }
- return res;
- }
-
- @Override
- public long getEstimatedSizeBytes(PipelineOptions options) throws Exception {
- return 8 * (to - from);
- }
-
- @Override
- public boolean producesSortedKeys(PipelineOptions options) throws Exception {
- return true;
- }
-
- @Override
- public BoundedReader<Integer> createReader(PipelineOptions options) throws IOException {
- return new RangeReader(this);
- }
-
- @Override
- public void validate() {}
-
- @Override
- public Coder<Integer> getDefaultOutputCoder() {
- return BigEndianIntegerCoder.of();
- }
-
- private class RangeReader extends BoundedReader<Integer> {
- private int current;
-
- public RangeReader(ReadSource source) {
- this.current = source.from - 1;
- }
-
- @Override
- public boolean start() throws IOException {
- return true;
- }
-
- @Override
- public boolean advance() throws IOException {
- current++;
- return (current < to);
- }
-
- @Override
- public Integer getCurrent() {
- return current;
- }
-
- @Override
- public void close() throws IOException {
- // Nothing
- }
-
- @Override
- public BoundedSource<Integer> getCurrentSource() {
- return ReadSource.this;
- }
- }
- }
-}
-
-
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java
deleted file mode 100644
index dc82d7d..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesEmptyITCase.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Joiner;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-import java.util.Collections;
-import java.util.List;
-
-
-public class RemoveDuplicatesEmptyITCase extends JavaProgramTestBase {
-
- protected String resultPath;
-
- public RemoveDuplicatesEmptyITCase(){
- }
-
- static final String[] EXPECTED_RESULT = new String[] {};
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
-
- List<String> strings = Collections.emptyList();
-
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- PCollection<String> input =
- p.apply(Create.of(strings))
- .setCoder(StringUtf8Coder.of());
-
- PCollection<String> output =
- input.apply(RemoveDuplicates.<String>create());
-
- output.apply(TextIO.Write.to(resultPath));
- p.run();
- }
-}
-
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java
deleted file mode 100644
index 78b48b5..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/RemoveDuplicatesITCase.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Joiner;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-import java.util.Arrays;
-import java.util.List;
-
-
-public class RemoveDuplicatesITCase extends JavaProgramTestBase {
-
- protected String resultPath;
-
- public RemoveDuplicatesITCase(){
- }
-
- static final String[] EXPECTED_RESULT = new String[] {
- "k1", "k5", "k2", "k3"};
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
-
- List<String> strings = Arrays.asList("k1", "k5", "k5", "k2", "k1", "k2", "k3");
-
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- PCollection<String> input =
- p.apply(Create.of(strings))
- .setCoder(StringUtf8Coder.of());
-
- PCollection<String> output =
- input.apply(RemoveDuplicates.<String>create());
-
- output.apply(TextIO.Write.to(resultPath));
- p.run();
- }
-}
-
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java
deleted file mode 100644
index 5cd7d78..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/SideInputITCase.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-import java.io.Serializable;
-
-public class SideInputITCase extends JavaProgramTestBase implements Serializable {
-
- private static final String expected = "Hello!";
-
- protected String resultPath;
-
- @Override
- protected void testProgram() throws Exception {
-
-
- Pipeline p = FlinkTestPipeline.createForBatch();
-
-
- final PCollectionView<String> sidesInput = p
- .apply(Create.of(expected))
- .apply(View.<String>asSingleton());
-
- p.apply(Create.of("bli"))
- .apply(ParDo.of(new DoFn<String, String>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- String s = c.sideInput(sidesInput);
- c.output(s);
- }
- }).withSideInputs(sidesInput)).apply(TextIO.Write.to(resultPath));
-
- p.run();
- }
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(expected, resultPath);
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java
deleted file mode 100644
index ceb0a3f..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/TfIdfITCase.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import org.apache.beam.runners.flink.examples.TFIDF;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.Keys;
-import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Joiner;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-import java.net.URI;
-
-
-public class TfIdfITCase extends JavaProgramTestBase {
-
- protected String resultPath;
-
- public TfIdfITCase(){
- }
-
- static final String[] EXPECTED_RESULT = new String[] {
- "a", "m", "n", "b", "c", "d"};
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
-
- Pipeline pipeline = FlinkTestPipeline.createForBatch();
-
- pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
-
- PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf = pipeline
- .apply(Create.of(
- KV.of(new URI("x"), "a b c d"),
- KV.of(new URI("y"), "a b c"),
- KV.of(new URI("z"), "a m n")))
- .apply(new TFIDF.ComputeTfIdf());
-
- PCollection<String> words = wordToUriAndTfIdf
- .apply(Keys.<String>create())
- .apply(RemoveDuplicates.<String>create());
-
- words.apply(TextIO.Write.to(resultPath));
-
- pipeline.run();
- }
-}
-
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java
deleted file mode 100644
index c2b6fdd..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountITCase.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import org.apache.beam.runners.flink.examples.WordCount;
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.MapElements;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.common.base.Joiner;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-import java.util.Arrays;
-import java.util.List;
-
-
-public class WordCountITCase extends JavaProgramTestBase {
-
- protected String resultPath;
-
- public WordCountITCase(){
- }
-
- static final String[] WORDS_ARRAY = new String[] {
- "hi there", "hi", "hi sue bob",
- "hi sue", "", "bob hi"};
-
- static final List<String> WORDS = Arrays.asList(WORDS_ARRAY);
-
- static final String[] COUNTS_ARRAY = new String[] {
- "hi: 5", "there: 1", "sue: 2", "bob: 2"};
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(COUNTS_ARRAY), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
-
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- PCollection<String> input = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of());
-
- input
- .apply(new WordCount.CountWords())
- .apply(MapElements.via(new WordCount.FormatAsTextFn()))
- .apply(TextIO.Write.to(resultPath));
-
- p.run();
- }
-}
-
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java
deleted file mode 100644
index d78434b..0000000
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/WordCountJoin2ITCase.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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.beam.runners.flink;
-
-import com.google.cloud.dataflow.sdk.Pipeline;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Count;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.base.Joiner;
-import org.apache.flink.test.util.JavaProgramTestBase;
-
-
-public class WordCountJoin2ITCase extends JavaProgramTestBase {
-
- static final String[] WORDS_1 = new String[] {
- "hi there", "hi", "hi sue bob",
- "hi sue", "", "bob hi"};
-
- static final String[] WORDS_2 = new String[] {
- "hi tim", "beauty", "hooray sue bob",
- "hi there", "", "please say hi"};
-
- static final String[] RESULTS = new String[] {
- "beauty -> Tag1: Tag2: 1",
- "bob -> Tag1: 2 Tag2: 1",
- "hi -> Tag1: 5 Tag2: 3",
- "hooray -> Tag1: Tag2: 1",
- "please -> Tag1: Tag2: 1",
- "say -> Tag1: Tag2: 1",
- "sue -> Tag1: 2 Tag2: 1",
- "there -> Tag1: 1 Tag2: 1",
- "tim -> Tag1: Tag2: 1"
- };
-
- static final TupleTag<Long> tag1 = new TupleTag<>("Tag1");
- static final TupleTag<Long> tag2 = new TupleTag<>("Tag2");
-
- protected String resultPath;
-
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
-
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(RESULTS), resultPath);
- }
-
- @Override
- protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.createForBatch();
-
- /* Create two PCollections and join them */
- PCollection<KV<String,Long>> occurences1 = p.apply(Create.of(WORDS_1))
- .apply(ParDo.of(new ExtractWordsFn()))
- .apply(Count.<String>perElement());
-
- PCollection<KV<String,Long>> occurences2 = p.apply(Create.of(WORDS_2))
- .apply(ParDo.of(new ExtractWordsFn()))
- .apply(Count.<String>perElement());
-
- /* CoGroup the two collections */
- PCollection<KV<String, CoGbkResult>> mergedOccurences = KeyedPCollectionTuple
- .of(tag1, occurences1)
- .and(tag2, occurences2)
- .apply(CoGroupByKey.<String>create());
-
- /* Format output */
- mergedOccurences.apply(ParDo.of(new FormatCountsFn()))
- .apply(TextIO.Write.named("test").to(resultPath));
-
- p.run();
- }
-
-
- static class ExtractWordsFn extends DoFn<String, String> {
-
- @Override
- public void startBundle(Context c) {
- }
-
- @Override
- public void processElement(ProcessContext c) {
- // Split the line into words.
- String[] words = c.element().split("[^a-zA-Z']+");
-
- // Output each word encountered into the output PCollection.
- for (String word : words) {
- if (!word.isEmpty()) {
- c.output(word);
- }
- }
- }
- }
-
- static class FormatCountsFn extends DoFn<KV<String, CoGbkResult>, String> {
- @Override
- public void processElement(ProcessContext c) {
- CoGbkResult value = c.element().getValue();
- String key = c.element().getKey();
- String countTag1 = tag1.getId() + ": ";
- String countTag2 = tag2.getId() + ": ";
- for (Long count : value.getAll(tag1)) {
- countTag1 += count + " ";
- }
- for (Long count : value.getAll(tag2)) {
- countTag2 += count;
- }
- c.output(key + " -> " + countTag1 + countTag2);
- }
- }
-
-
-}
[26/50] [abbrv] incubator-beam git commit: [flink] adjust directories
according to package name
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
new file mode 100644
index 0000000..ab23b92
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/TFIDF.java
@@ -0,0 +1,452 @@
+/*
+ * Copyright (C) 2015 Google Inc.
+ *
+ * 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://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.beam.runners.flink.examples;
+
+import org.apache.beam.runners.flink.FlinkPipelineOptions;
+import org.apache.beam.runners.flink.FlinkPipelineRunner;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.coders.StringDelegateCoder;
+import com.google.cloud.dataflow.sdk.coders.StringUtf8Coder;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.GcsOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.options.Validation;
+import com.google.cloud.dataflow.sdk.transforms.Count;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.Flatten;
+import com.google.cloud.dataflow.sdk.transforms.Keys;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.RemoveDuplicates;
+import com.google.cloud.dataflow.sdk.transforms.Values;
+import com.google.cloud.dataflow.sdk.transforms.View;
+import com.google.cloud.dataflow.sdk.transforms.WithKeys;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
+import com.google.cloud.dataflow.sdk.util.GcsUtil;
+import com.google.cloud.dataflow.sdk.util.gcsfs.GcsPath;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.PDone;
+import com.google.cloud.dataflow.sdk.values.PInput;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.HashSet;
+import java.util.Set;
+
+/**
+ * An example that computes a basic TF-IDF search table for a directory or GCS prefix.
+ *
+ * <p> Concepts: joining data; side inputs; logging
+ *
+ * <p> To execute this pipeline locally, specify general pipeline configuration:
+ * <pre>{@code
+ * --project=YOUR_PROJECT_ID
+ * }</pre>
+ * and a local output file or output prefix on GCS:
+ * <pre>{@code
+ * --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
+ * }</pre>
+ *
+ * <p> To execute this pipeline using the Dataflow service, specify pipeline configuration:
+ * <pre>{@code
+ * --project=YOUR_PROJECT_ID
+ * --stagingLocation=gs://YOUR_STAGING_DIRECTORY
+ * --runner=BlockingDataflowPipelineRunner
+ * and an output prefix on GCS:
+ * --output=gs://YOUR_OUTPUT_PREFIX
+ * }</pre>
+ *
+ * <p> The default input is {@code gs://dataflow-samples/shakespeare/} and can be overridden with
+ * {@code --input}.
+ */
+public class TFIDF {
+ /**
+ * Options supported by {@link TFIDF}.
+ * <p>
+ * Inherits standard configuration options.
+ */
+ private interface Options extends PipelineOptions, FlinkPipelineOptions {
+ @Description("Path to the directory or GCS prefix containing files to read from")
+ @Default.String("gs://dataflow-samples/shakespeare/")
+ String getInput();
+ void setInput(String value);
+
+ @Description("Prefix of output URI to write to")
+ @Validation.Required
+ String getOutput();
+ void setOutput(String value);
+ }
+
+ /**
+ * Lists documents contained beneath the {@code options.input} prefix/directory.
+ */
+ public static Set<URI> listInputDocuments(Options options)
+ throws URISyntaxException, IOException {
+ URI baseUri = new URI(options.getInput());
+
+ // List all documents in the directory or GCS prefix.
+ URI absoluteUri;
+ if (baseUri.getScheme() != null) {
+ absoluteUri = baseUri;
+ } else {
+ absoluteUri = new URI(
+ "file",
+ baseUri.getAuthority(),
+ baseUri.getPath(),
+ baseUri.getQuery(),
+ baseUri.getFragment());
+ }
+
+ Set<URI> uris = new HashSet<>();
+ if (absoluteUri.getScheme().equals("file")) {
+ File directory = new File(absoluteUri);
+ for (String entry : directory.list()) {
+ File path = new File(directory, entry);
+ uris.add(path.toURI());
+ }
+ } else if (absoluteUri.getScheme().equals("gs")) {
+ GcsUtil gcsUtil = options.as(GcsOptions.class).getGcsUtil();
+ URI gcsUriGlob = new URI(
+ absoluteUri.getScheme(),
+ absoluteUri.getAuthority(),
+ absoluteUri.getPath() + "*",
+ absoluteUri.getQuery(),
+ absoluteUri.getFragment());
+ for (GcsPath entry : gcsUtil.expand(GcsPath.fromUri(gcsUriGlob))) {
+ uris.add(entry.toUri());
+ }
+ }
+
+ return uris;
+ }
+
+ /**
+ * Reads the documents at the provided uris and returns all lines
+ * from the documents tagged with which document they are from.
+ */
+ public static class ReadDocuments
+ extends PTransform<PInput, PCollection<KV<URI, String>>> {
+ private static final long serialVersionUID = 0;
+
+ private Iterable<URI> uris;
+
+ public ReadDocuments(Iterable<URI> uris) {
+ this.uris = uris;
+ }
+
+ @Override
+ public Coder<?> getDefaultOutputCoder() {
+ return KvCoder.of(StringDelegateCoder.of(URI.class), StringUtf8Coder.of());
+ }
+
+ @Override
+ public PCollection<KV<URI, String>> apply(PInput input) {
+ Pipeline pipeline = input.getPipeline();
+
+ // Create one TextIO.Read transform for each document
+ // and add its output to a PCollectionList
+ PCollectionList<KV<URI, String>> urisToLines =
+ PCollectionList.empty(pipeline);
+
+ // TextIO.Read supports:
+ // - file: URIs and paths locally
+ // - gs: URIs on the service
+ for (final URI uri : uris) {
+ String uriString;
+ if (uri.getScheme().equals("file")) {
+ uriString = new File(uri).getPath();
+ } else {
+ uriString = uri.toString();
+ }
+
+ PCollection<KV<URI, String>> oneUriToLines = pipeline
+ .apply(TextIO.Read.from(uriString)
+ .named("TextIO.Read(" + uriString + ")"))
+ .apply("WithKeys(" + uriString + ")", WithKeys.<URI, String>of(uri));
+
+ urisToLines = urisToLines.and(oneUriToLines);
+ }
+
+ return urisToLines.apply(Flatten.<KV<URI, String>>pCollections());
+ }
+ }
+
+ /**
+ * A transform containing a basic TF-IDF pipeline. The input consists of KV objects
+ * where the key is the document's URI and the value is a piece
+ * of the document's content. The output is mapping from terms to
+ * scores for each document URI.
+ */
+ public static class ComputeTfIdf
+ extends PTransform<PCollection<KV<URI, String>>, PCollection<KV<String, KV<URI, Double>>>> {
+ private static final long serialVersionUID = 0;
+
+ public ComputeTfIdf() { }
+
+ @Override
+ public PCollection<KV<String, KV<URI, Double>>> apply(
+ PCollection<KV<URI, String>> uriToContent) {
+
+ // Compute the total number of documents, and
+ // prepare this singleton PCollectionView for
+ // use as a side input.
+ final PCollectionView<Long> totalDocuments =
+ uriToContent
+ .apply("GetURIs", Keys.<URI>create())
+ .apply("RemoveDuplicateDocs", RemoveDuplicates.<URI>create())
+ .apply(Count.<URI>globally())
+ .apply(View.<Long>asSingleton());
+
+ // Create a collection of pairs mapping a URI to each
+ // of the words in the document associated with that that URI.
+ PCollection<KV<URI, String>> uriToWords = uriToContent
+ .apply(ParDo.named("SplitWords").of(
+ new DoFn<KV<URI, String>, KV<URI, String>>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ URI uri = c.element().getKey();
+ String line = c.element().getValue();
+ for (String word : line.split("\\W+")) {
+ // Log INFO messages when the word “love” is found.
+ if (word.toLowerCase().equals("love")) {
+ LOG.info("Found {}", word.toLowerCase());
+ }
+
+ if (!word.isEmpty()) {
+ c.output(KV.of(uri, word.toLowerCase()));
+ }
+ }
+ }
+ }));
+
+ // Compute a mapping from each word to the total
+ // number of documents in which it appears.
+ PCollection<KV<String, Long>> wordToDocCount = uriToWords
+ .apply("RemoveDuplicateWords", RemoveDuplicates.<KV<URI, String>>create())
+ .apply(Values.<String>create())
+ .apply("CountDocs", Count.<String>perElement());
+
+ // Compute a mapping from each URI to the total
+ // number of words in the document associated with that URI.
+ PCollection<KV<URI, Long>> uriToWordTotal = uriToWords
+ .apply("GetURIs2", Keys.<URI>create())
+ .apply("CountWords", Count.<URI>perElement());
+
+ // Count, for each (URI, word) pair, the number of
+ // occurrences of that word in the document associated
+ // with the URI.
+ PCollection<KV<KV<URI, String>, Long>> uriAndWordToCount = uriToWords
+ .apply("CountWordDocPairs", Count.<KV<URI, String>>perElement());
+
+ // Adjust the above collection to a mapping from
+ // (URI, word) pairs to counts into an isomorphic mapping
+ // from URI to (word, count) pairs, to prepare for a join
+ // by the URI key.
+ PCollection<KV<URI, KV<String, Long>>> uriToWordAndCount = uriAndWordToCount
+ .apply(ParDo.named("ShiftKeys").of(
+ new DoFn<KV<KV<URI, String>, Long>, KV<URI, KV<String, Long>>>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ URI uri = c.element().getKey().getKey();
+ String word = c.element().getKey().getValue();
+ Long occurrences = c.element().getValue();
+ c.output(KV.of(uri, KV.of(word, occurrences)));
+ }
+ }));
+
+ // Prepare to join the mapping of URI to (word, count) pairs with
+ // the mapping of URI to total word counts, by associating
+ // each of the input PCollection<KV<URI, ...>> with
+ // a tuple tag. Each input must have the same key type, URI
+ // in this case. The type parameter of the tuple tag matches
+ // the types of the values for each collection.
+ final TupleTag<Long> wordTotalsTag = new TupleTag<>();
+ final TupleTag<KV<String, Long>> wordCountsTag = new TupleTag<>();
+ KeyedPCollectionTuple<URI> coGbkInput = KeyedPCollectionTuple
+ .of(wordTotalsTag, uriToWordTotal)
+ .and(wordCountsTag, uriToWordAndCount);
+
+ // Perform a CoGroupByKey (a sort of pre-join) on the prepared
+ // inputs. This yields a mapping from URI to a CoGbkResult
+ // (CoGroupByKey Result). The CoGbkResult is a mapping
+ // from the above tuple tags to the values in each input
+ // associated with a particular URI. In this case, each
+ // KV<URI, CoGbkResult> group a URI with the total number of
+ // words in that document as well as all the (word, count)
+ // pairs for particular words.
+ PCollection<KV<URI, CoGbkResult>> uriToWordAndCountAndTotal = coGbkInput
+ .apply("CoGroupByUri", CoGroupByKey.<URI>create());
+
+ // Compute a mapping from each word to a (URI, term frequency)
+ // pair for each URI. A word's term frequency for a document
+ // is simply the number of times that word occurs in the document
+ // divided by the total number of words in the document.
+ PCollection<KV<String, KV<URI, Double>>> wordToUriAndTf = uriToWordAndCountAndTotal
+ .apply(ParDo.named("ComputeTermFrequencies").of(
+ new DoFn<KV<URI, CoGbkResult>, KV<String, KV<URI, Double>>>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ URI uri = c.element().getKey();
+ Long wordTotal = c.element().getValue().getOnly(wordTotalsTag);
+
+ for (KV<String, Long> wordAndCount
+ : c.element().getValue().getAll(wordCountsTag)) {
+ String word = wordAndCount.getKey();
+ Long wordCount = wordAndCount.getValue();
+ Double termFrequency = wordCount.doubleValue() / wordTotal.doubleValue();
+ c.output(KV.of(word, KV.of(uri, termFrequency)));
+ }
+ }
+ }));
+
+ // Compute a mapping from each word to its document frequency.
+ // A word's document frequency in a corpus is the number of
+ // documents in which the word appears divided by the total
+ // number of documents in the corpus. Note how the total number of
+ // documents is passed as a side input; the same value is
+ // presented to each invocation of the DoFn.
+ PCollection<KV<String, Double>> wordToDf = wordToDocCount
+ .apply(ParDo
+ .named("ComputeDocFrequencies")
+ .withSideInputs(totalDocuments)
+ .of(new DoFn<KV<String, Long>, KV<String, Double>>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ String word = c.element().getKey();
+ Long documentCount = c.element().getValue();
+ Long documentTotal = c.sideInput(totalDocuments);
+ Double documentFrequency = documentCount.doubleValue()
+ / documentTotal.doubleValue();
+
+ c.output(KV.of(word, documentFrequency));
+ }
+ }));
+
+ // Join the term frequency and document frequency
+ // collections, each keyed on the word.
+ final TupleTag<KV<URI, Double>> tfTag = new TupleTag<>();
+ final TupleTag<Double> dfTag = new TupleTag<>();
+ PCollection<KV<String, CoGbkResult>> wordToUriAndTfAndDf = KeyedPCollectionTuple
+ .of(tfTag, wordToUriAndTf)
+ .and(dfTag, wordToDf)
+ .apply(CoGroupByKey.<String>create());
+
+ // Compute a mapping from each word to a (URI, TF-IDF) score
+ // for each URI. There are a variety of definitions of TF-IDF
+ // ("term frequency - inverse document frequency") score;
+ // here we use a basic version that is the term frequency
+ // divided by the log of the document frequency.
+
+ return wordToUriAndTfAndDf
+ .apply(ParDo.named("ComputeTfIdf").of(
+ new DoFn<KV<String, CoGbkResult>, KV<String, KV<URI, Double>>>() {
+ private static final long serialVersionUID1 = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ String word = c.element().getKey();
+ Double df = c.element().getValue().getOnly(dfTag);
+
+ for (KV<URI, Double> uriAndTf : c.element().getValue().getAll(tfTag)) {
+ URI uri = uriAndTf.getKey();
+ Double tf = uriAndTf.getValue();
+ Double tfIdf = tf * Math.log(1 / df);
+ c.output(KV.of(word, KV.of(uri, tfIdf)));
+ }
+ }
+ }));
+ }
+
+ // Instantiate Logger.
+ // It is suggested that the user specify the class name of the containing class
+ // (in this case ComputeTfIdf).
+ private static final Logger LOG = LoggerFactory.getLogger(ComputeTfIdf.class);
+ }
+
+ /**
+ * A {@link PTransform} to write, in CSV format, a mapping from term and URI
+ * to score.
+ */
+ public static class WriteTfIdf
+ extends PTransform<PCollection<KV<String, KV<URI, Double>>>, PDone> {
+ private static final long serialVersionUID = 0;
+
+ private String output;
+
+ public WriteTfIdf(String output) {
+ this.output = output;
+ }
+
+ @Override
+ public PDone apply(PCollection<KV<String, KV<URI, Double>>> wordToUriAndTfIdf) {
+ return wordToUriAndTfIdf
+ .apply(ParDo.named("Format").of(new DoFn<KV<String, KV<URI, Double>>, String>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ c.output(String.format("%s,\t%s,\t%f",
+ c.element().getKey(),
+ c.element().getValue().getKey(),
+ c.element().getValue().getValue()));
+ }
+ }))
+ .apply(TextIO.Write
+ .to(output)
+ .withSuffix(".csv"));
+ }
+ }
+
+ public static void main(String[] args) throws Exception {
+ Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+
+ options.setRunner(FlinkPipelineRunner.class);
+
+ Pipeline pipeline = Pipeline.create(options);
+ pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));
+
+ pipeline
+ .apply(new ReadDocuments(listInputDocuments(options)))
+ .apply(new ComputeTfIdf())
+ .apply(new WriteTfIdf(options.getOutput()));
+
+ pipeline.run();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
new file mode 100644
index 0000000..ba46301
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/WordCount.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.examples;
+
+import org.apache.beam.runners.flink.FlinkPipelineOptions;
+import org.apache.beam.runners.flink.FlinkPipelineRunner;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.*;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+public class WordCount {
+
+ public static class ExtractWordsFn extends DoFn<String, String> {
+ private final Aggregator<Long, Long> emptyLines =
+ createAggregator("emptyLines", new Sum.SumLongFn());
+
+ @Override
+ public void processElement(ProcessContext c) {
+ if (c.element().trim().isEmpty()) {
+ emptyLines.addValue(1L);
+ }
+
+ // Split the line into words.
+ String[] words = c.element().split("[^a-zA-Z']+");
+
+ // Output each word encountered into the output PCollection.
+ for (String word : words) {
+ if (!word.isEmpty()) {
+ c.output(word);
+ }
+ }
+ }
+ }
+
+ public static class CountWords extends PTransform<PCollection<String>,
+ PCollection<KV<String, Long>>> {
+ @Override
+ public PCollection<KV<String, Long>> apply(PCollection<String> lines) {
+
+ // Convert lines of text into individual words.
+ PCollection<String> words = lines.apply(
+ ParDo.of(new ExtractWordsFn()));
+
+ // Count the number of times each word occurs.
+ PCollection<KV<String, Long>> wordCounts =
+ words.apply(Count.<String>perElement());
+
+ return wordCounts;
+ }
+ }
+
+ /** A SimpleFunction that converts a Word and Count into a printable string. */
+ public static class FormatAsTextFn extends SimpleFunction<KV<String, Long>, String> {
+ @Override
+ public String apply(KV<String, Long> input) {
+ return input.getKey() + ": " + input.getValue();
+ }
+ }
+
+ /**
+ * Options supported by {@link WordCount}.
+ * <p>
+ * Inherits standard configuration options.
+ */
+ public interface Options extends PipelineOptions, FlinkPipelineOptions {
+ @Description("Path of the file to read from")
+ @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt")
+ String getInput();
+ void setInput(String value);
+
+ @Description("Path of the file to write to")
+ String getOutput();
+ void setOutput(String value);
+ }
+
+ public static void main(String[] args) {
+
+ Options options = PipelineOptionsFactory.fromArgs(args).withValidation()
+ .as(Options.class);
+ options.setRunner(FlinkPipelineRunner.class);
+
+ Pipeline p = Pipeline.create(options);
+
+ p.apply(TextIO.Read.named("ReadLines").from(options.getInput()))
+ .apply(new CountWords())
+ .apply(MapElements.via(new FormatAsTextFn()))
+ .apply(TextIO.Write.named("WriteCounts").to(options.getOutput()));
+
+ p.run();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
new file mode 100644
index 0000000..8168122
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/AutoComplete.java
@@ -0,0 +1,387 @@
+/*
+ * Copyright (C) 2015 Google Inc.
+ *
+ * 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://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.beam.runners.flink.examples.streaming;
+
+import org.apache.beam.runners.flink.FlinkPipelineRunner;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.coders.AvroCoder;
+import com.google.cloud.dataflow.sdk.coders.DefaultCoder;
+import com.google.cloud.dataflow.sdk.io.*;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.*;
+import com.google.cloud.dataflow.sdk.transforms.Partition.PartitionFn;
+import com.google.cloud.dataflow.sdk.transforms.windowing.*;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PBegin;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PCollectionList;
+import org.joda.time.Duration;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * To run the example, first open a socket on a terminal by executing the command:
+ * <li>
+ * <li>
+ * <code>nc -lk 9999</code>
+ * </li>
+ * </li>
+ * and then launch the example. Now whatever you type in the terminal is going to be
+ * the input to the program.
+ * */
+public class AutoComplete {
+
+ /**
+ * A PTransform that takes as input a list of tokens and returns
+ * the most common tokens per prefix.
+ */
+ public static class ComputeTopCompletions
+ extends PTransform<PCollection<String>, PCollection<KV<String, List<CompletionCandidate>>>> {
+ private static final long serialVersionUID = 0;
+
+ private final int candidatesPerPrefix;
+ private final boolean recursive;
+
+ protected ComputeTopCompletions(int candidatesPerPrefix, boolean recursive) {
+ this.candidatesPerPrefix = candidatesPerPrefix;
+ this.recursive = recursive;
+ }
+
+ public static ComputeTopCompletions top(int candidatesPerPrefix, boolean recursive) {
+ return new ComputeTopCompletions(candidatesPerPrefix, recursive);
+ }
+
+ @Override
+ public PCollection<KV<String, List<CompletionCandidate>>> apply(PCollection<String> input) {
+ PCollection<CompletionCandidate> candidates = input
+ // First count how often each token appears.
+ .apply(new Count.PerElement<String>())
+
+ // Map the KV outputs of Count into our own CompletionCandiate class.
+ .apply(ParDo.named("CreateCompletionCandidates").of(
+ new DoFn<KV<String, Long>, CompletionCandidate>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ CompletionCandidate cand = new CompletionCandidate(c.element().getKey(), c.element().getValue());
+ c.output(cand);
+ }
+ }));
+
+ // Compute the top via either a flat or recursive algorithm.
+ if (recursive) {
+ return candidates
+ .apply(new ComputeTopRecursive(candidatesPerPrefix, 1))
+ .apply(Flatten.<KV<String, List<CompletionCandidate>>>pCollections());
+ } else {
+ return candidates
+ .apply(new ComputeTopFlat(candidatesPerPrefix, 1));
+ }
+ }
+ }
+
+ /**
+ * Lower latency, but more expensive.
+ */
+ private static class ComputeTopFlat
+ extends PTransform<PCollection<CompletionCandidate>,
+ PCollection<KV<String, List<CompletionCandidate>>>> {
+ private static final long serialVersionUID = 0;
+
+ private final int candidatesPerPrefix;
+ private final int minPrefix;
+
+ public ComputeTopFlat(int candidatesPerPrefix, int minPrefix) {
+ this.candidatesPerPrefix = candidatesPerPrefix;
+ this.minPrefix = minPrefix;
+ }
+
+ @Override
+ public PCollection<KV<String, List<CompletionCandidate>>> apply(
+ PCollection<CompletionCandidate> input) {
+ return input
+ // For each completion candidate, map it to all prefixes.
+ .apply(ParDo.of(new AllPrefixes(minPrefix)))
+
+ // Find and return the top candiates for each prefix.
+ .apply(Top.<String, CompletionCandidate>largestPerKey(candidatesPerPrefix)
+ .withHotKeyFanout(new HotKeyFanout()));
+ }
+
+ private static class HotKeyFanout implements SerializableFunction<String, Integer> {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public Integer apply(String input) {
+ return (int) Math.pow(4, 5 - input.length());
+ }
+ }
+ }
+
+ /**
+ * Cheaper but higher latency.
+ *
+ * <p> Returns two PCollections, the first is top prefixes of size greater
+ * than minPrefix, and the second is top prefixes of size exactly
+ * minPrefix.
+ */
+ private static class ComputeTopRecursive
+ extends PTransform<PCollection<CompletionCandidate>,
+ PCollectionList<KV<String, List<CompletionCandidate>>>> {
+ private static final long serialVersionUID = 0;
+
+ private final int candidatesPerPrefix;
+ private final int minPrefix;
+
+ public ComputeTopRecursive(int candidatesPerPrefix, int minPrefix) {
+ this.candidatesPerPrefix = candidatesPerPrefix;
+ this.minPrefix = minPrefix;
+ }
+
+ private class KeySizePartitionFn implements PartitionFn<KV<String, List<CompletionCandidate>>> {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public int partitionFor(KV<String, List<CompletionCandidate>> elem, int numPartitions) {
+ return elem.getKey().length() > minPrefix ? 0 : 1;
+ }
+ }
+
+ private static class FlattenTops
+ extends DoFn<KV<String, List<CompletionCandidate>>, CompletionCandidate> {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ for (CompletionCandidate cc : c.element().getValue()) {
+ c.output(cc);
+ }
+ }
+ }
+
+ @Override
+ public PCollectionList<KV<String, List<CompletionCandidate>>> apply(
+ PCollection<CompletionCandidate> input) {
+ if (minPrefix > 10) {
+ // Base case, partitioning to return the output in the expected format.
+ return input
+ .apply(new ComputeTopFlat(candidatesPerPrefix, minPrefix))
+ .apply(Partition.of(2, new KeySizePartitionFn()));
+ } else {
+ // If a candidate is in the top N for prefix a...b, it must also be in the top
+ // N for a...bX for every X, which is typlically a much smaller set to consider.
+ // First, compute the top candidate for prefixes of size at least minPrefix + 1.
+ PCollectionList<KV<String, List<CompletionCandidate>>> larger = input
+ .apply(new ComputeTopRecursive(candidatesPerPrefix, minPrefix + 1));
+ // Consider the top candidates for each prefix of length minPrefix + 1...
+ PCollection<KV<String, List<CompletionCandidate>>> small =
+ PCollectionList
+ .of(larger.get(1).apply(ParDo.of(new FlattenTops())))
+ // ...together with those (previously excluded) candidates of length
+ // exactly minPrefix...
+ .and(input.apply(Filter.by(new SerializableFunction<CompletionCandidate, Boolean>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public Boolean apply(CompletionCandidate c) {
+ return c.getValue().length() == minPrefix;
+ }
+ })))
+ .apply("FlattenSmall", Flatten.<CompletionCandidate>pCollections())
+ // ...set the key to be the minPrefix-length prefix...
+ .apply(ParDo.of(new AllPrefixes(minPrefix, minPrefix)))
+ // ...and (re)apply the Top operator to all of them together.
+ .apply(Top.<String, CompletionCandidate>largestPerKey(candidatesPerPrefix));
+
+ PCollection<KV<String, List<CompletionCandidate>>> flattenLarger = larger
+ .apply("FlattenLarge", Flatten.<KV<String, List<CompletionCandidate>>>pCollections());
+
+ return PCollectionList.of(flattenLarger).and(small);
+ }
+ }
+ }
+
+ /**
+ * A DoFn that keys each candidate by all its prefixes.
+ */
+ private static class AllPrefixes
+ extends DoFn<CompletionCandidate, KV<String, CompletionCandidate>> {
+ private static final long serialVersionUID = 0;
+
+ private final int minPrefix;
+ private final int maxPrefix;
+ public AllPrefixes(int minPrefix) {
+ this(minPrefix, Integer.MAX_VALUE);
+ }
+ public AllPrefixes(int minPrefix, int maxPrefix) {
+ this.minPrefix = minPrefix;
+ this.maxPrefix = maxPrefix;
+ }
+ @Override
+ public void processElement(ProcessContext c) {
+ String word = c.element().value;
+ for (int i = minPrefix; i <= Math.min(word.length(), maxPrefix); i++) {
+ KV<String, CompletionCandidate> kv = KV.of(word.substring(0, i), c.element());
+ c.output(kv);
+ }
+ }
+ }
+
+ /**
+ * Class used to store tag-count pairs.
+ */
+ @DefaultCoder(AvroCoder.class)
+ static class CompletionCandidate implements Comparable<CompletionCandidate> {
+ private long count;
+ private String value;
+
+ public CompletionCandidate(String value, long count) {
+ this.value = value;
+ this.count = count;
+ }
+
+ public String getValue() {
+ return value;
+ }
+
+ // Empty constructor required for Avro decoding.
+ @SuppressWarnings("unused")
+ public CompletionCandidate() {}
+
+ @Override
+ public int compareTo(CompletionCandidate o) {
+ if (this.count < o.count) {
+ return -1;
+ } else if (this.count == o.count) {
+ return this.value.compareTo(o.value);
+ } else {
+ return 1;
+ }
+ }
+
+ @Override
+ public boolean equals(Object other) {
+ if (other instanceof CompletionCandidate) {
+ CompletionCandidate that = (CompletionCandidate) other;
+ return this.count == that.count && this.value.equals(that.value);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public int hashCode() {
+ return Long.valueOf(count).hashCode() ^ value.hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return "CompletionCandidate[" + value + ", " + count + "]";
+ }
+ }
+
+ static class ExtractWordsFn extends DoFn<String, String> {
+ private final Aggregator<Long, Long> emptyLines =
+ createAggregator("emptyLines", new Sum.SumLongFn());
+
+ @Override
+ public void processElement(ProcessContext c) {
+ if (c.element().trim().isEmpty()) {
+ emptyLines.addValue(1L);
+ }
+
+ // Split the line into words.
+ String[] words = c.element().split("[^a-zA-Z']+");
+
+ // Output each word encountered into the output PCollection.
+ for (String word : words) {
+ if (!word.isEmpty()) {
+ c.output(word);
+ }
+ }
+ }
+ }
+
+ /**
+ * Takes as input a the top candidates per prefix, and emits an entity
+ * suitable for writing to Datastore.
+ */
+ static class FormatForPerTaskLocalFile extends DoFn<KV<String, List<CompletionCandidate>>, String>
+ implements DoFn.RequiresWindowAccess{
+
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ StringBuilder str = new StringBuilder();
+ KV<String, List<CompletionCandidate>> elem = c.element();
+
+ str.append(elem.getKey() +" @ "+ c.window() +" -> ");
+ for(CompletionCandidate cand: elem.getValue()) {
+ str.append(cand.toString() + " ");
+ }
+ System.out.println(str.toString());
+ c.output(str.toString());
+ }
+ }
+
+ /**
+ * Options supported by this class.
+ *
+ * <p> Inherits standard Dataflow configuration options.
+ */
+ private interface Options extends WindowedWordCount.StreamingWordCountOptions {
+ @Description("Whether to use the recursive algorithm")
+ @Default.Boolean(true)
+ Boolean getRecursive();
+ void setRecursive(Boolean value);
+ }
+
+ public static void main(String[] args) throws IOException {
+ Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+ options.setStreaming(true);
+ options.setCheckpointingInterval(1000L);
+ options.setNumberOfExecutionRetries(5);
+ options.setExecutionRetryDelay(3000L);
+ options.setRunner(FlinkPipelineRunner.class);
+
+ PTransform<? super PBegin, PCollection<String>> readSource =
+ Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("WordStream");
+ WindowFn<Object, ?> windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize()));
+
+ // Create the pipeline.
+ Pipeline p = Pipeline.create(options);
+ PCollection<KV<String, List<CompletionCandidate>>> toWrite = p
+ .apply(readSource)
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Window.<String>into(windowFn)
+ .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes())
+ .apply(ComputeTopCompletions.top(10, options.getRecursive()));
+
+ toWrite
+ .apply(ParDo.named("FormatForPerTaskFile").of(new FormatForPerTaskLocalFile()))
+ .apply(TextIO.Write.to("./outputAutoComplete.txt"));
+
+ p.run();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java
new file mode 100644
index 0000000..3a8bdb0
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java
@@ -0,0 +1,158 @@
+/*
+ * Copyright (C) 2015 Google Inc.
+ *
+ * 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://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.beam.runners.flink.examples.streaming;
+
+import org.apache.beam.runners.flink.FlinkPipelineRunner;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.ParDo;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
+import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
+import com.google.cloud.dataflow.sdk.transforms.windowing.*;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PBegin;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import org.joda.time.Duration;
+
+/**
+ * To run the example, first open two sockets on two terminals by executing the commands:
+ * <li>
+ * <li>
+ * <code>nc -lk 9999</code>, and
+ * </li>
+ * <li>
+ * <code>nc -lk 9998</code>
+ * </li>
+ * </li>
+ * and then launch the example. Now whatever you type in the terminal is going to be
+ * the input to the program.
+ * */
+public class JoinExamples {
+
+ static PCollection<String> joinEvents(PCollection<String> streamA,
+ PCollection<String> streamB) throws Exception {
+
+ final TupleTag<String> firstInfoTag = new TupleTag<>();
+ final TupleTag<String> secondInfoTag = new TupleTag<>();
+
+ // transform both input collections to tuple collections, where the keys are country
+ // codes in both cases.
+ PCollection<KV<String, String>> firstInfo = streamA.apply(
+ ParDo.of(new ExtractEventDataFn()));
+ PCollection<KV<String, String>> secondInfo = streamB.apply(
+ ParDo.of(new ExtractEventDataFn()));
+
+ // country code 'key' -> CGBKR (<event info>, <country name>)
+ PCollection<KV<String, CoGbkResult>> kvpCollection = KeyedPCollectionTuple
+ .of(firstInfoTag, firstInfo)
+ .and(secondInfoTag, secondInfo)
+ .apply(CoGroupByKey.<String>create());
+
+ // Process the CoGbkResult elements generated by the CoGroupByKey transform.
+ // country code 'key' -> string of <event info>, <country name>
+ PCollection<KV<String, String>> finalResultCollection =
+ kvpCollection.apply(ParDo.named("Process").of(
+ new DoFn<KV<String, CoGbkResult>, KV<String, String>>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ KV<String, CoGbkResult> e = c.element();
+ String key = e.getKey();
+
+ String defaultA = "NO_VALUE";
+
+ // the following getOnly is a bit tricky because it expects to have
+ // EXACTLY ONE value in the corresponding stream and for the corresponding key.
+
+ String lineA = e.getValue().getOnly(firstInfoTag, defaultA);
+ for (String lineB : c.element().getValue().getAll(secondInfoTag)) {
+ // Generate a string that combines information from both collection values
+ c.output(KV.of(key, "Value A: " + lineA + " - Value B: " + lineB));
+ }
+ }
+ }));
+
+ return finalResultCollection
+ .apply(ParDo.named("Format").of(new DoFn<KV<String, String>, String>() {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ String result = c.element().getKey() + " -> " + c.element().getValue();
+ System.out.println(result);
+ c.output(result);
+ }
+ }));
+ }
+
+ static class ExtractEventDataFn extends DoFn<String, KV<String, String>> {
+ private static final long serialVersionUID = 0;
+
+ @Override
+ public void processElement(ProcessContext c) {
+ String line = c.element().toLowerCase();
+ String key = line.split("\\s")[0];
+ c.output(KV.of(key, line));
+ }
+ }
+
+ private interface Options extends WindowedWordCount.StreamingWordCountOptions {
+
+ }
+
+ public static void main(String[] args) throws Exception {
+ Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+ options.setStreaming(true);
+ options.setCheckpointingInterval(1000L);
+ options.setNumberOfExecutionRetries(5);
+ options.setExecutionRetryDelay(3000L);
+ options.setRunner(FlinkPipelineRunner.class);
+
+ PTransform<? super PBegin, PCollection<String>> readSourceA =
+ Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("FirstStream");
+ PTransform<? super PBegin, PCollection<String>> readSourceB =
+ Read.from(new UnboundedSocketSource<>("localhost", 9998, '\n', 3)).named("SecondStream");
+
+ WindowFn<Object, ?> windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize()));
+
+ Pipeline p = Pipeline.create(options);
+
+ // the following two 'applys' create multiple inputs to our pipeline, one for each
+ // of our two input sources.
+ PCollection<String> streamA = p.apply(readSourceA)
+ .apply(Window.<String>into(windowFn)
+ .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes());
+ PCollection<String> streamB = p.apply(readSourceB)
+ .apply(Window.<String>into(windowFn)
+ .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes());
+
+ PCollection<String> formattedResults = joinEvents(streamA, streamB);
+ formattedResults.apply(TextIO.Write.to("./outputJoin.txt"));
+ p.run();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
new file mode 100644
index 0000000..fa0c8e9
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java
@@ -0,0 +1,141 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.examples.streaming;
+
+import org.apache.beam.runners.flink.FlinkPipelineRunner;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSource;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.Read;
+import com.google.cloud.dataflow.sdk.io.TextIO;
+import com.google.cloud.dataflow.sdk.io.UnboundedSource;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.*;
+import com.google.cloud.dataflow.sdk.transforms.windowing.*;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer082;
+import org.apache.flink.streaming.util.serialization.SimpleStringSchema;
+import org.joda.time.Duration;
+
+import java.util.Properties;
+
+public class KafkaWindowedWordCountExample {
+
+ static final String KAFKA_TOPIC = "test"; // Default kafka topic to read from
+ static final String KAFKA_BROKER = "localhost:9092"; // Default kafka broker to contact
+ static final String GROUP_ID = "myGroup"; // Default groupId
+ static final String ZOOKEEPER = "localhost:2181"; // Default zookeeper to connect to for Kafka
+
+ public static class ExtractWordsFn extends DoFn<String, String> {
+ private final Aggregator<Long, Long> emptyLines =
+ createAggregator("emptyLines", new Sum.SumLongFn());
+
+ @Override
+ public void processElement(ProcessContext c) {
+ if (c.element().trim().isEmpty()) {
+ emptyLines.addValue(1L);
+ }
+
+ // Split the line into words.
+ String[] words = c.element().split("[^a-zA-Z']+");
+
+ // Output each word encountered into the output PCollection.
+ for (String word : words) {
+ if (!word.isEmpty()) {
+ c.output(word);
+ }
+ }
+ }
+ }
+
+ public static class FormatAsStringFn extends DoFn<KV<String, Long>, String> {
+ @Override
+ public void processElement(ProcessContext c) {
+ String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString();
+ System.out.println(row);
+ c.output(row);
+ }
+ }
+
+ public interface KafkaStreamingWordCountOptions extends WindowedWordCount.StreamingWordCountOptions {
+ @Description("The Kafka topic to read from")
+ @Default.String(KAFKA_TOPIC)
+ String getKafkaTopic();
+
+ void setKafkaTopic(String value);
+
+ @Description("The Kafka Broker to read from")
+ @Default.String(KAFKA_BROKER)
+ String getBroker();
+
+ void setBroker(String value);
+
+ @Description("The Zookeeper server to connect to")
+ @Default.String(ZOOKEEPER)
+ String getZookeeper();
+
+ void setZookeeper(String value);
+
+ @Description("The groupId")
+ @Default.String(GROUP_ID)
+ String getGroup();
+
+ void setGroup(String value);
+
+ }
+
+ public static void main(String[] args) {
+ PipelineOptionsFactory.register(KafkaStreamingWordCountOptions.class);
+ KafkaStreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).as(KafkaStreamingWordCountOptions.class);
+ options.setJobName("KafkaExample");
+ options.setStreaming(true);
+ options.setCheckpointingInterval(1000L);
+ options.setNumberOfExecutionRetries(5);
+ options.setExecutionRetryDelay(3000L);
+ options.setRunner(FlinkPipelineRunner.class);
+
+ System.out.println(options.getKafkaTopic() +" "+ options.getZookeeper() +" "+ options.getBroker() +" "+ options.getGroup() );
+ Pipeline pipeline = Pipeline.create(options);
+
+ Properties p = new Properties();
+ p.setProperty("zookeeper.connect", options.getZookeeper());
+ p.setProperty("bootstrap.servers", options.getBroker());
+ p.setProperty("group.id", options.getGroup());
+
+ // this is the Flink consumer that reads the input to
+ // the program from a kafka topic.
+ FlinkKafkaConsumer082 kafkaConsumer = new FlinkKafkaConsumer082<>(
+ options.getKafkaTopic(),
+ new SimpleStringSchema(), p);
+
+ PCollection<String> words = pipeline
+ .apply(Read.from(new UnboundedFlinkSource<String, UnboundedSource.CheckpointMark>(options, kafkaConsumer)).named("StreamingWordCount"))
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Window.<String>into(FixedWindows.of(Duration.standardSeconds(options.getWindowSize())))
+ .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes());
+
+ PCollection<KV<String, Long>> wordCounts =
+ words.apply(Count.<String>perElement());
+
+ wordCounts.apply(ParDo.of(new FormatAsStringFn()))
+ .apply(TextIO.Write.to("./outputKafka.txt"));
+
+ pipeline.run();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
new file mode 100644
index 0000000..6af044d
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java
@@ -0,0 +1,128 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.examples.streaming;
+
+import org.apache.beam.runners.flink.FlinkPipelineRunner;
+import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource;
+import com.google.cloud.dataflow.sdk.Pipeline;
+import com.google.cloud.dataflow.sdk.io.*;
+import com.google.cloud.dataflow.sdk.options.Default;
+import com.google.cloud.dataflow.sdk.options.Description;
+import com.google.cloud.dataflow.sdk.options.PipelineOptionsFactory;
+import com.google.cloud.dataflow.sdk.transforms.*;
+import com.google.cloud.dataflow.sdk.transforms.windowing.*;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+
+import org.joda.time.Duration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+/**
+ * To run the example, first open a socket on a terminal by executing the command:
+ * <li>
+ * <li>
+ * <code>nc -lk 9999</code>
+ * </li>
+ * </li>
+ * and then launch the example. Now whatever you type in the terminal is going to be
+ * the input to the program.
+ * */
+public class WindowedWordCount {
+
+ private static final Logger LOG = LoggerFactory.getLogger(WindowedWordCount.class);
+
+ static final long WINDOW_SIZE = 10; // Default window duration in seconds
+ static final long SLIDE_SIZE = 5; // Default window slide in seconds
+
+ static class FormatAsStringFn extends DoFn<KV<String, Long>, String> {
+ @Override
+ public void processElement(ProcessContext c) {
+ String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString();
+ c.output(row);
+ }
+ }
+
+ static class ExtractWordsFn extends DoFn<String, String> {
+ private final Aggregator<Long, Long> emptyLines =
+ createAggregator("emptyLines", new Sum.SumLongFn());
+
+ @Override
+ public void processElement(ProcessContext c) {
+ if (c.element().trim().isEmpty()) {
+ emptyLines.addValue(1L);
+ }
+
+ // Split the line into words.
+ String[] words = c.element().split("[^a-zA-Z']+");
+
+ // Output each word encountered into the output PCollection.
+ for (String word : words) {
+ if (!word.isEmpty()) {
+ c.output(word);
+ }
+ }
+ }
+ }
+
+ public interface StreamingWordCountOptions extends org.apache.beam.runners.flink.examples.WordCount.Options {
+ @Description("Sliding window duration, in seconds")
+ @Default.Long(WINDOW_SIZE)
+ Long getWindowSize();
+
+ void setWindowSize(Long value);
+
+ @Description("Window slide, in seconds")
+ @Default.Long(SLIDE_SIZE)
+ Long getSlide();
+
+ void setSlide(Long value);
+ }
+
+ public static void main(String[] args) throws IOException {
+ StreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(StreamingWordCountOptions.class);
+ options.setStreaming(true);
+ options.setWindowSize(10L);
+ options.setSlide(5L);
+ options.setCheckpointingInterval(1000L);
+ options.setNumberOfExecutionRetries(5);
+ options.setExecutionRetryDelay(3000L);
+ options.setRunner(FlinkPipelineRunner.class);
+
+ LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize() +
+ " sec. and a slide of " + options.getSlide());
+
+ Pipeline pipeline = Pipeline.create(options);
+
+ PCollection<String> words = pipeline
+ .apply(Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3)).named("StreamingWordCount"))
+ .apply(ParDo.of(new ExtractWordsFn()))
+ .apply(Window.<String>into(SlidingWindows.of(Duration.standardSeconds(options.getWindowSize()))
+ .every(Duration.standardSeconds(options.getSlide())))
+ .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes());
+
+ PCollection<KV<String, Long>> wordCounts =
+ words.apply(Count.<String>perElement());
+
+ wordCounts.apply(ParDo.of(new FormatAsStringFn()))
+ .apply(TextIO.Write.to("./outputWordCount.txt"));
+
+ pipeline.run();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java
new file mode 100644
index 0000000..cd25ba3
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/io/ConsoleIO.java
@@ -0,0 +1,80 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.io;
+
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.values.PCollection;
+import com.google.cloud.dataflow.sdk.values.PDone;
+
+/**
+ * Transform for printing the contents of a {@link com.google.cloud.dataflow.sdk.values.PCollection}.
+ * to standard output.
+ *
+ * This is Flink-specific and will only work when executed using the
+ * {@link org.apache.beam.runners.flink.FlinkPipelineRunner}.
+ */
+public class ConsoleIO {
+
+ /**
+ * A PTransform that writes a PCollection to a standard output.
+ */
+ public static class Write {
+
+ /**
+ * Returns a ConsoleIO.Write PTransform with a default step name.
+ */
+ public static Bound create() {
+ return new Bound();
+ }
+
+ /**
+ * Returns a ConsoleIO.Write PTransform with the given step name.
+ */
+ public static Bound named(String name) {
+ return new Bound().named(name);
+ }
+
+ /**
+ * A PTransform that writes a bounded PCollection to standard output.
+ */
+ public static class Bound extends PTransform<PCollection<?>, PDone> {
+ private static final long serialVersionUID = 0;
+
+ Bound() {
+ super("ConsoleIO.Write");
+ }
+
+ Bound(String name) {
+ super(name);
+ }
+
+ /**
+ * Returns a new ConsoleIO.Write PTransform that's like this one but with the given
+ * step
+ * name. Does not modify this object.
+ */
+ public Bound named(String name) {
+ return new Bound(name);
+ }
+
+ @Override
+ public PDone apply(PCollection<?> input) {
+ return PDone.in(input.getPipeline());
+ }
+ }
+ }
+}
+
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
new file mode 100644
index 0000000..5201423
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchPipelineTranslator.java
@@ -0,0 +1,149 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation;
+
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.runners.TransformTreeNode;
+import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
+import com.google.cloud.dataflow.sdk.transforms.PTransform;
+import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
+import com.google.cloud.dataflow.sdk.values.PValue;
+import org.apache.flink.api.java.ExecutionEnvironment;
+
+/**
+ * FlinkBatchPipelineTranslator knows how to translate Pipeline objects into Flink Jobs.
+ * This is based on {@link com.google.cloud.dataflow.sdk.runners.DataflowPipelineTranslator}
+ */
+public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
+
+ /**
+ * The necessary context in the case of a batch job.
+ */
+ private final FlinkBatchTranslationContext batchContext;
+
+ private int depth = 0;
+
+ /**
+ * Composite transform that we want to translate before proceeding with other transforms.
+ */
+ private PTransform<?, ?> currentCompositeTransform;
+
+ public FlinkBatchPipelineTranslator(ExecutionEnvironment env, PipelineOptions options) {
+ this.batchContext = new FlinkBatchTranslationContext(env, options);
+ }
+
+ // --------------------------------------------------------------------------------------------
+ // Pipeline Visitor Methods
+ // --------------------------------------------------------------------------------------------
+
+ @Override
+ public void enterCompositeTransform(TransformTreeNode node) {
+ System.out.println(genSpaces(this.depth) + "enterCompositeTransform- " + formatNodeName(node));
+
+ PTransform<?, ?> transform = node.getTransform();
+ if (transform != null && currentCompositeTransform == null) {
+
+ BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
+ if (translator != null) {
+ currentCompositeTransform = transform;
+ if (transform instanceof CoGroupByKey && node.getInput().expand().size() != 2) {
+ // we can only optimize CoGroupByKey for input size 2
+ currentCompositeTransform = null;
+ }
+ }
+ }
+ this.depth++;
+ }
+
+ @Override
+ public void leaveCompositeTransform(TransformTreeNode node) {
+ PTransform<?, ?> transform = node.getTransform();
+ if (transform != null && currentCompositeTransform == transform) {
+
+ BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
+ if (translator != null) {
+ System.out.println(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node));
+ applyBatchTransform(transform, node, translator);
+ currentCompositeTransform = null;
+ } else {
+ throw new IllegalStateException("Attempted to translate composite transform " +
+ "but no translator was found: " + currentCompositeTransform);
+ }
+ }
+ this.depth--;
+ System.out.println(genSpaces(this.depth) + "leaveCompositeTransform- " + formatNodeName(node));
+ }
+
+ @Override
+ public void visitTransform(TransformTreeNode node) {
+ System.out.println(genSpaces(this.depth) + "visitTransform- " + formatNodeName(node));
+ if (currentCompositeTransform != null) {
+ // ignore it
+ return;
+ }
+
+ // get the transformation corresponding to hte node we are
+ // currently visiting and translate it into its Flink alternative.
+
+ PTransform<?, ?> transform = node.getTransform();
+ BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
+ if (translator == null) {
+ System.out.println(node.getTransform().getClass());
+ throw new UnsupportedOperationException("The transform " + transform + " is currently not supported.");
+ }
+ applyBatchTransform(transform, node, translator);
+ }
+
+ @Override
+ public void visitValue(PValue value, TransformTreeNode producer) {
+ // do nothing here
+ }
+
+ private <T extends PTransform<?, ?>> void applyBatchTransform(PTransform<?, ?> transform, TransformTreeNode node, BatchTransformTranslator<?> translator) {
+
+ @SuppressWarnings("unchecked")
+ T typedTransform = (T) transform;
+
+ @SuppressWarnings("unchecked")
+ BatchTransformTranslator<T> typedTranslator = (BatchTransformTranslator<T>) translator;
+
+ // create the applied PTransform on the batchContext
+ batchContext.setCurrentTransform(AppliedPTransform.of(
+ node.getFullName(), node.getInput(), node.getOutput(), (PTransform) transform));
+ typedTranslator.translateNode(typedTransform, batchContext);
+ }
+
+ /**
+ * A translator of a {@link PTransform}.
+ */
+ public interface BatchTransformTranslator<Type extends PTransform> {
+ void translateNode(Type transform, FlinkBatchTranslationContext context);
+ }
+
+ private static String genSpaces(int n) {
+ String s = "";
+ for (int i = 0; i < n; i++) {
+ s += "| ";
+ }
+ return s;
+ }
+
+ private static String formatNodeName(TransformTreeNode node) {
+ return node.toString().split("@")[1] + node.getTransform();
+ }
+}
[24/50] [abbrv] incubator-beam git commit: [flink] adjust directories
according to package name
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java
new file mode 100644
index 0000000..bd8a968
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputDoFnFunction.java
@@ -0,0 +1,175 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.functions;
+
+import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.GlobalWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
+import com.google.cloud.dataflow.sdk.util.WindowedValue;
+import com.google.cloud.dataflow.sdk.util.WindowingInternals;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.collect.ImmutableList;
+import org.apache.flink.api.common.functions.RichMapPartitionFunction;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Encapsulates a {@link com.google.cloud.dataflow.sdk.transforms.DoFn} that uses side outputs
+ * inside a Flink {@link org.apache.flink.api.common.functions.RichMapPartitionFunction}.
+ *
+ * We get a mapping from {@link com.google.cloud.dataflow.sdk.values.TupleTag} to output index
+ * and must tag all outputs with the output number. Afterwards a filter will filter out
+ * those elements that are not to be in a specific output.
+ */
+public class FlinkMultiOutputDoFnFunction<IN, OUT> extends RichMapPartitionFunction<IN, RawUnionValue> {
+
+ private final DoFn<IN, OUT> doFn;
+ private transient PipelineOptions options;
+ private final Map<TupleTag<?>, Integer> outputMap;
+
+ public FlinkMultiOutputDoFnFunction(DoFn<IN, OUT> doFn, PipelineOptions options, Map<TupleTag<?>, Integer> outputMap) {
+ this.doFn = doFn;
+ this.options = options;
+ this.outputMap = outputMap;
+ }
+
+ private void writeObject(ObjectOutputStream out)
+ throws IOException, ClassNotFoundException {
+ out.defaultWriteObject();
+ ObjectMapper mapper = new ObjectMapper();
+ mapper.writeValue(out, options);
+ }
+
+ private void readObject(ObjectInputStream in)
+ throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ ObjectMapper mapper = new ObjectMapper();
+ options = mapper.readValue(in, PipelineOptions.class);
+
+ }
+
+ @Override
+ public void mapPartition(Iterable<IN> values, Collector<RawUnionValue> out) throws Exception {
+ ProcessContext context = new ProcessContext(doFn, out);
+ this.doFn.startBundle(context);
+ for (IN value : values) {
+ context.inValue = value;
+ doFn.processElement(context);
+ }
+ this.doFn.finishBundle(context);
+ }
+
+ private class ProcessContext extends DoFn<IN, OUT>.ProcessContext {
+
+ IN inValue;
+ Collector<RawUnionValue> outCollector;
+
+ public ProcessContext(DoFn<IN, OUT> fn, Collector<RawUnionValue> outCollector) {
+ fn.super();
+ this.outCollector = outCollector;
+ }
+
+ @Override
+ public IN element() {
+ return this.inValue;
+ }
+
+ @Override
+ public Instant timestamp() {
+ return Instant.now();
+ }
+
+ @Override
+ public BoundedWindow window() {
+ return GlobalWindow.INSTANCE;
+ }
+
+ @Override
+ public PaneInfo pane() {
+ return PaneInfo.NO_FIRING;
+ }
+
+ @Override
+ public WindowingInternals<IN, OUT> windowingInternals() {
+ return null;
+ }
+
+ @Override
+ public PipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view) {
+ List<T> sideInput = getRuntimeContext().getBroadcastVariable(view.getTagInternal()
+ .getId());
+ List<WindowedValue<?>> windowedValueList = new ArrayList<>(sideInput.size());
+ for (T input : sideInput) {
+ windowedValueList.add(WindowedValue.of(input, Instant.now(), ImmutableList.of(GlobalWindow.INSTANCE), pane()));
+ }
+ return view.fromIterableInternal(windowedValueList);
+ }
+
+ @Override
+ public void output(OUT value) {
+ // assume that index 0 is the default output
+ outCollector.collect(new RawUnionValue(0, value));
+ }
+
+ @Override
+ public void outputWithTimestamp(OUT output, Instant timestamp) {
+ // not FLink's way, just output normally
+ output(output);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public <T> void sideOutput(TupleTag<T> tag, T value) {
+ Integer index = outputMap.get(tag);
+ if (index != null) {
+ outCollector.collect(new RawUnionValue(index, value));
+ }
+ }
+
+ @Override
+ public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+ sideOutput(tag, output);
+ }
+
+ @Override
+ protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
+ SerializableFnAggregatorWrapper<AggInputT, AggOutputT> wrapper = new SerializableFnAggregatorWrapper<>(combiner);
+ getRuntimeContext().addAccumulator(name, wrapper);
+ return null;
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java
new file mode 100644
index 0000000..3e1cb65
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.functions;
+
+import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
+import org.apache.flink.api.common.functions.FlatMapFunction;
+import org.apache.flink.util.Collector;
+
+/**
+ * A FlatMap function that filters out those elements that don't belong in this output. We need
+ * this to implement MultiOutput ParDo functions.
+ */
+public class FlinkMultiOutputPruningFunction<T> implements FlatMapFunction<RawUnionValue, T> {
+
+ private final int outputTag;
+
+ public FlinkMultiOutputPruningFunction(int outputTag) {
+ this.outputTag = outputTag;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void flatMap(RawUnionValue rawUnionValue, Collector<T> collector) throws Exception {
+ if (rawUnionValue.getUnionTag() == outputTag) {
+ collector.collect((T) rawUnionValue.getValue());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java
new file mode 100644
index 0000000..1ff06ba
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.functions;
+
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.flink.api.common.functions.GroupCombineFunction;
+import org.apache.flink.util.Collector;
+
+import java.util.Iterator;
+
+/**
+ * Flink {@link org.apache.flink.api.common.functions.GroupCombineFunction} for executing a
+ * {@link com.google.cloud.dataflow.sdk.transforms.Combine.PerKey} operation. This reads the input
+ * {@link com.google.cloud.dataflow.sdk.values.KV} elements VI, extracts the key and emits accumulated
+ * values which have the intermediate format VA.
+ */
+public class FlinkPartialReduceFunction<K, VI, VA> implements GroupCombineFunction<KV<K, VI>, KV<K, VA>> {
+
+ private final Combine.KeyedCombineFn<K, VI, VA, ?> keyedCombineFn;
+
+ public FlinkPartialReduceFunction(Combine.KeyedCombineFn<K, VI, VA, ?>
+ keyedCombineFn) {
+ this.keyedCombineFn = keyedCombineFn;
+ }
+
+ @Override
+ public void combine(Iterable<KV<K, VI>> elements, Collector<KV<K, VA>> out) throws Exception {
+
+ final Iterator<KV<K, VI>> iterator = elements.iterator();
+ // create accumulator using the first elements key
+ KV<K, VI> first = iterator.next();
+ K key = first.getKey();
+ VI value = first.getValue();
+ VA accumulator = keyedCombineFn.createAccumulator(key);
+ accumulator = keyedCombineFn.addInput(key, accumulator, value);
+
+ while(iterator.hasNext()) {
+ value = iterator.next().getValue();
+ accumulator = keyedCombineFn.addInput(key, accumulator, value);
+ }
+
+ out.collect(KV.of(key, accumulator));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java
new file mode 100644
index 0000000..580ac01
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.functions;
+
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.values.KV;
+import com.google.common.collect.ImmutableList;
+import org.apache.flink.api.common.functions.GroupReduceFunction;
+import org.apache.flink.util.Collector;
+
+import java.util.Iterator;
+
+/**
+ * Flink {@link org.apache.flink.api.common.functions.GroupReduceFunction} for executing a
+ * {@link com.google.cloud.dataflow.sdk.transforms.Combine.PerKey} operation. This reads the input
+ * {@link com.google.cloud.dataflow.sdk.values.KV} elements, extracts the key and merges the
+ * accumulators resulting from the PartialReduce which produced the input VA.
+ */
+public class FlinkReduceFunction<K, VA, VO> implements GroupReduceFunction<KV<K, VA>, KV<K, VO>> {
+
+ private final Combine.KeyedCombineFn<K, ?, VA, VO> keyedCombineFn;
+
+ public FlinkReduceFunction(Combine.KeyedCombineFn<K, ?, VA, VO> keyedCombineFn) {
+ this.keyedCombineFn = keyedCombineFn;
+ }
+
+ @Override
+ public void reduce(Iterable<KV<K, VA>> values, Collector<KV<K, VO>> out) throws Exception {
+ Iterator<KV<K, VA>> it = values.iterator();
+
+ KV<K, VA> current = it.next();
+ K k = current.getKey();
+ VA accumulator = current.getValue();
+
+ while (it.hasNext()) {
+ current = it.next();
+ keyedCombineFn.mergeAccumulators(k, ImmutableList.of(accumulator, current.getValue()) );
+ }
+
+ out.collect(KV.of(k, keyedCombineFn.extractOutput(k, accumulator)));
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java
new file mode 100644
index 0000000..05f4415
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/UnionCoder.java
@@ -0,0 +1,150 @@
+/*
+ * Copyright (C) 2015 Google Inc.
+ *
+ * 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://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.beam.runners.flink.translation.functions;
+
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.StandardCoder;
+import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
+import com.google.cloud.dataflow.sdk.util.PropertyNames;
+import com.google.cloud.dataflow.sdk.util.VarInt;
+import com.google.cloud.dataflow.sdk.util.common.ElementByteSizeObserver;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+
+/**
+ * A UnionCoder encodes RawUnionValues.
+ *
+ * This file copied from {@link com.google.cloud.dataflow.sdk.transforms.join.UnionCoder}
+ */
+@SuppressWarnings("serial")
+public class UnionCoder extends StandardCoder<RawUnionValue> {
+ // TODO: Think about how to integrate this with a schema object (i.e.
+ // a tuple of tuple tags).
+ /**
+ * Builds a union coder with the given list of element coders. This list
+ * corresponds to a mapping of union tag to Coder. Union tags start at 0.
+ */
+ public static UnionCoder of(List<Coder<?>> elementCoders) {
+ return new UnionCoder(elementCoders);
+ }
+
+ @JsonCreator
+ public static UnionCoder jsonOf(
+ @JsonProperty(PropertyNames.COMPONENT_ENCODINGS)
+ List<Coder<?>> elements) {
+ return UnionCoder.of(elements);
+ }
+
+ private int getIndexForEncoding(RawUnionValue union) {
+ if (union == null) {
+ throw new IllegalArgumentException("cannot encode a null tagged union");
+ }
+ int index = union.getUnionTag();
+ if (index < 0 || index >= elementCoders.size()) {
+ throw new IllegalArgumentException(
+ "union value index " + index + " not in range [0.." +
+ (elementCoders.size() - 1) + "]");
+ }
+ return index;
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void encode(
+ RawUnionValue union,
+ OutputStream outStream,
+ Context context)
+ throws IOException {
+ int index = getIndexForEncoding(union);
+ // Write out the union tag.
+ VarInt.encode(index, outStream);
+
+ // Write out the actual value.
+ Coder<Object> coder = (Coder<Object>) elementCoders.get(index);
+ coder.encode(
+ union.getValue(),
+ outStream,
+ context);
+ }
+
+ @Override
+ public RawUnionValue decode(InputStream inStream, Context context)
+ throws IOException {
+ int index = VarInt.decodeInt(inStream);
+ Object value = elementCoders.get(index).decode(inStream, context);
+ return new RawUnionValue(index, value);
+ }
+
+ @Override
+ public List<? extends Coder<?>> getCoderArguments() {
+ return null;
+ }
+
+ @Override
+ public List<? extends Coder<?>> getComponents() {
+ return elementCoders;
+ }
+
+ /**
+ * Since this coder uses elementCoders.get(index) and coders that are known to run in constant
+ * time, we defer the return value to that coder.
+ */
+ @Override
+ public boolean isRegisterByteSizeObserverCheap(RawUnionValue union, Context context) {
+ int index = getIndexForEncoding(union);
+ @SuppressWarnings("unchecked")
+ Coder<Object> coder = (Coder<Object>) elementCoders.get(index);
+ return coder.isRegisterByteSizeObserverCheap(union.getValue(), context);
+ }
+
+ /**
+ * Notifies ElementByteSizeObserver about the byte size of the encoded value using this coder.
+ */
+ @Override
+ public void registerByteSizeObserver(
+ RawUnionValue union, ElementByteSizeObserver observer, Context context)
+ throws Exception {
+ int index = getIndexForEncoding(union);
+ // Write out the union tag.
+ observer.update(VarInt.getLength(index));
+ // Write out the actual value.
+ @SuppressWarnings("unchecked")
+ Coder<Object> coder = (Coder<Object>) elementCoders.get(index);
+ coder.registerByteSizeObserver(union.getValue(), observer, context);
+ }
+
+ /////////////////////////////////////////////////////////////////////////////
+
+ private final List<Coder<?>> elementCoders;
+
+ private UnionCoder(List<Coder<?>> elementCoders) {
+ this.elementCoders = elementCoders;
+ }
+
+ @Override
+ public void verifyDeterministic() throws NonDeterministicException {
+ verifyDeterministic(
+ "UnionCoder is only deterministic if all element coders are",
+ elementCoders);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java
new file mode 100644
index 0000000..ecfb95d
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderComparator.java
@@ -0,0 +1,216 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.types;
+
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+
+/**
+ * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for
+ * {@link com.google.cloud.dataflow.sdk.coders.Coder}.
+ */
+public class CoderComparator<T> extends TypeComparator<T> {
+
+ private Coder<T> coder;
+
+ // We use these for internal encoding/decoding for creating copies and comparing
+ // serialized forms using a Coder
+ private transient InspectableByteArrayOutputStream buffer1;
+ private transient InspectableByteArrayOutputStream buffer2;
+
+ // For storing the Reference in encoded form
+ private transient InspectableByteArrayOutputStream referenceBuffer;
+
+ public CoderComparator(Coder<T> coder) {
+ this.coder = coder;
+ buffer1 = new InspectableByteArrayOutputStream();
+ buffer2 = new InspectableByteArrayOutputStream();
+ referenceBuffer = new InspectableByteArrayOutputStream();
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ buffer1 = new InspectableByteArrayOutputStream();
+ buffer2 = new InspectableByteArrayOutputStream();
+ referenceBuffer = new InspectableByteArrayOutputStream();
+ }
+
+ @Override
+ public int hash(T record) {
+ return record.hashCode();
+ }
+
+ @Override
+ public void setReference(T toCompare) {
+ referenceBuffer.reset();
+ try {
+ coder.encode(toCompare, referenceBuffer, Coder.Context.OUTER);
+ } catch (IOException e) {
+ throw new RuntimeException("Could not set reference " + toCompare + ": " + e);
+ }
+ }
+
+ @Override
+ public boolean equalToReference(T candidate) {
+ try {
+ buffer2.reset();
+ coder.encode(candidate, buffer2, Coder.Context.OUTER);
+ byte[] arr = referenceBuffer.getBuffer();
+ byte[] arrOther = buffer2.getBuffer();
+ if (referenceBuffer.size() != buffer2.size()) {
+ return false;
+ }
+ int len = buffer2.size();
+ for(int i = 0; i < len; i++ ) {
+ if (arr[i] != arrOther[i]) {
+ return false;
+ }
+ }
+ return true;
+ } catch (IOException e) {
+ throw new RuntimeException("Could not compare reference.", e);
+ }
+ }
+
+ @Override
+ public int compareToReference(TypeComparator<T> other) {
+ InspectableByteArrayOutputStream otherReferenceBuffer = ((CoderComparator<T>) other).referenceBuffer;
+
+ byte[] arr = referenceBuffer.getBuffer();
+ byte[] arrOther = otherReferenceBuffer.getBuffer();
+ if (referenceBuffer.size() != otherReferenceBuffer.size()) {
+ return referenceBuffer.size() - otherReferenceBuffer.size();
+ }
+ int len = referenceBuffer.size();
+ for (int i = 0; i < len; i++) {
+ if (arr[i] != arrOther[i]) {
+ return arr[i] - arrOther[i];
+ }
+ }
+ return 0;
+ }
+
+ @Override
+ public int compare(T first, T second) {
+ try {
+ buffer1.reset();
+ buffer2.reset();
+ coder.encode(first, buffer1, Coder.Context.OUTER);
+ coder.encode(second, buffer2, Coder.Context.OUTER);
+ byte[] arr = buffer1.getBuffer();
+ byte[] arrOther = buffer2.getBuffer();
+ if (buffer1.size() != buffer2.size()) {
+ return buffer1.size() - buffer2.size();
+ }
+ int len = buffer1.size();
+ for(int i = 0; i < len; i++ ) {
+ if (arr[i] != arrOther[i]) {
+ return arr[i] - arrOther[i];
+ }
+ }
+ return 0;
+ } catch (IOException e) {
+ throw new RuntimeException("Could not compare: ", e);
+ }
+ }
+
+ @Override
+ public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
+ CoderTypeSerializer<T> serializer = new CoderTypeSerializer<>(coder);
+ T first = serializer.deserialize(firstSource);
+ T second = serializer.deserialize(secondSource);
+ return compare(first, second);
+ }
+
+ @Override
+ public boolean supportsNormalizedKey() {
+ return true;
+ }
+
+ @Override
+ public boolean supportsSerializationWithKeyNormalization() {
+ return false;
+ }
+
+ @Override
+ public int getNormalizeKeyLen() {
+ return Integer.MAX_VALUE;
+ }
+
+ @Override
+ public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
+ return true;
+ }
+
+ @Override
+ public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) {
+ buffer1.reset();
+ try {
+ coder.encode(record, buffer1, Coder.Context.OUTER);
+ } catch (IOException e) {
+ throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e);
+ }
+ final byte[] data = buffer1.getBuffer();
+ final int limit = offset + numBytes;
+
+ target.put(offset, data, 0, Math.min(numBytes, buffer1.size()));
+
+ offset += buffer1.size();
+
+ while (offset < limit) {
+ target.put(offset++, (byte) 0);
+ }
+ }
+
+ @Override
+ public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean invertNormalizedKey() {
+ return false;
+ }
+
+ @Override
+ public TypeComparator<T> duplicate() {
+ return new CoderComparator<>(coder);
+ }
+
+ @Override
+ public int extractKeys(Object record, Object[] target, int index) {
+ target[index] = record;
+ return 1;
+ }
+
+ @Override
+ public TypeComparator[] getFlatComparators() {
+ return new TypeComparator[] { this.duplicate() };
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
new file mode 100644
index 0000000..8880b48
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java
@@ -0,0 +1,116 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.types;
+
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.VoidCoder;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.AtomicType;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import com.google.common.base.Preconditions;
+
+/**
+ * Flink {@link org.apache.flink.api.common.typeinfo.TypeInformation} for
+ * Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s.
+ */
+public class CoderTypeInformation<T> extends TypeInformation<T> implements AtomicType<T> {
+
+ private final Coder<T> coder;
+
+ public CoderTypeInformation(Coder<T> coder) {
+ Preconditions.checkNotNull(coder);
+ this.coder = coder;
+ }
+
+ @Override
+ public boolean isBasicType() {
+ return false;
+ }
+
+ @Override
+ public boolean isTupleType() {
+ return false;
+ }
+
+ @Override
+ public int getArity() {
+ return 1;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public Class<T> getTypeClass() {
+ // We don't have the Class, so we have to pass null here. What a shame...
+ return (Class<T>) Object.class;
+ }
+
+ @Override
+ public boolean isKeyType() {
+ return true;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public TypeSerializer<T> createSerializer(ExecutionConfig config) {
+ if (coder instanceof VoidCoder) {
+ return (TypeSerializer<T>) new VoidCoderTypeSerializer();
+ }
+ return new CoderTypeSerializer<>(coder);
+ }
+
+ @Override
+ public int getTotalFields() {
+ return 2;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ CoderTypeInformation that = (CoderTypeInformation) o;
+
+ return coder.equals(that.coder);
+
+ }
+
+ @Override
+ public int hashCode() {
+ return coder.hashCode();
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof CoderTypeInformation;
+ }
+
+ @Override
+ public String toString() {
+ return "CoderTypeInformation{" +
+ "coder=" + coder +
+ '}';
+ }
+
+ @Override
+ public TypeComparator<T> createComparator(boolean sortOrderAscending, ExecutionConfig
+ executionConfig) {
+ return new CoderComparator<>(coder);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
new file mode 100644
index 0000000..481ee31
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
@@ -0,0 +1,152 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.types;
+
+import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper;
+import org.apache.beam.runners.flink.translation.wrappers.DataOutputViewWrapper;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.CoderException;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.ByteArrayInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+
+/**
+ * Flink {@link org.apache.flink.api.common.typeutils.TypeSerializer} for
+ * Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s
+ */
+public class CoderTypeSerializer<T> extends TypeSerializer<T> {
+
+ private Coder<T> coder;
+ private transient DataInputViewWrapper inputWrapper;
+ private transient DataOutputViewWrapper outputWrapper;
+
+ // We use this for internal encoding/decoding for creating copies using the Coder.
+ private transient InspectableByteArrayOutputStream buffer;
+
+ public CoderTypeSerializer(Coder<T> coder) {
+ this.coder = coder;
+ this.inputWrapper = new DataInputViewWrapper(null);
+ this.outputWrapper = new DataOutputViewWrapper(null);
+
+ buffer = new InspectableByteArrayOutputStream();
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ this.inputWrapper = new DataInputViewWrapper(null);
+ this.outputWrapper = new DataOutputViewWrapper(null);
+
+ buffer = new InspectableByteArrayOutputStream();
+ }
+
+ @Override
+ public boolean isImmutableType() {
+ return false;
+ }
+
+ @Override
+ public CoderTypeSerializer<T> duplicate() {
+ return new CoderTypeSerializer<>(coder);
+ }
+
+ @Override
+ public T createInstance() {
+ return null;
+ }
+
+ @Override
+ public T copy(T t) {
+ buffer.reset();
+ try {
+ coder.encode(t, buffer, Coder.Context.OUTER);
+ } catch (IOException e) {
+ throw new RuntimeException("Could not copy.", e);
+ }
+ try {
+ return coder.decode(new ByteArrayInputStream(buffer.getBuffer(), 0, buffer
+ .size()), Coder.Context.OUTER);
+ } catch (IOException e) {
+ throw new RuntimeException("Could not copy.", e);
+ }
+ }
+
+ @Override
+ public T copy(T t, T reuse) {
+ return copy(t);
+ }
+
+ @Override
+ public int getLength() {
+ return 0;
+ }
+
+ @Override
+ public void serialize(T t, DataOutputView dataOutputView) throws IOException {
+ outputWrapper.setOutputView(dataOutputView);
+ coder.encode(t, outputWrapper, Coder.Context.NESTED);
+ }
+
+ @Override
+ public T deserialize(DataInputView dataInputView) throws IOException {
+ try {
+ inputWrapper.setInputView(dataInputView);
+ return coder.decode(inputWrapper, Coder.Context.NESTED);
+ } catch (CoderException e) {
+ Throwable cause = e.getCause();
+ if (cause instanceof EOFException) {
+ throw (EOFException) cause;
+ } else {
+ throw e;
+ }
+ }
+ }
+
+ @Override
+ public T deserialize(T t, DataInputView dataInputView) throws IOException {
+ return deserialize(dataInputView);
+ }
+
+ @Override
+ public void copy(DataInputView dataInputView, DataOutputView dataOutputView) throws IOException {
+ serialize(deserialize(dataInputView), dataOutputView);
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ CoderTypeSerializer that = (CoderTypeSerializer) o;
+ return coder.equals(that.coder);
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof CoderTypeSerializer;
+ }
+
+ @Override
+ public int hashCode() {
+ return coder.hashCode();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java
new file mode 100644
index 0000000..619fa55
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/InspectableByteArrayOutputStream.java
@@ -0,0 +1,34 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.types;
+
+import java.io.ByteArrayOutputStream;
+
+/**
+ * Version of {@link java.io.ByteArrayOutputStream} that allows to retrieve the internal
+ * byte[] buffer without incurring an array copy.
+ */
+public class InspectableByteArrayOutputStream extends ByteArrayOutputStream {
+
+ /**
+ * Get the underlying byte array.
+ */
+ public byte[] getBuffer() {
+ return buf;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java
new file mode 100644
index 0000000..4599c6a
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderComperator.java
@@ -0,0 +1,264 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.types;
+
+import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+
+/**
+ * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for
+ * {@link com.google.cloud.dataflow.sdk.coders.KvCoder}. We have a special comparator
+ * for {@link KV} that always compares on the key only.
+ */
+public class KvCoderComperator <K, V> extends TypeComparator<KV<K, V>> {
+
+ private KvCoder<K, V> coder;
+ private Coder<K> keyCoder;
+
+ // We use these for internal encoding/decoding for creating copies and comparing
+ // serialized forms using a Coder
+ private transient InspectableByteArrayOutputStream buffer1;
+ private transient InspectableByteArrayOutputStream buffer2;
+
+ // For storing the Reference in encoded form
+ private transient InspectableByteArrayOutputStream referenceBuffer;
+
+
+ // For deserializing the key
+ private transient DataInputViewWrapper inputWrapper;
+
+ public KvCoderComperator(KvCoder<K, V> coder) {
+ this.coder = coder;
+ this.keyCoder = coder.getKeyCoder();
+
+ buffer1 = new InspectableByteArrayOutputStream();
+ buffer2 = new InspectableByteArrayOutputStream();
+ referenceBuffer = new InspectableByteArrayOutputStream();
+
+ inputWrapper = new DataInputViewWrapper(null);
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+
+ buffer1 = new InspectableByteArrayOutputStream();
+ buffer2 = new InspectableByteArrayOutputStream();
+ referenceBuffer = new InspectableByteArrayOutputStream();
+
+ inputWrapper = new DataInputViewWrapper(null);
+ }
+
+ @Override
+ public int hash(KV<K, V> record) {
+ K key = record.getKey();
+ if (key != null) {
+ return key.hashCode();
+ } else {
+ return 0;
+ }
+ }
+
+ @Override
+ public void setReference(KV<K, V> toCompare) {
+ referenceBuffer.reset();
+ try {
+ keyCoder.encode(toCompare.getKey(), referenceBuffer, Coder.Context.OUTER);
+ } catch (IOException e) {
+ throw new RuntimeException("Could not set reference " + toCompare + ": " + e);
+ }
+ }
+
+ @Override
+ public boolean equalToReference(KV<K, V> candidate) {
+ try {
+ buffer2.reset();
+ keyCoder.encode(candidate.getKey(), buffer2, Coder.Context.OUTER);
+ byte[] arr = referenceBuffer.getBuffer();
+ byte[] arrOther = buffer2.getBuffer();
+ if (referenceBuffer.size() != buffer2.size()) {
+ return false;
+ }
+ int len = buffer2.size();
+ for(int i = 0; i < len; i++ ) {
+ if (arr[i] != arrOther[i]) {
+ return false;
+ }
+ }
+ return true;
+ } catch (IOException e) {
+ throw new RuntimeException("Could not compare reference.", e);
+ }
+ }
+
+ @Override
+ public int compareToReference(TypeComparator<KV<K, V>> other) {
+ InspectableByteArrayOutputStream otherReferenceBuffer = ((KvCoderComperator<K, V>) other).referenceBuffer;
+
+ byte[] arr = referenceBuffer.getBuffer();
+ byte[] arrOther = otherReferenceBuffer.getBuffer();
+ if (referenceBuffer.size() != otherReferenceBuffer.size()) {
+ return referenceBuffer.size() - otherReferenceBuffer.size();
+ }
+ int len = referenceBuffer.size();
+ for (int i = 0; i < len; i++) {
+ if (arr[i] != arrOther[i]) {
+ return arr[i] - arrOther[i];
+ }
+ }
+ return 0;
+ }
+
+
+ @Override
+ public int compare(KV<K, V> first, KV<K, V> second) {
+ try {
+ buffer1.reset();
+ buffer2.reset();
+ keyCoder.encode(first.getKey(), buffer1, Coder.Context.OUTER);
+ keyCoder.encode(second.getKey(), buffer2, Coder.Context.OUTER);
+ byte[] arr = buffer1.getBuffer();
+ byte[] arrOther = buffer2.getBuffer();
+ if (buffer1.size() != buffer2.size()) {
+ return buffer1.size() - buffer2.size();
+ }
+ int len = buffer1.size();
+ for(int i = 0; i < len; i++ ) {
+ if (arr[i] != arrOther[i]) {
+ return arr[i] - arrOther[i];
+ }
+ }
+ return 0;
+ } catch (IOException e) {
+ throw new RuntimeException("Could not compare reference.", e);
+ }
+ }
+
+ @Override
+ public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
+
+ inputWrapper.setInputView(firstSource);
+ K firstKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED);
+ inputWrapper.setInputView(secondSource);
+ K secondKey = keyCoder.decode(inputWrapper, Coder.Context.NESTED);
+
+ try {
+ buffer1.reset();
+ buffer2.reset();
+ keyCoder.encode(firstKey, buffer1, Coder.Context.OUTER);
+ keyCoder.encode(secondKey, buffer2, Coder.Context.OUTER);
+ byte[] arr = buffer1.getBuffer();
+ byte[] arrOther = buffer2.getBuffer();
+ if (buffer1.size() != buffer2.size()) {
+ return buffer1.size() - buffer2.size();
+ }
+ int len = buffer1.size();
+ for(int i = 0; i < len; i++ ) {
+ if (arr[i] != arrOther[i]) {
+ return arr[i] - arrOther[i];
+ }
+ }
+ return 0;
+ } catch (IOException e) {
+ throw new RuntimeException("Could not compare reference.", e);
+ }
+ }
+
+ @Override
+ public boolean supportsNormalizedKey() {
+ return true;
+ }
+
+ @Override
+ public boolean supportsSerializationWithKeyNormalization() {
+ return false;
+ }
+
+ @Override
+ public int getNormalizeKeyLen() {
+ return Integer.MAX_VALUE;
+ }
+
+ @Override
+ public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
+ return true;
+ }
+
+ @Override
+ public void putNormalizedKey(KV<K, V> record, MemorySegment target, int offset, int numBytes) {
+ buffer1.reset();
+ try {
+ keyCoder.encode(record.getKey(), buffer1, Coder.Context.NESTED);
+ } catch (IOException e) {
+ throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e);
+ }
+ final byte[] data = buffer1.getBuffer();
+ final int limit = offset + numBytes;
+
+ int numBytesPut = Math.min(numBytes, buffer1.size());
+
+ target.put(offset, data, 0, numBytesPut);
+
+ offset += numBytesPut;
+
+ while (offset < limit) {
+ target.put(offset++, (byte) 0);
+ }
+ }
+
+ @Override
+ public void writeWithKeyNormalization(KV<K, V> record, DataOutputView target) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public KV<K, V> readWithKeyDenormalization(KV<K, V> reuse, DataInputView source) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean invertNormalizedKey() {
+ return false;
+ }
+
+ @Override
+ public TypeComparator<KV<K, V>> duplicate() {
+ return new KvCoderComperator<>(coder);
+ }
+
+ @Override
+ public int extractKeys(Object record, Object[] target, int index) {
+ KV<K, V> kv = (KV<K, V>) record;
+ K k = kv.getKey();
+ target[index] = k;
+ return 1;
+ }
+
+ @Override
+ public TypeComparator[] getFlatComparators() {
+ return new TypeComparator[] {new CoderComparator<>(keyCoder)};
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java
new file mode 100644
index 0000000..7a0d999
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvCoderTypeInformation.java
@@ -0,0 +1,186 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.types;
+
+import com.google.cloud.dataflow.sdk.coders.KvCoder;
+import com.google.cloud.dataflow.sdk.values.KV;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.CompositeType;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import com.google.common.base.Preconditions;
+
+import java.util.List;
+
+/**
+ * Flink {@link org.apache.flink.api.common.typeinfo.TypeInformation} for
+ * Dataflow {@link com.google.cloud.dataflow.sdk.coders.KvCoder}.
+ */
+public class KvCoderTypeInformation<K, V> extends CompositeType<KV<K, V>> {
+
+ private KvCoder<K, V> coder;
+
+ // We don't have the Class, so we have to pass null here. What a shame...
+ private static Object DUMMY = new Object();
+
+ @SuppressWarnings("unchecked")
+ public KvCoderTypeInformation(KvCoder<K, V> coder) {
+ super(((Class<KV<K,V>>) DUMMY.getClass()));
+ this.coder = coder;
+ Preconditions.checkNotNull(coder);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public TypeComparator<KV<K, V>> createComparator(int[] logicalKeyFields, boolean[] orders, int logicalFieldOffset, ExecutionConfig config) {
+ return new KvCoderComperator((KvCoder) coder);
+ }
+
+ @Override
+ public boolean isBasicType() {
+ return false;
+ }
+
+ @Override
+ public boolean isTupleType() {
+ return false;
+ }
+
+ @Override
+ public int getArity() {
+ return 2;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public Class<KV<K, V>> getTypeClass() {
+ return privateGetTypeClass();
+ }
+
+ @SuppressWarnings("unchecked")
+ private static <X> Class<X> privateGetTypeClass() {
+ return (Class<X>) Object.class;
+ }
+
+ @Override
+ public boolean isKeyType() {
+ return true;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public TypeSerializer<KV<K, V>> createSerializer(ExecutionConfig config) {
+ return new CoderTypeSerializer<>(coder);
+ }
+
+ @Override
+ public int getTotalFields() {
+ return 2;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ KvCoderTypeInformation that = (KvCoderTypeInformation) o;
+
+ return coder.equals(that.coder);
+
+ }
+
+ @Override
+ public int hashCode() {
+ return coder.hashCode();
+ }
+
+ @Override
+ public String toString() {
+ return "CoderTypeInformation{" +
+ "coder=" + coder +
+ '}';
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public <X> TypeInformation<X> getTypeAt(int pos) {
+ if (pos == 0) {
+ return (TypeInformation<X>) new CoderTypeInformation<>(coder.getKeyCoder());
+ } else if (pos == 1) {
+ return (TypeInformation<X>) new CoderTypeInformation<>(coder.getValueCoder());
+ } else {
+ throw new RuntimeException("Invalid field position " + pos);
+ }
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public <X> TypeInformation<X> getTypeAt(String fieldExpression) {
+ switch (fieldExpression) {
+ case "key":
+ return (TypeInformation<X>) new CoderTypeInformation<>(coder.getKeyCoder());
+ case "value":
+ return (TypeInformation<X>) new CoderTypeInformation<>(coder.getValueCoder());
+ default:
+ throw new UnsupportedOperationException("Only KvCoder has fields.");
+ }
+ }
+
+ @Override
+ public String[] getFieldNames() {
+ return new String[]{"key", "value"};
+ }
+
+ @Override
+ public int getFieldIndex(String fieldName) {
+ switch (fieldName) {
+ case "key":
+ return 0;
+ case "value":
+ return 1;
+ default:
+ return -1;
+ }
+ }
+
+ @Override
+ public void getFlatFields(String fieldExpression, int offset, List<FlatFieldDescriptor> result) {
+ CoderTypeInformation keyTypeInfo = new CoderTypeInformation<>(coder.getKeyCoder());
+ result.add(new FlatFieldDescriptor(0, keyTypeInfo));
+ }
+
+ @Override
+ protected TypeComparatorBuilder<KV<K, V>> createTypeComparatorBuilder() {
+ return new KvCoderTypeComparatorBuilder();
+ }
+
+ private class KvCoderTypeComparatorBuilder implements TypeComparatorBuilder<KV<K, V>> {
+
+ @Override
+ public void initializeTypeComparatorBuilder(int size) {}
+
+ @Override
+ public void addComparatorField(int fieldId, TypeComparator<?> comparator) {}
+
+ @Override
+ public TypeComparator<KV<K, V>> createTypeComparator(ExecutionConfig config) {
+ return new KvCoderComperator<>(coder);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java
new file mode 100644
index 0000000..c7b6ea2
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/VoidCoderTypeSerializer.java
@@ -0,0 +1,112 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.types;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+
+/**
+ * Special Flink {@link org.apache.flink.api.common.typeutils.TypeSerializer} for
+ * {@link com.google.cloud.dataflow.sdk.coders.VoidCoder}. We need this because Flink does not
+ * allow returning {@code null} from an input reader. We return a {@link VoidValue} instead
+ * that behaves like a {@code null}, hopefully.
+ */
+public class VoidCoderTypeSerializer extends TypeSerializer<VoidCoderTypeSerializer.VoidValue> {
+
+ @Override
+ public boolean isImmutableType() {
+ return false;
+ }
+
+ @Override
+ public VoidCoderTypeSerializer duplicate() {
+ return this;
+ }
+
+ @Override
+ public VoidValue createInstance() {
+ return VoidValue.INSTANCE;
+ }
+
+ @Override
+ public VoidValue copy(VoidValue from) {
+ return from;
+ }
+
+ @Override
+ public VoidValue copy(VoidValue from, VoidValue reuse) {
+ return from;
+ }
+
+ @Override
+ public int getLength() {
+ return 0;
+ }
+
+ @Override
+ public void serialize(VoidValue record, DataOutputView target) throws IOException {
+ target.writeByte(1);
+ }
+
+ @Override
+ public VoidValue deserialize(DataInputView source) throws IOException {
+ source.readByte();
+ return VoidValue.INSTANCE;
+ }
+
+ @Override
+ public VoidValue deserialize(VoidValue reuse, DataInputView source) throws IOException {
+ return deserialize(source);
+ }
+
+ @Override
+ public void copy(DataInputView source, DataOutputView target) throws IOException {
+ source.readByte();
+ target.writeByte(1);
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj instanceof VoidCoderTypeSerializer) {
+ VoidCoderTypeSerializer other = (VoidCoderTypeSerializer) obj;
+ return other.canEqual(this);
+ } else {
+ return false;
+ }
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return obj instanceof VoidCoderTypeSerializer;
+ }
+
+ @Override
+ public int hashCode() {
+ return 0;
+ }
+
+ public static class VoidValue {
+ private VoidValue() {}
+
+ public static VoidValue INSTANCE = new VoidValue();
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java
new file mode 100644
index 0000000..815765c
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/CombineFnAggregatorWrapper.java
@@ -0,0 +1,92 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers;
+
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.common.collect.Lists;
+import org.apache.flink.api.common.accumulators.Accumulator;
+
+import java.io.Serializable;
+
+/**
+ * Wrapper that wraps a {@link com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn}
+ * in a Flink {@link org.apache.flink.api.common.accumulators.Accumulator} for using
+ * the combine function as an aggregator in a {@link com.google.cloud.dataflow.sdk.transforms.ParDo}
+ * operation.
+ */
+public class CombineFnAggregatorWrapper<AI, AA, AR> implements Aggregator<AI, AR>, Accumulator<AI, Serializable> {
+
+ private AA aa;
+ private Combine.CombineFn<? super AI, AA, AR> combiner;
+
+ public CombineFnAggregatorWrapper() {
+ }
+
+ public CombineFnAggregatorWrapper(Combine.CombineFn<? super AI, AA, AR> combiner) {
+ this.combiner = combiner;
+ this.aa = combiner.createAccumulator();
+ }
+
+ @Override
+ public void add(AI value) {
+ combiner.addInput(aa, value);
+ }
+
+ @Override
+ public Serializable getLocalValue() {
+ return (Serializable) combiner.extractOutput(aa);
+ }
+
+ @Override
+ public void resetLocal() {
+ aa = combiner.createAccumulator();
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void merge(Accumulator<AI, Serializable> other) {
+ aa = combiner.mergeAccumulators(Lists.newArrayList(aa, ((CombineFnAggregatorWrapper<AI, AA, AR>)other).aa));
+ }
+
+ @Override
+ public Accumulator<AI, Serializable> clone() {
+ // copy it by merging
+ AA aaCopy = combiner.mergeAccumulators(Lists.newArrayList(aa));
+ CombineFnAggregatorWrapper<AI, AA, AR> result = new
+ CombineFnAggregatorWrapper<>(combiner);
+ result.aa = aaCopy;
+ return result;
+ }
+
+ @Override
+ public void addValue(AI value) {
+ add(value);
+ }
+
+ @Override
+ public String getName() {
+ return "CombineFn: " + combiner.toString();
+ }
+
+ @Override
+ public Combine.CombineFn getCombineFn() {
+ return combiner;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java
new file mode 100644
index 0000000..b56a90e
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers;
+
+import org.apache.flink.core.memory.DataInputView;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * Wrapper for {@link DataInputView}. We need this because Flink reads data using a
+ * {@link org.apache.flink.core.memory.DataInputView} while
+ * Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s expect an
+ * {@link java.io.InputStream}.
+ */
+public class DataInputViewWrapper extends InputStream {
+
+ private DataInputView inputView;
+
+ public DataInputViewWrapper(DataInputView inputView) {
+ this.inputView = inputView;
+ }
+
+ public void setInputView(DataInputView inputView) {
+ this.inputView = inputView;
+ }
+
+ @Override
+ public int read() throws IOException {
+ try {
+ return inputView.readUnsignedByte();
+ } catch (EOFException e) {
+ // translate between DataInput and InputStream,
+ // DataInput signals EOF by exception, InputStream does it by returning -1
+ return -1;
+ }
+ }
+
+ @Override
+ public int read(byte[] b, int off, int len) throws IOException {
+ return inputView.read(b, off, len);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
new file mode 100644
index 0000000..513d7f8
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers;
+
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * Wrapper for {@link org.apache.flink.core.memory.DataOutputView}. We need this because
+ * Flink writes data using a {@link org.apache.flink.core.memory.DataInputView} while
+ * Dataflow {@link com.google.cloud.dataflow.sdk.coders.Coder}s expect an
+ * {@link java.io.OutputStream}.
+ */
+public class DataOutputViewWrapper extends OutputStream {
+
+ private DataOutputView outputView;
+
+ public DataOutputViewWrapper(DataOutputView outputView) {
+ this.outputView = outputView;
+ }
+
+ public void setOutputView(DataOutputView outputView) {
+ this.outputView = outputView;
+ }
+
+ @Override
+ public void write(int b) throws IOException {
+ outputView.write(b);
+ }
+
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ outputView.write(b, off, len);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java
new file mode 100644
index 0000000..0d03f9f
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java
@@ -0,0 +1,91 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers;
+
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.apache.flink.api.common.accumulators.Accumulator;
+
+import java.io.Serializable;
+
+/**
+ * Wrapper that wraps a {@link com.google.cloud.dataflow.sdk.transforms.Combine.CombineFn}
+ * in a Flink {@link org.apache.flink.api.common.accumulators.Accumulator} for using
+ * the function as an aggregator in a {@link com.google.cloud.dataflow.sdk.transforms.ParDo}
+ * operation.
+ */
+public class SerializableFnAggregatorWrapper<AI, AO> implements Aggregator<AI, AO>, Accumulator<AI, Serializable> {
+
+ private AO aa;
+ private Combine.CombineFn<AI, ?, AO> combiner;
+
+ public SerializableFnAggregatorWrapper(Combine.CombineFn<AI, ?, AO> combiner) {
+ this.combiner = combiner;
+ resetLocal();
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void add(AI value) {
+ this.aa = combiner.apply(ImmutableList.of((AI) aa, value));
+ }
+
+ @Override
+ public Serializable getLocalValue() {
+ return (Serializable) aa;
+ }
+
+ @Override
+ public void resetLocal() {
+ this.aa = combiner.apply(ImmutableList.<AI>of());
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void merge(Accumulator<AI, Serializable> other) {
+ this.aa = combiner.apply(ImmutableList.of((AI) aa, (AI) other.getLocalValue()));
+ }
+
+ @Override
+ public void addValue(AI value) {
+ add(value);
+ }
+
+ @Override
+ public String getName() {
+ return "Aggregator :" + combiner.toString();
+ }
+
+ @Override
+ public Combine.CombineFn<AI, ?, AO> getCombineFn() {
+ return combiner;
+ }
+
+ @Override
+ public Accumulator<AI, Serializable> clone() {
+ // copy it by merging
+ AO resultCopy = combiner.apply(Lists.newArrayList((AI) aa));
+ SerializableFnAggregatorWrapper<AI, AO> result = new
+ SerializableFnAggregatorWrapper<>(combiner);
+
+ result.aa = resultCopy;
+ return result;
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java
new file mode 100644
index 0000000..d0423b9
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SinkOutputFormat.java
@@ -0,0 +1,121 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.cloud.dataflow.sdk.io.Sink;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.common.base.Preconditions;
+import com.google.cloud.dataflow.sdk.transforms.Write;
+import org.apache.flink.api.common.io.OutputFormat;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.util.AbstractID;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.lang.reflect.Field;
+
+/**
+ * Wrapper class to use generic Write.Bound transforms as sinks.
+ * @param <T> The type of the incoming records.
+ */
+public class SinkOutputFormat<T> implements OutputFormat<T> {
+
+ private final Sink<T> sink;
+
+ private transient PipelineOptions pipelineOptions;
+
+ private Sink.WriteOperation<T, ?> writeOperation;
+ private Sink.Writer<T, ?> writer;
+
+ private AbstractID uid = new AbstractID();
+
+ public SinkOutputFormat(Write.Bound<T> transform, PipelineOptions pipelineOptions) {
+ this.sink = extractSink(transform);
+ this.pipelineOptions = Preconditions.checkNotNull(pipelineOptions);
+ }
+
+ private Sink<T> extractSink(Write.Bound<T> transform) {
+ // TODO possibly add a getter in the upstream
+ try {
+ Field sinkField = transform.getClass().getDeclaredField("sink");
+ sinkField.setAccessible(true);
+ @SuppressWarnings("unchecked")
+ Sink<T> extractedSink = (Sink<T>) sinkField.get(transform);
+ return extractedSink;
+ } catch (NoSuchFieldException | IllegalAccessException e) {
+ throw new RuntimeException("Could not acquire custom sink field.", e);
+ }
+ }
+
+ @Override
+ public void configure(Configuration configuration) {
+ writeOperation = sink.createWriteOperation(pipelineOptions);
+ try {
+ writeOperation.initialize(pipelineOptions);
+ } catch (Exception e) {
+ throw new RuntimeException("Failed to initialize the write operation.", e);
+ }
+ }
+
+ @Override
+ public void open(int taskNumber, int numTasks) throws IOException {
+ try {
+ writer = writeOperation.createWriter(pipelineOptions);
+ } catch (Exception e) {
+ throw new IOException("Couldn't create writer.", e);
+ }
+ try {
+ writer.open(uid + "-" + String.valueOf(taskNumber));
+ } catch (Exception e) {
+ throw new IOException("Couldn't open writer.", e);
+ }
+ }
+
+ @Override
+ public void writeRecord(T record) throws IOException {
+ try {
+ writer.write(record);
+ } catch (Exception e) {
+ throw new IOException("Couldn't write record.", e);
+ }
+ }
+
+ @Override
+ public void close() throws IOException {
+ try {
+ writer.close();
+ } catch (Exception e) {
+ throw new IOException("Couldn't close writer.", e);
+ }
+ }
+
+ private void writeObject(ObjectOutputStream out) throws IOException, ClassNotFoundException {
+ out.defaultWriteObject();
+ ObjectMapper mapper = new ObjectMapper();
+ mapper.writeValue(out, pipelineOptions);
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ ObjectMapper mapper = new ObjectMapper();
+ pipelineOptions = mapper.readValue(in, PipelineOptions.class);
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
new file mode 100644
index 0000000..2d62416
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
@@ -0,0 +1,164 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import com.google.cloud.dataflow.sdk.io.BoundedSource;
+import com.google.cloud.dataflow.sdk.io.Source;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import org.apache.flink.api.common.io.InputFormat;
+import org.apache.flink.api.common.io.statistics.BaseStatistics;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.InputSplitAssigner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A Flink {@link org.apache.flink.api.common.io.InputFormat} that wraps a
+ * Dataflow {@link com.google.cloud.dataflow.sdk.io.Source}.
+ */
+public class SourceInputFormat<T> implements InputFormat<T, SourceInputSplit<T>> {
+ private static final Logger LOG = LoggerFactory.getLogger(SourceInputFormat.class);
+
+ private final BoundedSource<T> initialSource;
+ private transient PipelineOptions options;
+
+ private BoundedSource.BoundedReader<T> reader = null;
+ private boolean reachedEnd = true;
+
+ public SourceInputFormat(BoundedSource<T> initialSource, PipelineOptions options) {
+ this.initialSource = initialSource;
+ this.options = options;
+ }
+
+ private void writeObject(ObjectOutputStream out)
+ throws IOException, ClassNotFoundException {
+ out.defaultWriteObject();
+ ObjectMapper mapper = new ObjectMapper();
+ mapper.writeValue(out, options);
+ }
+
+ private void readObject(ObjectInputStream in)
+ throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ ObjectMapper mapper = new ObjectMapper();
+ options = mapper.readValue(in, PipelineOptions.class);
+ }
+
+ @Override
+ public void configure(Configuration configuration) {}
+
+ @Override
+ public void open(SourceInputSplit<T> sourceInputSplit) throws IOException {
+ reader = ((BoundedSource<T>) sourceInputSplit.getSource()).createReader(options);
+ reachedEnd = false;
+ }
+
+ @Override
+ public BaseStatistics getStatistics(BaseStatistics baseStatistics) throws IOException {
+ try {
+ final long estimatedSize = initialSource.getEstimatedSizeBytes(options);
+
+ return new BaseStatistics() {
+ @Override
+ public long getTotalInputSize() {
+ return estimatedSize;
+
+ }
+
+ @Override
+ public long getNumberOfRecords() {
+ return BaseStatistics.NUM_RECORDS_UNKNOWN;
+ }
+
+ @Override
+ public float getAverageRecordWidth() {
+ return BaseStatistics.AVG_RECORD_BYTES_UNKNOWN;
+ }
+ };
+ } catch (Exception e) {
+ LOG.warn("Could not read Source statistics: {}", e);
+ }
+
+ return null;
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public SourceInputSplit<T>[] createInputSplits(int numSplits) throws IOException {
+ long desiredSizeBytes;
+ try {
+ desiredSizeBytes = initialSource.getEstimatedSizeBytes(options) / numSplits;
+ List<? extends Source<T>> shards = initialSource.splitIntoBundles(desiredSizeBytes,
+ options);
+ List<SourceInputSplit<T>> splits = new ArrayList<>();
+ int splitCount = 0;
+ for (Source<T> shard: shards) {
+ splits.add(new SourceInputSplit<>(shard, splitCount++));
+ }
+ return splits.toArray(new SourceInputSplit[splits.size()]);
+ } catch (Exception e) {
+ throw new IOException("Could not create input splits from Source.", e);
+ }
+ }
+
+ @Override
+ public InputSplitAssigner getInputSplitAssigner(final SourceInputSplit[] sourceInputSplits) {
+ return new InputSplitAssigner() {
+ private int index = 0;
+ private final SourceInputSplit[] splits = sourceInputSplits;
+ @Override
+ public InputSplit getNextInputSplit(String host, int taskId) {
+ if (index < splits.length) {
+ return splits[index++];
+ } else {
+ return null;
+ }
+ }
+ };
+ }
+
+
+ @Override
+ public boolean reachedEnd() throws IOException {
+ return reachedEnd;
+ }
+
+ @Override
+ public T nextRecord(T t) throws IOException {
+
+ reachedEnd = !reader.advance();
+ if (!reachedEnd) {
+ return reader.getCurrent();
+ }
+ return null;
+ }
+
+ @Override
+ public void close() throws IOException {
+ reader.close();
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/51bec310/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
new file mode 100644
index 0000000..1b45ad7
--- /dev/null
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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.beam.runners.flink.translation.wrappers;
+
+import com.google.cloud.dataflow.sdk.io.Source;
+import org.apache.flink.core.io.InputSplit;
+
+/**
+ * {@link org.apache.flink.core.io.InputSplit} for
+ * {@link org.apache.beam.runners.flink.translation.wrappers.SourceInputFormat}. We pass
+ * the sharded Source around in the input split because Sources simply split up into several
+ * Sources for sharding. This is different to how Flink creates a separate InputSplit from
+ * an InputFormat.
+ */
+public class SourceInputSplit<T> implements InputSplit {
+
+ private Source<T> source;
+ private int splitNumber;
+
+ public SourceInputSplit() {
+ }
+
+ public SourceInputSplit(Source<T> source, int splitNumber) {
+ this.source = source;
+ this.splitNumber = splitNumber;
+ }
+
+ @Override
+ public int getSplitNumber() {
+ return splitNumber;
+ }
+
+ public Source<T> getSource() {
+ return source;
+ }
+
+}
[40/50] [abbrv] incubator-beam git commit: [flink] convert tabs to 2
spaces
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
index 01f9c32..fb240f4 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupAlsoByWindowTest.java
@@ -43,464 +43,464 @@ import java.util.concurrent.ConcurrentLinkedQueue;
public class GroupAlsoByWindowTest {
- private final Combine.CombineFn combiner = new Sum.SumIntegerFn();
-
- private final WindowingStrategy slidingWindowWithAfterWatermarkTriggerStrategy =
- WindowingStrategy.of(SlidingWindows.of(Duration.standardSeconds(10)).every(Duration.standardSeconds(5)))
- .withTrigger(AfterWatermark.pastEndOfWindow()).withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES);
-
- private final WindowingStrategy sessionWindowingStrategy =
- WindowingStrategy.of(Sessions.withGapDuration(Duration.standardSeconds(2)))
- .withTrigger(Repeatedly.forever(AfterWatermark.pastEndOfWindow()))
- .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES)
- .withAllowedLateness(Duration.standardSeconds(100));
-
- private final WindowingStrategy fixedWindowingStrategy =
- WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(10)));
-
- private final WindowingStrategy fixedWindowWithCountTriggerStrategy =
- fixedWindowingStrategy.withTrigger(AfterPane.elementCountAtLeast(5));
-
- private final WindowingStrategy fixedWindowWithAfterWatermarkTriggerStrategy =
- fixedWindowingStrategy.withTrigger(AfterWatermark.pastEndOfWindow());
-
- private final WindowingStrategy fixedWindowWithCompoundTriggerStrategy =
- fixedWindowingStrategy.withTrigger(
- AfterWatermark.pastEndOfWindow().withEarlyFirings(AfterPane.elementCountAtLeast(5))
- .withLateFirings(AfterPane.elementCountAtLeast(5)).buildTrigger());
-
- /**
- * The default accumulation mode is
- * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#DISCARDING_FIRED_PANES}.
- * This strategy changes it to
- * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#ACCUMULATING_FIRED_PANES}
- */
- private final WindowingStrategy fixedWindowWithCompoundTriggerStrategyAcc =
- fixedWindowWithCompoundTriggerStrategy
- .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES);
-
- @Test
- public void testWithLateness() throws Exception {
- WindowingStrategy strategy = WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(2)))
- .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES)
- .withAllowedLateness(Duration.millis(1000));
- long initialTime = 0L;
- Pipeline pipeline = FlinkTestPipeline.createForStreaming();
-
- KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
-
- FlinkGroupAlsoByWindowWrapper gbwOperaror =
- FlinkGroupAlsoByWindowWrapper.createForTesting(
- pipeline.getOptions(),
- pipeline.getCoderRegistry(),
- strategy,
- inputCoder,
- combiner.<String>asKeyedFn());
-
- OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
- new OneInputStreamOperatorTestHarness<>(gbwOperaror);
- testHarness.open();
-
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processWatermark(new Watermark(initialTime + 2000));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processWatermark(new Watermark(initialTime + 4000));
-
- ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 4),
- new Instant(initialTime + 1),
- new IntervalWindow(new Instant(0), new Instant(2000)),
- PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
- , initialTime + 1));
- expectedOutput.add(new Watermark(initialTime + 2000));
-
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 5),
- new Instant(initialTime + 1999),
- new IntervalWindow(new Instant(0), new Instant(2000)),
- PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 1, 1))
- , initialTime + 1999));
-
-
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 6),
- new Instant(initialTime + 1999),
- new IntervalWindow(new Instant(0), new Instant(2000)),
- PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 2, 2))
- , initialTime + 1999));
- expectedOutput.add(new Watermark(initialTime + 4000));
-
- TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
- testHarness.close();
- }
-
- @Test
- public void testSessionWindows() throws Exception {
- WindowingStrategy strategy = sessionWindowingStrategy;
-
- long initialTime = 0L;
- Pipeline pipeline = FlinkTestPipeline.createForStreaming();
-
- KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
-
- FlinkGroupAlsoByWindowWrapper gbwOperaror =
- FlinkGroupAlsoByWindowWrapper.createForTesting(
- pipeline.getOptions(),
- pipeline.getCoderRegistry(),
- strategy,
- inputCoder,
- combiner.<String>asKeyedFn());
-
- OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
- new OneInputStreamOperatorTestHarness<>(gbwOperaror);
- testHarness.open();
-
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 3500), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 3700), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 2700), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processWatermark(new Watermark(initialTime + 6000));
-
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 6700), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 6800), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 8900), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 7600), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 5600), null, PaneInfo.NO_FIRING), initialTime + 20));
-
- testHarness.processWatermark(new Watermark(initialTime + 12000));
-
- ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 6),
- new Instant(initialTime + 1),
- new IntervalWindow(new Instant(1), new Instant(5700)),
- PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
- , initialTime + 1));
- expectedOutput.add(new Watermark(initialTime + 6000));
-
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 11),
- new Instant(initialTime + 6700),
- new IntervalWindow(new Instant(1), new Instant(10900)),
- PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
- , initialTime + 6700));
- expectedOutput.add(new Watermark(initialTime + 12000));
-
- TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
- testHarness.close();
- }
-
- @Test
- public void testSlidingWindows() throws Exception {
- WindowingStrategy strategy = slidingWindowWithAfterWatermarkTriggerStrategy;
- long initialTime = 0L;
- OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
- createTestingOperatorAndState(strategy, initialTime);
- ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
- testHarness.processWatermark(new Watermark(initialTime + 25000));
-
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 6),
- new Instant(initialTime + 5000),
- new IntervalWindow(new Instant(0), new Instant(10000)),
- PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime + 5000));
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 6),
- new Instant(initialTime + 1),
- new IntervalWindow(new Instant(-5000), new Instant(5000)),
- PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime + 1));
- expectedOutput.add(new Watermark(initialTime + 10000));
-
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 11),
- new Instant(initialTime + 15000),
- new IntervalWindow(new Instant(10000), new Instant(20000)),
- PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime + 15000));
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 3),
- new Instant(initialTime + 10000),
- new IntervalWindow(new Instant(5000), new Instant(15000)),
- PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime + 10000));
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key2", 1),
- new Instant(initialTime + 19500),
- new IntervalWindow(new Instant(10000), new Instant(20000)),
- PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime + 19500));
- expectedOutput.add(new Watermark(initialTime + 20000));
-
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key2", 1),
- new Instant(initialTime + 20000),
- /**
- * this is 20000 and not 19500 because of a convention in dataflow where
- * timestamps of windowed values in a window cannot be smaller than the
- * end of a previous window. Checkout the documentation of the
- * {@link WindowFn#getOutputTime(Instant, BoundedWindow)}
- */
- new IntervalWindow(new Instant(15000), new Instant(25000)),
- PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime + 20000));
- expectedOutput.add(new StreamRecord<>(
- WindowedValue.of(KV.of("key1", 8),
- new Instant(initialTime + 20000),
- new IntervalWindow(new Instant(15000), new Instant(25000)),
- PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
- , initialTime + 20000));
- expectedOutput.add(new Watermark(initialTime + 25000));
-
- TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
- testHarness.close();
- }
-
- @Test
- public void testAfterWatermarkProgram() throws Exception {
- WindowingStrategy strategy = fixedWindowWithAfterWatermarkTriggerStrategy;
- long initialTime = 0L;
- OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
- createTestingOperatorAndState(strategy, initialTime);
- ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6),
- new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 1));
- expectedOutput.add(new Watermark(initialTime + 10000));
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11),
- new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 10000));
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500));
- expectedOutput.add(new Watermark(initialTime + 20000));
-
- TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
- testHarness.close();
- }
-
- @Test
- public void testAfterCountProgram() throws Exception {
- WindowingStrategy strategy = fixedWindowWithCountTriggerStrategy;
-
- long initialTime = 0L;
- OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
- createTestingOperatorAndState(strategy, initialTime);
- ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime + 1));
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime + 10000));
- expectedOutput.add(new Watermark(initialTime + 10000));
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME, 0, 0)), initialTime + 19500));
- expectedOutput.add(new Watermark(initialTime + 20000));
- TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
-
- testHarness.close();
- }
-
- @Test
- public void testCompoundProgram() throws Exception {
- WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategy;
-
- long initialTime = 0L;
- OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
- createTestingOperatorAndState(strategy, initialTime);
- ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
- /**
- * PaneInfo are:
- * isFirst (pane in window),
- * isLast, Timing (of triggering),
- * index (of pane in the window),
- * onTimeIndex (if it the 1st,2nd, ... pane that was fired on time)
- * */
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 1));
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 10000));
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime + 19500));
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1),
- new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime + 1200));
-
- expectedOutput.add(new Watermark(initialTime + 10000));
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime + 19500));
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500));
-
- expectedOutput.add(new Watermark(initialTime + 20000));
- TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
-
- testHarness.close();
- }
-
- @Test
- public void testCompoundAccumulatingPanesProgram() throws Exception {
- WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategyAcc;
- long initialTime = 0L;
- OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
- createTestingOperatorAndState(strategy, initialTime);
- ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 1));
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
- new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 10000));
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 10),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime + 19500));
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6),
- new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime + 1200));
-
- expectedOutput.add(new Watermark(initialTime + 10000));
-
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime + 19500));
- expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
- new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500));
-
- expectedOutput.add(new Watermark(initialTime + 20000));
- TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
-
- testHarness.close();
- }
-
- private OneInputStreamOperatorTestHarness createTestingOperatorAndState(WindowingStrategy strategy, long initialTime) throws Exception {
- Pipeline pipeline = FlinkTestPipeline.createForStreaming();
-
- KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
-
- FlinkGroupAlsoByWindowWrapper gbwOperaror =
- FlinkGroupAlsoByWindowWrapper.createForTesting(
- pipeline.getOptions(),
- pipeline.getCoderRegistry(),
- strategy,
- inputCoder,
- combiner.<String>asKeyedFn());
-
- OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
- new OneInputStreamOperatorTestHarness<>(gbwOperaror);
- testHarness.open();
-
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
-
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 10000), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 12100), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 14200), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 15300), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 16500), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
-
- testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
-
- testHarness.processWatermark(new Watermark(initialTime + 10000));
- testHarness.processWatermark(new Watermark(initialTime + 20000));
-
- return testHarness;
- }
-
- private static class ResultSortComparator implements Comparator<Object> {
- @Override
- public int compare(Object o1, Object o2) {
- if (o1 instanceof Watermark && o2 instanceof Watermark) {
- Watermark w1 = (Watermark) o1;
- Watermark w2 = (Watermark) o2;
- return (int) (w1.getTimestamp() - w2.getTimestamp());
- } else {
- StreamRecord<WindowedValue<KV<String, Integer>>> sr0 = (StreamRecord<WindowedValue<KV<String, Integer>>>) o1;
- StreamRecord<WindowedValue<KV<String, Integer>>> sr1 = (StreamRecord<WindowedValue<KV<String, Integer>>>) o2;
-
- int comparison = (int) (sr0.getValue().getTimestamp().getMillis() - sr1.getValue().getTimestamp().getMillis());
- if (comparison != 0) {
- return comparison;
- }
-
- comparison = sr0.getValue().getValue().getKey().compareTo(sr1.getValue().getValue().getKey());
- if(comparison == 0) {
- comparison = Integer.compare(
- sr0.getValue().getValue().getValue(),
- sr1.getValue().getValue().getValue());
- }
- if(comparison == 0) {
- Collection windowsA = sr0.getValue().getWindows();
- Collection windowsB = sr1.getValue().getWindows();
-
- if(windowsA.size() != 1 || windowsB.size() != 1) {
- throw new IllegalStateException("A value cannot belong to more than one windows after grouping.");
- }
-
- BoundedWindow windowA = (BoundedWindow) windowsA.iterator().next();
- BoundedWindow windowB = (BoundedWindow) windowsB.iterator().next();
- comparison = Long.compare(windowA.maxTimestamp().getMillis(), windowB.maxTimestamp().getMillis());
- }
- return comparison;
- }
- }
- }
-
- private <T> WindowedValue<T> makeWindowedValue(WindowingStrategy strategy,
- T output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
- final Instant inputTimestamp = timestamp;
- final WindowFn windowFn = strategy.getWindowFn();
-
- if (timestamp == null) {
- timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
- }
-
- if (windows == null) {
- try {
- windows = windowFn.assignWindows(windowFn.new AssignContext() {
- @Override
- public Object element() {
- throw new UnsupportedOperationException(
- "WindowFn attempted to access input element when none was available");
- }
-
- @Override
- public Instant timestamp() {
- if (inputTimestamp == null) {
- throw new UnsupportedOperationException(
- "WindowFn attempted to access input timestamp when none was available");
- }
- return inputTimestamp;
- }
-
- @Override
- public Collection<? extends BoundedWindow> windows() {
- throw new UnsupportedOperationException(
- "WindowFn attempted to access input windows when none were available");
- }
- });
- } catch (Exception e) {
- throw UserCodeException.wrap(e);
- }
- }
-
- return WindowedValue.of(output, timestamp, windows, pane);
- }
+ private final Combine.CombineFn combiner = new Sum.SumIntegerFn();
+
+ private final WindowingStrategy slidingWindowWithAfterWatermarkTriggerStrategy =
+ WindowingStrategy.of(SlidingWindows.of(Duration.standardSeconds(10)).every(Duration.standardSeconds(5)))
+ .withTrigger(AfterWatermark.pastEndOfWindow()).withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES);
+
+ private final WindowingStrategy sessionWindowingStrategy =
+ WindowingStrategy.of(Sessions.withGapDuration(Duration.standardSeconds(2)))
+ .withTrigger(Repeatedly.forever(AfterWatermark.pastEndOfWindow()))
+ .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES)
+ .withAllowedLateness(Duration.standardSeconds(100));
+
+ private final WindowingStrategy fixedWindowingStrategy =
+ WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(10)));
+
+ private final WindowingStrategy fixedWindowWithCountTriggerStrategy =
+ fixedWindowingStrategy.withTrigger(AfterPane.elementCountAtLeast(5));
+
+ private final WindowingStrategy fixedWindowWithAfterWatermarkTriggerStrategy =
+ fixedWindowingStrategy.withTrigger(AfterWatermark.pastEndOfWindow());
+
+ private final WindowingStrategy fixedWindowWithCompoundTriggerStrategy =
+ fixedWindowingStrategy.withTrigger(
+ AfterWatermark.pastEndOfWindow().withEarlyFirings(AfterPane.elementCountAtLeast(5))
+ .withLateFirings(AfterPane.elementCountAtLeast(5)).buildTrigger());
+
+ /**
+ * The default accumulation mode is
+ * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#DISCARDING_FIRED_PANES}.
+ * This strategy changes it to
+ * {@link com.google.cloud.dataflow.sdk.util.WindowingStrategy.AccumulationMode#ACCUMULATING_FIRED_PANES}
+ */
+ private final WindowingStrategy fixedWindowWithCompoundTriggerStrategyAcc =
+ fixedWindowWithCompoundTriggerStrategy
+ .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES);
+
+ @Test
+ public void testWithLateness() throws Exception {
+ WindowingStrategy strategy = WindowingStrategy.of(FixedWindows.of(Duration.standardSeconds(2)))
+ .withMode(WindowingStrategy.AccumulationMode.ACCUMULATING_FIRED_PANES)
+ .withAllowedLateness(Duration.millis(1000));
+ long initialTime = 0L;
+ Pipeline pipeline = FlinkTestPipeline.createForStreaming();
+
+ KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
+
+ FlinkGroupAlsoByWindowWrapper gbwOperaror =
+ FlinkGroupAlsoByWindowWrapper.createForTesting(
+ pipeline.getOptions(),
+ pipeline.getCoderRegistry(),
+ strategy,
+ inputCoder,
+ combiner.<String>asKeyedFn());
+
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ new OneInputStreamOperatorTestHarness<>(gbwOperaror);
+ testHarness.open();
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processWatermark(new Watermark(initialTime + 2000));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processWatermark(new Watermark(initialTime + 4000));
+
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 4),
+ new Instant(initialTime + 1),
+ new IntervalWindow(new Instant(0), new Instant(2000)),
+ PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
+ , initialTime + 1));
+ expectedOutput.add(new Watermark(initialTime + 2000));
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 5),
+ new Instant(initialTime + 1999),
+ new IntervalWindow(new Instant(0), new Instant(2000)),
+ PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 1, 1))
+ , initialTime + 1999));
+
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 6),
+ new Instant(initialTime + 1999),
+ new IntervalWindow(new Instant(0), new Instant(2000)),
+ PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 2, 2))
+ , initialTime + 1999));
+ expectedOutput.add(new Watermark(initialTime + 4000));
+
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+ testHarness.close();
+ }
+
+ @Test
+ public void testSessionWindows() throws Exception {
+ WindowingStrategy strategy = sessionWindowingStrategy;
+
+ long initialTime = 0L;
+ Pipeline pipeline = FlinkTestPipeline.createForStreaming();
+
+ KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
+
+ FlinkGroupAlsoByWindowWrapper gbwOperaror =
+ FlinkGroupAlsoByWindowWrapper.createForTesting(
+ pipeline.getOptions(),
+ pipeline.getCoderRegistry(),
+ strategy,
+ inputCoder,
+ combiner.<String>asKeyedFn());
+
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ new OneInputStreamOperatorTestHarness<>(gbwOperaror);
+ testHarness.open();
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 3500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 3700), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 2700), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processWatermark(new Watermark(initialTime + 6000));
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 6700), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 6800), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 8900), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 7600), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 5600), null, PaneInfo.NO_FIRING), initialTime + 20));
+
+ testHarness.processWatermark(new Watermark(initialTime + 12000));
+
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 6),
+ new Instant(initialTime + 1),
+ new IntervalWindow(new Instant(1), new Instant(5700)),
+ PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
+ , initialTime + 1));
+ expectedOutput.add(new Watermark(initialTime + 6000));
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 11),
+ new Instant(initialTime + 6700),
+ new IntervalWindow(new Instant(1), new Instant(10900)),
+ PaneInfo.createPane(true, false, PaneInfo.Timing.ON_TIME, 0, 0))
+ , initialTime + 6700));
+ expectedOutput.add(new Watermark(initialTime + 12000));
+
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+ testHarness.close();
+ }
+
+ @Test
+ public void testSlidingWindows() throws Exception {
+ WindowingStrategy strategy = slidingWindowWithAfterWatermarkTriggerStrategy;
+ long initialTime = 0L;
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ createTestingOperatorAndState(strategy, initialTime);
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+ testHarness.processWatermark(new Watermark(initialTime + 25000));
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 6),
+ new Instant(initialTime + 5000),
+ new IntervalWindow(new Instant(0), new Instant(10000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime + 5000));
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 6),
+ new Instant(initialTime + 1),
+ new IntervalWindow(new Instant(-5000), new Instant(5000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime + 1));
+ expectedOutput.add(new Watermark(initialTime + 10000));
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 11),
+ new Instant(initialTime + 15000),
+ new IntervalWindow(new Instant(10000), new Instant(20000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime + 15000));
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 3),
+ new Instant(initialTime + 10000),
+ new IntervalWindow(new Instant(5000), new Instant(15000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime + 10000));
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key2", 1),
+ new Instant(initialTime + 19500),
+ new IntervalWindow(new Instant(10000), new Instant(20000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime + 19500));
+ expectedOutput.add(new Watermark(initialTime + 20000));
+
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key2", 1),
+ new Instant(initialTime + 20000),
+ /**
+ * this is 20000 and not 19500 because of a convention in dataflow where
+ * timestamps of windowed values in a window cannot be smaller than the
+ * end of a previous window. Checkout the documentation of the
+ * {@link WindowFn#getOutputTime(Instant, BoundedWindow)}
+ */
+ new IntervalWindow(new Instant(15000), new Instant(25000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime + 20000));
+ expectedOutput.add(new StreamRecord<>(
+ WindowedValue.of(KV.of("key1", 8),
+ new Instant(initialTime + 20000),
+ new IntervalWindow(new Instant(15000), new Instant(25000)),
+ PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME))
+ , initialTime + 20000));
+ expectedOutput.add(new Watermark(initialTime + 25000));
+
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+ testHarness.close();
+ }
+
+ @Test
+ public void testAfterWatermarkProgram() throws Exception {
+ WindowingStrategy strategy = fixedWindowWithAfterWatermarkTriggerStrategy;
+ long initialTime = 0L;
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ createTestingOperatorAndState(strategy, initialTime);
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6),
+ new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 1));
+ expectedOutput.add(new Watermark(initialTime + 10000));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11),
+ new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 10000));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500));
+ expectedOutput.add(new Watermark(initialTime + 20000));
+
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+ testHarness.close();
+ }
+
+ @Test
+ public void testAfterCountProgram() throws Exception {
+ WindowingStrategy strategy = fixedWindowWithCountTriggerStrategy;
+
+ long initialTime = 0L;
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ createTestingOperatorAndState(strategy, initialTime);
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 1), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime + 1));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 10000), null, PaneInfo.createPane(true, true, PaneInfo.Timing.EARLY)), initialTime + 10000));
+ expectedOutput.add(new Watermark(initialTime + 10000));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME, 0, 0)), initialTime + 19500));
+ expectedOutput.add(new Watermark(initialTime + 20000));
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+ testHarness.close();
+ }
+
+ @Test
+ public void testCompoundProgram() throws Exception {
+ WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategy;
+
+ long initialTime = 0L;
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ createTestingOperatorAndState(strategy, initialTime);
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+ /**
+ * PaneInfo are:
+ * isFirst (pane in window),
+ * isLast, Timing (of triggering),
+ * index (of pane in the window),
+ * onTimeIndex (if it the 1st,2nd, ... pane that was fired on time)
+ * */
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 1));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 10000));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime + 19500));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1),
+ new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime + 1200));
+
+ expectedOutput.add(new Watermark(initialTime + 10000));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime + 19500));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500));
+
+ expectedOutput.add(new Watermark(initialTime + 20000));
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+ testHarness.close();
+ }
+
+ @Test
+ public void testCompoundAccumulatingPanesProgram() throws Exception {
+ WindowingStrategy strategy = fixedWindowWithCompoundTriggerStrategyAcc;
+ long initialTime = 0L;
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ createTestingOperatorAndState(strategy, initialTime);
+ ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 1), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 1));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 5),
+ new Instant(initialTime + 10000), null, PaneInfo.createPane(true, false, PaneInfo.Timing.EARLY)), initialTime + 10000));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 10),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(false, false, PaneInfo.Timing.EARLY, 1, -1)), initialTime + 19500));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 6),
+ new Instant(initialTime + 1200), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 1, 0)), initialTime + 1200));
+
+ expectedOutput.add(new Watermark(initialTime + 10000));
+
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 11),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(false, true, PaneInfo.Timing.ON_TIME, 2, 0)), initialTime + 19500));
+ expectedOutput.add(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1),
+ new Instant(initialTime + 19500), null, PaneInfo.createPane(true, true, PaneInfo.Timing.ON_TIME)), initialTime + 19500));
+
+ expectedOutput.add(new Watermark(initialTime + 20000));
+ TestHarnessUtil.assertOutputEqualsSorted("Output was not correct.", expectedOutput, testHarness.getOutput(), new ResultSortComparator());
+
+ testHarness.close();
+ }
+
+ private OneInputStreamOperatorTestHarness createTestingOperatorAndState(WindowingStrategy strategy, long initialTime) throws Exception {
+ Pipeline pipeline = FlinkTestPipeline.createForStreaming();
+
+ KvCoder<String, Integer> inputCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
+
+ FlinkGroupAlsoByWindowWrapper gbwOperaror =
+ FlinkGroupAlsoByWindowWrapper.createForTesting(
+ pipeline.getOptions(),
+ pipeline.getCoderRegistry(),
+ strategy,
+ inputCoder,
+ combiner.<String>asKeyedFn());
+
+ OneInputStreamOperatorTestHarness<WindowedValue<KV<String, Integer>>, WindowedValue<KV<String, Integer>>> testHarness =
+ new OneInputStreamOperatorTestHarness<>(gbwOperaror);
+ testHarness.open();
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1000), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 1200), null, PaneInfo.NO_FIRING), initialTime + 20));
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 10000), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 12100), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 14200), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 15300), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 16500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key1", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+
+ testHarness.processElement(new StreamRecord<>(makeWindowedValue(strategy, KV.of("key2", 1), new Instant(initialTime + 19500), null, PaneInfo.NO_FIRING), initialTime + 20));
+
+ testHarness.processWatermark(new Watermark(initialTime + 10000));
+ testHarness.processWatermark(new Watermark(initialTime + 20000));
+
+ return testHarness;
+ }
+
+ private static class ResultSortComparator implements Comparator<Object> {
+ @Override
+ public int compare(Object o1, Object o2) {
+ if (o1 instanceof Watermark && o2 instanceof Watermark) {
+ Watermark w1 = (Watermark) o1;
+ Watermark w2 = (Watermark) o2;
+ return (int) (w1.getTimestamp() - w2.getTimestamp());
+ } else {
+ StreamRecord<WindowedValue<KV<String, Integer>>> sr0 = (StreamRecord<WindowedValue<KV<String, Integer>>>) o1;
+ StreamRecord<WindowedValue<KV<String, Integer>>> sr1 = (StreamRecord<WindowedValue<KV<String, Integer>>>) o2;
+
+ int comparison = (int) (sr0.getValue().getTimestamp().getMillis() - sr1.getValue().getTimestamp().getMillis());
+ if (comparison != 0) {
+ return comparison;
+ }
+
+ comparison = sr0.getValue().getValue().getKey().compareTo(sr1.getValue().getValue().getKey());
+ if(comparison == 0) {
+ comparison = Integer.compare(
+ sr0.getValue().getValue().getValue(),
+ sr1.getValue().getValue().getValue());
+ }
+ if(comparison == 0) {
+ Collection windowsA = sr0.getValue().getWindows();
+ Collection windowsB = sr1.getValue().getWindows();
+
+ if(windowsA.size() != 1 || windowsB.size() != 1) {
+ throw new IllegalStateException("A value cannot belong to more than one windows after grouping.");
+ }
+
+ BoundedWindow windowA = (BoundedWindow) windowsA.iterator().next();
+ BoundedWindow windowB = (BoundedWindow) windowsB.iterator().next();
+ comparison = Long.compare(windowA.maxTimestamp().getMillis(), windowB.maxTimestamp().getMillis());
+ }
+ return comparison;
+ }
+ }
+ }
+
+ private <T> WindowedValue<T> makeWindowedValue(WindowingStrategy strategy,
+ T output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+ final Instant inputTimestamp = timestamp;
+ final WindowFn windowFn = strategy.getWindowFn();
+
+ if (timestamp == null) {
+ timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+ }
+
+ if (windows == null) {
+ try {
+ windows = windowFn.assignWindows(windowFn.new AssignContext() {
+ @Override
+ public Object element() {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input element when none was available");
+ }
+
+ @Override
+ public Instant timestamp() {
+ if (inputTimestamp == null) {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input timestamp when none was available");
+ }
+ return inputTimestamp;
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input windows when none were available");
+ }
+ });
+ } catch (Exception e) {
+ throw UserCodeException.wrap(e);
+ }
+ }
+
+ return WindowedValue.of(output, timestamp, windows, pane);
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java
index 5a412aa..52e9e25 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/GroupByNullKeyTest.java
@@ -39,83 +39,83 @@ import java.util.Arrays;
public class GroupByNullKeyTest extends StreamingProgramTestBase implements Serializable {
- protected String resultPath;
+ protected String resultPath;
- static final String[] EXPECTED_RESULT = new String[] {
- "k: null v: user1 user1 user1 user2 user2 user2 user2 user3"
- };
+ static final String[] EXPECTED_RESULT = new String[] {
+ "k: null v: user1 user1 user1 user2 user2 user2 user2 user3"
+ };
- public GroupByNullKeyTest(){
- }
+ public GroupByNullKeyTest(){
+ }
- @Override
- protected void preSubmit() throws Exception {
- resultPath = getTempDirPath("result");
- }
+ @Override
+ protected void preSubmit() throws Exception {
+ resultPath = getTempDirPath("result");
+ }
- @Override
- protected void postSubmit() throws Exception {
- compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
- }
+ @Override
+ protected void postSubmit() throws Exception {
+ compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath);
+ }
- public static class ExtractUserAndTimestamp extends DoFn<KV<Integer, String>, String> {
- private static final long serialVersionUID = 0;
+ public static class ExtractUserAndTimestamp extends DoFn<KV<Integer, String>, String> {
+ private static final long serialVersionUID = 0;
- @Override
- public void processElement(ProcessContext c) {
- KV<Integer, String> record = c.element();
- long now = System.currentTimeMillis();
- int timestamp = record.getKey();
- String userName = record.getValue();
- if (userName != null) {
- // Sets the implicit timestamp field to be used in windowing.
- c.outputWithTimestamp(userName, new Instant(timestamp + now));
- }
- }
- }
+ @Override
+ public void processElement(ProcessContext c) {
+ KV<Integer, String> record = c.element();
+ long now = System.currentTimeMillis();
+ int timestamp = record.getKey();
+ String userName = record.getValue();
+ if (userName != null) {
+ // Sets the implicit timestamp field to be used in windowing.
+ c.outputWithTimestamp(userName, new Instant(timestamp + now));
+ }
+ }
+ }
- @Override
- protected void testProgram() throws Exception {
+ @Override
+ protected void testProgram() throws Exception {
- Pipeline p = FlinkTestPipeline.createForStreaming();
+ Pipeline p = FlinkTestPipeline.createForStreaming();
- PCollection<String> output =
- p.apply(Create.of(Arrays.asList(
- KV.<Integer, String>of(0, "user1"),
- KV.<Integer, String>of(1, "user1"),
- KV.<Integer, String>of(2, "user1"),
- KV.<Integer, String>of(10, "user2"),
- KV.<Integer, String>of(1, "user2"),
- KV.<Integer, String>of(15000, "user2"),
- KV.<Integer, String>of(12000, "user2"),
- KV.<Integer, String>of(25000, "user3"))))
- .apply(ParDo.of(new ExtractUserAndTimestamp()))
- .apply(Window.<String>into(FixedWindows.of(Duration.standardHours(1)))
- .triggering(AfterWatermark.pastEndOfWindow())
- .withAllowedLateness(Duration.ZERO)
- .discardingFiredPanes())
+ PCollection<String> output =
+ p.apply(Create.of(Arrays.asList(
+ KV.<Integer, String>of(0, "user1"),
+ KV.<Integer, String>of(1, "user1"),
+ KV.<Integer, String>of(2, "user1"),
+ KV.<Integer, String>of(10, "user2"),
+ KV.<Integer, String>of(1, "user2"),
+ KV.<Integer, String>of(15000, "user2"),
+ KV.<Integer, String>of(12000, "user2"),
+ KV.<Integer, String>of(25000, "user3"))))
+ .apply(ParDo.of(new ExtractUserAndTimestamp()))
+ .apply(Window.<String>into(FixedWindows.of(Duration.standardHours(1)))
+ .triggering(AfterWatermark.pastEndOfWindow())
+ .withAllowedLateness(Duration.ZERO)
+ .discardingFiredPanes())
- .apply(ParDo.of(new DoFn<String, KV<Void, String>>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- String elem = c.element();
- c.output(KV.<Void, String>of((Void) null, elem));
- }
- }))
- .apply(GroupByKey.<Void, String>create())
- .apply(ParDo.of(new DoFn<KV<Void, Iterable<String>>, String>() {
- @Override
- public void processElement(ProcessContext c) throws Exception {
- KV<Void, Iterable<String>> elem = c.element();
- StringBuilder str = new StringBuilder();
- str.append("k: " + elem.getKey() + " v:");
- for (String v : elem.getValue()) {
- str.append(" " + v);
- }
- c.output(str.toString());
- }
- }));
- output.apply(TextIO.Write.to(resultPath));
- p.run();
- }
+ .apply(ParDo.of(new DoFn<String, KV<Void, String>>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ String elem = c.element();
+ c.output(KV.<Void, String>of((Void) null, elem));
+ }
+ }))
+ .apply(GroupByKey.<Void, String>create())
+ .apply(ParDo.of(new DoFn<KV<Void, Iterable<String>>, String>() {
+ @Override
+ public void processElement(ProcessContext c) throws Exception {
+ KV<Void, Iterable<String>> elem = c.element();
+ StringBuilder str = new StringBuilder();
+ str.append("k: " + elem.getKey() + " v:");
+ for (String v : elem.getValue()) {
+ str.append(" " + v);
+ }
+ c.output(str.toString());
+ }
+ }));
+ output.apply(TextIO.Write.to(resultPath));
+ p.run();
+ }
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/8852eb15/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java
index 7489fcc..d5b1043 100644
--- a/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java
+++ b/runners/flink/src/test/java/com/dataartisans/flink/dataflow/streaming/StateSerializationTest.java
@@ -43,261 +43,261 @@ import static org.junit.Assert.assertEquals;
public class StateSerializationTest {
- private static final StateNamespace NAMESPACE_1 = StateNamespaces.global();
- private static final String KEY_PREFIX = "TEST_";
-
- // TODO: This can be replaced with the standard Sum.SumIntererFn once the state no longer needs
- // to create a StateTag at the point of restoring state. Currently StateTags are compared strictly
- // by type and combiners always use KeyedCombineFnWithContext rather than KeyedCombineFn or CombineFn.
- private static CombineWithContext.KeyedCombineFnWithContext<Object, Integer, int[], Integer> SUM_COMBINER =
- new CombineWithContext.KeyedCombineFnWithContext<Object, Integer, int[], Integer>() {
- @Override
- public int[] createAccumulator(Object key, CombineWithContext.Context c) {
- return new int[1];
- }
-
- @Override
- public int[] addInput(Object key, int[] accumulator, Integer value, CombineWithContext.Context c) {
- accumulator[0] += value;
- return accumulator;
- }
-
- @Override
- public int[] mergeAccumulators(Object key, Iterable<int[]> accumulators, CombineWithContext.Context c) {
- int[] r = new int[1];
- for (int[] a : accumulators) {
- r[0] += a[0];
- }
- return r;
- }
-
- @Override
- public Integer extractOutput(Object key, int[] accumulator, CombineWithContext.Context c) {
- return accumulator[0];
- }
- };
-
- private static Coder<int[]> INT_ACCUM_CODER = DelegateCoder.of(
- VarIntCoder.of(),
- new DelegateCoder.CodingFunction<int[], Integer>() {
- @Override
- public Integer apply(int[] accumulator) {
- return accumulator[0];
- }
- },
- new DelegateCoder.CodingFunction<Integer, int[]>() {
- @Override
- public int[] apply(Integer value) {
- int[] a = new int[1];
- a[0] = value;
- return a;
- }
- });
-
- private static final StateTag<Object, ValueState<String>> STRING_VALUE_ADDR =
- StateTags.value("stringValue", StringUtf8Coder.of());
- private static final StateTag<Object, ValueState<Integer>> INT_VALUE_ADDR =
- StateTags.value("stringValue", VarIntCoder.of());
- private static final StateTag<Object, AccumulatorCombiningState<Integer, int[], Integer>> SUM_INTEGER_ADDR =
- StateTags.keyedCombiningValueWithContext("sumInteger", INT_ACCUM_CODER, SUM_COMBINER);
- private static final StateTag<Object, BagState<String>> STRING_BAG_ADDR =
- StateTags.bag("stringBag", StringUtf8Coder.of());
- private static final StateTag<Object, WatermarkHoldState<BoundedWindow>> WATERMARK_BAG_ADDR =
- StateTags.watermarkStateInternal("watermark", OutputTimeFns.outputAtEarliestInputTimestamp());
-
- private Map<String, FlinkStateInternals<String>> statePerKey = new HashMap<>();
-
- private Map<String, Set<TimerInternals.TimerData>> activeTimers = new HashMap<>();
-
- private void initializeStateAndTimers() throws CannotProvideCoderException {
- for (int i = 0; i < 10; i++) {
- String key = KEY_PREFIX + i;
-
- FlinkStateInternals state = initializeStateForKey(key);
- Set<TimerInternals.TimerData> timers = new HashSet<>();
- for (int j = 0; j < 5; j++) {
- TimerInternals.TimerData timer = TimerInternals
- .TimerData.of(NAMESPACE_1,
- new Instant(1000 + i + j), TimeDomain.values()[j % 3]);
- timers.add(timer);
- }
-
- statePerKey.put(key, state);
- activeTimers.put(key, timers);
- }
- }
-
- private FlinkStateInternals<String> initializeStateForKey(String key) throws CannotProvideCoderException {
- FlinkStateInternals<String> state = createState(key);
-
- ValueState<String> value = state.state(NAMESPACE_1, STRING_VALUE_ADDR);
- value.write("test");
-
- ValueState<Integer> value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR);
- value2.write(4);
- value2.write(5);
-
- AccumulatorCombiningState<Integer, int[], Integer> combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR);
- combiningValue.add(1);
- combiningValue.add(2);
-
- WatermarkHoldState<BoundedWindow> watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
- watermark.add(new Instant(1000));
-
- BagState<String> bag = state.state(NAMESPACE_1, STRING_BAG_ADDR);
- bag.add("v1");
- bag.add("v2");
- bag.add("v3");
- bag.add("v4");
- return state;
- }
-
- private boolean restoreAndTestState(DataInputView in) throws Exception {
- StateCheckpointReader reader = new StateCheckpointReader(in);
- final ClassLoader userClassloader = this.getClass().getClassLoader();
- Coder<? extends BoundedWindow> windowCoder = IntervalWindow.getCoder();
- Coder<String> keyCoder = StringUtf8Coder.of();
-
- boolean comparisonRes = true;
-
- for (String key : statePerKey.keySet()) {
- comparisonRes &= checkStateForKey(key);
- }
-
- // restore the timers
- Map<String, Set<TimerInternals.TimerData>> restoredTimersPerKey = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder);
- if (activeTimers.size() != restoredTimersPerKey.size()) {
- return false;
- }
-
- for (String key : statePerKey.keySet()) {
- Set<TimerInternals.TimerData> originalTimers = activeTimers.get(key);
- Set<TimerInternals.TimerData> restoredTimers = restoredTimersPerKey.get(key);
- comparisonRes &= checkTimersForKey(originalTimers, restoredTimers);
- }
-
- // restore the state
- Map<String, FlinkStateInternals<String>> restoredPerKeyState =
- StateCheckpointUtils.decodeState(reader, OutputTimeFns.outputAtEarliestInputTimestamp(), keyCoder, windowCoder, userClassloader);
- if (restoredPerKeyState.size() != statePerKey.size()) {
- return false;
- }
-
- for (String key : statePerKey.keySet()) {
- FlinkStateInternals<String> originalState = statePerKey.get(key);
- FlinkStateInternals<String> restoredState = restoredPerKeyState.get(key);
- comparisonRes &= checkStateForKey(originalState, restoredState);
- }
- return comparisonRes;
- }
-
- private boolean checkStateForKey(String key) throws CannotProvideCoderException {
- FlinkStateInternals<String> state = statePerKey.get(key);
-
- ValueState<String> value = state.state(NAMESPACE_1, STRING_VALUE_ADDR);
- boolean comp = value.read().equals("test");
-
- ValueState<Integer> value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR);
- comp &= value2.read().equals(5);
-
- AccumulatorCombiningState<Integer, int[], Integer> combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR);
- comp &= combiningValue.read().equals(3);
-
- WatermarkHoldState<BoundedWindow> watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
- comp &= watermark.read().equals(new Instant(1000));
-
- BagState<String> bag = state.state(NAMESPACE_1, STRING_BAG_ADDR);
- Iterator<String> it = bag.read().iterator();
- int i = 0;
- while (it.hasNext()) {
- comp &= it.next().equals("v" + (++i));
- }
- return comp;
- }
-
- private void storeState(AbstractStateBackend.CheckpointStateOutputView out) throws Exception {
- StateCheckpointWriter checkpointBuilder = StateCheckpointWriter.create(out);
- Coder<String> keyCoder = StringUtf8Coder.of();
-
- // checkpoint the timers
- StateCheckpointUtils.encodeTimers(activeTimers, checkpointBuilder, keyCoder);
-
- // checkpoint the state
- StateCheckpointUtils.encodeState(statePerKey, checkpointBuilder, keyCoder);
- }
-
- private boolean checkTimersForKey(Set<TimerInternals.TimerData> originalTimers, Set<TimerInternals.TimerData> restoredTimers) {
- boolean comp = true;
- if (restoredTimers == null) {
- return false;
- }
-
- if (originalTimers.size() != restoredTimers.size()) {
- return false;
- }
-
- for (TimerInternals.TimerData timer : originalTimers) {
- comp &= restoredTimers.contains(timer);
- }
- return comp;
- }
-
- private boolean checkStateForKey(FlinkStateInternals<String> originalState, FlinkStateInternals<String> restoredState) throws CannotProvideCoderException {
- if (restoredState == null) {
- return false;
- }
-
- ValueState<String> orValue = originalState.state(NAMESPACE_1, STRING_VALUE_ADDR);
- ValueState<String> resValue = restoredState.state(NAMESPACE_1, STRING_VALUE_ADDR);
- boolean comp = orValue.read().equals(resValue.read());
-
- ValueState<Integer> orIntValue = originalState.state(NAMESPACE_1, INT_VALUE_ADDR);
- ValueState<Integer> resIntValue = restoredState.state(NAMESPACE_1, INT_VALUE_ADDR);
- comp &= orIntValue.read().equals(resIntValue.read());
-
- AccumulatorCombiningState<Integer, int[], Integer> combOrValue = originalState.state(NAMESPACE_1, SUM_INTEGER_ADDR);
- AccumulatorCombiningState<Integer, int[], Integer> combResValue = restoredState.state(NAMESPACE_1, SUM_INTEGER_ADDR);
- comp &= combOrValue.read().equals(combResValue.read());
-
- WatermarkHoldState orWatermark = originalState.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
- WatermarkHoldState resWatermark = restoredState.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
- comp &= orWatermark.read().equals(resWatermark.read());
-
- BagState<String> orBag = originalState.state(NAMESPACE_1, STRING_BAG_ADDR);
- BagState<String> resBag = restoredState.state(NAMESPACE_1, STRING_BAG_ADDR);
-
- Iterator<String> orIt = orBag.read().iterator();
- Iterator<String> resIt = resBag.read().iterator();
-
- while (orIt.hasNext() && resIt.hasNext()) {
- comp &= orIt.next().equals(resIt.next());
- }
-
- return !((orIt.hasNext() && !resIt.hasNext()) || (!orIt.hasNext() && resIt.hasNext())) && comp;
- }
-
- private FlinkStateInternals<String> createState(String key) throws CannotProvideCoderException {
- return new FlinkStateInternals<>(
- key,
- StringUtf8Coder.of(),
- IntervalWindow.getCoder(),
- OutputTimeFns.outputAtEarliestInputTimestamp());
- }
-
- @Test
- public void test() throws Exception {
- StateSerializationTest test = new StateSerializationTest();
- test.initializeStateAndTimers();
+ private static final StateNamespace NAMESPACE_1 = StateNamespaces.global();
+ private static final String KEY_PREFIX = "TEST_";
+
+ // TODO: This can be replaced with the standard Sum.SumIntererFn once the state no longer needs
+ // to create a StateTag at the point of restoring state. Currently StateTags are compared strictly
+ // by type and combiners always use KeyedCombineFnWithContext rather than KeyedCombineFn or CombineFn.
+ private static CombineWithContext.KeyedCombineFnWithContext<Object, Integer, int[], Integer> SUM_COMBINER =
+ new CombineWithContext.KeyedCombineFnWithContext<Object, Integer, int[], Integer>() {
+ @Override
+ public int[] createAccumulator(Object key, CombineWithContext.Context c) {
+ return new int[1];
+ }
+
+ @Override
+ public int[] addInput(Object key, int[] accumulator, Integer value, CombineWithContext.Context c) {
+ accumulator[0] += value;
+ return accumulator;
+ }
+
+ @Override
+ public int[] mergeAccumulators(Object key, Iterable<int[]> accumulators, CombineWithContext.Context c) {
+ int[] r = new int[1];
+ for (int[] a : accumulators) {
+ r[0] += a[0];
+ }
+ return r;
+ }
+
+ @Override
+ public Integer extractOutput(Object key, int[] accumulator, CombineWithContext.Context c) {
+ return accumulator[0];
+ }
+ };
+
+ private static Coder<int[]> INT_ACCUM_CODER = DelegateCoder.of(
+ VarIntCoder.of(),
+ new DelegateCoder.CodingFunction<int[], Integer>() {
+ @Override
+ public Integer apply(int[] accumulator) {
+ return accumulator[0];
+ }
+ },
+ new DelegateCoder.CodingFunction<Integer, int[]>() {
+ @Override
+ public int[] apply(Integer value) {
+ int[] a = new int[1];
+ a[0] = value;
+ return a;
+ }
+ });
+
+ private static final StateTag<Object, ValueState<String>> STRING_VALUE_ADDR =
+ StateTags.value("stringValue", StringUtf8Coder.of());
+ private static final StateTag<Object, ValueState<Integer>> INT_VALUE_ADDR =
+ StateTags.value("stringValue", VarIntCoder.of());
+ private static final StateTag<Object, AccumulatorCombiningState<Integer, int[], Integer>> SUM_INTEGER_ADDR =
+ StateTags.keyedCombiningValueWithContext("sumInteger", INT_ACCUM_CODER, SUM_COMBINER);
+ private static final StateTag<Object, BagState<String>> STRING_BAG_ADDR =
+ StateTags.bag("stringBag", StringUtf8Coder.of());
+ private static final StateTag<Object, WatermarkHoldState<BoundedWindow>> WATERMARK_BAG_ADDR =
+ StateTags.watermarkStateInternal("watermark", OutputTimeFns.outputAtEarliestInputTimestamp());
+
+ private Map<String, FlinkStateInternals<String>> statePerKey = new HashMap<>();
+
+ private Map<String, Set<TimerInternals.TimerData>> activeTimers = new HashMap<>();
+
+ private void initializeStateAndTimers() throws CannotProvideCoderException {
+ for (int i = 0; i < 10; i++) {
+ String key = KEY_PREFIX + i;
+
+ FlinkStateInternals state = initializeStateForKey(key);
+ Set<TimerInternals.TimerData> timers = new HashSet<>();
+ for (int j = 0; j < 5; j++) {
+ TimerInternals.TimerData timer = TimerInternals
+ .TimerData.of(NAMESPACE_1,
+ new Instant(1000 + i + j), TimeDomain.values()[j % 3]);
+ timers.add(timer);
+ }
+
+ statePerKey.put(key, state);
+ activeTimers.put(key, timers);
+ }
+ }
+
+ private FlinkStateInternals<String> initializeStateForKey(String key) throws CannotProvideCoderException {
+ FlinkStateInternals<String> state = createState(key);
+
+ ValueState<String> value = state.state(NAMESPACE_1, STRING_VALUE_ADDR);
+ value.write("test");
+
+ ValueState<Integer> value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR);
+ value2.write(4);
+ value2.write(5);
+
+ AccumulatorCombiningState<Integer, int[], Integer> combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+ combiningValue.add(1);
+ combiningValue.add(2);
+
+ WatermarkHoldState<BoundedWindow> watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
+ watermark.add(new Instant(1000));
+
+ BagState<String> bag = state.state(NAMESPACE_1, STRING_BAG_ADDR);
+ bag.add("v1");
+ bag.add("v2");
+ bag.add("v3");
+ bag.add("v4");
+ return state;
+ }
+
+ private boolean restoreAndTestState(DataInputView in) throws Exception {
+ StateCheckpointReader reader = new StateCheckpointReader(in);
+ final ClassLoader userClassloader = this.getClass().getClassLoader();
+ Coder<? extends BoundedWindow> windowCoder = IntervalWindow.getCoder();
+ Coder<String> keyCoder = StringUtf8Coder.of();
+
+ boolean comparisonRes = true;
+
+ for (String key : statePerKey.keySet()) {
+ comparisonRes &= checkStateForKey(key);
+ }
+
+ // restore the timers
+ Map<String, Set<TimerInternals.TimerData>> restoredTimersPerKey = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder);
+ if (activeTimers.size() != restoredTimersPerKey.size()) {
+ return false;
+ }
+
+ for (String key : statePerKey.keySet()) {
+ Set<TimerInternals.TimerData> originalTimers = activeTimers.get(key);
+ Set<TimerInternals.TimerData> restoredTimers = restoredTimersPerKey.get(key);
+ comparisonRes &= checkTimersForKey(originalTimers, restoredTimers);
+ }
+
+ // restore the state
+ Map<String, FlinkStateInternals<String>> restoredPerKeyState =
+ StateCheckpointUtils.decodeState(reader, OutputTimeFns.outputAtEarliestInputTimestamp(), keyCoder, windowCoder, userClassloader);
+ if (restoredPerKeyState.size() != statePerKey.size()) {
+ return false;
+ }
+
+ for (String key : statePerKey.keySet()) {
+ FlinkStateInternals<String> originalState = statePerKey.get(key);
+ FlinkStateInternals<String> restoredState = restoredPerKeyState.get(key);
+ comparisonRes &= checkStateForKey(originalState, restoredState);
+ }
+ return comparisonRes;
+ }
+
+ private boolean checkStateForKey(String key) throws CannotProvideCoderException {
+ FlinkStateInternals<String> state = statePerKey.get(key);
+
+ ValueState<String> value = state.state(NAMESPACE_1, STRING_VALUE_ADDR);
+ boolean comp = value.read().equals("test");
+
+ ValueState<Integer> value2 = state.state(NAMESPACE_1, INT_VALUE_ADDR);
+ comp &= value2.read().equals(5);
+
+ AccumulatorCombiningState<Integer, int[], Integer> combiningValue = state.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+ comp &= combiningValue.read().equals(3);
+
+ WatermarkHoldState<BoundedWindow> watermark = state.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
+ comp &= watermark.read().equals(new Instant(1000));
+
+ BagState<String> bag = state.state(NAMESPACE_1, STRING_BAG_ADDR);
+ Iterator<String> it = bag.read().iterator();
+ int i = 0;
+ while (it.hasNext()) {
+ comp &= it.next().equals("v" + (++i));
+ }
+ return comp;
+ }
+
+ private void storeState(AbstractStateBackend.CheckpointStateOutputView out) throws Exception {
+ StateCheckpointWriter checkpointBuilder = StateCheckpointWriter.create(out);
+ Coder<String> keyCoder = StringUtf8Coder.of();
+
+ // checkpoint the timers
+ StateCheckpointUtils.encodeTimers(activeTimers, checkpointBuilder, keyCoder);
+
+ // checkpoint the state
+ StateCheckpointUtils.encodeState(statePerKey, checkpointBuilder, keyCoder);
+ }
+
+ private boolean checkTimersForKey(Set<TimerInternals.TimerData> originalTimers, Set<TimerInternals.TimerData> restoredTimers) {
+ boolean comp = true;
+ if (restoredTimers == null) {
+ return false;
+ }
+
+ if (originalTimers.size() != restoredTimers.size()) {
+ return false;
+ }
+
+ for (TimerInternals.TimerData timer : originalTimers) {
+ comp &= restoredTimers.contains(timer);
+ }
+ return comp;
+ }
+
+ private boolean checkStateForKey(FlinkStateInternals<String> originalState, FlinkStateInternals<String> restoredState) throws CannotProvideCoderException {
+ if (restoredState == null) {
+ return false;
+ }
+
+ ValueState<String> orValue = originalState.state(NAMESPACE_1, STRING_VALUE_ADDR);
+ ValueState<String> resValue = restoredState.state(NAMESPACE_1, STRING_VALUE_ADDR);
+ boolean comp = orValue.read().equals(resValue.read());
+
+ ValueState<Integer> orIntValue = originalState.state(NAMESPACE_1, INT_VALUE_ADDR);
+ ValueState<Integer> resIntValue = restoredState.state(NAMESPACE_1, INT_VALUE_ADDR);
+ comp &= orIntValue.read().equals(resIntValue.read());
+
+ AccumulatorCombiningState<Integer, int[], Integer> combOrValue = originalState.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+ AccumulatorCombiningState<Integer, int[], Integer> combResValue = restoredState.state(NAMESPACE_1, SUM_INTEGER_ADDR);
+ comp &= combOrValue.read().equals(combResValue.read());
+
+ WatermarkHoldState orWatermark = originalState.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
+ WatermarkHoldState resWatermark = restoredState.state(NAMESPACE_1, WATERMARK_BAG_ADDR);
+ comp &= orWatermark.read().equals(resWatermark.read());
+
+ BagState<String> orBag = originalState.state(NAMESPACE_1, STRING_BAG_ADDR);
+ BagState<String> resBag = restoredState.state(NAMESPACE_1, STRING_BAG_ADDR);
+
+ Iterator<String> orIt = orBag.read().iterator();
+ Iterator<String> resIt = resBag.read().iterator();
+
+ while (orIt.hasNext() && resIt.hasNext()) {
+ comp &= orIt.next().equals(resIt.next());
+ }
+
+ return !((orIt.hasNext() && !resIt.hasNext()) || (!orIt.hasNext() && resIt.hasNext())) && comp;
+ }
+
+ private FlinkStateInternals<String> createState(String key) throws CannotProvideCoderException {
+ return new FlinkStateInternals<>(
+ key,
+ StringUtf8Coder.of(),
+ IntervalWindow.getCoder(),
+ OutputTimeFns.outputAtEarliestInputTimestamp());
+ }
+
+ @Test
+ public void test() throws Exception {
+ StateSerializationTest test = new StateSerializationTest();
+ test.initializeStateAndTimers();
- MemoryStateBackend.MemoryCheckpointOutputStream memBackend = new MemoryStateBackend.MemoryCheckpointOutputStream(32048);
- AbstractStateBackend.CheckpointStateOutputView out = new AbstractStateBackend.CheckpointStateOutputView(memBackend);
-
- test.storeState(out);
+ MemoryStateBackend.MemoryCheckpointOutputStream memBackend = new MemoryStateBackend.MemoryCheckpointOutputStream(32048);
+ AbstractStateBackend.CheckpointStateOutputView out = new AbstractStateBackend.CheckpointStateOutputView(memBackend);
+
+ test.storeState(out);
- byte[] contents = memBackend.closeAndGetBytes();
- DataInputView in = new DataInputDeserializer(contents, 0, contents.length);
+ byte[] contents = memBackend.closeAndGetBytes();
+ DataInputView in = new DataInputDeserializer(contents, 0, contents.length);
- assertEquals(test.restoreAndTestState(in), true);
- }
+ assertEquals(test.restoreAndTestState(in), true);
+ }
}
[06/50] [abbrv] incubator-beam git commit: [runner] add streaming
support with checkpointing
Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java
deleted file mode 100644
index c1d78c0..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/FlinkTransformTranslators.java
+++ /dev/null
@@ -1,594 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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 com.dataartisans.flink.dataflow.translation;
-
-import com.dataartisans.flink.dataflow.io.ConsoleIO;
-import com.dataartisans.flink.dataflow.translation.functions.FlinkCoGroupKeyedListAggregator;
-import com.dataartisans.flink.dataflow.translation.functions.FlinkCreateFunction;
-import com.dataartisans.flink.dataflow.translation.functions.FlinkDoFnFunction;
-import com.dataartisans.flink.dataflow.translation.functions.FlinkKeyedListAggregationFunction;
-import com.dataartisans.flink.dataflow.translation.functions.FlinkMultiOutputDoFnFunction;
-import com.dataartisans.flink.dataflow.translation.functions.FlinkMultiOutputPruningFunction;
-import com.dataartisans.flink.dataflow.translation.functions.FlinkPartialReduceFunction;
-import com.dataartisans.flink.dataflow.translation.functions.FlinkReduceFunction;
-import com.dataartisans.flink.dataflow.translation.functions.UnionCoder;
-import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation;
-import com.dataartisans.flink.dataflow.translation.types.KvCoderTypeInformation;
-import com.dataartisans.flink.dataflow.translation.wrappers.SinkOutputFormat;
-import com.dataartisans.flink.dataflow.translation.wrappers.SourceInputFormat;
-import com.google.api.client.util.Maps;
-import com.google.cloud.dataflow.sdk.coders.CannotProvideCoderException;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.io.AvroIO;
-import com.google.cloud.dataflow.sdk.io.BoundedSource;
-import com.google.cloud.dataflow.sdk.io.Read;
-import com.google.cloud.dataflow.sdk.io.TextIO;
-import com.google.cloud.dataflow.sdk.transforms.Combine;
-import com.google.cloud.dataflow.sdk.transforms.Create;
-import com.google.cloud.dataflow.sdk.transforms.DoFn;
-import com.google.cloud.dataflow.sdk.transforms.Flatten;
-import com.google.cloud.dataflow.sdk.transforms.GroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.transforms.ParDo;
-import com.google.cloud.dataflow.sdk.transforms.View;
-import com.google.cloud.dataflow.sdk.transforms.Write;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResult;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGbkResultSchema;
-import com.google.cloud.dataflow.sdk.transforms.join.CoGroupByKey;
-import com.google.cloud.dataflow.sdk.transforms.join.KeyedPCollectionTuple;
-import com.google.cloud.dataflow.sdk.transforms.join.RawUnionValue;
-import com.google.cloud.dataflow.sdk.values.KV;
-import com.google.cloud.dataflow.sdk.values.PCollection;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TupleTag;
-import com.google.common.collect.Lists;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.io.AvroInputFormat;
-import org.apache.flink.api.java.io.AvroOutputFormat;
-import org.apache.flink.api.java.io.TextInputFormat;
-import org.apache.flink.api.java.operators.CoGroupOperator;
-import org.apache.flink.api.java.operators.DataSink;
-import org.apache.flink.api.java.operators.DataSource;
-import org.apache.flink.api.java.operators.FlatMapOperator;
-import org.apache.flink.api.java.operators.GroupCombineOperator;
-import org.apache.flink.api.java.operators.GroupReduceOperator;
-import org.apache.flink.api.java.operators.Grouping;
-import org.apache.flink.api.java.operators.Keys;
-import org.apache.flink.api.java.operators.MapPartitionOperator;
-import org.apache.flink.api.java.operators.UnsortedGrouping;
-import org.apache.flink.core.fs.Path;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Translators for transforming
- * Dataflow {@link com.google.cloud.dataflow.sdk.transforms.PTransform}s to
- * Flink {@link org.apache.flink.api.java.DataSet}s
- */
-public class FlinkTransformTranslators {
-
- // --------------------------------------------------------------------------------------------
- // Transform Translator Registry
- // --------------------------------------------------------------------------------------------
-
- @SuppressWarnings("rawtypes")
- private static final Map<Class<? extends PTransform>, FlinkPipelineTranslator.TransformTranslator> TRANSLATORS = new HashMap<>();
-
- // register the known translators
- static {
- TRANSLATORS.put(View.CreatePCollectionView.class, new CreatePCollectionViewTranslator());
-
- TRANSLATORS.put(Combine.PerKey.class, new CombinePerKeyTranslator());
- // we don't need this because we translate the Combine.PerKey directly
- //TRANSLATORS.put(Combine.GroupedValues.class, new CombineGroupedValuesTranslator());
-
- TRANSLATORS.put(Create.Values.class, new CreateTranslator());
-
- TRANSLATORS.put(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator());
-
- TRANSLATORS.put(GroupByKey.GroupByKeyOnly.class, new GroupByKeyOnlyTranslator());
- // TODO we're currently ignoring windows here but that has to change in the future
- TRANSLATORS.put(GroupByKey.class, new GroupByKeyTranslator());
-
- TRANSLATORS.put(ParDo.BoundMulti.class, new ParDoBoundMultiTranslator());
- TRANSLATORS.put(ParDo.Bound.class, new ParDoBoundTranslator());
-
- TRANSLATORS.put(CoGroupByKey.class, new CoGroupByKeyTranslator());
-
- TRANSLATORS.put(AvroIO.Read.Bound.class, new AvroIOReadTranslator());
- TRANSLATORS.put(AvroIO.Write.Bound.class, new AvroIOWriteTranslator());
-
- TRANSLATORS.put(Read.Bounded.class, new ReadSourceTranslator());
- TRANSLATORS.put(Write.Bound.class, new WriteSinkTranslator());
-
- TRANSLATORS.put(TextIO.Read.Bound.class, new TextIOReadTranslator());
- TRANSLATORS.put(TextIO.Write.Bound.class, new TextIOWriteTranslator());
-
- // Flink-specific
- TRANSLATORS.put(ConsoleIO.Write.Bound.class, new ConsoleIOWriteTranslator());
-
- }
-
-
- public static FlinkPipelineTranslator.TransformTranslator<?> getTranslator(PTransform<?, ?> transform) {
- return TRANSLATORS.get(transform.getClass());
- }
-
- private static class ReadSourceTranslator<T> implements FlinkPipelineTranslator.TransformTranslator<Read.Bounded<T>> {
-
- @Override
- public void translateNode(Read.Bounded<T> transform, TranslationContext context) {
- String name = transform.getName();
- BoundedSource<T> source = transform.getSource();
- PCollection<T> output = context.getOutput(transform);
- Coder<T> coder = output.getCoder();
-
- TypeInformation<T> typeInformation = context.getTypeInfo(output);
-
- DataSource<T> dataSource = new DataSource<>(context.getExecutionEnvironment(), new SourceInputFormat<>(source, context.getPipelineOptions(), coder), typeInformation, name);
-
- context.setOutputDataSet(output, dataSource);
- }
- }
-
- private static class AvroIOReadTranslator<T> implements FlinkPipelineTranslator.TransformTranslator<AvroIO.Read.Bound<T>> {
- private static final Logger LOG = LoggerFactory.getLogger(AvroIOReadTranslator.class);
-
- @Override
- public void translateNode(AvroIO.Read.Bound<T> transform, TranslationContext context) {
- String path = transform.getFilepattern();
- String name = transform.getName();
-// Schema schema = transform.getSchema();
- PValue output = context.getOutput(transform);
-
- TypeInformation<T> typeInformation = context.getTypeInfo(output);
-
- // This is super hacky, but unfortunately we cannot get the type otherwise
- Class<T> extractedAvroType;
- try {
- Field typeField = transform.getClass().getDeclaredField("type");
- typeField.setAccessible(true);
- @SuppressWarnings("unchecked")
- Class<T> avroType = (Class<T>) typeField.get(transform);
- extractedAvroType = avroType;
- } catch (NoSuchFieldException | IllegalAccessException e) {
- // we know that the field is there and it is accessible
- throw new RuntimeException("Could not access type from AvroIO.Bound", e);
- }
-
- DataSource<T> source = new DataSource<>(context.getExecutionEnvironment(),
- new AvroInputFormat<>(new Path(path), extractedAvroType),
- typeInformation, name);
-
- context.setOutputDataSet(output, source);
- }
- }
-
- private static class AvroIOWriteTranslator<T> implements FlinkPipelineTranslator.TransformTranslator<AvroIO.Write.Bound<T>> {
- private static final Logger LOG = LoggerFactory.getLogger(AvroIOWriteTranslator.class);
-
- @Override
- public void translateNode(AvroIO.Write.Bound<T> transform, TranslationContext context) {
- DataSet<T> inputDataSet = context.getInputDataSet(context.getInput(transform));
- String filenamePrefix = transform.getFilenamePrefix();
- String filenameSuffix = transform.getFilenameSuffix();
- int numShards = transform.getNumShards();
- String shardNameTemplate = transform.getShardNameTemplate();
-
- // TODO: Implement these. We need Flink support for this.
- LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.",
- filenameSuffix);
- LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
-
- // This is super hacky, but unfortunately we cannot get the type otherwise
- Class<T> extractedAvroType;
- try {
- Field typeField = transform.getClass().getDeclaredField("type");
- typeField.setAccessible(true);
- @SuppressWarnings("unchecked")
- Class<T> avroType = (Class<T>) typeField.get(transform);
- extractedAvroType = avroType;
- } catch (NoSuchFieldException | IllegalAccessException e) {
- // we know that the field is there and it is accessible
- throw new RuntimeException("Could not access type from AvroIO.Bound", e);
- }
-
- DataSink<T> dataSink = inputDataSet.output(new AvroOutputFormat<>(new Path
- (filenamePrefix), extractedAvroType));
-
- if (numShards > 0) {
- dataSink.setParallelism(numShards);
- }
- }
- }
-
- private static class TextIOReadTranslator implements FlinkPipelineTranslator.TransformTranslator<TextIO.Read.Bound<String>> {
- private static final Logger LOG = LoggerFactory.getLogger(TextIOReadTranslator.class);
-
- @Override
- public void translateNode(TextIO.Read.Bound<String> transform, TranslationContext context) {
- String path = transform.getFilepattern();
- String name = transform.getName();
-
- TextIO.CompressionType compressionType = transform.getCompressionType();
- boolean needsValidation = transform.needsValidation();
-
- // TODO: Implement these. We need Flink support for this.
- LOG.warn("Translation of TextIO.CompressionType not yet supported. Is: {}.", compressionType);
- LOG.warn("Translation of TextIO.Read.needsValidation not yet supported. Is: {}.", needsValidation);
-
- PValue output = context.getOutput(transform);
-
- TypeInformation<String> typeInformation = context.getTypeInfo(output);
-
- DataSource<String> source = new DataSource<>(context.getExecutionEnvironment(), new TextInputFormat(new Path(path)), typeInformation, name);
-
- context.setOutputDataSet(output, source);
- }
- }
-
- private static class TextIOWriteTranslator<T> implements FlinkPipelineTranslator.TransformTranslator<TextIO.Write.Bound<T>> {
- private static final Logger LOG = LoggerFactory.getLogger(TextIOWriteTranslator.class);
-
- @Override
- public void translateNode(TextIO.Write.Bound<T> transform, TranslationContext context) {
- PValue input = context.getInput(transform);
- DataSet<T> inputDataSet = context.getInputDataSet(input);
-
- String filenamePrefix = transform.getFilenamePrefix();
- String filenameSuffix = transform.getFilenameSuffix();
- boolean needsValidation = transform.needsValidation();
- int numShards = transform.getNumShards();
- String shardNameTemplate = transform.getShardNameTemplate();
-
- // TODO: Implement these. We need Flink support for this.
- LOG.warn("Translation of TextIO.Write.needsValidation not yet supported. Is: {}.", needsValidation);
- LOG.warn("Translation of TextIO.Write.filenameSuffix not yet supported. Is: {}.", filenameSuffix);
- LOG.warn("Translation of TextIO.Write.shardNameTemplate not yet supported. Is: {}.", shardNameTemplate);
-
- //inputDataSet.print();
- DataSink<T> dataSink = inputDataSet.writeAsText(filenamePrefix);
-
- if (numShards > 0) {
- dataSink.setParallelism(numShards);
- }
- }
- }
-
- private static class ConsoleIOWriteTranslator implements FlinkPipelineTranslator.TransformTranslator<ConsoleIO.Write.Bound> {
- @Override
- public void translateNode(ConsoleIO.Write.Bound transform, TranslationContext context) {
- PValue input = context.getInput(transform);
- DataSet<?> inputDataSet = context.getInputDataSet(input);
- inputDataSet.printOnTaskManager(transform.getName());
- }
- }
-
- private static class WriteSinkTranslator<T> implements FlinkPipelineTranslator.TransformTranslator<Write.Bound<T>> {
-
- @Override
- public void translateNode(Write.Bound<T> transform, TranslationContext context) {
- String name = transform.getName();
- PValue input = context.getInput(transform);
- DataSet<T> inputDataSet = context.getInputDataSet(input);
-
- inputDataSet.output(new SinkOutputFormat<>(transform, context.getPipelineOptions())).name(name);
- }
- }
-
- private static class GroupByKeyOnlyTranslator<K, V> implements FlinkPipelineTranslator.TransformTranslator<GroupByKey.GroupByKeyOnly<K, V>> {
-
- @Override
- public void translateNode(GroupByKey.GroupByKeyOnly<K, V> transform, TranslationContext context) {
- DataSet<KV<K, V>> inputDataSet = context.getInputDataSet(context.getInput(transform));
- GroupReduceFunction<KV<K, V>, KV<K, Iterable<V>>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>();
-
- TypeInformation<KV<K, Iterable<V>>> typeInformation = context.getTypeInfo(context.getOutput(transform));
-
- Grouping<KV<K, V>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType()));
-
- GroupReduceOperator<KV<K, V>, KV<K, Iterable<V>>> outputDataSet =
- new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
- context.setOutputDataSet(context.getOutput(transform), outputDataSet);
- }
- }
-
- /**
- * Translates a GroupByKey while ignoring window assignments. This is identical to the {@link GroupByKeyOnlyTranslator}
- */
- private static class GroupByKeyTranslator<K, V> implements FlinkPipelineTranslator.TransformTranslator<GroupByKey<K, V>> {
-
- @Override
- public void translateNode(GroupByKey<K, V> transform, TranslationContext context) {
- DataSet<KV<K, V>> inputDataSet = context.getInputDataSet(context.getInput(transform));
- GroupReduceFunction<KV<K, V>, KV<K, Iterable<V>>> groupReduceFunction = new FlinkKeyedListAggregationFunction<>();
-
- TypeInformation<KV<K, Iterable<V>>> typeInformation = context.getTypeInfo(context.getOutput(transform));
-
- Grouping<KV<K, V>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet.getType()));
-
- GroupReduceOperator<KV<K, V>, KV<K, Iterable<V>>> outputDataSet =
- new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
-
- context.setOutputDataSet(context.getOutput(transform), outputDataSet);
- }
- }
-
- private static class CombinePerKeyTranslator<K, VI, VA, VO> implements FlinkPipelineTranslator.TransformTranslator<Combine.PerKey<K, VI, VO>> {
-
- @Override
- public void translateNode(Combine.PerKey<K, VI, VO> transform, TranslationContext context) {
- DataSet<KV<K, VI>> inputDataSet = context.getInputDataSet(context.getInput(transform));
-
- @SuppressWarnings("unchecked")
- Combine.KeyedCombineFn<K, VI, VA, VO> keyedCombineFn = (Combine.KeyedCombineFn<K, VI, VA, VO>) transform.getFn();
-
- KvCoder<K, VI> inputCoder = (KvCoder<K, VI>) context.getInput(transform).getCoder();
-
- Coder<VA> accumulatorCoder =
- null;
- try {
- accumulatorCoder = keyedCombineFn.getAccumulatorCoder(context.getInput(transform).getPipeline().getCoderRegistry(), inputCoder.getKeyCoder(), inputCoder.getValueCoder());
- } catch (CannotProvideCoderException e) {
- e.printStackTrace();
- // TODO
- }
-
- TypeInformation<KV<K, VI>> kvCoderTypeInformation = new KvCoderTypeInformation<>(inputCoder);
- TypeInformation<KV<K, VA>> partialReduceTypeInfo = new KvCoderTypeInformation<>(KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder));
-
- Grouping<KV<K, VI>> inputGrouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{"key"}, kvCoderTypeInformation));
-
- FlinkPartialReduceFunction<K, VI, VA> partialReduceFunction = new FlinkPartialReduceFunction<>(keyedCombineFn);
-
- // Partially GroupReduce the values into the intermediate format VA (combine)
- GroupCombineOperator<KV<K, VI>, KV<K, VA>> groupCombine =
- new GroupCombineOperator<>(inputGrouping, partialReduceTypeInfo, partialReduceFunction,
- "GroupCombine: " + transform.getName());
-
- // Reduce fully to VO
- GroupReduceFunction<KV<K, VA>, KV<K, VO>> reduceFunction = new FlinkReduceFunction<>(keyedCombineFn);
-
- TypeInformation<KV<K, VO>> reduceTypeInfo = context.getTypeInfo(context.getOutput(transform));
-
- Grouping<KV<K, VA>> intermediateGrouping = new UnsortedGrouping<>(groupCombine, new Keys.ExpressionKeys<>(new String[]{"key"}, groupCombine.getType()));
-
- // Fully reduce the values and create output format VO
- GroupReduceOperator<KV<K, VA>, KV<K, VO>> outputDataSet =
- new GroupReduceOperator<>(intermediateGrouping, reduceTypeInfo, reduceFunction, transform.getName());
-
- context.setOutputDataSet(context.getOutput(transform), outputDataSet);
- }
- }
-
-// private static class CombineGroupedValuesTranslator<K, VI, VO> implements FlinkPipelineTranslator.TransformTranslator<Combine.GroupedValues<K, VI, VO>> {
-//
-// @Override
-// public void translateNode(Combine.GroupedValues<K, VI, VO> transform, TranslationContext context) {
-// DataSet<KV<K, VI>> inputDataSet = context.getInputDataSet(transform.getInput());
-//
-// Combine.KeyedCombineFn<? super K, ? super VI, ?, VO> keyedCombineFn = transform.getFn();
-//
-// GroupReduceFunction<KV<K, VI>, KV<K, VO>> groupReduceFunction = new FlinkCombineFunction<>(keyedCombineFn);
-//
-// TypeInformation<KV<K, VO>> typeInformation = context.getTypeInfo(transform.getOutput());
-//
-// Grouping<KV<K, VI>> grouping = new UnsortedGrouping<>(inputDataSet, new Keys.ExpressionKeys<>(new String[]{""}, inputDataSet.getType()));
-//
-// GroupReduceOperator<KV<K, VI>, KV<K, VO>> outputDataSet =
-// new GroupReduceOperator<>(grouping, typeInformation, groupReduceFunction, transform.getName());
-// context.setOutputDataSet(transform.getOutput(), outputDataSet);
-// }
-// }
-
- private static class ParDoBoundTranslator<IN, OUT> implements FlinkPipelineTranslator.TransformTranslator<ParDo.Bound<IN, OUT>> {
- private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundTranslator.class);
-
- @Override
- public void translateNode(ParDo.Bound<IN, OUT> transform, TranslationContext context) {
- DataSet<IN> inputDataSet = context.getInputDataSet(context.getInput(transform));
-
- final DoFn<IN, OUT> doFn = transform.getFn();
-
- TypeInformation<OUT> typeInformation = context.getTypeInfo(context.getOutput(transform));
-
- FlinkDoFnFunction<IN, OUT> doFnWrapper = new FlinkDoFnFunction<>(doFn, context.getPipelineOptions());
- MapPartitionOperator<IN, OUT> outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName());
-
- transformSideInputs(transform.getSideInputs(), outputDataSet, context);
-
- context.setOutputDataSet(context.getOutput(transform), outputDataSet);
- }
- }
-
- private static class ParDoBoundMultiTranslator<IN, OUT> implements FlinkPipelineTranslator.TransformTranslator<ParDo.BoundMulti<IN, OUT>> {
- private static final Logger LOG = LoggerFactory.getLogger(ParDoBoundMultiTranslator.class);
-
- @Override
- public void translateNode(ParDo.BoundMulti<IN, OUT> transform, TranslationContext context) {
- DataSet<IN> inputDataSet = context.getInputDataSet(context.getInput(transform));
-
- final DoFn<IN, OUT> doFn = transform.getFn();
-
- Map<TupleTag<?>, PCollection<?>> outputs = context.getOutput(transform).getAll();
-
- Map<TupleTag<?>, Integer> outputMap = Maps.newHashMap();
- // put the main output at index 0, FlinkMultiOutputDoFnFunction also expects this
- outputMap.put(transform.getMainOutputTag(), 0);
- int count = 1;
- for (TupleTag<?> tag: outputs.keySet()) {
- if (!outputMap.containsKey(tag)) {
- outputMap.put(tag, count++);
- }
- }
-
- // collect all output Coders and create a UnionCoder for our tagged outputs
- List<Coder<?>> outputCoders = Lists.newArrayList();
- for (PCollection<?> coll: outputs.values()) {
- outputCoders.add(coll.getCoder());
- }
-
- UnionCoder unionCoder = UnionCoder.of(outputCoders);
-
- @SuppressWarnings("unchecked")
- TypeInformation<RawUnionValue> typeInformation = new CoderTypeInformation<>(unionCoder);
-
- @SuppressWarnings("unchecked")
- FlinkMultiOutputDoFnFunction<IN, OUT> doFnWrapper = new FlinkMultiOutputDoFnFunction(doFn, context.getPipelineOptions(), outputMap);
- MapPartitionOperator<IN, RawUnionValue> outputDataSet = new MapPartitionOperator<>(inputDataSet, typeInformation, doFnWrapper, transform.getName());
-
- transformSideInputs(transform.getSideInputs(), outputDataSet, context);
-
- for (Map.Entry<TupleTag<?>, PCollection<?>> output: outputs.entrySet()) {
- TypeInformation<Object> outputType = context.getTypeInfo(output.getValue());
- int outputTag = outputMap.get(output.getKey());
- FlinkMultiOutputPruningFunction<Object> pruningFunction = new FlinkMultiOutputPruningFunction<>(outputTag);
- FlatMapOperator<RawUnionValue, Object> pruningOperator = new
- FlatMapOperator<>(outputDataSet, outputType,
- pruningFunction, output.getValue().getName());
- context.setOutputDataSet(output.getValue(), pruningOperator);
-
- }
- }
- }
-
- private static class FlattenPCollectionTranslator<T> implements FlinkPipelineTranslator.TransformTranslator<Flatten.FlattenPCollectionList<T>> {
-
- @Override
- public void translateNode(Flatten.FlattenPCollectionList<T> transform, TranslationContext context) {
- List<PCollection<T>> allInputs = context.getInput(transform).getAll();
- DataSet<T> result = null;
- for(PCollection<T> collection : allInputs) {
- DataSet<T> current = context.getInputDataSet(collection);
- if (result == null) {
- result = current;
- } else {
- result = result.union(current);
- }
- }
- context.setOutputDataSet(context.getOutput(transform), result);
- }
- }
-
- private static class CreatePCollectionViewTranslator<R, T> implements FlinkPipelineTranslator.TransformTranslator<View.CreatePCollectionView<R, T>> {
- @Override
- public void translateNode(View.CreatePCollectionView<R, T> transform, TranslationContext context) {
- DataSet<T> inputDataSet = context.getInputDataSet(context.getInput(transform));
- PCollectionView<T> input = transform.apply(null);
- context.setSideInputDataSet(input, inputDataSet);
- }
- }
-
- private static class CreateTranslator<OUT> implements FlinkPipelineTranslator.TransformTranslator<Create.Values<OUT>> {
-
- @Override
- public void translateNode(Create.Values<OUT> transform, TranslationContext context) {
- TypeInformation<OUT> typeInformation = context.getOutputTypeInfo();
- Iterable<OUT> elements = transform.getElements();
-
- // we need to serialize the elements to byte arrays, since they might contain
- // elements that are not serializable by Java serialization. We deserialize them
- // in the FlatMap function using the Coder.
-
- List<byte[]> serializedElements = Lists.newArrayList();
- Coder<OUT> coder = context.getOutput(transform).getCoder();
- for (OUT element: elements) {
- ByteArrayOutputStream bao = new ByteArrayOutputStream();
- try {
- coder.encode(element, bao, Coder.Context.OUTER);
- serializedElements.add(bao.toByteArray());
- } catch (IOException e) {
- throw new RuntimeException("Could not serialize Create elements using Coder: " + e);
- }
- }
-
- DataSet<Integer> initDataSet = context.getExecutionEnvironment().fromElements(1);
- FlinkCreateFunction<Integer, OUT> flatMapFunction = new FlinkCreateFunction<>(serializedElements, coder);
- FlatMapOperator<Integer, OUT> outputDataSet = new FlatMapOperator<>(initDataSet, typeInformation, flatMapFunction, transform.getName());
-
- context.setOutputDataSet(context.getOutput(transform), outputDataSet);
- }
- }
-
- private static void transformSideInputs(List<PCollectionView<?>> sideInputs,
- MapPartitionOperator<?, ?> outputDataSet,
- TranslationContext context) {
- // get corresponding Flink broadcast DataSets
- for(PCollectionView<?> input : sideInputs) {
- DataSet<?> broadcastSet = context.getSideInputDataSet(input);
- outputDataSet.withBroadcastSet(broadcastSet, input.getTagInternal().getId());
- }
- }
-
-// Disabled because it depends on a pending pull request to the DataFlowSDK
- /**
- * Special composite transform translator. Only called if the CoGroup is two dimensional.
- * @param <K>
- */
- private static class CoGroupByKeyTranslator<K, V1, V2> implements FlinkPipelineTranslator.TransformTranslator<CoGroupByKey<K>> {
-
- @Override
- public void translateNode(CoGroupByKey<K> transform, TranslationContext context) {
- KeyedPCollectionTuple<K> input = context.getInput(transform);
-
- CoGbkResultSchema schema = input.getCoGbkResultSchema();
- List<KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?>> keyedCollections = input.getKeyedCollections();
-
- KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?> taggedCollection1 = keyedCollections.get(0);
- KeyedPCollectionTuple.TaggedKeyedPCollection<K, ?> taggedCollection2 = keyedCollections.get(1);
-
- TupleTag<?> tupleTag1 = taggedCollection1.getTupleTag();
- TupleTag<?> tupleTag2 = taggedCollection2.getTupleTag();
-
- PCollection<? extends KV<K, ?>> collection1 = taggedCollection1.getCollection();
- PCollection<? extends KV<K, ?>> collection2 = taggedCollection2.getCollection();
-
- DataSet<KV<K,V1>> inputDataSet1 = context.getInputDataSet(collection1);
- DataSet<KV<K,V2>> inputDataSet2 = context.getInputDataSet(collection2);
-
- TypeInformation<KV<K,CoGbkResult>> typeInfo = context.getOutputTypeInfo();
-
- FlinkCoGroupKeyedListAggregator<K,V1,V2> aggregator = new FlinkCoGroupKeyedListAggregator<>(schema, tupleTag1, tupleTag2);
-
- Keys.ExpressionKeys<KV<K,V1>> keySelector1 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet1.getType());
- Keys.ExpressionKeys<KV<K,V2>> keySelector2 = new Keys.ExpressionKeys<>(new String[]{"key"}, inputDataSet2.getType());
-
- DataSet<KV<K, CoGbkResult>> out = new CoGroupOperator<>(inputDataSet1, inputDataSet2,
- keySelector1, keySelector2,
- aggregator, typeInfo, null, transform.getName());
- context.setOutputDataSet(context.getOutput(transform), out);
- }
- }
-
- // --------------------------------------------------------------------------------------------
- // Miscellaneous
- // --------------------------------------------------------------------------------------------
-
- private FlinkTransformTranslators() {}
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java
deleted file mode 100644
index af46109..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/TranslationContext.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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 com.dataartisans.flink.dataflow.translation;
-
-import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation;
-import com.dataartisans.flink.dataflow.translation.types.KvCoderTypeInformation;
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import com.google.cloud.dataflow.sdk.coders.KvCoder;
-import com.google.cloud.dataflow.sdk.options.PipelineOptions;
-import com.google.cloud.dataflow.sdk.transforms.AppliedPTransform;
-import com.google.cloud.dataflow.sdk.transforms.PTransform;
-import com.google.cloud.dataflow.sdk.values.PCollectionView;
-import com.google.cloud.dataflow.sdk.values.PInput;
-import com.google.cloud.dataflow.sdk.values.POutput;
-import com.google.cloud.dataflow.sdk.values.PValue;
-import com.google.cloud.dataflow.sdk.values.TypedPValue;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.typeutils.GenericTypeInfo;
-
-import java.util.HashMap;
-import java.util.Map;
-
-public class TranslationContext {
-
- private final Map<PValue, DataSet<?>> dataSets;
- private final Map<PCollectionView<?>, DataSet<?>> broadcastDataSets;
-
- private final ExecutionEnvironment env;
- private final PipelineOptions options;
-
- private AppliedPTransform<?, ?, ?> currentTransform;
-
- // ------------------------------------------------------------------------
-
- public TranslationContext(ExecutionEnvironment env, PipelineOptions options) {
- this.env = env;
- this.options = options;
- this.dataSets = new HashMap<>();
- this.broadcastDataSets = new HashMap<>();
- }
-
- // ------------------------------------------------------------------------
-
- public ExecutionEnvironment getExecutionEnvironment() {
- return env;
- }
-
- public PipelineOptions getPipelineOptions() {
- return options;
- }
-
- @SuppressWarnings("unchecked")
- public <T> DataSet<T> getInputDataSet(PValue value) {
- return (DataSet<T>) dataSets.get(value);
- }
-
- public void setOutputDataSet(PValue value, DataSet<?> set) {
- if (!dataSets.containsKey(value)) {
- dataSets.put(value, set);
- }
- }
-
- /**
- * Sets the AppliedPTransform which carries input/output.
- * @param currentTransform
- */
- public void setCurrentTransform(AppliedPTransform<?, ?, ?> currentTransform) {
- this.currentTransform = currentTransform;
- }
-
- @SuppressWarnings("unchecked")
- public <T> DataSet<T> getSideInputDataSet(PCollectionView<?> value) {
- return (DataSet<T>) broadcastDataSets.get(value);
- }
-
- public void setSideInputDataSet(PCollectionView<?> value, DataSet<?> set) {
- if (!broadcastDataSets.containsKey(value)) {
- broadcastDataSets.put(value, set);
- }
- }
-
- @SuppressWarnings("unchecked")
- public <T> TypeInformation<T> getTypeInfo(PInput output) {
- if (output instanceof TypedPValue) {
- Coder<?> outputCoder = ((TypedPValue) output).getCoder();
- if (outputCoder instanceof KvCoder) {
- return new KvCoderTypeInformation((KvCoder) outputCoder);
- } else {
- return new CoderTypeInformation(outputCoder);
- }
- }
- return new GenericTypeInfo<>((Class<T>)Object.class);
- }
-
- public <T> TypeInformation<T> getInputTypeInfo() {
- return getTypeInfo(currentTransform.getInput());
- }
-
- public <T> TypeInformation<T> getOutputTypeInfo() {
- return getTypeInfo((PValue) currentTransform.getOutput());
- }
-
- @SuppressWarnings("unchecked")
- <I extends PInput> I getInput(PTransform<I, ?> transform) {
- return (I) currentTransform.getInput();
- }
-
- @SuppressWarnings("unchecked")
- <O extends POutput> O getOutput(PTransform<?, O> transform) {
- return (O) currentTransform.getOutput();
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java
new file mode 100644
index 0000000..e433589
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComparator.java
@@ -0,0 +1,216 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation.types;
+
+import com.google.cloud.dataflow.sdk.coders.Coder;
+import org.apache.flink.api.common.typeutils.TypeComparator;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+
+/**
+ * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for
+ * {@link com.google.cloud.dataflow.sdk.coders.Coder}.
+ */
+public class CoderComparator<T> extends TypeComparator<T> {
+
+ private Coder<T> coder;
+
+ // We use these for internal encoding/decoding for creating copies and comparing
+ // serialized forms using a Coder
+ private transient InspectableByteArrayOutputStream buffer1;
+ private transient InspectableByteArrayOutputStream buffer2;
+
+ // For storing the Reference in encoded form
+ private transient InspectableByteArrayOutputStream referenceBuffer;
+
+ public CoderComparator(Coder<T> coder) {
+ this.coder = coder;
+ buffer1 = new InspectableByteArrayOutputStream();
+ buffer2 = new InspectableByteArrayOutputStream();
+ referenceBuffer = new InspectableByteArrayOutputStream();
+ }
+
+ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ buffer1 = new InspectableByteArrayOutputStream();
+ buffer2 = new InspectableByteArrayOutputStream();
+ referenceBuffer = new InspectableByteArrayOutputStream();
+ }
+
+ @Override
+ public int hash(T record) {
+ return record.hashCode();
+ }
+
+ @Override
+ public void setReference(T toCompare) {
+ referenceBuffer.reset();
+ try {
+ coder.encode(toCompare, referenceBuffer, Coder.Context.OUTER);
+ } catch (IOException e) {
+ throw new RuntimeException("Could not set reference " + toCompare + ": " + e);
+ }
+ }
+
+ @Override
+ public boolean equalToReference(T candidate) {
+ try {
+ buffer2.reset();
+ coder.encode(candidate, buffer2, Coder.Context.OUTER);
+ byte[] arr = referenceBuffer.getBuffer();
+ byte[] arrOther = buffer2.getBuffer();
+ if (referenceBuffer.size() != buffer2.size()) {
+ return false;
+ }
+ int len = buffer2.size();
+ for(int i = 0; i < len; i++ ) {
+ if (arr[i] != arrOther[i]) {
+ return false;
+ }
+ }
+ return true;
+ } catch (IOException e) {
+ throw new RuntimeException("Could not compare reference.", e);
+ }
+ }
+
+ @Override
+ public int compareToReference(TypeComparator<T> other) {
+ InspectableByteArrayOutputStream otherReferenceBuffer = ((CoderComparator<T>) other).referenceBuffer;
+
+ byte[] arr = referenceBuffer.getBuffer();
+ byte[] arrOther = otherReferenceBuffer.getBuffer();
+ if (referenceBuffer.size() != otherReferenceBuffer.size()) {
+ return referenceBuffer.size() - otherReferenceBuffer.size();
+ }
+ int len = referenceBuffer.size();
+ for (int i = 0; i < len; i++) {
+ if (arr[i] != arrOther[i]) {
+ return arr[i] - arrOther[i];
+ }
+ }
+ return 0;
+ }
+
+ @Override
+ public int compare(T first, T second) {
+ try {
+ buffer1.reset();
+ buffer2.reset();
+ coder.encode(first, buffer1, Coder.Context.OUTER);
+ coder.encode(second, buffer2, Coder.Context.OUTER);
+ byte[] arr = buffer1.getBuffer();
+ byte[] arrOther = buffer2.getBuffer();
+ if (buffer1.size() != buffer2.size()) {
+ return buffer1.size() - buffer2.size();
+ }
+ int len = buffer1.size();
+ for(int i = 0; i < len; i++ ) {
+ if (arr[i] != arrOther[i]) {
+ return arr[i] - arrOther[i];
+ }
+ }
+ return 0;
+ } catch (IOException e) {
+ throw new RuntimeException("Could not compare: ", e);
+ }
+ }
+
+ @Override
+ public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
+ CoderTypeSerializer<T> serializer = new CoderTypeSerializer<>(coder);
+ T first = serializer.deserialize(firstSource);
+ T second = serializer.deserialize(secondSource);
+ return compare(first, second);
+ }
+
+ @Override
+ public boolean supportsNormalizedKey() {
+ return true;
+ }
+
+ @Override
+ public boolean supportsSerializationWithKeyNormalization() {
+ return false;
+ }
+
+ @Override
+ public int getNormalizeKeyLen() {
+ return Integer.MAX_VALUE;
+ }
+
+ @Override
+ public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
+ return true;
+ }
+
+ @Override
+ public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) {
+ buffer1.reset();
+ try {
+ coder.encode(record, buffer1, Coder.Context.OUTER);
+ } catch (IOException e) {
+ throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e);
+ }
+ final byte[] data = buffer1.getBuffer();
+ final int limit = offset + numBytes;
+
+ target.put(offset, data, 0, Math.min(numBytes, buffer1.size()));
+
+ offset += buffer1.size();
+
+ while (offset < limit) {
+ target.put(offset++, (byte) 0);
+ }
+ }
+
+ @Override
+ public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ public boolean invertNormalizedKey() {
+ return false;
+ }
+
+ @Override
+ public TypeComparator<T> duplicate() {
+ return new CoderComparator<>(coder);
+ }
+
+ @Override
+ public int extractKeys(Object record, Object[] target, int index) {
+ target[index] = record;
+ return 1;
+ }
+
+ @Override
+ public TypeComparator[] getFlatComparators() {
+ return new TypeComparator[] { this.duplicate() };
+ }
+}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java
deleted file mode 100644
index ade826d..0000000
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderComperator.java
+++ /dev/null
@@ -1,218 +0,0 @@
-/*
- * Copyright 2015 Data Artisans GmbH
- *
- * 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://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 com.dataartisans.flink.dataflow.translation.types;
-
-import com.google.cloud.dataflow.sdk.coders.Coder;
-import org.apache.flink.api.common.typeutils.TypeComparator;
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.MemorySegment;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-
-/**
- * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for
- * {@link com.google.cloud.dataflow.sdk.coders.Coder}.
- */
-public class CoderComperator<T> extends TypeComparator<T> {
-
- private Coder<T> coder;
-
- // We use these for internal encoding/decoding for creating copies and comparing
- // serialized forms using a Coder
- private transient InspectableByteArrayOutputStream buffer1;
- private transient InspectableByteArrayOutputStream buffer2;
-
- // For storing the Reference in encoded form
- private transient InspectableByteArrayOutputStream referenceBuffer;
-
- public CoderComperator(Coder<T> coder) {
- this.coder = coder;
- buffer1 = new InspectableByteArrayOutputStream();
- buffer2 = new InspectableByteArrayOutputStream();
- referenceBuffer = new InspectableByteArrayOutputStream();
- }
-
- private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
- in.defaultReadObject();
-
- buffer1 = new InspectableByteArrayOutputStream();
- buffer2 = new InspectableByteArrayOutputStream();
- referenceBuffer = new InspectableByteArrayOutputStream();
-
- }
-
- @Override
- public int hash(T record) {
- return record.hashCode();
- }
-
- @Override
- public void setReference(T toCompare) {
- referenceBuffer.reset();
- try {
- coder.encode(toCompare, referenceBuffer, Coder.Context.OUTER);
- } catch (IOException e) {
- throw new RuntimeException("Could not set reference " + toCompare + ": " + e);
- }
- }
-
- @Override
- public boolean equalToReference(T candidate) {
- try {
- buffer2.reset();
- coder.encode(candidate, buffer2, Coder.Context.OUTER);
- byte[] arr = referenceBuffer.getBuffer();
- byte[] arrOther = buffer2.getBuffer();
- if (referenceBuffer.size() != buffer2.size()) {
- return false;
- }
- int len = buffer2.size();
- for(int i = 0; i < len; i++ ) {
- if (arr[i] != arrOther[i]) {
- return false;
- }
- }
- return true;
- } catch (IOException e) {
- throw new RuntimeException("Could not compare reference.", e);
- }
- }
-
- @Override
- public int compareToReference(TypeComparator<T> other) {
- InspectableByteArrayOutputStream otherReferenceBuffer = ((CoderComperator<T>) other).referenceBuffer;
-
- byte[] arr = referenceBuffer.getBuffer();
- byte[] arrOther = otherReferenceBuffer.getBuffer();
- if (referenceBuffer.size() != otherReferenceBuffer.size()) {
- return referenceBuffer.size() - otherReferenceBuffer.size();
- }
- int len = referenceBuffer.size();
- for (int i = 0; i < len; i++) {
- if (arr[i] != arrOther[i]) {
- return arr[i] - arrOther[i];
- }
- }
- return 0;
- }
-
- @Override
- public int compare(T first, T second) {
- try {
- buffer1.reset();
- buffer2.reset();
- coder.encode(first, buffer1, Coder.Context.OUTER);
- coder.encode(second, buffer2, Coder.Context.OUTER);
- byte[] arr = buffer1.getBuffer();
- byte[] arrOther = buffer2.getBuffer();
- if (buffer1.size() != buffer2.size()) {
- return buffer1.size() - buffer2.size();
- }
- int len = buffer1.size();
- for(int i = 0; i < len; i++ ) {
- if (arr[i] != arrOther[i]) {
- return arr[i] - arrOther[i];
- }
- }
- return 0;
- } catch (IOException e) {
- throw new RuntimeException("Could not compare: ", e);
- }
- }
-
- @Override
- public int compareSerialized(DataInputView firstSource, DataInputView secondSource) throws IOException {
- CoderTypeSerializer<T> serializer = new CoderTypeSerializer<>(coder);
- T first = serializer.deserialize(firstSource);
- T second = serializer.deserialize(secondSource);
- return compare(first, second);
- }
-
- @Override
- public boolean supportsNormalizedKey() {
- return true;
- }
-
- @Override
- public boolean supportsSerializationWithKeyNormalization() {
- return false;
- }
-
- @Override
- public int getNormalizeKeyLen() {
- return Integer.MAX_VALUE;
- }
-
- @Override
- public boolean isNormalizedKeyPrefixOnly(int keyBytes) {
- return true;
- }
-
- @Override
- public void putNormalizedKey(T record, MemorySegment target, int offset, int numBytes) {
- buffer1.reset();
- try {
- coder.encode(record, buffer1, Coder.Context.OUTER);
- } catch (IOException e) {
- throw new RuntimeException("Could not serializer " + record + " using coder " + coder + ": " + e);
- }
- final byte[] data = buffer1.getBuffer();
- final int limit = offset + numBytes;
-
- target.put(offset, data, 0, Math.min(numBytes, buffer1.size()));
-
- offset += buffer1.size();
-
- while (offset < limit) {
- target.put(offset++, (byte) 0);
- }
- }
-
- @Override
- public void writeWithKeyNormalization(T record, DataOutputView target) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public T readWithKeyDenormalization(T reuse, DataInputView source) throws IOException {
- throw new UnsupportedOperationException();
- }
-
- @Override
- public boolean invertNormalizedKey() {
- return false;
- }
-
- @Override
- public TypeComparator<T> duplicate() {
- return new CoderComperator<>(coder);
- }
-
- @Override
- public int extractKeys(Object record, Object[] target, int index) {
- target[index] = record;
- return 1;
- }
-
- @Override
- public TypeComparator[] getFlatComparators() {
- return new TypeComparator[] { this.duplicate() };
- }
-}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java
index 56192cd..80e451a 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeInformation.java
@@ -32,12 +32,12 @@ import org.apache.flink.shaded.com.google.common.base.Preconditions;
*/
public class CoderTypeInformation<T> extends TypeInformation<T> implements AtomicType<T> {
- private Coder<T> coder;
+ private final Coder<T> coder;
@SuppressWarnings("unchecked")
public CoderTypeInformation(Coder<T> coder) {
- this.coder = coder;
Preconditions.checkNotNull(coder);
+ this.coder = coder;
}
@Override
@@ -112,6 +112,6 @@ public class CoderTypeInformation<T> extends TypeInformation<T> implements Atomi
@Override
public TypeComparator<T> createComparator(boolean sortOrderAscending, ExecutionConfig
executionConfig) {
- return new CoderComperator<>(coder);
+ return new CoderComparator<>(coder);
}
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java
index 9715477..f739397 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/CoderTypeSerializer.java
@@ -137,9 +137,7 @@ public class CoderTypeSerializer<T> extends TypeSerializer<T> {
if (o == null || getClass() != o.getClass()) return false;
CoderTypeSerializer that = (CoderTypeSerializer) o;
-
return coder.equals(that.coder);
-
}
@Override
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java
index 940dba6..815569d 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/KvCoderComperator.java
@@ -259,6 +259,6 @@ public class KvCoderComperator <K, V> extends TypeComparator<KV<K, V>> {
@Override
public TypeComparator[] getFlatComparators() {
- return new TypeComparator[] {new CoderComperator<>(keyCoder)};
+ return new TypeComparator[] {new CoderComparator<>(keyCoder)};
}
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java
index 2096e27..7ce484a 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/types/VoidCoderTypeSerializer.java
@@ -109,5 +109,4 @@ public class VoidCoderTypeSerializer extends TypeSerializer<VoidCoderTypeSeriali
public static VoidValue INSTANCE = new VoidValue();
}
-
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
index 8c9c59c..afb15da 100644
--- a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/SourceInputFormat.java
@@ -18,7 +18,6 @@
package com.dataartisans.flink.dataflow.translation.wrappers;
import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.api.client.util.Lists;
import com.google.cloud.dataflow.sdk.coders.Coder;
import com.google.cloud.dataflow.sdk.io.BoundedSource;
import com.google.cloud.dataflow.sdk.io.Source;
@@ -34,6 +33,7 @@ import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.io.ObjectInputStream;
import java.io.ObjectOutputStream;
+import java.util.ArrayList;
import java.util.List;
/**
@@ -116,7 +116,7 @@ public class SourceInputFormat<T> implements InputFormat<T, SourceInputSplit<T>>
desiredSizeBytes = initialSource.getEstimatedSizeBytes(options) / numSplits;
List<? extends Source<T>> shards = initialSource.splitIntoBundles(desiredSizeBytes,
options);
- List<SourceInputSplit<T>> splits = Lists.newArrayList();
+ List<SourceInputSplit<T>> splits = new ArrayList<SourceInputSplit<T>>();
int splitCount = 0;
for (Source<T> shard: shards) {
splits.add(new SourceInputSplit<>(shard, splitCount++));
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
new file mode 100644
index 0000000..53bb177
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java
@@ -0,0 +1,274 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation.wrappers.streaming;
+
+import com.dataartisans.flink.dataflow.translation.wrappers.SerializableFnAggregatorWrapper;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
+import com.google.cloud.dataflow.sdk.transforms.windowing.WindowFn;
+import com.google.cloud.dataflow.sdk.util.*;
+import com.google.cloud.dataflow.sdk.values.PCollectionView;
+import com.google.cloud.dataflow.sdk.values.TupleTag;
+import com.google.common.base.Throwables;
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.accumulators.AccumulatorHelper;
+import org.apache.flink.api.common.functions.RichFlatMapFunction;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+import org.joda.time.format.PeriodFormat;
+
+import java.util.Collection;
+
+public abstract class FlinkAbstractParDoWrapper<IN, OUTDF, OUTFL> extends RichFlatMapFunction<WindowedValue<IN>, WindowedValue<OUTFL>> {
+
+ private final DoFn<IN, OUTDF> doFn;
+ private final WindowingStrategy<?, ?> windowingStrategy;
+ private transient PipelineOptions options;
+
+ private DoFnProcessContext context;
+
+ public FlinkAbstractParDoWrapper(PipelineOptions options, WindowingStrategy<?, ?> windowingStrategy, DoFn<IN, OUTDF> doFn) {
+ Preconditions.checkNotNull(options);
+ Preconditions.checkNotNull(windowingStrategy);
+ Preconditions.checkNotNull(doFn);
+
+ this.doFn = doFn;
+ this.options = options;
+ this.windowingStrategy = windowingStrategy;
+ }
+
+// protected void writeObject(ObjectOutputStream out)
+// throws IOException, ClassNotFoundException {
+// out.defaultWriteObject();
+// ObjectMapper mapper = new ObjectMapper();
+// mapper.writeValue(out, options);
+// }
+//
+// protected void readObject(ObjectInputStream in)
+// throws IOException, ClassNotFoundException {
+// in.defaultReadObject();
+// ObjectMapper mapper = new ObjectMapper();
+// options = mapper.readValue(in, PipelineOptions.class);
+// }
+
+ private void initContext(DoFn<IN, OUTDF> function, Collector<WindowedValue<OUTFL>> outCollector) {
+ if (this.context == null) {
+ this.context = new DoFnProcessContext(function, outCollector);
+ }
+ }
+
+ @Override
+ public void flatMap(WindowedValue<IN> value, Collector<WindowedValue<OUTFL>> out) throws Exception {
+ this.initContext(doFn, out);
+
+ // for each window the element belongs to, create a new copy here.
+ Collection<? extends BoundedWindow> windows = value.getWindows();
+ if (windows.size() <= 1) {
+ processElement(value);
+ } else {
+ for (BoundedWindow window : windows) {
+ processElement(WindowedValue.of(
+ value.getValue(), value.getTimestamp(), window, value.getPane()));
+ }
+ }
+ }
+
+ private void processElement(WindowedValue<IN> value) throws Exception {
+ this.context.setElement(value);
+ this.doFn.startBundle(context);
+ doFn.processElement(context);
+ this.doFn.finishBundle(context);
+ }
+
+ private class DoFnProcessContext extends DoFn<IN, OUTDF>.ProcessContext {
+
+ private final DoFn<IN, OUTDF> fn;
+
+ protected final Collector<WindowedValue<OUTFL>> collector;
+
+ private WindowedValue<IN> element;
+
+ private DoFnProcessContext(DoFn<IN, OUTDF> function, Collector<WindowedValue<OUTFL>> outCollector) {
+ function.super();
+ super.setupDelegateAggregators();
+
+ this.fn = function;
+ this.collector = outCollector;
+ }
+
+ public void setElement(WindowedValue<IN> value) {
+ this.element = value;
+ }
+
+ @Override
+ public IN element() {
+ return this.element.getValue();
+ }
+
+ @Override
+ public Instant timestamp() {
+ return this.element.getTimestamp();
+ }
+
+ @Override
+ public BoundedWindow window() {
+// if (!(fn instanceof DoFn.RequiresWindowAccess)) {
+// throw new UnsupportedOperationException(
+// "window() is only available in the context of a DoFn marked as RequiresWindow.");
+// }
+
+ Collection<? extends BoundedWindow> windows = this.element.getWindows();
+ if (windows.size() != 1) {
+ throw new IllegalArgumentException("Each element is expected to belong to 1 window. " +
+ "This belongs to " + windows.size() + ".");
+ }
+ return windows.iterator().next();
+ }
+
+ @Override
+ public PaneInfo pane() {
+ return this.element.getPane();
+ }
+
+ @Override
+ public WindowingInternals<IN, OUTDF> windowingInternals() {
+ return windowingInternalsHelper(element, collector);
+ }
+
+ @Override
+ public PipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view) {
+ throw new RuntimeException("sideInput() is not supported in Streaming mode.");
+ }
+
+ @Override
+ public void output(OUTDF output) {
+ outputWithTimestamp(output, this.element.getTimestamp());
+ }
+
+ @Override
+ public void outputWithTimestamp(OUTDF output, Instant timestamp) {
+ outputWithTimestampHelper(element, output, timestamp, collector);
+ }
+
+ @Override
+ public <T> void sideOutput(TupleTag<T> tag, T output) {
+ sideOutputWithTimestamp(tag, output, this.element.getTimestamp());
+ }
+
+ @Override
+ public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+ sideOutputWithTimestampHelper(element, output, timestamp, collector, tag);
+ }
+
+ @Override
+ protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
+ Accumulator acc = getRuntimeContext().getAccumulator(name);
+ if (acc != null) {
+ AccumulatorHelper.compareAccumulatorTypes(name,
+ SerializableFnAggregatorWrapper.class, acc.getClass());
+ return (Aggregator<AggInputT, AggOutputT>) acc;
+ }
+
+ SerializableFnAggregatorWrapper<AggInputT, AggOutputT> accumulator =
+ new SerializableFnAggregatorWrapper<>(combiner);
+ getRuntimeContext().addAccumulator(name, accumulator);
+ return accumulator;
+ }
+ }
+
+ protected void checkTimestamp(WindowedValue<IN> ref, Instant timestamp) {
+ if (timestamp.isBefore(ref.getTimestamp().minus(doFn.getAllowedTimestampSkew()))) {
+ throw new IllegalArgumentException(String.format(
+ "Cannot output with timestamp %s. Output timestamps must be no earlier than the "
+ + "timestamp of the current input (%s) minus the allowed skew (%s). See the "
+ + "DoFn#getAllowedTimestmapSkew() Javadoc for details on changing the allowed skew.",
+ timestamp, ref.getTimestamp(),
+ PeriodFormat.getDefault().print(doFn.getAllowedTimestampSkew().toPeriod())));
+ }
+ }
+
+ protected <T> WindowedValue<T> makeWindowedValue(
+ T output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+ final Instant inputTimestamp = timestamp;
+ final WindowFn windowFn = windowingStrategy.getWindowFn();
+
+ if (timestamp == null) {
+ timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
+ }
+
+ if (windows == null) {
+ try {
+ windows = windowFn.assignWindows(windowFn.new AssignContext() {
+ @Override
+ public Object element() {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input element when none was available"); // TODO: 12/16/15 aljoscha's comment in slack
+ }
+
+ @Override
+ public Instant timestamp() {
+ if (inputTimestamp == null) {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input timestamp when none was available");
+ }
+ return inputTimestamp;
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ throw new UnsupportedOperationException(
+ "WindowFn attempted to access input windows when none were available");
+ }
+ });
+ } catch (Exception e) {
+ Throwables.propagateIfInstanceOf(e, UserCodeException.class);
+ throw new UserCodeException(e);
+ }
+ }
+
+ return WindowedValue.of(output, timestamp, windows, pane);
+ }
+
+ /////////// ABSTRACT METHODS TO BE IMPLEMENTED BY SUBCLASSES /////////////////
+
+ public abstract void outputWithTimestampHelper(
+ WindowedValue<IN> inElement,
+ OUTDF output,
+ Instant timestamp,
+ Collector<WindowedValue<OUTFL>> outCollector);
+
+ public abstract <T> void sideOutputWithTimestampHelper(
+ WindowedValue<IN> inElement,
+ T output,
+ Instant timestamp,
+ Collector<WindowedValue<OUTFL>> outCollector,
+ TupleTag<T> tag);
+
+ public abstract WindowingInternals<IN, OUTDF> windowingInternalsHelper(
+ WindowedValue<IN> inElement,
+ Collector<WindowedValue<OUTFL>> outCollector);
+
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/edff0785/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
----------------------------------------------------------------------
diff --git a/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
new file mode 100644
index 0000000..c52fabe
--- /dev/null
+++ b/runners/flink/src/main/java/com/dataartisans/flink/dataflow/translation/wrappers/streaming/FlinkGroupAlsoByWindowWrapper.java
@@ -0,0 +1,601 @@
+/*
+ * Copyright 2015 Data Artisans GmbH
+ *
+ * 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://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 com.dataartisans.flink.dataflow.translation.wrappers.streaming;
+
+import com.dataartisans.flink.dataflow.translation.types.CoderTypeInformation;
+import com.dataartisans.flink.dataflow.translation.wrappers.SerializableFnAggregatorWrapper;
+import com.dataartisans.flink.dataflow.translation.wrappers.streaming.state.*;
+import com.google.cloud.dataflow.sdk.coders.*;
+import com.google.cloud.dataflow.sdk.options.PipelineOptions;
+import com.google.cloud.dataflow.sdk.repackaged.com.google.common.base.Preconditions;
+import com.google.cloud.dataflow.sdk.transforms.Aggregator;
+import com.google.cloud.dataflow.sdk.transforms.Combine;
+import com.google.cloud.dataflow.sdk.transforms.DoFn;
+import com.google.cloud.dataflow.sdk.transforms.windowing.BoundedWindow;
+import com.google.cloud.dataflow.sdk.transforms.windowing.PaneInfo;
+import com.google.cloud.dataflow.sdk.util.*;
+import com.google.cloud.dataflow.sdk.values.*;
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.accumulators.AccumulatorHelper;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.runtime.state.StateBackend;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.KeyedStream;
+import org.apache.flink.streaming.api.operators.*;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
+import org.apache.flink.util.Collector;
+import org.joda.time.Instant;
+
+import java.io.IOException;
+import java.util.*;
+
+/**
+ * This class is the key class implementing all the windowing/triggering logic of Google Dataflow.
+ * To provide full compatibility and support all the windowing/triggering combinations offered by
+ * Datadlow, we opted for a strategy that uses the SDK's code for doing these operations
+ * ({@link com.google.cloud.dataflow.sdk.util.StreamingGroupAlsoByWindowsDoFn}.
+ * <p>
+ * In a nutshell, when the execution arrives to this operator, we expect to have a stream <b>already
+ * grouped by key</b>. Each of the elements that enter here, registers a timer
+ * (see {@link TimerInternals#setTimer(TimerInternals.TimerData)} in the
+ * {@link FlinkGroupAlsoByWindowWrapper#activeTimers}.
+ * This is essentially a timestamp indicating when to trigger the computation over the window this
+ * element belongs to.
+ * <p>
+ * When a watermark arrives, all the registered timers are checked to see which ones are ready to
+ * fire (see {@link FlinkGroupAlsoByWindowWrapper#processWatermark(Watermark)}). These are deregistered from
+ * the {@link FlinkGroupAlsoByWindowWrapper#activeTimers}
+ * list, and are fed into the {@link com.google.cloud.dataflow.sdk.util.StreamingGroupAlsoByWindowsDoFn}
+ * for furhter processing.
+ */
+public class FlinkGroupAlsoByWindowWrapper<K, VIN, VACC, VOUT>
+ extends AbstractStreamOperator<WindowedValue<KV<K, VOUT>>>
+ implements OneInputStreamOperator<WindowedValue<KV<K, VIN>>, WindowedValue<KV<K, VOUT>>> {
+
+ private static final long serialVersionUID = 1L;
+
+ private transient PipelineOptions options;
+
+ private transient CoderRegistry coderRegistry;
+
+ private StreamingGroupAlsoByWindowsDoFn operator;
+
+ private ProcessContext context;
+
+ private final WindowingStrategy<?, ?> windowingStrategy;
+
+ private final Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combineFn;
+
+ private final KvCoder<K, VIN> inputKvCoder;
+
+ /**
+ * State is kept <b>per-key</b>. This data structure keeps this mapping between an active key, i.e. a
+ * key whose elements are currently waiting to be processed, and its associated state.
+ */
+ private Map<K, FlinkStateInternals<K>> perKeyStateInternals = new HashMap<>();
+
+ /**
+ * Timers waiting to be processed.
+ */
+ private Map<K, Set<TimerInternals.TimerData>> activeTimers = new HashMap<>();
+
+ private FlinkTimerInternals timerInternals = new FlinkTimerInternals();
+
+ /**
+ * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy.
+ * This method assumes that <b>elements are already grouped by key</b>.
+ * <p>
+ * The difference with {@link #createForIterable(PipelineOptions, PCollection, KeyedStream)}
+ * is that this method assumes that a combiner function is provided
+ * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}).
+ * A combiner helps at increasing the speed and, in most of the cases, reduce the per-window state.
+ *
+ * @param options the general job configuration options.
+ * @param input the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}.
+ * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key.
+ * @param combiner the combiner to be used.
+ * @param outputKvCoder the type of the output values.
+ */
+ public static <K, VIN, VACC, VOUT> DataStream<WindowedValue<KV<K, VOUT>>> create(
+ PipelineOptions options,
+ PCollection input,
+ KeyedStream<WindowedValue<KV<K, VIN>>, K> groupedStreamByKey,
+ Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner,
+ KvCoder<K, VOUT> outputKvCoder) {
+
+ KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) input.getCoder();
+ FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper<>(options,
+ input.getPipeline().getCoderRegistry(), input.getWindowingStrategy(), inputKvCoder, combiner);
+
+ Coder<WindowedValue<KV<K, VOUT>>> windowedOutputElemCoder = WindowedValue.FullWindowedValueCoder.of(
+ outputKvCoder,
+ input.getWindowingStrategy().getWindowFn().windowCoder());
+
+ CoderTypeInformation<WindowedValue<KV<K, VOUT>>> outputTypeInfo =
+ new CoderTypeInformation<>(windowedOutputElemCoder);
+
+ DataStream<WindowedValue<KV<K, VOUT>>> groupedByKeyAndWindow = groupedStreamByKey
+ .transform("GroupByWindowWithCombiner",
+ new CoderTypeInformation<>(outputKvCoder),
+ windower)
+ .returns(outputTypeInfo);
+
+ return groupedByKeyAndWindow;
+ }
+
+ /**
+ * Creates an DataStream where elements are grouped in windows based on the specified windowing strategy.
+ * This method assumes that <b>elements are already grouped by key</b>.
+ * <p>
+ * The difference with {@link #create(PipelineOptions, PCollection, KeyedStream, Combine.KeyedCombineFn, KvCoder)}
+ * is that this method assumes no combiner function
+ * (see {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn}).
+ *
+ * @param options the general job configuration options.
+ * @param input the input Dataflow {@link com.google.cloud.dataflow.sdk.values.PCollection}.
+ * @param groupedStreamByKey the input stream, it is assumed to already be grouped by key.
+ */
+ public static <K, VIN> DataStream<WindowedValue<KV<K, Iterable<VIN>>>> createForIterable(
+ PipelineOptions options,
+ PCollection input,
+ KeyedStream<WindowedValue<KV<K, VIN>>, K> groupedStreamByKey) {
+
+ KvCoder<K, VIN> inputKvCoder = (KvCoder<K, VIN>) input.getCoder();
+ Coder<K> keyCoder = inputKvCoder.getKeyCoder();
+ Coder<VIN> inputValueCoder = inputKvCoder.getValueCoder();
+
+ FlinkGroupAlsoByWindowWrapper windower = new FlinkGroupAlsoByWindowWrapper(options,
+ input.getPipeline().getCoderRegistry(), input.getWindowingStrategy(), inputKvCoder, null);
+
+ Coder<Iterable<VIN>> valueIterCoder = IterableCoder.of(inputValueCoder);
+ KvCoder<K, Iterable<VIN>> outputElemCoder = KvCoder.of(keyCoder, valueIterCoder);
+
+ Coder<WindowedValue<KV<K, Iterable<VIN>>>> windowedOutputElemCoder = WindowedValue.FullWindowedValueCoder.of(
+ outputElemCoder,
+ input.getWindowingStrategy().getWindowFn().windowCoder());
+
+ CoderTypeInformation<WindowedValue<KV<K, Iterable<VIN>>>> outputTypeInfo =
+ new CoderTypeInformation<>(windowedOutputElemCoder);
+
+ DataStream<WindowedValue<KV<K, Iterable<VIN>>>> groupedByKeyAndWindow = groupedStreamByKey
+ .transform("GroupByWindow",
+ new CoderTypeInformation<>(windowedOutputElemCoder),
+ windower)
+ .returns(outputTypeInfo);
+
+ return groupedByKeyAndWindow;
+ }
+
+ public static <K, VIN, VACC, VOUT> FlinkGroupAlsoByWindowWrapper createForTesting(PipelineOptions options,
+ CoderRegistry registry,
+ WindowingStrategy<?, ?> windowingStrategy,
+ KvCoder<K, VIN> inputCoder,
+ Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner) {
+ return new FlinkGroupAlsoByWindowWrapper(options, registry, windowingStrategy, inputCoder, combiner);
+ }
+
+ private FlinkGroupAlsoByWindowWrapper(PipelineOptions options,
+ CoderRegistry registry,
+ WindowingStrategy<?, ?> windowingStrategy,
+ KvCoder<K, VIN> inputCoder,
+ Combine.KeyedCombineFn<K, VIN, VACC, VOUT> combiner) {
+
+ this.options = Preconditions.checkNotNull(options);
+ this.coderRegistry = Preconditions.checkNotNull(registry);
+ this.inputKvCoder = Preconditions.checkNotNull(inputCoder);//(KvCoder<K, VIN>) input.getCoder();
+ this.combineFn = combiner;
+ this.windowingStrategy = Preconditions.checkNotNull(windowingStrategy);//input.getWindowingStrategy();
+ this.operator = createGroupAlsoByWindowOperator();
+ this.chainingStrategy = ChainingStrategy.ALWAYS;
+ }
+
+ @Override
+ public void open() throws Exception {
+ super.open();
+ this.context = new ProcessContext(operator, new TimestampedCollector<>(output), this.timerInternals);
+
+ // this is to cover the case that this is the state after a recovery.
+ // In this case, the restoreState() has already initialized the timerInternals to a certain value.
+ TimerOrElement<WindowedValue<KV<K, VIN>>> element = this.timerInternals.getElement();
+ if (element != null) {
+ if (element.isTimer()) {
+ throw new RuntimeException("The recovered element cannot be a Timer.");
+ }
+ K key = element.element().getValue().getKey();
+ FlinkStateInternals<K> stateForKey = getStateInternalsForKey(key);
+ this.context.setElement(element, stateForKey);
+ }
+ }
+
+ /**
+ * Create the adequate {@link com.google.cloud.dataflow.sdk.util.StreamingGroupAlsoByWindowsDoFn},
+ * <b> if not already created</b>.
+ * If a {@link com.google.cloud.dataflow.sdk.transforms.Combine.KeyedCombineFn} was provided, then
+ * a function with that combiner is created, so that elements are combined as they arrive. This is
+ * done for speed and (in most of the cases) for reduction of the per-window state.
+ */
+ private StreamingGroupAlsoByWindowsDoFn createGroupAlsoByWindowOperator() {
+ if (this.operator == null) {
+ if (this.combineFn == null) {
+ Coder<VIN> inputValueCoder = inputKvCoder.getValueCoder();
+
+ this.operator = StreamingGroupAlsoByWindowsDoFn.createForIterable(
+ this.windowingStrategy, inputValueCoder);
+ } else {
+
+ Coder<K> inputKeyCoder = inputKvCoder.getKeyCoder();
+ //CoderRegistry dataflowRegistry = input.getPipeline().getCoderRegistry();
+
+ AppliedCombineFn<K, VIN, VACC, VOUT> appliedCombineFn = AppliedCombineFn
+ .withInputCoder(combineFn, coderRegistry, inputKvCoder);
+
+ this.operator = StreamingGroupAlsoByWindowsDoFn.create(
+ this.windowingStrategy, appliedCombineFn, inputKeyCoder);
+ }
+ }
+ return this.operator;
+ }
+
+
+ @Override
+ public void processElement(StreamRecord<WindowedValue<KV<K, VIN>>> element) throws Exception {
+ WindowedValue<KV<K, VIN>> value = element.getValue();
+ TimerOrElement<WindowedValue<KV<K, VIN>>> elem = TimerOrElement.element(value);
+ processElementOrTimer(elem);
+ }
+
+ @Override
+ public void processWatermark(Watermark mark) throws Exception {
+
+ context.setCurrentWatermark(new Instant(mark.getTimestamp()));
+
+ Set<TimerOrElement> toFire = getTimersReadyToProcess(mark.getTimestamp());
+ if (!toFire.isEmpty()) {
+ for (TimerOrElement timer : toFire) {
+ processElementOrTimer(timer);
+ }
+ }
+
+ /**
+ * This is to take into account the different semantics of the Watermark in Flink and
+ * in Dataflow. To understand the reasoning behind the Dataflow semantics and its
+ * watermark holding logic, see the documentation of
+ * {@link WatermarkHold#addHold(ReduceFn.ProcessValueContext, boolean)}
+ * */
+ long millis = Long.MAX_VALUE;
+ for (FlinkStateInternals state : perKeyStateInternals.values()) {
+ Instant watermarkHold = state.getWatermarkHold();
+ if (watermarkHold != null && watermarkHold.getMillis() < millis) {
+ millis = watermarkHold.getMillis();
+ }
+ }
+
+ if (mark.getTimestamp() < millis) {
+ millis = mark.getTimestamp();
+ }
+
+ // Don't forget to re-emit the watermark for further operators down the line.
+ // This is critical for jobs with multiple aggregation steps.
+ // Imagine a job with a groupByKey() on key K1, followed by a map() that changes
+ // the key K1 to K2, and another groupByKey() on K2. In this case, if the watermark
+ // is not re-emitted, the second aggregation would never be triggered, and no result
+ // will be produced.
+ output.emitWatermark(new Watermark(millis));
+ }
+
+ private void processElementOrTimer(TimerOrElement<WindowedValue<KV<K, VIN>>> timerOrElement) throws Exception {
+ K key = timerOrElement.isTimer() ?
+ (K) timerOrElement.key() :
+ timerOrElement.element().getValue().getKey();
+
+ context.setElement(timerOrElement, getStateInternalsForKey(key));
+
+ operator.startBundle(context);
+ operator.processElement(context);
+ operator.finishBundle(context);
+ }
+
+ private void registerActiveTimer(K key, TimerInternals.TimerData timer) {
+ Set<TimerInternals.TimerData> timersForKey = activeTimers.get(key);
+ if (timersForKey == null) {
+ timersForKey = new HashSet<>();
+ }
+ timersForKey.add(timer);
+ activeTimers.put(key, timersForKey);
+ }
+
+ private void unregisterActiveTimer(K key, TimerInternals.TimerData timer) {
+ Set<TimerInternals.TimerData> timersForKey = activeTimers.get(key);
+ if (timersForKey != null) {
+ timersForKey.remove(timer);
+ if (timersForKey.isEmpty()) {
+ activeTimers.remove(key);
+ } else {
+ activeTimers.put(key, timersForKey);
+ }
+ }
+ }
+
+ /**
+ * Returns the list of timers that are ready to fire. These are the timers
+ * that are registered to be triggered at a time before the current watermark.
+ * We keep these timers in a Set, so that they are deduplicated, as the same
+ * timer can be registered multiple times.
+ */
+ private Set<TimerOrElement> getTimersReadyToProcess(long currentWatermark) {
+
+ // we keep the timers to return in a different list and launch them later
+ // because we cannot prevent a trigger from registering another trigger,
+ // which would lead to concurrent modification exception.
+ Set<TimerOrElement> toFire = new HashSet<>();
+
+ Iterator<Map.Entry<K, Set<TimerInternals.TimerData>>> it = activeTimers.entrySet().iterator();
+ while (it.hasNext()) {
+ Map.Entry<K, Set<TimerInternals.TimerData>> keyWithTimers = it.next();
+
+ Iterator<TimerInternals.TimerData> timerIt = keyWithTimers.getValue().iterator();
+ while (timerIt.hasNext()) {
+ TimerInternals.TimerData timerData = timerIt.next();
+ if (timerData.getTimestamp().isBefore(currentWatermark)) {
+ TimerOrElement timer = TimerOrElement.timer(keyWithTimers.getKey(), timerData);
+ toFire.add(timer);
+ timerIt.remove();
+ }
+ }
+
+ if (keyWithTimers.getValue().isEmpty()) {
+ it.remove();
+ }
+ }
+ return toFire;
+ }
+
+ /**
+ * Gets the state associated with the specified key.
+ *
+ * @param key the key whose state we want.
+ * @return The {@link FlinkStateInternals}
+ * associated with that key.
+ */
+ private FlinkStateInternals<K> getStateInternalsForKey(K key) {
+ FlinkStateInternals<K> stateInternals = perKeyStateInternals.get(key);
+ if (stateInternals == null) {
+ Coder<? extends BoundedWindow> windowCoder = this.windowingStrategy.getWindowFn().windowCoder();
+ stateInternals = new FlinkStateInternals<>(key, inputKvCoder.getKeyCoder(), windowCoder, combineFn);
+ perKeyStateInternals.put(key, stateInternals);
+ }
+ return stateInternals;
+ }
+
+ private class FlinkTimerInternals extends AbstractFlinkTimerInternals<K, VIN> {
+
+ @Override
+ protected void registerTimer(K key, TimerData timerKey) {
+ registerActiveTimer(key, timerKey);
+ }
+
+ @Override
+ protected void unregisterTimer(K key, TimerData timerKey) {
+ unregisterActiveTimer(key, timerKey);
+ }
+ }
+
+ private class ProcessContext extends DoFn<TimerOrElement<WindowedValue<KV<K, VIN>>>, KV<K, VOUT>>.ProcessContext {
+
+ private final FlinkTimerInternals timerInternals;
+
+ private final DoFn<TimerOrElement<WindowedValue<KV<K, VIN>>>, KV<K, VOUT>> fn;
+
+ private final Collector<WindowedValue<KV<K, VOUT>>> collector;
+
+ private FlinkStateInternals<K> stateInternals;
+
+ private TimerOrElement<WindowedValue<KV<K, VIN>>> element;
+
+ public ProcessContext(DoFn<TimerOrElement<WindowedValue<KV<K, VIN>>>, KV<K, VOUT>> function,
+ Collector<WindowedValue<KV<K, VOUT>>> outCollector,
+ FlinkTimerInternals timerInternals) {
+ function.super();
+ super.setupDelegateAggregators();
+
+ this.fn = Preconditions.checkNotNull(function);
+ this.collector = Preconditions.checkNotNull(outCollector);
+ this.timerInternals = Preconditions.checkNotNull(timerInternals);
+ }
+
+ public void setElement(TimerOrElement<WindowedValue<KV<K, VIN>>> value,
+ FlinkStateInternals<K> stateForKey) {
+ this.element = value;
+ this.stateInternals = stateForKey;
+ this.timerInternals.setElement(value);
+ }
+
+ public void setCurrentWatermark(Instant watermark) {
+ this.timerInternals.setCurrentWatermark(watermark);
+ }
+
+ @Override
+ public TimerOrElement element() {
+ if (element != null && !this.element.isTimer()) {
+ return TimerOrElement.element(this.element.element().getValue());
+ }
+ return this.element;
+ }
+
+ @Override
+ public Instant timestamp() {
+ return this.element.isTimer() ?
+ this.element.getTimer().getTimestamp() :
+ this.element.element().getTimestamp();
+ }
+
+ @Override
+ public PipelineOptions getPipelineOptions() {
+ return options;
+ }
+
+ @Override
+ public void output(KV<K, VOUT> output) {
+ throw new UnsupportedOperationException(
+ "output() is not available when grouping by window.");
+ }
+
+ @Override
+ public void outputWithTimestamp(KV<K, VOUT> output, Instant timestamp) {
+ throw new UnsupportedOperationException(
+ "outputWithTimestamp() is not available when grouping by window.");
+ }
+
+ @Override
+ public PaneInfo pane() {
+ return this.element.element().getPane();
+ }
+
+ @Override
+ public BoundedWindow window() {
+ if (!(fn instanceof DoFn.RequiresWindowAccess)) {
+ throw new UnsupportedOperationException(
+ "window() is only available in the context of a DoFn marked as RequiresWindow.");
+ }
+
+ Collection<? extends BoundedWindow> windows = this.element.element().getWindows();
+ if (windows.size() != 1) {
+ throw new IllegalArgumentException("Each element is expected to belong to 1 window. " +
+ "This belongs to " + windows.size() + ".");
+ }
+ return windows.iterator().next();
+ }
+
+ @Override
+ public WindowingInternals<TimerOrElement<WindowedValue<KV<K, VIN>>>, KV<K, VOUT>> windowingInternals() {
+ return new WindowingInternals<TimerOrElement<WindowedValue<KV<K, VIN>>>, KV<K, VOUT>>() {
+
+ @Override
+ public com.google.cloud.dataflow.sdk.util.state.StateInternals stateInternals() {
+ return stateInternals;
+ }
+
+ @Override
+ public void outputWindowedValue(KV<K, VOUT> output, Instant timestamp, Collection<? extends BoundedWindow> windows, PaneInfo pane) {
+ collector.collect(WindowedValue.of(output, timestamp, windows, pane));
+ }
+
+ @Override
+ public TimerInternals timerInternals() {
+ return timerInternals;
+ }
+
+ @Override
+ public Collection<? extends BoundedWindow> windows() {
+ return element.element().getWindows();
+ }
+
+ @Override
+ public PaneInfo pane() {
+ return element.element().getPane();
+ }
+
+ @Override
+ public <T> void writePCollectionViewData(TupleTag<?> tag, Iterable<WindowedValue<T>> data, Coder<T> elemCoder) throws IOException {
+ throw new RuntimeException("writePCollectionViewData() not supported in Streaming mode.");
+ }
+ };
+ }
+
+ @Override
+ public <T> T sideInput(PCollectionView<T> view) {
+ throw new RuntimeException("sideInput() is not supported in Streaming mode.");
+ }
+
+ @Override
+ public <T> void sideOutput(TupleTag<T> tag, T output) {
+ // ignore the side output, this can happen when a user does not register
+ // side outputs but then outputs using a freshly created TupleTag.
+ throw new RuntimeException("sideOutput() is not available when grouping by window.");
+ }
+
+ @Override
+ public <T> void sideOutputWithTimestamp(TupleTag<T> tag, T output, Instant timestamp) {
+ sideOutput(tag, output);
+ }
+
+ @Override
+ protected <AggInputT, AggOutputT> Aggregator<AggInputT, AggOutputT> createAggregatorInternal(String name, Combine.CombineFn<AggInputT, ?, AggOutputT> combiner) {
+ Accumulator acc = getRuntimeContext().getAccumulator(name);
+ if (acc != null) {
+ AccumulatorHelper.compareAccumulatorTypes(name,
+ SerializableFnAggregatorWrapper.class, acc.getClass());
+ return (Aggregator<AggInputT, AggOutputT>) acc;
+ }
+
+ SerializableFnAggregatorWrapper<AggInputT, AggOutputT> accumulator =
+ new SerializableFnAggregatorWrapper<>(combiner);
+ getRuntimeContext().addAccumulator(name, accumulator);
+ return accumulator;
+ }
+ }
+
+ ////////////// Checkpointing implementation ////////////////
+
+ @Override
+ public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception {
+ StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
+ StateBackend.CheckpointStateOutputView out = getStateBackend().createCheckpointStateOutputView(checkpointId, timestamp);
+ StateCheckpointWriter writer = StateCheckpointWriter.create(out);
+ Coder<K> keyCoder = inputKvCoder.getKeyCoder();
+
+ // checkpoint the timers
+ StateCheckpointUtils.encodeTimers(activeTimers, writer, keyCoder);
+
+ // checkpoint the state
+ StateCheckpointUtils.encodeState(perKeyStateInternals, writer, keyCoder);
+
+ // checkpoint the timerInternals
+ context.timerInternals.encodeTimerInternals(context, writer,
+ inputKvCoder, windowingStrategy.getWindowFn().windowCoder());
+
+ taskState.setOperatorState(out.closeAndGetHandle());
+ return taskState;
+ }
+
+ @Override
+ public void restoreState(StreamTaskState taskState) throws Exception {
+ super.restoreState(taskState);
+
+ final ClassLoader userClassloader = getUserCodeClassloader();
+
+ Coder<? extends BoundedWindow> windowCoder = this.windowingStrategy.getWindowFn().windowCoder();
+ Coder<K> keyCoder = inputKvCoder.getKeyCoder();
+
+ @SuppressWarnings("unchecked")
+ StateHandle<DataInputView> inputState = (StateHandle<DataInputView>) taskState.getOperatorState();
+ DataInputView in = inputState.getState(userClassloader);
+ StateCheckpointReader reader = new StateCheckpointReader(in);
+
+ // restore the timers
+ this.activeTimers = StateCheckpointUtils.decodeTimers(reader, windowCoder, keyCoder);
+
+ // restore the state
+ this.perKeyStateInternals = StateCheckpointUtils.decodeState(
+ reader, combineFn, keyCoder, windowCoder, userClassloader);
+
+ // restore the timerInternals.
+ this.timerInternals.restoreTimerInternals(reader, inputKvCoder, windowCoder);
+ }
+}
\ No newline at end of file