[
https://issues.apache.org/jira/browse/AVRO-1554?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14078425#comment-14078425
]
Sachin Goyal commented on AVRO-1554:
------------------------------------
Thank you for reviewing.
I did think about adding the configuration directly to ReflectData but created
a new class AvroConfig to create a more logical space (in my understand I
confess) for such options. This class can also serve to populate more
configuration in the future. But I am open to purge this class. Please confirm
and I will submit another patch.
The index needs to be discarded so that clients do not handle auto-generated
UNION-type schemas. Consider the schema generated for a class having two fields
- uuid and date:
{code:javascript}
{
"type":"record",
"name":"Types",
"namespace":"org.apache.avro.reflect",
"fields":[
{
"name":"uuid",
"type":[
"null",
{
"type":"string",
"CustomEncoding":"UuidAsStringEncoding"
}
],
"default":null
}
]
}
{code}
But the user only specified:
{code}
{
"type":"string",
"CustomEncoding":"UuidAsStringEncoding"
}
{code}
as the schema.It turned into a UNION because of ReflectData.AllowNull (or some
other things like CircularRef that may be added in future). The
index-discarding is required to:
# Unwind the parser to the right symbol
# Get rid of the integer written by resolveIndex while writing.
If the index is not read and discarded, then clients writing custom-encodings
will need to handle such auto-generated unions. Please suggest if this can be
improved.
> Avro should have support for common constructs like UUID and Date
> -----------------------------------------------------------------
>
> Key: AVRO-1554
> URL: https://issues.apache.org/jira/browse/AVRO-1554
> Project: Avro
> Issue Type: Bug
> Components: java
> Affects Versions: 1.7.6
> Reporter: Sachin Goyal
> Attachments: AVRO-1554.patch
>
>
> Consider the following code:
> {code}
> public class AvroExample
> {
> public static void main (String [] args) throws Exception
> {
> ReflectData rdata = ReflectData.AllowNull.get();
> Schema schema = rdata.getSchema(Temp.class);
>
> ReflectDatumWriter<Temp> datumWriter =
> new ReflectDatumWriter (Temp.class, rdata);
> DataFileWriter<Temp> fileWriter =
> new DataFileWriter<Temp> (datumWriter);
> ByteArrayOutputStream baos = new ByteArrayOutputStream();
> fileWriter.create(schema, baos);
> fileWriter.append(new Temp());
> fileWriter.close();
> byte[] bytes = baos.toByteArray();
> GenericDatumReader<GenericRecord> datumReader =
> new GenericDatumReader<GenericRecord> ();
> SeekableByteArrayInput avroInputStream =
> new SeekableByteArrayInput(bytes);
> DataFileReader<GenericRecord> fileReader =
> new DataFileReader<GenericRecord>(avroInputStream,
> datumReader);
> schema = fileReader.getSchema();
> GenericRecord record = null;
> record = fileReader.next(record);
> System.out.println (record);
> System.out.println (record.get("id"));
> }
> }
> class Temp
> {
> UUID id = UUID.randomUUID();
> Date date = new Date();
> BigInteger bi = BigInteger.TEN;
> }
> {code}
> Output from this code is:
> {code:javascript}
> {"id": {}, "date": {}, "bi": "10"}
> {code}
> UUID and Date type fields are very common in Java and can be found a lot in
> third-party code as well (where it may be difficult to put annotations).
> So Avro should include a default serialization/deserialization support for
> such fields.
--
This message was sent by Atlassian JIRA
(v6.2#6252)