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
Original file line number Diff line number Diff line change
Expand Up @@ -75,7 +75,7 @@ private class ProcCtxt extends SparkProcessContext<I, O, WindowedValue<O>> {
@Override
public synchronized void output(O o) {
outputs.add(windowedValue != null ? windowedValue.withValue(o) :
WindowedValue.valueInEmptyWindows(o));
WindowedValue.valueInGlobalWindow(o));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,9 @@
import org.apache.beam.sdk.transforms.Aggregator;
import org.apache.beam.sdk.transforms.AppliedPTransform;
import org.apache.beam.sdk.transforms.PTransform;
import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
import org.apache.beam.sdk.transforms.windowing.WindowFn;
import org.apache.beam.sdk.util.WindowedValue;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollectionView;
Expand Down Expand Up @@ -76,12 +79,13 @@ public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline) {
*/
private class RDDHolder<T> {

private Iterable<T> values;
private Iterable<WindowedValue<T>> windowedValues;
private Coder<T> coder;
private JavaRDDLike<WindowedValue<T>, ?> rdd;

RDDHolder(Iterable<T> values, Coder<T> coder) {
this.values = values;
this.windowedValues =
Iterables.transform(values, WindowingHelpers.<T>windowValueFunction());
this.coder = coder;
}

Expand All @@ -91,14 +95,6 @@ private class RDDHolder<T> {

JavaRDDLike<WindowedValue<T>, ?> getRDD() {
if (rdd == null) {
Iterable<WindowedValue<T>> windowedValues = Iterables.transform(values,
new Function<T, WindowedValue<T>>() {
@Override
public WindowedValue<T> apply(T t) {
// TODO: this is wrong if T is a TimestampedValue
return WindowedValue.valueInEmptyWindows(t);
}
});
WindowedValue.ValueOnlyWindowedValueCoder<T> windowCoder =
WindowedValue.getValueOnlyCoder(coder);
rdd = jsc.parallelize(CoderHelpers.toByteArrays(windowedValues, windowCoder))
Expand All @@ -107,29 +103,31 @@ public WindowedValue<T> apply(T t) {
return rdd;
}

Iterable<T> getValues(PCollection<T> pcollection) {
if (values == null) {
coder = pcollection.getCoder();
JavaRDDLike<byte[], ?> bytesRDD = rdd.map(WindowingHelpers.<T>unwindowFunction())
.map(CoderHelpers.toByteFunction(coder));
Iterable<WindowedValue<T>> getValues(PCollection<T> pcollection) {
if (windowedValues == null) {
WindowFn<?, ?> windowFn =
pcollection.getWindowingStrategy().getWindowFn();
Coder<? extends BoundedWindow> windowCoder = windowFn.windowCoder();
final WindowedValue.WindowedValueCoder<T> windowedValueCoder;
if (windowFn instanceof GlobalWindows) {
windowedValueCoder =
WindowedValue.ValueOnlyWindowedValueCoder.of(pcollection.getCoder());
} else {
windowedValueCoder =
WindowedValue.FullWindowedValueCoder.of(pcollection.getCoder(), windowCoder);
}
JavaRDDLike<byte[], ?> bytesRDD =
rdd.map(CoderHelpers.toByteFunction(windowedValueCoder));
List<byte[]> clientBytes = bytesRDD.collect();
values = Iterables.transform(clientBytes, new Function<byte[], T>() {
windowedValues = Iterables.transform(clientBytes,
new Function<byte[], WindowedValue<T>>() {
@Override
public T apply(byte[] bytes) {
return CoderHelpers.fromByteArray(bytes, coder);
public WindowedValue<T> apply(byte[] bytes) {
return CoderHelpers.fromByteArray(bytes, windowedValueCoder);
}
});
}
return values;
}

Iterable<WindowedValue<T>> getWindowedValues(PCollection<T> pcollection) {
return Iterables.transform(get(pcollection), new Function<T, WindowedValue<T>>() {
@Override
public WindowedValue<T> apply(T t) {
return WindowedValue.valueInEmptyWindows(t); // TODO: not the right place?
}
});
return windowedValues;
}
}

Expand Down Expand Up @@ -266,13 +264,14 @@ public <T> AggregatorValues<T> getAggregatorValues(Aggregator<?, T> aggregator)
public <T> Iterable<T> get(PCollection<T> pcollection) {
@SuppressWarnings("unchecked")
RDDHolder<T> rddHolder = (RDDHolder<T>) pcollections.get(pcollection);
return rddHolder.getValues(pcollection);
Iterable<WindowedValue<T>> windowedValues = rddHolder.getValues(pcollection);
return Iterables.transform(windowedValues, WindowingHelpers.<T>unwindowValueFunction());
}

<T> Iterable<WindowedValue<T>> getWindowedValues(PCollection<T> pcollection) {
@SuppressWarnings("unchecked")
RDDHolder<T> rddHolder = (RDDHolder<T>) pcollections.get(pcollection);
return rddHolder.getWindowedValues(pcollection);
return rddHolder.getValues(pcollection);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,8 @@ private WindowingHelpers() {
}

/**
* A function for converting a value to a {@link WindowedValue}. The resulting
* {@link WindowedValue} will be in no windows, and will have the default timestamp
* A Spark function for converting a value to a {@link WindowedValue}. The resulting
* {@link WindowedValue} will be in a global windows, and will have the default timestamp == MIN
* and pane.
*
* @param <T> The type of the object.
Expand All @@ -40,13 +40,13 @@ public static <T> Function<T, WindowedValue<T>> windowFunction() {
return new Function<T, WindowedValue<T>>() {
@Override
public WindowedValue<T> call(T t) {
return WindowedValue.valueInEmptyWindows(t);
return WindowedValue.valueInGlobalWindow(t);
}
};
}

/**
* A function for extracting the value from a {@link WindowedValue}.
* A Spark function for extracting the value from a {@link WindowedValue}.
*
* @param <T> The type of the object.
* @return A function that accepts a {@link WindowedValue} and returns its value.
Expand All @@ -59,4 +59,34 @@ public T call(WindowedValue<T> t) {
}
};
}

/**
* Same as windowFunction but for non-RDD values - not an RDD transformation!
*
* @param <T> The type of the object.
* @return A function that accepts an object and returns its {@link WindowedValue}.
*/
public static <T> com.google.common.base.Function<T, WindowedValue<T>> windowValueFunction() {
return new com.google.common.base.Function<T, WindowedValue<T>>() {
@Override
public WindowedValue<T> apply(T t) {
return WindowedValue.valueInGlobalWindow(t);
}
};
}

/**
* Same as unwindowFunction but for non-RDD values - not an RDD transformation!
*
* @param <T> The type of the object.
* @return A function that accepts an object and returns its {@link WindowedValue}.
*/
public static <T> com.google.common.base.Function<WindowedValue<T>, T> unwindowValueFunction() {
return new com.google.common.base.Function<WindowedValue<T>, T>() {
@Override
public T apply(WindowedValue<T> t) {
return t.getValue();
}
};
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,18 @@

package org.apache.beam.runners.spark;

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

import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
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.testing.PAssert;
import org.apache.beam.sdk.transforms.Aggregator;
import org.apache.beam.sdk.transforms.Count;
Expand All @@ -37,8 +40,14 @@
import org.apache.beam.sdk.transforms.Sum;
import org.apache.beam.sdk.values.KV;
import org.apache.beam.sdk.values.PCollection;
import org.apache.commons.io.FileUtils;
import org.junit.rules.TemporaryFolder;
import org.junit.Rule;
import org.junit.Test;

import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.junit.Assert.assertThat;

public class SimpleWordCountTest {
private static final String[] WORDS_ARRAY = {
"hi there", "hi", "hi sue bob",
Expand All @@ -48,7 +57,7 @@ public class SimpleWordCountTest {
ImmutableSet.of("hi: 5", "there: 1", "sue: 2", "bob: 2");

@Test
public void testRun() throws Exception {
public void testInMem() throws Exception {
SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
options.setRunner(SparkPipelineRunner.class);
Pipeline p = Pipeline.create(options);
Expand All @@ -62,6 +71,29 @@ public void testRun() throws Exception {
res.close();
}

@Rule
public TemporaryFolder testFolder = new TemporaryFolder();

@Test
public void testOutputFile() throws Exception {
SparkPipelineOptions options = SparkPipelineOptionsFactory.create();
options.setRunner(SparkPipelineRunner.class);
Pipeline p = Pipeline.create(options);
PCollection<String> inputWords = p.apply(Create.of(WORDS)).setCoder(StringUtf8Coder
Copy link
Member

Choose a reason for hiding this comment

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

Not particularly relevant to the content of your change, but the recommended way to set the coder here would be Create.of(WORDS).withCoder(StringUtf8Coder.of()).

Separately, it isn't entirely necessary: it is a bit of a hack, but the methods in the default Create implementation that infer coders based on the values are made public just so that runner overrides can still invoke them, as the DataflowPipelineRunner does.

I hope the new whole-graph analysis will make it so individual runners no longer deal with this, but until then since it is not too hard to do, you might consider doing it for the Spark runner.

.of());
PCollection<String> output = inputWords.apply(new CountWords());

File outputFile = testFolder.newFile();
output.apply(
TextIO.Write.named("WriteCounts").to(outputFile.getAbsolutePath()).withoutSharding());

EvaluationResult res = SparkPipelineRunner.create().run(p);
res.close();

assertThat(Sets.newHashSet(FileUtils.readLines(outputFile)),
containsInAnyOrder(EXPECTED_COUNT_SET.toArray()));
}

/**
* A DoFn that tokenizes lines of text into individual words.
*/
Expand Down