Skip to content
Merged
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
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@

package org.apache.iceberg.data.parquet;

import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.time.Instant;
import java.time.LocalDate;
import java.time.LocalDateTime;
Expand All @@ -28,6 +30,7 @@
import java.time.temporal.ChronoUnit;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.iceberg.Schema;
import org.apache.iceberg.parquet.ParquetSchemaUtil;
import org.apache.iceberg.parquet.ParquetValueReader;
Expand Down Expand Up @@ -299,6 +302,10 @@ public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveTy
case INT64:
case DOUBLE:
return new ParquetValueReaders.UnboxedReader<>(desc);
case INT96:
// Impala & Spark used to write timestamps as INT96 without a logical type. For backwards
// compatibility we try to read INT96 as timestamps.
return new TimestampInt96Reader(desc);
default:
throw new UnsupportedOperationException("Unsupported type: " + primitive);
}
Expand Down Expand Up @@ -345,6 +352,25 @@ public LocalDateTime read(LocalDateTime reuse) {
}
}

private static class TimestampInt96Reader extends ParquetValueReaders.PrimitiveReader<LocalDateTime> {
private static final long UNIX_EPOCH_JULIAN = 2_440_588L;

private TimestampInt96Reader(ColumnDescriptor desc) {
super(desc);
}

@Override
public LocalDateTime read(LocalDateTime reuse) {
final ByteBuffer byteBuffer = column.nextBinary().toByteBuffer().order(ByteOrder.LITTLE_ENDIAN);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Note for reviewers (and future me): toByteBuffer returns a duplicate of the internal buffer so that it is safe for uses of it to modify the buffer's position with methods like getLong.

final long timeOfDayNanos = byteBuffer.getLong();
final int julianDay = byteBuffer.getInt();

return Instant
.ofEpochMilli(TimeUnit.DAYS.toMillis(julianDay - UNIX_EPOCH_JULIAN))
.plusNanos(timeOfDayNanos).atOffset(ZoneOffset.UTC).toLocalDateTime();
}
}

