Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
23 commits
Select commit Hold shift + click to select a range
5e45de8
Add Window.Bound translator to Flink batch
kennknowles May 2, 2016
95f915a
Add TestFlinkPipelineRunner to FlinkRunnerRegistrar
kennknowles May 2, 2016
c15be55
Configure RunnableOnService tests for Flink in batch mode
kennknowles May 2, 2016
9071bbe
Fix Dangling Flink DataSets
aljoscha May 6, 2016
f76d09d
Add hamcrest dep to Flink Runner
aljoscha May 6, 2016
bdb77da
Fix Flink Create and GroupByNullKeyTest, Remove Special VoidSerializer
aljoscha May 6, 2016
7ad190b
Fix faulty Flink Flatten with empty PCollectionList
aljoscha May 6, 2016
eddbbb1
Fix Flink Batch Partial Combine/Combine
aljoscha May 6, 2016
ba1e4d4
Disable MaybeEmptyTestITCase
aljoscha May 6, 2016
2ac9998
Add RequiresFixedWindows test category
kennknowles May 6, 2016
21614a1
Exclude RequiresFixedWindows test category from Flink batch tests
kennknowles May 6, 2016
ff7051f
Remove unused threadCount from integration tests
kennknowles May 6, 2016
1df8038
Disable Flink streaming integration tests for now
kennknowles May 6, 2016
155dc26
Special casing job exec AssertionError in TestFlinkPipelineRunner
kennknowles May 6, 2016
f418a22
Use Int instead of Void in Combine.globally default insertion
aljoscha May 9, 2016
76450e8
Fix accumulator update in Flink Reduce Function
aljoscha May 9, 2016
bd404b0
Use Int instead of Void in Sample
aljoscha May 9, 2016
76ae7a4
Use Int instead of Void in FlattenTest
aljoscha May 9, 2016
0452755
Add RequiresTimestampControl category and tag some tests
kennknowles May 9, 2016
29a7752
Exclude groups from Flink batch integration tests
kennknowles May 9, 2016
1dc63f8
Fix checkstyle of FlattenTest
kennknowles May 9, 2016
6a13182
Add RequiresTimestampControl to WithTimestampsTest
kennknowles May 12, 2016
31cb37d
Temporarily disable failing Spark streaming tests
kennknowles 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
131 changes: 98 additions & 33 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,7 +62,8 @@
<artifactId>flink-avro_2.10</artifactId>
<version>${flink.version}</version>
</dependency>
<!-- Beam -->

<!--- Beam -->
<dependency>
<groupId>org.apache.beam</groupId>
<artifactId>java-sdk-all</artifactId>
Expand All @@ -111,6 +87,37 @@
</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 @@ -133,12 +140,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 @@ -168,10 +169,74 @@
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<executions>
Copy link
Member Author

Choose a reason for hiding this comment

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

@davorbonaci @jasonkuster @lukecwik my maven-fu might be lacking. I think I can remove the phase and goals here, since the execution has an id, and maybe lift some of the configuration, but most importantly, I could not get runnableOnServicePipelineOptions to do anything, instead having to set beamTestPipelineOptions. This seems mostly fine but please advise.

Copy link
Member

@lukecwik lukecwik May 5, 2016

Choose a reason for hiding this comment

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

I think you want to drop the whole plugin configuration and either:

  • do as we did in the dataflow runner pom to have a profile dependent on runnableOnService existing and then jenkins could be configured to build the flink pipeline and dependent modules with a system property on the jenkins command line which sets the runnableOnServicePipelineOptions as you have listed before
  • always run the jenkins tests part of the regular integration-test run by hardcoding runnableOnServicePipelineOptions in this module and having a trivial configuration part of builds/plugins:
         <plugin>
            <groupId>org.apache.maven.plugins</groupId>
            <artifactId>maven-surefire-plugin</artifactId>
            <executions>
                <id>runnable-on-service-tests</id>
              </execution>
            </executions>
          </plugin>

I don't know if you need the executions block or not, I would think it would inherit it from plugin but could be wrong.

Copy link
Member Author

Choose a reason for hiding this comment

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

We definitely need two executions for now, since we need to pass --streaming=true and --streaming=false. It actually seems to confuse Jenkins, since the test class is duplicated. It probably will also confuse it with the direct runner and spark runner tests.

For Flink and Spark, running against a local endpoint as a unit test (not really an integration test per se) is reasonable. We want to support both, so Jenkins can also run a real integration test against a cluster by overrides on the command line.

<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>
<excludedGroups>
org.apache.beam.sdk.testing.capabilities.RequiresFixedWindows,
org.apache.beam.sdk.testing.capabilities.RequiresTimestampControl
</excludedGroups>
<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>
<excludes>
<!-- Tests that use unsupported windowing -->
<exclude>**/org/apache/beam/sdk/transforms/CombineTest.java</exclude>
<exclude>**/org/apache/beam/sdk/transforms/GroupByKeyTest.java</exclude>
<exclude>**/org/apache/beam/sdk/transforms/ViewTest.java</exclude>
<exclude>**/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java</exclude>
<exclude>**/org/apache/beam/sdk/transforms/windowing/WindowTest.java</exclude>
<exclude>**/org/apache/beam/sdk/transforms/windowing/WindowingTest.java</exclude>
<exclude>**/org/apache/beam/sdk/util/ReshuffleTest.java</exclude>
</excludes>
</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 @@ -108,7 +108,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 @@ -138,20 +138,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,14 +17,17 @@
*/
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;

Expand All @@ -47,6 +50,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 Down
Loading