[jira] Commented: (PIG-794) Use Avro serialization in Pig

2010-09-08 Thread Doug Cutting (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-794?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12907280#action_12907280
 ] 

Doug Cutting commented on PIG-794:
--

Jeff, please instead use current trunk or the 1.4.0 build that I expect to be 
released tomorrow (http://people.apache.org/~cutting/avro-1.4.0-rc4/).  There 
was a bug that caused a similar failure in the snapshot you're using, but that 
should only happen in multi-threaded applications, which I doubt yours is, but 
it's better to either test against trunk or a release so we don't chase ghosts.

Further, while debugging a DatumWriter and DatumReader, you might use a 
ValidatingEncoder and ValidatingDecoder to ensure that what you write and read 
conforms to your schema.  You might also test by reading and printing your data 
with GenericDatumReader to see that you've written what you meant to write.  If 
you've written data that does not conform to your declared schema then it 
cannot be read correctly.  If this is the case, we should attempt to improve 
the error message here.


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



[jira] Commented: (PIG-794) Use Avro serialization in Pig

2010-09-07 Thread Doug Cutting (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-794?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12906871#action_12906871
 ] 

Doug Cutting commented on PIG-794:
--

Jeff, what version of Avro are you using?

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



[jira] Commented: (PIG-794) Use Avro serialization in Pig

2010-09-02 Thread Doug Cutting (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-794?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12905663#action_12905663
 ] 

Doug Cutting commented on PIG-794:
--

Some quick comments on the new patch:
  - you might define a java enum type for the union elements, using 
Enum#ordinal() for the union indexes
  - instead of name.equals(union), s.getType()==Type.UNION would be faster, 
but better yet would be to simply call read() recursively, since it already 
handles unions, no?
 - peekArray() can simply return null, and that might be faster



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



[jira] Commented: (PIG-794) Use Avro serialization in Pig

2010-08-31 Thread Doug Cutting (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-794?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=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 MapClass,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 
CollectionTuple.  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.



[jira] Commented: (PIG-794) Use Avro serialization in Pig

2009-05-14 Thread Doug Cutting (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-794?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12709608#action_12709608
 ] 

Doug Cutting commented on PIG-794:
--

Looking at the patch, I have a few questions and remarks:
 - Why not name the records Tuple and Bag instead of T and B?  The 
names are not written in the data, so there's little advantage to shorter names.
 - Why not, instead of parsing the schema from Json, construct the schema using 
the Java Schema API?  Then you would not need to walk the schema afterwards to 
find union indexes, and you'd get compile-time API checking rather than 
potential load-time JSON parse errors.
 - Why not extend GenericDatumReader and override newRecord() to create either 
a Bag or a Tuple, then override addField() to add values to either a bag or 
tuple?  This would make the patch much smaller, and potentially permit you to 
eventually take advantage of GenericDatumReader features like projection and 
object reuse.
 - Finally, since you're using a pre-release version of Avro, you should 
probably name the jar with the subversion revision number.  Also note that, 
since Avro is not yet stable, it should not be yet used for persistent data in 
production systems.


 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
 Fix For: 0.2.0

 Attachments: avro-0.1-dev-java.jar, AvroStorage.patch, 
 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.



[jira] Commented: (PIG-794) Use Avro serialization in Pig

2009-05-05 Thread Doug Cutting (JIRA)

[ 
https://issues.apache.org/jira/browse/PIG-794?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=12706197#action_12706197
 ] 

Doug Cutting commented on PIG-794:
--

 I think we have to ask the Avro team to support this (current position in the 
 stream) for us to proceed with this. 

ValueReader performs no buffering, so its position is always the same as the 
InputStream that it wraps.  See DataFileReader#SeekableBufferedInput for an 
example of an input stream that tracks its position.

Note that AVRO-25 proposes to add buffering to ValueWriter, so that the 
position of the underlying stream might be different than that of the 
ValueWriter, but I do not forsee a need to add this to ValueReader, the concern 
here.

 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
 Attachments: AvroBinStorage.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.