From f98addcec327bfd41ce357f61d0fc5136b882e71 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 19 Apr 2016 16:12:32 -0700 Subject: [PATCH] Override Create in the SparkPipelineRunner This allows existing pipelines to continue to function by keeping the graph structure identical while replacing Create with a Read. --- .../runners/spark/SparkPipelineRunner.java | 5 ++ .../util/SinglePrimitiveOutputPTransform.java | 48 +++++++++++++++++++ 2 files changed, 53 insertions(+) create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/util/SinglePrimitiveOutputPTransform.java diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java index 2b33e7acc965..8635cfb60331 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java @@ -27,12 +27,14 @@ import org.apache.beam.runners.spark.translation.streaming.StreamingEvaluationContext; import org.apache.beam.runners.spark.translation.streaming.StreamingTransformTranslator; import org.apache.beam.runners.spark.translation.streaming.StreamingWindowPipelineDetector; +import org.apache.beam.runners.spark.util.SinglePrimitiveOutputPTransform; 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; +import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly; @@ -124,6 +126,9 @@ public OT apply( if (transform instanceof GroupByKey) { return (OT) ((PCollection) input).apply( new GroupByKeyViaGroupByKeyOnly((GroupByKey) transform)); + } else if (transform instanceof Create.Values) { + return (OT) super.apply( + new SinglePrimitiveOutputPTransform((Create.Values) transform), input); } else { return super.apply(transform, input); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SinglePrimitiveOutputPTransform.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SinglePrimitiveOutputPTransform.java new file mode 100644 index 000000000000..9a8aa2ed15cc --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SinglePrimitiveOutputPTransform.java @@ -0,0 +1,48 @@ +/* + * 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.util; + +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.apache.beam.sdk.values.PInput; + +public class SinglePrimitiveOutputPTransform extends PTransform> { + private PTransform> transform; + + public SinglePrimitiveOutputPTransform(PTransform> transform) { + this.transform = transform; + } + + @Override + public PCollection apply(PInput input) { + try { + PCollection collection = PCollection.createPrimitiveOutputInternal( + input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + collection.setCoder(transform.getDefaultOutputCoder(input, collection)); + return collection; + } catch (CannotProvideCoderException e) { + throw new IllegalArgumentException( + "Unable to infer a coder and no Coder was specified. " + + "Please set a coder by invoking Create.withCoder() explicitly.", + e); + } + } +}