[ https://issues.apache.org/jira/browse/PARQUET-1879?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17598759#comment-17598759 ]
Daniel Dai commented on PARQUET-1879: ------------------------------------- This seems to be a backward-incompatible change. We cannot read parquet file created pre-1.11.1 using the new version. Here is a same error message: {code:java} org.apache.parquet.io.InvalidRecordException: key_value not found in optional group canonicals (MAP) { repeated group map (MAP_KEY_VALUE) { required binary key (ENUM); optional group value { optional int32 index; optional int64 pinId; optional group indexableTextIndexes (LIST) { repeated int32 indexableTextIndexes_tuple; } optional int32 indexExpLq; optional int32 indexExp; optional boolean imageOnly; optional boolean link404; optional boolean unsafe; optional boolean imageNotOnPage; optional boolean linkStatusError; } } } at org.apache.parquet.schema.GroupType.getFieldIndex(GroupType.java:176) at org.apache.parquet.schema.GroupType.getType(GroupType.java:208) at org.apache.parquet.schema.GroupType.checkGroupContains(GroupType.java:348) at org.apache.parquet.schema.GroupType.checkContains(GroupType.java:339) at org.apache.parquet.schema.GroupType.checkGroupContains(GroupType.java:349) at org.apache.parquet.schema.MessageType.checkContains(MessageType.java:124) at org.apache.parquet.hadoop.api.ReadSupport.getSchemaForRead(ReadSupport.java:56) at org.apache.parquet.hadoop.thrift.ThriftReadSupport.init(ThriftReadSupport.java:187) at org.apache.parquet.hadoop.InternalParquetRecordReader.initialize(InternalParquetRecordReader.java:200) at org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:182) at org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140) at org.apache.spark.rdd.NewHadoopRDD$$anon$1.liftedTree1$1(NewHadoopRDD.scala:216) at org.apache.spark.rdd.NewHadoopRDD$$anon$1.<init>(NewHadoopRDD.scala:213) at org.apache.spark.rdd.NewHadoopRDD.compute(NewHadoopRDD.scala:168) at org.apache.spark.rdd.NewHadoopRDD.compute(NewHadoopRDD.scala:71) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:373) at org.apache.spark.rdd.RDD.iterator(RDD.scala:337) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90) at org.apache.spark.scheduler.Task.run(Task.scala:131) at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:506) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1462) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:509) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) {code} I am not sure what's the best way to fix it. I am thinking about adding a walker in the construct of FileMetaData to fix the schema, is it a good idea? > Apache Arrow can not read a Parquet File written with Parqet-Avro 1.11.0 with > a Map field > ----------------------------------------------------------------------------------------- > > Key: PARQUET-1879 > URL: https://issues.apache.org/jira/browse/PARQUET-1879 > Project: Parquet > Issue Type: Bug > Components: parquet-avro, parquet-format > Affects Versions: 1.11.0 > Reporter: Matthew McMahon > Assignee: Matthew McMahon > Priority: Critical > Fix For: 1.12.0, 1.11.1 > > > From my > [StackOverflow|https://stackoverflow.com/questions/62504757/issue-with-loading-parquet-data-into-snowflake-cloud-database-when-written-with] > in relation to an issue I'm having with getting Snowflake (Cloud DB) to load > Parquet files written with version 1.11.0 > ---- > The problem only appears when using a map schema field in the Avro schema. > For example: > {code:java} > { > "name": "FeatureAmounts", > "type": { > "type": "map", > "values": "records.MoneyDecimal" > } > } > {code} > When using Parquet-Avro to write the file, a bad Parquet schema ends up with, > for example > {code:java} > message record.ResponseRecord { > required binary GroupId (STRING); > required int64 EntryTime (TIMESTAMP(MILLIS,true)); > required int64 HandlingDuration; > required binary Id (STRING); > optional binary ResponseId (STRING); > required binary RequestId (STRING); > optional fixed_len_byte_array(12) CostInUSD (DECIMAL(28,15)); > required group FeatureAmounts (MAP) { > repeated group map (MAP_KEY_VALUE) { > required binary key (STRING); > required fixed_len_byte_array(12) value (DECIMAL(28,15)); > } > } > } > {code} > From the great answer to my StackOverflow, it seems the issue is that the > 1.11.0 Parquet-Avro is still using the legacy MAP_KEY_VALUE converted type, > that has no logical type equivalent. From the comment on > [LogicalTypeAnnotation|https://github.com/apache/parquet-mr/blob/84c954d8a4feef2d9bdad7a236a7268ef71a1c25/parquet-column/src/main/java/org/apache/parquet/schema/LogicalTypeAnnotation.java#L904] > {code:java} > // This logical type annotation is implemented to support backward > compatibility with ConvertedType. > // The new logical type representation in parquet-format doesn't have any > key-value type, > // thus this annotation is mapped to UNKNOWN. This type shouldn't be used. > {code} > However, it seems this is being written with the latest 1.11.0, which then > causes Apache Arrow to fail with > {code:java} > Logical type Null can not be applied to group node > {code} > As it appears that > [Arrow|https://github.com/apache/arrow/blob/master/cpp/src/parquet/types.cc#L629-L632] > only looks for the new logical type of Map or List, therefore this causes an > error. > I have seen in Parquet Formats that > [LogicalTypes|https://github.com/apache/parquet-format/blob/master/LogicalTypes.md] > should be something like > {code:java} > // Map<String, Integer> > required group my_map (MAP) { > repeated group key_value { > required binary key (UTF8); > optional int32 value; > } > } > {code} > Is this on the correct path? -- This message was sent by Atlassian Jira (v8.20.10#820010)