1
0

[HUDI-2069] Fix KafkaAvroSchemaDeserializer to not rely on reflection (#3111)

[HUDI-2069] KafkaAvroSchemaDeserializer should get sourceSchema passed instead using Reflection
This commit is contained in:
Sebastian Bernauer
2021-06-24 15:08:21 +02:00
committed by GitHub
parent 84dd3ca18b
commit b32855545b
5 changed files with 20 additions and 26 deletions

View File

@@ -51,13 +51,12 @@ public class TestKafkaAvroSchemaDeserializer extends UtilitiesTestBase {
private final String topic;
private final Schema origSchema = createUserSchema();
private final Schema evolSchema = createExtendUserSchema();
private Properties defaultConfig = new Properties();
private Properties config = new Properties();
public TestKafkaAvroSchemaDeserializer() {
defaultConfig.put(KafkaAvroDeserializerConfig.SCHEMA_REGISTRY_URL_CONFIG, "bogus");
defaultConfig.put("hoodie.deltastreamer.schemaprovider.class", SchemaTestProvider.class.getName());
config.put(KafkaAvroDeserializerConfig.SCHEMA_REGISTRY_URL_CONFIG, "bogus");
schemaRegistry = new MockSchemaRegistryClient();
avroSerializer = new KafkaAvroSerializer(schemaRegistry, new HashMap(defaultConfig));
avroSerializer = new KafkaAvroSerializer(schemaRegistry, new HashMap(config));
topic = "test";
}
@@ -66,8 +65,7 @@ public class TestKafkaAvroSchemaDeserializer extends UtilitiesTestBase {
+ "\"name\": \"User\","
+ "\"fields\": [{\"name\": \"name\", \"type\": \"string\"}]}";
Schema.Parser parser = new Schema.Parser();
Schema schema = parser.parse(userSchema);
return schema;
return parser.parse(userSchema);
}
private IndexedRecord createUserRecord() {
@@ -83,8 +81,7 @@ public class TestKafkaAvroSchemaDeserializer extends UtilitiesTestBase {
+ "\"fields\": [{\"name\": \"name\", \"type\": \"string\"}, "
+ "{\"name\": \"age\", \"type\": [\"null\", \"int\"], \"default\": null}]}";
Schema.Parser parser = new Schema.Parser();
Schema schema = parser.parse(userSchema);
return schema;
return parser.parse(userSchema);
}
private IndexedRecord createExtendUserRecord() {
@@ -102,9 +99,10 @@ public class TestKafkaAvroSchemaDeserializer extends UtilitiesTestBase {
public void testKafkaAvroSchemaDeserializer() {
byte[] bytesOrigRecord;
IndexedRecord avroRecord = createUserRecord();
SchemaTestProvider.schemaToReturn.set(origSchema);
KafkaAvroSchemaDeserializer avroDeserializer = new KafkaAvroSchemaDeserializer(schemaRegistry, new HashMap(defaultConfig));
avroDeserializer.configure(new HashMap(defaultConfig), false);
config.put("hoodie.deltastreamer.source.kafka.value.deserializer.schema", origSchema.toString());
KafkaAvroSchemaDeserializer avroDeserializer = new KafkaAvroSchemaDeserializer(schemaRegistry, new HashMap(config));
avroDeserializer.configure(new HashMap(config), false);
bytesOrigRecord = avroSerializer.serialize(topic, avroRecord);
// record is serialized in orig schema and deserialized using same schema.
assertEquals(avroRecord, avroDeserializer.deserialize(false, topic, false, bytesOrigRecord, origSchema));
@@ -113,8 +111,9 @@ public class TestKafkaAvroSchemaDeserializer extends UtilitiesTestBase {
byte[] bytesExtendedRecord = avroSerializer.serialize(topic, avroRecordWithAllField);
SchemaTestProvider.schemaToReturn.set(evolSchema);
avroDeserializer = new KafkaAvroSchemaDeserializer(schemaRegistry, new HashMap(defaultConfig));
avroDeserializer.configure(new HashMap(defaultConfig), false);
config.put("hoodie.deltastreamer.source.kafka.value.deserializer.schema", evolSchema.toString());
avroDeserializer = new KafkaAvroSchemaDeserializer(schemaRegistry, new HashMap(config));
avroDeserializer.configure(new HashMap(config), false);
// record is serialized w/ evolved schema, and deserialized w/ evolved schema
IndexedRecord avroRecordWithAllFieldActual = (IndexedRecord) avroDeserializer.deserialize(false, topic, false, bytesExtendedRecord, evolSchema);
assertEquals(avroRecordWithAllField, avroRecordWithAllFieldActual);