private void setupBigQueryTable( String projectId, String datasetId, String tableId, TableSchema schema) throws IOException { if (bigQueryClient == null) { bigQueryClient = Transport.newBigQueryClient(options.as(BigQueryOptions.class)).build(); } Datasets datasetService = bigQueryClient.datasets(); if (executeNullIfNotFound(datasetService.get(projectId, datasetId)) == null) { Dataset newDataset = new Dataset() .setDatasetReference( new DatasetReference().setProjectId(projectId).setDatasetId(datasetId)); datasetService.insert(projectId, newDataset).execute(); } Tables tableService = bigQueryClient.tables(); Table table = executeNullIfNotFound(tableService.get(projectId, datasetId, tableId)); if (table == null) { Table newTable = new Table() .setSchema(schema) .setTableReference( new TableReference() .setProjectId(projectId) .setDatasetId(datasetId) .setTableId(tableId)); tableService.insert(projectId, datasetId, newTable).execute(); } else if (!table.getSchema().equals(schema)) { throw new RuntimeException( "Table exists and schemas do not match, expecting: " + schema.toPrettyString() + ", actual: " + table.getSchema().toPrettyString()); } }
@Test public void testConvertGenericRecordToTableRow() throws Exception { TableSchema tableSchema = new TableSchema(); tableSchema.setFields(fields); Schema avroSchema = AvroCoder.of(Bird.class).getSchema(); { // Test nullable fields. GenericRecord record = new GenericData.Record(avroSchema); record.put("number", 5L); TableRow convertedRow = BigQueryAvroUtils.convertGenericRecordToTableRow(record, tableSchema); TableRow row = new TableRow().set("number", "5").set("associates", new ArrayList<TableRow>()); assertEquals(row, convertedRow); } { // Test type conversion for: // INTEGER, FLOAT, TIMESTAMP, BOOLEAN, BYTES, DATE, DATETIME, TIME. GenericRecord record = new GenericData.Record(avroSchema); byte[] soundBytes = "chirp,chirp".getBytes(); ByteBuffer soundByteBuffer = ByteBuffer.wrap(soundBytes); soundByteBuffer.rewind(); record.put("number", 5L); record.put("quality", 5.0); record.put("birthday", 5L); record.put("flighted", Boolean.TRUE); record.put("sound", soundByteBuffer); record.put("anniversaryDate", new Utf8("2000-01-01")); record.put("anniversaryDatetime", new String("2000-01-01 00:00:00.000005")); record.put("anniversaryTime", new Utf8("00:00:00.000005")); TableRow convertedRow = BigQueryAvroUtils.convertGenericRecordToTableRow(record, tableSchema); TableRow row = new TableRow() .set("number", "5") .set("birthday", "1970-01-01 00:00:00.000005 UTC") .set("quality", 5.0) .set("associates", new ArrayList<TableRow>()) .set("flighted", Boolean.TRUE) .set("sound", BaseEncoding.base64().encode(soundBytes)) .set("anniversaryDate", "2000-01-01") .set("anniversaryDatetime", "2000-01-01 00:00:00.000005") .set("anniversaryTime", "00:00:00.000005"); assertEquals(row, convertedRow); } { // Test repeated fields. Schema subBirdSchema = AvroCoder.of(Bird.SubBird.class).getSchema(); GenericRecord nestedRecord = new GenericData.Record(subBirdSchema); nestedRecord.put("species", "other"); GenericRecord record = new GenericData.Record(avroSchema); record.put("number", 5L); record.put("associates", Lists.<GenericRecord>newArrayList(nestedRecord)); TableRow convertedRow = BigQueryAvroUtils.convertGenericRecordToTableRow(record, tableSchema); TableRow row = new TableRow() .set( "associates", Lists.<TableRow>newArrayList(new TableRow().set("species", "other"))) .set("number", "5"); assertEquals(row, convertedRow); } }
@Test public void testConvertBigQuerySchemaToAvroSchema() { TableSchema tableSchema = new TableSchema(); tableSchema.setFields(fields); Schema avroSchema = BigQueryAvroUtils.toGenericAvroSchema("testSchema", tableSchema.getFields()); assertThat(avroSchema.getField("number").schema(), equalTo(Schema.create(Type.LONG))); assertThat( avroSchema.getField("species").schema(), equalTo(Schema.createUnion(Schema.create(Type.NULL), Schema.create(Type.STRING)))); assertThat( avroSchema.getField("quality").schema(), equalTo(Schema.createUnion(Schema.create(Type.NULL), Schema.create(Type.DOUBLE)))); assertThat( avroSchema.getField("quantity").schema(), equalTo(Schema.createUnion(Schema.create(Type.NULL), Schema.create(Type.LONG)))); assertThat( avroSchema.getField("birthday").schema(), equalTo(Schema.createUnion(Schema.create(Type.NULL), Schema.create(Type.LONG)))); assertThat( avroSchema.getField("flighted").schema(), equalTo(Schema.createUnion(Schema.create(Type.NULL), Schema.create(Type.BOOLEAN)))); assertThat( avroSchema.getField("sound").schema(), equalTo(Schema.createUnion(Schema.create(Type.NULL), Schema.create(Type.BYTES)))); assertThat( avroSchema.getField("anniversaryDate").schema(), equalTo(Schema.createUnion(Schema.create(Type.NULL), Schema.create(Type.STRING)))); assertThat( avroSchema.getField("anniversaryDatetime").schema(), equalTo(Schema.createUnion(Schema.create(Type.NULL), Schema.create(Type.STRING)))); assertThat( avroSchema.getField("anniversaryTime").schema(), equalTo(Schema.createUnion(Schema.create(Type.NULL), Schema.create(Type.STRING)))); assertThat( avroSchema.getField("scion").schema(), equalTo( Schema.createUnion( Schema.create(Type.NULL), Schema.createRecord( "scion", "org.apache.beam.sdk.io.gcp.bigquery", "Translated Avro Schema for scion", false, ImmutableList.of( new Field( "species", Schema.createUnion( Schema.create(Type.NULL), Schema.create(Type.STRING)), null, (Object) null)))))); assertThat( avroSchema.getField("associates").schema(), equalTo( Schema.createArray( Schema.createRecord( "associates", "org.apache.beam.sdk.io.gcp.bigquery", "Translated Avro Schema for associates", false, ImmutableList.of( new Field( "species", Schema.createUnion( Schema.create(Type.NULL), Schema.create(Type.STRING)), null, (Object) null)))))); }