From 44d218165f1261924347100746505d4347e519eb Mon Sep 17 00:00:00 2001 From: Limian Zhang <75445426+rzhang10@users.noreply.github.com> Date: Mon, 12 Dec 2022 16:33:42 -0700 Subject: [PATCH] Enhance unit test --- .../data/TestSparkOrcWriteWithoutIds.java | 32 +++++++++++++++---- 1 file changed, 25 insertions(+), 7 deletions(-) diff --git a/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcWriteWithoutIds.java b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcWriteWithoutIds.java index 337bd7261..1088cd0c3 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcWriteWithoutIds.java +++ b/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcWriteWithoutIds.java @@ -21,10 +21,12 @@ import java.io.File; import java.io.IOException; +import java.util.Iterator; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; 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.orc.ORC; import org.apache.iceberg.orc.ORCSchemaUtil; @@ -38,6 +40,7 @@ import org.junit.rules.TemporaryFolder; import static org.apache.iceberg.TableProperties.ORC_DO_NOT_WRITE_FIELD_IDS; +import static org.apache.iceberg.spark.data.TestHelpers.assertEquals; import static org.apache.iceberg.types.Types.NestedField.optional; public class TestSparkOrcWriteWithoutIds { @@ -56,17 +59,32 @@ public void testWriteWithoutIds() throws IOException { Assert.assertTrue("Delete should succeed", testFile.delete()); Iterable rows = RandomData.generateSpark(SCHEMA, 1, 0L); - FileAppender writer = ORC.write(Files.localOutput(testFile)) + try (FileAppender writer = ORC.write(Files.localOutput(testFile)) .config(ORC_DO_NOT_WRITE_FIELD_IDS, String.valueOf(true)) .createWriterFunc(SparkOrcWriter::new) .schema(SCHEMA) - .build(); + .build()) { + writer.addAll(rows); + } - writer.addAll(rows); - writer.close(); + // read back using ORC native file reader and test file schema + try (Reader fileReader = OrcFile.createReader(new Path(testFile.getPath()), + new OrcFile.ReaderOptions(new Configuration()))) { + Assert.assertFalse(ORCSchemaUtil.hasIds(fileReader.getSchema())); + } - // read back using ORC native file reader - Reader reader = OrcFile.createReader(new Path(testFile.getPath()), new OrcFile.ReaderOptions(new Configuration())); - Assert.assertFalse(ORCSchemaUtil.hasIds(reader.getSchema())); + // read back using Spark ORC reader and test overall readability + try (CloseableIterable reader = ORC.read(Files.localInput(testFile)) + .project(SCHEMA) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(SCHEMA, readOrcSchema)) + .build()) { + final Iterator actualRows = reader.iterator(); + final Iterator expectedRows = rows.iterator(); + while (expectedRows.hasNext()) { + Assert.assertTrue("Should have expected number of rows", actualRows.hasNext()); + assertEquals(SCHEMA, expectedRows.next(), actualRows.next()); + } + Assert.assertFalse("Should not have extra rows", actualRows.hasNext()); + } } }