[jira] [Work logged] (BEAM-3433) Allow BigQueryIO to use a different project for the load job in batch mode.

2018-05-17 Thread ASF GitHub Bot (JIRA)

 [ 
https://issues.apache.org/jira/browse/BEAM-3433?focusedWorklogId=103015=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-103015
 ]

ASF GitHub Bot logged work on BEAM-3433:


Author: ASF GitHub Bot
Created on: 17/May/18 17:53
Start Date: 17/May/18 17:53
Worklog Time Spent: 10m 
  Work Description: jkff closed pull request #5178: [BEAM-3433] Allow a GCP 
project to be explicitly set for a load job
URL: https://github.com/apache/beam/pull/5178
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java
 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java
index c3f3134523d..daebf47c5eb 100644
--- 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java
+++ 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java
@@ -28,6 +28,7 @@
 import com.google.common.collect.Lists;
 import java.util.List;
 import java.util.concurrent.ThreadLocalRandom;
+import javax.annotation.Nullable;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.IterableCoder;
@@ -73,6 +74,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+
 /** PTransform that uses BigQuery batch-load jobs to write a PCollection to 
BigQuery. */
 class BatchLoads
 extends PTransform>, WriteResult> {
@@ -127,12 +129,14 @@
   private int numFileShards;
   private Duration triggeringFrequency;
   private ValueProvider customGcsTempLocation;
+  private ValueProvider loadJobProjectId;
 
   BatchLoads(WriteDisposition writeDisposition, CreateDisposition 
createDisposition,
  boolean singletonTable,
  DynamicDestinations dynamicDestinations,
  Coder destinationCoder,
- ValueProvider customGcsTempLocation) {
+ ValueProvider customGcsTempLocation,
+ @Nullable ValueProvider loadJobProjectId) {
 bigQueryServices = new BigQueryServicesImpl();
 this.writeDisposition = writeDisposition;
 this.createDisposition = createDisposition;
@@ -144,6 +148,7 @@
 this.numFileShards = DEFAULT_NUM_FILE_SHARDS;
 this.triggeringFrequency = null;
 this.customGcsTempLocation = customGcsTempLocation;
+this.loadJobProjectId = loadJobProjectId;
   }
 
   void setTestServices(BigQueryServices bigQueryServices) {
@@ -507,7 +512,8 @@ public void processElement(ProcessContext c) {
 WriteDisposition.WRITE_EMPTY,
 CreateDisposition.CREATE_IF_NEEDED,
 sideInputs,
-dynamicDestinations));
+dynamicDestinations,
+loadJobProjectId));
   }
 
   // In the case where the files fit into a single load job, there's no need 
