You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by "Ryan Skraba (Jira)" <ji...@apache.org> on 2021/08/25 10:02:00 UTC

[jira] [Commented] (AVRO-2872) 'conversions' array is not populated for Avro Union Logicaltype fields

    [ https://issues.apache.org/jira/browse/AVRO-2872?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17404331#comment-17404331 ] 

Ryan Skraba commented on AVRO-2872:
-----------------------------------

Hello [~icecreamhead] -- I'd really like to get this merged too!  Your spike project cleared up some of the issues for me -- it seems that there's a couple of things going on, including custom logical types and nullable types.  

As you note, the spike project still breaks for nullable custom conversions.  This works today if you add the following code "someplace":

{code}
  static {
    LogicalTypes.register(ZONED_DATE_TIME_TYPE.getName(), new TypeFactory());
  }
{code}

This needs to be called before the {{SCHEMA$}} is initialized inside the generated specific record.   This [unit test|https://github.com/apache/avro/blob/ccbbb8353fcc168563c567580cb773d5fe442704/lang/java/integration-test/codegen-test/src/test/java/org/apache/avro/codegentest/TestCustomConversion.java#L32] does it in {{@BeforeClass}} but it's probably cleaner to add it as a static initializer inside the [custom conversion|https://github.com/icecreamhead/avro-logical-type-testing/blob/main/conversion/src/main/java/ZonedDateTimeConversion.java#L12].

If we don't want to _enforce_ a workaround like that, then the generated specific record should probably take care of registering all of the custom conversions that it uses in it's Schema.  What do you think?  If we go that route, maybe we should consider merging this PR (which fixes exactly the problem in this JIRA description) and creating a new JIRA to fix the actual symptom you're seeing.



> 'conversions' array is not populated for Avro Union Logicaltype fields 
> -----------------------------------------------------------------------
>
>                 Key: AVRO-2872
>                 URL: https://issues.apache.org/jira/browse/AVRO-2872
>             Project: Apache Avro
>          Issue Type: Bug
>          Components: logical types
>    Affects Versions: 1.9.2
>         Environment: * Apache Avro Version [1.9.2]
> * Java Version [11]
>            Reporter: Pádraig de Buitléar
>            Assignee: Josh Cooke
>            Priority: Minor
>             Fix For: 1.11.0
>
>
> Steps to reproduce :
>  # Using the maven/gradle plugin generate code with the following avsc:
>    
> {code:java}
> {
>   "type": "record",
>   "name": "Messages",
>   "namespace": "com.somedomain",
>   "fields": [
>     {
>       "name": "start",
>       "type": {
>         "type": "long",
>         "logicalType": "timestamp-millis"
>       }
>     },
>     {
>       "name": "optional_date",
>       "type": [
>         "null",
>         {
>           "type": "long",
>           "logicalType": "timestamp-millis"
>         }
>       ],
>       "default": null
>     }
>   ]
> }{code}
>  
> *Actual behavior*
>  In the generated code, the return types of the getter methods for both fields are correct, however the conversions array only has an element for the field which isn't a union.
>   
> {code:java}
>   private static final org.apache.avro.Conversion<?>[] conversions =
>       new org.apache.avro.Conversion<?>[] {
>       new org.apache.avro.data.TimeConversions.TimestampMillisConversion(),
>       null,
>       null
>   };
> {code}
>  
> *Expected output:*
>  Based on the above avsc the following is expected.
>   
> {code:java}
>   private static final org.apache.avro.Conversion[] conversions =
>       new org.apache.avro.Conversion[] {
>       new org.apache.avro.data.TimeConversions.TimestampMillisConversion(),
>       new org.apache.avro.data.TimeConversions.TimestampMillisConversion(),
>       null
>   };
> {code}
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)