Return-Path: Delivered-To: apmail-hadoop-avro-dev-archive@minotaur.apache.org Received: (qmail 92354 invoked from network); 15 Apr 2010 17:12:12 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 15 Apr 2010 17:12:12 -0000 Received: (qmail 61783 invoked by uid 500); 15 Apr 2010 17:12:11 -0000 Delivered-To: apmail-hadoop-avro-dev-archive@hadoop.apache.org Received: (qmail 61707 invoked by uid 500); 15 Apr 2010 17:12:11 -0000 Mailing-List: contact avro-dev-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: avro-dev@hadoop.apache.org Delivered-To: mailing list avro-dev@hadoop.apache.org Received: (qmail 61697 invoked by uid 99); 15 Apr 2010 17:12:11 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 15 Apr 2010 17:12:11 +0000 X-ASF-Spam-Status: No, hits=-1294.7 required=10.0 tests=ALL_TRUSTED,AWL X-Spam-Check-By: apache.org Received: from [140.211.11.22] (HELO thor.apache.org) (140.211.11.22) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 15 Apr 2010 17:12:10 +0000 Received: from thor (localhost [127.0.0.1]) by thor.apache.org (8.13.8+Sun/8.13.8) with ESMTP id o3FHBnQv023739 for ; Thu, 15 Apr 2010 13:11:49 -0400 (EDT) Message-ID: <13614506.145201271351509978.JavaMail.jira@thor> Date: Thu, 15 Apr 2010 13:11:49 -0400 (EDT) From: "Scott Carey (JIRA)" To: avro-dev@hadoop.apache.org Subject: [jira] Commented: (AVRO-517) Resolving Decoder fails in some cases In-Reply-To: <9201816.145111271351269362.JavaMail.jira@thor> MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/AVRO-517?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12857417#action_12857417 ] Scott Carey commented on AVRO-517: ---------------------------------- Sample code that shows this issue: {code} import java.io.File; import java.io.IOException; import org.apache.avro.Schema; import org.apache.avro.file.DataFileReader; import org.apache.avro.file.DataFileWriter; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericData.Record; import org.apache.avro.util.Utf8; public class AddressBook { String fileName = "AddressBook.db"; String prefix = "{\"type\":\"record\",\"name\": \"Person\",\"fields\":["; String suffix = "]}"; String fieldFirst = "{\"name\":\"First\",\"type\":\"string\"}"; String fieldLast = "{\"name\":\"Last\",\"type\":\"string\"}"; String fieldAge = "{\"name\":\"Age\",\"type\":\"int\"}"; Schema personSchema = Schema.parse(prefix + fieldFirst + "," + fieldLast + "," + fieldAge + suffix); Schema ageSchema = Schema.parse(prefix + fieldAge + suffix); Schema extractSchema = Schema.parse(prefix + fieldFirst + "," + fieldLast + suffix); /** * @param args * @throws IOException */ public static void main(String[] args) throws IOException { AddressBook ab = new AddressBook(); ab.init(); ab.browseAge(); ab.browseName(); } public void init() throws IOException { DataFileWriter writer = new DataFileWriter( new GenericDatumWriter(personSchema)).create( personSchema, new File(fileName)); try { writer.append(createPerson("Dante", "Hicks", 27)); writer.append(createPerson("Randal", "Graves", 20)); writer.append(createPerson("Steve", "Jobs", 31)); } finally { writer.close(); } } private Record createPerson(String first, String last, int age) { Record person = new GenericData.Record(personSchema); person.put("First", new Utf8(first)); person.put("Last", new Utf8(last)); person.put("Age", age); return person; } public void browseAge() throws IOException { GenericDatumReader dr = new GenericDatumReader(); dr.setExpected(ageSchema); DataFileReader reader = new DataFileReader(new File( fileName), dr); try { while (reader.hasNext()) { Record person = reader.next(); System.out.println(person.get("Age").toString()); } } finally { reader.close(); } } public void browseName() throws IOException { GenericDatumReader dr = new GenericDatumReader(); dr.setExpected(extractSchema); DataFileReader reader = new DataFileReader(new File( fileName), dr); try { while (reader.hasNext()) { Record person = reader.next(); System.out.println(person.get("First").toString() + " " + person.get("Last").toString() + "\t"); } } finally { reader.close(); } } } {code} User comments: {quote} Hi, 27 20 31 Dante Hicks Exception in thread "main" org.apache.avro.AvroRuntimeException: java.io.EOFException at org.apache.avro.file.DataFileStream.next(DataFileStream.java:184) at cn.znest.test.avro.AddressBook.browseName(AddressBook.java:91) at cn.znest.test.avro.AddressBook.main(AddressBook.java:43) Caused by: java.io.EOFException at org.apache.avro.io.BinaryDecoder.readInt(BinaryDecoder.java:163) at org.apache.avro.io.BinaryDecoder.readString(BinaryDecoder.java:262) at org.apache.avro.io.ValidatingDecoder.readString(ValidatingDecoder.java:93) at org.apache.avro.generic.GenericDatumReader.readString(GenericDatumReader.java:277) at org.apache.avro.generic.GenericDatumReader.readString(GenericDatumReader.java:271) at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:83) at org.apache.avro.generic.GenericDatumReader.readRecord(GenericDatumReader.java:105) at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:77) at org.apache.avro.generic.GenericDatumReader.read(GenericDatumReader.java:70) at org.apache.avro.file.DataFileStream.next(DataFileStream.java:195) at org.apache.avro.file.DataFileStream.next(DataFileStream.java:182) ... 2 more My code is below. In this example, I create three record: person(3 fileds: First Last Age), age(Age), extract(First Last). The record "age" has the last filed of "Person", so AddressBook.browseAge() will be executed successfully. But the record "extract" does not have the last filed of "Person", so executing AddressBook.browseName() will cause an exception. In avro/c, read_record (datum_read.c) loops every write_schema fileds. In avro/java, GenericDatumReader.readRecord loops every read_schema fileds. I think that's the point. {quote} The above may be correct, the loop is over the read schema (expected) rather than actual. However, it works fine if the read schema is at the "end" of the writer schema. I am not familiar enough with the resolving decoder internals yet to and locate the bug quickly. It is cleanly reproducible. > Resolving Decoder fails in some cases > ------------------------------------- > > Key: AVRO-517 > URL: https://issues.apache.org/jira/browse/AVRO-517 > Project: Avro > Issue Type: Bug > Components: java > Affects Versions: 1.3.2 > Reporter: Scott Carey > Priority: Critical > > User reports that reading an 'actual' schema of > string, string, int > fails when using an expected schema of: > string, string > Sample code and details in the comments. -- This message is automatically generated by JIRA. - If you think it was sent incorrectly contact one of the administrators: https://issues.apache.org/jira/secure/Administrators.jspa - For more information on JIRA, see: http://www.atlassian.com/software/jira