to write temporary
@@ -536,7 +542,8 @@ void writeSinglePartition(
 writeDisposition,
 createDisposition,
 sideInputs,
-dynamicDestinations));
+dynamicDestinations,
+loadJobProjectId));
   }
 
   private WriteResult writeResult(Pipeline p) {
diff --git 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
index e9eb3cb4fd3..e5aa287b2b7 100644
--- 
a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
+++ 
b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
@@ -1122,6 +1122,8 @@ static String getExtractDestinationUri(String 
extractDestinationDir) {
 
 abstract Method getMethod();
 
+@Nullable abstract ValueProvider getLoadJobProjectId();
+
 @Nullable abstract InsertRetryPolicy getFailedInsertRetryPolicy();
 
 @Nullable abstract ValueProvider getCustomGcsTempLocation();
@@ -1151,6 +1153,7 @@ static String getExtractDestinationUri(String 
extractDestinationDir) {
   abstract Builder setTriggeringFrequency(Duration triggeringFrequency);
 
   abstract Builder setMethod(Method method);
+  abstract Builder setLoadJobProjectId(ValueProvider 
loadJobProjectId);
 
   abstract Builder setFailedInsertRetryPolicy(InsertRetryPolicy 
retryPolicy);
 
@@ -1391,6 +1394,20 @@ static String getExtractDestinationUri(String 
extractDestinationDir) {
   return toBuilder().setMethod(method).build();
 }
 
+/**
+ * Set the 

[jira] [Work logged] (BEAM-3433) Allow BigQueryIO to use a different project for the load job in batch mode.

2018-05-16 Thread ASF GitHub Bot (JIRA)

 [ 
https://issues.apache.org/jira/browse/BEAM-3433?focusedWorklogId=102745=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-102745
 ]

ASF GitHub Bot logged work on BEAM-3433:


Author: ASF GitHub Bot
Created on: 17/May/18 01:15
Start Date: 17/May/18 01:15
Worklog Time Spent: 10m 
  Work Description: chamikaramj commented on issue #5178: [BEAM-3433] Allow 
a GCP project to be explicitly set for a load job
URL: https://github.com/apache/beam/pull/5178#issuecomment-389713747
 
 
   LGTM.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 102745)
Time Spent: 1h 40m  (was: 1.5h)

> Allow BigQueryIO to use a different project for the load job in batch mode.
> ---
>
> Key: BEAM-3433
> URL: https://issues.apache.org/jira/browse/BEAM-3433
> Project: Beam
>  Issue Type: Bug
>  Components: io-java-gcp
>Reporter: Kevin Peterson
>Assignee: Chamikara Jayalath
>Priority: Minor
>  Time Spent: 1h 40m
>  Remaining Estimate: 0h
>
> BigQueryIO is currently configured to always run a batch load job using the 
> same projects as the destination table: 
> https://github.com/apache/beam/blob/192b4c70927901860312f8c8acd27bd47e4a4259/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java#L256
> This may not always be desirable, since a pipeline may have write access to a 
> dataset in a different project, but not jobs.create access in that project. 
> This parameter should be settable in the interface.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Work logged] (BEAM-3433) Allow BigQueryIO to use a different project for the load job in batch mode.

2018-05-16 Thread ASF GitHub Bot (JIRA)

 [ 
https://issues.apache.org/jira/browse/BEAM-3433?focusedWorklogId=102638=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-102638
 ]

ASF GitHub Bot logged work on BEAM-3433:


Author: ASF GitHub Bot
Created on: 16/May/18 20:23
Start Date: 16/May/18 20:23
Worklog Time Spent: 10m 
  Work Description: kvncp commented on issue #5178: [BEAM-3433] Allow a GCP 
project to be explicitly set for a load job
URL: https://github.com/apache/beam/pull/5178#issuecomment-389653310
 
 
   @jfkk, @chamikaramj, I've addressed comments, and tests are passing.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 102638)
Time Spent: 1.5h  (was: 1h 20m)

> Allow BigQueryIO to use a different project for the load job in batch mode.
> ---
>
> Key: BEAM-3433
> URL: https://issues.apache.org/jira/browse/BEAM-3433
> Project: Beam
>  Issue Type: Bug
>  Components: io-java-gcp
>Reporter: Kevin Peterson
>Assignee: Chamikara Jayalath
>Priority: Minor
>  Time Spent: 1.5h
>  Remaining Estimate: 0h
>
> BigQueryIO is currently configured to always run a batch load job using the 
> same projects as the destination table: 
> https://github.com/apache/beam/blob/192b4c70927901860312f8c8acd27bd47e4a4259/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java#L256
> This may not always be desirable, since a pipeline may have write access to a 
> dataset in a different project, but not jobs.create access in that project. 
> This parameter should be settable in the interface.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Work logged] (BEAM-3433) Allow BigQueryIO to use a different project for the load job in batch mode.

2018-05-16 Thread ASF GitHub Bot (JIRA)

 [ 
https://issues.apache.org/jira/browse/BEAM-3433?focusedWorklogId=102529=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-102529
 ]

ASF GitHub Bot logged work on BEAM-3433:


Author: ASF GitHub Bot
Created on: 16/May/18 16:18
Start Date: 16/May/18 16:18
Worklog Time Spent: 10m 
  Work Description: kvncp commented on a change in pull request #5178: 
[BEAM-3433] Allow a GCP project to be explicitly set for a load job
URL: https://github.com/apache/beam/pull/5178#discussion_r188687147
 
 

 ##
 File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
 ##
 @@ -1391,6 +1394,16 @@ static String getExtractDestinationUri(String 
extractDestinationDir) {
   return toBuilder().setMethod(method).build();
 }
 
+/**
+ * Set the project the BigQuery load job will be initiated from. This is 
only applicable when
+ * the write method is set to {@link Method#FILE_LOADS}. If omitted, the 
project of the
+ * destination table is used.
+ */
+public Write withLoadJobProjectId(String loadJobProjectId) {
 
 Review comment:
   Done


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 102529)
Time Spent: 1h 20m  (was: 1h 10m)

> Allow BigQueryIO to use a different project for the load job in batch mode.
> ---
>
> Key: BEAM-3433
> URL: https://issues.apache.org/jira/browse/BEAM-3433
> Project: Beam
>  Issue Type: Bug
>  Components: io-java-gcp
>Reporter: Kevin Peterson
>Assignee: Chamikara Jayalath
>Priority: Minor
>  Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> BigQueryIO is currently configured to always run a batch load job using the 
> same projects as the destination table: 
> https://github.com/apache/beam/blob/192b4c70927901860312f8c8acd27bd47e4a4259/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java#L256
> This may not always be desirable, since a pipeline may have write access to a 
> dataset in a different project, but not jobs.create access in that project. 
> This parameter should be settable in the interface.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Work logged] (BEAM-3433) Allow BigQueryIO to use a different project for the load job in batch mode.

2018-05-16 Thread ASF GitHub Bot (JIRA)

 [ 
https://issues.apache.org/jira/browse/BEAM-3433?focusedWorklogId=102520=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-102520
 ]

ASF GitHub Bot logged work on BEAM-3433:


Author: ASF GitHub Bot
Created on: 16/May/18 15:41
Start Date: 16/May/18 15:41
Worklog Time Spent: 10m 
  Work Description: kvncp commented on issue #5178: [BEAM-3433] Allow a GCP 
project to be explicitly set for a load job
URL: https://github.com/apache/beam/pull/5178#issuecomment-389565965
 
 
   Sorry, was on vacation recently and just catching up here. Addressing 
comments now!


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 102520)
Time Spent: 1h 10m  (was: 1h)

> Allow BigQueryIO to use a different project for the load job in batch mode.
> ---
>
> Key: BEAM-3433
> URL: https://issues.apache.org/jira/browse/BEAM-3433
> Project: Beam
>  Issue Type: Bug
>  Components: io-java-gcp
>Reporter: Kevin Peterson
>Assignee: Chamikara Jayalath
>Priority: Minor
>  Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> BigQueryIO is currently configured to always run a batch load job using the 
> same projects as the destination table: 
> https://github.com/apache/beam/blob/192b4c70927901860312f8c8acd27bd47e4a4259/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java#L256
> This may not always be desirable, since a pipeline may have write access to a 
> dataset in a different project, but not jobs.create access in that project. 
> This parameter should be settable in the interface.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Work logged] (BEAM-3433) Allow BigQueryIO to use a different project for the load job in batch mode.

