Skip to content

Commit 209efcc

Browse files
Issue 149: Change AvroDeserializer deserialize method to reduce performance overhead on DatumReader creation (#151)
* [changed] AvroDeserializer deserialize method to reduce performance overhead on DatumReader creation Signed-off-by: jingerbread <[email protected]> * Issue 149: Change AvroDeserializer deserialize method to reduce performance overhead on DatumReader creation [updated] GenericAvroDeserializer to avoid creating data readers [used] computeIfAbsent Signed-off-by: jingerbread <[email protected]> * [removed] getKnownSchemaReaders() Signed-off-by: jingerbread <[email protected]> * [changed] visibility of createDatumReader and getKnownSchemaReaders methods for unit test [added] unit test for AvroDeserializer Signed-off-by: jingerbread <[email protected]> * [fixed] failing gradle check for import of io.pravega.schemaregistry.serializers.SerializerFactory Signed-off-by: jingerbread <[email protected]> * [added] licencing headers [removed] Preconditions.checkNotNull(readerSchemaInfo) for AvroGenericDeserializer Signed-off-by: jingerbread <[email protected]> Co-authored-by: shivesh ranjan <[email protected]>
1 parent 9f207de commit 209efcc

File tree

11 files changed

+3093
-29
lines changed

11 files changed

+3093
-29
lines changed

serializers/avro/src/main/java/io/pravega/schemaregistry/serializer/avro/impl/AvroDeserializer.java

+32-20
Original file line numberDiff line numberDiff line change
@@ -9,8 +9,9 @@
99
*/
1010
package io.pravega.schemaregistry.serializer.avro.impl;
1111

12-
import com.google.common.base.Charsets;
12+
import com.google.common.annotations.VisibleForTesting;
1313
import com.google.common.base.Preconditions;
14+
import com.google.common.collect.ImmutableMap;
1415
import io.pravega.schemaregistry.serializer.avro.schemas.AvroSchema;
1516
import io.pravega.schemaregistry.client.SchemaRegistryClient;
1617
import io.pravega.schemaregistry.contract.data.SchemaInfo;
@@ -19,48 +20,59 @@
1920
import io.pravega.schemaregistry.serializer.shared.impl.SerializerConfig;
2021
import org.apache.avro.Schema;
2122
import org.apache.avro.io.BinaryDecoder;
23+
import org.apache.avro.io.DatumReader;
2224
import org.apache.avro.io.DecoderFactory;
2325
import org.apache.avro.reflect.ReflectDatumReader;
2426
import org.apache.avro.specific.SpecificDatumReader;
2527
import org.apache.avro.specific.SpecificRecordBase;
2628

2729
import java.io.IOException;
2830
import java.io.InputStream;
31+
import java.nio.ByteBuffer;
2932
import java.util.concurrent.ConcurrentHashMap;
3033

3134
class AvroDeserializer<T> extends AbstractDeserializer<T> {
32-
private final AvroSchema<T> avroSchema;
33-
private final ConcurrentHashMap<SchemaInfo, Schema> knownSchemas;
35+
private final ConcurrentHashMap<ByteBuffer, DatumReader<T>> knownSchemaReaders;
36+
private final boolean specific;
37+
private final Schema readerSchema;
3438

3539
AvroDeserializer(String groupId, SchemaRegistryClient client,
3640
AvroSchema<T> schema,
3741
SerializerConfig.Decoders decoder, EncodingCache encodingCache) {
3842
super(groupId, client, schema, false, decoder, encodingCache, true);
3943
Preconditions.checkNotNull(schema);
40-
this.avroSchema = schema;
41-
this.knownSchemas = new ConcurrentHashMap<>();
44+
this.knownSchemaReaders = new ConcurrentHashMap<>();
45+
specific = SpecificRecordBase.class.isAssignableFrom(schema.getTClass());
46+
readerSchema = schema.getSchema();
47+
ByteBuffer schemaData = schema.getSchemaInfo().getSchemaData();
48+
knownSchemaReaders.put(schemaData, createDatumReader(readerSchema, readerSchema, specific));
4249
}
4350

4451
@Override
4552
public final T deserialize(InputStream inputStream, SchemaInfo writerSchemaInfo, SchemaInfo readerSchemaInfo) throws IOException {
4653
Preconditions.checkNotNull(writerSchemaInfo);
47-
Schema writerSchema;
48-
if (knownSchemas.containsKey(writerSchemaInfo)) {
49-
writerSchema = knownSchemas.get(writerSchemaInfo);
50-
} else {
51-
String schemaString = new String(writerSchemaInfo.getSchemaData().array(), Charsets.UTF_8);
52-
writerSchema = new Schema.Parser().parse(schemaString);
53-
knownSchemas.put(writerSchemaInfo, writerSchema);
54-
}
55-
Schema readerSchema = avroSchema.getSchema();
54+
final ByteBuffer writerSchemaData = writerSchemaInfo.getSchemaData();
55+
DatumReader<T> datumReader = knownSchemaReaders.computeIfAbsent(writerSchemaData, key -> {
56+
Schema writerSchema = AvroSchema.from(writerSchemaInfo).getSchema();
57+
return createDatumReader(writerSchema, this.readerSchema, specific);
58+
});
5659
BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
57-
58-
if (SpecificRecordBase.class.isAssignableFrom(avroSchema.getTClass())) {
59-
SpecificDatumReader<T> datumReader = new SpecificDatumReader<>(writerSchema, readerSchema);
60-
return datumReader.read(null, decoder);
60+
return datumReader.read(null, decoder);
61+
}
62+
63+
@VisibleForTesting
64+
DatumReader<T> createDatumReader(Schema writerSchema, Schema readerSchema, boolean specific) {
65+
DatumReader<T> datumReader;
66+
if (specific) {
67+
datumReader = new SpecificDatumReader<>(writerSchema, readerSchema);
6168
} else {
62-
ReflectDatumReader<T> datumReader = new ReflectDatumReader<>(writerSchema, readerSchema);
63-
return datumReader.read(null, decoder);
69+
datumReader = new ReflectDatumReader<>(writerSchema, readerSchema);
6470
}
71+
return datumReader;
72+
}
73+
74+
@VisibleForTesting
75+
ImmutableMap<ByteBuffer, DatumReader<T>> getKnownSchemaReaders() {
76+
return ImmutableMap.copyOf(knownSchemaReaders);
6577
}
6678
}

serializers/avro/src/main/java/io/pravega/schemaregistry/serializer/avro/impl/AvroGenericDeserializer.java

+17-9
Original file line numberDiff line numberDiff line change
@@ -9,41 +9,49 @@
99
*/
1010
package io.pravega.schemaregistry.serializer.avro.impl;
1111

12+
import com.google.common.annotations.VisibleForTesting;
1213
import com.google.common.base.Preconditions;
13-
import io.pravega.schemaregistry.serializer.avro.schemas.AvroSchema;
14+
import com.google.common.collect.ImmutableMap;
1415
import io.pravega.schemaregistry.client.SchemaRegistryClient;
1516
import io.pravega.schemaregistry.contract.data.SchemaInfo;
17+
import io.pravega.schemaregistry.serializer.avro.schemas.AvroSchema;
1618
import io.pravega.schemaregistry.serializer.shared.impl.AbstractDeserializer;
1719
import io.pravega.schemaregistry.serializer.shared.impl.EncodingCache;
1820
import io.pravega.schemaregistry.serializer.shared.impl.SerializerConfig;
1921
import org.apache.avro.Schema;
2022
import org.apache.avro.generic.GenericDatumReader;
2123
import org.apache.avro.io.BinaryDecoder;
2224
import org.apache.avro.io.DecoderFactory;
23-
25+
import org.apache.commons.lang3.tuple.Pair;
2426
import javax.annotation.Nullable;
2527
import java.io.IOException;
2628
import java.io.InputStream;
2729
import java.util.concurrent.ConcurrentHashMap;
2830

2931
public class AvroGenericDeserializer extends AbstractDeserializer<Object> {
30-
private final ConcurrentHashMap<SchemaInfo, Schema> knownSchemas;
32+
private final ConcurrentHashMap<Pair<SchemaInfo, SchemaInfo>, GenericDatumReader<Object>> knownSchemaReaders;
3133

3234
public AvroGenericDeserializer(String groupId, SchemaRegistryClient client, @Nullable AvroSchema<Object> schema,
3335
SerializerConfig.Decoders decoder, EncodingCache encodingCache) {
3436
super(groupId, client, schema, false, decoder, encodingCache, true);
35-
this.knownSchemas = new ConcurrentHashMap<>();
37+
this.knownSchemaReaders = new ConcurrentHashMap<>();
3638
}
3739

3840
@Override
3941
public final Object deserialize(InputStream inputStream, SchemaInfo writerSchemaInfo, SchemaInfo readerSchemaInfo) throws IOException {
4042
Preconditions.checkNotNull(writerSchemaInfo);
41-
Schema writerSchema = knownSchemas.computeIfAbsent(writerSchemaInfo, x -> AvroSchema.from(x).getSchema());
42-
Schema readerSchema = knownSchemas.computeIfAbsent(readerSchemaInfo, x -> AvroSchema.from(x).getSchema());
43-
44-
GenericDatumReader<Object> genericDatumReader = new GenericDatumReader<>(writerSchema, readerSchema);
45-
43+
final Pair<SchemaInfo, SchemaInfo> keyPair = Pair.of(writerSchemaInfo, readerSchemaInfo);
44+
GenericDatumReader<Object> genericDatumReader = knownSchemaReaders.computeIfAbsent(keyPair, key -> {
45+
Schema writerSchema = AvroSchema.from(writerSchemaInfo).getSchema();
46+
Schema readerSchema = AvroSchema.from(readerSchemaInfo).getSchema();
47+
return new GenericDatumReader<>(writerSchema, readerSchema);
48+
});
4649
BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
4750
return genericDatumReader.read(null, decoder);
4851
}
52+
53+
@VisibleForTesting
54+
ImmutableMap<Pair<SchemaInfo, SchemaInfo>, GenericDatumReader<Object>> getKnownSchemaReaders() {
55+
return ImmutableMap.copyOf(knownSchemaReaders);
56+
}
4957
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,123 @@
1+
/**
2+
* Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved.
3+
*
4+
* Licensed under the Apache License, Version 2.0 (the "License");
5+
* you may not use this file except in compliance with the License.
6+
* You may obtain a copy of the License at
7+
*
8+
* http://www.apache.org/licenses/LICENSE-2.0
9+
*/
10+
package io.pravega.schemaregistry.serializer.avro.impl;
11+
12+
import com.google.common.collect.ImmutableMap;
13+
import io.pravega.client.stream.Serializer;
14+
import io.pravega.schemaregistry.client.SchemaRegistryClient;
15+
import io.pravega.schemaregistry.contract.data.*;
16+
import io.pravega.schemaregistry.serializer.avro.schemas.AvroSchema;
17+
import io.pravega.schemaregistry.serializer.avro.testobjs.generated.avro.AddressEntry;
18+
import io.pravega.schemaregistry.serializer.avro.testobjs.generated.avro.User;
19+
import io.pravega.schemaregistry.serializer.shared.codec.Codecs;
20+
import io.pravega.schemaregistry.serializer.shared.impl.SerializerConfig;
21+
import lombok.extern.slf4j.Slf4j;
22+
import org.apache.avro.generic.GenericDatumReader;
23+
import org.apache.avro.io.DatumReader;
24+
import org.apache.commons.lang3.tuple.Pair;
25+
import org.junit.Assert;
26+
import org.junit.Before;
27+
import org.junit.Test;
28+
import org.mockito.Mockito;
29+
import java.nio.ByteBuffer;
30+
import java.nio.charset.StandardCharsets;
31+
32+
import static org.mockito.ArgumentMatchers.*;
33+
import static org.mockito.Mockito.doAnswer;
34+
import static org.mockito.Mockito.mock;
35+
36+
@Slf4j
37+
public class AvroDeserializerTest {
38+
39+
private Serializer<User> serializer;
40+
private AvroDeserializer<User> avroDeserializer;
41+
private AvroGenericDeserializer genericDeserializer;
42+
private User user;
43+
44+
@Before
45+
public void init() {
46+
AvroSchema<User> userAvroSchema = AvroSchema.of(User.class);
47+
log.info("Aliases: {}", userAvroSchema.getSchema().getAliases());
48+
VersionInfo versionInfo1 = new VersionInfo("avroUser1", 0, 0);
49+
SchemaRegistryClient client = mock(SchemaRegistryClient.class);
50+
doAnswer(x -> true).when(client).canReadUsing(anyString(), any());
51+
doAnswer(x -> new EncodingId(0)).when(client).getEncodingId(anyString(), any(), any());
52+
doAnswer(x -> new EncodingInfo(versionInfo1, userAvroSchema.getSchemaInfo(), Codecs.None.getCodec().getCodecType())).when(client).getEncodingInfo(anyString(), eq(new EncodingId(0)));
53+
SerializerConfig serializerConfig = SerializerConfig.builder().registryClient(client).groupId("avroUser1")
54+
.createGroup(SerializationFormat.Avro).registerSchema(true).build();
55+
this.serializer = AvroSerializerFactory
56+
.serializer(serializerConfig, userAvroSchema);
57+
this.avroDeserializer = Mockito.spy((AvroDeserializer<User>)AvroSerializerFactory.deserializer(
58+
serializerConfig, userAvroSchema));
59+
60+
org.apache.avro.Schema schema = userAvroSchema.getSchema();
61+
AvroSchema<Object> objectAvroSchema = AvroSchema.of(schema);
62+
this.genericDeserializer = Mockito.spy((AvroGenericDeserializer)AvroSerializerFactory.genericDeserializer(
63+
serializerConfig, objectAvroSchema));
64+
65+
this.user = User.newBuilder()
66+
.setUserId("111111111111")
67+
.setBiography("Greg Egan was born 20 August 1961")
68+
.setName("Greg Egan")
69+
.setEventTimestamp(System.currentTimeMillis())
70+
.setKeyValues(null)
71+
.setKeyValues2(null)
72+
.setKeyValues3(null)
73+
.setAddress(AddressEntry.newBuilder().setCity("Perth")
74+
.setPostalCode(5018)
75+
.setStreetAddress("4/19 Gardner Road").build()).build();
76+
}
77+
78+
@Test
79+
public void testCreatingReadersOnceForSchemaGeneric() {
80+
ImmutableMap<Pair<SchemaInfo, SchemaInfo>, GenericDatumReader<Object>> knownSchemaReaders1 = genericDeserializer.getKnownSchemaReaders();
81+
Assert.assertTrue(knownSchemaReaders1.isEmpty());
82+
Assert.assertEquals(0, knownSchemaReaders1.size());
83+
84+
ByteBuffer serialized = serializer.serialize(user);
85+
int payloadSize = serialized.limit();
86+
log.info("serialized into {}", payloadSize);
87+
Assert.assertEquals(100, payloadSize);
88+
byte[] bytes = serialized.array();
89+
log.info("bytes: {}", new String(bytes, StandardCharsets.UTF_8));
90+
Object user1 = genericDeserializer.deserialize(ByteBuffer.wrap(bytes));
91+
log.info("deserialized {}", user1);
92+
ImmutableMap<Pair<SchemaInfo, SchemaInfo>, GenericDatumReader<Object>> knownSchemaReaders2 = genericDeserializer.getKnownSchemaReaders();
93+
Assert.assertEquals(1, knownSchemaReaders2.size());
94+
}
95+
96+
@Test
97+
public void testCreatingReadersOnceForSchema() {
98+
ImmutableMap<ByteBuffer, DatumReader<User>> knownSchemaReaders1 = avroDeserializer.getKnownSchemaReaders();
99+
Assert.assertFalse(knownSchemaReaders1.isEmpty());
100+
Assert.assertEquals(1, knownSchemaReaders1.size());
101+
AvroSchema<User> userAvroSchema = AvroSchema.of(User.class);
102+
DatumReader<User> datumReader = knownSchemaReaders1.get(userAvroSchema.getSchemaInfo().getSchemaData());
103+
Assert.assertNotNull(datumReader);
104+
105+
ByteBuffer serialized = serializer.serialize(user);
106+
int payloadSize = serialized.limit();
107+
log.info("serialized into {}", payloadSize);
108+
Assert.assertEquals(100, payloadSize);
109+
byte[] bytes = serialized.array();
110+
log.info("bytes: {}", new String(bytes, StandardCharsets.UTF_8));
111+
User user1 = avroDeserializer.deserialize(ByteBuffer.wrap(bytes));
112+
113+
log.info("deserialized {}", user1);
114+
Assert.assertEquals(user, user1);
115+
serializer.serialize(user1);
116+
ImmutableMap<ByteBuffer, DatumReader<User>> knownSchemaReaders2 = avroDeserializer.getKnownSchemaReaders();
117+
Assert.assertEquals(1, knownSchemaReaders2.size());
118+
Assert.assertEquals(knownSchemaReaders1, knownSchemaReaders2);
119+
// called zero times outside constructor
120+
Mockito.verify(avroDeserializer, Mockito.times(0)).createDatumReader(Mockito.any(), Mockito.any(), Mockito.anyBoolean());
121+
}
122+
123+
}

serializers/avro/src/test/java/io/pravega/schemaregistry/serializer/avro/impl/SerializerTest.java

+1
Original file line numberDiff line numberDiff line change
@@ -9,6 +9,7 @@
99
*/
1010
package io.pravega.schemaregistry.serializer.avro.impl;
1111

12+
1213
import io.pravega.client.stream.Serializer;
1314
import io.pravega.schemaregistry.serializer.avro.schemas.AvroSchema;
1415
import io.pravega.schemaregistry.client.SchemaRegistryClient;
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,16 @@
1+
/**
2+
* Copyright (c) Dell Inc., or its subsidiaries. All Rights Reserved.
3+
*
4+
* Licensed under the Apache License, Version 2.0 (the "License");
5+
* you may not use this file except in compliance with the License.
6+
* You may obtain a copy of the License at
7+
*
8+
* http://www.apache.org/licenses/LICENSE-2.0
9+
*/
10+
package io.pravega.schemaregistry.serializer.avro.testobjs;
11+
12+
public interface EventTimestampAware {
13+
void setEventTimestamp(Long value);
14+
15+
Long getEventTimestamp();
16+
}

0 commit comments

Comments
 (0)