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
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import org.apache.beam.runners.spark.translation.EvaluationContext;
import org.apache.beam.runners.spark.translation.SparkContextFactory;
import org.apache.beam.runners.spark.translation.SparkPipelineEvaluator;
import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
import org.apache.beam.runners.spark.translation.SparkPipelineTranslator;
import org.apache.beam.runners.spark.translation.SparkProcessContext;
import org.apache.beam.runners.spark.translation.TransformTranslator;
Expand All @@ -30,6 +29,7 @@
import org.apache.beam.runners.spark.translation.streaming.StreamingWindowPipelineDetector;
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.runners.TransformTreeNode;
Expand All @@ -40,6 +40,7 @@
import org.apache.beam.sdk.values.PInput;
import org.apache.beam.sdk.values.POutput;
import org.apache.beam.sdk.values.PValue;

import org.apache.spark.SparkException;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.streaming.Duration;
Expand Down Expand Up @@ -85,7 +86,8 @@ public final class SparkPipelineRunner extends PipelineRunner<EvaluationResult>
* @return A pipeline runner with default options.
*/
public static SparkPipelineRunner create() {
SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
SparkPipelineOptions options = PipelineOptionsFactory.as(SparkPipelineOptions.class);
options.setRunner(SparkPipelineRunner.class);
return new SparkPipelineRunner(options);
}

Expand Down

This file was deleted.

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -18,20 +18,22 @@

package org.apache.beam.runners.spark;

import java.util.Arrays;
import java.util.List;
import java.util.Set;

import com.google.common.collect.ImmutableSet;
import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.RemoveDuplicates;
import org.apache.beam.sdk.values.PCollection;

import com.google.common.collect.ImmutableSet;

import org.junit.Test;

import java.util.Arrays;
import java.util.List;
import java.util.Set;

