Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Hive fails to insert optional struct into avro backed table. #20

Open
IllyaYalovyy opened this issue May 21, 2012 · 11 comments
Open

Hive fails to insert optional struct into avro backed table. #20

IllyaYalovyy opened this issue May 21, 2012 · 11 comments

Comments

@IllyaYalovyy
Copy link

  • Environment:
    CDH3U3, avro-1.5.4 and avro-mapred-1.5.4, haivvreo-1.0.7

  • Avro schema:

    
    {
    "namespace":"com.expedia.edw.gco.navigator",
    "name":"item",
    "type":"record",
    "fields":[
      {"name":"a","type":["null","string"]},
      {"name":"b","type":["null","int"]},
      {"name":"c", "type": ["null",{"type":"record", "name":"c_t", "fields":[{"name":"col1","type":["null","string"]}]}]}
    ]
    }
    
  • Query:

    
    insert overwrite table illya_avro_2 select "asd", "sdf", struct("dfg") from default.dual;
    
  • Exception:

    2012-05-20 21:11:12,821 WARN org.apache.hadoop.mapred.Child: Error running child
    java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row {"dummy":"1"}
    at org.apache.hadoop.hive.ql.exec.ExecMapper.map(ExecMapper.java:161)
    at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:50)
    at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:391)
    at org.apache.hadoop.mapred.MapTask.run(MapTask.java:325)
    at org.apache.hadoop.mapred.Child$4.run(Child.java:270)
    at java.security.AccessController.doPrivileged(Native Method)
    at javax.security.auth.Subject.doAs(Subject.java:396)
    at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1157)
    at org.apache.hadoop.mapred.Child.main(Child.java:264)
    Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row {"dummy":"1"}
    at org.apache.hadoop.hive.ql.exec.MapOperator.process(MapOperator.java:546)
    at org.apache.hadoop.hive.ql.exec.ExecMapper.map(ExecMapper.java:143)
    ... 8 more
    Caused by: org.apache.avro.AvroRuntimeException: Not a record: ["null",{"type":"record","name":"c_t","namespace":"com.expedia.edw.gco.navigator","fields":[{"name":"col1","type":["null","string"]}]}]
    at org.apache.avro.Schema.getFields(Schema.java:227)
    at com.linkedin.haivvreo.AvroSerializer.serializeStruct(AvroSerializer.java:104)
    at com.linkedin.haivvreo.AvroSerializer.serialize(AvroSerializer.java:97)
    at com.linkedin.haivvreo.AvroSerializer.serialize(AvroSerializer.java:65)
    at com.linkedin.haivvreo.AvroSerDe.serialize(AvroSerDe.java:75)
    at org.apache.hadoop.hive.ql.exec.FileSinkOperator.processOp(FileSinkOperator.java:553)
    at org.apache.hadoop.hive.ql.exec.Operator.process(Operator.java:471)
    at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:744)
    at org.apache.hadoop.hive.ql.exec.SelectOperator.processOp(SelectOperator.java:84)
    at org.apache.hadoop.hive.ql.exec.Operator.process(Operator.java:471)
    at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:744)
    at org.apache.hadoop.hive.ql.exec.SelectOperator.processOp(SelectOperator.java:84)
    at org.apache.hadoop.hive.ql.exec.Operator.process(Operator.java:471)
    at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:744)
    at org.apache.hadoop.hive.ql.exec.TableScanOperator.processOp(TableScanOperator.java:78)
    at org.apache.hadoop.hive.ql.exec.Operator.process(Operator.java:471)
    at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:744)
    at org.apache.hadoop.hive.ql.exec.MapOperator.process(MapOperator.java:527)
    ... 9 more
    
@jghoman
Copy link
Owner

jghoman commented Jun 2, 2012

@IllyaYalovyy I'm playing with this to see what's up. Stay tuned.

@IllyaYalovyy
Copy link
Author

As a workaround I replaced optional records with mandatory records that are populated with NULLs. This approach leads to higher space consumption and performance degradation, but it works.

@houckman
Copy link

houckman commented Aug 8, 2012

Hey... I am running into this (or a similar) issue as well... and have posted an issue on the Cloudera customer support portal. Was there any resolution to this?? Thanks!!!

@IllyaYalovyy
Copy link
Author

Sure. Send me more details on your problem, and I hope I can help you.

@houckman
Copy link

Thanks... I have tried to create an example as close as this initial issue as possible. My schema looks like this:

