Skip to content
Merged
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 @@ -30,7 +30,9 @@
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 @@ -59,11 +61,10 @@
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.dstream.ConstantInputDStream;
import org.apache.spark.streaming.api.java.JavaInputDStream;
import scala.Tuple2;
import scala.collection.JavaConverters;

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

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());

Queue<JavaRDD<WindowedValue<byte[]>>> rddQueue = new LinkedBlockingQueue<>();
rddQueue.offer(emptyByteArrayRDD);
JavaInputDStream<WindowedValue<byte[]>> emptyByteArrayStream =
context.getStreamingContext().queueStream(rddQueue, true /* oneAtATime */);
UnboundedDataset<byte[]> output =
new UnboundedDataset<>(stream, Collections.singletonList(inputDStream.id()));
new UnboundedDataset<>(
emptyByteArrayStream,
Collections.singletonList(emptyByteArrayStream.inputDStream().id()));

// Add watermark to holder and advance to infinity to ensure future watermarks can be updated
GlobalWatermarkHolder.SparkWatermarks sparkWatermark =
Expand Down Expand Up @@ -307,11 +305,9 @@ private static <T> void translateFlatten(
List<Integer> streamSources = new ArrayList<>();

if (inputsMap.isEmpty()) {
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());
Queue<JavaRDD<WindowedValue<T>>> q = new LinkedBlockingQueue<>();
q.offer(context.getSparkContext().emptyRDD());
unifiedStreams = context.getStreamingContext().queueStream(q);
} else {
List<JavaDStream<WindowedValue<T>>> dStreams = new ArrayList<>();
for (String inputId : inputsMap.values()) {
Expand All @@ -322,13 +318,11 @@ private static <T> void translateFlatten(
dStreams.add(unboundedDataset.getDStream());
} else {
// create a single RDD stream.
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());

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);
}
}
Expand Down
Loading