Uploaded image for project: 'Parquet'
  1. Parquet
  2. PARQUET-952

Avro union with single type fails with 'is not a group'

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 1.11.0
    • parquet-mr
    • None

    Description

      When one uses Avro schema with a union that has only one type specified, the AvroParquetWriter throws an exception. See the following repro test case:

        @Test
        public void reproCase() throws Exception {
          System.out.println("Parquet version: " + Version.FULL_VERSION);
      
          // Schema with a single field 'value' with type of union that have a single item (=string)
          Schema avroSchema = Schema.parse("{" +
            "\"type\": \"record\", " +
            "\"name\": \"RandomRecord\", " +
            "\"fields\": [" +
            "{\"name\": \"value\", \"type\": [\"string\"] }" +
            "]" +
            "}");
      
          // Parquet writer
          ParquetWriter parquetWriter = AvroParquetWriter.builder(path).withSchema(avroSchema)
            .withConf(new Configuration())
            .build();
      
          GenericRecord record = new GenericRecordBuilder(avroSchema)
            .set("value", "Surprise!")
            .build();
      
          parquetWriter.write(record);
        }
      

      Will result in:

      Parquet version: parquet-mr version 1.9.0 (build 38262e2c80015d0935dad20f8e18f2d6f9fbd03c)
      
      java.lang.ClassCastException: required binary value (UTF8) is not a group
      
      	at org.apache.parquet.schema.Type.asGroupType(Type.java:202)
      	at org.apache.parquet.avro.AvroWriteSupport.writeValueWithoutConversion(AvroWriteSupport.java:357)
      	at org.apache.parquet.avro.AvroWriteSupport.writeValue(AvroWriteSupport.java:274)
      	at org.apache.parquet.avro.AvroWriteSupport.writeRecordFields(AvroWriteSupport.java:187)
      	at org.apache.parquet.avro.AvroWriteSupport.write(AvroWriteSupport.java:161)
      	at org.apache.parquet.hadoop.InternalParquetRecordWriter.write(InternalParquetRecordWriter.java:123)
      	at org.apache.parquet.hadoop.ParquetWriter.write(ParquetWriter.java:292)
      	at net.jarcec.AvroParquet.reproCase(AvroParquet.java:49)
      	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
      	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
      	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      	at java.lang.reflect.Method.invoke(Method.java:498)
      	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
      	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
      	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
      	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
      	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
      	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
      	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
      	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
      	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
      	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
      	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
      	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
      	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
      	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
      	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
      	at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
      	at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:68)
      	at com.intellij.rt.execution.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:51)
      	at com.intellij.rt.execution.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:237)
      	at com.intellij.rt.execution.junit.JUnitStarter.main(JUnitStarter.java:70)
      

      I'm attaching a small maven project with all the dependencies to make it easier to reproduce locally.

      Trying to isolate the problem further, it seems that the AvroSchemaConverter converts the Avro schema to just required binary value (UTF8); (e.g. primitive type). But then the writer will go based on the Avro schema (which is a union) and tries to call asGroupType() on the primite type.

      Attachments

        1. PARQUET-952-repro.tar.gz
          13 kB
          Jarek Jarcec Cecho

        Issue Links

          Activity

            People

              nkollar Nándor Kollár
              jarcec Jarek Jarcec Cecho
              Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: