Michael Heuer created PARQUET-1441:
--------------------------------------

             Summary: SchemaParseException: Can't redefine: list in 
AvroIndexedRecordConverter
                 Key: PARQUET-1441
                 URL: https://issues.apache.org/jira/browse/PARQUET-1441
             Project: Parquet
          Issue Type: Bug
          Components: parquet-avro
            Reporter: Michael Heuer


The following unit test added to TestAvroSchemaConverter fails
{code:java}
@Test
public void testConvertedSchemaToStringCantRedefineList() throws Exception {
  String parquet = "message spark_schema {\n" +
      "  optional group annotation {\n" +
      "    optional group transcriptEffects (LIST) {\n" +
      "      repeated group list {\n" +
      "        optional group element {\n" +
      "          optional group effects (LIST) {\n" +
      "            repeated group list {\n" +
      "              optional binary element (UTF8);\n" +
      "            }\n" +
      "          }\n" +
      "        }\n" +
      "      }\n" +
      "    }\n" +
      "  }\n" +
      "}\n";

  Configuration conf = new Configuration(false);
  AvroSchemaConverter avroSchemaConverter = new AvroSchemaConverter(conf);
  Schema schema = 
avroSchemaConverter.convert(MessageTypeParser.parseMessageType(parquet));
  schema.toString();
}
{code}

while this one succeeds
{code:java}
@Test
public void testConvertedSchemaToStringCantRedefineList() throws Exception {
  String parquet = "message spark_schema {\n" +
      "  optional group annotation {\n" +
      "    optional group transcriptEffects (LIST) {\n" +
      "      repeated group list {\n" +
      "        optional group element {\n" +
      "          optional group effects (LIST) {\n" +
      "            repeated group list {\n" +
      "              optional binary element (UTF8);\n" +
      "            }\n" +
      "          }\n" +
      "        }\n" +
      "      }\n" +
      "    }\n" +
      "  }\n" +
      "}\n";
 
  Configuration conf = new Configuration(false);
  conf.setBoolean("parquet.avro.add-list-element-records", false);
  AvroSchemaConverter avroSchemaConverter = new AvroSchemaConverter(conf);
  Schema schema = 
avroSchemaConverter.convert(MessageTypeParser.parseMessageType(parquet));
  schema.toString();
}
{code}

I don't see a way to influence the code path in AvroIndexedRecordConverter to 
respect this configuration, resulting in the following stack trace downstream
{noformat}
  Cause: org.apache.avro.SchemaParseException: Can't redefine: list
  at org.apache.avro.Schema$Names.put(Schema.java:1128)
  at org.apache.avro.Schema$NamedSchema.writeNameRef(Schema.java:562)
  at org.apache.avro.Schema$RecordSchema.toJson(Schema.java:690)
  at org.apache.avro.Schema$ArraySchema.toJson(Schema.java:805)
  at org.apache.avro.Schema$UnionSchema.toJson(Schema.java:882)
  at org.apache.avro.Schema$RecordSchema.fieldsToJson(Schema.java:716)
  at org.apache.avro.Schema$RecordSchema.toJson(Schema.java:701)
  at org.apache.avro.Schema$UnionSchema.toJson(Schema.java:882)
  at org.apache.avro.Schema$RecordSchema.fieldsToJson(Schema.java:716)
  at org.apache.avro.Schema$RecordSchema.toJson(Schema.java:701)
  at org.apache.avro.Schema.toString(Schema.java:324)
  at 
org.apache.avro.SchemaCompatibility.checkReaderWriterCompatibility(SchemaCompatibility.java:68)
  at 
org.apache.parquet.avro.AvroRecordConverter.isElementType(AvroRecordConverter.java:866)
  at 
org.apache.parquet.avro.AvroIndexedRecordConverter$AvroArrayConverter.<init>(AvroIndexedRecordConverter.java:333)
  at 
org.apache.parquet.avro.AvroIndexedRecordConverter.newConverter(AvroIndexedRecordConverter.java:172)
  at 
org.apache.parquet.avro.AvroIndexedRecordConverter.<init>(AvroIndexedRecordConverter.java:94)
  at 
org.apache.parquet.avro.AvroIndexedRecordConverter.newConverter(AvroIndexedRecordConverter.java:168)
  at 
org.apache.parquet.avro.AvroIndexedRecordConverter.<init>(AvroIndexedRecordConverter.java:94)
  at 
org.apache.parquet.avro.AvroIndexedRecordConverter.<init>(AvroIndexedRecordConverter.java:66)
  at 
org.apache.parquet.avro.AvroCompatRecordMaterializer.<init>(AvroCompatRecordMaterializer.java:34)
  at 
org.apache.parquet.avro.AvroReadSupport.newCompatMaterializer(AvroReadSupport.java:144)
  at 
org.apache.parquet.avro.AvroReadSupport.prepareForRead(AvroReadSupport.java:136)
  at 
org.apache.parquet.hadoop.InternalParquetRecordReader.initialize(InternalParquetRecordReader.java:204)
  at 
org.apache.parquet.hadoop.ParquetRecordReader.initializeInternalReader(ParquetRecordReader.java:182)
  at 
org.apache.parquet.hadoop.ParquetRecordReader.initialize(ParquetRecordReader.java:140)
...
{noformat}

See also downstream issues
https://issues.apache.org/jira/browse/SPARK-25588
[https://github.com/bigdatagenomics/adam/issues/2058]



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to