[ https://issues.apache.org/jira/browse/BEAM-4454?focusedWorklogId=171585&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-171585 ]
ASF GitHub Bot logged work on BEAM-4454: ---------------------------------------- Author: ASF GitHub Bot Created on: 03/Dec/18 13:44 Start Date: 03/Dec/18 13:44 Worklog Time Spent: 10m Work Description: kanterov commented on a change in pull request #7181: [BEAM-4454] Add more AVRO utilities to convert between Beam and Avro. URL: https://github.com/apache/beam/pull/7181#discussion_r238271611 ########## File path: sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java ########## @@ -20,119 +20,408 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import java.math.BigDecimal; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import org.apache.avro.Conversions; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema.Type; import org.apache.avro.generic.GenericEnumSymbol; import org.apache.avro.generic.GenericFixed; import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.GenericRecordBuilder; +import org.apache.avro.reflect.ReflectData; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.Field; +import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.schemas.Schema.TypeName; import org.apache.beam.sdk.values.Row; +import org.joda.time.Instant; +import org.joda.time.ReadableInstant; /** Utils to convert AVRO records to Beam rows. */ @Experimental(Experimental.Kind.SCHEMAS) public class AvroUtils { + // Unwrap an AVRO schema into the base type an whether it is nullable. + static class TypeWithNullability { + public final org.apache.avro.Schema type; + public final boolean nullable; + + TypeWithNullability(org.apache.avro.Schema avroSchema) { + if (avroSchema.getType() == org.apache.avro.Schema.Type.UNION) { + List<org.apache.avro.Schema> types = avroSchema.getTypes(); + + // optional fields in AVRO have form of: + // {"name": "foo", "type": ["null", "something"]} + + // don't need recursion because nested unions aren't supported in AVRO + List<org.apache.avro.Schema> nonNullTypes = + types + .stream() + .filter(x -> x.getType() != org.apache.avro.Schema.Type.NULL) + .collect(Collectors.toList()); + + if (nonNullTypes.size() == types.size() || nonNullTypes.isEmpty()) { + // union without `null` or all 'null' union, keep as is. + type = avroSchema; + nullable = false; + } else if (nonNullTypes.size() > 1) { + type = org.apache.avro.Schema.createUnion(nonNullTypes); + nullable = true; + } else { + // One non-null type. + type = nonNullTypes.get(0); + nullable = true; + } + } else { + type = avroSchema; + nullable = false; + } + } + } + private AvroUtils() {} /** * Converts AVRO schema to Beam row schema. * * @param schema schema of type RECORD */ - public static Schema toSchema(@Nonnull org.apache.avro.Schema schema) { + public static Schema toBeamSchema(org.apache.avro.Schema schema) { Schema.Builder builder = Schema.builder(); for (org.apache.avro.Schema.Field field : schema.getFields()) { - org.apache.avro.Schema unwrapped = unwrapNullableSchema(field.schema()); - - if (!unwrapped.equals(field.schema())) { - builder.addNullableField(field.name(), toFieldType(unwrapped)); - } else { - builder.addField(field.name(), toFieldType(unwrapped)); + TypeWithNullability nullableType = new TypeWithNullability(field.schema()); + Field beamField = Field.of(field.name(), toFieldType(nullableType)); + if (field.doc() != null) { + beamField = beamField.withDescription(field.doc()); } + builder.addField(beamField); } return builder.build(); } - /** Converts AVRO schema to Beam field. */ - public static Schema.FieldType toFieldType(@Nonnull org.apache.avro.Schema avroSchema) { - switch (avroSchema.getType()) { - case RECORD: - return Schema.FieldType.row(toSchema(avroSchema)); + /** Converts a Beam Schema into an AVRO schema. */ + public static org.apache.avro.Schema toAvroSchema(Schema beamSchema) { + List<org.apache.avro.Schema.Field> fields = Lists.newArrayList(); + for (Schema.Field field : beamSchema.getFields()) { + org.apache.avro.Schema fieldSchema = getFieldSchema(field.getType()); + org.apache.avro.Schema.Field recordField = + new org.apache.avro.Schema.Field( + field.getName(), fieldSchema, field.getDescription(), (Object) null); + fields.add(recordField); + } + org.apache.avro.Schema avroSchema = org.apache.avro.Schema.createRecord(fields); + return avroSchema; + } - case ENUM: - return Schema.FieldType.STRING; + /** + * Strict conversion from AVRO to Beam, strict because it doesn't do widening or narrowing during + * conversion. If Schema is not provided, one is inferred from the AVRO schema. + */ + public static Row toBeamRowStrict(GenericRecord record, @Nullable Schema schema) { + if (schema == null) { + schema = toBeamSchema(record.getSchema()); + } - case ARRAY: - Schema.FieldType elementType = toFieldType(avroSchema.getElementType()); - return Schema.FieldType.array(elementType); + Row.Builder builder = Row.withSchema(schema); + org.apache.avro.Schema avroSchema = record.getSchema(); - case MAP: - return Schema.FieldType.map( - Schema.FieldType.STRING, toFieldType(avroSchema.getValueType())); + for (Schema.Field field : schema.getFields()) { + Object value = record.get(field.getName()); + org.apache.avro.Schema fieldAvroSchema = avroSchema.getField(field.getName()).schema(); - case FIXED: - return Schema.FieldType.BYTES; + if (value == null) { + builder.addValue(null); + } else { + builder.addValue(convertAvroFieldStrict(value, fieldAvroSchema, field.getType())); + } + } - case STRING: - return Schema.FieldType.STRING; + return builder.build(); + } - case BYTES: - return Schema.FieldType.BYTES; + /** + * Convert from a Beam Row to an AVRO GenericRecord. If a Schema is not provided, one is inferred + * from the Beam schema on the orw. + */ + public static GenericRecord toGenericRecord( + Row row, @Nullable org.apache.avro.Schema avroSchema) { + Schema beamSchema = row.getSchema(); + // Use the provided AVRO schema if present, otherwise infer an AVRO schema from the row + // schema. + if (avroSchema != null && avroSchema.getFields().size() != beamSchema.getFieldCount()) { + throw new IllegalArgumentException( + "AVRO schema doesn't match row schema. Row schema " + + beamSchema + + ". AVRO schema + " + + avroSchema); + } + if (avroSchema == null) { + avroSchema = toAvroSchema(beamSchema); + } - case INT: - return Schema.FieldType.INT32; + List<org.apache.avro.Schema.Field> avroFields = + Lists.newArrayListWithExpectedSize(beamSchema.getFieldCount()); + GenericRecordBuilder builder = new GenericRecordBuilder(avroSchema); + for (int i = 0; i < beamSchema.getFieldCount(); ++i) { + Schema.Field field = beamSchema.getField(i); + builder.set( + field.getName(), + genericFromBeamField( + field.getType(), avroSchema.getField(field.getName()).schema(), row.getValue(i))); + } + return builder.build(); + } - case LONG: - return Schema.FieldType.INT64; + /** Converts AVRO schema to Beam field. */ + private static Schema.FieldType toFieldType(TypeWithNullability type) { + Schema.FieldType fieldType = null; + org.apache.avro.Schema avroSchema = type.type; + + LogicalType logicalType = LogicalTypes.fromSchema(avroSchema); + if (logicalType != null) { + if (logicalType instanceof LogicalTypes.Decimal) { + fieldType = FieldType.DECIMAL; + } else if (logicalType instanceof LogicalTypes.TimestampMillis) { + // TODO: There is a desire to move Beam schema DATETIME to a micros representation. When + // this is done, this logical type needs to be changed. + fieldType = FieldType.DATETIME; + } + } + + if (fieldType == null) { + switch (type.type.getType()) { + case RECORD: + fieldType = Schema.FieldType.row(toBeamSchema(avroSchema)); + break; + + case ENUM: + fieldType = Schema.FieldType.STRING; + break; + + case ARRAY: + Schema.FieldType elementType = + toFieldType(new TypeWithNullability(avroSchema.getElementType())); + fieldType = Schema.FieldType.array(elementType); + break; + + case MAP: + fieldType = + Schema.FieldType.map( + Schema.FieldType.STRING, + toFieldType(new TypeWithNullability(avroSchema.getValueType()))); + break; + + case FIXED: + fieldType = Schema.FieldType.BYTES; + break; + + case STRING: + fieldType = Schema.FieldType.STRING; + break; + + case BYTES: + fieldType = Schema.FieldType.BYTES; + break; + + case INT: + fieldType = Schema.FieldType.INT32; + break; + + case LONG: + fieldType = Schema.FieldType.INT64; + break; + + case FLOAT: + fieldType = Schema.FieldType.FLOAT; + break; + + case DOUBLE: + fieldType = Schema.FieldType.DOUBLE; + break; + + case BOOLEAN: + fieldType = Schema.FieldType.BOOLEAN; + break; + + case UNION: + throw new RuntimeException("Can't convert 'union' to FieldType"); + + case NULL: + throw new RuntimeException("Can't convert 'null' to FieldType"); + + default: + throw new AssertionError("Unexpected AVRO Schema.Type: " + avroSchema.getType()); + } + } + fieldType = fieldType.withNullable(type.nullable); + return fieldType; + } + + private static org.apache.avro.Schema getFieldSchema(Schema.FieldType fieldType) { + org.apache.avro.Schema baseType; + switch (fieldType.getTypeName()) { + case BYTE: + case INT16: + case INT32: + baseType = org.apache.avro.Schema.create(Type.INT); + break; + + case INT64: + baseType = org.apache.avro.Schema.create(Type.LONG); + break; + + case DECIMAL: + baseType = + LogicalTypes.decimal(Integer.MAX_VALUE) + .addToSchema(org.apache.avro.Schema.create(Type.BYTES)); + break; case FLOAT: - return Schema.FieldType.FLOAT; + baseType = org.apache.avro.Schema.create(Type.FLOAT); + break; case DOUBLE: - return Schema.FieldType.DOUBLE; + baseType = org.apache.avro.Schema.create(Type.DOUBLE); + break; + + case STRING: + baseType = org.apache.avro.Schema.create(Type.STRING); + break; + + case DATETIME: + // TODO: There is a desire to move Beam schema DATETIME to a micros representation. When + // this is done, this logical type needs to be changed. + baseType = + LogicalTypes.timestampMillis().addToSchema(org.apache.avro.Schema.create(Type.LONG)); + break; case BOOLEAN: - return Schema.FieldType.BOOLEAN; + baseType = org.apache.avro.Schema.create(Type.BOOLEAN); + break; - case UNION: - throw new RuntimeException("Can't convert 'union' to FieldType"); + case BYTES: + baseType = org.apache.avro.Schema.create(Type.BYTES); + break; - case NULL: - throw new RuntimeException("Can't convert 'null' to FieldType"); + case ARRAY: + baseType = + org.apache.avro.Schema.createArray( + getFieldSchema(fieldType.getCollectionElementType())); + break; + + case MAP: + if (fieldType.getMapKeyType().getTypeName().isStringType()) { + // Avro only supports string keys in maps. + baseType = org.apache.avro.Schema.createMap(getFieldSchema(fieldType.getMapValueType())); + } else { + throw new IllegalArgumentException("Avro only supports maps with string keys"); + } + break; + + case ROW: + baseType = toAvroSchema(fieldType.getRowSchema()); + break; default: - throw new AssertionError("Unexpected AVRO Schema.Type: " + avroSchema.getType()); + throw new IllegalArgumentException("Unexpected type " + fieldType); } + return fieldType.getNullable() ? ReflectData.makeNullable(baseType) : baseType; } - /** - * Strict conversion from AVRO to Beam, strict because it doesn't do widening or narrowing during - * conversion. - */ - public static Row toRowStrict(@Nonnull GenericRecord record, @Nonnull Schema schema) { - Row.Builder builder = Row.withSchema(schema); - org.apache.avro.Schema avroSchema = record.getSchema(); + private static Object genericFromBeamField( + Schema.FieldType fieldType, org.apache.avro.Schema avroSchema, Object value) { + org.apache.avro.Schema expectedSchema = getFieldSchema(fieldType); + switch (fieldType.getTypeName()) { + case BYTE: + case INT16: + case INT32: + case INT64: + case FLOAT: + case DOUBLE: + case STRING: Review comment: It happens because we when we create `Map` we should supply `Utf8` as keys, otherwise, equality doesn't work. Something like: ``` - entry.getKey() + new Utf8((String)entry.getKey()), ``` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 171585) Time Spent: 4.5h (was: 4h 20m) > Provide automatic schema registration for AVROs > ----------------------------------------------- > > Key: BEAM-4454 > URL: https://issues.apache.org/jira/browse/BEAM-4454 > Project: Beam > Issue Type: Sub-task > Components: sdk-java-core > Reporter: Reuven Lax > Assignee: Reuven Lax > Priority: Major > Time Spent: 4.5h > Remaining Estimate: 0h > > Need to make sure this is a compatible change -- This message was sent by Atlassian JIRA (v7.6.3#76005)