2018-05-09 Thread ASF GitHub Bot (JIRA)

 [ 
https://issues.apache.org/jira/browse/BEAM-3433?focusedWorklogId=100193=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-100193
 ]

ASF GitHub Bot logged work on BEAM-3433:


Author: ASF GitHub Bot
Created on: 09/May/18 18:33
Start Date: 09/May/18 18:33
Worklog Time Spent: 10m 
  Work Description: chamikaramj commented on issue #5178: [BEAM-3433] Allow 
a GCP project to be explicitly set for a load job
URL: https://github.com/apache/beam/pull/5178#issuecomment-387834168
 
 
   @kvncp, please address comments to get this into next release.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 100193)
Time Spent: 1h  (was: 50m)

> Allow BigQueryIO to use a different project for the load job in batch mode.
> ---
>
> Key: BEAM-3433
> URL: https://issues.apache.org/jira/browse/BEAM-3433
> Project: Beam
>  Issue Type: Bug
>  Components: io-java-gcp
>Reporter: Kevin Peterson
>Assignee: Chamikara Jayalath
>Priority: Minor
>  Time Spent: 1h
>  Remaining Estimate: 0h
>
> BigQueryIO is currently configured to always run a batch load job using the 
> same projects as the destination table: 
> https://github.com/apache/beam/blob/192b4c70927901860312f8c8acd27bd47e4a4259/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java#L256
> This may not always be desirable, since a pipeline may have write access to a 
> dataset in a different project, but not jobs.create access in that project. 
> This parameter should be settable in the interface.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Work logged] (BEAM-3433) Allow BigQueryIO to use a different project for the load job in batch mode.

2018-05-08 Thread ASF GitHub Bot (JIRA)

 [ 
https://issues.apache.org/jira/browse/BEAM-3433?focusedWorklogId=99742=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-99742
 ]

