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

ASF GitHub Bot logged work on BEAM-8458:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 03/Feb/20 18:51
            Start Date: 03/Feb/20 18:51
    Worklog Time Spent: 10m 
      Work Description: chamikaramj commented on pull request #9852: 
[BEAM-8458] Add option to set temp dataset in BigQueryIO.Read
URL: https://github.com/apache/beam/pull/9852#discussion_r374276451
 
 

 ##########
 File path: 
sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySourceDef.java
 ##########
 @@ -115,19 +127,26 @@ TableReference getTableReference(BigQueryOptions 
bqOptions, String stepUuid)
         useLegacySql,
         priority,
         location,
+        queryTempDataset,
         kmsKey);
   }
 
   void cleanupTempResource(BigQueryOptions bqOptions, String stepUuid) throws 
Exception {
+    Optional<String> queryTempDatasetOpt = 
Optional.ofNullable(queryTempDataset);
     TableReference tableToRemove =
         createTempTableReference(
-            bqOptions.getProject(), createJobIdToken(bqOptions.getJobName(), 
stepUuid));
+            bqOptions.getProject(),
+            createJobIdToken(bqOptions.getJobName(), stepUuid),
+            queryTempDatasetOpt);
 
     BigQueryServices.DatasetService tableService = 
bqServices.getDatasetService(bqOptions);
     LOG.info("Deleting temporary table with query results {}", tableToRemove);
     tableService.deleteTable(tableToRemove);
-    LOG.info("Deleting temporary dataset with query results {}", 
tableToRemove.getDatasetId());
-    tableService.deleteDataset(tableToRemove.getProjectId(), 
tableToRemove.getDatasetId());
+    if (queryTempDatasetOpt.isPresent()) {
 
 Review comment:
   Probably extract "queryTempDatasetOpt.isPresent()" to an instance variable 
boolean datasetProvidedByUser (before this point).
 
----------------------------------------------------------------
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:
us...@infra.apache.org


Issue Time Tracking
-------------------

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

> BigQueryIO.Read needs permissions to create datasets to be able to run queries
> ------------------------------------------------------------------------------
>
>                 Key: BEAM-8458
>                 URL: https://issues.apache.org/jira/browse/BEAM-8458
>             Project: Beam
>          Issue Type: Bug
>          Components: io-java-gcp
>            Reporter: Israel Herraiz
>            Assignee: Israel Herraiz
>            Priority: Major
>          Time Spent: 1h
>  Remaining Estimate: 0h
>
> When using {{fromQuery}}, BigQueryIO creates a temp dataset to store the 
> results of the query.
> Therefore, Beam requires permissions to create datasets just to be able to 
> run a query. In practice, this means that Beam requires the role 
> bigQuery.User just to run queries, whereas if you use {{from}} (to read from 
> a table), the role bigQuery.jobUser suffices.
> BigQueryIO.Read should have an option to set an existing dataset  to write 
> the temp results of
>  a query, so it would be enough with having the role bigQuery.jobUser.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to