Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
25 commits
Select commit Hold shift + click to select a range
3cb9973
Add Window.Bound translator to Flink batch
kennknowles May 2, 2016
b7335f9
Add TestFlinkPipelineRunner to FlinkRunnerRegistrar
kennknowles May 2, 2016
093dc3e
Configure RunnableOnService tests for Flink in batch mode
kennknowles May 2, 2016
6d00cba
Fix Dangling Flink DataSets
aljoscha May 6, 2016
af7f858
Add hamcrest dep to Flink Runner
aljoscha May 6, 2016
0cc7363
Fix Flink Create and GroupByNullKeyTest, Remove Special VoidSerializer
aljoscha May 6, 2016
5fa21df
Fix faulty Flink Flatten with empty PCollectionList
aljoscha May 6, 2016
30818e8
Fix Flink Batch Partial Combine/Combine
aljoscha May 6, 2016
e663de1
Disable MaybeEmptyTestITCase
aljoscha May 6, 2016
abad037
Add RequiresFixedWindows test category
kennknowles May 6, 2016
b98ee40
Exclude RequiresFixedWindows test category from Flink batch tests
kennknowles May 6, 2016
1306655
Remove unused threadCount from integration tests
kennknowles May 6, 2016
c3f71e2
Disable Flink streaming integration tests for now
kennknowles May 6, 2016
a0376ff
Change PAssert's dummy inputs from (Void) null to integer 0
kennknowles May 6, 2016
d60229d
Special casing job exec AssertionError in TestFlinkPipelineRunner
kennknowles May 6, 2016
054a42c
Use Int instead of Void in Combine.globally default insertion
aljoscha May 9, 2016
8f7f9fb
Fix accumulator update in Flink Reduce Function
aljoscha May 9, 2016
7abf64b
Use Int instead of Void as dummy key in Combine.globally
aljoscha May 9, 2016
6023361
Use Int instead of Void in Sample
aljoscha May 9, 2016
fa0f769
Use Int instead of Void in FlattenTest
aljoscha May 9, 2016
19c4cd3
Add RequiresTimestampControl category and tag some tests
kennknowles May 9, 2016
55efae4
Exclude groups from Flink batch integration tests
kennknowles May 9, 2016
99fe94c
fixup! fix checkstyle of FlattenTest
kennknowles May 9, 2016
1ee5685
[BEAM-270] Support Timestamps/Windows in Flink Batch
aljoscha May 10, 2016
a2c988c
fixup, remove WordCount changes, remove KvCoderComparator code
aljoscha May 12, 2016
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
116 changes: 84 additions & 32 deletions runners/flink/runner/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -34,31 +34,6 @@

<packaging>jar</packaging>

<profiles>
<profile>
<id>disable-runnable-on-service-tests</id>
<activation>
<activeByDefault>true</activeByDefault>
</activation>
<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<executions>
<execution>
<id>runnable-on-service-tests</id>
<configuration>
<skip>true</skip>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</profile>
</profiles>

<dependencies>
<!-- Flink dependencies -->
<dependency>
Expand Down Expand Up @@ -87,6 +62,7 @@
<artifactId>flink-avro_2.10</artifactId>
<version>${flink.version}</version>
</dependency>

<!--- Beam -->
<dependency>
<groupId>org.apache.beam</groupId>
Expand Down Expand Up @@ -120,7 +96,39 @@
</exclusion>
</exclusions>
</dependency>

<!-- Test scoped -->

<dependency>
<groupId>org.hamcrest</groupId>
<artifactId>hamcrest-all</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>junit</groupId>
<artifactId>junit</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-all</artifactId>
<scope>test</scope>
</dependency>

