github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/io/gcp/bigquery_schema_tools.py (about) 1 # 2 # Licensed to the Apache Software Foundation (ASF) under one or more 3 # contributor license agreements. See the NOTICE file distributed with 4 # this work for additional information regarding copyright ownership. 5 # The ASF licenses this file to You under the Apache License, Version 2.0 6 # (the "License"); you may not use this file except in compliance with 7 # the License. You may obtain a copy of the License at 8 # 9 # http://www.apache.org/licenses/LICENSE-2.0 10 # 11 # Unless required by applicable law or agreed to in writing, software 12 # distributed under the License is distributed on an "AS IS" BASIS, 13 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 # See the License for the specific language governing permissions and 15 # limitations under the License. 16 # 17 18 """Tools used tool work with Schema types in the context of BigQuery. 19 Classes, constants and functions in this file are experimental and have no 20 backwards compatibility guarantees. 21 NOTHING IN THIS FILE HAS BACKWARDS COMPATIBILITY GUARANTEES. 22 """ 23 24 import datetime 25 from typing import Optional 26 from typing import Sequence 27 28 import numpy as np 29 30 import apache_beam as beam 31 import apache_beam.io.gcp.bigquery_tools 32 import apache_beam.typehints.schemas 33 import apache_beam.utils.proto_utils 34 import apache_beam.utils.timestamp 35 from apache_beam.io.gcp.internal.clients import bigquery 36 from apache_beam.portability.api import schema_pb2 37 from apache_beam.transforms import DoFn 38 39 # BigQuery types as listed in 40 # https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types 41 # with aliases (RECORD, BOOLEAN, FLOAT, INTEGER) as defined in 42 # https://developers.google.com/resources/api-libraries/documentation/bigquery/v2/java/latest/com/google/api/services/bigquery/model/TableFieldSchema.html#setType-java.lang.String- 43 BIG_QUERY_TO_PYTHON_TYPES = { 44 "STRING": str, 45 "INTEGER": np.int64, 46 "FLOAT64": np.float64, 47 "FLOAT": np.float64, 48 "BOOLEAN": bool, 49 "BYTES": bytes, 50 "TIMESTAMP": apache_beam.utils.timestamp.Timestamp 51 #TODO(https://github.com/apache/beam/issues/20810): 52 # Finish mappings for all BQ types 53 } 54 55 56 def generate_user_type_from_bq_schema(the_table_schema): 57 #type: (bigquery.TableSchema) -> type 58 59 """Convert a schema of type TableSchema into a pcollection element. 60 Args: 61 the_table_schema: A BQ schema of type TableSchema 62 Returns: 63 type: type that can be used to work with pCollections. 64 """ 65 66 the_schema = beam.io.gcp.bigquery_tools.get_dict_table_schema( 67 the_table_schema) 68 if the_schema == {}: 69 raise ValueError("Encountered an empty schema") 70 dict_of_tuples = [] 71 for i in range(len(the_schema['fields'])): 72 if the_schema['fields'][i]['type'] in BIG_QUERY_TO_PYTHON_TYPES: 73 typ = bq_field_to_type( 74 the_schema['fields'][i]['type'], the_schema['fields'][i]['mode']) 75 else: 76 raise ValueError( 77 f"Encountered " 78 f"an unsupported type: {the_schema['fields'][i]['type']!r}") 79 # TODO svetaksundhar@: Map remaining BQ types 80 dict_of_tuples.append((the_schema['fields'][i]['name'], typ)) 81 sample_schema = beam.typehints.schemas.named_fields_to_schema(dict_of_tuples) 82 usertype = beam.typehints.schemas.named_tuple_from_schema(sample_schema) 83 return usertype 84 85 86 def bq_field_to_type(field, mode): 87 if mode == 'NULLABLE' or mode is None or mode == '': 88 return Optional[BIG_QUERY_TO_PYTHON_TYPES[field]] 89 elif mode == 'REPEATED': 90 return Sequence[BIG_QUERY_TO_PYTHON_TYPES[field]] 91 elif mode == 'REQUIRED': 92 return BIG_QUERY_TO_PYTHON_TYPES[field] 93 else: 94 raise ValueError(f"Encountered an unsupported mode: {mode!r}") 95 96 97 def convert_to_usertype(table_schema): 98 usertype = generate_user_type_from_bq_schema(table_schema) 99 return beam.ParDo(BeamSchemaConversionDoFn(usertype)) 100 101 102 class BeamSchemaConversionDoFn(DoFn): 103 def __init__(self, pcoll_val_ctor): 104 self._pcoll_val_ctor = pcoll_val_ctor 105 106 def process(self, dict_of_tuples): 107 for k, v in dict_of_tuples.items(): 108 if isinstance(v, datetime.datetime): 109 dict_of_tuples[k] = beam.utils.timestamp.Timestamp.from_utc_datetime(v) 110 yield self._pcoll_val_ctor(**dict_of_tuples) 111 112 def infer_output_type(self, input_type): 113 return self._pcoll_val_ctor 114 115 @classmethod 116 def _from_serialized_schema(cls, schema_str): 117 return cls( 118 apache_beam.typehints.schemas.named_tuple_from_schema( 119 apache_beam.utils.proto_utils.parse_Bytes( 120 schema_str, schema_pb2.Schema))) 121 122 def __reduce__(self): 123 # when pickling, use bytes representation of the schema. 124 return ( 125 self._from_serialized_schema, 126 ( 127 beam.typehints.schemas.named_tuple_to_schema( 128 self._pcoll_val_ctor).SerializeToString(), ))