You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by "Scott Carey (JIRA)" <ji...@apache.org> on 2010/04/15 19:07:49 UTC
[jira] Created: (AVRO-517) Resolving Decoder fails in some cases
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
[jira] Commented: (AVRO-517) Resolving Decoder fails in some cases
Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/AVRO-517?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12857920#action_12857920 ]
Scott Carey commented on AVRO-517:
----------------------------------
+1
This patch looks good to me.
> 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
> Assignee: Thiruvalluvan M. G.
> Priority: Critical
> Attachments: AVRO-517.patch
>
>
> 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
[jira] Updated: (AVRO-517) Resolving Decoder fails in some cases
Posted by "Thiruvalluvan M. G. (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/AVRO-517?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Thiruvalluvan M. G. updated AVRO-517:
-------------------------------------
Attachment: AVRO-517.patch
The trouble is that the ResolvingDecoder does not take care of the trailing field in the underlying BinaryDecoder. So part of the data belonging to the current object is left in the BinaryDecoder. GenericDatumReader constructs a new ResolvingDecoder for the next object. So the leftover integer field is read as a string of the next object.
This problem will not occur if the same ResolvingDecoder is used for all the objects. But that approach requires quite a bit of changes to GenericDatumReader. So I added a new method drain() in ResolvingDecoder, which,if called after reading the entire record as per reader's schema, drains the remaining unused portions.
> 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
> Assignee: Thiruvalluvan M. G.
> Priority: Critical
> Attachments: AVRO-517.patch
>
>
> 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
[jira] Updated: (AVRO-517) Resolving Decoder fails in some cases
Posted by "Thiruvalluvan M. G. (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/AVRO-517?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Thiruvalluvan M. G. updated AVRO-517:
-------------------------------------
Status: Patch Available (was: Open)
Thanks Scott for catching this intricate bug.
> 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
> Assignee: Thiruvalluvan M. G.
> Priority: Critical
> Attachments: AVRO-517.patch
>
>
> 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
[jira] Updated: (AVRO-517) Resolving Decoder fails in some cases
Posted by "Thiruvalluvan M. G. (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/AVRO-517?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Thiruvalluvan M. G. updated AVRO-517:
-------------------------------------
Status: Resolved (was: Patch Available)
Fix Version/s: 1.4.0
Resolution: Fixed
Committed revision 935526.
Thanks Scott for reporting the issue and reviewing the fix.
> 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
> Assignee: Thiruvalluvan M. G.
> Priority: Critical
> Fix For: 1.4.0
>
> Attachments: AVRO-517.patch
>
>
> 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.
-
You can reply to this email to add a comment to the issue online.
[jira] Commented: (AVRO-517) Resolving Decoder fails in some cases
Posted by "Scott Carey (JIRA)" <ji...@apache.org>.
[ 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<Record> writer = new DataFileWriter<Record>(
new GenericDatumWriter<Record>(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<Record> dr = new GenericDatumReader<Record>();
dr.setExpected(ageSchema);
DataFileReader<Record> reader = new DataFileReader<Record>(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<Record> dr = new GenericDatumReader<Record>();
dr.setExpected(extractSchema);
DataFileReader<Record> reader = new DataFileReader<Record>(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
[jira] Assigned: (AVRO-517) Resolving Decoder fails in some cases
Posted by "Thiruvalluvan M. G. (JIRA)" <ji...@apache.org>.
[ https://issues.apache.org/jira/browse/AVRO-517?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Thiruvalluvan M. G. reassigned AVRO-517:
----------------------------------------
Assignee: Thiruvalluvan M. G.
> 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
> Assignee: Thiruvalluvan M. G.
> 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