hadoop-pig-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Scott Carey (JIRA)" <j...@apache.org>
Subject [jira] Commented: (PIG-794) Use Avro serialization in Pig
Date Tue, 31 Aug 2010 16:59:57 GMT

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

Scott Carey commented on PIG-794:
---------------------------------

So a summary of the differences I can see quickly are:

h5. Schema usage:
This creates a 'generic' Avro schema that can be used for any pig data.  Each field in a Tuple
is a Union of all possible pig types, and each Tuple is a list of fields.  It does not preserve
the field names or types -- these are not important for intermediate data anyway.

AVRO-592 translates the Pig schema into a specific Avro schema that persists the field names
and types, so that:
STORE foo INTO 'file' USING AvroStorage();
Will create a file that
foo2 = LOAD 'file' USING AvroStorage(); 
will be able to re-create the exact schema for use in a script.

h5. Serialization and Deserialization:
This uses the same style as Avro's GenericRecord, which traverses the schema on the fly and
writes fields for each record.

AVRO-592 constructs a state machine for each specific schema to optimally traverse a Tuple
to serialize a record or create a Tuple when deserializing.  This should be faster but the
code is definitely harder to read (but easy to unit test -- AVRO-592 has 98% unit test code
coverage on that portion).


Integrating these should not be too hard.  I'll try and put my latest version of AVRO-592
up there late today or tomorrow.




> 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
>            Assignee: Dmitriy V. Ryaboy
>         Attachments: avro-0.1-dev-java_r765402.jar, AvroStorage.patch, AvroStorage_2.patch,
AvroStorage_3.patch, AvroTest.java, 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