On May 14, 2010, at 12:18 PM, Doug Cutting wrote:

> On 05/14/2010 11:50 AM, Scott Carey wrote:
>> It doesn't cost on the serialization size but currently it costs a lot on 
>> the performance side.
> 
> Not necessarily. For Pig data in Java I think one might reasonably write 
> a custom reader and writer that reads and writes Pig data structures 
> directly.  This could look something like readSchema, writeSchema, 
> readJson and writeJson methods in the patch for AVRO-251.
> 
> https://issues.apache.org/jira/browse/AVRO-251
> 
> Start by looking at json.avsc (an Avro schema for arbitrary JSON data) 
> then see the readJson() method.  It directly reads Avro data 
> corresponding to that schema into a Jackson JsonNode.  ResolvingDecoder 
> enforces the schema.

This has to work through Hadoop via an InputFormat / OutputFormat 0.20 apis, 
and the o.a.h.mapreduce API implementations I have made closely resemble those 
you made for the old o.a.h.mapred api.  These currently require using the 
Generic or Specific API, though one could make one that took a Pig Tuple 
directly instead of an avro object.

Longer term, I think we can get the Specific and Reflect APIs to perform as 
well as the above by dynamically compiling a schema's serialization and 
deserialization via ASM into direct decoder/encoder API calls.  But thats the 
sort of research project I only wish I had time for :)

> 
>> Yes, it helps a lot.  One question remains, how can I construct a recursive 
>> schema programmatically?
>> I have a couple options for the pig Tuple avro schema -- write it in JSON 
>> and put that in the source code or programmatically construct it.
>> I'm currently programmatically constructing a schema specific to the Pig 
>> schema that is serialized, which is straightforward until I hit the map type 
>> and recursion.
> 
> If you're not using a universal Pig schema then the above strategy may 
> or may not work.  It might still work if the specific schema is always a 
> subset of the universal Pig schema, which I suspect it is.

One requirement I may not have communicated well is that I want to persist the 
Pig schema, not just the pig data.   So if someone in Pig does:

STORE FOO into 'directory' using AvroStorage();

and FOO has pig schema:
{ firstName: string, lastName: string, departmentId: int, salary: long }
The avro schema would ideally be:
{ "name": "FOO", "type": "record", "fields": [
  { "name": "firstName", "type": "string" },
  { "name": "lastName", "type": "string" },
  { "name": "departmentId", "type": "int" },
  { "name": "salary", "type": "long"}
]

Then, when the avro container file is loaded back in pig (or in Java M/R, or 
Hive, or Cascading, or external applications, etc) the field names are 
preserved.  Without this feature much of the power of Avro goes away -- 
resolving schema evolution is manual and lock-step across all data consumers, 
etc.

I will create a unique schema for each store, and when a Map is encountered the 
value will be an avro record that can contain generic, nameless pig data using 
something much like Thiru's schema.

> 
> To construct a recursive schema programmatically you need to do what the 
> schema parser does: create a record schema with createSchema(), create 
> it's fields, including one or more that references the record schema, 
> then call setFields() with the fields.

Excellent.  I'll do that.  This means that the recursion points that avro 
supports can only be records, right?  Only a record has the equivalent of 
setFields(), everything else that can contain another element (arrays, unions) 
must declare the inner element(s) at creation time.

> 
> Doug
> 
> 

Thanks Doug,

-Scott

Reply via email to