Return-Path: Delivered-To: apmail-hadoop-avro-user-archive@minotaur.apache.org Received: (qmail 68405 invoked from network); 17 Aug 2009 19:05:06 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 17 Aug 2009 19:05:06 -0000 Received: (qmail 17776 invoked by uid 500); 17 Aug 2009 19:05:12 -0000 Delivered-To: apmail-hadoop-avro-user-archive@hadoop.apache.org Received: (qmail 17724 invoked by uid 500); 17 Aug 2009 19:05:12 -0000 Mailing-List: contact avro-user-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: avro-user@hadoop.apache.org Delivered-To: mailing list avro-user@hadoop.apache.org Received: (qmail 17715 invoked by uid 99); 17 Aug 2009 19:05:12 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 17 Aug 2009 19:05:12 +0000 X-ASF-Spam-Status: No, hits=0.0 required=10.0 tests= X-Spam-Check-By: apache.org Received: from [140.211.11.9] (HELO minotaur.apache.org) (140.211.11.9) by apache.org (qpsmtpd/0.29) with SMTP; Mon, 17 Aug 2009 19:05:09 +0000 Received: (qmail 68301 invoked by uid 99); 17 Aug 2009 19:04:43 -0000 Received: from localhost.apache.org (HELO [192.168.168.109]) (127.0.0.1) (smtp-auth username cutting, mechanism plain) by minotaur.apache.org (qpsmtpd/0.29) with ESMTP; Mon, 17 Aug 2009 19:04:43 +0000 Message-ID: <4A89A9D0.1020202@apache.org> Date: Mon, 17 Aug 2009 12:04:48 -0700 From: Doug Cutting User-Agent: Thunderbird 2.0.0.22 (X11/20090608) MIME-Version: 1.0 To: avro-user@hadoop.apache.org Subject: Re: mixing types using reflection References: In-Reply-To: Content-Type: text/plain; charset=ISO-8859-1; format=flowed Content-Transfer-Encoding: 7bit X-Virus-Checked: Checked by ClamAV on apache.org Eelco Hillenius wrote: > There are two show stoppers for me right now, AVRO-93 and AVRO-95. > Now, my question is about the latter one, which is about mixing > multiple types in one data file using reflection. I submitted a unit > test for it that shows the bug, but I'm wondering if the way I'm using > the API is as it is intended. No, it is not. An Avro data file is expected to contain instances that all conform to a single schema. If you have multiple classes that you'd like to store in a single file then you can use a union schema, e.g.: List records = new ArrayList(); records.add(ReflectData.getSchema(MyRecord1.class)); records.add(ReflectData.getSchema(MyRecord2.class)); records.add(ReflectData.getSchema(MyRecord3.class)); Schema union = Schema.createUnion(records); DataFileWriter writer = new DataFileWriter(union, fos, new ReflectDatumWriter(union); I have not tested this code, but it should work. It should also work to add new schemas to the list as you write the file, before adding records of each type. Doug