Hmmm, I guess I was fuzzy on the details. I thought the SpecificData model
understood the standard logical types even when it was used on "generic"
data, which is apparently not the case!  I guess these conversions are
*only* built-in automatically when you use generated code.

For the reasoning, keep in mind the schema and binary representation didn't
really change between 1.7.x and 1.8.x!  A producer and consumer of Avro
binary data could be using two different versions of Avro and still
communicate successfully (1.7.x ignoring the logical type).  More
importantly, the API between the two was impressively compatible.  This was
particularly important in 2015/2016 when there were so many different
versions (and distributions) of Hadoop and Spark that were very gradually
transitioning between 1.7.x and 1.8.x -- I know we were writing libraries
using Avro that should run correctly with either 1.7.x or 1.8.x, even with
data generated by 1.8.x.

Versioning and compatibility is still a hot topic of conversation for Avro
today... it's definitely not a solved problem.

I know that Beam lets you specify the model in AvroIO.  If Parquet is
exclusively using the singleton SpecificData, is it alright with Beam
pipelines using generic data?  If you can't succeed in getting it to work,
please do raise a JIRA because it's likely a problem in other big data
execution engines as well!




On Wed, Oct 7, 2020 at 12:28 AM Bashir Sadjad <bas...@google.com> wrote:

