You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by ke...@apache.org on 2017/04/21 16:30:19 UTC

[1/2] beam git commit: Inline and remove POutputValueBase

Repository: beam
Updated Branches:
  refs/heads/master 8160924e1 -> ea0f37db7


Inline and remove POutputValueBase


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/66803115
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/66803115
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/66803115

Branch: refs/heads/master
Commit: 66803115d3aaa225881137307486a955d1e0dc7b
Parents: 2866549
Author: Kenneth Knowles <kl...@google.com>
Authored: Thu Apr 20 18:47:57 2017 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Thu Apr 20 19:03:19 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/values/PDone.java  | 30 ++++++--
 .../beam/sdk/values/POutputValueBase.java       | 74 --------------------
 .../java/org/apache/beam/sdk/values/PValue.java |  2 +
 .../org/apache/beam/sdk/values/PValueBase.java  | 29 ++++++--
 .../beam/sdk/io/gcp/bigquery/WriteResult.java   | 37 +++++++++-
 5 files changed, 83 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam/blob/66803115/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java
index 1e5c4dc..5c9800d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PDone.java
@@ -21,13 +21,20 @@ import java.util.Collections;
 import java.util.Map;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.WriteFiles;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
 
 /**
  * {@link PDone} is the output of a {@link PTransform} that has a trivial result,
  * such as a {@link WriteFiles}.
  */
