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 @@ -5,5 +5,6 @@
"https://github.com/apache/beam/pull/32546": "noting that PR #32546 should run this test",
"https://github.com/apache/beam/pull/33267": "noting that PR #33267 should run this test",
"https://github.com/apache/beam/pull/33322": "noting that PR #33322 should run this test",
"https://github.com/apache/beam/pull/34123": "noting that PR #34123 should run this test"
"https://github.com/apache/beam/pull/34123": "noting that PR #34123 should run this test",
"https://github.com/apache/beam/pull/34080": "noting that PR #34080 should run this test"
}
Original file line number Diff line number Diff line change
Expand Up @@ -4,5 +4,6 @@
"https://github.com/apache/beam/pull/31798": "noting that PR #31798 should run this test",
"https://github.com/apache/beam/pull/32546": "noting that PR #32546 should run this test",
"https://github.com/apache/beam/pull/33267": "noting that PR #33267 should run this test",
"https://github.com/apache/beam/pull/34123": "noting that PR #34123 should run this test"
"https://github.com/apache/beam/pull/34123": "noting that PR #34123 should run this test",
"https://github.com/apache/beam/pull/34080": "noting that PR #34080 should run this test"
}
Original file line number Diff line number Diff line change
Expand Up @@ -4,5 +4,6 @@
"https://github.com/apache/beam/pull/31798": "noting that PR #31798 should run this test",
"https://github.com/apache/beam/pull/32546": "noting that PR #32546 should run this test",
"https://github.com/apache/beam/pull/33267": "noting that PR #33267 should run this test",
"https://github.com/apache/beam/pull/33322": "noting that PR #33322 should run this test"
"https://github.com/apache/beam/pull/33322": "noting that PR #33322 should run this test",
"https://github.com/apache/beam/pull/34080": "noting that PR #34080 should run this test"
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,12 @@ public interface TestSparkPipelineOptions extends SparkPipelineOptions, TestPipe

void setStopPipelineWatermark(Long stopPipelineWatermark);

@Description("Whether to delete the checkpoint directory after the pipeline execution.")
@Default.Boolean(true)
boolean isDeleteCheckpointDir();

void setDeleteCheckpointDir(boolean deleteCheckpointDir);

