You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hive.apache.org by "David Maughan (JIRA)" <ji...@apache.org> on 2016/11/30 11:53:00 UTC

[jira] [Updated] (HIVE-15316) CTAS STORED AS AVRO: AvroTypeException Found default.record_0, expecting union

     [ https://issues.apache.org/jira/browse/HIVE-15316?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

David Maughan updated HIVE-15316:
---------------------------------
    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:

{code}
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
{code}

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"}}.

{code}
{
  "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
    }
  ]
}
{code}

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 a 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 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|https://issues.apache.org/jira/browse/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.

  was:
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:

{code}
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
{code}

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"}}.

{code}
{
  "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
    }
  ]
}
{code}

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 a expected a union, it is specifically expected 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 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|https://issues.apache.org/jira/browse/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.


> CTAS STORED AS AVRO: AvroTypeException Found default.record_0, expecting union
> ------------------------------------------------------------------------------
>
>                 Key: HIVE-15316
>                 URL: https://issues.apache.org/jira/browse/HIVE-15316
>             Project: Hive
>          Issue Type: Bug
>          Components: Hive
>    Affects Versions: 2.1.0
>            Reporter: David Maughan
>            Priority: Minor
>
> 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:
> {code}
> 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
> {code}
> 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"}}.
> {code}
> {
>   "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
>     }
>   ]
> }
> {code}
> 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 a 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 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|https://issues.apache.org/jira/browse/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.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)