You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@avro.apache.org by "Dan Lipofsky (Jira)" <ji...@apache.org> on 2020/07/29 20:35:01 UTC

[jira] [Comment Edited] (AVRO-2471) Java maven plugin code generation doesn't add conversion for timestamp-micros

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

Dan Lipofsky edited comment on AVRO-2471 at 7/29/20, 8:34 PM:
--------------------------------------------------------------

I have the same problem with Avro 1.10.0. My IDL contains
{noformat}
  record Foobar1 {
    string id;
    @logicalType("timestamp-micros") long ts;
  }
  record Foobar2 {
    string id;
    union { null, @logicalType("timestamp-micros") long } ts = null;
  }
  record Foobar3 {
    string id;
    union { null, timestamp_ms } ts = null;
  }
{noformat}
Foobar1 and 3 work while 2 fails.
My test looks like
{noformat}
    public void foobar2() throws IOException {
        final Foobar2 f = Foobar2.newBuilder().setId("aaa").setTs(Instant.now()).build();
        try (final ByteArrayOutputStream out = new ByteArrayOutputStream()) {
            SpecificDatumWriter<SpecificRecord> writer = new SpecificDatumWriter<>(
                    ((SpecificRecord) f).getSchema());
            Encoder encoder = EncoderFactory.get().binaryEncoder(out, null);
            writer.write(f, encoder);
        }
    }
{noformat}
and fails like
{noformat}
org.apache.avro.AvroRuntimeException: Unknown datum type java.time.Instant: 2020-07-22T21:57:44.177315Z

	at org.apache.avro.generic.GenericData.getSchemaName(GenericData.java:912)
	at org.apache.avro.specific.SpecificData.getSchemaName(SpecificData.java:413)
	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.specific.SpecificDatumWriter.writeField(SpecificDatumWriter.java:98)
	at org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:210)
	at org.apache.avro.specific.SpecificDatumWriter.writeRecord(SpecificDatumWriter.java:83)
	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 com.....SerDeTest.foobar2(SerDeTest.java:49)
{noformat}
and if I add this right after the object is created it works:
{noformat}
        f.getSpecificData().addLogicalTypeConversion(new TimeConversions.TimestampMicrosConversion());
{noformat}
but I would think that should be unnecessary.



was (Author: dlipofsky):
I have the same problem with Avro 1.10.0. My IDL contains
{noformat}
  record Foobar1 {
    string id;
    @logicalType("timestamp-micros") long ts;
  }
  record Foobar2 {
    string id;
    union { null, @logicalType("timestamp-micros") long } ts = null;
  }
  record Foobar3 {
    string id;
    union { null, timestamp_ms } ts = null;
  }
{noformat}
Foobar1 and 3 work while 2 fails.
My test looks like
{noformat}
    public void foobar2() throws IOException {
        final Foobar2 f = Foobar2.newBuilder().setId("aaa").setTs(Instant.now()).build();
        try (final ByteArrayOutputStream out = new ByteArrayOutputStream()) {
            SpecificDatumWriter<SpecificRecord> writer = new SpecificDatumWriter<>(
                    ((SpecificRecord) f).getSchema());
            Encoder encoder = EncoderFactory.get().binaryEncoder(out, null);
            writer.write(f, encoder);
        }
    }
{noformat}
and fails like
{noformat}
org.apache.avro.AvroRuntimeException: Unknown datum type java.time.Instant: 2020-07-22T21:57:44.177315Z

	at org.apache.avro.generic.GenericData.getSchemaName(GenericData.java:912)
	at org.apache.avro.specific.SpecificData.getSchemaName(SpecificData.java:413)
	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.specific.SpecificDatumWriter.writeField(SpecificDatumWriter.java:98)
	at org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:210)
	at org.apache.avro.specific.SpecificDatumWriter.writeRecord(SpecificDatumWriter.java:83)
	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 com.....SerDeTest.foobar2(SerDeTest.java:49)
{noformat}
and if I add this right after the object is created it works:
{noformat}
        f.getSpecificData().addLogicalTypeConversion(new TimeConversions.TimestampMicrosConversion());
{noformat}
but clearly I shouldn't have to do that.