/**
* A factory to provide the default watermark to stop a pipeline that reads from an unbounded
* source.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -106,8 +106,10 @@ public SparkPipelineResult run(Pipeline pipeline) {
isOneOf(PipelineResult.State.STOPPED, PipelineResult.State.DONE));
} finally {
try {
// cleanup checkpoint dir.
FileUtils.deleteDirectory(new File(testSparkOptions.getCheckpointDir()));
if (testSparkOptions.isDeleteCheckpointDir()) {
// cleanup checkpoint dir.
FileUtils.deleteDirectory(new File(testSparkOptions.getCheckpointDir()));
}
} catch (IOException e) {
throw new RuntimeException("Failed to clear checkpoint tmp dir.", e);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* 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.translation;

import org.apache.spark.api.java.JavaSparkContext$;
import org.apache.spark.rdd.RDD;
import org.apache.spark.streaming.StreamingContext;
import org.apache.spark.streaming.Time;
import org.apache.spark.streaming.dstream.ConstantInputDStream;
import org.apache.spark.streaming.dstream.QueueInputDStream;
import scala.Option;

/**
* A specialized {@link ConstantInputDStream} that emits its RDD exactly once. Alternative to {@link
* QueueInputDStream} when checkpointing is required.
*
* <p>Features:
*
* <ul>
* <li>Supports checkpointing
* <li>Guarantees single emission of data
* <li>Returns empty RDD after first emission
* </ul>
*
* @param <T> The type of elements in the RDD
*/
public class SingleEmitInputDStream<T> extends ConstantInputDStream<T> {

private boolean emitted = false;

public SingleEmitInputDStream(StreamingContext ssc, RDD<T> rdd) {
super(ssc, rdd, JavaSparkContext$.MODULE$.fakeClassTag());
}

@Override
public Option<RDD<T>> compute(Time validTime) {
if (this.emitted) {
return Option.apply(this.emptyRDD());
} else {
this.emitted = true;
return super.compute(validTime);
}
}

private RDD<T> emptyRDD() {
return this.context().sparkContext().emptyRDD(JavaSparkContext$.MODULE$.fakeClassTag());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,7 @@
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.LinkedBlockingQueue;
import org.apache.beam.model.pipeline.v1.RunnerApi;
import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
import org.apache.beam.runners.spark.SparkPipelineOptions;
Expand Down Expand Up @@ -61,10 +59,11 @@
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.JavaSparkContext$;
import org.apache.spark.broadcast.Broadcast;
import org.apache.spark.storage.StorageLevel;
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaInputDStream;
import org.apache.spark.streaming.dstream.ConstantInputDStream;
import scala.Tuple2;
import scala.collection.JavaConverters;

Expand Down Expand Up @@ -158,15 +157,17 @@ private static void translateImpulse(
.parallelize(CoderHelpers.toByteArrays(windowedValues, windowCoder))
.map(CoderHelpers.fromByteFunction(windowCoder));

Queue<JavaRDD<WindowedValue<byte[]>>> rddQueue = new LinkedBlockingQueue<>();
rddQueue.offer(emptyByteArrayRDD);
JavaInputDStream<WindowedValue<byte[]>> emptyByteArrayStream =
context.getStreamingContext().queueStream(rddQueue, true /* oneAtATime */);
final ConstantInputDStream<WindowedValue<byte[]>> inputDStream =
new ConstantInputDStream<>(
context.getStreamingContext().ssc(),
emptyByteArrayRDD.rdd(),
JavaSparkContext$.MODULE$.fakeClassTag());

final JavaDStream<WindowedValue<byte[]>> stream =
JavaDStream.fromDStream(inputDStream, JavaSparkContext$.MODULE$.fakeClassTag());

UnboundedDataset<byte[]> output =
new UnboundedDataset<>(
emptyByteArrayStream,
Collections.singletonList(emptyByteArrayStream.inputDStream().id()));
new UnboundedDataset<>(stream, Collections.singletonList(inputDStream.id()));

// Add watermark to holder and advance to infinity to ensure future watermarks can be updated
GlobalWatermarkHolder.SparkWatermarks sparkWatermark =
Expand All @@ -175,7 +176,6 @@ private static void translateImpulse(
BoundedWindow.TIMESTAMP_MAX_VALUE,
context.getFirstTimestamp());
GlobalWatermarkHolder.add(output.getStreamSources().get(0), sparkWatermark);

context.pushDataset(getOutputId(transformNode), output);
}

Expand Down Expand Up @@ -297,6 +297,7 @@ public void setName(String name) {
}
}

@SuppressWarnings("unchecked")
private static <T> void translateFlatten(
PTransformNode transformNode,
RunnerApi.Pipeline pipeline,
Expand All @@ -306,9 +307,11 @@ private static <T> void translateFlatten(
List<Integer> streamSources = new ArrayList<>();

if (inputsMap.isEmpty()) {
Queue<JavaRDD<WindowedValue<T>>> q = new LinkedBlockingQueue<>();
q.offer(context.getSparkContext().emptyRDD());
unifiedStreams = context.getStreamingContext().queueStream(q);
final JavaRDD<WindowedValue<T>> emptyRDD = context.getSparkContext().emptyRDD();
final SingleEmitInputDStream<WindowedValue<T>> singleEmitInputDStream =
new SingleEmitInputDStream<>(context.getStreamingContext().ssc(), emptyRDD.rdd());
unifiedStreams =
JavaDStream.fromDStream(singleEmitInputDStream, JavaSparkContext$.MODULE$.fakeClassTag());
} else {
List<JavaDStream<WindowedValue<T>>> dStreams = new ArrayList<>();
for (String inputId : inputsMap.values()) {
Expand All @@ -319,11 +322,13 @@ private static <T> void translateFlatten(
dStreams.add(unboundedDataset.getDStream());
} else {
// create a single RDD stream.
Queue<JavaRDD<WindowedValue<T>>> q = new LinkedBlockingQueue<>();
q.offer(((BoundedDataset) dataset).getRDD());
// TODO (https://github.com/apache/beam/issues/20426): this is not recoverable from
// checkpoint!
JavaDStream<WindowedValue<T>> dStream = context.getStreamingContext().queueStream(q);
final SingleEmitInputDStream<WindowedValue<T>> singleEmitInputDStream =
new SingleEmitInputDStream<WindowedValue<T>>(
context.getStreamingContext().ssc(), ((BoundedDataset) dataset).getRDD().rdd());
final JavaDStream<WindowedValue<T>> dStream =
JavaDStream.fromDStream(
singleEmitInputDStream, JavaSparkContext$.MODULE$.fakeClassTag());

dStreams.add(dStream);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import org.apache.beam.runners.spark.translation.EvaluationContext;
import org.apache.beam.runners.spark.translation.GroupCombineFunctions;
import org.apache.beam.runners.spark.translation.MultiDoFnFunction;
import org.apache.beam.runners.spark.translation.SingleEmitInputDStream;
import org.apache.beam.runners.spark.translation.SparkAssignWindowFn;
import org.apache.beam.runners.spark.translation.SparkCombineFn;
import org.apache.beam.runners.spark.translation.SparkPCollectionView;
Expand Down Expand Up @@ -88,6 +89,7 @@
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.JavaSparkContext$;
import org.apache.spark.streaming.StreamingContext;
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaInputDStream;
import org.apache.spark.streaming.api.java.JavaPairDStream;
Expand Down Expand Up @@ -291,12 +293,8 @@ public void evaluate(Flatten.PCollections<T> transform, EvaluationContext contex
dStreams.add(unboundedDataset.getDStream());
} else {
// create a single RDD stream.
Queue<JavaRDD<WindowedValue<T>>> q = new LinkedBlockingQueue<>();
q.offer(((BoundedDataset) dataset).getRDD());
// TODO (https://github.com/apache/beam/issues/20426): this is not recoverable from
// checkpoint!
JavaDStream<WindowedValue<T>> dStream = context.getStreamingContext().queueStream(q);
dStreams.add(dStream);
dStreams.add(
this.buildDStream(context.getStreamingContext().ssc(), (BoundedDataset) dataset));
}
}
// start by unifying streams into a single stream.
Expand All @@ -305,6 +303,15 @@ public void evaluate(Flatten.PCollections<T> transform, EvaluationContext contex
context.putDataset(transform, new UnboundedDataset<>(unifiedStreams, streamingSources));
}

private JavaDStream<WindowedValue<T>> buildDStream(
final StreamingContext ssc, final BoundedDataset<T> dataset) {

final SingleEmitInputDStream<WindowedValue<T>> singleEmitDStream =
new SingleEmitInputDStream<>(ssc, dataset.getRDD().rdd());

return JavaDStream.fromDStream(singleEmitDStream, JavaSparkContext$.MODULE$.fakeClassTag());
}

@Override
public String toNativeString() {
return "streamingContext.union(...)";
Expand Down
Loading
Loading