diff --git orc/src/java/org/apache/orc/impl/SchemaEvolution.java orc/src/java/org/apache/orc/impl/SchemaEvolution.java index 7379de9..1f2409e 100644 --- orc/src/java/org/apache/orc/impl/SchemaEvolution.java +++ orc/src/java/org/apache/orc/impl/SchemaEvolution.java @@ -191,8 +191,9 @@ private boolean validatePPDConversion(final TypeDescription fileType, void buildConversionFileTypesArray(TypeDescription fileType, TypeDescription readerType) { - // if the column isn't included, don't map it - if (included != null && !included[readerType.getId()]) { + // if the column isn't included or missing, don't map it + if (included != null && readerType.getId() < included.length + && !included[readerType.getId()]) { return; } boolean isOk = true; diff --git orc/src/test/org/apache/orc/impl/TestSchemaEvolution.java orc/src/test/org/apache/orc/impl/TestSchemaEvolution.java index c28af94..1cd1401 100644 --- orc/src/test/org/apache/orc/impl/TestSchemaEvolution.java +++ orc/src/test/org/apache/orc/impl/TestSchemaEvolution.java @@ -43,7 +43,7 @@ @Rule public TestName testCaseName = new TestName(); - + Configuration conf; Path testFilePath; FileSystem fs; @@ -466,4 +466,30 @@ public void testSafePpdEvaluationForStrings() throws IOException { assertFalse(schemaEvolution.isPPDSafeConversion(0)); assertFalse(schemaEvolution.isPPDSafeConversion(1)); } + + @Test + public void testStructFieldChange() throws IOException { + TypeDescription nestedFileStruct1 = TypeDescription.createStruct() + .addField("f1", TypeDescription.createInt()) + .addField("f3", TypeDescription.createDecimal().withPrecision(38).withScale(10)); + TypeDescription fileStruct1 = TypeDescription.createStruct() + .addField("row1", nestedFileStruct1) + .addField("row2", TypeDescription.createInt()); + TypeDescription nestedReaderStruct1 = TypeDescription.createStruct() + .addField("f1", TypeDescription.createInt()) + .addField("f2", TypeDescription.createString()) + .addField("f3", TypeDescription.createDecimal().withPrecision(38).withScale(10)); + TypeDescription readerStruct1 = TypeDescription.createStruct() + .addField("row1", nestedReaderStruct1) + .addField("row2", TypeDescription.createInt()); + + SchemaEvolution both1 = new SchemaEvolution(fileStruct1, readerStruct1, null); + assertTrue(both1.hasConversion()); + + // one item per root row + boolean[] included2 = {true, true, true, true, true}; + SchemaEvolution both2 = new SchemaEvolution(fileStruct1, readerStruct1, included2); + assertTrue(both2.hasConversion()); + assertTrue(both1.isPPDSafeConversion(5)); + } }