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

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

                Author: ASF GitHub Bot
            Created on: 03/Oct/18 17:01
            Start Date: 03/Oct/18 17:01
    Worklog Time Spent: 10m 
      Work Description: udim commented on a change in pull request #6463: 
[BEAM-5457][BEAM-1909] Make BigQuerySource work for the EU
URL: https://github.com/apache/beam/pull/6463#discussion_r222389468
 
 

 ##########
 File path: sdks/python/apache_beam/io/gcp/bigquery.py
 ##########
 @@ -659,11 +661,36 @@ def __init__(self, source, test_bigquery_client=None, 
use_legacy_sql=True,
     else:
       self.query = self.source.query
 
+  def _get_source_query_location(self):
+    referenced_locations = self.client.get_referenced_locations(
+        self.executing_project,
+        self.source.query,
+        use_legacy_sql=self.source.use_legacy_sql)
+
+    logger.debug("Referenced locations: %r", referenced_locations)
+
+    locations = set(referenced_locations.values())
+
+    if len(locations) > 1:
 
 Review comment:
   For example, the Java SDK takes the location of the first referenced table:
   
https://github.com/apache/beam/blob/279a05604b83a54e8e5a79e13d8761f94841f326/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQuerySource.java#L117-L120
   
   I believe that is sufficient. Also note that the docs for referencedTables 
states:
   > Queries that reference more than 50 tables will not have a complete list.

----------------------------------------------------------------
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:
[email protected]


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

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

> BigQuerySource(query=...) in DirectRunner creates temp dataset in the wrong 
> location
> ------------------------------------------------------------------------------------
>
>                 Key: BEAM-5457
>                 URL: https://issues.apache.org/jira/browse/BEAM-5457
>             Project: Beam
>          Issue Type: Bug
>          Components: sdk-py-core
>    Affects Versions: 2.6.0
>            Reporter: Joar Wandborg
>            Assignee: Chamikara Jayalath
>            Priority: Major
>          Time Spent: 1.5h
>  Remaining Estimate: 0h
>
> I'm in the EU, if I have a
>  
> {code:java}
> BigQuerySource(
>     query="SELECT x, y FROM `my-other-project.mydataset.my_european_table`",
>     project="myproject",
>     use_standard_sql=True
> ){code}
> And then run the Pipeline through the DirectRunner I get the following 
> warning and error:
> {noformat}
> 2018-09-21 11:39:52,620 WARNING root create_temporary_dataset
> Dataset myproject:temp_dataset_0bbb28f014a24225b668a67341f4f71e does not 
> exist so we will create it as temporary with location=None {noformat}
> {noformat}
> HttpBadRequestError: HttpError accessing 
> <https://www.googleapis.com/bigquery/v2/projects/myproject/queries/xyz123?alt=json&maxResults=10000>:
>  response: <{'status': '400', 'content-length': '354', 'x-xss-protection': 
> '1; mode=block', 'x-content-type-options': 'nosniff', 'transfer-encoding': 
> 'chunked', 'vary': 'Origin, X-Origin, Referer', 'server': 'ESF', 
> '-content-encoding': 'gzip', 'cache-control': 'private', 'date': 'Fri, 21 Sep 
> 2018 09:39:55 GMT', 'x-frame-options': 'SAMEORIGIN', 'alt-svc': 'quic=":443"; 
> ma=2592000; v="44,43,39,35"', 'content-type': 'application/json; 
> charset=UTF-8'}>, content <{
>   "error": {
>     "code": 400,
>     "message": "Cannot read and write in different locations: source: EU, 
> destination: US",
>     "errors": [
>       {
>         "message": "Cannot read and write in different locations: source: EU, 
> destination: US",
>         "domain": "global",
>         "reason": "invalid"
>       }
>     ],
>     "status": "INVALID_ARGUMENT"
>   }
> {noformat}
> There's a TODO in the code that looks very related: 
> [https://github.com/apache/beam/blob/d691a86b8fd082efd0fd71c3cb58b7d61442717d/sdks/python/apache_beam/io/gcp/bigquery.py#L665|https://github.com/apache/beam/blob/d691a86b8fd082efd0fd71c3cb58b7d61442717d/sdks/python/apache_beam/io/gcp/bigquery.py#L665,]
>  



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

Reply via email to