You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2020/09/29 02:17:27 UTC

[GitHub] [beam] frankzhao opened a new pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

frankzhao opened a new pull request #12960:
URL: https://github.com/apache/beam/pull/12960


   Allow ReadFromBigQuery to use a user pre-configured dataset for the temporary dataset.
   Using a DatasetReference will also allow for cross project temporary dataset configuration.
   
   There are some uses cases that require temporary datasets created when using beam.io.BigQuerySource to be in a different GCP project (e.g service account permissions, billing).
   
   We could make the project where temporary datasets configurable, so that the temp dataset is created in a user specified project. Example:
   
   ```
   pipeline = p | "Read from BQ source" >> beam.io.ReadFromBigQuery(
         query=q,
         use_standard_sql=True,
         project='frankzhao-test-project',
         temp_dataset=bigquery.DatasetReference(
             projectId='bigquery-project', datasetId='bq_temp'))
   ```
   
   This PR adds the temp_dataset option to allow user configuration of the temporary dataset location when using ReadFromBigQuery, which defaults to the execution project. A DatasetReference is used to allow for the dataset location to be in a different project (BEAM-10859).
   
   ------------------------
   
   Thank you for your contribution! Follow this checklist to help us incorporate your contribution quickly and easily:
   
    - [ ] [**Choose reviewer(s)**](https://beam.apache.org/contribute/#make-your-change) and mention them in a comment (`R: @username`).
    - [x] Format the pull request title like `[BEAM-XXX] Fixes bug in ApproximateQuantiles`, where you replace `BEAM-XXX` with the appropriate JIRA issue, if applicable. This will automatically link the pull request to the issue.
    - [ ] Update `CHANGES.md` with noteworthy changes.
    - [ ] If this contribution is large, please file an Apache [Individual Contributor License Agreement](https://www.apache.org/licenses/icla.pdf).
   
   See the [Contributor Guide](https://beam.apache.org/contribute) for more tips on [how to make review process smoother](https://beam.apache.org/contribute/#make-reviewers-job-easier).
   
   Post-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   Lang | SDK | Dataflow | Flink | Samza | Spark | Twister2
   --- | --- | --- | --- | --- | --- | ---
   Go | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Flink/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Go_VR_Spark/lastCompletedBuild/) | ---
   Java | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Dataflow_Java11/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/badge/i
 con)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Flink_Java11/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Batch/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Flink_Streaming/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Samza/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Spark/lastCompletedBuild/)<br>[![Build Status](htt
 ps://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_PVR_Spark_Batch/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Java_ValidatesRunner_Twister2/lastCompletedBuild/)
   Python | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python36/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python37/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python38/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python38/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_VR_Dataflow_V2/lastCompletedBuild/)<br>[![Build Status](https://ci-beam
 .apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Py_ValCont/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Python_PVR_Flink_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Python_PVR_Flink_Cron/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Flink/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_Python_VR_Spark/lastCompletedBuild/) | ---
   XLang | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Direct/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Flink/lastCompletedBuild/) | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PostCommit_XVR_Spark/lastCompletedBuild/) | ---
   
   Pre-Commit Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   
   --- |Java | Python | Go | Website | Whitespace | Typescript
   --- | --- | --- | --- | --- | --- | ---
   Non-portable | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Java_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Python_Cron/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_PythonLint_Cron/lastCompletedBuild/)<br>[![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_PythonDocker_Cron/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_PythonDocker_Cron/lastCompletedBuild/) <br>[![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_PythonDocs_Cron/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_PythonDocs_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/be
 am_PreCommit_Go_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Go_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Website_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Whitespace_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Whitespace_Cron/lastCompletedBuild/) | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Typescript_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Typescript_Cron/lastCompletedBuild/)
   Portable | --- | [![Build Status](https://ci-beam.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/badge/icon)](https://ci-beam.apache.org/job/beam_PreCommit_Portable_Python_Cron/lastCompletedBuild/) | --- | --- | --- | ---
   
   See [.test-infra/jenkins/README](https://github.com/apache/beam/blob/master/.test-infra/jenkins/README.md) for trigger phrase, status and link of all Jenkins jobs.
   
   
   GitHub Actions Tests Status (on master branch)
   ------------------------------------------------------------------------------------------------
   [![Build python source distribution and wheels](https://github.com/apache/beam/workflows/Build%20python%20source%20distribution%20and%20wheels/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Build+python+source+distribution+and+wheels%22+branch%3Amaster+event%3Aschedule)
   [![Python tests](https://github.com/apache/beam/workflows/Python%20tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Python+Tests%22+branch%3Amaster+event%3Aschedule)
   [![Java tests](https://github.com/apache/beam/workflows/Java%20Tests/badge.svg?branch=master&event=schedule)](https://github.com/apache/beam/actions?query=workflow%3A%22Java+Tests%22+branch%3Amaster+event%3Aschedule)
   
   See [CI.md](https://github.com/apache/beam/blob/master/CI.md) for more information about GitHub Actions CI.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
pabloem commented on pull request #12960:
URL: https://github.com/apache/beam/pull/12960#issuecomment-724916983


   this looks much nicer. Thanks Frank! LGTM. I'll merge once tests pass


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] frankzhao commented on pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
frankzhao commented on pull request #12960:
URL: https://github.com/apache/beam/pull/12960#issuecomment-700713637


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] frankzhao commented on pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
frankzhao commented on pull request #12960:
URL: https://github.com/apache/beam/pull/12960#issuecomment-703063721


   Sure, added pydoc


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on a change in pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #12960:
URL: https://github.com/apache/beam/pull/12960#discussion_r504209460



##########
File path: sdks/python/apache_beam/io/gcp/bigquery_tools.py
##########
@@ -1096,7 +1114,8 @@ def __init__(
       test_bigquery_client=None,
       use_legacy_sql=True,
       flatten_results=True,
-      kms_key=None):
+      kms_key=None,
+      temp_dataset=None):

Review comment:
       The `BigQueryReader` is only used in the Dataflow native BigQuerySource - I don't think this is compatible, so we should probably revert changes to it.
   Let's focus on temp_dataset for _CustomBigQuerySource.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] frankzhao commented on pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
frankzhao commented on pull request #12960:
URL: https://github.com/apache/beam/pull/12960#issuecomment-701738038


   R: @pabloem


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] frankzhao commented on a change in pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
frankzhao commented on a change in pull request #12960:
URL: https://github.com/apache/beam/pull/12960#discussion_r513107429



##########
File path: sdks/python/apache_beam/io/gcp/bigquery_tools.py
##########
@@ -1096,7 +1114,8 @@ def __init__(
       test_bigquery_client=None,
       use_legacy_sql=True,
       flatten_results=True,
-      kms_key=None):
+      kms_key=None,
+      temp_dataset=None):

Review comment:
       reverted changes to `BigQueryReader`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] frankzhao removed a comment on pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
frankzhao removed a comment on pull request #12960:
URL: https://github.com/apache/beam/pull/12960#issuecomment-700713637


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] otourzan commented on pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
otourzan commented on pull request #12960:
URL: https://github.com/apache/beam/pull/12960#issuecomment-774637281


   I may not have understood this change correctly but I see 2 issues here.
   
   1- _BigQuerySource is the private one and users don't call it. It supposed to be called through BigQuerySource [1]. Those 2 classes don't use kwargs and temp_data set is not defined in init parameters of public one as well so it's not accessible to users, right?
   
   2- ReadFromBigQuery and _CustomBigQuerySource handle kwargs, but I think the temp_daatset needs to be documented in ReadFromBigQuery docs as the public method users use.
   
   [1]: https://github.com/apache/beam/blob/b74fcf7b30d956fb42830d652a57b265a1546973/sdks/python/apache_beam/io/gcp/bigquery.py#L492


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
pabloem commented on pull request #12960:
URL: https://github.com/apache/beam/pull/12960#issuecomment-718112424


   Run Python_PVR_Flink PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] frankzhao commented on pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
frankzhao commented on pull request #12960:
URL: https://github.com/apache/beam/pull/12960#issuecomment-724651451


   Refactored to have the `dataset_id` in `BigQueryWrapper` and squashed the commits. Let me know what you think @pabloem 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] frankzhao commented on pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
frankzhao commented on pull request #12960:
URL: https://github.com/apache/beam/pull/12960#issuecomment-774670479


   Regarding (1): BigQuerySource is deprecated since 2.25.0 https://github.com/apache/beam/blob/b74fcf7b30d956fb42830d652a57b265a1546973/sdks/python/apache_beam/io/gcp/bigquery.py#L479 and users should use ReadFromBigQuery, which handles the `temp_dataset` kwarg.
   
   Regarding (2): Yes, we should probably have the same doc comment for `temp_dataset` in ReadFromBigQuery.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
pabloem commented on pull request #12960:
URL: https://github.com/apache/beam/pull/12960#issuecomment-724900046


   Run Python 3.8 PostCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
pabloem commented on pull request #12960:
URL: https://github.com/apache/beam/pull/12960#issuecomment-717702550


   Run Python 3.8 PostCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
pabloem commented on pull request #12960:
URL: https://github.com/apache/beam/pull/12960#issuecomment-702934278


   thanks @frankzhao ! I'm taking a look. In the meantime, can you add pydoc for the new argument to ReadFromBigQuery?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on a change in pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #12960:
URL: https://github.com/apache/beam/pull/12960#discussion_r514516617



##########
File path: sdks/python/apache_beam/io/gcp/bigquery_tools.py
##########
@@ -731,16 +744,19 @@ def create_temporary_dataset(self, project_id, location):
   @retry.with_exponential_backoff(
       num_retries=MAX_RETRIES,
       retry_filter=retry.retry_on_server_errors_and_timeout_filter)
-  def clean_up_temporary_dataset(self, project_id):
+  def clean_up_temporary_dataset(self, project_id, dataset_reference=None):
+    if dataset_reference:
+      project_id = dataset_reference.projectId
     temp_table = self._get_temp_table(project_id)
+    dataset_id = dataset_reference.datasetId if dataset_reference \

Review comment:
       in beam, we usually break lines using parentheses, kind of like this:
   ```
   dataset_id = (
       dataset_reference.datasetId if dataset_reference else temp_table.datasetId)
   ```

##########
File path: sdks/python/apache_beam/io/gcp/bigquery.py
##########
@@ -712,6 +713,7 @@ def __init__(
     self.bq_io_metadata = None  # Populate in setup, as it may make an RPC
     self.bigquery_job_labels = bigquery_job_labels or {}
     self.use_json_exports = use_json_exports
+    self.temp_dataset = temp_dataset

Review comment:
       how about we add a `get_temporary_dataset` function to `BigqueryWrapper`, and we can define a `self.temp_dataset = self.temp_dataset or bq.get_temporary_dataset()` (this logic would need to run in the `split`, and `- and once we've done that, we can just pass the dataset name around to every call? That way we will treat the user-defined dataset and the automatic dataset the same way?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on a change in pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
pabloem commented on a change in pull request #12960:
URL: https://github.com/apache/beam/pull/12960#discussion_r514520584



##########
File path: sdks/python/apache_beam/io/gcp/bigquery.py
##########
@@ -712,6 +713,7 @@ def __init__(
     self.bq_io_metadata = None  # Populate in setup, as it may make an RPC
     self.bigquery_job_labels = bigquery_job_labels or {}
     self.use_json_exports = use_json_exports
+    self.temp_dataset = temp_dataset

Review comment:
       let me know what you think about this implementation @frankzhao
   
   I understand changing it may be troublesome, but doing this would simplify the logic quite a bit




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem commented on pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
pabloem commented on pull request #12960:
URL: https://github.com/apache/beam/pull/12960#issuecomment-718135801


   Run Python_PVR_Flink PreCommit


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] pabloem merged pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
pabloem merged pull request #12960:
URL: https://github.com/apache/beam/pull/12960


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [beam] frankzhao commented on a change in pull request #12960: [BEAM-9804] Allow user configuration of BigQuery temporary dataset

Posted by GitBox <gi...@apache.org>.
frankzhao commented on a change in pull request #12960:
URL: https://github.com/apache/beam/pull/12960#discussion_r514589859



##########
File path: sdks/python/apache_beam/io/gcp/bigquery.py
##########
@@ -712,6 +713,7 @@ def __init__(
     self.bq_io_metadata = None  # Populate in setup, as it may make an RPC
     self.bigquery_job_labels = bigquery_job_labels or {}
     self.use_json_exports = use_json_exports
+    self.temp_dataset = temp_dataset

Review comment:
       I agree, let me take look at it




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org