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

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

A few comments about the attached code:
 - is there a reason you don't subclass GenericDatumReader and 
GenericDatumWriter, overriding readRecord() and writeRecord()?  That would 
simplify things and better guarantee that you're conforming to a schema.  
Currently, e.g., your writeMap() doesn't appear to write a valid Avro map, 
writeArray() doesn't write a valid Avro array, etc., so the data written is not 
interoperable,.
 - my guess is that a lot of time is spent in findSchemaIndex().  if that's 
right, you might improve this in various ways, e.g.:
 -- sort this by the most common types.  the order in Pig's DataType.java is 
probably a good one.
 -- try using a static Map<Class,Integer> cache of indexes
- have you run this under a profiler?

I don't see where this specifies an Avro schema for Pig data.  It's possible to 
construct a generic schema for all Pig data.  In this, a Bag should be record 
with a single field, an array of Tuples.  A Tuple should be a record with a 
single field, an array of a union of all types.  Given such a schema, one could 
then write a DatumReader/Writer using the control logic of Pig's 
DataReaderWriter (i.e., a switch based on the value of DataType.findType(), 
but, instead of calling DataInput/Output methods, use Encoder/Decoder methods 
with a ValidatingEncoder (at least while debugging) to ensure you conform to 
that schema.

Alternately, in Avro 1.4 (snapshot in Maven now, release this week, hopefully) 
Avro arrays can be arbitrary Collection implementations.  Bag already 
implements all of the required Collection methods -- clear(), add(), size(), & 
iterator(), so there's no reason I can see for Bag not to implement 
Collection<Tuple>.  So then one could subclass GenericData, GenericDatumReader 
& Writer, overriding:

{code}
protected boolean isRecord(Object datum) {
  return datum instanceof Tuple || datum instanceof Bag;
}
protected void writeRecord(Schema schema, Object datum, Encoder out) throws 
IOException {
  if (TUPLE_NAME.equals(schema.getFullName()))
    datum = ((Tuple)datum.getAll();
  writeArray(schema.getFields().get(0).getType(), datum, out);
}
protected Object readRecord(Object old, Schema expected, ResolvingDecoder in) 
throws IOException {
  Object result;
  if (TUPLE_NAME.equals(schema.getFullName())) {
    old = new ArrayList();
    result = new Tuple(old);
  } else {
    old = result = new Bag();
  }
  readArray(old, expected.getFields().get(0).getType(), in);
  return result;
}
{code}
   
Finally, if you knew the schema for the dataset being processed, rather than 
using a fully-general Pig schema, then you could translate that schema to an 
Avro schema.  This schema in most cases would not likely have a huge, 
compute-intensive-to-write union in it .  Or you might use something like what 
Scott's proposed in AVRO-592.


> 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.

Reply via email to