Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
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
12 changes: 12 additions & 0 deletions examples/java8/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,18 @@
</configuration>
</plugin>

<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<configuration>
<systemPropertyVariables>
<beamTestPipelineOptions>
</beamTestPipelineOptions>
</systemPropertyVariables>
</configuration>
</plugin>


<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-dependency-plugin</artifactId>
Expand Down
117 changes: 84 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,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 @@ -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();
}
}


This file was deleted.

Loading