-public class PDone extends POutputValueBase {
+public class PDone implements POutput {
+
+  private final Pipeline pipeline;
+
+  private PDone(Pipeline pipeline) {
+    this.pipeline = pipeline;
+  }
 
   /**
    * Creates a {@link PDone} in the given {@link Pipeline}.
@@ -37,12 +44,25 @@ public class PDone extends POutputValueBase {
   }
 
   @Override
+  public Pipeline getPipeline() {
+    return pipeline;
+  }
+
+  /**
+   * A {@link PDone} contains no {@link PValue PValues}.
+   */
+  @Override
   public Map<TupleTag<?>, PValue> expand() {
-    // A PDone contains no PValues.
     return Collections.emptyMap();
   }
 
-  private PDone(Pipeline pipeline) {
-    super(pipeline);
-  }
+  /** Does nothing; there are no concrete outputs to record. */
+  @Override
+  public void recordAsOutput(AppliedPTransform<?, ?, ?> transform) {}
+
+  /**
+   * Does nothing; there is nothing to finish specifying.
+   */
+  @Override
+  public void finishSpecifyingOutput(PInput input, PTransform<?, ?> transform) { }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/66803115/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutputValueBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutputValueBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutputValueBase.java
deleted file mode 100644
index 5bd424d..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/POutputValueBase.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * 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.sdk.values;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.transforms.AppliedPTransform;
-import org.apache.beam.sdk.transforms.PTransform;
-
-/**
- * A {@link POutputValueBase} is the abstract base class of
- * {@code PTransform} outputs.
- *
- * <p>A {@link PValueBase} that adds tracking of its producing
- * {@link AppliedPTransform}.
- *
- * <p>For internal use.
- */
-public abstract class POutputValueBase implements POutput {
-
-  private final Pipeline pipeline;
-
-  protected POutputValueBase(Pipeline pipeline) {
-    this.pipeline = pipeline;
-  }
-
-  /**
-   * No-arg constructor for Java serialization only.
-   * The resulting {@link POutputValueBase} is unlikely to be
-   * valid.
-   */
-  protected POutputValueBase() {
-    pipeline = null;
-  }
-
-  @Override
-  public Pipeline getPipeline() {
-    return pipeline;
-  }
-
-  /**
-   * Records that this {@link POutputValueBase} is an output with the given name of the given {@link
-   * AppliedPTransform}.
-   *
-   * <p>By default, does nothing.
-   *
-   * <p>To be invoked only by {@link POutput#recordAsOutput} implementations. Not to be invoked
-   * directly by user code.
-   */
-  @Override
-  public void recordAsOutput(AppliedPTransform<?, ?, ?> transform) {}
-
-  /**
-   * Default behavior for {@link #finishSpecifyingOutput(PInput, PTransform)}} is
-   * to do nothing. Override if your {@link PValue} requires
-   * finalization.
-   */
-  @Override
-  public void finishSpecifyingOutput(PInput input, PTransform<?, ?> transform) { }
-}

http://git-wip-us.apache.org/repos/asf/beam/blob/66803115/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java
index 06546aa..d9f6920 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java
@@ -22,6 +22,8 @@ import org.apache.beam.sdk.transforms.PTransform;
 
 /**
  * The interface for values that can be input to and output from {@link PTransform PTransforms}.
+ *
+ * <p>It is recommended to extend {@link PValueBase}
  */
 public interface PValue extends POutput, PInput {
 

http://git-wip-us.apache.org/repos/asf/beam/blob/66803115/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java
index 91ee392..9f151ec 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java
@@ -37,7 +37,10 @@ import org.apache.beam.sdk.util.NameUtils;
  *
  * <p>For internal use.
  */
-public abstract class PValueBase extends POutputValueBase implements PValue {
+public abstract class PValueBase implements PValue {
+
+  private final Pipeline pipeline;
+
   /**
    * Returns the name of this {@link PValueBase}.
    *
@@ -56,7 +59,7 @@ public abstract class PValueBase extends POutputValueBase implements PValue {
   }
 
   /**
-   * Sets the name of this {@link PValueBase}.  Returns {@code this}.
+   * Sets the name of this {@link PValueBase}. Returns {@code this}.
    *
    * @throws IllegalStateException if this {@link PValueBase} has
    * already been finalized and may no longer be set.
@@ -73,7 +76,7 @@ public abstract class PValueBase extends POutputValueBase implements PValue {
   /////////////////////////////////////////////////////////////////////////////
 
   protected PValueBase(Pipeline pipeline) {
-    super(pipeline);
+    this.pipeline = pipeline;
   }
 
   /**
@@ -82,11 +85,11 @@ public abstract class PValueBase extends POutputValueBase implements PValue {
    * valid.
    */
   protected PValueBase() {
-    super();
+    this.pipeline = null;
   }
 
   /**
-   * The name of this {@link PValueBase}, or null if not yet set.
+   * The name of this {@link PValueBase}, or {@code null} if not yet set.
    */
   private String name;
 
@@ -107,7 +110,7 @@ public abstract class PValueBase extends POutputValueBase implements PValue {
   }
 
   /**
-   * Records that this {@link POutputValueBase} is an output with the
+   * Records that this {@link PValueBase} is an output with the
    * given name of the given {@link AppliedPTransform} in the given
    * {@link Pipeline}.
    *
@@ -116,7 +119,6 @@ public abstract class PValueBase extends POutputValueBase implements PValue {
    */
   protected void recordAsOutput(AppliedPTransform<?, ?, ?> transform,
                                 String outName) {
-    super.recordAsOutput(transform);
     if (name == null) {
       name = transform.getFullName() + "." + outName;
     }
@@ -157,4 +159,17 @@ public abstract class PValueBase extends POutputValueBase implements PValue {
   protected String getKindString() {
     return NameUtils.approximateSimpleName(getClass());
   }
+
+  @Override
+  public Pipeline getPipeline() {
+    return pipeline;
+  }
+
+  /**
+   * Default behavior for {@link #finishSpecifyingOutput(PInput, PTransform)}} is
+   * to do nothing. Override if your {@link PValue} requires
+   * finalization.
+   */
+  @Override
+  public void finishSpecifyingOutput(PInput input, PTransform<?, ?> transform) { }
 }

http://git-wip-us.apache.org/repos/asf/beam/blob/66803115/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java
----------------------------------------------------------------------
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java
index 3e0f51c..d137f05 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java
@@ -20,14 +20,20 @@ package org.apache.beam.sdk.io.gcp.bigquery;
 import java.util.Collections;
 import java.util.Map;
 import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.values.POutputValueBase;
+import org.apache.beam.sdk.transforms.AppliedPTransform;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 
 /**
  * The result of a {@link BigQueryIO.Write} transform.
  */
-final class WriteResult extends POutputValueBase {
+final class WriteResult implements POutput {
+
+  private final Pipeline pipeline;
+
   /**
    * Creates a {@link WriteResult} in the given {@link Pipeline}.
    */
@@ -41,6 +47,31 @@ final class WriteResult extends POutputValueBase {
   }
 
   private WriteResult(Pipeline pipeline) {
-    super(pipeline);
+    this.pipeline = pipeline;
   }
+
+  @Override
+  public Pipeline getPipeline() {
+    return pipeline;
+  }
+
+  /**
+   * Records that this {@link WriteResult} is an output with the given name of the given {@link
+   * AppliedPTransform}.
+   *
+   * <p>By default, does nothing.
+   *
+   * <p>To be invoked only by {@link POutput#recordAsOutput} implementations. Not to be invoked
+   * directly by user code.
+   */
+  @Override
+  public void recordAsOutput(AppliedPTransform<?, ?, ?> transform) {}
+
+  /**
+   * Default behavior for {@link #finishSpecifyingOutput(PInput, PTransform)}} is
+   * to do nothing. Override if your {@link PValue} requires
+   * finalization.
+   */
+  @Override
+  public void finishSpecifyingOutput(PInput input, PTransform<?, ?> transform) { }
 }


[2/2] beam git commit: This closes #2623: Inline and remove POutputValueBase

Posted by ke...@apache.org.
This closes #2623: Inline and remove POutputValueBase


Project: http://git-wip-us.apache.org/repos/asf/beam/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/ea0f37db
Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/ea0f37db
Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/ea0f37db

Branch: refs/heads/master
Commit: ea0f37db76b18efa3b7d145e0168e3d3e4bede28
Parents: 8160924 6680311
Author: Kenneth Knowles <kl...@google.com>
Authored: Fri Apr 21 09:30:01 2017 -0700
Committer: Kenneth Knowles <kl...@google.com>
Committed: Fri Apr 21 09:30:01 2017 -0700

----------------------------------------------------------------------
 .../java/org/apache/beam/sdk/values/PDone.java  | 30 ++++++--
 .../beam/sdk/values/POutputValueBase.java       | 74 --------------------
 .../java/org/apache/beam/sdk/values/PValue.java |  2 +
 .../org/apache/beam/sdk/values/PValueBase.java  | 29 ++++++--
 .../beam/sdk/io/gcp/bigquery/WriteResult.java   | 37 +++++++++-
 5 files changed, 83 insertions(+), 89 deletions(-)
----------------------------------------------------------------------