Uploaded image for project: 'Hive'
  1. Hive
  2. HIVE-15316

CTAS STORED AS AVRO: AvroTypeException Found default.record_0, expecting union

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Minor
    • Resolution: Unresolved
    • 2.1.0
    • None
    • Hive

    Description

      There's an issue when querying a table that has been created as Avro via CTAS when the target struct is at least 2 struct-levels deep. It can be replicated with the following steps:

      CREATE TABLE a
        STORED AS AVRO
        AS
      SELECT named_struct('c', named_struct('d', 1)) as b;
      
      SELECT b FROM a;
      
      org.apache.avro.AvroTypeException: Found default.record_0, expecting union
      

      The reason for this is that during table creation, the Avro schema is generated from the Hive columns in AvroSerDe and then passed through the Avro Schema Parser: new Schema.Parser().parse(schema.toString()). For the above example, this creates the below schema in the Avro file. Note that the lowest level struct, record_0 has "namespace": "default".

      {
        "type": "record",
        "name": "a",
        "namespace": "default",
        "fields": [
          {
            "name": "b",
            "type": [
              "null",
              {
                "type": "record",
                "name": "record_1",
                "namespace": "",
                "doc": "struct<c:struct<d:int>>",
                "fields": [
                  {
                    "name": "c",
                    "type": [
                      "null",
                      {
                        "type": "record",
                        "name": "record_0",
                        "namespace": "default",
                        "doc": "struct<d:int>",
                        "fields": [
                          {
                            "name": "d",
                            "type": [ "null", "int" ],
                            "doc": "int",
                            "default": null
                          }
                        ]
                      }
                    ],
                    "doc": "struct<d:int>",
                    "default": null
                  }
                ]
              }
            ],
            "default": null
          }
        ]
      }
      

      On a subsequent select query, the Avro schema is again generated from the Hive columns. However, this time it is not passed through the Avro Schema Parser and the namespace attribute is not present in record_0. The actual Error message "Found default.record_0, expecting union" is slightly misleading. Although it is expecting a union, it is specifically expecting a null or a record named record_0 but it finds default.record_0.

      I believe this is a bug in Avro. I'm not sure whether the correct behaviour is to cascade the namespace down or not but it is definitely an inconsistency between creating a schema via the builders and parser. I've created AVRO-1965 for this. However, I believe that defensively passing the schema through the Avro Schema Parser on a select query would fix this issue in Hive without an Avro fix and version bump in Hive.

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              nahguam Dave Maughan
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated: