Skip to content
Merged
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
112 changes: 75 additions & 37 deletions runners/spark/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,62 @@
<spark.version>1.6.1</spark.version>
</properties>

<profiles>
<profile>
<id>jacoco</id>
<build>
<plugins>
<plugin>
<groupId>org.jacoco</groupId>
<artifactId>jacoco-maven-plugin</artifactId>
</plugin>
</plugins>
</build>
</profile>

<profile>
<!-- This profile adds execution of RunnableOnService integration tests
against a local Spark endpoint. -->
<id>runnable-on-service-tests</id>
<activation><activeByDefault>false</activeByDefault></activation>
<build>
<pluginManagement>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-surefire-plugin</artifactId>
<executions>
<execution>
<id>runnable-on-service-tests</id>
<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>
<excludes>
org.apache.beam.sdk.io.BoundedReadFromUnboundedSourceTest
</excludes>
<systemPropertyVariables>
<beamTestPipelineOptions>
[
"--runner=org.apache.beam.runners.spark.TestSparkPipelineRunner",
"--streaming=false"
]
</beamTestPipelineOptions>
<dataflow.spark.test.reuseSparkContext>true</dataflow.spark.test.reuseSparkContext>
</systemPropertyVariables>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</pluginManagement>
</build>
</profile>
</profiles>

<dependencies>
<dependency>
<groupId>org.apache.spark</groupId>
Expand Down Expand Up @@ -122,6 +178,25 @@
<groupId>org.apache.beam</groupId>
<artifactId>beam-runners-direct-java</artifactId>
<version>0.2.0-incubating-SNAPSHOT</version>
</dependency>

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

<dependency>
<groupId>org.mockito</groupId>
Copy link
Member Author

Choose a reason for hiding this comment

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

There's something amiss with the maven config, as omitting this results in ClassDefNotFound looking for Mockito classes.

Copy link
Member

Choose a reason for hiding this comment

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

According to the table here: https://maven.apache.org/guides/introduction/introduction-to-dependency-mechanism.html the runner has a scope test dependency on java-sdk-all (classifier tests), and it (java-sdk-all) has a scope test dependency on mockito, so it's not transitive anymore..
Actually it seems as if scope test is never transitive..

Copy link
Member Author

Choose a reason for hiding this comment

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

Ah, yes, I think I see. To get the behavior automatically, we would probably need a java-sdk-tests artifact with a compile-scoped dependency for everything that is currently a test-scoped dependency. I propose solving this in the future instead of now.

Copy link
Member

Choose a reason for hiding this comment

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

Agree

<artifactId>mockito-all</artifactId>
<scope>test</scope>
</dependency>
</dependencies>
Expand Down Expand Up @@ -237,41 +312,4 @@
</plugins>
</build>

<profiles>
<profile>
<id>jacoco</id>
<build>
<plugins>
<plugin>
<groupId>org.jacoco</groupId>
<artifactId>jacoco-maven-plugin</artifactId>
</plugin>
</plugins>
</build>
</profile>

<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>

</project>
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,8 @@ private SparkRunnerRegistrar() {}
public static class Runner implements PipelineRunnerRegistrar {
@Override
public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
return ImmutableList.<Class<? extends PipelineRunner<?>>>of(SparkPipelineRunner.class);
return ImmutableList.<Class<? extends PipelineRunner<?>>>of(
SparkPipelineRunner.class, TestSparkPipelineRunner.class);
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,77 @@
/*
* 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.spark;

import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.options.PipelineOptions;
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;

/**
* The SparkPipelineRunner translate operations defined on a pipeline to a representation executable
* by Spark, and then submitting the job to Spark to be executed. If we wanted to run a dataflow
* pipeline with the default options of a single threaded spark instance in local mode, we would do
* the following:
*
* {@code
* Pipeline p = [logic for pipeline creation]
* EvaluationResult result = SparkPipelineRunner.create().run(p);
* }
*
* To create a pipeline runner to run against a different spark cluster, with a custom master url we
* would do the following:
*
* {@code
* Pipeline p = [logic for pipeline creation]
* SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
* options.setSparkMaster("spark://host:port");
* EvaluationResult result = SparkPipelineRunner.create(options).run(p);
* }
*
* To create a Spark streaming pipeline runner use {@link SparkStreamingPipelineOptions}
*/
public final class TestSparkPipelineRunner extends PipelineRunner<EvaluationResult> {
Copy link
Member

Choose a reason for hiding this comment

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

Why do we need the TestSparkPipelineRunner ?

Copy link
Member Author

Choose a reason for hiding this comment

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

Perhaps you don't - it is really just a hook to set configuration options. I see some test cases that seem to set them up prior to the run, so this might be a place for that knowledge to reside. You could do it in the pom, too.

Copy link
Member

Choose a reason for hiding this comment

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

I don't really mind the additional class, but am wondering if you see some future use for it ?
To rephrase that, do you think runners will benefit from having a TestPipelineRunner ? taking it to the next level, should it be a part of the runner API ?

Copy link
Member Author

Choose a reason for hiding this comment

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

This comment of yours was prescient; I don't know if it is in the way you meant. The TestDataflowPipelineRunner does some additional bookkeeping to make sure that PAsserts ran as expected, using aggregators. Otherwise you might have an assertion that just never ran because the PCollection was empty, giving a false test pass. The logic really should be extracted into a generic TestPipelineRunner which could have a "sub" runner or some such. (Or it could just be a utility used in Java code to create the various test wrappers).


private SparkPipelineRunner delegate;

private TestSparkPipelineRunner(SparkPipelineOptions options) {
this.delegate = SparkPipelineRunner.fromOptions(options);
}

public static TestSparkPipelineRunner fromOptions(PipelineOptions options) {
// Default options suffice to set it up as a test runner
SparkPipelineOptions sparkOptions =
PipelineOptionsValidator.validate(SparkPipelineOptions.class, options);
return new TestSparkPipelineRunner(sparkOptions);
}

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

@Override
public EvaluationResult run(Pipeline pipeline) {
return delegate.run(pipeline);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ public void testOptions() {

@Test
public void testRunners() {
assertEquals(ImmutableList.of(SparkPipelineRunner.class),
assertEquals(ImmutableList.of(SparkPipelineRunner.class, TestSparkPipelineRunner.class),
new SparkRunnerRegistrar.Runner().getPipelineRunners());
}

Expand Down