-
Notifications
You must be signed in to change notification settings - Fork 3k
Align the records written by GenericOrcWriter and SparkOrcWriter #1271
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
Merged
Merged
Changes from all commits
Commits
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
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
148 changes: 148 additions & 0 deletions
148
spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java
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 |
|---|---|---|
| @@ -0,0 +1,148 @@ | ||
| /* | ||
| * 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.File; | ||
| import java.io.IOException; | ||
| import java.math.BigDecimal; | ||
| import java.util.Iterator; | ||
| import java.util.List; | ||
| import org.apache.iceberg.Files; | ||
| import org.apache.iceberg.Schema; | ||
| import org.apache.iceberg.data.GenericRecord; | ||
| import org.apache.iceberg.data.RandomGenericData; | ||
| import org.apache.iceberg.data.Record; | ||
| import org.apache.iceberg.data.orc.GenericOrcReader; | ||
| import org.apache.iceberg.data.orc.GenericOrcWriter; | ||
| import org.apache.iceberg.io.CloseableIterable; | ||
| import org.apache.iceberg.io.FileAppender; | ||
| import org.apache.iceberg.orc.ORC; | ||
| import org.apache.iceberg.relocated.com.google.common.collect.Lists; | ||
| import org.apache.iceberg.types.Types; | ||
| import org.apache.spark.sql.catalyst.InternalRow; | ||
| import org.junit.Assert; | ||
| import org.junit.Test; | ||
|
|
||
| import static org.apache.iceberg.types.Types.NestedField.required; | ||
|
|
||
| public class TestSparkRecordOrcReaderWriter extends AvroDataTest { | ||
| private static final int NUM_RECORDS = 200; | ||
|
|
||
| private void writeAndValidate(Schema schema, List<Record> expectedRecords) throws IOException { | ||
| final File originalFile = temp.newFile(); | ||
| Assert.assertTrue("Delete should succeed", originalFile.delete()); | ||
|
|
||
| // Write few generic records into the original test file. | ||
| try (FileAppender<Record> writer = ORC.write(Files.localOutput(originalFile)) | ||
| .createWriterFunc(GenericOrcWriter::buildWriter) | ||
| .schema(schema) | ||
| .build()) { | ||
| writer.addAll(expectedRecords); | ||
| } | ||
|
|
||
| // Read into spark InternalRow from the original test file. | ||
| List<InternalRow> internalRows = Lists.newArrayList(); | ||
| try (CloseableIterable<InternalRow> reader = ORC.read(Files.localInput(originalFile)) | ||
| .project(schema) | ||
| .createReaderFunc(readOrcSchema -> new SparkOrcReader(schema, readOrcSchema)) | ||
| .build()) { | ||
| reader.forEach(internalRows::add); | ||
| assertEqualsUnsafe(schema.asStruct(), expectedRecords, reader, expectedRecords.size()); | ||
| } | ||
|
|
||
| final File anotherFile = temp.newFile(); | ||
| Assert.assertTrue("Delete should succeed", anotherFile.delete()); | ||
|
|
||
| // Write those spark InternalRows into a new file again. | ||
| try (FileAppender<InternalRow> writer = ORC.write(Files.localOutput(anotherFile)) | ||
| .createWriterFunc(SparkOrcWriter::new) | ||
| .schema(schema) | ||
| .build()) { | ||
| writer.addAll(internalRows); | ||
| } | ||
|
|
||
| // Check whether the InternalRows are expected records. | ||
| try (CloseableIterable<InternalRow> reader = ORC.read(Files.localInput(anotherFile)) | ||
| .project(schema) | ||
| .createReaderFunc(readOrcSchema -> new SparkOrcReader(schema, readOrcSchema)) | ||
| .build()) { | ||
| assertEqualsUnsafe(schema.asStruct(), expectedRecords, reader, expectedRecords.size()); | ||
| } | ||
|
|
||
| // Read into iceberg GenericRecord and check again. | ||
| try (CloseableIterable<Record> reader = ORC.read(Files.localInput(anotherFile)) | ||
| .createReaderFunc(typeDesc -> GenericOrcReader.buildReader(schema, typeDesc)) | ||
| .project(schema) | ||
| .build()) { | ||
| assertRecordEquals(expectedRecords, reader, expectedRecords.size()); | ||
| } | ||
| } | ||
|
|
||
| @Override | ||
| protected void writeAndValidate(Schema schema) throws IOException { | ||
| List<Record> expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1992L); | ||
| writeAndValidate(schema, expectedRecords); | ||
| } | ||
|
|
||
| @Test | ||
| public void testDecimalWithTrailingZero() throws IOException { | ||
| Schema schema = new Schema( | ||
| required(1, "d1", Types.DecimalType.of(10, 2)), | ||
| required(2, "d2", Types.DecimalType.of(20, 5)), | ||
| required(3, "d3", Types.DecimalType.of(38, 20)) | ||
| ); | ||
|
|
||
| List<Record> expected = Lists.newArrayList(); | ||
|
|
||
| GenericRecord record = GenericRecord.create(schema); | ||
| record.set(0, new BigDecimal("101.00")); | ||
| record.set(1, new BigDecimal("10.00E-3")); | ||
| record.set(2, new BigDecimal("1001.0000E-16")); | ||
|
|
||
| expected.add(record.copy()); | ||
|
|
||
| writeAndValidate(schema, expected); | ||
| } | ||
|
|
||
| private static void assertRecordEquals(Iterable<Record> expected, Iterable<Record> actual, int size) { | ||
| Iterator<Record> expectedIter = expected.iterator(); | ||
| Iterator<Record> actualIter = actual.iterator(); | ||
| for (int i = 0; i < size; i += 1) { | ||
| Assert.assertTrue("Expected iterator should have more rows", expectedIter.hasNext()); | ||
| Assert.assertTrue("Actual iterator should have more rows", actualIter.hasNext()); | ||
| Assert.assertEquals("Should have same rows.", expectedIter.next(), actualIter.next()); | ||
| } | ||
| Assert.assertFalse("Expected iterator should not have any extra rows.", expectedIter.hasNext()); | ||
| Assert.assertFalse("Actual iterator should not have any extra rows.", actualIter.hasNext()); | ||
| } | ||
|
|
||
| private static void assertEqualsUnsafe(Types.StructType struct, Iterable<Record> expected, | ||
| Iterable<InternalRow> actual, int size) { | ||
| Iterator<Record> expectedIter = expected.iterator(); | ||
| Iterator<InternalRow> actualIter = actual.iterator(); | ||
| for (int i = 0; i < size; i += 1) { | ||
| Assert.assertTrue("Expected iterator should have more rows", expectedIter.hasNext()); | ||
| Assert.assertTrue("Actual iterator should have more rows", actualIter.hasNext()); | ||
| GenericsHelpers.assertEqualsUnsafe(struct, expectedIter.next(), actualIter.next()); | ||
| } | ||
| Assert.assertFalse("Expected iterator should not have any extra rows.", expectedIter.hasNext()); | ||
| Assert.assertFalse("Actual iterator should not have any extra rows.", actualIter.hasNext()); | ||
| } | ||
| } |
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.
I'm not sure we need to check the precision either. If we read a value, then we should return it, right?
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.
It is necessary to do this check. we need to make sure that there's no bug when written a decimal into ORC. For example, for decimal(3, 0) data type we encounter a hive decimal
10000(whose precision is 5), that should be something wrong. Throwing an exception is the correct way in that case.