beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From dhalp...@apache.org
Subject [1/2] incubator-beam git commit: Improve BigQueryIO validation for streaming WriteDisposition
Date Thu, 16 Jun 2016 06:43:49 GMT
Repository: incubator-beam
Updated Branches:
  refs/heads/master f8c59bd6c -> 3b66f5919


Improve BigQueryIO validation for streaming WriteDisposition


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

Branch: refs/heads/master
Commit: df9a78b9ae67f9d05da1fcd000a85c412c4bcbd9
Parents: f8c59bd
Author: Scott Wegner <swegner@google.com>
Authored: Mon Jun 13 11:05:52 2016 -0700
Committer: Dan Halperin <dhalperi@google.com>
Committed: Wed Jun 15 23:43:36 2016 -0700

----------------------------------------------------------------------
 .../core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java  | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/df9a78b9/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
----------------------------------------------------------------------
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
index 38009bf..1c666ed 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BigQueryIO.java
@@ -201,8 +201,8 @@ import javax.annotation.Nullable;
  *
  * <p>See {@link BigQueryIO.Write} for details on how to specify if a write should
  * append to an existing table, replace the table, or verify that the table is
- * empty. Note that the dataset being written to must already exist. Write
- * dispositions are not supported in streaming mode.
+ * empty. Note that the dataset being written to must already exist. Unbounded PCollections
can only
+ * be written using {@link WriteDisposition#WRITE_EMPTY} or {@link WriteDisposition#WRITE_APPEND}.
  *
  * <h3>Sharding BigQuery output tables</h3>
  * <p>A common use case is to dynamically generate BigQuery table names based on
@@ -1674,7 +1674,7 @@ public class BigQueryIO {
           }
         }
 
-        if (options.isStreaming() || tableRefFunction != null) {
+        if (input.isBounded() == PCollection.IsBounded.UNBOUNDED || tableRefFunction != null)
{
           // We will use BigQuery's streaming write API -- validate supported dispositions.
           checkArgument(
               createDisposition != CreateDisposition.CREATE_NEVER,


Mime
View raw message