You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/12/17 19:26:30 UTC

[GitHub] [iceberg] jackye1995 commented on a change in pull request #3723: Fix Iceberg's parquet reader returning nulls incorrectly for parquet files written by writers that don't use list and element as names.

jackye1995 commented on a change in pull request #3723:
URL: https://github.com/apache/iceberg/pull/3723#discussion_r771629679



##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java
##########
@@ -88,6 +96,31 @@ public Type primitive(PrimitiveType primitive) {
     return field == null ? primitive : primitive.withId(field.id());
   }
 
+  @Override
+  public void beforeElementField(Type element) {
+    super.beforeElementField(makeElement(element));
+  }
+
+  @Override
+  public void afterElementField(Type element) {
+    super.afterElementField(makeElement(element));
+  }
+
+  private Type makeElement(Type element) {
+    // List's element in 3-level lists can be named differently across different parquet writers.
+    // For example, hive names it "array_element", whereas new parquet writers names it as "element".
+    if (element.getName().equals("element") || element.isPrimitive()) {

Review comment:
       nit: static variable for "element"

##########
File path: site/docs/spark-procedures.md
##########
@@ -334,6 +334,10 @@ CALL catalog_name.system.rewrite_manifests('db.sample', false)
 
 The `snapshot` and `migrate` procedures help test and migrate existing Hive or Spark tables to Iceberg.
 
+**Note** Parquet files written with Parquet writers that use names other than `list` and `element` for repeated group
+and element of the list respectively are **read incorrectly** by Iceberg upto 0.12.1 Iceberg versions. Parquet files
+generated by Hive fall in this category.

Review comment:
       use `!!! Note` for note.
   
   Also "Parquet files generated by Hive" sounds a bit vague. I think we can specify example cases are Parquet files written using (1) `org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe` in Hive, or (2) `spark.sql.parquet.writeLegacyFormat=true` in Spark.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org