You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by "Scott Carey (JIRA)" <ji...@apache.org> on 2010/07/06 21:07:50 UTC

[jira] Commented: (AVRO-592) Pig to Avro translation -- Pig DatumReader/Writer

    [ https://issues.apache.org/jira/browse/AVRO-592?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12885642#action_12885642 ] 

Scott Carey commented on AVRO-592:
----------------------------------

Previous discussion on the mailing list led me down several paths.

What I have so far needs some work in a couple areas and needs more testing, but It is in a good state to share the core.

Essentially, I wrote a DatumReader / DatumWriter pair for Pig.

My first attempt was to mimic GenericDatumReader/Writer, but there were a lot of branches in the code that were going to cause a more work for me to complete good unit testing coverage.  Plus, I wanted to use this as a place to try out a DatumReader/Writer pair that did not require traversing the avro Schema for every tuple.  

In some sense this is a prototype for a performance improvement to GenericDatumReader/Writer but Pig is simpler because there is no recursion or schema resolution.   I will have to learn the Parser better in order to leverage that to create something similar.

h3.  Schema Translation Design
The core classes where the tricky stuff happens are org.apache.avro.pig.PigDataAssembly and org.apache.avro.pig.SchemaTranslator.  

PigDataAssembly takes an Avro schema that represents Pig data, and builds a data structure that can read / write Tuples to avro.  Its really just a linked list of Action objects (private inner classes) for each type that can exist in a Tuple -- if this was C they would be structs with a function pointer and some misc data.  But its Java so they're private inner classes that override a method from a supertype.
Once this data structure is created, it can efficiently read data from a Decoder or write to an Encoder without parsing the schema.     However this makes it stateful and like a ResolvingDecoder, the initialization is not that cheap (it is still fairly fast).

SchemaTranslator is responsible for handling Pig Schemas (ResourceSchema) and converting these to Avro Schemas as well as validating that an Avro schema actually represents a Pig Tuple created by the same class.  The current restriction is that you can't read an arbitrary Avro record and make a Tuple out of it, even though the total number of possible Avro schemas that can be coerced into a Tuple is much larger than supported, I wanted to support that in a separate place.  This one is strict and constrained to reading and writing from pig -- although one can easily read data written by this with GenericDatumReader.
One of the primary goals was to be able to persist the Pig schema so that when the data was read by Pig, the loader would provide the same schema back that was stored originally.    To accomplish this the Schema translation makes a custom schema from the Pig schema, preserving field names and types.  The only exception is the value type of a Pig Map, which must be generic with no field names and is untyped from the POV of  a Pig schema.  There is a "GENERIC_ELEMENT" Schema that handles this, and corresponding Bag, and Tuple types to handle all types of map values.

h3. Integration with Hadoop / Pig
The next set of complexities are around Hadoop integration.  Pig 0.7+ requires using the org.apache.hadoop.mapreduce API, not the legacy mapred API.  So, I made org.apache.avro.mapreduce classes -- AvroInputFormat, AvroOutputFormat, and AvroRecordReader modeled mostly after the mapred ones.  However, these are limited to input and output -- not meant to be used for intermediate data between Map and Reduce.
Pig currently has several hoops you have to jump through to get a StoreFunc and LoadFunc to work correctly.  In particular, you can't easily read/write to the job configuration and only have a couple points of access to move information from the front-end to the back-end.
I think the right thing to do is to remove the org.apache.avro.mapreduce classes I have here and simply embed them in the only classes that use them: PigAvroInputFormat and PigAvroOutputFormat. That would significantly clean things up and prevent users from wondering which API to use for non-Pig mapreduce stuff.

h3.  Rough Edges
The unit test I've done hits the schema translation and serialization / deserialization well, but I have _not_ tested all of the Pig/Hadoop wrappers.

h3.  Other Potential Avro Changes / Additions
DatumWriter/Reader are problematic here.  Because the PigDataAssembly is stateful, it would be best to force a DatumWriter or DatumReader to take a Schema and Encoder/Decoder on the constructor and disallow setting them later.  In particular, changing the Schema is expensive.
The fact that we need to construct a DatumReader before creating a DataFileReader means that the schema has to be set later.  Perhaps we can delay construction by passing around a Factory rather than an already constructed object?  Then we can delay creation of the DatumReader/Writer until later when both the Schema and Decoder/Encoder are available.

As a more general concept, this is moving Schema resolution and Object serialization out as separate concerns from the Encoder / Decoder.  Right now, a DatumReader/Writer handles object serialization, but the Decoder/Encoder handles Schema resolution.  From a performance and code consolidation perspective, these two actions should be moved closer together -- they both require parsing Schemas and can be optimized together by doing:
[writer Schema] ->> [Shared Assembly Construction configured for a type -- Generic, Specific, Pig, Hive, etc] ->> SerializationAssembly
[reader Schema, actual Schema] ->> [Shared Assembly Construction for a type] ->> SerializationAssembly
The shared assembly stuff could leverage the parser, but I'm not knowledgeable enough with that to do that yet.

> Pig to Avro translation -- Pig DatumReader/Writer
> -------------------------------------------------
>
>                 Key: AVRO-592
>                 URL: https://issues.apache.org/jira/browse/AVRO-592
>             Project: Avro
>          Issue Type: New Feature
>          Components: java
>            Reporter: Scott Carey
>            Assignee: Scott Carey
>             Fix For: 1.4.0
>
>
> It would be great to use Avro to store Pig outputs.   Because Avro persists the schema as well, one can store data in one script, then load it in another and preserve the schema.
> Additionally, one can serialize pig Tuples to Avro and read Avro into pig Tuples.  Avro Schemas are significantly more rich than Pig schemas, but a limited translation is possible.

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