/**
* A test based on {@code DeDupExample} from the SDK.
*/
Expand All @@ -46,7 +48,7 @@ public class DeDupTest {

@Test
public void testRun() throws Exception {
SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
SparkPipelineOptions options = PipelineOptionsFactory.as(SparkPipelineOptions.class);
options.setRunner(SparkPipelineRunner.class);
Pipeline p = Pipeline.create(options);
PCollection<String> input = p.apply(Create.of(LINES)).setCoder(StringUtf8Coder.of());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,24 +20,27 @@

import static org.junit.Assert.assertEquals;

import java.util.Collections;
import java.util.List;

import com.google.common.collect.Iterables;
import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.transforms.Combine;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.SerializableFunction;
import org.apache.beam.sdk.values.PCollection;

import com.google.common.collect.Iterables;

import org.junit.Test;

import java.util.Collections;
import java.util.List;

public class EmptyInputTest {

@Test
public void test() throws Exception {
SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
SparkPipelineOptions options = PipelineOptionsFactory.as(SparkPipelineOptions.class);
options.setRunner(SparkPipelineRunner.class);
Pipeline p = Pipeline.create(options);
List<String> empty = Collections.emptyList();
PCollection<String> inputWords = p.apply(Create.of(empty)).setCoder(StringUtf8Coder.of());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,9 @@

package org.apache.beam.runners.spark;

import java.util.Arrays;
import java.util.List;
import java.util.Set;
import java.util.regex.Pattern;

import com.google.common.collect.ImmutableSet;
import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.testing.PAssert;
import org.apache.beam.sdk.transforms.Aggregator;
import org.apache.beam.sdk.transforms.Count;
Expand All @@ -37,8 +31,16 @@
import org.apache.beam.sdk.transforms.Sum;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;

import com.google.common.collect.ImmutableSet;

import org.junit.Test;

import java.util.Arrays;
import java.util.List;
import java.util.Set;
import java.util.regex.Pattern;

public class SimpleWordCountTest {
private static final String[] WORDS_ARRAY = {
"hi there", "hi", "hi sue bob",
Expand All @@ -49,7 +51,7 @@ public class SimpleWordCountTest {

@Test
public void testRun() throws Exception {
SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
SparkPipelineOptions options = PipelineOptionsFactory.as(SparkPipelineOptions.class);
options.setRunner(SparkPipelineRunner.class);
Pipeline p = Pipeline.create(options);
PCollection<String> inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,6 @@

package org.apache.beam.runners.spark;

import java.net.URI;
import java.util.Arrays;

import org.apache.beam.examples.complete.TfIdf;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.StringDelegateCoder;
Expand All @@ -31,16 +28,22 @@
import org.apache.beam.sdk.transforms.RemoveDuplicates;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;

import org.junit.Test;

import java.net.URI;
import java.util.Arrays;

/**
* A test based on {@code TfIdf} from the SDK.
*/
public class TfIdfTest {

@Test
public void testTfIdf() throws Exception {
Pipeline pipeline = Pipeline.create(PipelineOptionsFactory.create());
SparkPipelineOptions opts = PipelineOptionsFactory.as(SparkPipelineOptions.class);
opts.setRunner(SparkPipelineRunner.class);
Pipeline pipeline = Pipeline.create(opts);

pipeline.getCoderRegistry().registerCoder(URI.class, StringDelegateCoder.of(URI.class));

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.beam.runners.spark.coders;

import org.apache.beam.sdk.testing.CoderProperties;

import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.NullWritable;
import org.junit.Test;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,31 +20,33 @@

import static org.junit.Assert.assertEquals;

import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.util.List;
import org.apache.beam.runners.spark.EvaluationResult;
import org.apache.beam.runners.spark.SparkPipelineRunner;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.io.AvroIO;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.values.PCollection;

import com.google.common.collect.Lists;
import com.google.common.io.Resources;

import org.apache.avro.Schema;
import org.apache.avro.file.DataFileReader;
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.generic.GenericRecord;
import org.apache.beam.runners.spark.EvaluationResult;
import org.apache.beam.runners.spark.SparkPipelineRunner;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.io.AvroIO;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.values.PCollection;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;

import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.util.List;

public class AvroPipelineTest {

private File inputFile;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,32 +21,34 @@
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;

import java.io.File;
import java.io.FileFilter;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Set;

import com.google.common.base.Charsets;
import com.google.common.collect.Sets;
import com.google.common.io.Files;
import org.apache.beam.examples.WordCount;
import org.apache.beam.runners.spark.EvaluationResult;
import org.apache.beam.runners.spark.SparkPipelineOptions;
import org.apache.beam.runners.spark.SparkPipelineRunner;
import org.apache.beam.runners.spark.translation.SparkPipelineOptionsFactory;
import org.apache.beam.sdk.Pipeline;
import org.apache.beam.sdk.coders.StringUtf8Coder;
import org.apache.beam.sdk.io.TextIO;
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.transforms.Create;
import org.apache.beam.sdk.transforms.MapElements;
import org.apache.beam.sdk.values.PCollection;

import com.google.common.base.Charsets;
import com.google.common.collect.Sets;
import com.google.common.io.Files;

import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;

import java.io.File;
import java.io.FileFilter;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Set;

public class NumShardsTest {

private static final String[] WORDS_ARRAY = {
Expand All @@ -67,7 +69,7 @@ public void setUp() throws IOException {

@Test
public void testText() throws Exception {
SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
SparkPipelineOptions options = PipelineOptionsFactory.as(SparkPipelineOptions.class);
options.setRunner(SparkPipelineRunner.class);
Pipeline p = Pipeline.create(options);
PCollection<String> inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder.of());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,6 @@

import static org.junit.Assert.assertEquals;

import java.io.File;
import java.io.IOException;

import org.apache.beam.runners.spark.EvaluationResult;
import org.apache.beam.runners.spark.SparkPipelineRunner;
import org.apache.beam.runners.spark.coders.WritableCoder;
Expand All @@ -31,6 +28,7 @@
import org.apache.beam.sdk.options.PipelineOptionsFactory;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
Expand All @@ -46,6 +44,9 @@
import org.junit.Test;
import org.junit.rules.TemporaryFolder;

import java.io.File;
import java.io.IOException;

public class HadoopFileFormatPipelineTest {

private File inputFile;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFileTemplate;
import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardCount;
import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardNumber;

import static org.junit.Assert.assertEquals;

import org.junit.Test;
Expand Down
Loading