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 @@ -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;
Expand Down Expand Up @@ -124,6 +126,9 @@ public <OT extends POutput, IT extends PInput> 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);
}
Expand Down
Original file line number Diff line number Diff line change
@@ -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<T> extends PTransform<PInput, PCollection<T>> {
Copy link
Member

Choose a reason for hiding this comment

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

Why the name change?

Copy link
Member Author

Choose a reason for hiding this comment

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

?

This could be a more specific override (probably should be, in order to facilitate quick removal), but as written is a relatively general override

Copy link
Member

Choose a reason for hiding this comment

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

I'm actually a bit confused as to how/whether this works. The translator is expecting Create.Values

I think the two options are:

  1. Wait until the Spark runner supports Read.
  2. Override Create.Values to a Spark-specific clone of it as here, but alter the translator to translate the new class.

Am I missing something?

private PTransform<PInput, PCollection<T>> transform;

public SinglePrimitiveOutputPTransform(PTransform<PInput, PCollection<T>> transform) {
this.transform = transform;
}

@Override
public PCollection<T> apply(PInput input) {
try {
PCollection<T> collection = PCollection.<T>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);
}
}
}