diff --git a/flink-connector-bigquery-common/src/main/java/com/google/cloud/flink/bigquery/common/utils/AvroToBigQuerySchemaTransform.java b/flink-connector-bigquery-common/src/main/java/com/google/cloud/flink/bigquery/common/utils/AvroToBigQuerySchemaTransform.java new file mode 100644 index 00000000..6f1e4283 --- /dev/null +++ b/flink-connector-bigquery-common/src/main/java/com/google/cloud/flink/bigquery/common/utils/AvroToBigQuerySchemaTransform.java @@ -0,0 +1,509 @@ +/* + * Copyright (C) 2024 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.common.utils; + +import org.apache.flink.annotation.VisibleForTesting; + +import com.google.cloud.bigquery.Field; +import com.google.cloud.bigquery.FieldList; +import com.google.cloud.bigquery.LegacySQLTypeName; +import com.google.cloud.bigquery.StandardSQLTypeName; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; + +import java.util.ArrayList; +import java.util.EnumMap; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * Utility class for transforming Avro {@link Schema} to BigQuery {@link + * com.google.cloud.bigquery.Schema}. + */ +public class AvroToBigQuerySchemaTransform { + + /** + * Maximum nesting level for BigQuery schemas (15 levels). See BigQuery nested and + * repeated field limitations. + */ + private static final int MAX_NESTED_LEVEL = 15; + + private static final String GEOGRAPHY_LOGICAL_TYPE_NAME = "geography_wkt"; + private static final String JSON_LOGICAL_TYPE_NAME = "Json"; + private static final int MAX_BQ_NUMERIC_PRECISION = 38; + private static final int MAX_BQ_NUMERIC_SCALE = 9; + private static final int MAX_BQ_BIGNUMERIC_PRECISION = 76; + private static final int MAX_BQ_BIGNUMERIC_SCALE = 38; + private static final Map PRIMITIVE_AVRO_TYPES_TO_BQ_TYPES; + private static final Map LOGICAL_AVRO_TYPES_TO_BQ_TYPES; + + // ----------- Initialize Maps between Avro Schema to BigQuery schema ------------- + static { + /* + * Map Avro Schema Type to StandardSQLTypeName which converts AvroSchema + * Primitive Type to StandardSQLTypeName. + * PRIMITIVE_AVRO_TYPES_TO_BQ_TYPES: containing mapping from Primitive Avro Schema Type + * to StandardSQLTypeName. + */ + PRIMITIVE_AVRO_TYPES_TO_BQ_TYPES = new EnumMap<>(Schema.Type.class); + PRIMITIVE_AVRO_TYPES_TO_BQ_TYPES.put(Schema.Type.INT, StandardSQLTypeName.INT64); + PRIMITIVE_AVRO_TYPES_TO_BQ_TYPES.put(Schema.Type.FIXED, StandardSQLTypeName.BYTES); + PRIMITIVE_AVRO_TYPES_TO_BQ_TYPES.put(Schema.Type.LONG, StandardSQLTypeName.INT64); + PRIMITIVE_AVRO_TYPES_TO_BQ_TYPES.put(Schema.Type.FLOAT, StandardSQLTypeName.FLOAT64); + PRIMITIVE_AVRO_TYPES_TO_BQ_TYPES.put(Schema.Type.DOUBLE, StandardSQLTypeName.FLOAT64); + PRIMITIVE_AVRO_TYPES_TO_BQ_TYPES.put(Schema.Type.STRING, StandardSQLTypeName.STRING); + PRIMITIVE_AVRO_TYPES_TO_BQ_TYPES.put(Schema.Type.BOOLEAN, StandardSQLTypeName.BOOL); + PRIMITIVE_AVRO_TYPES_TO_BQ_TYPES.put(Schema.Type.ENUM, StandardSQLTypeName.STRING); + PRIMITIVE_AVRO_TYPES_TO_BQ_TYPES.put(Schema.Type.BYTES, StandardSQLTypeName.BYTES); + + /* + * Map Logical Avro Schema Type to StandardSQLTypeName Type, which converts + * AvroSchema Logical Type to StandardSQLTypeName. + * LOGICAL_AVRO_TYPES_TO_BQ_TYPES: Map containing mapping from Logical Avro Schema Type to + * StandardSQLTypeName. + */ + LOGICAL_AVRO_TYPES_TO_BQ_TYPES = new HashMap<>(); + LOGICAL_AVRO_TYPES_TO_BQ_TYPES.put(LogicalTypes.date().getName(), StandardSQLTypeName.DATE); + LOGICAL_AVRO_TYPES_TO_BQ_TYPES.put( + LogicalTypes.timestampMicros().getName(), StandardSQLTypeName.TIMESTAMP); + LOGICAL_AVRO_TYPES_TO_BQ_TYPES.put( + LogicalTypes.timestampMillis().getName(), StandardSQLTypeName.TIMESTAMP); + LOGICAL_AVRO_TYPES_TO_BQ_TYPES.put( + LogicalTypes.uuid().getName(), StandardSQLTypeName.STRING); + LOGICAL_AVRO_TYPES_TO_BQ_TYPES.put( + LogicalTypes.timeMillis().getName(), StandardSQLTypeName.TIME); + LOGICAL_AVRO_TYPES_TO_BQ_TYPES.put( + LogicalTypes.timeMicros().getName(), StandardSQLTypeName.TIME); + LOGICAL_AVRO_TYPES_TO_BQ_TYPES.put( + LogicalTypes.localTimestampMillis().getName(), StandardSQLTypeName.DATETIME); + LOGICAL_AVRO_TYPES_TO_BQ_TYPES.put( + LogicalTypes.localTimestampMicros().getName(), StandardSQLTypeName.DATETIME); + LOGICAL_AVRO_TYPES_TO_BQ_TYPES.put( + GEOGRAPHY_LOGICAL_TYPE_NAME, StandardSQLTypeName.GEOGRAPHY); + LOGICAL_AVRO_TYPES_TO_BQ_TYPES.put(JSON_LOGICAL_TYPE_NAME, StandardSQLTypeName.JSON); + } + + // Private Constructor to ensure no instantiation. + private AvroToBigQuerySchemaTransform() {} + + /** + * Converts an Avro schema to a BigQuery schema. + * + *

This method transforms an Avro {@link Schema} into a BigQuery {@link + * com.google.cloud.bigquery.Schema}. It iterates through the fields of the Avro schema, + * converts each field to its BigQuery equivalent, and constructs a BigQuery schema with the + * resulting fields. + * + *

For each Avro field, the method extracts the field's name and documentation (if + * available), and uses this information to build the corresponding BigQuery field. + * + *

The Avro schema must be of type Avro RECORD + * + * @param avroSchema The Avro schema to convert. + * @return The converted BigQuery {@link com.google.cloud.bigquery.Schema}. + */ + public static com.google.cloud.bigquery.Schema getBigQuerySchema(Schema avroSchema) { + if (avroSchema == null) { + throw new IllegalArgumentException( + "Could not convert avro schema of expected output to BigQuery table schema. " + + "The avro schema of the expected output record cannot be null"); + } + if (avroSchema.getType() == null) { + throw new IllegalArgumentException( + "Could not convert avro schema of expected output to BigQuery table schema. " + + "The avro schema of the output record must have a type"); + } + if (avroSchema.getType() != Schema.Type.RECORD) { + throw new IllegalArgumentException( + String.format( + "Could not convert avro schema of expected output to BigQuery table " + + "schema. Avro schema must be of type RECORD, found %s", + avroSchema.getType())); + } + if (avroSchema.getFields().isEmpty()) { + throw new IllegalArgumentException( + "Could not convert avro schema of expected output to BigQuery table schema. " + + "The avro schema of the output record must have at least one field."); + } + // Iterate over each record field and add them to the BigQuery schema. + List fields = + avroSchema.getFields().stream() + .map( + avroField -> { + if (avroField.name() == null) { + throw new IllegalArgumentException( + String.format( + "Could not convert avro field: %s of " + + "expected output to BigQuery " + + "table schema. The output avro " + + "field must have a name " + + "attribute to convert to a " + + "BigQuery table schema.", + avroField.toString())); + } + Field.Builder bigQueryFieldBuilder = + convertAvroFieldToBigQueryField( + avroField.schema(), avroField.name(), 0) + .toBuilder(); + bigQueryFieldBuilder.setName(avroField.name()); + if (avroField.doc() != null) { + bigQueryFieldBuilder.setDescription(avroField.doc()); + } + return bigQueryFieldBuilder.build(); + }) + .collect(Collectors.toList()); + return com.google.cloud.bigquery.Schema.of(fields); + } + + /** + * Converts an Avro field to a BigQuery field. + * + * @param avroField the Avro field to convert + * @param name the name of the field + * @param nestedLevel the current nesting level of the field + */ + private static Field convertAvroFieldToBigQueryField( + Schema avroField, String name, int nestedLevel) { + if (nestedLevel > MAX_NESTED_LEVEL) { + throw new IllegalArgumentException( + String.format( + "Could not convert avro field: %s of expected output to BigQuery table " + + "schema. Field nesting in expected output's schema exceeds " + + "15. BigQuery allows nesting up to a depth of 15.", + name)); + } + if (avroField == null) { + throw new IllegalArgumentException( + String.format( + "Could not convert avro field: %s of expected output to BigQuery table " + + "schema. The avro schema of the expected output record cannot" + + " be null.", + name)); + } + if (avroField.getType() == null) { + throw new IllegalArgumentException( + String.format( + "Could not convert avro field: %s of expected output to BigQuery table " + + "schema. The avro schema of the output record must have a " + + "type", + name)); + } + switch (avroField.getType()) { + case RECORD: + return convertAvroRecordFieldToBigQueryField(avroField, name, nestedLevel); + case ARRAY: + return convertAvroRepeatedFieldToBigQueryField(avroField, name, nestedLevel); + case UNION: + return convertAvroUnionFieldToBigQueryField(avroField, name, nestedLevel); + case MAP: + throw new UnsupportedOperationException( + getUnsupportedTypeErrorMessage(avroField.getType().toString(), name)); + default: + return convertAvroRequiredFieldToBigQueryField(avroField, name); + } + } + // --------------- Helper Functions to convert AvroSchema to BigQuerySchema --------------- + + /** + * Converts an Avro record field to a BigQuery record field. This method iterates through each + * field in the Avro record schema, recursively converts them to BigQuery fields, adding them as + * sub-fields, and then constructs a BigQuery record field with the converted fields. + * + * @param avroSchema the Avro schema of the record field + * @param name the name of the field + * @param nestedLevel the current nesting level of the field + * @return the converted BigQuery record field with nested fields. + */ + private static Field convertAvroRecordFieldToBigQueryField( + Schema avroSchema, String name, int nestedLevel) { + List fields = new ArrayList<>(); + if (avroSchema.getFields().isEmpty()) { + throw new IllegalArgumentException( + String.format( + "Could not convert avro field: %s of expected output to BigQuery table " + + "schema. The avro schema of the output record must have at " + + "least one field.", + name)); + } + // Iterate over each record field and obtain the nested record fields. + for (Schema.Field field : avroSchema.getFields()) { + Schema nestedAvroSchema = field.schema(); + fields.add( + convertAvroFieldToBigQueryField( + nestedAvroSchema, field.name(), nestedLevel + 1)); + } + FieldList nestedBigQueryFields = FieldList.of(fields); + return Field.newBuilder(name, LegacySQLTypeName.RECORD, nestedBigQueryFields) + .setDescription((avroSchema.getDoc() != null) ? avroSchema.getDoc() : null) + .setMode(Field.Mode.REQUIRED) + .build(); + } + + /** + * Helper function to convert the UNION type schema field to a BigQuery Field. + * + *

BigQuery supports nullable fields but not unions with multiple non-null types. This method + * validates the Avro UNION schema to ensure it conforms to BigQuery's requirements. + * + *

Valid UNION schemas are: + * + *

+ * + *

Invalid UNION schemas include: + * + *

+ * + *

If the UNION schema is valid, this method returns a BigQuery field with the schema of the + * non-null datatype. Otherwise, it throws an {@link IllegalArgumentException}. + * + * @param avroSchema The Avro UNION schema to process. + * @return The BigQuery field corresponding to the non-null type in the UNION. + * @throws IllegalArgumentException if the UNION schema is invalid for BigQuery. + */ + private static Field convertAvroUnionFieldToBigQueryField( + Schema avroSchema, String name, int nestedLevel) { + List unionTypes = avroSchema.getTypes(); + + // Case, when there is only a single type in UNION. + // Can be ['null'] - ERROR + // [Valid-Datatype] - Not Nullable, element type + // Then it is essentially the same as not having a UNION. + if (unionTypes.size() == 1 && unionTypes.get(0).getType() != Schema.Type.NULL) { + return convertAvroFieldToBigQueryField(unionTypes.get(0), name, nestedLevel); + } + + // Extract all the nonNull Datatypes. + // ['datatype, 'null'] and ['null', datatype] are only valid cases. + if (unionTypes.size() == 2 + && ((unionTypes.get(0).getType() == Schema.Type.NULL + && unionTypes.get(1).getType() != Schema.Type.ARRAY) + || (unionTypes.get(1).getType() == Schema.Type.NULL + && unionTypes.get(0).getType() != Schema.Type.ARRAY))) { + Schema actualDataType = + unionTypes.get(0).getType() == Schema.Type.NULL + ? unionTypes.get(1) + : unionTypes.get(0); + return convertAvroNullableFieldToBigQueryField(actualDataType, name, nestedLevel); + } + + /* UNION of type ARRAY is not supported. + ARRAY is mapped to REPEATED type in Bigquery, which cannot be NULLABLE. + If we have the datatype is ["null", "ARRAY"] or ["ARRAY", "null], + IllegalArgumentException is thrown. + For any other unsupported union types, exception is thrown. */ + throw new IllegalArgumentException( + String.format( + "Could not convert schema of avro field: %s to BigQuery table schema. " + + "Found unsupported Avro Field of type UNION. Only supported " + + "types " + + "for BigQuery table schema are non-array types of " + + "\"['datatype'], ['null', 'datatype'] or ['datatype', 'null']\"", + name)); + } + + private static Field convertAvroNullableFieldToBigQueryField( + Schema avroSchema, String name, int nestedLevel) { + return convertAvroFieldToBigQueryField(avroSchema, name, nestedLevel) + .toBuilder() + .setMode(Field.Mode.NULLABLE) + .build(); + } + + /** + * Converts an Avro ARRAY field to a BigQuery REPEATED field. + * + *

The following restrictions imposed by BigQuery's schema definition: + * + *

+ * + *

If any of these restrictions are violated, an {@link IllegalArgumentException} is thrown. + * + * @param avroSchema The Avro schema of the repeated field. + * @param name The name of the field. + * @param nestedLevel The nesting level of the field. + * @return The converted BigQuery {@link Field} with mode set to {@link Field.Mode#REPEATED}. + * @throws IllegalArgumentException if the Avro schema violates any of the restrictions for + * BigQuery repeated fields. + */ + private static Field convertAvroRepeatedFieldToBigQueryField( + Schema avroSchema, String name, int nestedLevel) { + Schema arrayAvroSchema = avroSchema.getElementType(); + if (arrayAvroSchema.getType() == Schema.Type.ARRAY) { + throw new IllegalArgumentException( + String.format( + "Could not convert schema of avro field: %s to BigQuery table schema. " + + "BigQuery ARRAY cannot have recursive ARRAY fields.", + name)); + } + Field innerArrayField = convertAvroFieldToBigQueryField(arrayAvroSchema, name, nestedLevel); + if (innerArrayField.getMode() != Field.Mode.REQUIRED) { + throw new IllegalArgumentException( + String.format( + "Could not convert schema of avro field: %s to BigQuery table schema. " + + "Array cannot have a NULLABLE element in BigQuery", + name)); + } + return innerArrayField.toBuilder().setMode(Field.Mode.REPEATED).build(); + } + + /** + * Helper function convert Avro Field to BigQuery Field for Primitive and Logical Datatypes. + * + *

LOGICAL: Use elementType.getProp() to obtain the string for the property name and + * search for its corresponding mapping in the LOGICAL_AVRO_TYPES_TO_BQ_TYPES map. + * + *

PRIMITIVE: If there is no match for the logical type (or there is no logical type + * present), the field data type is attempted to be mapped to a PRIMITIVE type map. + * + * @param avroSchema the Avro schema of the required field + * @param name the name of the field + * @return the converted BigQuery field with the appropriate data type and mode set to REQUIRED + * @throws UnsupportedOperationException if the Avro type is not supported + * @throws IllegalArgumentException if the Avro schema is invalid for a decimal logical type + */ + private static Field convertAvroRequiredFieldToBigQueryField(Schema avroSchema, String name) + throws UnsupportedOperationException, IllegalArgumentException { + StandardSQLTypeName dataType; + + // Handle decimal logical types by extracting precision and setting the appropriate + // StandardSQLTypeName. + // The conversions are according to + // https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types#decimal_types + if ("decimal".equals(avroSchema.getProp(LogicalType.LOGICAL_TYPE_PROP))) { + dataType = handleDecimalLogicalType(avroSchema, name); + if (dataType == StandardSQLTypeName.NUMERIC + || dataType == StandardSQLTypeName.BIGNUMERIC) { + // The precision and scale is also set in the BigQuery Field + long precision = + ((LogicalTypes.Decimal) avroSchema.getLogicalType()).getPrecision(); + long scale = ((LogicalTypes.Decimal) avroSchema.getLogicalType()).getScale(); + return Field.newBuilder(name, dataType) + .setMode(Field.Mode.REQUIRED) + .setPrecision(precision) + .setScale(scale) + .build(); + } + } else { + dataType = + Optional.ofNullable(avroSchema.getProp(LogicalType.LOGICAL_TYPE_PROP)) + .map(LOGICAL_AVRO_TYPES_TO_BQ_TYPES::get) + .orElse(PRIMITIVE_AVRO_TYPES_TO_BQ_TYPES.get(avroSchema.getType())); + } + if (dataType == null) { + throw new UnsupportedOperationException( + getUnsupportedTypeErrorMessage(avroSchema.getType().toString(), name)); + } + + return Field.newBuilder(name, dataType).setMode(Field.Mode.REQUIRED).build(); + } + + /** + * Helper method to handle Avro decimal logical types by determining the appropriate BigQuery + * data type based on the precision of the decimal. + * + * @param avroSchema the Avro schema of the decimal field + * @param name the name of the field + */ + private static StandardSQLTypeName handleDecimalLogicalType(Schema avroSchema, String name) { + validatePrecisionAndScale(avroSchema, name); + long precision = ((LogicalTypes.Decimal) avroSchema.getLogicalType()).getPrecision(); + long scale = ((LogicalTypes.Decimal) avroSchema.getLogicalType()).getScale(); + + if (precision > 0 + && precision <= (scale + (MAX_BQ_NUMERIC_PRECISION - MAX_BQ_NUMERIC_SCALE)) + && scale <= MAX_BQ_NUMERIC_SCALE) { + return StandardSQLTypeName.NUMERIC; + } else if (precision > 0 + && precision <= (scale + (MAX_BQ_BIGNUMERIC_PRECISION - MAX_BQ_BIGNUMERIC_SCALE)) + && scale <= MAX_BQ_BIGNUMERIC_SCALE) { + return StandardSQLTypeName.BIGNUMERIC; + } + throw new IllegalArgumentException( + String.format( + "Could not convert schema of avro field: %s to BigQuery table schema. " + + "BigQuery does not support Decimal types with " + + "precision %d and scale %d.", + name, precision, scale)); + } + + /** + * Validates the precision and scale of an Avro decimal logical type. Ensures precision and + * scale are non-negative and that scale does not exceed precision. + * + * @param schema the Avro schema of the decimal field + * @param name the name of the field + * @throws IllegalArgumentException if precision or scale are invalid + */ + private static void validatePrecisionAndScale(Schema schema, String name) { + LogicalTypes.Decimal decimalLogicalSchema = + ((LogicalTypes.Decimal) schema.getLogicalType()); + // The Avro schema of logical type "decimal" has "precision" and "scale" attributes + // inherently defined so null checks for these attributes are not required. + long precision = decimalLogicalSchema.getPrecision(); + long scale = decimalLogicalSchema.getScale(); + if (precision <= 0) { + throw new IllegalArgumentException( + String.format( + "Could not convert avro field: %s of expected output to BigQuery table " + + "schema. Precision of decimal avro field must be " + + "non-negative. Saw: %d", + name, precision)); + } + if (scale < 0) { + throw new IllegalArgumentException( + String.format( + "Could not convert avro field: %s of expected output to BigQuery table " + + "schema. Scale of decimal avro field must be " + + "non-negative. Saw: %d", + name, scale)); + } + if (precision < scale) { + throw new IllegalArgumentException( + String.format( + "Could not convert avro field: %s of expected output to BigQuery table " + + "schema. Scale of the decimal avro field cannot exceed " + + "precision. Saw scale: %d, precision: %d", + name, scale, precision)); + } + } + + @VisibleForTesting + public static String getUnsupportedTypeErrorMessage( + String unsupportedAvroType, String fieldName) { + return String.format( + "Could not convert avro field: %s of expected output to BigQuery table schema. " + + "The avro type: %s of the field is not supported by BigQuery", + fieldName, unsupportedAvroType); + } +} diff --git a/flink-connector-bigquery-common/src/test/java/com/google/cloud/flink/bigquery/common/utils/AvroToBigQuerySchemaTransformTest.java b/flink-connector-bigquery-common/src/test/java/com/google/cloud/flink/bigquery/common/utils/AvroToBigQuerySchemaTransformTest.java new file mode 100644 index 00000000..20c35537 --- /dev/null +++ b/flink-connector-bigquery-common/src/test/java/com/google/cloud/flink/bigquery/common/utils/AvroToBigQuerySchemaTransformTest.java @@ -0,0 +1,620 @@ +/* + * Copyright (C) 2024 Google Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); you may not + * use this file except in compliance with the License. You may obtain a copy of + * the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package com.google.cloud.flink.bigquery.common.utils; + +import com.google.cloud.bigquery.Field; +import com.google.cloud.bigquery.FieldList; +import com.google.cloud.bigquery.LegacySQLTypeName; +import com.google.cloud.bigquery.StandardSQLTypeName; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.Schema.Parser; +import org.apache.avro.Schema.Type; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static com.google.cloud.flink.bigquery.common.utils.AvroToBigQuerySchemaTransform.getUnsupportedTypeErrorMessage; +import static com.google.common.truth.Truth.assertThat; +import static org.junit.Assert.assertThrows; + +/** Unit tests for {@link AvroToBigQuerySchemaTransform}. */ +public class AvroToBigQuerySchemaTransformTest { + + private static final String GEOGRAPHY_LOGICAL_TYPE_NAME = "geography_wkt"; + private static final String JSON_LOGICAL_TYPE_NAME = "Json"; + + private static Field createRequiredBigqueryField(String name, StandardSQLTypeName type) { + return Field.newBuilder(name, type).setMode(Field.Mode.REQUIRED).build(); + } + + private static Field createNullableBigqueryField(String name, StandardSQLTypeName type) { + return Field.newBuilder(name, type).setMode(Field.Mode.NULLABLE).build(); + } + + private static Field createRecordField(String name, Field... subFields) { + return Field.newBuilder(name, LegacySQLTypeName.RECORD, FieldList.of(subFields)) + .setMode(Field.Mode.REQUIRED) + .build(); + } + + private static Field createRepeatedBigqueryField(String name, StandardSQLTypeName type) { + return Field.newBuilder(name, type).setMode(Field.Mode.REPEATED).build(); + } + + /** Tests Avro Schema with all Primitive Data Types. */ + @Test + public void testAllTypesSchemaSuccessful() { + String avroSchemaString = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"allTypes\",\n" + + " \"fields\": [\n" + + " {\"name\": \"string_field\", \"type\": \"string\"},\n" + + " {\"name\": \"bytes_field\", \"type\": \"bytes\"},\n" + + " {\"name\": \"integer_field\", \"type\": \"long\"},\n" + + " {\"name\": \"array_field\", \"type\": {\"type\": \"array\", " + + "\"items\": \"double\"}},\n" + + " {\"name\": \"numeric_field\", \"type\": [\"null\", {\"type\": " + + "\"bytes\", \"logicalType\": \"decimal\", \"precision\": 38, \"scale\":" + + " 9}]},\n" + + " {\"name\": \"bignumeric_field\", \"type\": [\"null\", {\"type\": " + + "\"bytes\", \"logicalType\": \"decimal\", \"precision\": 76, \"scale\":" + + " 38}]},\n" + + " {\"name\": \"boolean_field\", \"type\": [\"null\", \"boolean\"]},\n" + + " {\"name\": \"ts_field\", \"type\": [\"null\", {\"type\": \"long\"," + + " \"logicalType\": \"timestamp-micros\"}]},\n" + + " {\"name\": \"date_field\", \"type\": [\"null\", {\"type\": " + + "\"int\", \"logicalType\": \"date\"}]},\n" + + " {\"name\": \"time_field\", \"type\": [\"null\", {\"type\": " + + "\"long\", \"logicalType\": \"time-micros\"}]},\n" + + " {\"name\": \"datetime_field\", \"type\": [\"null\", {\"type\": " + + "\"long\", \"logicalType\": \"local-timestamp-micros\"}]},\n" + + " {\"name\": \"geography_field\", \"type\": [\"null\", {\"type\": " + + "\"string\", \"logicalType\": \"geography_wkt\"}]},\n" + + " {\"name\": \"record_field\", \"type\": {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"record_field\",\n" + + " \"fields\": [\n" + + " {\"name\": \"json_field\", \"type\": [\"null\", {\"type\": " + + "\"string\", \"logicalType\": \"Json\"}]},\n" + + " {\"name\": \"geography_field\", \"type\": [\"null\", " + + "{\"type\": \"string\", \"logicalType\": \"geography_wkt\"}]}\n" + + " ]\n" + + " }}\n" + + " ]\n" + + "}"; + Schema allTypesSchema = new Parser().parse(avroSchemaString); + + com.google.cloud.bigquery.Schema expectedBqSchema = + com.google.cloud.bigquery.Schema.of( + createRequiredBigqueryField("string_field", StandardSQLTypeName.STRING), + createRequiredBigqueryField("bytes_field", StandardSQLTypeName.BYTES), + createRequiredBigqueryField("integer_field", StandardSQLTypeName.INT64), + createRepeatedBigqueryField("array_field", StandardSQLTypeName.FLOAT64), + createNullableBigqueryField("numeric_field", StandardSQLTypeName.NUMERIC) + .toBuilder() + .setPrecision(38L) + .setScale(9L) + .build(), + createNullableBigqueryField( + "bignumeric_field", StandardSQLTypeName.BIGNUMERIC) + .toBuilder() + .setPrecision(76L) + .setScale(38L) + .build(), + createNullableBigqueryField("boolean_field", StandardSQLTypeName.BOOL), + createNullableBigqueryField("ts_field", StandardSQLTypeName.TIMESTAMP), + createNullableBigqueryField("date_field", StandardSQLTypeName.DATE), + createNullableBigqueryField("time_field", StandardSQLTypeName.TIME), + createNullableBigqueryField("datetime_field", StandardSQLTypeName.DATETIME), + createNullableBigqueryField( + "geography_field", StandardSQLTypeName.GEOGRAPHY), + Field.newBuilder( + "record_field", + LegacySQLTypeName.RECORD, + FieldList.of( + createNullableBigqueryField( + "json_field", StandardSQLTypeName.JSON), + createNullableBigqueryField( + "geography_field", + StandardSQLTypeName.GEOGRAPHY))) + .setMode(Field.Mode.REQUIRED) + .build()); + + com.google.cloud.bigquery.Schema bqSchema = + AvroToBigQuerySchemaTransform.getBigQuerySchema(allTypesSchema); + assertExactSchema(bqSchema, expectedBqSchema); + } + + /** Tests Avro Schema with all Logical Data Types. */ + @Test + public void testLogicalTypesSuccessful() { + // Create an Avro schema with logical types + Schema avroSchema = Schema.createRecord("RecordWithLogicalTypes", "", "", false); + ArrayList logicalFields = new ArrayList<>(); + logicalFields.add( + new Schema.Field( + "dateField", + LogicalTypes.date().addToSchema(Schema.create(Type.INT)), + null, + null)); + logicalFields.add( + new Schema.Field( + "timeMillisField", + LogicalTypes.timeMillis().addToSchema(Schema.create(Type.INT)), + null, + null)); + logicalFields.add( + new Schema.Field( + "timestampMillisField", + LogicalTypes.timestampMillis().addToSchema(Schema.create(Type.LONG)), + null, + null)); + logicalFields.add( + new Schema.Field( + "localTimestampMillisField", + LogicalTypes.localTimestampMillis().addToSchema(Schema.create(Type.LONG)), + null, + null)); + logicalFields.add( + new Schema.Field( + "decimalField", + LogicalTypes.decimal(10, 2).addToSchema(Schema.create(Type.BYTES)), + "Decimal Field Description", + null)); + logicalFields.add( + new Schema.Field( + "uuidField", + LogicalTypes.uuid().addToSchema(Schema.create(Type.STRING)), + null, + null)); + Schema durationSchema = Schema.create(Type.BYTES); + durationSchema.addProp(LogicalType.LOGICAL_TYPE_PROP, "duration"); + logicalFields.add(new Schema.Field("durationField", durationSchema, null, null)); + Schema geoSchema = Schema.create(Schema.Type.STRING); + geoSchema.addProp(LogicalType.LOGICAL_TYPE_PROP, GEOGRAPHY_LOGICAL_TYPE_NAME); + logicalFields.add(new Schema.Field("geographyWKTField", geoSchema, null, null)); + Schema jsonSchema = Schema.create(Schema.Type.STRING); + jsonSchema.addProp(LogicalType.LOGICAL_TYPE_PROP, JSON_LOGICAL_TYPE_NAME); + logicalFields.add(new Schema.Field("jsonField", jsonSchema, null, null)); + avroSchema.setFields(logicalFields); + + com.google.cloud.bigquery.Schema expectedBqSchema = + com.google.cloud.bigquery.Schema.of( + createRequiredBigqueryField("dateField", StandardSQLTypeName.DATE), + createRequiredBigqueryField("timeMillisField", StandardSQLTypeName.TIME), + createRequiredBigqueryField( + "timestampMillisField", StandardSQLTypeName.TIMESTAMP), + createRequiredBigqueryField( + "localTimestampMillisField", StandardSQLTypeName.DATETIME), + createRequiredBigqueryField("decimalField", StandardSQLTypeName.NUMERIC) + .toBuilder() + .setPrecision(10L) + .setDescription("Decimal Field Description") + .build(), + createRequiredBigqueryField("uuidField", StandardSQLTypeName.STRING), + createRequiredBigqueryField("durationField", StandardSQLTypeName.BYTES), + createRequiredBigqueryField( + "geographyWKTField", StandardSQLTypeName.GEOGRAPHY), + createRequiredBigqueryField("jsonField", StandardSQLTypeName.JSON)); + + com.google.cloud.bigquery.Schema bqSchema = + AvroToBigQuerySchemaTransform.getBigQuerySchema(avroSchema); + assertExactSchema(bqSchema, expectedBqSchema); + } + + /** Tests that an exception is thrown for invalid decimal precision and scale. */ + @Test + public void testHandleDecimalLogicalTypeThrowsException() { + Schema avroSchema = Schema.createRecord("RecordWithDecimalType", "", "", false); + ArrayList decimalFields = new ArrayList<>(); + decimalFields.add( + new Schema.Field( + "decimalField", + LogicalTypes.decimal(78, 10).addToSchema(Schema.create(Type.BYTES)), + "Decimal Field Description", + null)); + avroSchema.setFields(decimalFields); + IllegalArgumentException exception = + assertThrows( + IllegalArgumentException.class, + () -> AvroToBigQuerySchemaTransform.getBigQuerySchema(avroSchema)); + + assertThat(exception.getMessage()) + .isEqualTo( + "Could not convert schema of avro field: decimalField to BigQuery table " + + "schema. BigQuery does not support Decimal types with precision" + + " 78 and scale 10."); + } + + /** Tests Avro record schema with nesting upto 15 levels. */ + @Test + public void testDeeplyNestedSchemaSuccessful() { + Schema currentSchema = null; + for (int i = 13; i >= 0; i--) { + Schema nextSchema = Schema.createRecord("level_" + i, null, null, false); + if (currentSchema != null) { + ArrayList nestedFields = new ArrayList<>(); + nestedFields.add((new Schema.Field("level_" + (i + 1), currentSchema, null, null))); + nextSchema.setFields(nestedFields); + } else { + ArrayList nestedFields = new ArrayList<>(); + nestedFields.add(new Schema.Field("value", Schema.create(Type.LONG), null, null)); + nextSchema.setFields(nestedFields); + } + currentSchema = nextSchema; + } + Schema level0Schema = currentSchema; + Schema nestedSchema = Schema.createRecord("nestedTypeIT", null, null, false); + ArrayList recordFields = new ArrayList<>(); + recordFields.add(new Schema.Field("level_0", level0Schema, null, null)); + nestedSchema.setFields(recordFields); + + Field currentField = createRequiredBigqueryField("value", StandardSQLTypeName.INT64); + for (int i = 13; i >= 0; i--) { + currentField = createRecordField("level_" + i, currentField); + } + Field level0Field = currentField; + + com.google.cloud.bigquery.Schema expectedBqSchema = + com.google.cloud.bigquery.Schema.of(level0Field); + + com.google.cloud.bigquery.Schema bqSchema = + AvroToBigQuerySchemaTransform.getBigQuerySchema(nestedSchema); + assertExactSchema(bqSchema, expectedBqSchema); + } + + /** + * Tests Avro Schema: {"type": "record", "name": "LongList", "fields" : [{"name": "value", + * "type": "long"}, {"name": "next", "type": ["null", "LongList"]}]}. + * + *

This should throw Exception as this is an infinite recursion and is not supported by + * BigQuery + */ + @Test + public void testInfiniteRecursiveSchemaThrowsException() { + // Build the Avro schema programmatically + Schema longListSchema = Schema.createRecord("LongList", "", "", false); + longListSchema.addAlias("LinkedLongs"); + + ArrayList fields = new ArrayList<>(); + fields.add(new Schema.Field("value", Schema.create(Type.LONG), "", null)); + + Schema nullableLongListSchema = + Schema.createUnion(Schema.create(Type.NULL), longListSchema); + fields.add(new Schema.Field("next", nullableLongListSchema, "", null)); + longListSchema.setFields(fields); + + assertThrows( + IllegalArgumentException.class, + () -> AvroToBigQuerySchemaTransform.getBigQuerySchema(longListSchema)); + } + + /** + * Tests Avro record schema with more than 15 levels of nesting. This should throw an Exception. + */ + @Test + public void testNestedRecordExceedsLimitThrowsException() { + Schema nestedSchema = Schema.createRecord("NestedRecord", "", "", false); + List fields = new ArrayList<>(); + Schema currentSchema = nestedSchema; + for (int i = 0; i < 16; i++) { + Schema nextSchema = Schema.createRecord("NestedRecord" + i, "", "", false); + fields.add(new Schema.Field("nestedField", nextSchema, "", null)); + currentSchema.setFields(fields); + currentSchema = nextSchema; + fields = new ArrayList<>(); + } + + Schema nestedRecord = Schema.createRecord("NestedRecord", "", "", false); + ArrayList nestedFields = new ArrayList<>(); + nestedFields.add(new Schema.Field("nestedField", nestedRecord, "", null)); + nestedRecord.setFields(nestedFields); + + assertThrows( + IllegalArgumentException.class, + () -> AvroToBigQuerySchemaTransform.getBigQuerySchema(nestedRecord)); + } + + /** + * Tests Avro schema: { "type": "record", "name": "ArrayRecord", "fields": [ { "name": + * "arrayField", "type": { "type": "array", "items": { "type": "array", "items": "int" } } } ] + * }. It should throw an exception since recursive Arrays are not supported by BigQuery + */ + @Test + public void testArrayOfArraysThrowsException() { + String avroSchemaString = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"ArrayRecord\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"arrayField\",\n" + + " \"type\": {\n" + + " \"type\": \"array\",\n" + + " \"items\": {\n" + + " \"type\": \"array\",\n" + + " \"items\": \"int\"\n" + + " }\n" + + " }\n" + + " }\n" + + " ]\n" + + "}"; + Schema arrayRecord = new Parser().parse(avroSchemaString); + + assertThrows( + IllegalArgumentException.class, + () -> AvroToBigQuerySchemaTransform.getBigQuerySchema(arrayRecord)); + } + + /** Tests that an exception is thrown because BigQuery doesn't support nullable array types. */ + @Test + public void testNullableArrayThrowsException() { + String avroSchemaString = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"RecordWithNullableArray\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"nullableArray\",\n" + + " \"type\": [\"null\", {\"type\": \"array\", \"items\": \"int\"}]\n" + + " }\n" + + " ]\n" + + "}"; + Schema recordSchema = new Parser().parse(avroSchemaString); + + assertThrows( + IllegalArgumentException.class, + () -> AvroToBigQuerySchemaTransform.getBigQuerySchema(recordSchema)); + } + + /** Tests that an Avro array with a nullable inner type throws an exception. */ + @Test + public void testArrayWithNullableInnerTypeThrowsException() { + String avroSchemaString = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"ArrayRecord\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"arrayField\",\n" + + " \"type\": {\n" + + " \"type\": \"array\",\n" + + " \"items\": [\"null\", \"int\"]\n" + + " }\n" + + " }\n" + + " ]\n" + + "}"; + Schema arrayRecord = new Parser().parse(avroSchemaString); + + assertThrows( + IllegalArgumentException.class, + () -> AvroToBigQuerySchemaTransform.getBigQuerySchema(arrayRecord)); + } + + /** Tests that an Avro array with multiple datatypes throws an exception. */ + @Test + public void testArrayWithMultipleDatatypesThrowsException() { + String avroSchemaString = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"ArrayRecord\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"arrayMultipleDataTypesField\",\n" + + " \"type\": {\n" + + " \"type\": \"array\",\n" + + " \"items\": [\"int\", \"string\", \"float\"]\n" + + " }\n" + + " }\n" + + " ]\n" + + "}"; + Schema arrayRecord = new Parser().parse(avroSchemaString); + + assertThrows( + IllegalArgumentException.class, + () -> AvroToBigQuerySchemaTransform.getBigQuerySchema(arrayRecord)); + } + + /** Tests that an Avro array of records is correctly converted to a BigQuery schema. */ + @Test + public void testArrayOfRecordsDatatypeSuccessful() { + String avroSchemaString = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"OuterRecord\",\n" + + " \"fields\": [\n" + + " {\"name\": \"stringField\", \"type\": \"string\"},\n" + + " {\n" + + " \"name\": \"arrayField\",\n" + + " \"type\": {\n" + + " \"type\": \"array\",\n" + + " \"items\": {\n" + + " \"type\": \"record\",\n" + + " \"name\": \"InnerRecord\",\n" + + " \"fields\": [\n" + + " {\"name\": \"stringField\", \"type\": \"string\"},\n" + + " {\"name\": \"intField\", \"type\": \"int\"}\n" + + " ]\n" + + " }\n" + + " }\n" + + " },\n" + + " {\"name\": \"tsField\", \"type\": {\"type\": \"long\", " + + "\"logicalType\": \"timestamp-micros\"}}\n" + + " ]\n" + + "}"; + Schema arrayRecordSchema = new Parser().parse(avroSchemaString); + + ArrayList bqRecordFields = new ArrayList<>(); + bqRecordFields.add(createRequiredBigqueryField("stringField", StandardSQLTypeName.STRING)); + bqRecordFields.add(createRequiredBigqueryField("intField", StandardSQLTypeName.INT64)); + com.google.cloud.bigquery.Schema expectedBqSchema = + com.google.cloud.bigquery.Schema.of( + createRequiredBigqueryField("stringField", StandardSQLTypeName.STRING), + Field.newBuilder( + "arrayField", + LegacySQLTypeName.RECORD, + FieldList.of(bqRecordFields)) + .setMode(Field.Mode.REPEATED) + .build(), + createRequiredBigqueryField("tsField", StandardSQLTypeName.TIMESTAMP)); + + com.google.cloud.bigquery.Schema bqSchema = + AvroToBigQuerySchemaTransform.getBigQuerySchema(arrayRecordSchema); + assertExactSchema(bqSchema, expectedBqSchema); + } + + /** Tests that an Avro union field with only a null type throws an exception. */ + @Test + public void testUnionFieldWithOnlyNullTypeThrowsException() { + String avroSchemaString = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"Record\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"nullableField\",\n" + + " \"type\": [\"null\"]\n" + + " }\n" + + " ]\n" + + "}"; + Schema recordSchema = new Parser().parse(avroSchemaString); + + assertThrows( + IllegalArgumentException.class, + () -> AvroToBigQuerySchemaTransform.getBigQuerySchema(recordSchema)); + } + + /** Tests that an Avro union field with multiple non-null types throws an exception. */ + @Test + public void testUnionFieldWithMultipleNonNullTypesThrowsException() { + String avroSchemaString = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"Record\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"unionField\",\n" + + " \"type\": [\"int\", \"string\"]\n" + + " }\n" + + " ]\n" + + "}"; + Schema recordSchema = new Parser().parse(avroSchemaString); + + assertThrows( + IllegalArgumentException.class, + () -> AvroToBigQuerySchemaTransform.getBigQuerySchema(recordSchema)); + } + + /** Tests that an Avro nullable field with a valid union is correctly converted. */ + @Test + public void testNullableFieldWithValidUnion() { + String avroSchemaString = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"Record\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"nullableStringField\",\n" + + " \"type\": [\"null\", \"string\"]\n" + + " }\n" + + " ]\n" + + "}"; + Schema recordSchema = new Parser().parse(avroSchemaString); + + com.google.cloud.bigquery.Schema expectedBqSchema = + com.google.cloud.bigquery.Schema.of( + createNullableBigqueryField( + "nullableStringField", StandardSQLTypeName.STRING)); + + com.google.cloud.bigquery.Schema bqSchema = + AvroToBigQuerySchemaTransform.getBigQuerySchema(recordSchema); + assertExactSchema(bqSchema, expectedBqSchema); + } + + /** + * Tested Avro schema tested: "string" It should throw an Exception since this schema has no + * property as "name". + */ + @Test + public void testSchemaWithoutNamedFieldsThrowsException() { + Schema avroUnNamedSchema = Schema.create(Type.STRING); + + assertThrows( + IllegalArgumentException.class, + () -> AvroToBigQuerySchemaTransform.getBigQuerySchema(avroUnNamedSchema)); + } + + /** + * Tests that an Avro schema with a "map" type throws an exception with the correct error + * message. + */ + @Test + public void testUnsupportedMapTypeThrowsExceptionWithCorrectMessage() { + // Use an unsupported Avro type "map" + String avroSchemaString = + "{\n" + + " \"type\": \"record\",\n" + + " \"name\": \"RecordWithMapType\",\n" + + " \"fields\": [\n" + + " {\n" + + " \"name\": \"mapField\",\n" + + " \"type\": {\n" + + " \"type\": \"map\",\n" + + " \"values\": \"string\"\n" + + " }\n" + + " }\n" + + " ]\n" + + "}"; + Schema recordSchema = new Parser().parse(avroSchemaString); + + UnsupportedOperationException exception = + assertThrows( + UnsupportedOperationException.class, + () -> AvroToBigQuerySchemaTransform.getBigQuerySchema(recordSchema)); + + String expectedMessage = getUnsupportedTypeErrorMessage("MAP", "mapField"); + assertThat(exception.getMessage()).isEqualTo(expectedMessage); + } + + // Helper function to assert equality of two BigQuery schemas + private void assertExactSchema( + com.google.cloud.bigquery.Schema actual, com.google.cloud.bigquery.Schema expected) { + assertThat(actual.getFields().size()).isEqualTo(expected.getFields().size()); + for (int i = 0; i < actual.getFields().size(); i++) { + Field actualField = actual.getFields().get(i); + Field expectedField = expected.getFields().get(i); + assertThat(actualField.getName()).isEqualTo(expectedField.getName()); + assertThat(actualField.getType()).isEqualTo(expectedField.getType()); + assertThat(actualField.getMode()).isEqualTo(expectedField.getMode()); + if (expectedField.getPrecision() != null) { + assertThat(actualField.getPrecision()).isEqualTo(expectedField.getPrecision()); + } + if (expectedField.getScale() != null) { + assertThat(actualField.getScale()).isEqualTo(expectedField.getScale()); + } + if (actualField.getType() == LegacySQLTypeName.RECORD) { + assertExactSchema( + com.google.cloud.bigquery.Schema.of(actualField.getSubFields()), + com.google.cloud.bigquery.Schema.of(expectedField.getSubFields())); + } + } + } +}