Repository: incubator-beam Updated Branches: refs/heads/python-sdk 8365b6838 -> 72fa21f98
Make the legacy SQL flag consistent between Java and Python Renamed the BigQuery use_legacy_sql parameter to use_standard_sql. Project: http://git-wip-us.apache.org/repos/asf/incubator-beam/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-beam/commit/72721031 Tree: http://git-wip-us.apache.org/repos/asf/incubator-beam/tree/72721031 Diff: http://git-wip-us.apache.org/repos/asf/incubator-beam/diff/72721031 Branch: refs/heads/python-sdk Commit: 727210318404a585bb7742591ade0a09ccc20444 Parents: 8365b68 Author: Sourabh Bajaj <sourabhba...@google.com> Authored: Fri Dec 2 16:45:19 2016 -0800 Committer: Robert Bradshaw <rober...@gmail.com> Committed: Fri Dec 2 22:10:21 2016 -0800 ---------------------------------------------------------------------- sdks/python/apache_beam/examples/snippets/snippets.py | 2 +- sdks/python/apache_beam/io/bigquery.py | 9 +++++---- sdks/python/apache_beam/io/bigquery_test.py | 4 ++-- 3 files changed, 8 insertions(+), 7 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/72721031/sdks/python/apache_beam/examples/snippets/snippets.py ---------------------------------------------------------------------- diff --git a/sdks/python/apache_beam/examples/snippets/snippets.py b/sdks/python/apache_beam/examples/snippets/snippets.py index 580a3d7..6dcf05e 100644 --- a/sdks/python/apache_beam/examples/snippets/snippets.py +++ b/sdks/python/apache_beam/examples/snippets/snippets.py @@ -962,7 +962,7 @@ def model_bigqueryio(): 'ReadYearAndTemp', beam.io.BigQuerySource( query='SELECT year, mean_temp FROM `samples.weather_stations`', - use_legacy_sql=False)) + use_standard_sql=True)) # [END model_bigqueryio_query_standard_sql] # [START model_bigqueryio_schema] http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/72721031/sdks/python/apache_beam/io/bigquery.py ---------------------------------------------------------------------- diff --git a/sdks/python/apache_beam/io/bigquery.py b/sdks/python/apache_beam/io/bigquery.py index 0885e3a..ce75e10 100644 --- a/sdks/python/apache_beam/io/bigquery.py +++ b/sdks/python/apache_beam/io/bigquery.py @@ -323,7 +323,7 @@ class BigQuerySource(dataflow_io.NativeSource): """A source based on a BigQuery table.""" def __init__(self, table=None, dataset=None, project=None, query=None, - validate=False, coder=None, use_legacy_sql=True): + validate=False, coder=None, use_standard_sql=False): """Initialize a BigQuerySource. Args: @@ -351,8 +351,8 @@ class BigQuerySource(dataflow_io.NativeSource): in a file as a JSON serialized dictionary. This argument needs a value only in special cases when returning table rows as dictionaries is not desirable. - useLegacySql: Specifies whether to use BigQuery's legacy - SQL dialect for this query. The default value is true. If set to false, + use_standard_sql: Specifies whether to use BigQuery's standard + SQL dialect for this query. The default value is False. If set to True, the query will use BigQuery's updated SQL dialect with improved standards compliance. This parameter is ignored for table inputs. @@ -374,7 +374,8 @@ class BigQuerySource(dataflow_io.NativeSource): self.use_legacy_sql = True else: self.query = query - self.use_legacy_sql = use_legacy_sql + # TODO(BEAM-1082): Change the internal flag to be standard_sql + self.use_legacy_sql = not use_standard_sql self.table_reference = None self.validate = validate http://git-wip-us.apache.org/repos/asf/incubator-beam/blob/72721031/sdks/python/apache_beam/io/bigquery_test.py ---------------------------------------------------------------------- diff --git a/sdks/python/apache_beam/io/bigquery_test.py b/sdks/python/apache_beam/io/bigquery_test.py index e263e13..a2cf947 100644 --- a/sdks/python/apache_beam/io/bigquery_test.py +++ b/sdks/python/apache_beam/io/bigquery_test.py @@ -199,7 +199,7 @@ class TestBigQuerySource(unittest.TestCase): hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items)) def test_specify_query_sql_format(self): - source = beam.io.BigQuerySource(query='my_query', use_legacy_sql=False) + source = beam.io.BigQuerySource(query='my_query', use_standard_sql=True) self.assertEqual(source.query, 'my_query') self.assertFalse(source.use_legacy_sql) @@ -371,7 +371,7 @@ class TestBigQueryReader(unittest.TestCase): jobComplete=True, rows=table_rows, schema=schema) actual_rows = [] with beam.io.BigQuerySource( - query='query', use_legacy_sql=False).reader(client) as reader: + query='query', use_standard_sql=True).reader(client) as reader: for row in reader: actual_rows.append(row) self.assertEqual(actual_rows, expected_rows)