[
https://issues.apache.org/jira/browse/BEAM-11587?focusedWorklogId=753773&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-753773
]
ASF GitHub Bot logged work on BEAM-11587:
-----------------------------------------
Author: ASF GitHub Bot
Created on: 06/Apr/22 23:29
Start Date: 06/Apr/22 23:29
Worklog Time Spent: 10m
Work Description: TheNeuralBit commented on code in PR #17159:
URL: https://github.com/apache/beam/pull/17159#discussion_r844504179
##########
sdks/python/apache_beam/io/gcp/bigquery_schema_tools.py:
##########
@@ -0,0 +1,115 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""Tools used tool work with Schema types in the context of BigQuery.
+Classes, constants and functions in this file are experimental and have no
+backwards compatibility guarantees.
+NOTHING IN THIS FILE HAS BACKWARDS COMPATIBILITY GUARANTEES.
+"""
+import apache_beam as beam
+import numpy as np
+from typing import Optional
+from typing import Sequence
+
+# BigQuery types as listed in
+# https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types
+# with aliases (RECORD, BOOLEAN, FLOAT, INTEGER) as defined in
+#
https://developers.google.com/resources/api-libraries/documentation/bigquery/v2/java/latest/com/google/api/services/bigquery/model/TableFieldSchema.html#setType-java.lang.String-
+BIG_QUERY_TO_PYTHON_TYPES = {
+ "STRING": "str",
Review Comment:
```suggestion
"STRING": str,
```
(similarly for the other fields), then you should be able to use this
mapping in your function, right?
##########
sdks/python/apache_beam/io/gcp/bigquery_schema_tools.py:
##########
@@ -0,0 +1,115 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""Tools used tool work with Schema types in the context of BigQuery.
+Classes, constants and functions in this file are experimental and have no
+backwards compatibility guarantees.
+NOTHING IN THIS FILE HAS BACKWARDS COMPATIBILITY GUARANTEES.
Review Comment:
Haha this is a pretty loud warning, but it looks like it's consistent with
other BQIO files so no need to change it :)
##########
sdks/python/apache_beam/io/gcp/bigquery_schema_tools.py:
##########
@@ -0,0 +1,115 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""Tools used tool work with Schema types in the context of BigQuery.
+Classes, constants and functions in this file are experimental and have no
+backwards compatibility guarantees.
+NOTHING IN THIS FILE HAS BACKWARDS COMPATIBILITY GUARANTEES.
+"""
+import apache_beam as beam
+import numpy as np
+from typing import Optional
+from typing import Sequence
+
+# BigQuery types as listed in
+# https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types
+# with aliases (RECORD, BOOLEAN, FLOAT, INTEGER) as defined in
+#
https://developers.google.com/resources/api-libraries/documentation/bigquery/v2/java/latest/com/google/api/services/bigquery/model/TableFieldSchema.html#setType-java.lang.String-
+BIG_QUERY_TO_PYTHON_TYPES = {
+ "STRING": "str",
+ "INTEGER": "np.int64",
+ "FLOAT64": "np.float64",
+ "BOOLEAN": "bool",
+ "BYTES": "bytes",
+ "TIMESTAMP": "apache_beam.utils.timestamp.Timestamp",
+ #TODO svetaksundhar@: Finish mappings for all BQ types
+}
+
+
+def produce_pcoll_with_schema(self, the_table_schema):
+ #type: [TableSchema] -> [UserType]
+
+ """Convert a schema of type TableSchema into a pcollection element.
+ Args:
+ the_table_schema: A BQ schema of type TableSchema
+ Returns:
+ usertype: type that can be used to work with pCollections.
+ """
+
+ the_schema = beam.io.gcp.bigquery_tools.get_dict_table_schema(
+ the_table_schema)
+ if the_schema == {}:
+ raise ValueError("The schema is empty")
+ i = 0
+ dict_of_tuples = []
+ for x in the_schema['fields']:
+ if the_schema['fields'][i]['type'] == 'STRING':
+ if the_schema['fields'][i]['mode'] == 'NULLABLE':
+ typ = Optional[str]
+ elif the_schema['fields'][i]['mode'] == 'REPEATED':
+ typ = Sequence[str]
Review Comment:
I think we should be able to do this in a way that doesn't duplicate the
logic for each supported type. There are a number of ways to do it, my
suggestion would be to break out a function that does the mapping for a single
field, and call it from here:
```py
typ = bq_field_to_type(the_schema['fields'][i])
```
`bq_field_to_type` will have logic that recurses based on NULLABLE and
REPEATED like:
```py
if field['mode'] == 'NULLABLE':
return Optional[bq_field_to_type(field_with_mode_erased)]
```
and finally if no mode is set it just does the regular type mapping (ideally
leveraging `BIG_QUERY_TO_PYTHON_TYPES`)
##########
sdks/python/apache_beam/io/gcp/bigquery_schema_tools.py:
##########
@@ -0,0 +1,115 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""Tools used tool work with Schema types in the context of BigQuery.
+Classes, constants and functions in this file are experimental and have no
+backwards compatibility guarantees.
+NOTHING IN THIS FILE HAS BACKWARDS COMPATIBILITY GUARANTEES.
+"""
+import apache_beam as beam
+import numpy as np
+from typing import Optional
+from typing import Sequence
+
+# BigQuery types as listed in
+# https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types
+# with aliases (RECORD, BOOLEAN, FLOAT, INTEGER) as defined in
+#
https://developers.google.com/resources/api-libraries/documentation/bigquery/v2/java/latest/com/google/api/services/bigquery/model/TableFieldSchema.html#setType-java.lang.String-
+BIG_QUERY_TO_PYTHON_TYPES = {
+ "STRING": "str",
+ "INTEGER": "np.int64",
+ "FLOAT64": "np.float64",
+ "BOOLEAN": "bool",
+ "BYTES": "bytes",
+ "TIMESTAMP": "apache_beam.utils.timestamp.Timestamp",
+ #TODO svetaksundhar@: Finish mappings for all BQ types
+}
+
+
+def produce_pcoll_with_schema(self, the_table_schema):
+ #type: [TableSchema] -> [UserType]
Review Comment:
I think mypy will complain about this, this should work better:
```suggestion
#type: (TableSchema) -> type
```
##########
sdks/python/apache_beam/io/gcp/bigquery_schema_tools.py:
##########
@@ -0,0 +1,115 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""Tools used tool work with Schema types in the context of BigQuery.
+Classes, constants and functions in this file are experimental and have no
+backwards compatibility guarantees.
+NOTHING IN THIS FILE HAS BACKWARDS COMPATIBILITY GUARANTEES.
+"""
+import apache_beam as beam
+import numpy as np
+from typing import Optional
+from typing import Sequence
+
+# BigQuery types as listed in
+# https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types
+# with aliases (RECORD, BOOLEAN, FLOAT, INTEGER) as defined in
+#
https://developers.google.com/resources/api-libraries/documentation/bigquery/v2/java/latest/com/google/api/services/bigquery/model/TableFieldSchema.html#setType-java.lang.String-
+BIG_QUERY_TO_PYTHON_TYPES = {
+ "STRING": "str",
+ "INTEGER": "np.int64",
+ "FLOAT64": "np.float64",
+ "BOOLEAN": "bool",
+ "BYTES": "bytes",
+ "TIMESTAMP": "apache_beam.utils.timestamp.Timestamp",
+ #TODO svetaksundhar@: Finish mappings for all BQ types
+}
+
+
+def produce_pcoll_with_schema(self, the_table_schema):
+ #type: [TableSchema] -> [UserType]
+
+ """Convert a schema of type TableSchema into a pcollection element.
+ Args:
+ the_table_schema: A BQ schema of type TableSchema
+ Returns:
+ usertype: type that can be used to work with pCollections.
+ """
+
+ the_schema = beam.io.gcp.bigquery_tools.get_dict_table_schema(
+ the_table_schema)
+ if the_schema == {}:
+ raise ValueError("The schema is empty")
+ i = 0
+ dict_of_tuples = []
+ for x in the_schema['fields']:
+ if the_schema['fields'][i]['type'] == 'STRING':
+ if the_schema['fields'][i]['mode'] == 'NULLABLE':
+ typ = Optional[str]
+ elif the_schema['fields'][i]['mode'] == 'REPEATED':
+ typ = Sequence[str]
+ else:
+ typ = str
Review Comment:
nit: I'd make this `elif mode == ''` case, and the else case should raise an
error. That way we won't accidentally accept a mode we don't understand.
##########
sdks/python/apache_beam/io/gcp/bigquery_schema_tools.py:
##########
@@ -0,0 +1,115 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""Tools used tool work with Schema types in the context of BigQuery.
+Classes, constants and functions in this file are experimental and have no
+backwards compatibility guarantees.
+NOTHING IN THIS FILE HAS BACKWARDS COMPATIBILITY GUARANTEES.
+"""
+import apache_beam as beam
+import numpy as np
+from typing import Optional
+from typing import Sequence
+
+# BigQuery types as listed in
+# https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types
+# with aliases (RECORD, BOOLEAN, FLOAT, INTEGER) as defined in
+#
https://developers.google.com/resources/api-libraries/documentation/bigquery/v2/java/latest/com/google/api/services/bigquery/model/TableFieldSchema.html#setType-java.lang.String-
+BIG_QUERY_TO_PYTHON_TYPES = {
+ "STRING": "str",
+ "INTEGER": "np.int64",
+ "FLOAT64": "np.float64",
+ "BOOLEAN": "bool",
+ "BYTES": "bytes",
+ "TIMESTAMP": "apache_beam.utils.timestamp.Timestamp",
+ #TODO svetaksundhar@: Finish mappings for all BQ types
+}
+
+
+def produce_pcoll_with_schema(self, the_table_schema):
Review Comment:
```suggestion
def produce_pcoll_with_schema(the_table_schema):
```
This is a stand-alone function, no need for self.
Issue Time Tracking
-------------------
Worklog Id: (was: 753773)
Time Spent: 3h 20m (was: 3h 10m)
> Support pd.read_gbq and DataFrame.to_gbq
> ----------------------------------------
>
> Key: BEAM-11587
> URL: https://issues.apache.org/jira/browse/BEAM-11587
> Project: Beam
> Issue Type: New Feature
> Components: dsl-dataframe, io-py-gcp, sdk-py-core
> Reporter: Brian Hulette
> Assignee: Svetak Vihaan Sundhar
> Priority: P3
> Labels: dataframe-api
> Time Spent: 3h 20m
> Remaining Estimate: 0h
>
> We should support
> [read_gbq|https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.read_gbq.html]
> andÂ
> [to_gbq|https://pandas.pydata.org/docs/reference/api/pandas.DataFrame.to_gbq.html]
> in the DataFrame API when gcp extras are installed.
--
This message was sent by Atlassian Jira
(v8.20.1#820001)