Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Avro: Add variant type support #11934

Draft
wants to merge 4 commits into
base: main
Choose a base branch
from
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 8 additions & 0 deletions api/src/main/java/org/apache/iceberg/types/Type.java
Original file line number Diff line number Diff line change
Expand Up @@ -81,6 +81,10 @@ default Types.MapType asMapType() {
throw new IllegalArgumentException("Not a map type: " + this);
}

default Types.VariantType asVariantType() {
throw new IllegalArgumentException("Not a variant type: " + this);
}

default boolean isNestedType() {
return false;
}
Expand All @@ -97,6 +101,10 @@ default boolean isMapType() {
return false;
}

default boolean isVariantType() {
return false;
}

default NestedType asNestedType() {
throw new IllegalArgumentException("Not a nested type: " + this);
}
Expand Down
7 changes: 7 additions & 0 deletions api/src/main/java/org/apache/iceberg/types/TypeUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -709,6 +709,10 @@ public T map(Types.MapType map, Supplier<T> keyResult, Supplier<T> valueResult)
return null;
}

public T variant() {
return null;
}

public T primitive(Type.PrimitiveType primitive) {
return null;
}
Expand Down Expand Up @@ -785,6 +789,9 @@ public static <T> T visit(Type type, CustomOrderSchemaVisitor<T> visitor) {
new VisitFuture<>(map.keyType(), visitor),
new VisitFuture<>(map.valueType(), visitor));

case VARIANT:
return visitor.variant();

default:
return visitor.primitive(type.asPrimitiveType());
}
Expand Down
18 changes: 18 additions & 0 deletions api/src/main/java/org/apache/iceberg/types/Types.java
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,14 @@ private Types() {}
private static final Pattern DECIMAL =
Pattern.compile("decimal\\(\\s*(\\d+)\\s*,\\s*(\\d+)\\s*\\)");

public static Type typeFromTypeString(String typeString) {
if (VariantType.get().toString().equalsIgnoreCase(typeString)) {
return Types.VariantType.get();
}

return Types.fromPrimitiveString(typeString);
}

