def test_row_as_table_row(self): schema_definition = [ ('s', 'STRING'), ('i', 'INTEGER'), ('f', 'FLOAT'), ('b', 'BOOLEAN'), ('r', 'RECORD')] data_defination = [ 'abc', 123, 123.456, True, {'a': 'b'}] str_def = '{"s": "abc", "i": 123, "f": 123.456, "b": true, "r": {"a": "b"}}' schema = bigquery.TableSchema( fields=[bigquery.TableFieldSchema(name=k, type=v) for k, v in schema_definition]) coder = TableRowJsonCoder(table_schema=schema) test_row = bigquery.TableRow( f=[bigquery.TableCell(v=to_json_value(e)) for e in data_defination]) self.assertEqual(str_def, coder.encode(test_row)) self.assertEqual(test_row, coder.decode(coder.encode(test_row))) # A coder without schema can still decode. self.assertEqual( test_row, TableRowJsonCoder().decode(coder.encode(test_row)))
def field(name, kind='string', mode='nullable'): f = bigquery.TableFieldSchema() f.name = name f.type = kind f.mode = mode return f
def test_parse_table_schema_from_json(self): string_field = bigquery.TableFieldSchema( name='s', type='STRING', mode='NULLABLE', description='s description') number_field = bigquery.TableFieldSchema( name='n', type='INTEGER', mode='REQUIRED', description='n description') record_field = bigquery.TableFieldSchema( name='r', type='RECORD', mode='REQUIRED', description='r description', fields=[string_field, number_field]) expected_schema = bigquery.TableSchema(fields=[record_field]) json_str = json.dumps({'fields': [ {'name': 'r', 'type': 'RECORD', 'mode': 'REQUIRED', 'description': 'r description', 'fields': [ {'name': 's', 'type': 'STRING', 'mode': 'NULLABLE', 'description': 's description'}, {'name': 'n', 'type': 'INTEGER', 'mode': 'REQUIRED', 'description': 'n description'}]}]}) self.assertEqual(parse_table_schema_from_json(json_str), expected_schema)
def test_nested_schema_as_json(self): string_field = bigquery.TableFieldSchema( name='s', type='STRING', mode='NULLABLE', description='s description') number_field = bigquery.TableFieldSchema( name='n', type='INTEGER', mode='REQUIRED', description='n description') record_field = bigquery.TableFieldSchema( name='r', type='RECORD', mode='REQUIRED', description='r description', fields=[string_field, number_field]) schema = bigquery.TableSchema(fields=[record_field]) sink = beam.io.BigQuerySink('dataset.table', schema=schema) self.assertEqual( {'fields': [ {'name': 'r', 'type': 'RECORD', 'mode': 'REQUIRED', 'description': 'r description', 'fields': [ {'name': 's', 'type': 'STRING', 'mode': 'NULLABLE', 'description': 's description'}, {'name': 'n', 'type': 'INTEGER', 'mode': 'REQUIRED', 'description': 'n description'}]}]}, json.loads(sink.schema_as_json()))
def json_compliance_exception(self, value): with self.assertRaises(ValueError) as exn: schema_definition = [('f', 'FLOAT')] schema = bigquery.TableSchema( fields=[bigquery.TableFieldSchema(name=k, type=v) for k, v in schema_definition]) coder = TableRowJsonCoder(table_schema=schema) test_row = bigquery.TableRow( f=[bigquery.TableCell(v=to_json_value(value))]) coder.encode(test_row) self.assertTrue(bigquery.JSON_COMPLIANCE_ERROR in exn.exception.message)
def _parse_schema_field(field): """Parse a single schema field from dictionary. Args: field: Dictionary object containing serialized schema. Returns: A TableFieldSchema for a single column in BigQuery. """ schema = bigquery.TableFieldSchema() schema.name = field['name'] schema.type = field['type'] if 'mode' in field: schema.mode = field['mode'] else: schema.mode = 'NULLABLE' if 'description' in field: schema.description = field['description'] if 'fields' in field: schema.fields = [_parse_schema_field(x) for x in field['fields']] return schema
def __init__(self, table, dataset=None, project=None, schema=None, create_disposition=BigQueryDisposition.CREATE_IF_NEEDED, write_disposition=BigQueryDisposition.WRITE_EMPTY, validate=False, coder=None): """Initialize a BigQuerySink. Args: table: The ID of the table. The ID must contain only letters (a-z, A-Z), numbers (0-9), or underscores (_). If dataset argument is None then the table argument must contain the entire table reference specified as: 'DATASET.TABLE' or 'PROJECT:DATASET.TABLE'. dataset: The ID of the dataset containing this table or null if the table reference is specified entirely by the table argument. project: The ID of the project containing this table or null if the table reference is specified entirely by the table argument. schema: The schema to be used if the BigQuery table to write has to be created. This can be either specified as a 'bigquery.TableSchema' object or a single string of the form 'field1:type1,field2:type2,field3:type3' that defines a comma separated list of fields. Here 'type' should specify the BigQuery type of the field. Single string based schemas do not support nested fields, repeated fields, or specifying a BigQuery mode for fields (mode will always be set to 'NULLABLE'). create_disposition: A string describing what happens if the table does not exist. Possible values are: - BigQueryDisposition.CREATE_IF_NEEDED: create if does not exist. - BigQueryDisposition.CREATE_NEVER: fail the write if does not exist. write_disposition: A string describing what happens if the table has already some data. Possible values are: - BigQueryDisposition.WRITE_TRUNCATE: delete existing rows. - BigQueryDisposition.WRITE_APPEND: add to existing rows. - BigQueryDisposition.WRITE_EMPTY: fail the write if table not empty. validate: If true, various checks will be done when sink gets initialized (e.g., is table present given the disposition arguments?). This should be True for most scenarios in order to catch errors as early as possible (pipeline construction instead of pipeline execution). It should be False if the table is created during pipeline execution by a previous step. coder: The coder for the table rows if serialized to disk. If None, then the default coder is RowAsDictJsonCoder, which will interpret every element written to the sink as a dictionary that will be JSON serialized as a line in a file. This argument needs a value only in special cases when writing table rows as dictionaries is not desirable. Raises: TypeError: if the schema argument is not a string or a TableSchema object. ValueError: if the table reference as a string does not match the expected format. """ self.table_reference = _parse_table_reference(table, dataset, project) # Transform the table schema into a bigquery.TableSchema instance. if isinstance(schema, basestring): # TODO(silviuc): Should add a regex-based validation of the format. table_schema = bigquery.TableSchema() schema_list = [s.strip(' ') for s in schema.split(',')] for field_and_type in schema_list: field_name, field_type = field_and_type.split(':') field_schema = bigquery.TableFieldSchema() field_schema.name = field_name field_schema.type = field_type field_schema.mode = 'NULLABLE' table_schema.fields.append(field_schema) self.table_schema = table_schema elif schema is None: # TODO(silviuc): Should check that table exists if no schema specified. self.table_schema = schema elif isinstance(schema, bigquery.TableSchema): self.table_schema = schema else: raise TypeError('Unexpected schema argument: %s.' % schema) self.create_disposition = BigQueryDisposition.validate_create( create_disposition) self.write_disposition = BigQueryDisposition.validate_write( write_disposition) self.validate = validate self.coder = coder or RowAsDictJsonCoder()
def get_test_rows(self): now = time.time() dt = datetime.datetime.utcfromtimestamp(float(now)) ts = dt.strftime('%Y-%m-%d %H:%M:%S.%f UTC') expected_rows = [{ 'i': 1, 's': 'abc', 'f': 2.3, 'b': True, 't': ts, 'dt': '2016-10-31', 'ts': '22:39:12.627498', 'dt_ts': '2008-12-25T07:30:00', 'r': { 's2': 'b' }, 'rpr': [{ 's3': 'c', 'rpr2': [{ 'rs': ['d', 'e'], 's4': 'f' }] }] }, { 'i': 10, 's': 'xyz', 'f': -3.14, 'b': False, 'rpr': [] }] nested_schema = [ bigquery.TableFieldSchema(name='s2', type='STRING', mode='NULLABLE') ] nested_schema_2 = [ bigquery.TableFieldSchema(name='s3', type='STRING', mode='NULLABLE'), bigquery.TableFieldSchema( name='rpr2', type='RECORD', mode='REPEATED', fields=[ bigquery.TableFieldSchema(name='rs', type='STRING', mode='REPEATED'), bigquery.TableFieldSchema(name='s4', type='STRING', mode='NULLABLE') ]) ] schema = bigquery.TableSchema(fields=[ bigquery.TableFieldSchema( name='b', type='BOOLEAN', mode='REQUIRED'), bigquery.TableFieldSchema(name='f', type='FLOAT', mode='REQUIRED'), bigquery.TableFieldSchema( name='i', type='INTEGER', mode='REQUIRED'), bigquery.TableFieldSchema(name='s', type='STRING', mode='REQUIRED'), bigquery.TableFieldSchema( name='t', type='TIMESTAMP', mode='NULLABLE'), bigquery.TableFieldSchema(name='dt', type='DATE', mode='NULLABLE'), bigquery.TableFieldSchema(name='ts', type='TIME', mode='NULLABLE'), bigquery.TableFieldSchema( name='dt_ts', type='DATETIME', mode='NULLABLE'), bigquery.TableFieldSchema( name='r', type='RECORD', mode='NULLABLE', fields=nested_schema), bigquery.TableFieldSchema(name='rpr', type='RECORD', mode='REPEATED', fields=nested_schema_2) ]) table_rows = [ bigquery.TableRow(f=[ bigquery.TableCell(v=to_json_value('true')), bigquery.TableCell(v=to_json_value(str(2.3))), bigquery.TableCell(v=to_json_value(str(1))), bigquery.TableCell(v=to_json_value('abc')), # For timestamps cannot use str() because it will truncate the # number representing the timestamp. bigquery.TableCell(v=to_json_value('%f' % now)), bigquery.TableCell(v=to_json_value('2016-10-31')), bigquery.TableCell(v=to_json_value('22:39:12.627498')), bigquery.TableCell(v=to_json_value('2008-12-25T07:30:00')), # For record we cannot use dict because it doesn't create nested # schemas correctly so we have to use this f,v based format bigquery.TableCell(v=to_json_value({'f': [{ 'v': 'b' }]})), bigquery.TableCell(v=to_json_value([{ 'v': { 'f': [{ 'v': 'c' }, { 'v': [{ 'v': { 'f': [{ 'v': [{ 'v': 'd' }, { 'v': 'e' }] }, { 'v': 'f' }] } }] }] } }])) ]), bigquery.TableRow(f=[ bigquery.TableCell(v=to_json_value('false')), bigquery.TableCell(v=to_json_value(str(-3.14))), bigquery.TableCell(v=to_json_value(str(10))), bigquery.TableCell(v=to_json_value('xyz')), bigquery.TableCell(v=None), bigquery.TableCell(v=None), bigquery.TableCell(v=None), bigquery.TableCell(v=None), bigquery.TableCell(v=None), bigquery.TableCell(v=to_json_value([])) ]) ] return table_rows, schema, expected_rows
def run(argv=None): """Run the workflow.""" parser = argparse.ArgumentParser() parser.add_argument( '--output', required=True, help= ('Output BigQuery table for results specified as: PROJECT:DATASET.TABLE ' 'or DATASET.TABLE.')) known_args, pipeline_args = parser.parse_known_args(argv) p = beam.Pipeline(argv=pipeline_args) from apache_beam.internal.clients import bigquery # pylint: disable=wrong-import-order, wrong-import-position table_schema = bigquery.TableSchema() # Fields that use standard types. kind_schema = bigquery.TableFieldSchema() kind_schema.name = 'kind' kind_schema.type = 'string' kind_schema.mode = 'nullable' table_schema.fields.append(kind_schema) full_name_schema = bigquery.TableFieldSchema() full_name_schema.name = 'fullName' full_name_schema.type = 'string' full_name_schema.mode = 'required' table_schema.fields.append(full_name_schema) age_schema = bigquery.TableFieldSchema() age_schema.name = 'age' age_schema.type = 'integer' age_schema.mode = 'nullable' table_schema.fields.append(age_schema) gender_schema = bigquery.TableFieldSchema() gender_schema.name = 'gender' gender_schema.type = 'string' gender_schema.mode = 'nullable' table_schema.fields.append(gender_schema) # A nested field phone_number_schema = bigquery.TableFieldSchema() phone_number_schema.name = 'phoneNumber' phone_number_schema.type = 'record' phone_number_schema.mode = 'nullable' area_code = bigquery.TableFieldSchema() area_code.name = 'areaCode' area_code.type = 'integer' area_code.mode = 'nullable' phone_number_schema.fields.append(area_code) number = bigquery.TableFieldSchema() number.name = 'number' number.type = 'integer' number.mode = 'nullable' phone_number_schema.fields.append(number) table_schema.fields.append(phone_number_schema) # A repeated field. children_schema = bigquery.TableFieldSchema() children_schema.name = 'children' children_schema.type = 'string' children_schema.mode = 'repeated' table_schema.fields.append(children_schema) def create_random_record(record_id): return {'kind': 'kind' + record_id, 'fullName': 'fullName'+record_id, 'age': int(record_id) * 10, 'gender': 'male', 'phoneNumber': { 'areaCode': int(record_id) * 100, 'number': int(record_id) * 100000}, 'children': ['child' + record_id + '1', 'child' + record_id + '2', 'child' + record_id + '3'] } # pylint: disable=expression-not-assigned record_ids = p | 'CreateIDs' >> beam.Create(['1', '2', '3', '4', '5']) records = record_ids | 'CreateRecords' >> beam.Map(create_random_record) records | beam.io.Write( 'write', beam.io.BigQuerySink( known_args.output, schema=table_schema, create_disposition=beam.io.BigQueryDisposition.CREATE_IF_NEEDED, write_disposition=beam.io.BigQueryDisposition.WRITE_TRUNCATE)) # Run the pipeline (all operations are deferred until run() is called). p.run()