You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@pig.apache.org by "Anyi Li (JIRA)" <ji...@apache.org> on 2017/03/01 03:53:45 UTC

[jira] [Commented] (PIG-5115) Builtin AvroStorage generates incorrect avro schema when the same pig field name appears in the alias

    [ https://issues.apache.org/jira/browse/PIG-5115?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15889453#comment-15889453 ] 

Anyi Li commented on PIG-5115:
------------------------------

I do agree this patch will change the previous output while using builtin AvroStorage. However, if you like the pig schema I described, we did see the problem, it will throw exception and fail the whole job although the passed-in pig schema was valid. There may be a better way to fix that but I think it will the similar thing by generating unique avro schema field names for different avro types. However, if you look at the example I gave above, the duplicated field name which causes the problems are from the wrapper of avro records, which is invisible for the next step even you load it again, you will only refer to the field names on the avro record types rather than the outer wrapper. I think if you have concerns on break ingthe old output. I could make a switch, so we can switch on or off.

 

> Builtin AvroStorage generates incorrect avro schema when the same pig field name appears in the alias
> -----------------------------------------------------------------------------------------------------
>
>                 Key: PIG-5115
>                 URL: https://issues.apache.org/jira/browse/PIG-5115
>             Project: Pig
>          Issue Type: Bug
>    Affects Versions: 0.17.0
>            Reporter: Anyi Li
>            Assignee: Anyi Li
>             Fix For: 0.17.0
>
>         Attachments: PIG-5115.patch
>
>
> Pig ResourceSchema allows to use same field names but different types when they are not in the same level. The pig schema like
> {quote}
> data: {col1: (col2: (col1_data: chararray)),col2: {col2: (col2_data: chararray)}}
> {quote}
> Although _col2_ has been redefined, they are not appeared in the same level, it is a totally valid pig schema. 
> However, once it is translated by AvroStorage, it will throw exception 
> {noformat}
> Can't redefine: col2
>         at org.apache.pig.newplan.logical.visitor.InputOutputFileValidatorVisitor.visit(InputOutputFileValidatorVisitor.java:64)
>         at org.apache.pig.newplan.logical.relational.LOStore.accept(LOStore.java:66)
>         at org.apache.pig.newplan.DepthFirstWalker.depthFirst(DepthFirstWalker.java:64)
>         at org.apache.pig.newplan.DepthFirstWalker.depthFirst(DepthFirstWalker.java:66)
>         at org.apache.pig.newplan.DepthFirstWalker.depthFirst(DepthFirstWalker.java:66)
>         at org.apache.pig.newplan.DepthFirstWalker.walk(DepthFirstWalker.java:53)
>         at org.apache.pig.newplan.PlanVisitor.visit(PlanVisitor.java:52)
>         at org.apache.pig.newplan.logical.relational.LogicalPlan.validate(LogicalPlan.java:212)
>         at org.apache.pig.PigServer$Graph.compile(PigServer.java:1767)
>         at org.apache.pig.PigServer$Graph.access$300(PigServer.java:1443)
>         at org.apache.pig.PigServer.execute(PigServer.java:1356)
>         at org.apache.pig.PigServer.executeBatch(PigServer.java:415)
>         at org.apache.pig.PigServer.executeBatch(PigServer.java:398)
>         at org.apache.pig.tools.grunt.GruntParser.executeBatch(GruntParser.java:171)
>         at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:234)
>         at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:205)
>         at org.apache.pig.tools.grunt.Grunt.exec(Grunt.java:81)
>         at org.apache.pig.Main.run(Main.java:631)
>         at org.apache.pig.Main.main(Main.java:177)
> Caused by: org.apache.avro.SchemaParseException: Can't redefine: col2
>         at org.apache.avro.Schema$Names.put(Schema.java:1042)
>         at org.apache.avro.Schema$NamedSchema.writeNameRef(Schema.java:511)
>         at org.apache.avro.Schema$RecordSchema.toJson(Schema.java:626)
>         at org.apache.avro.Schema$ArraySchema.toJson(Schema.java:737)
>         at org.apache.avro.Schema$UnionSchema.toJson(Schema.java:814)
>         at org.apache.avro.Schema$RecordSchema.fieldsToJson(Schema.java:648)
>         at org.apache.avro.Schema$RecordSchema.toJson(Schema.java:635)
>         at org.apache.avro.Schema.toString(Schema.java:297)
>         at org.apache.avro.Schema.toString(Schema.java:287)
>         at org.apache.pig.builtin.AvroStorage.setOutputAvroSchema(AvroStorage.java:442)
>         at org.apache.pig.builtin.AvroStorage.checkSchema(AvroStorage.java:433)
>         at org.apache.pig.newplan.logical.visitor.InputOutputFileValidatorVisitor.visit(InputOutputFileValidatorVisitor.java:54)
>         ... 18 more
> {noformat}
> It is caused by a bug in AvroStorageSchemaConversionUtilities class which uses tuple name as GenericRecord name as well as the fieldname that wraps the record. 
> So it would like to  produces the avro schema like the following 
> {noformat}
> {
>   "type": "record",
>   "name": "data",
>   "fields": [
>     {
>       "name": "col1",
>       "type": {
>         "type": "record",
>         "name": "col1_1",
>         "fields": [
>           {
>             "name": "col2",
>             "type": {
>               "type": "record",
>               "name": "col2",
>               "fields": [
>                 {
>                   "name": "col1_data",
>                   "type": "string"
>                 }
>               ]
>             }
>           }
>         ]
>       }
>     },
>     {
>       "name": "col2",
>       "type": {
>         "type": "array",
>         "items": {
>           "type": "record",
>           "name": "col2",
>           "fields": [
>             {
>               "name": "col2_data",
>               "type": "string"
>             }
>           ]
>         }
>       }
>     }
>   ]
> }
> {noformat}
> But according to the avro 1.7.7  specs ([https://avro.apache.org/docs/1.7.7/spec.html#Names]), _col2_ has been defined as record and redefined as array later, it is an invalid schema, unless the fullname (namespace + name) is unique. 
> Since AvroStorageSchemaConversionUtilities will generate avro record if the pig schema is a tuple, we need a way to generate unique _recordName_. 
> {code}
> public static Schema resourceSchemaToAvroSchema(final ResourceSchema rs,
>       String recordName, final String recordNameSpace,
>       final Map<String, List<Schema>> definedRecordNames,
>       final Boolean doubleColonsToDoubleUnderscores) throws IOException {
>     if (rs == null) {
>       return null;
>     }
>     recordName = toAvroName(recordName, doubleColonsToDoubleUnderscores);
>     List<Schema.Field> fields = new ArrayList<Schema.Field>();
>     Schema newSchema = Schema.createRecord(
>             recordName, null, recordNameSpace, false);
> {code}
> The AvroStorage class from piggybank solved this problem by defining a static method and generate unique _recordName_. We can implement the similar method for the builtin AvroStorage 
>  



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)