public static PrimitiveType fromPrimitiveString(String typeString) {
String lowerTypeString = typeString.toLowerCase(Locale.ROOT);
if (TYPES.containsKey(lowerTypeString)) {
Expand Down Expand Up @@ -429,6 +437,16 @@ public String toString() {
return "variant";
}

@Override
public VariantType asVariantType() {
return this;
}

@Override
public boolean isVariantType() {
return true;
}

@Override
public boolean equals(Object o) {
if (this == o) {
Expand Down
55 changes: 55 additions & 0 deletions api/src/test/java/org/apache/iceberg/util/RandomUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,9 @@
*/
package org.apache.iceberg.util;

import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ObjectNode;
import java.math.BigDecimal;
import java.math.BigInteger;
import java.util.Arrays;
Expand Down Expand Up @@ -228,4 +231,56 @@ private static BigInteger randomUnscaled(int precision, Random random) {

return new BigInteger(sb.toString());
}

public static String generateRandomJsonString(int numFields, Random random)
throws JsonProcessingException {
ObjectMapper mapper = new ObjectMapper();
ObjectNode jsonObject = mapper.createObjectNode();
for (int i = 0; i < numFields; i++) {
String key = randomString(random);
Object value = randomValue(random);
jsonObject.putPOJO(key, value);
}

return mapper.writeValueAsString(jsonObject);
}

private static Object randomValue(Random random) {
Type.PrimitiveType primitive = randomPrimitiveType(random);
return generatePrimitive(primitive, random);
}

private static Type.PrimitiveType randomPrimitiveType(Random random) {
int choice = random.nextInt(13);
switch (choice) {
case 0:
return Types.BooleanType.get();
case 1:
return Types.IntegerType.get();
case 2:
return Types.LongType.get();
case 3:
return Types.FloatType.get();
case 4:
return Types.DoubleType.get();
case 5:
return Types.DateType.get();
case 6:
return Types.TimeType.get();
case 7:
return Types.TimestampType.withoutZone();
case 8:
return Types.StringType.get();
case 9:
return Types.UUIDType.get();
case 10:
return Types.FixedType.ofLength(16);
case 11:
return Types.BinaryType.get();
case 12:
return Types.DecimalType.of(38, 9);
default:
throw new IllegalArgumentException("Unknown type choice: " + choice);
}
}
}
4 changes: 3 additions & 1 deletion core/src/main/java/org/apache/iceberg/SchemaParser.java
Original file line number Diff line number Diff line change
Expand Up @@ -145,6 +145,8 @@ static void toJson(Type.PrimitiveType primitive, JsonGenerator generator) throws
static void toJson(Type type, JsonGenerator generator) throws IOException {
if (type.isPrimitiveType()) {
toJson(type.asPrimitiveType(), generator);
} else if (type.isVariantType()) {
generator.writeString(type.toString());
} else {
Type.NestedType nested = type.asNestedType();
switch (type.typeId()) {
Expand Down Expand Up @@ -179,7 +181,7 @@ public static String toJson(Schema schema, boolean pretty) {

private static Type typeFromJson(JsonNode json) {
if (json.isTextual()) {
return Types.fromPrimitiveString(json.asText());
return Types.typeFromTypeString(json.asText());
} else if (json.isObject()) {
JsonNode typeObj = json.get(TYPE);
if (typeObj != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,10 @@ class BuildAvroProjection extends AvroCustomOrderSchemaVisitor<Schema, Schema.Fi
@Override
@SuppressWarnings("checkstyle:CyclomaticComplexity")
public Schema record(Schema record, List<String> names, Iterable<Schema.Field> schemaIterable) {
if (current.isVariantType()) {
return variant(record);
}

Preconditions.checkArgument(
current.isNestedType() && current.asNestedType().isStructType(),
"Cannot project non-struct: %s",
Expand Down Expand Up @@ -130,6 +134,17 @@ public Schema record(Schema record, List<String> names, Iterable<Schema.Field> s
return record;
}

private Schema variant(Schema record) {
Preconditions.checkArgument(
current.isVariantType()
&& record.getField("value") != null
&& record.getField("metadata") != null,
"Expect variant type with value and metadata fields: %s",
current);

return record;
}

@Override
public Schema.Field field(Schema.Field field, Supplier<Schema> fieldResult) {
Types.StructType struct = current.asNestedType().asStructType();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -112,11 +112,14 @@ public ValueReader<?> record(Type partner, Schema record, List<ValueReader<?>> f
return ValueReaders.skipStruct(fieldResults);
}

Types.StructType expected = partner.asStructType();
List<Pair<Integer, ValueReader<?>>> readPlan =
ValueReaders.buildReadPlan(expected, record, fieldResults, idToConstant);

return recordReader(readPlan, avroSchemas.get(partner), record.getFullName());
if (partner.isVariantType()) {
return ValueReaders.record(fieldResults, record);
} else {
Types.StructType expected = partner.asStructType();
List<Pair<Integer, ValueReader<?>>> readPlan =
ValueReaders.buildReadPlan(expected, record, fieldResults, idToConstant);
return recordReader(readPlan, avroSchemas.get(partner), record.getFullName());
}
}

@SuppressWarnings("unchecked")
Expand Down
4 changes: 4 additions & 0 deletions core/src/main/java/org/apache/iceberg/avro/SchemaToType.java
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,10 @@ public Type record(Schema record, List<String> names, List<Type> fieldTypes) {
List<Schema.Field> fields = record.getFields();
List<Types.NestedField> newFields = Lists.newArrayListWithExpectedSize(fields.size());

if ("variant".equals(record.getName())) {
return Types.VariantType.get();
}

if (Objects.equals(root, record)) {
this.nextId = 0;
}
Expand Down
16 changes: 16 additions & 0 deletions core/src/main/java/org/apache/iceberg/avro/TypeToSchema.java
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,15 @@ abstract class TypeToSchema extends TypeUtil.SchemaVisitor<Schema> {
private static final Schema UUID_SCHEMA =
LogicalTypes.uuid().addToSchema(Schema.createFixed("uuid_fixed", null, null, 16));
private static final Schema BINARY_SCHEMA = Schema.create(Schema.Type.BYTES);
private static final Schema VARIANT_SCHEMA =
Schema.createRecord(
"variant",
null,
null,
false,
List.of(
new Schema.Field("metadata", BINARY_SCHEMA),
new Schema.Field("value", BINARY_SCHEMA)));

static {
TIMESTAMP_SCHEMA.addProp(AvroSchemaUtil.ADJUST_TO_UTC_PROP, false);
Expand Down Expand Up @@ -187,6 +196,13 @@ public Schema map(Types.MapType map, Schema keySchema, Schema valueSchema) {
return mapSchema;
}

@Override
public Schema variant() {
cacheSchema(Types.VariantType.get(), VARIANT_SCHEMA);

return VARIANT_SCHEMA;
}

@Override
public Schema primitive(Type.PrimitiveType primitive) {
Schema primitiveSchema;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,8 +61,8 @@ private SerializedArray(SerializedMetadata metadata, ByteBuffer value, int heade
this.array = new VariantValue[numElements];
}

@VisibleForTesting
int numElements() {
@Override
public int numElements() {
return array.length;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.util.Pair;
Expand Down Expand Up @@ -112,8 +111,8 @@ private void initOffsetsAndLengths(int numElements) {
}
}

@VisibleForTesting
int numElements() {
@Override
public int numElements() {
return fieldIds.length;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,10 @@

/** An variant array value. */
public interface VariantArray extends VariantValue {
default int numElements() {
throw new UnsupportedOperationException();
}

/** Returns the {@link VariantValue} at {@code index} in this array. */
VariantValue get(int index);

Expand Down
Loading