> Thanks a lot Ryan, this was very helpful. It resolved the immediate
> problem I had (in the minimal example I posted before). But a more complete
> context of the issue that I had is described here
> <https://lists.apache.org/thread.html/rabfc94fd696651c1f28f245fd682366ba5a4e552317ccd5ccc3f7a63%40%3Cuser.beam.apache.org%3E>
> where after serializing these GenericRecords I was writing them into
> Parquet files. With your hint, I could resolve another related issue there
> too. I just put it here for future reference:
>
> The issue was in AvroWriteSupport of Parquet, specifically here
> in writeValueWithoutConversion
> <https://github.com/apache/parquet-mr/blob/0a4e3eea991f7588c9c5e056e9d7b32a76eed5da/parquet-avro/src/main/java/org/apache/parquet/avro/AvroWriteSupport.java#L348>
>  where
> I was getting a cast exception:
>
> java.lang.ClassCastException: class java.math.BigDecimal cannot be cast to
> class java.nio.ByteBuffer (java.math.BigDecimal and java.nio.ByteBuffer are
> in module java.base of loader 'bootstrap')
>
> I debugged this a little more and the way AvroWriteSupport instance is
> created is by using SpecificData as its model (e.g., here
> <https://github.com/apache/parquet-mr/blob/0a4e3eea991f7588c9c5e056e9d7b32a76eed5da/parquet-avro/src/main/java/org/apache/parquet/avro/AvroParquetWriter.java#L163>).
> Using your trick, I could resolve that other issue with adding this to my
> code:
>
> SpecificData.get().addLogicalTypeConversion(new DecimalConversion());
>
> I haven't yet tried reading those Parquet files but I might need to do
> something similar to be able to read those BigDecimals properly.
>
> BTW, I did not fully understand the reasoning behind why support for these
> logical types are not in GenericData by default (I mean if logical types
> were not present before 1.8.x why backward compatibility for reading was an
> issue? Older schema did not have logical types, right?). I am sure there
> are good reasons that I don't understand because I don't have the full
> context but this certainly was not trivial to add DecimalConversion
> manually.
>
> Thanks again for your help.
>
> -B
>
> On Tue, Oct 6, 2020 at 10:56 AM Ryan Skraba <r...@skraba.com> wrote:
>
>> Hello!  This is a frequent stumbling block for logical types.
>>
>> You should explicitly add the Decimal logical type conversion to the
>> data model that interprets the Java datum being serialized to your
>> file, like this:
>>
>> GenericData model = new GenericData();
>> model.addLogicalTypeConversion(new Conversions.DecimalConversion());
>> DatumWriter<GenericRecord> datumWriter = new
>> GenericDatumWriter<>(r.getSchema(), model);
>>
>> (You can also add it to the singleton GenericData.get() instance used
>> by your application.  I tend to prefer explicitly setting the model.)
>>
>> As an explanation: when logical types were added to Avro in 1.8.x, the
>> "standard" logical types weren't automatically added to the
>> GenericData model, likely to ensure that the reading behaviour remains
>> unchanged from 1.7.x (unless specifically requested).  Although I've
>> seldom seen user-defined logical types with Avro, they would also need
>> to be added explicitly.  This problem doesn't occur with generated
>> code and specific records, since the conversions are decided when the
>> code is generated.
>>
>> I hope this is useful!  Best regards, Ryan
>>
>>
>> On Tue, Oct 6, 2020 at 7:21 AM Bashir Sadjad <bas...@google.com> wrote:
>> >
>> > Hi all,
>> >
>> > I do not have a lot of experience using Avro, so hopefully this is not
>> an obvious question (and I hope this is the right place to ask):
>> >
>> > I have a schema with a decimal logical type, e.g.,
>> >
>> > {
>> >   "type" : "record",
>> >   "name" : "testRecord",
>> >   "namespace" : "org.example",
>> >   "doc" : "",
>> >   "fields" : [
>> >     {
>> >       "name" : "value",
>> >       "type":[
>> >         "null",
>> >         {
>> >           "type":"bytes",
>> >           "logicalType":"decimal",
>> >           "precision":12,
>> >           "scale":4
>> >         }
>> >       ],
>> >       "doc":"",
>> >       "default":null
>> >     }
>> >   ]
>> > }
>> >
>> > And I have a code that parses this schema and creates a GenericRecord
>> based on that then puts a BigDecimal for "value" (I have copied the full
>> code at the end). The problem is that when I write this record to file, I
>> get the following exception which IIUC is coming from the fact that there
>> are no conversions registered for BigDecimal here:
>> >
>> > org.apache.avro.file.DataFileWriter$AppendWriteException:
>> org.apache.avro.AvroRuntimeException: Unknown datum type
>> java.math.BigDecimal: 10.0
>> >    at org.apache.avro.file.DataFileWriter.append
>> (DataFileWriter.java:317)
>> >    at org.openmrs.analytics.TestAvro.main (TestAvro.java:25)
>> >    at org.codehaus.mojo.exec.ExecJavaMojo$1.run (ExecJavaMojo.java:254)
>> >    at java.lang.Thread.run (Thread.java:834)
>> > Caused by: org.apache.avro.AvroRuntimeException: Unknown datum type
>> java.math.BigDecimal: 10.0
>> >    at org.apache.avro.generic.GenericData.getSchemaName
>> (GenericData.java:912)
>> >    at org.apache.avro.generic.GenericData.resolveUnion
>> (GenericData.java:874)
>> >    at org.apache.avro.generic.GenericDatumWriter.resolveUnion
>> (GenericDatumWriter.java:272)
>> >    at org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion
>> (GenericDatumWriter.java:143)
>> >    at org.apache.avro.generic.GenericDatumWriter.write
>> (GenericDatumWriter.java:83)
>> >    at org.apache.avro.generic.GenericDatumWriter.writeField
>> (GenericDatumWriter.java:221)
>> >    at org.apache.avro.generic.GenericDatumWriter.writeRecord
>> (GenericDatumWriter.java:210)
>> >    at org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion
>> (GenericDatumWriter.java:131)
>> >    at org.apache.avro.generic.GenericDatumWriter.write
>> (GenericDatumWriter.java:83)
>> >    at org.apache.avro.generic.GenericDatumWriter.write
>> (GenericDatumWriter.java:73)
>> >    at org.apache.avro.file.DataFileWriter.append
>> (DataFileWriter.java:314)
>> >    at org.openmrs.analytics.TestAvro.main (TestAvro.java:25)
>> >    at org.codehaus.mojo.exec.ExecJavaMojo$1.run (ExecJavaMojo.java:254)
>> >    at java.lang.Thread.run (Thread.java:834)
>> >
>> > But my understanding is that BigDecimal is the right type for the Avro
>> decimal logical type, is that correct? If yes, shouldn't this approach
>> work? I can do a conversion from BigDecimal to ByteBuffer but that is
>> something that I want to avoid because in my real use case, I am receiving
>> GenericRecord from another library that creates the schema too, possibly
>> with many such logical types. Here is the full code:
>> >
>> > import java.io.File;
>> > import java.io.IOException;
>> > import java.math.BigDecimal;
>> >
>> > import org.apache.avro.Schema;
>> > import org.apache.avro.file.DataFileWriter;
>> > import org.apache.avro.generic.GenericData.Record;
>> > import org.apache.avro.generic.GenericDatumWriter;
>> > import org.apache.avro.generic.GenericRecord;
>> > import org.apache.avro.io.DatumWriter;
>> >
>> > public class TestAvro {
>> >   public static void main(String[] args) throws IOException {
>> >     Schema testSchema = new Schema.Parser().parse(new
>> File("tmp/test_decimal_union.avsc"));
>> >     GenericRecord testRecord = new Record(testSchema);
>> >     testRecord.put("value", BigDecimal.valueOf(10.0));
>> >     DatumWriter<GenericRecord> datumWriter = new
>> GenericDatumWriter<>(testSchema);
>> >     DataFileWriter<GenericRecord> dataFileWriter = new
>> DataFileWriter<GenericRecord>(datumWriter);
>> >     File testFile = new File("tmp/test_decimal.avro");
>> >     dataFileWriter.create(testSchema, testFile);
>> >     dataFileWriter.append(testRecord);
>> >     dataFileWriter.close();
>> > }
>> >
>> > Any help is appreciated.
>> >
>> > -B
>>
>

Reply via email to