Skip to content

Commit

Permalink
Avro: Support default values for generic data (#11786)
Browse files Browse the repository at this point in the history
  • Loading branch information
rdblue authored Dec 16, 2024
1 parent 57ea310 commit b9b61b1
Show file tree
Hide file tree
Showing 20 changed files with 533 additions and 42 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,10 @@
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;

/**
* @deprecated will be removed in 2.0.0; use {@link PlannedDataReader} instead.
*/
@Deprecated
public class DataReader<T> implements DatumReader<T>, SupportsRowPosition {

public static <D> DataReader<D> create(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.iceberg.data.Record;
import org.apache.iceberg.types.Types.StructType;
import org.apache.iceberg.util.DateTimeUtil;
import org.apache.iceberg.util.Pair;

class GenericReaders {
private GenericReaders() {}
Expand All @@ -52,6 +53,11 @@ static ValueReader<OffsetDateTime> timestamptz() {
return TimestamptzReader.INSTANCE;
}

static ValueReader<Record> struct(
List<Pair<Integer, ValueReader<?>>> readPlan, StructType struct) {
return new PlannedRecordReader(readPlan, struct);
}

static ValueReader<Record> struct(
StructType struct, List<ValueReader<?>> readers, Map<Integer, ?> idToConstant) {
return new GenericRecordReader(readers, struct, idToConstant);
Expand Down Expand Up @@ -101,6 +107,34 @@ public OffsetDateTime read(Decoder decoder, Object reuse) throws IOException {
}
}

private static class PlannedRecordReader extends ValueReaders.PlannedStructReader<Record> {
private final StructType structType;

private PlannedRecordReader(List<Pair<Integer, ValueReader<?>>> readPlan, StructType struct) {
super(readPlan);
this.structType = struct;
}

@Override
protected Record reuseOrCreate(Object reuse) {
if (reuse instanceof Record) {
return (Record) reuse;
} else {
return GenericRecord.create(structType);
}
}

@Override
protected Object get(Record struct, int pos) {
return struct.get(pos);
}

@Override
protected void set(Record struct, int pos, Object value) {
struct.set(pos, value);
}
}

private static class GenericRecordReader extends ValueReaders.StructReader<Record> {
private final StructType structType;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -103,9 +103,7 @@ public void addSchema(org.apache.iceberg.Schema writeSchema) {

private void addSchema(Schema writeSchema) {
long fp = SchemaNormalization.parsingFingerprint64(writeSchema);
RawDecoder decoder =
new RawDecoder<>(
readSchema, avroSchema -> DataReader.create(readSchema, avroSchema), writeSchema);
RawDecoder<D> decoder = RawDecoder.create(readSchema, PlannedDataReader::create, writeSchema);
decoders.put(fp, decoder);
}

Expand Down
187 changes: 187 additions & 0 deletions core/src/main/java/org/apache/iceberg/data/avro/PlannedDataReader.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,187 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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 org.apache.iceberg.data.avro;

import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.function.Supplier;
import org.apache.avro.LogicalType;
import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
import org.apache.avro.io.DatumReader;
import org.apache.avro.io.Decoder;
import org.apache.iceberg.avro.AvroSchemaUtil;
import org.apache.iceberg.avro.AvroWithPartnerVisitor;
import org.apache.iceberg.avro.SupportsRowPosition;
import org.apache.iceberg.avro.ValueReader;
import org.apache.iceberg.avro.ValueReaders;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.util.Pair;

public class PlannedDataReader<T> implements DatumReader<T>, SupportsRowPosition {

public static <D> PlannedDataReader<D> create(org.apache.iceberg.Schema expectedSchema) {
return create(expectedSchema, ImmutableMap.of());
}

public static <D> PlannedDataReader<D> create(
org.apache.iceberg.Schema expectedSchema, Map<Integer, ?> idToConstant) {
return new PlannedDataReader<>(expectedSchema, idToConstant);
}

private final org.apache.iceberg.Schema expectedSchema;
private final Map<Integer, ?> idToConstant;
private ValueReader<T> reader;

protected PlannedDataReader(
org.apache.iceberg.Schema expectedSchema, Map<Integer, ?> idToConstant) {
this.expectedSchema = expectedSchema;
this.idToConstant = idToConstant;
}

@Override
@SuppressWarnings("unchecked")
public void setSchema(Schema fileSchema) {
this.reader =
(ValueReader<T>)
AvroWithPartnerVisitor.visit(
expectedSchema.asStruct(),
fileSchema,
new ReadBuilder(idToConstant),
AvroWithPartnerVisitor.FieldIDAccessors.get());
}

@Override
public T read(T reuse, Decoder decoder) throws IOException {
return reader.read(decoder, reuse);
}

@Override
public void setRowPositionSupplier(Supplier<Long> posSupplier) {
if (reader instanceof SupportsRowPosition) {
((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier);
}
}

private static class ReadBuilder extends AvroWithPartnerVisitor<Type, ValueReader<?>> {
private final Map<Integer, ?> idToConstant;

private ReadBuilder(Map<Integer, ?> idToConstant) {
this.idToConstant = idToConstant;
}

@Override
public ValueReader<?> record(Type partner, Schema record, List<ValueReader<?>> fieldReaders) {
if (partner == null) {
return ValueReaders.skipStruct(fieldReaders);
}

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

return GenericReaders.struct(readPlan, expected);
}

@Override
public ValueReader<?> union(Type partner, Schema union, List<ValueReader<?>> options) {
return ValueReaders.union(options);
}

@Override
public ValueReader<?> array(Type ignored, Schema array, ValueReader<?> elementReader) {
return ValueReaders.array(elementReader);
}

@Override
public ValueReader<?> arrayMap(
Type ignored, Schema map, ValueReader<?> keyReader, ValueReader<?> valueReader) {
return ValueReaders.arrayMap(keyReader, valueReader);
}

@Override
public ValueReader<?> map(Type ignored, Schema map, ValueReader<?> valueReader) {
return ValueReaders.map(ValueReaders.strings(), valueReader);
}

@Override
public ValueReader<?> primitive(Type partner, Schema primitive) {
LogicalType logicalType = primitive.getLogicalType();
if (logicalType != null) {
switch (logicalType.getName()) {
case "date":
return GenericReaders.dates();

case "time-micros":
return GenericReaders.times();

case "timestamp-micros":
if (AvroSchemaUtil.isTimestamptz(primitive)) {
return GenericReaders.timestamptz();
}
return GenericReaders.timestamps();

case "decimal":
return ValueReaders.decimal(
ValueReaders.decimalBytesReader(primitive),
((LogicalTypes.Decimal) logicalType).getScale());

case "uuid":
return ValueReaders.uuids();

default:
throw new IllegalArgumentException("Unknown logical type: " + logicalType);
}
}

switch (primitive.getType()) {
case NULL:
return ValueReaders.nulls();
case BOOLEAN:
return ValueReaders.booleans();
case INT:
if (partner != null && partner.typeId() == Type.TypeID.LONG) {
return ValueReaders.intsAsLongs();
}
return ValueReaders.ints();
case LONG:
return ValueReaders.longs();
case FLOAT:
if (partner != null && partner.typeId() == Type.TypeID.DOUBLE) {
return ValueReaders.floatsAsDoubles();
}
return ValueReaders.floats();
case DOUBLE:
return ValueReaders.doubles();
case STRING:
// might want to use a binary-backed container like Utf8
return ValueReaders.strings();
case FIXED:
return ValueReaders.fixed(primitive.getFixedSize());
case BYTES:
return ValueReaders.byteBuffers();
default:
throw new IllegalArgumentException("Unsupported type: " + primitive);
}
}
}
}
32 changes: 32 additions & 0 deletions core/src/main/java/org/apache/iceberg/data/avro/RawDecoder.java
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,27 @@
public class RawDecoder<D> extends MessageDecoder.BaseDecoder<D> {
private static final ThreadLocal<BinaryDecoder> DECODER = new ThreadLocal<>();

/**
* Creates a new {@link MessageDecoder} that constructs datum instances described by the {@link
* Schema readSchema}.
*
* <p>The {@code readSchema} is used for the expected schema and the {@code writeSchema} is the
* schema used to decode buffers. The {@code writeSchema} must be the schema that was used to
* encode all buffers decoded by this class.
*
* @param readSchema an Iceberg schema to produce when reading
* @param readerFunction a function that produces a DatumReader from the read schema
* @param writeSchema an Avro schema that describes serialized data to be read
*/
public static <D> RawDecoder<D> create(
org.apache.iceberg.Schema readSchema,
Function<org.apache.iceberg.Schema, DatumReader<D>> readerFunction,
Schema writeSchema) {
DatumReader<D> reader = readerFunction.apply(readSchema);
reader.setSchema(writeSchema);
return new RawDecoder<>(reader);
}

private final DatumReader<D> reader;

/**
Expand All @@ -42,7 +63,11 @@ public class RawDecoder<D> extends MessageDecoder.BaseDecoder<D> {
* <p>The {@code readSchema} is used for the expected schema and the {@code writeSchema} is the
* schema used to decode buffers. The {@code writeSchema} must be the schema that was used to
* encode all buffers decoded by this class.
*
* @deprecated will be removed in 2.0.0; use {@link #create(org.apache.iceberg.Schema, Function,
* Schema)} instead
*/
@Deprecated
public RawDecoder(
org.apache.iceberg.Schema readSchema,
Function<Schema, DatumReader<?>> readerFunction,
Expand All @@ -51,6 +76,13 @@ public RawDecoder(
this.reader.setSchema(writeSchema);
}

/**
* Creates a new {@link MessageDecoder} that constructs datum instances using the {@code reader}.
*/
private RawDecoder(DatumReader<D> reader) {
this.reader = reader;
}

@Override
public D decode(InputStream stream, D reuse) {
BinaryDecoder decoder = DecoderFactory.get().directBinaryDecoder(stream, DECODER.get());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ public StandardKeyMetadata decode(InputStream stream, StandardKeyMetadata reuse)
RawDecoder<StandardKeyMetadata> decoder = decoders.get(writeSchemaVersion);

if (decoder == null) {
decoder = new RawDecoder<>(readSchema, GenericAvroReader::create, writeSchema);
decoder = RawDecoder.create(readSchema, GenericAvroReader::create, writeSchema);

decoders.put(writeSchemaVersion, decoder);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@
import org.apache.iceberg.SortOrder;
import org.apache.iceberg.data.GenericRecord;
import org.apache.iceberg.data.Record;
import org.apache.iceberg.data.avro.DataReader;
import org.apache.iceberg.data.avro.PlannedDataReader;
import org.apache.iceberg.io.DataWriter;
import org.apache.iceberg.io.OutputFile;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
Expand Down Expand Up @@ -103,7 +103,7 @@ public void testDataWriter() throws IOException {
try (AvroIterable<Record> reader =
Avro.read(file.toInputFile())
.project(SCHEMA)
.createReaderFunc(DataReader::create)
.createResolvingReader(PlannedDataReader::create)
.build()) {
writtenRecords = Lists.newArrayList(reader);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,8 +33,8 @@
import org.apache.iceberg.Schema;
import org.apache.iceberg.data.GenericRecord;
import org.apache.iceberg.data.Record;
import org.apache.iceberg.data.avro.DataReader;
import org.apache.iceberg.data.avro.DataWriter;
import org.apache.iceberg.data.avro.PlannedDataReader;
import org.apache.iceberg.deletes.EqualityDeleteWriter;
import org.apache.iceberg.deletes.PositionDelete;
import org.apache.iceberg.deletes.PositionDeleteWriter;
Expand Down Expand Up @@ -102,7 +102,10 @@ public void testEqualityDeleteWriter() throws IOException {

List<Record> deletedRecords;
try (AvroIterable<Record> reader =
Avro.read(out.toInputFile()).project(SCHEMA).createReaderFunc(DataReader::create).build()) {
Avro.read(out.toInputFile())
.project(SCHEMA)
.createResolvingReader(PlannedDataReader::create)
.build()) {
deletedRecords = Lists.newArrayList(reader);
}

Expand Down Expand Up @@ -158,7 +161,7 @@ public void testPositionDeleteWriter() throws IOException {
try (AvroIterable<Record> reader =
Avro.read(out.toInputFile())
.project(deleteSchema)
.createReaderFunc(DataReader::create)
.createResolvingReader(PlannedDataReader::create)
.build()) {
deletedRecords = Lists.newArrayList(reader);
}
Expand Down Expand Up @@ -212,7 +215,7 @@ public void testPositionDeleteWriterWithEmptyRow() throws IOException {
try (AvroIterable<Record> reader =
Avro.read(out.toInputFile())
.project(deleteSchema)
.createReaderFunc(DataReader::create)
.createResolvingReader(PlannedDataReader::create)
.build()) {
deletedRecords = Lists.newArrayList(reader);
}
Expand Down
Loading

0 comments on commit b9b61b1

Please sign in to comment.