private static class TimestamptzReader extends ParquetValueReaders.PrimitiveReader<OffsetDateTime> {
private TimestamptzReader(ColumnDescriptor desc) {
super(desc);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,13 @@ public Long next() {
return nextLong();
}
};
case INT96:
return (ColumnIterator<T>) new ColumnIterator<Binary>(desc, writerVersion) {
@Override
public Binary next() {
return nextBinary();
}
};
case FLOAT:
return (ColumnIterator<T>) new ColumnIterator<Float>(desc, writerVersion) {
@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,12 @@ public Long next() {
return nextLong();
}
};
case INT96:
return (PageIterator<T>) new PageIterator<Binary>(desc, writerVersion) {
public Binary next() {
return nextBinary();
}
};
case FLOAT:
return (PageIterator<T>) new PageIterator<Float>(desc, writerVersion) {
@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -681,6 +681,7 @@ private <E> Setter<I> newSetter(ParquetValueReader<E> reader, Type type) {
return (record, pos, ignored) -> setFloat(record, pos, unboxed.readFloat());
case DOUBLE:
return (record, pos, ignored) -> setDouble(record, pos, unboxed.readDouble());
case INT96:
case FIXED_LEN_BYTE_ARRAY:
case BINARY:
return (record, pos, ignored) -> set(record, pos, unboxed.readBinary());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,9 +22,11 @@
import java.math.BigDecimal;
import java.math.BigInteger;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.apache.iceberg.Schema;
import org.apache.iceberg.parquet.ParquetSchemaUtil;
import org.apache.iceberg.parquet.ParquetValueReader;
Expand Down Expand Up @@ -277,6 +279,10 @@ public ParquetValueReader<?> primitive(org.apache.iceberg.types.Type.PrimitiveTy
case INT64:
case DOUBLE:
return new UnboxedReader<>(desc);
case INT96:
// Impala & Spark used to write timestamps as INT96 without a logical type. For backwards
// compatibility we try to read INT96 as timestamps.
return new TimestampInt96Reader(desc);
default:
throw new UnsupportedOperationException("Unsupported type: " + primitive);
}
Expand Down Expand Up @@ -350,6 +356,29 @@ public long readLong() {
}
}

private static class TimestampInt96Reader extends UnboxedReader<Long> {
private static final long UNIX_EPOCH_JULIAN = 2_440_588L;

TimestampInt96Reader(ColumnDescriptor desc) {
super(desc);
}

@Override
public Long read(Long ignored) {
return readLong();
}

@Override
public long readLong() {
final ByteBuffer byteBuffer = column.nextBinary().toByteBuffer().order(ByteOrder.LITTLE_ENDIAN);
final long timeOfDayNanos = byteBuffer.getLong();
final int julianDay = byteBuffer.getInt();

return TimeUnit.DAYS.toMicros(julianDay - UNIX_EPOCH_JULIAN) +
TimeUnit.NANOSECONDS.toMicros(timeOfDayNanos);
}
}

private static class StringReader extends PrimitiveReader<UTF8String> {
StringReader(ColumnDescriptor desc) {
super(desc);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,19 +23,36 @@
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import org.apache.avro.generic.GenericData;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.Files;
import org.apache.iceberg.MetricsConfig;
import org.apache.iceberg.Schema;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.FileAppender;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.parquet.Parquet;
import org.apache.iceberg.parquet.ParquetWriteAdapter;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.types.TypeUtil;
import org.apache.iceberg.types.Types;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.api.WriteSupport;
import org.apache.parquet.hadoop.util.HadoopOutputFile;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.types.DataTypes;
import org.apache.spark.sql.types.Metadata;
import org.apache.spark.sql.types.StructField;
import org.apache.spark.sql.types.StructType;
import org.hamcrest.CoreMatchers;
import org.junit.Assert;
import org.junit.Assume;
import org.junit.Test;

import static org.apache.iceberg.spark.data.TestHelpers.assertEqualsUnsafe;
import static org.apache.iceberg.types.Types.NestedField.required;

public class TestSparkParquetReader extends AvroDataTest {
@Override
Expand Down Expand Up @@ -67,4 +84,76 @@ protected void writeAndValidate(Schema schema) throws IOException {
Assert.assertFalse("Should not have extra rows", rows.hasNext());
}
}

protected List<InternalRow> rowsFromFile(InputFile inputFile, Schema schema) throws IOException {
try (CloseableIterable<InternalRow> reader =
Parquet.read(inputFile)
.project(schema)
.createReaderFunc(type -> SparkParquetReaders.buildReader(schema, type))
.build()) {
return Lists.newArrayList(reader);
}
}

@Test
public void testInt96TimestampProducedBySparkIsReadCorrectly() throws IOException {
String outputFilePath = String.format("%s/%s", temp.getRoot().getAbsolutePath(), "parquet_int96.parquet");
HadoopOutputFile outputFile =
HadoopOutputFile.fromPath(
new org.apache.hadoop.fs.Path(outputFilePath), new Configuration());
Schema schema = new Schema(required(1, "ts", Types.TimestampType.withZone()));
StructType sparkSchema =
new StructType(
new StructField[] {
new StructField("ts", DataTypes.TimestampType, true, Metadata.empty())
});
List<InternalRow> rows = Lists.newArrayList(RandomData.generateSpark(schema, 10, 0L));

try (FileAppender<InternalRow> writer =
new ParquetWriteAdapter<>(
new NativeSparkWriterBuilder(outputFile)
.set("org.apache.spark.sql.parquet.row.attributes", sparkSchema.json())
.set("spark.sql.parquet.writeLegacyFormat", "false")
.set("spark.sql.parquet.outputTimestampType", "INT96")
.build(),
MetricsConfig.getDefault())) {
writer.addAll(rows);
}

List<InternalRow> readRows = rowsFromFile(Files.localInput(outputFilePath), schema);
Assert.assertEquals(rows.size(), readRows.size());
Assert.assertThat(readRows, CoreMatchers.is(rows));
}

/**
* Native Spark ParquetWriter.Builder implementation so that we can write timestamps using Spark's native
* ParquetWriteSupport.
*/
private static class NativeSparkWriterBuilder
extends ParquetWriter.Builder<InternalRow, NativeSparkWriterBuilder> {
private final Map<String, String> config = Maps.newHashMap();

NativeSparkWriterBuilder(org.apache.parquet.io.OutputFile path) {
super(path);
}

public NativeSparkWriterBuilder set(String property, String value) {
this.config.put(property, value);
return self();
}

@Override
protected NativeSparkWriterBuilder self() {
return this;
}

@Override
protected WriteSupport<InternalRow> getWriteSupport(Configuration configuration) {
for (Map.Entry<String, String> entry : config.entrySet()) {
configuration.set(entry.getKey(), entry.getValue());
}

return new org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport();
}
}
}