This is an automated email from the ASF dual-hosted git repository.

blue pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new 7ecf672  Parquet: Fix NPE in value reader building for projections 
(#1164)
7ecf672 is described below

commit 7ecf6724a9a01462729e667b47e623b625a6caae
Author: Ryan Blue <[email protected]>
AuthorDate: Mon Jul 6 08:55:44 2020 -0700

    Parquet: Fix NPE in value reader building for projections (#1164)
    
    Co-authored-by: 范欣欣 <[email protected]>
---
 .../iceberg/data/parquet/BaseParquetReaders.java   | 37 ++++++++++++++++------
 .../apache/iceberg/data/TestReadProjection.java    |  5 ++-
 2 files changed, 32 insertions(+), 10 deletions(-)

diff --git 
a/data/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java 
b/data/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java
index 36d0144..651a688 100644
--- a/data/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java
+++ b/data/src/main/java/org/apache/iceberg/data/parquet/BaseParquetReaders.java
@@ -94,10 +94,13 @@ public abstract class BaseParquetReaders<T> {
       List<Type> types = 
Lists.newArrayListWithExpectedSize(fieldReaders.size());
       List<Type> fields = struct.getFields();
       for (int i = 0; i < fields.size(); i += 1) {
-        Type fieldType = fields.get(i);
-        int fieldD = type().getMaxDefinitionLevel(path(fieldType.getName())) - 
1;
-        newFields.add(ParquetValueReaders.option(fieldType, fieldD, 
fieldReaders.get(i)));
-        types.add(fieldType);
+        ParquetValueReader<?> fieldReader = fieldReaders.get(i);
+        if (fieldReader != null) {
+          Type fieldType = fields.get(i);
+          int fieldD = type().getMaxDefinitionLevel(path(fieldType.getName())) 
- 1;
+          newFields.add(ParquetValueReaders.option(fieldType, fieldD, 
fieldReader));
+          types.add(fieldType);
+        }
       }
 
       return createStructReader(types, newFields, expected);
@@ -127,11 +130,14 @@ public abstract class BaseParquetReaders<T> {
       Map<Integer, Type> typesById = Maps.newHashMap();
       List<Type> fields = struct.getFields();
       for (int i = 0; i < fields.size(); i += 1) {
-        Type fieldType = fields.get(i);
-        int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1;
-        int id = fieldType.getId().intValue();
-        readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, 
fieldReaders.get(i)));
-        typesById.put(id, fieldType);
+        ParquetValueReader<?> fieldReader = fieldReaders.get(i);
+        if (fieldReader != null) {
+          Type fieldType = fields.get(i);
+          int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 
1;
+          int id = fieldType.getId().intValue();
+          readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, 
fieldReader));
+          typesById.put(id, fieldType);
+        }
       }
 
       List<Types.NestedField> expectedFields = expected != null ?
@@ -163,6 +169,10 @@ public abstract class BaseParquetReaders<T> {
     @Override
     public ParquetValueReader<?> list(Types.ListType expectedList, GroupType 
array,
                                       ParquetValueReader<?> elementReader) {
+      if (expectedList == null) {
+        return null;
+      }
+
       GroupType repeated = array.getFields().get(0).asGroupType();
       String[] repeatedPath = currentPath();
 
@@ -180,6 +190,10 @@ public abstract class BaseParquetReaders<T> {
     public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map,
                                      ParquetValueReader<?> keyReader,
                                      ParquetValueReader<?> valueReader) {
+      if (expectedMap == null) {
+        return null;
+      }
+
       GroupType repeatedKeyValue = map.getFields().get(0).asGroupType();
       String[] repeatedPath = currentPath();
 
@@ -197,8 +211,13 @@ public abstract class BaseParquetReaders<T> {
     }
 
     @Override
+    @SuppressWarnings("checkstyle:CyclomaticComplexity")
     public ParquetValueReader<?> 
primitive(org.apache.iceberg.types.Type.PrimitiveType expected,
                                            PrimitiveType primitive) {
+      if (expected == null) {
+        return null;
+      }
+
       ColumnDescriptor desc = type.getColumnDescription(currentPath());
 
       if (primitive.getOriginalType() != null) {
diff --git a/data/src/test/java/org/apache/iceberg/data/TestReadProjection.java 
b/data/src/test/java/org/apache/iceberg/data/TestReadProjection.java
index 2dee8d1..1a1c599 100644
--- a/data/src/test/java/org/apache/iceberg/data/TestReadProjection.java
+++ b/data/src/test/java/org/apache/iceberg/data/TestReadProjection.java
@@ -20,6 +20,7 @@
 package org.apache.iceberg.data;
 
 import java.io.IOException;
+import java.time.OffsetDateTime;
 import java.util.List;
 import java.util.Map;
 import org.apache.iceberg.Schema;
@@ -192,12 +193,14 @@ public abstract class TestReadProjection {
   public void testBasicProjection() throws Exception {
     Schema writeSchema = new Schema(
         Types.NestedField.required(0, "id", Types.LongType.get()),
-        Types.NestedField.optional(1, "data", Types.StringType.get())
+        Types.NestedField.optional(1, "data", Types.StringType.get()),
+        Types.NestedField.optional(2, "time", Types.TimestampType.withZone())
     );
 
     Record record = GenericRecord.create(writeSchema.asStruct());
     record.setField("id", 34L);
     record.setField("data", "test");
+    record.setField("time", OffsetDateTime.now());
 
     Schema idOnly = new Schema(
         Types.NestedField.required(0, "id", Types.LongType.get())

Reply via email to