Skip to content

Commit

Permalink
Spark 3.3: Backport support for default values (#11988)
Browse files Browse the repository at this point in the history
  • Loading branch information
rdblue authored Jan 21, 2025
1 parent e13a87f commit 5b13760
Show file tree
Hide file tree
Showing 27 changed files with 1,369 additions and 816 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,35 +18,45 @@
*/
package org.apache.iceberg.spark;

import java.math.BigDecimal;
import java.nio.ByteBuffer;
import java.sql.Date;
import java.sql.Timestamp;
import java.util.List;
import java.util.Map;
import java.util.function.BiFunction;
import java.util.function.Function;
import java.util.stream.Collectors;
import org.apache.avro.generic.GenericData;
import org.apache.avro.util.Utf8;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.PartitionField;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.StructLike;
import org.apache.iceberg.relocated.com.google.common.base.Joiner;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.transforms.Transform;
import org.apache.iceberg.transforms.UnknownTransform;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.TypeUtil;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.util.ByteBuffers;
import org.apache.iceberg.util.Pair;
import org.apache.spark.sql.RuntimeConfig;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.catalyst.expressions.BoundReference;
import org.apache.spark.sql.catalyst.expressions.EqualTo;
import org.apache.spark.sql.catalyst.expressions.Expression;
import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
import org.apache.spark.sql.catalyst.expressions.Literal;
import org.apache.spark.sql.connector.expressions.NamedReference;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.DataTypes;
import org.apache.spark.sql.types.Decimal;
import org.apache.spark.sql.types.StructType;
import org.apache.spark.unsafe.types.UTF8String;
import org.joda.time.DateTime;

public class SparkUtil {
Expand Down Expand Up @@ -282,4 +292,60 @@ public static String toColumnName(NamedReference ref) {
public static boolean caseSensitive(SparkSession spark) {
return Boolean.parseBoolean(spark.conf().get("spark.sql.caseSensitive"));
}

/**
* Converts a value to pass into Spark from Iceberg's internal object model.
*
* @param type an Iceberg type
* @param value a value that is an instance of {@link Type.TypeID#javaClass()}
* @return the value converted for Spark
*/
public static Object internalToSpark(Type type, Object value) {
if (value == null) {
return null;
}

switch (type.typeId()) {
case DECIMAL:
return Decimal.apply((BigDecimal) value);
case UUID:
case STRING:
if (value instanceof Utf8) {
Utf8 utf8 = (Utf8) value;
return UTF8String.fromBytes(utf8.getBytes(), 0, utf8.getByteLength());
}
return UTF8String.fromString(value.toString());
case FIXED:
if (value instanceof byte[]) {
return value;
} else if (value instanceof GenericData.Fixed) {
return ((GenericData.Fixed) value).bytes();
}
return ByteBuffers.toByteArray((ByteBuffer) value);
case BINARY:
return ByteBuffers.toByteArray((ByteBuffer) value);
case STRUCT:
Types.StructType structType = (Types.StructType) type;

if (structType.fields().isEmpty()) {
return new GenericInternalRow();
}

List<Types.NestedField> fields = structType.fields();
Object[] values = new Object[fields.size()];
StructLike struct = (StructLike) value;

for (int index = 0; index < fields.size(); index++) {
Types.NestedField field = fields.get(index);
Type fieldType = field.type();
values[index] =
internalToSpark(fieldType, struct.get(index, fieldType.typeId().javaClass()));
}

return new GenericInternalRow(values);
default:
}

return value;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,16 +37,28 @@
import org.apache.iceberg.types.Types;
import org.apache.spark.sql.catalyst.InternalRow;

/**
* @deprecated will be removed in 1.8.0; use SparkPlannedAvroReader instead.
*/
@Deprecated
public class SparkAvroReader implements DatumReader<InternalRow>, SupportsRowPosition {

private final Schema readSchema;
private final ValueReader<InternalRow> reader;
private Schema fileSchema = null;

/**
* @deprecated will be removed in 1.8.0; use SparkPlannedAvroReader instead.
*/
@Deprecated
public SparkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) {
this(expectedSchema, readSchema, ImmutableMap.of());
}

/**
* @deprecated will be removed in 1.8.0; use SparkPlannedAvroReader instead.
*/
@Deprecated
@SuppressWarnings("unchecked")
public SparkAvroReader(
org.apache.iceberg.Schema expectedSchema, Schema readSchema, Map<Integer, ?> constants) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.spark.SparkUtil;
import org.apache.iceberg.types.Type.TypeID;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.util.UUIDUtil;
Expand Down Expand Up @@ -165,6 +166,7 @@ public ParquetValueReader<?> struct(
int defaultMaxDefinitionLevel = type.getMaxDefinitionLevel(currentPath());
for (Types.NestedField field : expectedFields) {
int id = field.fieldId();
ParquetValueReader<?> reader = readersById.get(id);
if (idToConstant.containsKey(id)) {
// containsKey is used because the constant may be null
int fieldMaxDefinitionLevel =
Expand All @@ -178,15 +180,21 @@ public ParquetValueReader<?> struct(
} else if (id == MetadataColumns.IS_DELETED.fieldId()) {
reorderedFields.add(ParquetValueReaders.constant(false));
types.add(null);
} else if (reader != null) {
reorderedFields.add(reader);
types.add(typesById.get(id));
} else if (field.initialDefault() != null) {
reorderedFields.add(
ParquetValueReaders.constant(
SparkUtil.internalToSpark(field.type(), field.initialDefault()),
maxDefinitionLevelsById.getOrDefault(id, defaultMaxDefinitionLevel)));
types.add(typesById.get(id));
} else if (field.isOptional()) {
reorderedFields.add(ParquetValueReaders.nulls());
types.add(null);
} else {
ParquetValueReader<?> reader = readersById.get(id);
if (reader != null) {
reorderedFields.add(reader);
types.add(typesById.get(id));
} else {
reorderedFields.add(ParquetValueReaders.nulls());
types.add(null);
}
throw new IllegalArgumentException(
String.format("Missing required field: %s", field.name()));
}
}

Expand Down Expand Up @@ -250,7 +258,7 @@ public ParquetValueReader<?> primitive(
if (expected != null && expected.typeId() == Types.LongType.get().typeId()) {
return new IntAsLongReader(desc);
} else {
return new UnboxedReader(desc);
return new UnboxedReader<>(desc);
}
case DATE:
case INT_64:
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,192 @@
/*
* 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.spark.data;

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.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.spark.SparkUtil;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;
import org.apache.iceberg.util.Pair;
import org.apache.spark.sql.catalyst.InternalRow;

public class SparkPlannedAvroReader implements DatumReader<InternalRow>, SupportsRowPosition {

private final Types.StructType expectedType;
private final Map<Integer, ?> idToConstant;
private ValueReader<InternalRow> reader;

public static SparkPlannedAvroReader create(org.apache.iceberg.Schema schema) {
return create(schema, ImmutableMap.of());
}

public static SparkPlannedAvroReader create(
org.apache.iceberg.Schema schema, Map<Integer, ?> constants) {
return new SparkPlannedAvroReader(schema, constants);
}

private SparkPlannedAvroReader(
org.apache.iceberg.Schema expectedSchema, Map<Integer, ?> constants) {
this.expectedType = expectedSchema.asStruct();
this.idToConstant = constants;
}

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

@Override
public InternalRow read(InternalRow 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, SparkUtil::internalToSpark);

// TODO: should this pass expected so that struct.get can reuse containers?
return SparkValueReaders.struct(readPlan, expected.fields().size());
}

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

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

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

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

@Override
public ValueReader<?> primitive(Type partner, Schema primitive) {
LogicalType logicalType = primitive.getLogicalType();
if (logicalType != null) {
switch (logicalType.getName()) {
case "date":
// Spark uses the same representation
return ValueReaders.ints();

case "timestamp-millis":
// adjust to microseconds
ValueReader<Long> longs = ValueReaders.longs();
return (ValueReader<Long>) (decoder, ignored) -> longs.read(decoder, null) * 1000L;

case "timestamp-micros":
// Spark uses the same representation
return ValueReaders.longs();

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

case "uuid":
return SparkValueReaders.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:
return SparkValueReaders.strings();
case FIXED:
return ValueReaders.fixed(primitive.getFixedSize());
case BYTES:
return ValueReaders.bytes();
case ENUM:
return SparkValueReaders.enums(primitive.getEnumSymbols());
default:
throw new IllegalArgumentException("Unsupported type: " + primitive);
}
}
}
}
Loading

0 comments on commit 5b13760

Please sign in to comment.