Skip to content

Commit

Permalink
Merge pull request #14964: [BEAM-12460] Provide a simpler interface t…
Browse files Browse the repository at this point in the history
…o convert Beam Row to GenericRecord
  • Loading branch information
iemejia authored Jun 10, 2021
2 parents 9cd7479 + c13a0cc commit 67164a1
Show file tree
Hide file tree
Showing 2 changed files with 15 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -369,6 +369,14 @@ public static Row toBeamRowStrict(GenericRecord record, @Nullable Schema schema)
return builder.build();
}

/**
* Convert from a Beam Row to an AVRO GenericRecord. The Avro Schema is inferred from the Beam
* schema on the row.
*/
public static GenericRecord toGenericRecord(Row row) {
return toGenericRecord(row, null);
}

/**
* Convert from a Beam Row to an AVRO GenericRecord. If a Schema is not provided, one is inferred
* from the Beam schema on the row.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -721,6 +721,13 @@ public void testBeamRowToGenericRecord() {
assertEquals(getGenericRecord(), genericRecord);
}

@Test
public void testBeamRowToGenericRecordInferSchema() {
GenericRecord genericRecord = AvroUtils.toGenericRecord(getBeamRow());
assertEquals(getAvroSchema(), genericRecord.getSchema());
assertEquals(getGenericRecord(), genericRecord);
}

@Test
public void testRowToGenericRecordFunction() {
SerializableUtils.ensureSerializable(AvroUtils.getRowToGenericRecordFunction(NULL_SCHEMA));
Expand Down

0 comments on commit 67164a1

Please sign in to comment.