> Java maven plugin code generation doesn't add conversion for timestamp-micros
> -----------------------------------------------------------------------------
>
>                 Key: AVRO-2471
>                 URL: https://issues.apache.org/jira/browse/AVRO-2471
>             Project: Apache Avro
>          Issue Type: Bug
>          Components: java
>    Affects Versions: 1.9.0
>            Reporter: Marek Tracz
>            Priority: Major
>
> Field in schema: (there is no single field with timestamp-millis logical type)
> {code:java}
> {
>   "name": "RECORDING_TIME",
>   "type": [
>     "null",
>     {
>       "type": "long",
>       "logicalType": "timestamp-micros"
>     }
>   ],
>   "default": null
> }
> {code}
> Maven plugin configuration:
> {code:xml}
> <plugin>
> 	<groupId>org.apache.avro</groupId>
> 	<artifactId>avro-maven-plugin</artifactId>
> 	<version>1.9.0</version>
> 	<executions>
> 		<execution>
> 			<goals>
> 				<goal>schema</goal>
> 			</goals>
> 			<configuration>
> 				<stringType>String</stringType>
> 				<enableDecimalLogicalType>true</enableDecimalLogicalType>
> 				<sourceDirectory>${project.basedir}/src/main/resources/</sourceDirectory>
> 			</configuration>
> 		</execution>
> 	</executions>
> </plugin>
> {code}
> Part of the generated class:
> {code:java}
>   private static SpecificData MODEL$ = new SpecificData();
> static {
>     MODEL$.addLogicalTypeConversion(new org.apache.avro.data.TimeConversions.DateConversion());
>     MODEL$.addLogicalTypeConversion(new org.apache.avro.data.TimeConversions.TimestampMillisConversion()); // <--- this should be TimestampMicrosConversion
>     MODEL$.addLogicalTypeConversion(new org.apache.avro.Conversions.DecimalConversion());
>   }
> {code}
> For example this code:
> {code:java}
> Data data = Data.newBuilder()
> 		.setRECORDINGTIME(Instant.now())
>                 .build();
> {code}
> Fails during comparison:
> {noformat}
> org.apache.kafka.common.errors.SerializationException: Error serializing Avro message
> Caused by: org.apache.avro.AvroRuntimeException: Unknown datum type java.time.Instant: 2019-07-12T14:24:47.322Z
> 	at org.apache.avro.generic.GenericData.getSchemaName(GenericData.java:887)
> 	at org.apache.avro.specific.SpecificData.getSchemaName(SpecificData.java:420)
> 	at org.apache.avro.generic.GenericData.resolveUnion(GenericData.java:850)
> 	at org.apache.avro.generic.GenericDatumWriter.resolveUnion(GenericDatumWriter.java:249)
> 	at org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion(GenericDatumWriter.java:142)
> 	at org.apache.avro.specific.SpecificDatumWriter.writeField(SpecificDatumWriter.java:98)
> 	at org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:195)
> 	at org.apache.avro.specific.SpecificDatumWriter.writeRecord(SpecificDatumWriter.java:83)
> 	at org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion(GenericDatumWriter.java:130)
> 	at org.apache.avro.specific.SpecificDatumWriter.writeField(SpecificDatumWriter.java:98)
> 	at org.apache.avro.generic.GenericDatumWriter.writeRecord(GenericDatumWriter.java:195)
> 	at org.apache.avro.specific.SpecificDatumWriter.writeRecord(SpecificDatumWriter.java:83)
> 	at org.apache.avro.generic.GenericDatumWriter.writeWithoutConversion(GenericDatumWriter.java:130)
> 	at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:82)
> 	at org.apache.avro.generic.GenericDatumWriter.write(GenericDatumWriter.java:72)
> 	at io.confluent.kafka.serializers.AbstractKafkaAvroSerializer.serializeImpl(AbstractKafkaAvroSerializer.java:92)
> 	at io.confluent.kafka.serializers.KafkaAvroSerializer.serialize(KafkaAvroSerializer.java:53)
> 	at org.apache.kafka.common.serialization.ExtendedSerializer$Wrapper.serialize(ExtendedSerializer.java:65)
> 	at org.apache.kafka.common.serialization.ExtendedSerializer$Wrapper.serialize(ExtendedSerializer.java:55)
> 	at org.apache.kafka.clients.producer.KafkaProducer.doSend(KafkaProducer.java:841)
> 	at org.apache.kafka.clients.producer.KafkaProducer.send(KafkaProducer.java:803)
> 	at org.apache.kafka.clients.producer.KafkaProducer.send(KafkaProducer.java:690)
> {noformat}
> When manually changed to *org.apache.avro.data.TimeConversions.TimestampMicrosConversion* everything works properly.



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