{
"name":"item",
"type":"record",
"fields":[
{"name":"a","type":"string"},
{"name":"b","type":"int"},
{"name":"c", "type": [{"type":"record", "name":"subitem", "fields":[{"name":"d","type":"string"}]}, "null"], "default": "null"}
]
}

I create two tables (mytable1 and mytable2) using this schema as follows....

CREATE external TABLE mytable1
ROW FORMAT SERDE
'com.linkedin.haivvreo.AvroSerDe'
WITH SERDEPROPERTIES (
'schema.url'='hdfs:///avro/schema/mytable.avsc')
STORED AS INPUTFORMAT
'com.linkedin.haivvreo.AvroContainerInputFormat'
OUTPUTFORMAT
'com.linkedin.haivvreo.AvroContainerOutputFormat'
LOCATION
'/avro/data/mytable1';

Describing one of the tables shows....

a string from deserializer
b int from deserializer
c struct<d:string> from deserializer

I assume here that since hive generally allows nullable, and my avro schema is a union with null, that it more-or-less ignores the union from a hive perspective.

I am given a file using the avro schema provided (I can also create this file from a java app).

Then I run a 'insert overwrite table mytable2 select * from mytable1;

2012-08-13 10:11:56,223 WARN org.apache.hadoop.mapred.Child: Error running child
java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row {"a":"a", "b": "1", "c":{"d":"d"}}
at org.apache.hadoop.hive.ql.exec.ExecMapper.map(ExecMapper.java:161)
at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:50)
at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:391)
at org.apache.hadoop.mapred.MapTask.run(MapTask.java:325)
at org.apache.hadoop.mapred.Child$4.run(Child.java:270)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:396)
at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1157)
at org.apache.hadoop.mapred.Child.main(Child.java:264)
Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row {"a":"a", "b": "1", "c":{"d":"d"}}
at org.apache.hadoop.hive.ql.exec.MapOperator.process(MapOperator.java:546)
at org.apache.hadoop.hive.ql.exec.ExecMapper.map(ExecMapper.java:143)
... 8 more
Caused by: org.apache.avro.AvroRuntimeException: Not a record: [{"type":"record", "name":"subitem", "fields":[{"name":"d","type":"string"}]}, "null"]
at org.apache.avro.Schema.getFields(Schema.java:227)
at com.linkedin.haivvreo.AvroSerializer.serializeStruct(AvroSerializer.java:104)
at com.linkedin.haivvreo.AvroSerializer.serialize(AvroSerializer.java:97)
at com.linkedin.haivvreo.AvroSerializer.serialize(AvroSerializer.java:65)
at com.linkedin.haivvreo.AvroSerDe.serialize(AvroSerDe.java:75)
at org.apache.hadoop.hive.ql.exec.FileSinkOperator.processOp(FileSinkOperator.java:553)
at org.apache.hadoop.hive.ql.exec.Operator.process(Operator.java:471)
at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:744)
at org.apache.hadoop.hive.ql.exec.SelectOperator.processOp(SelectOperator.java:84)
at org.apache.hadoop.hive.ql.exec.Operator.process(Operator.java:471)
at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:744)
at org.apache.hadoop.hive.ql.exec.SelectOperator.processOp(SelectOperator.java:84)
at org.apache.hadoop.hive.ql.exec.Operator.process(Operator.java:471)
at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:744)
at org.apache.hadoop.hive.ql.exec.TableScanOperator.processOp(TableScanOperator.java:78)
at org.apache.hadoop.hive.ql.exec.Operator.process(Operator.java:471)
at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:744)
at org.apache.hadoop.hive.ql.exec.MapOperator.process(MapOperator.java:527)
... 9 more

I can write a MR job that does this without error. Like you found out, if I change the subrecord to not be a union, but just a record, it works fine with haivvreo.. that may be an option longer term.. but not for now.. I am going to download the haivvreo code tonight and see if I can figure this out (more for my own edification that anything). I was looking at com.linkedin.haivvreo.AvroSerializer (specifically, the serialize() method). I noticed that it uses the TypeInfo instance to determine the hive type, and it resolves to a "struct". The Avro schema type, however, is a union for this field. So, I thought I would redirect it to the serializeUnion() method if the Schema is a type union. Not sure if I am on the right path... but thought it would be fun to try.

I appreciate any thoughts you might have.

Thanks!!!!!

@mischuh
Copy link

mischuh commented Sep 17, 2012

Hi,

i am facing the same problems with unions and records.

  • Environment
    CHD 4.0.1, avro-1.5.4 and avro-mapred-1.5.4, haivvreo-1.0.12-avro15.jar
  • Avro backed table
