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

Jeff Zhang commented on PIG-794:
--------------------------------

Scott & Doug, thanks for your review.

Scott, 
It seems we are doing the same thing to integrate avro into pig. seems you put 
all the read/writer logic in PigDataAssembly.java, and BTW, you miss one type 
of InternalMap which pig use it when there's a order by statement in pig 
script. The key type of InternalMap can been any type that pig can handle such 
as tuple.

Doug,

{quote}
is there a reason you subclass GenericDatumReader and GenericDatumWriter, 
overriding readRecord() and writeRecord()? 
{quote}
Actually at first I try to extend GenericDatumReader and GenericDatumWriter, 
but I found it needs to override many methods (The AvroStorage_4.patch has the 
implementation code), and one weird thing is that it can not handle 
InternalMap, PigData's main method illustrate the problem (maybe I do not use 
avro api correctly) .

{quote}
your writeMap() doesn't appear to write a valid Avro map, writeArray() doesn't 
write a valid Avro array
{quote}
Do you mean I should follow the steps of writeArray in GenericDatumWriter like 
following:
{code}
    out.writeArrayStart();
    out.setItemCount(size);
    for (Iterator<? extends Object> it = getArrayElements(datum); 
it.hasNext();) {
      out.startItem();
      write(element, it.next(), out);
    }
    out.writeArrayEnd();
{code}

{quote}
my guess is that a lot of time is spent in findSchemaIndex().
{quote}
Yes, I have optimized in AvroStorage_3.patch

{quote}
I don't see where this specifies an Avro schema for Pig data.
{quote}
I construct Pig's avro schema in PigData.java, I use the avro api to construct 
the schema rather than construct it from json.




> 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