avroTypes = BIG_QUERY_TO_AVRO_TYPES.get(bigQueryField.getType());
+ if (avroTypes.isEmpty()) {
+ throw new IllegalArgumentException(
+ "Unable to map BigQuery field type " + bigQueryField.getType() + " to avro type.");
+ }
+
+ Type avroType = avroTypes.iterator().next();
Schema elementSchema;
if (avroType == Type.RECORD) {
elementSchema = toGenericAvroSchema(bigQueryField.getName(), bigQueryField.getFields());
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
index f17c3900e8daa..5987a29d36aec 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
@@ -276,9 +276,20 @@
* To write to a BigQuery table, apply a {@link BigQueryIO.Write} transformation. This consumes a
* {@link PCollection} of a user-defined type when using {@link BigQueryIO#write()} (recommended),
* or a {@link PCollection} of {@link TableRow TableRows} as input when using {@link
- * BigQueryIO#writeTableRows()} (not recommended). When using a user-defined type, a function must
- * be provided to turn this type into a {@link TableRow} using {@link
- * BigQueryIO.Write#withFormatFunction(SerializableFunction)}.
+ * BigQueryIO#writeTableRows()} (not recommended). When using a user-defined type, one of the
+ * following must be provided.
+ *
+ *
+ * - {@link BigQueryIO.Write#withAvroFormatFunction(SerializableFunction)} (recommended) to
+ * write data using avro records.
+ *
- {@link BigQueryIO.Write#withFormatFunction(SerializableFunction)} to write data as json
+ * encoded {@link TableRow TableRows}.
+ *
+ *
+ * If {@link BigQueryIO.Write#withAvroFormatFunction(SerializableFunction)} is used, the table
+ * schema MUST be specified using one of the {@link Write#withJsonSchema(String)}, {@link
+ * Write#withJsonSchema(ValueProvider)}, {@link Write#withSchemaFromView(PCollectionView)} methods,
+ * or {@link Write#to(DynamicDestinations)}.
*
* {@code
* class Quote {
@@ -465,6 +476,15 @@ public class BigQueryIO {
*/
static final SerializableFunction IDENTITY_FORMATTER = input -> input;
+ private static final SerializableFunction
+ DEFAULT_AVRO_SCHEMA_FACTORY =
+ new SerializableFunction() {
+ @Override
+ public org.apache.avro.Schema apply(TableSchema input) {
+ return BigQueryAvroUtils.toGenericAvroSchema("root", input.getFields());
+ }
+ };
+
/**
* @deprecated Use {@link #read(SerializableFunction)} or {@link #readTableRows} instead. {@link
* #readTableRows()} does exactly the same as {@link #read}, however {@link
@@ -1662,6 +1682,12 @@ public enum Method {
@Nullable
abstract SerializableFunction getFormatFunction();
+ @Nullable
+ abstract SerializableFunction, GenericRecord> getAvroFormatFunction();
+
+ @Nullable
+ abstract SerializableFunction getAvroSchemaFactory();
+
@Nullable
abstract DynamicDestinations getDynamicDestinations();
@@ -1738,6 +1764,12 @@ abstract Builder setTableFunction(
abstract Builder setFormatFunction(SerializableFunction formatFunction);
+ abstract Builder setAvroFormatFunction(
+ SerializableFunction, GenericRecord> avroFormatFunction);
+
+ abstract Builder setAvroSchemaFactory(
+ SerializableFunction avroSchemaFactory);
+
abstract Builder setDynamicDestinations(DynamicDestinations dynamicDestinations);
abstract Builder setSchemaFromView(PCollectionView