Skip to content

Commit

Permalink
Enhance unit test
Browse files Browse the repository at this point in the history
  • Loading branch information
rzhang10 committed Dec 12, 2022
1 parent 7970ace commit 44d2181
Showing 1 changed file with 25 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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 {
Expand All @@ -56,17 +59,32 @@ public void testWriteWithoutIds() throws IOException {
Assert.assertTrue("Delete should succeed", testFile.delete());

Iterable<InternalRow> rows = RandomData.generateSpark(SCHEMA, 1, 0L);
FileAppender<InternalRow> writer = ORC.write(Files.localOutput(testFile))
try (FileAppender<InternalRow> 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<InternalRow> reader = ORC.read(Files.localInput(testFile))
.project(SCHEMA)
.createReaderFunc(readOrcSchema -> new SparkOrcReader(SCHEMA, readOrcSchema))
.build()) {
final Iterator<InternalRow> actualRows = reader.iterator();
final Iterator<InternalRow> 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());
}
}
}

0 comments on commit 44d2181

Please sign in to comment.