<!-- Depend on test jar to scan for RunnableOnService tests -->
<dependency>
<groupId>org.apache.beam</groupId>
<artifactId>java-sdk-all</artifactId>
<classifier>tests</classifier>
<scope>test</scope>
<exclusions>
<exclusion>
<groupId>org.slf4j</groupId>
<artifactId>slf4j-jdk14</artifactId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>org.apache.beam</groupId>
<artifactId>java-examples-all</artifactId>
Expand All @@ -143,12 +151,6 @@
<groupId>org.apache.flink</groupId>
<artifactId>flink-test-utils_2.10</artifactId>
<version>${flink.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-all</artifactId>
<scope>test</scope>
</dependency>
<!-- Optional Pipeline Registration -->
<dependency>
Expand Down Expand Up @@ -178,10 +180,60 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<executions>
<execution>
<id>runnable-on-service-tests</id>
<phase>integration-test</phase>
<goals>
<goal>test</goal>
</goals>
<configuration>
<groups>org.apache.beam.sdk.testing.RunnableOnService</groups>
<parallel>none</parallel>
<failIfNoTests>true</failIfNoTests>
<dependenciesToScan>
<dependency>org.apache.beam:java-sdk-all</dependency>
</dependenciesToScan>
<systemPropertyVariables>
<beamTestPipelineOptions>
[
"--runner=org.apache.beam.runners.flink.TestFlinkPipelineRunner",
"--streaming=false"
]
</beamTestPipelineOptions>
</systemPropertyVariables>
</configuration>
</execution>
<execution>
<id>streaming-runnable-on-service-tests</id>
<phase>integration-test</phase>
<goals>
<goal>test</goal>
</goals>
<configuration>
<skip>true</skip>
<groups>org.apache.beam.sdk.testing.RunnableOnService</groups>
<parallel>none</parallel>
<failIfNoTests>true</failIfNoTests>
<dependenciesToScan>
<dependency>org.apache.beam:java-sdk-all</dependency>
</dependenciesToScan>
<systemPropertyVariables>
<beamTestPipelineOptions>
[
"--runner=org.apache.beam.runners.flink.TestFlinkPipelineRunner",
"--streaming=true"
]
</beamTestPipelineOptions>
</systemPropertyVariables>
<excludes>
</excludes>
</configuration>
</execution>
</executions>
</plugin>

</plugins>

</build>

</project>
Original file line number Diff line number Diff line change
Expand Up @@ -110,7 +110,7 @@ public FlinkRunnerResult run(Pipeline pipeline) {
this.flinkJobEnv.translate(pipeline);

LOG.info("Starting execution of Flink program.");

JobExecutionResult result;
try {
result = this.flinkJobEnv.executePipeline();
Expand Down Expand Up @@ -140,20 +140,6 @@ 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) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,9 @@ private FlinkRunnerRegistrar() { }
public static class Runner implements PipelineRunnerRegistrar {
@Override
public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
return ImmutableList.<Class<? extends PipelineRunner<?>>>of(FlinkPipelineRunner.class);
return ImmutableList.<Class<? extends PipelineRunner<?>>>of(
FlinkPipelineRunner.class,
TestFlinkPipelineRunner.class);
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.beam.runners.flink;

import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.options.PipelineOptionsValidator;
import org.apache.beam.sdk.runners.PipelineRunner;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.values.PInput;
import org.apache.beam.sdk.values.POutput;

import org.apache.flink.runtime.client.JobExecutionException;

public class TestFlinkPipelineRunner extends PipelineRunner<FlinkRunnerResult> {

private FlinkPipelineRunner delegate;

private TestFlinkPipelineRunner(FlinkPipelineOptions options) {
// We use [auto] for testing since this will make it pick up the Testing ExecutionEnvironment
options.setFlinkMaster("[auto]");
this.delegate = FlinkPipelineRunner.fromOptions(options);
}

public static TestFlinkPipelineRunner fromOptions(PipelineOptions options) {
FlinkPipelineOptions flinkOptions = PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options);
return new TestFlinkPipelineRunner(flinkOptions);
}

public static TestFlinkPipelineRunner create(boolean streaming) {
FlinkPipelineOptions flinkOptions = PipelineOptionsFactory.as(FlinkPipelineOptions.class);
flinkOptions.setStreaming(streaming);
return TestFlinkPipelineRunner.fromOptions(flinkOptions);
}

@Override
public <OutputT extends POutput, InputT extends PInput>
OutputT apply(PTransform<InputT,OutputT> transform, InputT input) {
return delegate.apply(transform, input);
}

@Override
public FlinkRunnerResult run(Pipeline pipeline) {
try {
return delegate.run(pipeline);
} catch (RuntimeException e) {
// Special case hack to pull out assertion errors from PAssert; instead there should
// probably be a better story along the lines of UserCodeException.
if (e.getCause() != null
&& e.getCause() instanceof JobExecutionException
&& e.getCause().getCause() instanceof AssertionError) {
throw (AssertionError) e.getCause().getCause();
} else {
throw e;
}
}
}

public PipelineOptions getPipelineOptions() {
return delegate.getPipelineOptions();
}
}


Original file line number Diff line number Diff line change
Expand Up @@ -17,20 +17,23 @@
*/
package org.apache.beam.runners.flink.translation;

import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.runners.TransformTreeNode;
import org.apache.beam.sdk.transforms.AppliedPTransform;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.join.CoGroupByKey;
import org.apache.beam.sdk.values.PValue;

import org.apache.flink.api.java.DataSet;
import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.io.DiscardingOutputFormat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* FlinkBatchPipelineTranslator knows how to translate Pipeline objects into Flink Jobs.
* This is based on {@link org.apache.beam.runners.dataflow.DataflowPipelineTranslator}
* {@link Pipeline.PipelineVisitor} for executing a {@link Pipeline} as a
* Flink batch job.
*/
public class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {

Expand All @@ -52,6 +55,17 @@ public FlinkBatchPipelineTranslator(ExecutionEnvironment env, PipelineOptions op
this.batchContext = new FlinkBatchTranslationContext(env, options);
}

@Override
@SuppressWarnings("rawtypes, unchecked")
public void translate(Pipeline pipeline) {
super.translate(pipeline);

// terminate dangling DataSets
for (DataSet<?> dataSet: batchContext.getDanglingDataSets().values()) {
dataSet.output(new DiscardingOutputFormat());
}
}

// --------------------------------------------------------------------------------------------
// Pipeline Visitor Methods
// --------------------------------------------------------------------------------------------
Expand All @@ -63,10 +77,13 @@ public void enterCompositeTransform(TransformTreeNode node) {
PTransform<?, ?> transform = node.getTransform();
if (transform != null && currentCompositeTransform == null) {

BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
BatchTransformTranslator<?> translator =
FlinkBatchTransformTranslators.getTranslator(transform);

if (translator != null) {
currentCompositeTransform = transform;
if (transform instanceof CoGroupByKey && node.getInput().expand().size() != 2) {
if (transform instanceof CoGroupByKey &&
node.getInput().expand().size() != 2) {
// we can only optimize CoGroupByKey for input size 2
currentCompositeTransform = null;
}
Expand All @@ -80,7 +97,9 @@ public void leaveCompositeTransform(TransformTreeNode node) {
PTransform<?, ?> transform = node.getTransform();
if (transform != null && currentCompositeTransform == transform) {

BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
BatchTransformTranslator<?> translator =
FlinkBatchTransformTranslators.getTranslator(transform);

if (translator != null) {
LOG.info(genSpaces(this.depth) + "doingCompositeTransform- " + formatNodeName(node));
applyBatchTransform(transform, node, translator);
Expand All @@ -106,10 +125,13 @@ public void visitTransform(TransformTreeNode node) {
// currently visiting and translate it into its Flink alternative.

PTransform<?, ?> transform = node.getTransform();
BatchTransformTranslator<?> translator = FlinkBatchTransformTranslators.getTranslator(transform);
BatchTransformTranslator<?> translator =
FlinkBatchTransformTranslators.getTranslator(transform);

if (translator == null) {
LOG.info(node.getTransform().getClass().toString());
throw new UnsupportedOperationException("The transform " + transform + " is currently not supported.");
throw new UnsupportedOperationException(
"The transform " + transform + " is currently not supported.");
}
applyBatchTransform(transform, node, translator);
}
Expand All @@ -119,7 +141,10 @@ public void visitValue(PValue value, TransformTreeNode producer) {
// do nothing here
}

private <T extends PTransform<?, ?>> void applyBatchTransform(PTransform<?, ?> transform, TransformTreeNode node, BatchTransformTranslator<?> translator) {
private <T extends PTransform<?, ?>> void applyBatchTransform(
PTransform<?, ?> transform,
TransformTreeNode node,
BatchTransformTranslator<?> translator) {

@SuppressWarnings("unchecked")
T typedTransform = (T) transform;
Expand All @@ -136,8 +161,8 @@ public void visitValue(PValue value, TransformTreeNode producer) {
/**
* A translator of a {@link PTransform}.
*/
public interface BatchTransformTranslator<Type extends PTransform> {
void translateNode(Type transform, FlinkBatchTranslationContext context);
public interface BatchTransformTranslator<TransformT extends PTransform> {
void translateNode(TransformT transform, FlinkBatchTranslationContext context);
}

private static String genSpaces(int n) {
Expand Down
Loading