-
Notifications
You must be signed in to change notification settings - Fork 2.9k
🐛 fix Flink Read support for parquet int96 timestamps #3987
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
Closed
Closed
Changes from all commits
Commits
Show all changes
20 commits
Select commit
Hold shift + click to select a range
8b7cfb2
fix(flink parquet reader support int96): flink parquet reader support…
kingeasternsun be8b45f
Merge branch 'apache:master' into flink-read-int96
kingeasternsun a4ada94
:art: format code according Configuring Code Formatter for IntelliJ …
d886d80
:white_check_mark: finkv1.14 read parquet-int96 test pass
9371226
:white_check_mark: finkv1.14 read parquet-int96 test pass
1fee14e
:doc: add note
f24d0ff
:white_check_mark: flink v1.13 read parquet-int96 test pass
636c2c5
:white_check_mark: generate InternalRow directly in flink v1.13 read …
cc6ba8a
:fire: delete unused file RandomData.java that just used to test flin…
068163b
:art: checkstyle fix
0e7c819
:art: revert the code style of old code as before, keep the modificat…
a49fc8c
:art: undo the support of parquet-int96 flinkv1.13
85d6471
:recycle: add new method microsFromInt96 in DateTimeUtil for parquet…
7db1116
:recycle: add new method microsFromInt96 in DateTimeUtil for parquet…
02570a4
remove final in local
036fd64
:recycle: flink parquet read int96 as TimestampData ; add new test …
51c1ea4
:fire: remove unused import
150f800
:art: checkstyle fix
f29b145
:art: checkstyle fix
6610357
Merge branch 'master' into flink-read-int96
kingeasternsun File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -23,33 +23,57 @@ | |
| import java.io.File; | ||
| import java.io.IOException; | ||
| import java.nio.ByteBuffer; | ||
| import java.time.OffsetDateTime; | ||
| import java.util.ArrayList; | ||
| import java.util.Iterator; | ||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Random; | ||
| import org.apache.avro.generic.GenericData; | ||
| import org.apache.avro.generic.GenericRecord; | ||
| import org.apache.avro.generic.GenericRecordBuilder; | ||
| import org.apache.flink.table.data.GenericRowData; | ||
| import org.apache.flink.table.data.RowData; | ||
| import org.apache.flink.table.data.TimestampData; | ||
| import org.apache.flink.table.types.logical.LogicalType; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.apache.iceberg.Files; | ||
| import org.apache.iceberg.MetricsConfig; | ||
| import org.apache.iceberg.Schema; | ||
| import org.apache.iceberg.avro.AvroSchemaUtil; | ||
| import org.apache.iceberg.data.DataTest; | ||
| import org.apache.iceberg.data.RandomGenericData; | ||
| import org.apache.iceberg.data.Record; | ||
| import org.apache.iceberg.data.parquet.GenericParquetReaders; | ||
| import org.apache.iceberg.data.parquet.GenericParquetWriter; | ||
| import org.apache.iceberg.flink.FlinkSchemaUtil; | ||
| import org.apache.iceberg.flink.TestHelpers; | ||
| 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.Types; | ||
| import org.apache.iceberg.util.RandomUtil; | ||
| import org.apache.parquet.avro.AvroParquetWriter; | ||
| 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.catalyst.expressions.GenericInternalRow; | ||
| 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.junit.Assert; | ||
| import org.junit.Test; | ||
|
|
||
| import static org.apache.iceberg.types.Types.NestedField.optional; | ||
| import static org.apache.iceberg.types.Types.NestedField.required; | ||
|
|
||
| public class TestFlinkParquetReader extends DataTest { | ||
| private static final int NUM_RECORDS = 100; | ||
|
|
||
|
|
@@ -135,4 +159,106 @@ protected void writeAndValidate(Schema schema) throws IOException { | |
| RandomGenericData.generateFallbackRecords(schema, NUM_RECORDS, 21124, NUM_RECORDS / 20), | ||
| schema); | ||
| } | ||
|
|
||
| protected List<RowData> flinkReadRowDataFromFile(InputFile inputFile, Schema schema) throws IOException { | ||
| try (CloseableIterable<RowData> reader = | ||
| Parquet.read(inputFile) | ||
| .project(schema) | ||
| .createReaderFunc(type -> FlinkParquetReaders.buildReader(schema, type)) | ||
| .build()) { | ||
| return Lists.newArrayList(reader); | ||
| } | ||
| } | ||
|
|
||
| protected List<org.apache.iceberg.data.GenericRecord> genericReadRowDataFromFile(InputFile inputFile, | ||
| Schema schema) throws IOException { | ||
| try (CloseableIterable<org.apache.iceberg.data.GenericRecord> reader = | ||
| Parquet.read(inputFile) | ||
| .project(schema) | ||
| .createReaderFunc(type -> GenericParquetReaders.buildReader(schema, type)) | ||
| .build()) { | ||
| return Lists.newArrayList(reader); | ||
| } | ||
| } | ||
|
|
||
| @Test | ||
| public void testInt96TimestampProducedBySparkIsReadCorrectly() throws IOException { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I will suggest to write few rows by using the flink native writers, and then use the the following readers to assert the their results:
|
||
| 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.withoutZone())); | ||
| StructType sparkSchema = | ||
| new StructType( | ||
| new StructField[] { | ||
| new StructField("ts", DataTypes.TimestampType, true, Metadata.empty()) | ||
| }); | ||
|
|
||
| final Random random = new Random(0L); | ||
| List<InternalRow> rows = Lists.newArrayList(); | ||
| for (int i = 0; i < 10; i++) { | ||
| rows.add(new GenericInternalRow(new Object[] { | ||
| RandomUtil.generatePrimitive(schema.asStruct().fieldType("ts").asPrimitiveType(), random)})); | ||
| } | ||
|
|
||
| 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); | ||
| } | ||
|
|
||
| InputFile parquetInputFile = Files.localInput(outputFilePath); | ||
| List<RowData> flinkReadDataRows = flinkReadRowDataFromFile(parquetInputFile, schema); | ||
| List<org.apache.iceberg.data.GenericRecord> genericReadDataRows = genericReadRowDataFromFile(parquetInputFile, | ||
| schema); | ||
|
|
||
| Assert.assertEquals(rows.size(), flinkReadDataRows.size()); | ||
| Assert.assertEquals(rows.size(), genericReadDataRows.size()); | ||
| for (int i = 0; i < rows.size(); i += 1) { | ||
| TimestampData actual = ((TimestampData) ((GenericRowData) flinkReadDataRows.get(i)).getField(0)); | ||
| Assert.assertEquals( | ||
| rows.get(i).getLong(0), | ||
| actual.getMillisecond() * 1000 + actual.getNanoOfMillisecond() / 1000); | ||
|
|
||
| OffsetDateTime expect = ((OffsetDateTime) genericReadDataRows.get(i).getField("ts")); | ||
| Assert.assertTrue(expect.toLocalDateTime().equals(actual.toLocalDateTime())); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * 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(); | ||
| } | ||
| } | ||
| } | ||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Seems like this dependency was added for writing the timestamp as int96 in the unit test, but in fact we apache flink's ParquetRowDataWriter support writing a timestamp_with_local_time_zone into an INT96. So I will suggest to use the flink parquet writer rather than the spark parquet writer. (It's strange for me to introduce a spark module in in the flink module).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I actually prefer using the Spark module, unless Flink natively supports writing INT96 timestamps to Parquet. The benefit of using the Spark module is that support has been around for a long time and is relatively trusted to produce correct INT96 timestamp values.