@Override public POutput buildIOWriter(PCollection<Row> input) { return input.apply( BigQueryIO.<Row>write() .withSchema(BigQueryUtils.toTableSchema(getSchema())) .withFormatFunction(BigQueryUtils.toTableRow()) .to(tableSpec)); }
private List<Row> bqRowsToBeamRows( TableSchema bqSchema, List<TableRow> bqRows, Schema rowSchema) { if (bqRows == null) { return Collections.emptyList(); } return bqRows .stream() .map(bqRow -> toBeamRow(rowSchema, bqSchema, bqRow)) .collect(Collectors.toList()); }
private static Object toBeamValue(FieldType fieldType, Object jsonBQValue) { if (jsonBQValue instanceof String && JSON_VALUE_PARSERS.containsKey(fieldType.getTypeName())) { return JSON_VALUE_PARSERS.get(fieldType.getTypeName()).apply((String) jsonBQValue); } if (jsonBQValue instanceof List) { return ((List<Object>) jsonBQValue) .stream() .map(v -> ((Map<String, Object>) v).get("v")) .map(v -> toBeamValue(fieldType.getCollectionElementType(), v)) .collect(toList()); } throw new UnsupportedOperationException( "Converting BigQuery type '" + jsonBQValue.getClass() + "' to '" + fieldType + "' is not supported"); } }
@Override public TableRow apply(Row input) { return toTableRow(input); } }
/** Convert a Beam {@link PCollection} to a BigQuery {@link TableSchema}. */ public static TableSchema toTableSchema(PCollection<Row> rows) { RowCoder coder = (RowCoder) rows.getCoder(); return toTableSchema(coder.getSchema()); }
private static List<TableFieldSchema> toTableFieldSchema(Schema schema) { List<TableFieldSchema> fields = new ArrayList<>(schema.getFieldCount()); for (Field schemaField : schema.getFields()) { FieldType type = schemaField.getType(); TableFieldSchema field = new TableFieldSchema().setName(schemaField.getName()); if (schemaField.getDescription() != null && !"".equals(schemaField.getDescription())) { field.setDescription(schemaField.getDescription()); } if (!schemaField.getNullable()) { field.setMode(Mode.REQUIRED.toString()); } if (TypeName.ARRAY == type.getTypeName()) { type = type.getCollectionElementType(); field.setMode(Mode.REPEATED.toString()); } if (TypeName.ROW == type.getTypeName()) { Schema subType = type.getRowSchema(); field.setFields(toTableFieldSchema(subType)); } field.setType(toStandardSQLTypeName(type).toString()); fields.add(field); } return fields; }
/** Convert a Beam {@link Schema} to a BigQuery {@link TableSchema}. */ public static TableSchema toTableSchema(Schema schema) { return new TableSchema().setFields(toTableFieldSchema(schema)); }
List<TableRow> tableRows = new ArrayList<>(rows.size()); for (int j = 0; j < rows.size(); j++) { tableRows.add(toTableRow(rows.get(j))); value = toTableRow((Row) value); break; case DATETIME:
@Test public void testToTableSchema_flat() { TableSchema schema = toTableSchema(FLAT_TYPE); assertThat(schema.getFields(), containsInAnyOrder(ID, VALUE, NAME, TIMESTAMP, VALID)); }
@Test public void testToTableRow_array() { TableRow row = toTableRow().apply(ARRAY_ROW); assertThat(row, hasEntry("ids", Arrays.asList(123L, 124L))); assertThat(row.size(), equalTo(1)); }
@Test public void testToTableSchema_array() { TableSchema schema = toTableSchema(ARRAY_TYPE); assertThat(schema.getFields(), contains(IDS)); }
@Override public Row apply(SchemaAndRecord input) { GenericRecord record = input.getRecord(); checkState( schema.getFields().size() == record.getSchema().getFields().size(), "Schema sizes are different."); return toBeamRow(record, schema); } }
/** * Tries to parse the JSON {@link TableRow} from BigQuery. * * <p>Only supports basic types and arrays. Doesn't support date types. */ public static Row toBeamRow(Schema rowSchema, TableSchema bqSchema, TableRow jsonBqRow) { List<TableFieldSchema> bqFields = bqSchema.getFields(); Map<String, Integer> bqFieldIndices = IntStream.range(0, bqFields.size()) .boxed() .collect(toMap(i -> bqFields.get(i).getName(), i -> i)); List<Object> rawJsonValues = rowSchema .getFields() .stream() .map(field -> bqFieldIndices.get(field.getName())) .map(index -> jsonBqRow.getF().get(index).getV()) .collect(toList()); return IntStream.range(0, rowSchema.getFieldCount()) .boxed() .map(index -> toBeamValue(rowSchema.getField(index).getType(), rawJsonValues.get(index))) .collect(toRow(rowSchema)); }
@Test public void testToTableRow_array_row() { TableRow row = toTableRow().apply(ARRAY_ROW_ROW); assertThat(row.size(), equalTo(1)); row = ((List<TableRow>) row.get("rows")).get(0); assertThat(row.size(), equalTo(5)); assertThat(row, hasEntry("id", 123L)); assertThat(row, hasEntry("value", 123.456)); assertThat(row, hasEntry("name", "test")); assertThat(row, hasEntry("valid", false)); }
@Test public void testToTableSchema_row() { TableSchema schema = toTableSchema(ROW_TYPE); assertThat(schema.getFields().size(), equalTo(1)); TableFieldSchema field = schema.getFields().get(0); assertThat(field.getName(), equalTo("row")); assertThat(field.getType(), equalTo(StandardSQLTypeName.STRUCT.toString())); assertThat(field.getMode(), nullValue()); assertThat(field.getFields(), containsInAnyOrder(ID, VALUE, NAME, TIMESTAMP, VALID)); }
@Override public PCollection<Row> buildIOReader(PBegin begin) { // TODO: make this more generic. return begin .apply(BigQueryIO.read(BigQueryUtils.toBeamRow(schema)).from(tableSpec)) .setRowSchema(getSchema()); }
@Test public void testToTableRow_row() { TableRow row = toTableRow().apply(ROW_ROW); assertThat(row.size(), equalTo(1)); row = (TableRow) row.get("row"); assertThat(row.size(), equalTo(5)); assertThat(row, hasEntry("id", 123L)); assertThat(row, hasEntry("value", 123.456)); assertThat(row, hasEntry("name", "test")); assertThat(row, hasEntry("valid", false)); }
@Test public void testToTableSchema_array_row() { TableSchema schema = toTableSchema(ARRAY_ROW_TYPE); assertThat(schema.getFields().size(), equalTo(1)); TableFieldSchema field = schema.getFields().get(0); assertThat(field.getName(), equalTo("rows")); assertThat(field.getType(), equalTo(StandardSQLTypeName.STRUCT.toString())); assertThat(field.getMode(), equalTo(Mode.REPEATED.toString())); assertThat(field.getFields(), containsInAnyOrder(ID, VALUE, NAME, TIMESTAMP, VALID)); }
@Test public void testToTableRow_flat() { TableRow row = toTableRow().apply(FLAT_ROW); assertThat(row.size(), equalTo(5)); assertThat(row, hasEntry("id", 123L)); assertThat(row, hasEntry("value", 123.456)); assertThat(row, hasEntry("name", "test")); assertThat(row, hasEntry("valid", false)); }
private Table createTable(Description description) throws IOException, InterruptedException { TableReference tableReference = new TableReference() .setProjectId(pipelineOptions.getProject()) .setDatasetId(pipelineOptions.getTargetDataset()) .setTableId(createRandomizedName(description)); table = new Table() .setTableReference(tableReference) .setSchema(BigQueryUtils.toTableSchema(schema)) .setDescription( "Table created for " + description.getDisplayName() + " by TestBigQueryRule. " + "Should be automatically cleaned up after test completion."); if (datasetService.getTable(tableReference) != null) { throw new IllegalStateException( "Table '" + tableReference + "' already exists. " + "It should have been cleaned up by the test rule."); } datasetService.createTable(table); return table; }