ASF GitHub Bot logged work on BEAM-3433:


Author: ASF GitHub Bot
Created on: 08/May/18 22:00
Start Date: 08/May/18 22:00
Worklog Time Spent: 10m 
  Work Description: jkff commented on issue #5178: [BEAM-3433] Allow a GCP 
project to be explicitly set for a load job
URL: https://github.com/apache/beam/pull/5178#issuecomment-387556158
 
 
   Hey @kvncp , any updates on this one?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 99742)
Time Spent: 50m  (was: 40m)

> Allow BigQueryIO to use a different project for the load job in batch mode.
> ---
>
> Key: BEAM-3433
> URL: https://issues.apache.org/jira/browse/BEAM-3433
> Project: Beam
>  Issue Type: Bug
>  Components: io-java-gcp
>Reporter: Kevin Peterson
>Assignee: Chamikara Jayalath
>Priority: Minor
>  Time Spent: 50m
>  Remaining Estimate: 0h
>
> BigQueryIO is currently configured to always run a batch load job using the 
> same projects as the destination table: 
> https://github.com/apache/beam/blob/192b4c70927901860312f8c8acd27bd47e4a4259/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java#L256
> This may not always be desirable, since a pipeline may have write access to a 
> dataset in a different project, but not jobs.create access in that project. 
> This parameter should be settable in the interface.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Work logged] (BEAM-3433) Allow BigQueryIO to use a different project for the load job in batch mode.

2018-04-27 Thread ASF GitHub Bot (JIRA)

 [ 
https://issues.apache.org/jira/browse/BEAM-3433?focusedWorklogId=96186=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-96186
 ]

ASF GitHub Bot logged work on BEAM-3433:


Author: ASF GitHub Bot
Created on: 27/Apr/18 18:49
Start Date: 27/Apr/18 18:49
Worklog Time Spent: 10m 
  Work Description: jkff commented on a change in pull request #5178: 
[BEAM-3433] Allow a GCP project to be explicitly set for a load job
URL: https://github.com/apache/beam/pull/5178#discussion_r184776409
 
 

 ##
 File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
 ##
 @@ -1391,6 +1394,16 @@ static String getExtractDestinationUri(String 
extractDestinationDir) {
   return toBuilder().setMethod(method).build();
 }
 
+/**
+ * Set the project the BigQuery load job will be initiated from. This is 
only applicable when
+ * the write method is set to {@link Method#FILE_LOADS}. If omitted, the 
project of the
+ * destination table is used.
+ */
+public Write withLoadJobProjectId(String loadJobProjectId) {
 
 Review comment:
   Could you add a ValueProvider version too?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 96186)
Time Spent: 40m  (was: 0.5h)

> Allow BigQueryIO to use a different project for the load job in batch mode.
> ---
>
> Key: BEAM-3433
> URL: https://issues.apache.org/jira/browse/BEAM-3433
> Project: Beam
>  Issue Type: Bug
>  Components: io-java-gcp
>Reporter: Kevin Peterson
>Assignee: Chamikara Jayalath
>Priority: Minor
>  Time Spent: 40m
>  Remaining Estimate: 0h
>
> BigQueryIO is currently configured to always run a batch load job using the 
> same projects as the destination table: 
> https://github.com/apache/beam/blob/192b4c70927901860312f8c8acd27bd47e4a4259/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java#L256
> This may not always be desirable, since a pipeline may have write access to a 
> dataset in a different project, but not jobs.create access in that project. 
> This parameter should be settable in the interface.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Work logged] (BEAM-3433) Allow BigQueryIO to use a different project for the load job in batch mode.

2018-04-27 Thread ASF GitHub Bot (JIRA)

 [ 
https://issues.apache.org/jira/browse/BEAM-3433?focusedWorklogId=96185=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-96185
 ]

ASF GitHub Bot logged work on BEAM-3433:


Author: ASF GitHub Bot
Created on: 27/Apr/18 18:45
Start Date: 27/Apr/18 18:45
Worklog Time Spent: 10m 
  Work Description: chamikaramj commented on issue #5178: [BEAM-3433] Allow 
a GCP project to be explicitly set for a load job
URL: https://github.com/apache/beam/pull/5178#issuecomment-385059985
 
 
   Thanks. Will take a look.
   
   cc: @jkff 


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 96185)
Time Spent: 0.5h  (was: 20m)