CREATE TABLE my_table 
  ROW FORMAT SERDE
  'com.linkedin.haivvreo.AvroSerDe'
  WITH SERDEPROPERTIES (
    'schema.literal'='{
      "type": "record",
      "name": "record_test",
      "namespace": "com.test.haivvreo.record",
      "fields": [
        {
          "name": "test_record_1",
          "type": [{
            "type": "record",
            "name": "record_1",
            "fields": [
                {
                  "name": "double_value",
                  "type": "double"
                },
                {
                  "name": "string_value",
                  "type": "string"
                },
                {
                  "name": "array_values",
                  "type": {
                    "type": "array",
                    "items": "string"
                  }
                }
              ]
            }, "null" ]
          },
          {
            "name": "test_record_2",
            "type": [{
              "type": "record",
              "name": "record_2",
              "fields": [
                  {
                    "name": "double_value",
                    "type": "double"
                  },
                  {
                    "name": "string_value",
                    "type": "string"
                  },
                  {
                    "name": "array_values",
                    "type": {
                      "type": "array",
                      "items": "string"
                    }
                  }
                ]
            }]
          }
        ]
      }'
    )
  STORED AS INPUTFORMAT
  'com.linkedin.haivvreo.AvroContainerInputFormat'
  OUTPUTFORMAT
  'com.linkedin.haivvreo.AvroContainerOutputFormat';
  • Describe on table leads to following output
desc my_table;                                      
OK
col_name    data_type   comment
test_record_1   struct<double_value:double,string_value:string,array_values:array<string>>  from deserializer
test_record_2   uniontype<struct<double_value:double,string_value:string,array_values:array<string>>>   from deserializer

As you can see if I have defined test_record_1 as an union type in the schema as it should be [record, null] and
have omitted the "null" value for the second item test_record_2.
The strange thing is that test_record_1 is a struct type in the table definition which I have thought has to be a uniontype
while test_record_2, which should be syntactically incorrect, leads to an uniontype with only one entry (struct).

If I am inserting data into the test_record_1 column I am getting the error "AvroRuntimeException: Not a record:..." as @IllyaYalovyy mentioned above.
This error occures in the reduce part of the inserting job.

I am receiving following error in the mapping phase if I am trying to insert data into test_record_2:

java.io.IOException: org.apache.avro.AvroTypeException: Found "null", expecting [ {
    [{
      "type": "record",
      "name": "record_2",
      "namespace": "com.test.haivvreo.record",
      "fields": [
          {
            "name": "double_value",
            "type": "double"
          },
          {
            "name": "string_value",
            "type": "string"
          },
          {
            "name": "array_values",
            "type": {
              "type": "array",
              "items": "string"
            }
          }
        ]
    }]
} ]

What's wrong here?
Any help or hints are welcome.

@houckman
Copy link

I am not sure this is the same issue I ran into... but the one was having
trouble with is incompatibility between an avro record and a Hive struct.
Assuming you are interested, read on!!!

I believe this to be a bug in the haivvreo/avro code. When I was trying to
recreated it, I was able to do so using both haivvreo with CDH3u4 and with
CDH3u5 (which has native Hive/Avro support). I started looking into it and
found that the code is really the very similar (which is good). Using the
CDH3u5 code, I determined the following:

The actual AvroRuntimeException occurs on line 227 of
org.apache.avro.Schema.getFields()... and again the error indicates that
the code is looking for a record, but finding something else. The error is
"Caused by: org.apache.avro.AvroRuntimeException: Not a record:
[{"type":"record", "name":"subitem",
"fields":[{"name":"d","type":"string"}]}, "null"] ".

So, I started reviewing the code that occurs before this, and ran into a
switch statement in
org.apache.hadoop.hive.serde2.avro.AvroSerializer.serialize().. line 81.
Here the code is checking the "hive" type and is returning a "STRUCT",
which make sense. However, there is also a Schema object here and a
schema.getType() shows that Avro thinks this field is a "UNION", which also
makes sense. These, however, an incompatible. I read somewhere (cannot
remember where) that hive takes an Avro UNION and, if it is a UNION with
null (which mine is), it really just makes that a nullable field of the
regular type ("STRUCT") in my case. So, doing that makes sense, Hive sees
it as a "STRUCT" and Avro sees it as a "UNION". Again, this is incompatible
because Avro is not aware of that little rule I just mentioned.. and still
expects a "UNION". So, all I did what add a check at the beginning of the
serializeStruct() method.

if (schema.getType() == Schema.Type.UNION) {
schema = getBaseType(schema);
}

... and I added the method getBaseType(Schema), which I copied from some
Avro code somewhere, which does this...

