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
6 changes: 3 additions & 3 deletions runners/spark/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ Switch to the Spark runner directory:
Then run the [word count example][wc] from the SDK using a single threaded Spark instance
in local mode:

mvn exec:exec -DmainClass=com.google.cloud.dataflow.examples.WordCount \
mvn exec:exec -DmainClass=org.apache.beam.examples.WordCount \
-Dinput=/tmp/kinglear.txt -Doutput=/tmp/out -Drunner=SparkPipelineRunner \
-DsparkMaster=local

Expand All @@ -104,7 +104,7 @@ Check the output by running:
__Note: running examples using `mvn exec:exec` only works for Spark local mode at the
moment. See the next section for how to run on a cluster.__

[wc]: https://github.com/apache/incubator-beam/blob/master/examples/src/main/java/com/google/cloud/dataflow/examples/WordCount.java
[wc]: https://github.com/apache/incubator-beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/WordCount.java
## Running on a Cluster

Spark Beam pipelines can be run on a cluster using the `spark-submit` command.
Expand All @@ -117,7 +117,7 @@ Then run the word count example using Spark submit with the `yarn-client` master
(`yarn-cluster` works just as well):

spark-submit \
--class com.google.cloud.dataflow.examples.WordCount \
--class org.apache.beam.examples.WordCount \
--master yarn-client \
target/spark-runner-*-spark-app.jar \
--inputFile=kinglear.txt --output=out --runner=SparkPipelineRunner --sparkMaster=yarn-client
Expand Down
6 changes: 6 additions & 0 deletions runners/spark/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,12 @@
<artifactId>guava</artifactId>
<version>${guava.version}</version>
</dependency>
<dependency>
<groupId>com.google.auto.service</groupId>
<artifactId>auto-service</artifactId>
<version>1.0-rc2</version>
<optional>true</optional>
</dependency>
<dependency>
<groupId>org.apache.beam</groupId>
<artifactId>java-sdk-all</artifactId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
/*
* 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 com.google.auto.service.AutoService;
import com.google.common.collect.ImmutableList;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
import org.apache.beam.sdk.runners.PipelineRunner;
import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;

/**
* Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the
* {@link SparkPipelineRunner}.
*
* {@link AutoService} will register Spark's implementations of the {@link PipelineRunner}
* and {@link PipelineOptions} as available pipeline runner services.
*/
public final class SparkRunnerRegistrar {
private SparkRunnerRegistrar() {}

/**
* Registers the {@link SparkPipelineRunner}.
*/
@AutoService(PipelineRunnerRegistrar.class)
public static class Runner implements PipelineRunnerRegistrar {
@Override
public Iterable<Class<? extends PipelineRunner<?>>> getPipelineRunners() {
return ImmutableList.<Class<? extends PipelineRunner<?>>>of(SparkPipelineRunner.class);
}
}

/**
* Registers the {@link SparkPipelineOptions} and {@link SparkStreamingPipelineOptions}.
*/
@AutoService(PipelineOptionsRegistrar.class)
public static class Options implements PipelineOptionsRegistrar {
@Override
public Iterable<Class<? extends PipelineOptions>> getPipelineOptions() {
return ImmutableList.<Class<? extends PipelineOptions>>of(
SparkPipelineOptions.class,
SparkStreamingPipelineOptions.class);
}
}
}

This file was deleted.

This file was deleted.

This file was deleted.

This file was deleted.

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/*
* 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 com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
import org.apache.beam.sdk.runners.PipelineRunnerRegistrar;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.JUnit4;

import java.util.ServiceLoader;

import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;

/**
* Test {@link SparkRunnerRegistrar}.
*/
@RunWith(JUnit4.class)
public class SparkRunnerRegistrarTest {
Copy link
Member

Choose a reason for hiding this comment

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

Please annotate class with:
@RunWith(JUnit4.class)

@Test
public void testOptions() {
assertEquals(
ImmutableList.of(SparkPipelineOptions.class, SparkStreamingPipelineOptions.class),
new SparkRunnerRegistrar.Options().getPipelineOptions());
}

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

@Test
public void testServiceLoaderForOptions() {
for (PipelineOptionsRegistrar registrar :
Lists.newArrayList(ServiceLoader.load(PipelineOptionsRegistrar.class).iterator())) {
if (registrar instanceof SparkRunnerRegistrar.Options) {
return;
}
}
fail("Expected to find " + SparkRunnerRegistrar.Options.class);
}

@Test
public void testServiceLoaderForRunner() {
for (PipelineRunnerRegistrar registrar :
Lists.newArrayList(ServiceLoader.load(PipelineRunnerRegistrar.class).iterator())) {
if (registrar instanceof SparkRunnerRegistrar.Runner) {
return;
}
}
fail("Expected to find " + SparkRunnerRegistrar.Runner.class);
}
}