> Allow BigQueryIO to use a different project for the load job in batch mode.
> ---
>
> Key: BEAM-3433
> URL: https://issues.apache.org/jira/browse/BEAM-3433
> Project: Beam
>  Issue Type: Bug
>  Components: io-java-gcp
>Reporter: Kevin Peterson
>Assignee: Chamikara Jayalath
>Priority: Minor
>  Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> BigQueryIO is currently configured to always run a batch load job using the 
> same projects as the destination table: 
> https://github.com/apache/beam/blob/192b4c70927901860312f8c8acd27bd47e4a4259/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java#L256
> This may not always be desirable, since a pipeline may have write access to a 
> dataset in a different project, but not jobs.create access in that project. 
> This parameter should be settable in the interface.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Work logged] (BEAM-3433) Allow BigQueryIO to use a different project for the load job in batch mode.

2018-04-18 Thread ASF GitHub Bot (JIRA)

 [ 
https://issues.apache.org/jira/browse/BEAM-3433?focusedWorklogId=92345=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-92345
 ]

ASF GitHub Bot logged work on BEAM-3433:


Author: ASF GitHub Bot
Created on: 19/Apr/18 02:06
Start Date: 19/Apr/18 02:06
Worklog Time Spent: 10m 
  Work Description: kvncp opened a new pull request #5178: [BEAM-3433] 
Allow a GCP project to be explicitly set for a load job
URL: https://github.com/apache/beam/pull/5178
 
 
   I've added the needed string parameter to the BigQueryIO.write() function, 
and passed it through to the underlying class. Wanted to get some feedback 
before trying to write a test.
   
   1. Should I also add a ValueProvider interface?
   2. I've modified the constructor for WriteTables, which is public. Should I 
instead add a setter for that function or overload the constructor?
   3. Should I validate the this parameter is not set unless the Method is 
FILE_LOADS? It isn't harmful to set it otherwise, it is just ignored. Not sure 
what the recommendation is in that case.


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 92345)
Time Spent: 10m
Remaining Estimate: 0h

> Allow BigQueryIO to use a different project for the load job in batch mode.
> ---
>
> Key: BEAM-3433
> URL: https://issues.apache.org/jira/browse/BEAM-3433
> Project: Beam
>  Issue Type: Bug
>  Components: io-java-gcp
>Reporter: Kevin Peterson
>Assignee: Chamikara Jayalath
>Priority: Minor
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> BigQueryIO is currently configured to always run a batch load job using the 
> same projects as the destination table: 
> https://github.com/apache/beam/blob/192b4c70927901860312f8c8acd27bd47e4a4259/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java#L256
> This may not always be desirable, since a pipeline may have write access to a 
> dataset in a different project, but not jobs.create access in that project. 
> This parameter should be settable in the interface.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Work logged] (BEAM-3433) Allow BigQueryIO to use a different project for the load job in batch mode.

2018-04-18 Thread ASF GitHub Bot (JIRA)

 [ 
https://issues.apache.org/jira/browse/BEAM-3433?focusedWorklogId=92346=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-92346
 ]

ASF GitHub Bot logged work on BEAM-3433:


Author: ASF GitHub Bot
Created on: 19/Apr/18 02:06
Start Date: 19/Apr/18 02:06
Worklog Time Spent: 10m 
  Work Description: kvncp commented on issue #5178: [BEAM-3433] Allow a GCP 
project to be explicitly set for a load job
URL: https://github.com/apache/beam/pull/5178#issuecomment-382586056
 
 
   Hi @chamikaramj , can you please take a look?


This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 92346)
Time Spent: 20m  (was: 10m)

> Allow BigQueryIO to use a different project for the load job in batch mode.
> ---
>
> Key: BEAM-3433
> URL: https://issues.apache.org/jira/browse/BEAM-3433
> Project: Beam
>  Issue Type: Bug
>  Components: io-java-gcp
>Reporter: Kevin Peterson
>Assignee: Chamikara Jayalath
>Priority: Minor
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> BigQueryIO is currently configured to always run a batch load job using the 
> same projects as the destination table: 
> https://github.com/apache/beam/blob/192b4c70927901860312f8c8acd27bd47e4a4259/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java#L256
> This may not always be desirable, since a pipeline may have write access to a 
> dataset in a different project, but not jobs.create access in that project. 
> This parameter should be settable in the interface.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)