Skip to content

Commit

Permalink
[java] BQ: Add avro schema to BQ TableSchema conversion (#33389)
Browse files Browse the repository at this point in the history
* [java] BQ: Add avro schema to BQ TableSchema conversion

* Use name that avoid compiler class with overload
  • Loading branch information
RustedBones authored Feb 15, 2025
1 parent 23ba9fc commit 53080f1
Show file tree
Hide file tree
Showing 4 changed files with 457 additions and 39 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.avro.Conversions;
import org.apache.avro.LogicalType;
import org.apache.avro.LogicalTypes;
Expand Down Expand Up @@ -530,4 +531,140 @@ private static Field convertField(
bigQueryField.getDescription(),
(Object) null /* Cast to avoid deprecated JsonNode constructor. */);
}

static TableSchema fromGenericAvroSchema(Schema schema) {
return fromGenericAvroSchema(schema, true);
}

static TableSchema fromGenericAvroSchema(Schema schema, Boolean useAvroLogicalTypes) {
verify(
schema.getType() == Type.RECORD,
"Expected Avro schema type RECORD, not %s",
schema.getType());

List<TableFieldSchema> fields =
schema.getFields().stream()
.map(f -> fromAvroFieldSchema(f, useAvroLogicalTypes))
.collect(Collectors.toList());
return new TableSchema().setFields(fields);
}

private static TableFieldSchema fromAvroFieldSchema(
Schema.Field avrofield, Boolean useAvroLogicalTypes) {
Schema fieldSchema = avrofield.schema();
TableFieldSchema field;
switch (fieldSchema.getType()) {
case UNION:
List<Schema> types = fieldSchema.getTypes();
verify(
types.size() == 2 && types.get(0).getType() == Type.NULL,
"Avro union field %s should be of null and another type, not %s",
avrofield.name(),
fieldSchema);
field = typedTableFieldSchema(types.get(1), useAvroLogicalTypes).setMode("NULLABLE");
break;
case ARRAY:
field =
typedTableFieldSchema(fieldSchema.getElementType(), useAvroLogicalTypes)
.setMode("REPEATED");
break;
case MAP:
TableFieldSchema key =
new TableFieldSchema().setType("STRING").setName("key").setMode("REQUIRED");
TableFieldSchema value =
typedTableFieldSchema(fieldSchema.getValueType(), useAvroLogicalTypes)
.setName("value")
.setMode("REQUIRED");
List<TableFieldSchema> mapTableSchema = new ArrayList<>();
mapTableSchema.add(key);
mapTableSchema.add(value);
field =
new TableFieldSchema().setType("RECORD").setFields(mapTableSchema).setMode("REPEATED");
break;
default:
field = typedTableFieldSchema(fieldSchema, useAvroLogicalTypes).setMode("REQUIRED");
}

return field.setName(avrofield.name()).setDescription(avrofield.doc());
}

private static TableFieldSchema typedTableFieldSchema(Schema type, Boolean useAvroLogicalTypes) {
TableFieldSchema fieldSchema = new TableFieldSchema();
LogicalType logicalType = useAvroLogicalTypes ? type.getLogicalType() : null;
String sqlType = useAvroLogicalTypes ? type.getProp("sqlType") : null;
switch (type.getType()) {
case INT:
if (logicalType instanceof LogicalTypes.Date) {
return fieldSchema.setType("DATE");
} else if (logicalType instanceof LogicalTypes.TimeMillis) {
return fieldSchema.setType("TIME");
} else {
return fieldSchema.setType("INTEGER");
}
case LONG:
if (logicalType instanceof LogicalTypes.TimeMicros) {
return fieldSchema.setType("TIME");
} else if (!(VERSION_AVRO.startsWith("1.8") || VERSION_AVRO.startsWith("1.9"))
&& (logicalType instanceof LogicalTypes.LocalTimestampMillis
|| logicalType instanceof LogicalTypes.LocalTimestampMicros)) {
return fieldSchema.setType("DATETIME");
} else if (logicalType instanceof LogicalTypes.TimestampMillis
|| logicalType instanceof LogicalTypes.TimestampMicros) {
return fieldSchema.setType("TIMESTAMP");
} else {
return fieldSchema.setType("INTEGER");
}
case FLOAT:
case DOUBLE:
return fieldSchema.setType("FLOAT");
case BOOLEAN:
return fieldSchema.setType("BOOLEAN");
case STRING:
if ("GEOGRAPHY".equals(sqlType)) {
return fieldSchema.setType("GEOGRAPHY");
} else if ("JSON".equals(sqlType)) {
return fieldSchema.setType("JSON");
} else {
return fieldSchema.setType("STRING");
}
case BYTES:
if (logicalType instanceof LogicalTypes.Decimal) {
LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType;
int precision = decimal.getPrecision();
int scale = decimal.getScale();
if (scale <= 9 && precision - scale <= 29) {
fieldSchema.setType("NUMERIC");
if (!(precision == 38 && scale == 9)) {
fieldSchema.setPrecision((long) precision);
if (scale != 0) {
fieldSchema.setScale((long) scale);
}
}
} else {
fieldSchema.setType("BIGNUMERIC");
if (!(precision == 77 && scale == 38)) {
fieldSchema.setPrecision((long) precision);
if (scale != 0) {
fieldSchema.setScale((long) scale);
}
}
}
return fieldSchema;
} else {
return fieldSchema.setType("BYTES");
}
case ENUM:
return fieldSchema.setType("STRING");
case FIXED:
return fieldSchema.setType("BYTES");
case RECORD:
List<TableFieldSchema> recordFields =
type.getFields().stream()
.map(f -> fromAvroFieldSchema(f, useAvroLogicalTypes))
.collect(Collectors.toList());
return new TableFieldSchema().setType("RECORD").setFields(recordFields);
default:
throw new IllegalArgumentException("Unknown Avro type: " + type.getType());
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -456,12 +456,23 @@ public static Schema fromTableSchema(TableSchema tableSchema, SchemaConversionOp
return fromTableFieldSchema(tableSchema.getFields(), options);
}

/** Convert a list of BigQuery {@link TableSchema} to Avro {@link org.apache.avro.Schema}. */
/** Convert a BigQuery {@link TableSchema} to Avro {@link org.apache.avro.Schema}. */
public static org.apache.avro.Schema toGenericAvroSchema(TableSchema tableSchema) {
return toGenericAvroSchema(tableSchema, false);
}

/** Convert a list of BigQuery {@link TableSchema} to Avro {@link org.apache.avro.Schema}. */
/** Convert an Avro {@link org.apache.avro.Schema} to a BigQuery {@link TableSchema}. */
public static TableSchema fromGenericAvroSchema(org.apache.avro.Schema schema) {
return fromGenericAvroSchema(schema, false);
}

/** Convert an Avro {@link org.apache.avro.Schema} to a BigQuery {@link TableSchema}. */
public static TableSchema fromGenericAvroSchema(
org.apache.avro.Schema schema, Boolean useAvroLogicalTypes) {
return BigQueryAvroUtils.fromGenericAvroSchema(schema, useAvroLogicalTypes);
}

/** Convert a BigQuery {@link TableSchema} to Avro {@link org.apache.avro.Schema}. */
public static org.apache.avro.Schema toGenericAvroSchema(
TableSchema tableSchema, Boolean useAvroLogicalTypes) {
return toGenericAvroSchema("root", tableSchema.getFields(), useAvroLogicalTypes);
Expand Down
Loading

0 comments on commit 53080f1

Please sign in to comment.