This is an automated email from the ASF dual-hosted git repository. lhotari pushed a commit to branch branch-4.2 in repository https://gitbox.apache.org/repos/asf/pulsar.git
commit c11ddd937b9e6568a6ee90e0a80b828c094f63e7 Author: Lari Hotari <[email protected]> AuthorDate: Wed May 13 15:44:45 2026 +0300 [fix][client] Apply Avro logical type conversions when decoding schema without classloader (#25759) (cherry picked from commit 30ca23f463412de0fe49007d8d5215252e693c56) --- .../client/impl/schema/reader/AvroReader.java | 24 ++++--------- .../pulsar/client/impl/schema/AvroSchemaTest.java | 40 ++++++++++++++++++++++ 2 files changed, 47 insertions(+), 17 deletions(-) diff --git a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/reader/AvroReader.java b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/reader/AvroReader.java index 3ad43dbd9f2..0c9b8d37ae2 100644 --- a/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/reader/AvroReader.java +++ b/pulsar-client/src/main/java/org/apache/pulsar/client/impl/schema/reader/AvroReader.java @@ -18,6 +18,7 @@ */ package org.apache.pulsar.client.impl.schema.reader; +import com.google.common.annotations.VisibleForTesting; import java.io.IOException; import java.io.InputStream; import java.util.Optional; @@ -39,32 +40,21 @@ public class AvroReader<T> implements SchemaReader<T> { new ThreadLocal<>(); private final Schema schema; + @VisibleForTesting public AvroReader(Schema schema) { - this.reader = new ReflectDatumReader<>(schema); - this.schema = schema; + this(schema, schema, null, false); } public AvroReader(Schema schema, ClassLoader classLoader, boolean jsr310ConversionEnabled) { - this.schema = schema; - if (classLoader != null) { - ReflectData reflectData = new ReflectData(classLoader); - AvroSchema.addLogicalTypeConversions(reflectData, jsr310ConversionEnabled); - this.reader = new ReflectDatumReader<>(schema, schema, reflectData); - } else { - this.reader = new ReflectDatumReader<>(schema); - } + this(schema, schema, classLoader, jsr310ConversionEnabled); } public AvroReader(Schema writerSchema, Schema readerSchema, ClassLoader classLoader, boolean jsr310ConversionEnabled) { this.schema = readerSchema; - if (classLoader != null) { - ReflectData reflectData = new ReflectData(classLoader); - AvroSchema.addLogicalTypeConversions(reflectData, jsr310ConversionEnabled); - this.reader = new ReflectDatumReader<>(writerSchema, readerSchema, reflectData); - } else { - this.reader = new ReflectDatumReader<>(writerSchema, readerSchema); - } + ReflectData reflectData = classLoader != null ? new ReflectData(classLoader) : new ReflectData(); + AvroSchema.addLogicalTypeConversions(reflectData, jsr310ConversionEnabled); + this.reader = new ReflectDatumReader<>(writerSchema, readerSchema, reflectData); } @Override diff --git a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/AvroSchemaTest.java b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/AvroSchemaTest.java index e0a6cdc54fc..7e7602594d1 100644 --- a/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/AvroSchemaTest.java +++ b/pulsar-client/src/test/java/org/apache/pulsar/client/impl/schema/AvroSchemaTest.java @@ -553,4 +553,44 @@ public class AvroSchemaTest { LocalDateTimePojo pojo = avroSchema.decode(bytes); assertEquals(pojo.getValue().truncatedTo(ChronoUnit.MILLIS), now.truncatedTo(ChronoUnit.MILLIS)); } + + public static class UuidLogicalTypeDto { + public UUID id; + public String name; + + public UuidLogicalTypeDto() { + } + + public UuidLogicalTypeDto(UUID id, String name) { + this.id = id; + this.name = name; + } + } + + @Test + public void testDecodeUuidLogicalTypeWithClass() { + UUID id = UUID.randomUUID(); + String name = "name"; + AvroSchema<UuidLogicalTypeDto> schema = AvroSchema.of(UuidLogicalTypeDto.class); + byte[] encoded = schema.encode(new UuidLogicalTypeDto(id, name)); + + UuidLogicalTypeDto decoded = schema.decode(encoded); + assertEquals(decoded.id, id); + assertEquals(decoded.name, name); + } + + @Test + public void testDecodeUuidLogicalTypeWithJsonDef() { + UUID id = UUID.randomUUID(); + String name = "name"; + AvroSchema<UuidLogicalTypeDto> schemaFromClass = AvroSchema.of(UuidLogicalTypeDto.class); + byte[] encoded = schemaFromClass.encode(new UuidLogicalTypeDto(id, name)); + String jsonDef = schemaFromClass.getSchemaInfo().getSchemaDefinition(); + + AvroSchema<UuidLogicalTypeDto> schemaFromJsonDef = AvroSchema.of( + SchemaDefinition.<UuidLogicalTypeDto>builder().withJsonDef(jsonDef).build()); + UuidLogicalTypeDto decoded = schemaFromJsonDef.decode(encoded); + assertEquals(decoded.id, id); + assertEquals(decoded.name, name); + } }
