[
https://issues.apache.org/jira/browse/BEAM-5457?focusedWorklogId=151107&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-151107
]
ASF GitHub Bot logged work on BEAM-5457:
----------------------------------------
Author: ASF GitHub Bot
Created on: 04/Oct/18 09:43
Start Date: 04/Oct/18 09:43
Worklog Time Spent: 10m
Work Description: joar 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_r222602287
##########
File path: sdks/python/apache_beam/io/gcp/bigquery.py
##########
@@ -659,27 +659,41 @@ def __init__(self, source, test_bigquery_client=None,
use_legacy_sql=True,
else:
self.query = self.source.query
- def _get_source_table_location(self):
- tr = self.source.table_reference
- if tr is None:
- # TODO: implement location retrieval for query sources
- return
+ def _get_source_location(self):
+ """
+ Get the source location (e.g. ``"EU"`` or ``"US"``) from either
- if tr.projectId is None:
- source_project_id = self.executing_project
- else:
- source_project_id = tr.projectId
+ - :data:`source.table_reference`
+ or
+ - The first referenced table in :data:`source.query`
+
+ See Also:
+ - :meth:`BigQueryWrapper.get_query_location`
+ - :meth:`BigQueryWrapper.get_table_location`
- source_dataset_id = tr.datasetId
- source_table_id = tr.tableId
- source_location = self.client.get_table_location(
- source_project_id, source_dataset_id, source_table_id)
- return source_location
+ Returns:
+ Optional[str]: The source location, if any.
+ """
+ if self.source.table_reference is not None:
+ tr = self.source.table_reference
+ return self.client.get_table_location(
+ tr.projectId if tr.projectId is not None else self.executing_project,
+ tr.datasetId, tr.tableId)
+ elif self.source.query is not None:
+ return self.client.get_query_location(
+ self.executing_project,
+ self.source.query,
+ self.source.use_legacy_sql)
+ else:
+ # Enforce the "modes" enforced by BigQuerySource.__init__.
+ # If this exception has been raised, the BigQuerySource "modes" have
+ # changed and this method will need to be updated as well.
+ raise ValueError("BigQuerySource must have either a table or query")
Review comment:
@udim re: https://github.com/apache/beam/pull/6463#discussion_r222597668 I
pushed this, could revert easily.
----------------------------------------------------------------
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: 151107)
Time Spent: 3h (was: 2h 50m)
> 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: 3h
> 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)