Skip to content
Merged
Show file tree
Hide file tree
Changes from 2 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 @@ -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 @@ -21,21 +21,32 @@

import java.io.File;
import java.io.IOException;
import java.nio.file.Path;
import java.time.Instant;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import org.apache.avro.generic.GenericData;
import org.apache.iceberg.Files;
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.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.types.TypeUtil;
import org.apache.iceberg.types.Types;
import org.apache.spark.sql.Encoders;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.catalyst.InternalRow;
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.optional;

public class TestSparkParquetReader extends AvroDataTest {
@Override
Expand Down Expand Up @@ -67,4 +78,49 @@ 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 {
final SparkSession spark =
SparkSession.builder()
.master("local[2]")
.config("spark.sql.parquet.int96AsTimestamp", "false")
.getOrCreate();
Copy link
Contributor

Choose a reason for hiding this comment

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

Is it possible to avoid creating a Spark session just to write a timestamp? What about calling Spark's FileFormat to write directly instead?

We wrap Spark's FileFormat in our DSv2 table implementation: https://github.com/Netflix/iceberg/blob/netflix-spark-2.4/metacat/src/main/java/com/netflix/iceberg/batch/BatchPatternWrite.java#L90

This test would run much faster by using that to create a file instead of creating a Spark context.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, I would much rather avoid creating a SparkSession here if possible. However, looking into ParquetFileFormat it seems like we would still need to pass a SparkSession to create the writer.

I can look at ParquetOutputWriter but I might need to match the configuration there with what Spark uses to write int96.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Another approach would be to check-in a parquet file written by a spark and have the test just read it?

A drawback with that approach is that updating this file would be brittle, but I can check in the code that writes the file in an ignored test, but that should avoid us from creating a spark session during unit tests. What do you think @rdblue?

Copy link
Contributor

Choose a reason for hiding this comment

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

At one point, we supported writing to Parquet using Spark's built-in ReadSupport. I think we can probably get that working again to create the files.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, looking at one of the tests we do support writing parquet files using Spark's WriteSupport.

To be able to use a FileAppender I had to add a TimestampAsInt96 type (that can only be written using Spark's builtin WriteSupport) so that schema conversion within Iceberg's ParquetWriteSupport knows that this timestamps should be encoded as int96 in the parquet schema.


final String parquetPath = temp.getRoot().getAbsolutePath() + "/parquet_int96";
final java.sql.Timestamp ts = java.sql.Timestamp.valueOf("2014-01-01 23:00:01");
spark.createDataset(ImmutableList.of(ts), Encoders.TIMESTAMP()).write().parquet(parquetPath);
Copy link
Contributor

Choose a reason for hiding this comment

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

Using Spark's FileFormat would also make this test easier. You'd be able to pass in a value in micros and validate that you get the same value back, unmodified. You'd also not need to locate the Parquet file using find.

spark.stop();

// Get the single parquet file produced by spark.
List<Path> parquetOutputs =
java.nio.file.Files.find(
java.nio.file.Paths.get(parquetPath),
1,
(path, basicFileAttributes) -> path.toString().endsWith(".parquet"))
.collect(Collectors.toList());
Assert.assertEquals(1, parquetOutputs.size());

List<InternalRow> rows =
rowsFromFile(
Files.localInput(parquetOutputs.get(0).toFile()),
new Schema(optional(1, "timestamp", Types.TimestampType.withoutZone())));
Assert.assertEquals(1, rows.size());
Assert.assertEquals(1, rows.get(0).numFields());

// Spark represents Timestamps as epoch micros and are stored as longs.
Assert.assertEquals(
ts.toInstant(),
Instant.ofEpochMilli(TimeUnit.MICROSECONDS.toMillis(rows.get(0).getLong(0))));
}
}