hadoop-pig-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Doug Cutting (JIRA)" <j...@apache.org>
Subject [jira] Commented: (PIG-794) Use Avro serialization in Pig
Date Thu, 14 May 2009 22:04:45 GMT

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

Doug Cutting commented on PIG-794:
----------------------------------

Looking at the patch, I have a few questions and remarks:
 - Why not name the records "Tuple" and "Bag" instead of "T" and "B"?  The names are not written
in the data, so there's little advantage to shorter names.
 - Why not, instead of parsing the schema from Json, construct the schema using the Java Schema
API?  Then you would not need to walk the schema afterwards to find union indexes, and you'd
get compile-time API checking rather than potential load-time JSON parse errors.
 - Why not extend GenericDatumReader and override newRecord() to create either a Bag or a
Tuple, then override addField() to add values to either a bag or tuple?  This would make the
patch much smaller, and potentially permit you to eventually take advantage of GenericDatumReader
features like projection and object reuse.
 - Finally, since you're using a pre-release version of Avro, you should probably name the
jar with the subversion revision number.  Also note that, since Avro is not yet stable, it
should not be yet used for persistent data in production systems.


> Use Avro serialization in Pig
> -----------------------------
>
>                 Key: PIG-794
>                 URL: https://issues.apache.org/jira/browse/PIG-794
>             Project: Pig
>          Issue Type: Improvement
>          Components: impl
>    Affects Versions: 0.2.0
>            Reporter: Rakesh Setty
>             Fix For: 0.2.0
>
>         Attachments: avro-0.1-dev-java.jar, AvroStorage.patch, jackson-asl-0.9.4.jar,
PIG-794.patch
>
>
> We would like to use Avro serialization in Pig to pass data between MR jobs instead of
the current BinStorage. Attached is an implementation of AvroBinStorage which performs significantly
better compared to BinStorage on our benchmarks.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.


Mime
View raw message