You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by lc...@apache.org on 2016/07/06 17:20:41 UTC
[35/50] [abbrv] incubator-beam git commit: Make PTransform.apply
Abstract
Make PTransform.apply Abstract
Primitive PTransforms should create the appropriate primitive output
type. Composite PTransforms should override apply.
Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/9a17d496
Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/9a17d496
Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/9a17d496
Branch: refs/heads/runners-spark2
Commit: 9a17d496bc67b0ebd601b717cc810d290142481f
Parents: 8f7e24a
Author: Thomas Groh <tg...@google.com>
Authored: Wed Jun 29 16:12:21 2016 -0700
Committer: Luke Cwik <lc...@google.com>
Committed: Wed Jul 6 10:18:52 2016 -0700
----------------------------------------------------------------------
.../runners/direct/CommittedResultTest.java | 4 ++++
.../apache/beam/sdk/transforms/PTransform.java | 20 ++++----------------
.../beam/sdk/transforms/PTransformTest.java | 7 ++++++-
.../display/DisplayDataMatchersTest.java | 18 ++++++++++++++++--
.../sdk/transforms/display/DisplayDataTest.java | 6 +++++-
.../apache/beam/sdk/util/StringUtilsTest.java | 19 +++++++++++++++++--
6 files changed, 52 insertions(+), 22 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9a17d496/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
----------------------------------------------------------------------
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
index 1e51b55..c0c06e5 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
@@ -52,6 +52,10 @@ public class CommittedResultTest implements Serializable {
private transient PCollection<Integer> created = p.apply(Create.of(1, 2));
private transient AppliedPTransform<?, ?, ?> transform =
AppliedPTransform.of("foo", p.begin(), PDone.in(p), new PTransform<PBegin, PDone>() {
+ @Override
+ public PDone apply(PBegin begin) {
+ throw new IllegalArgumentException("Should never be applied");
+ }
});
private transient BundleFactory bundleFactory = ImmutableListBundleFactory.create();
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9a17d496/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java
index a56eefc..fe6e8ad 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PTransform.java
@@ -181,18 +181,8 @@ public abstract class PTransform<InputT extends PInput, OutputT extends POutput>
* transforms, which do not apply any transforms internally, should return
* a new unbound output and register evaluators (via backend-specific
* registration methods).
- *
- * <p>The default implementation throws an exception. A derived class must
- * either implement apply, or else each runner must supply a custom
- * implementation via
- * {@link org.apache.beam.sdk.runners.PipelineRunner#apply}.
*/
- public OutputT apply(InputT input) {
- throw new IllegalArgumentException(
- "Runner " + input.getPipeline().getRunner()
- + " has not registered an implementation for the required primitive operation "
- + this);
- }
+ public abstract OutputT apply(InputT input);
/**
* Called before invoking apply (which may be intercepted by the runner) to
@@ -201,7 +191,7 @@ public abstract class PTransform<InputT extends PInput, OutputT extends POutput>
*
* <p>By default, does nothing.
*/
- public void validate(InputT input) { }
+ public void validate(InputT input) {}
/**
* Returns the transform name.
@@ -279,8 +269,7 @@ public abstract class PTransform<InputT extends PInput, OutputT extends POutput>
* @throws CannotProvideCoderException if no coder can be inferred
*/
protected Coder<?> getDefaultOutputCoder() throws CannotProvideCoderException {
- throw new CannotProvideCoderException(
- "PTransform.getDefaultOutputCoder called.");
+ throw new CannotProvideCoderException("PTransform.getDefaultOutputCoder called.");
}
/**
@@ -319,6 +308,5 @@ public abstract class PTransform<InputT extends PInput, OutputT extends POutput>
* to provide their own display data.
*/
@Override
- public void populateDisplayData(Builder builder) {
- }
+ public void populateDisplayData(Builder builder) {}
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9a17d496/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java
index 0ef13c5..f7074c3 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java
@@ -35,7 +35,12 @@ public class PTransformTest {
@Test
public void testPopulateDisplayDataDefaultBehavior() {
PTransform<PCollection<String>, PCollection<String>> transform =
- new PTransform<PCollection<String>, PCollection<String>>() {};
+ new PTransform<PCollection<String>, PCollection<String>>() {
+ @Override
+ public PCollection<String> apply(PCollection<String> begin) {
+ throw new IllegalArgumentException("Should never be applied");
+ }
+ };
DisplayData displayData = DisplayData.from(transform);
assertThat(displayData.items(), empty());
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9a17d496/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java
index f9f2911..07029e9 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchersTest.java
@@ -22,7 +22,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasName
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType;
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue;
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom;
-
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
@@ -79,6 +78,11 @@ public class DisplayDataMatchersTest {
DisplayData data = DisplayData.from(new PTransform<PCollection<String>, PCollection<String>>() {
@Override
+ public PCollection<String> apply(PCollection<String> input) {
+ throw new IllegalArgumentException("Should never be applied");
+ }
+
+ @Override
public void populateDisplayData(Builder builder) {
builder.add(DisplayData.item("foo", DisplayDataMatchersTest.class));
}
@@ -101,7 +105,12 @@ public class DisplayDataMatchersTest {
Matcher<DisplayData> matcher = hasDisplayItem(hasNamespace(SampleTransform.class));
assertFalse(matcher.matches(DisplayData.from(
- new PTransform<PCollection<String>, PCollection<String>>(){})));
+ new PTransform<PCollection<String>, PCollection<String>>(){
+ @Override
+ public PCollection<String> apply(PCollection<String> input) {
+ throw new IllegalArgumentException("Should never be applied");
+ }
+ })));
assertThat(createDisplayDataWithItem("foo", "bar"), matcher);
}
@@ -149,6 +158,11 @@ public class DisplayDataMatchersTest {
}
@Override
+ public PCollection<String> apply(PCollection<String> input) {
+ throw new IllegalArgumentException("Should never be applied");
+ }
+
+ @Override
public void populateDisplayData(Builder builder) {
builder.add(DisplayData.item(key, value));
}
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9a17d496/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java
index 88973ff..a1189bb 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java
@@ -23,7 +23,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasName
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType;
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue;
import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom;
-
import static org.hamcrest.Matchers.allOf;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.everyItem;
@@ -112,6 +111,11 @@ public class DisplayDataTest implements Serializable {
Instant startTime = defaultStartTime;
@Override
+ public PCollection<String> apply(PCollection<String> begin) {
+ throw new IllegalArgumentException("Should never be applied");
+ }
+
+ @Override
public void populateDisplayData(DisplayData.Builder builder) {
builder
.include(subComponent1)
http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/9a17d496/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java
index 408af83..7e68df9 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StringUtilsTest.java
@@ -69,7 +69,17 @@ public class StringUtilsTest {
}
private class EmbeddedPTransform extends PTransform<PBegin, PDone> {
- private class Bound extends PTransform<PBegin, PDone> {}
+ @Override
+ public PDone apply(PBegin begin) {
+ throw new IllegalArgumentException("Should never be applied");
+ }
+
+ private class Bound extends PTransform<PBegin, PDone> {
+ @Override
+ public PDone apply(PBegin begin) {
+ throw new IllegalArgumentException("Should never be applied");
+ }
+ }
private Bound getBound() {
return new Bound();
@@ -118,7 +128,12 @@ public class StringUtilsTest {
@Test
public void testPTransformNameWithAnonOuterClass() throws Exception {
AnonymousClass anonymousClassObj = new AnonymousClass() {
- class NamedInnerClass extends PTransform<PBegin, PDone> {}
+ class NamedInnerClass extends PTransform<PBegin, PDone> {
+ @Override
+ public PDone apply(PBegin begin) {
+ throw new IllegalArgumentException("Should never be applied");
+ }
+ }
@Override
public Object getInnerClassInstance() {