private Schema getBaseType(Schema schema) {
Schema retSchema = schema;
if (schema.getType == Schema.Type.UNION) {
for (Schema subSchema : schema.getTypes()) {
if (subSchema.getType() == Schema.Type.NULL) {
continue;
}
retSchema = getBaseSchema(subSchema);
} else {
retSchema = schema;
}
return retSchema;
}

This all worked for me. I contacted Cloudera and explained the issue to
them... they said they would open a ticket on it, but have not heard back
on that yet.

On Mon, Sep 17, 2012 at 5:21 AM, mischuh [email protected] wrote:

Hi,

i am facing the same problems with unions and records.

Environment
CHD 4.0.1, avro-1.5.4 and avro-mapred-1.5.4, haivvreo-1.0.12-avro15.jar

Avro backed table

CREATE TABLE my_table ROW FORMAT SERDE'com.linkedin.haivvreo.AvroSerDe'WITH SERDEPROPERTIES (
'schema.literal'='{ "type": "record", "name": "record_test", "namespace": "com.test.haivvreo.record", "fields": [ { "name": "test_record_1", "type": [{ "type": "record", "name": "record_1", "fields": [ { "name": "double_value", "type": "double" }, { "name": "string_value", "type": "string" }, { "name": "array_values", "type": { "type": "array", "items": "string" } } ] }, "null" ] }, { "name": "test_record_2", "type": [{ "type": "record", "name": "record_2", "fields": [ { "name": "double_value", "type": "double" }, { "name": "string_value", "type": "string" }, { "name": "array_values", "type": { "type": "array", "items": "string" } } ] }] } ] }'
)STORED AS INPUTFORMAT'com.linkedin.haivvreo.AvroContainerInputFormat'OUTPUTFORMAT'com.linkedin.haivvreo.AvroContainerOutputFormat';

-

Describe on table leads to following output

desc my_table;
OK
col_name data_type comment
test_record_1 struct<double_value:double,string_value:string,array_values:array> from deserializer
test_record_2 uniontype<struct<double_value:double,string_value:string,array_values:array>> from deserializer

As you can see if I have defined test_record_1 as an union type in the
schema as it should be [record, null] and
have omitted the "null" value for the second item test_record_2.
The strange thing is that test_record_1 is a struct type in the table
definition which I have thought has to be a uniontype
while test_record_2, which should be syntactically incorrect, leads to an
uniontype with only one entry (struct).

If I am inserting data into the test_record_1 column I am getting the
error "AvroRuntimeException: Not a record:..." as @IllyaYalovyyhttps://github.com/IllyaYalovyymentioned above.
This error occures in the reduce part of the inserting job.

I am receiving following error in the mapping phase if I am trying to
insert data into test_record_2:

java.io.IOException: org.apache.avro.AvroTypeException: Found "null", expecting [ {
[{
"type": "record",
"name": "record_2",
"namespace": "com.test.haivvreo.record",
"fields": [
{
"name": "double_value",
"type": "double"
},
{
"name": "string_value",
"type": "string"
},
{
"name": "array_values",
"type": {
"type": "array",
"items": "string"
}
}
]
}]
} ]

What's wrong here?
Any help or hints are welcome.


Reply to this email directly or view it on GitHubhttps://github.com//issues/20#issuecomment-8608481.

Doug Houck
[email protected]
(571) 207-5863

"I swear by my life and my love of it that I will never live for the sake
of another man, nor ask another man to live for mine." -- John Galt

@busbey
Copy link

busbey commented Feb 14, 2013

FYI, this was fixed in the integrated-into-hive Avro support in HIVE-3528.

@IllyaYalovyy
Copy link
Author

Thanks! It was really helpful! Unfortunately we stuck with CDH4 (Hive 0.9)

@jghoman
Copy link
Owner

jghoman commented Feb 14, 2013

@busbey Any chance this got merged back to CDH's fork of Haivvreo? I'd be happy to merge it here.

@busbey
Copy link

busbey commented Feb 22, 2013

@jghoman I don't think it's been merged into any part of CDH yet. I'll likely merge it into CDH4's Hive, but that'll still be the built in Avro SerDe and not the Haivvreo support from CDH3.

@IllyaYalovyy are you likely to follow further CDH4 releases, or is there some particular reason you're staying with Haivvreo on CDH4 instead of the builtin SerDe? I believe CDH4 with Hive 0.9 means you're atleast on CDH 4.1.0 and the built in version has been available since then -- see the "New Feature" section.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

No branches or pull requests

5 participants