Return-Path: Delivered-To: apmail-avro-user-archive@www.apache.org Received: (qmail 52447 invoked from network); 16 Jun 2010 03:10:56 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 16 Jun 2010 03:10:56 -0000 Received: (qmail 34409 invoked by uid 500); 16 Jun 2010 03:10:56 -0000 Delivered-To: apmail-avro-user-archive@avro.apache.org Received: (qmail 34212 invoked by uid 500); 16 Jun 2010 03:10:54 -0000 Mailing-List: contact user-help@avro.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@avro.apache.org Delivered-To: mailing list user@avro.apache.org Received: (qmail 33519 invoked by uid 99); 16 Jun 2010 03:10:53 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 16 Jun 2010 03:10:53 +0000 X-ASF-Spam-Status: No, hits=-0.4 required=10.0 tests=AWL,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: local policy) Received: from [64.78.17.17] (HELO EXHUB018-2.exch018.msoutlookonline.net) (64.78.17.17) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 16 Jun 2010 03:10:48 +0000 Received: from EXVMBX018-1.exch018.msoutlookonline.net ([64.78.17.47]) by EXHUB018-2.exch018.msoutlookonline.net ([64.78.17.17]) with mapi; Tue, 15 Jun 2010 20:10:28 -0700 From: Scott Carey To: "user@avro.apache.org" Date: Tue, 15 Jun 2010 20:10:36 -0700 Subject: Re: Serializing / Deserializing Java Objects Thread-Topic: Serializing / Deserializing Java Objects Thread-Index: AcsNAXiFJHIxVF2ZTumX9EGjGPNgWw== Message-ID: References: In-Reply-To: Accept-Language: en-US Content-Language: en-US X-MS-Has-Attach: X-MS-TNEF-Correlator: acceptlanguage: en-US Content-Type: text/plain; charset="us-ascii" Content-Transfer-Encoding: quoted-printable MIME-Version: 1.0 QueueItem myItem =3D new QueueItem(); GenericArray cols =3D new GenericArray( ... ) ... Since the Columns field is public, instead of: myItem.put(index, cols); do: myItem.Columns =3D cols; On Jun 15, 2010, at 7:54 PM, Bradford Stephens wrote: > Ah, interesting. >=20 > Then, is there a way to avoid manually making the .put(int, object) > call that relies on the magic number? >=20 > Or rather, what is the best practice for getting my Java object data > into a generated Avro class so that it can be written? >=20 > -B >=20 >=20 >=20 > On Tue, Jun 15, 2010 at 7:44 PM, Scott Carey wr= ote: >> This iteration of the SpecificAPI simply has public fields that are inte= nded to be set directly. >>=20 >> The current best practice is to use wrapper classes or static helpers to= interact with the generated objects so that most of your code is abstracte= d from the implementation details. >>=20 >> put(field, value) is there for other internal Avro code moreso than user= s -- specifically it allows a ResolvingDecoder to automatically figure out = where the data goes if the reader and writer's schemas differ. >>=20 >> Definitely do NOT depend on the 'magic number' in your code. We should= document that better. There is some discussion about the future of the Sp= ecific API so that it can generate getters/setters, and provide user contro= lled features -- potentially something like whether to use String[] or List= or Utf8[], etc to represent data in memory. More suggestions on h= ow to improve the API are welcome. >>=20 >> -Scott >>=20 >> On Jun 15, 2010, at 7:32 PM, Bradford Stephens wrote: >>=20 >>> Another thing to help me understand the Avro philosophy... >>>=20 >>> When doing, public void put(int field$, java.lang.Object value$) >>>=20 >>> Why is field an integer? >>>=20 >>> For instance, I have a String[] Column in my object. In protobuf, it >>> would generate java methods like .putColumn(String[] item). Is there a >>> reason avro can't do that? Or did I run the code generator in >>> avro-tools wrong? >>>=20 >>> If that doesn't work, could we generate an enum of field names to pass >>> in, instead? I don't like having to know "Magic Numbers" :) >>>=20 >>> Cheers, >>> B >>>=20 >>>=20 >>>=20 >>>=20 >>> On Tue, Jun 15, 2010 at 7:26 PM, Bradford Stephens >>> wrote: >>>> That's.... erm, kinda bizarre. >>>>=20 >>>> But hey, it works! Thanks! >>>>=20 >>>>=20 >>>>=20 >>>> On Tue, Jun 15, 2010 at 6:56 PM, Scott Carey = wrote: >>>>> Use GenericArray. The schema given to the generic array is not the s= chema of the member elements, but the actual array schema (yes it is confus= ing). >>>>>=20 >>>>> new GenericData.Array(size, Schema.createArray(Schema.create(Ty= pe.STRING)); >>>>>=20 >>>>> It would be useful to be able to simply use Utf8[] or List for = the Specific API, but at this time it leverages GenericData. >>>>>=20 >>>>>=20 >>>>> On Jun 15, 2010, at 6:25 PM, Bradford Stephens wrote: >>>>>=20 >>>>>> That makes sense -- I'm getting errors during serialization, though. >>>>>>=20 >>>>>> I convert my List to Utf8[]. >>>>>>=20 >>>>>> I then do a QueueItem.put() and get "Exception in thread "main" >>>>>> java.lang.ClassCastException: [Lorg.apache.avro.util.Utf8; cannot be >>>>>> cast to org.apache.avro.generic.GenericArray" >>>>>>=20 >>>>>> How do I handle this Java->Avro interop? It seems to me that it shou= ld >>>>>> be a lot simpler... >>>>>>=20 >>>>>> If I try to create a GenericArray and add items to that, it >>>>>> complains that my schema doesn't look right...so that doesn't feel >>>>>> like the right path. >>>>>>=20 >>>>>> My generated class looks like this: >>>>>>=20 >>>>>> @SuppressWarnings("all") >>>>>> public class QueueItem extends >>>>>> org.apache.avro.specific.SpecificRecordBase implements >>>>>> org.apache.avro.specific.SpecificRecord { >>>>>> public static final org.apache.avro.Schema SCHEMA$ =3D >>>>>> org.apache.avro.Schema.parse("{\"type\":\"record\",\"name\":\"QueueI= tem\",\"namespace\":\"com.dts\",\"fields\":[{\"name\":\"Columns\",\"type\":= [\"null\",{\"type\":\"array\",\"items\":\"string\"}]}]}"); >>>>>>=20 >>>>>> public org.apache.avro.generic.GenericArray >>>>>> Columns; >>>>>> public org.apache.avro.Schema getSchema() { return SCHEMA$; } >>>>>> public java.lang.Object get(int field$) { >>>>>>=20 >>>>>>=20 >>>>>> switch (field$) { >>>>>> case 0: return Columns; >>>>>> default: throw new org.apache.avro.AvroRuntimeException("Bad index= "); >>>>>> } >>>>>> } >>>>>> @SuppressWarnings(value=3D"unchecked") >>>>>> public void put(int field$, java.lang.Object value$) { >>>>>> switch (field$) { >>>>>> case 0: Columns =3D >>>>>> (org.apache.avro.generic.GenericArray)val= ue$; >>>>>> break; >>>>>> default: throw new org.apache.avro.AvroRuntimeException("Bad index= "); >>>>>> } >>>>>> } >>>>>> } >>>>>>=20 >>>>>>=20 >>>>>>=20 >>>>>>=20 >>>>>> On Tue, Jun 15, 2010 at 8:57 AM, Philip Zeyliger wrote: >>>>>>> Hi Bradford, >>>>>>> I believe you use a SpecificDatumReader. Something like: >>>>>>>=20 >>>>>>> final static SpecicificDatumReader QUEUE_ITEM_READER = =3D new >>>>>>> SpecificDatumReader(QueueItem.class); >>>>>>> QueueItem q =3D QUEUE_ITEM_READER.read(null, decoder); >>>>>>> There doesn't seem to be a test that exercises this code path in an >>>>>>> explanatory way, but java/src/java/org/apache/avro/ipc/Requestor.ja= va uses >>>>>>> something quite similar. >>>>>>> -- Philip >>>>>>>=20 >>>>>>> On Mon, Jun 14, 2010 at 9:20 PM, Bradford Stephens >>>>>>> wrote: >>>>>>>>=20 >>>>>>>> Greetings, >>>>>>>>=20 >>>>>>>> I've poked around for examples of this, but I can't find any. I >>>>>>>> imagine it's a fairly common use case. >>>>>>>>=20 >>>>>>>> I'm serializing some simple objects into bytes for placement onto >>>>>>>> RabbitMQ. My java class is pretty simple (but it'll grow in comple= xity >>>>>>>> in time).: >>>>>>>>=20 >>>>>>>> { >>>>>>>> String[] Columns; >>>>>>>> } >>>>>>>>=20 >>>>>>>>=20 >>>>>>>> So, I made a .json schema containing this: >>>>>>>> { >>>>>>>> "namespace": "com.dts", >>>>>>>> "name": "QueueItem", >>>>>>>> "type": "record", >>>>>>>> "fields": [ >>>>>>>> {"name": "Columns", "type": ["null", {"type": "array", >>>>>>>> "items":"string"}]} >>>>>>>> ] >>>>>>>> } >>>>>>>>=20 >>>>>>>>=20 >>>>>>>> And generated a java class ... >>>>>>>>=20 >>>>>>>> Now, I'm writing a test to serialize and deserialize some items. I= can >>>>>>>> figure out the serialization, but not deserialization back to obje= cts. >>>>>>>>=20 >>>>>>>> Schema s =3D Schema.parse(new File("queuetype.json")); >>>>>>>>=20 >>>>>>>> ByteArrayOutputStream bao =3D new ByteArrayOutputStream(= ); >>>>>>>> GenericDatumWriter w =3D new GenericDatumWriter(s); >>>>>>>> Encoder e =3D new BinaryEncoder(bao); >>>>>>>> e.init (bao); >>>>>>>>=20 >>>>>>>>=20 >>>>>>>> QueueItem r =3D new QueueItem(); >>>>>>>> r.put(0, items); >>>>>>>> w.write(r, e); >>>>>>>> e.flush(); >>>>>>>>=20 >>>>>>>> ByteArrayInputStream is =3D new >>>>>>>> ByteArrayInputStream(bao.toByteArray()); >>>>>>>> DecoderFactory df =3D new DecoderFactory(); >>>>>>>> Decoder d =3D df.createBinaryDecoder(is, null); >>>>>>>>=20 >>>>>>>> QueueItem itemout =3D (QueueItem) ..... >>>>>>>>=20 >>>>>>>>=20 >>>>>>>> And that's what I can't figure out -- how do I use a decoder metho= d to >>>>>>>> create QueueItems? >>>>>>>>=20 >>>>>>>> Cheers, >>>>>>>> B >>>>>>>>=20 >>>>>>>> radford Stephens, >>>>>>>> Founder, Drawn to Scale >>>>>>>> drawntoscalehq.com >>>>>>>> 727.697.7528 >>>>>>>>=20 >>>>>>>> http://www.drawntoscalehq.com -- The intuitive, cloud-scale data >>>>>>>> solution. Process, store, query, search, and serve all your data. >>>>>>>>=20 >>>>>>>> http://www.roadtofailure.com -- The Fringes of Scalability, Social >>>>>>>> Media, and Computer Science >>>>>>>=20 >>>>>>>=20 >>>>>>=20 >>>>>>=20 >>>>>>=20 >>>>>> -- >>>>>> Bradford Stephens, >>>>>> Founder, Drawn to Scale >>>>>> drawntoscalehq.com >>>>>> 727.697.7528 >>>>>>=20 >>>>>> http://www.drawntoscalehq.com -- The intuitive, cloud-scale data >>>>>> solution. Process, store, query, search, and serve all your data. >>>>>>=20 >>>>>> http://www.roadtofailure.com -- The Fringes of Scalability, Social >>>>>> Media, and Computer Science >>>>>=20 >>>>>=20 >>>>=20 >>>>=20 >>>>=20 >>>> -- >>>> Bradford Stephens, >>>> Founder, Drawn to Scale >>>> drawntoscalehq.com >>>> 727.697.7528 >>>>=20 >>>> http://www.drawntoscalehq.com -- The intuitive, cloud-scale data >>>> solution. Process, store, query, search, and serve all your data. >>>>=20 >>>> http://www.roadtofailure.com -- The Fringes of Scalability, Social >>>> Media, and Computer Science >>>>=20 >>>=20 >>>=20 >>>=20 >>> -- >>> Bradford Stephens, >>> Founder, Drawn to Scale >>> drawntoscalehq.com >>> 727.697.7528 >>>=20 >>> http://www.drawntoscalehq.com -- The intuitive, cloud-scale data >>> solution. Process, store, query, search, and serve all your data. >>>=20 >>> http://www.roadtofailure.com -- The Fringes of Scalability, Social >>> Media, and Computer Science >>=20 >>=20 >=20 >=20 >=20 > -- > Bradford Stephens, > Founder, Drawn to Scale > drawntoscalehq.com > 727.697.7528 >=20 > http://www.drawntoscalehq.com -- The intuitive, cloud-scale data > solution. Process, store, query, search, and serve all your data. >=20 > http://www.roadtofailure.com -- The Fringes of Scalability, Social > Media, and Computer Science