[HUDI-1650] Custom avro kafka deserializer. (#2619)
* Custom avro kafka deserializer Co-authored-by: volodymyr.burenin <volodymyr.burenin@cloudkitchens.com> Co-authored-by: Sivabalan Narayanan <sivabala@uber.com>
This commit is contained in:
committed by
GitHub
parent
161d530f93
commit
900de34e45
@@ -373,4 +373,11 @@ object DataSourceWriteOptions {
|
||||
// Async Compaction - Enabled by default for MOR
|
||||
val ASYNC_COMPACT_ENABLE_OPT_KEY = "hoodie.datasource.compaction.async.enable"
|
||||
val DEFAULT_ASYNC_COMPACT_ENABLE_OPT_VAL = "true"
|
||||
|
||||
// Avro Kafka Source configs
|
||||
val KAFKA_AVRO_VALUE_DESERIALIZER = "hoodie.deltastreamer.source.kafka.value.deserializer.class"
|
||||
|
||||
// Schema provider class to be set to be used in custom kakfa deserializer
|
||||
val SCHEMA_PROVIDER_CLASS_PROP = "hoodie.deltastreamer.schemaprovider.class"
|
||||
|
||||
}
|
||||
|
||||
@@ -0,0 +1,91 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.utilities.deser;
|
||||
|
||||
import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
|
||||
import io.confluent.kafka.serializers.KafkaAvroDeserializer;
|
||||
import org.apache.avro.Schema;
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.kafka.common.errors.SerializationException;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Extending {@link KafkaAvroSchemaDeserializer} as we need to be able to inject reader schema during deserialization.
|
||||
*/
|
||||
public class KafkaAvroSchemaDeserializer extends KafkaAvroDeserializer {
|
||||
|
||||
private Schema sourceSchema;
|
||||
|
||||
public KafkaAvroSchemaDeserializer() {}
|
||||
|
||||
public KafkaAvroSchemaDeserializer(SchemaRegistryClient client, Map<String, ?> props) {
|
||||
super(client, props);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configure(Map<String, ?> configs, boolean isKey) {
|
||||
super.configure(configs, isKey);
|
||||
try {
|
||||
TypedProperties props = getConvertToTypedProperties(configs);
|
||||
String className = props.getString(DataSourceWriteOptions.SCHEMA_PROVIDER_CLASS_PROP());
|
||||
SchemaProvider schemaProvider = (SchemaProvider) ReflectionUtils.loadClass(className, props);
|
||||
sourceSchema = Objects.requireNonNull(schemaProvider).getSourceSchema();
|
||||
} catch (Throwable e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* We need to inject sourceSchema instead of reader schema during deserialization or later stages of the pipeline.
|
||||
*
|
||||
* @param includeSchemaAndVersion
|
||||
* @param topic
|
||||
* @param isKey
|
||||
* @param payload
|
||||
* @param readerSchema
|
||||
* @return
|
||||
* @throws SerializationException
|
||||
*/
|
||||
@Override
|
||||
protected Object deserialize(
|
||||
boolean includeSchemaAndVersion,
|
||||
String topic,
|
||||
Boolean isKey,
|
||||
byte[] payload,
|
||||
Schema readerSchema)
|
||||
throws SerializationException {
|
||||
return super.deserialize(includeSchemaAndVersion, topic, isKey, payload, sourceSchema);
|
||||
}
|
||||
|
||||
protected TypedProperties getConvertToTypedProperties(Map<String, ?> configs) {
|
||||
TypedProperties typedProperties = new TypedProperties();
|
||||
for (Entry<String, ?> entry : configs.entrySet()) {
|
||||
typedProperties.put(entry.getKey(), entry.getValue());
|
||||
}
|
||||
return typedProperties;
|
||||
}
|
||||
}
|
||||
@@ -38,6 +38,10 @@ public abstract class SchemaProvider implements Serializable {
|
||||
|
||||
protected JavaSparkContext jssc;
|
||||
|
||||
public SchemaProvider(TypedProperties props) {
|
||||
this(props, null);
|
||||
}
|
||||
|
||||
protected SchemaProvider(TypedProperties props, JavaSparkContext jssc) {
|
||||
this.config = props;
|
||||
this.jssc = jssc;
|
||||
|
||||
@@ -18,8 +18,11 @@
|
||||
|
||||
package org.apache.hudi.utilities.sources;
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamerMetrics;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
import org.apache.hudi.utilities.sources.helpers.KafkaOffsetGen;
|
||||
@@ -43,17 +46,36 @@ import org.apache.spark.streaming.kafka010.OffsetRange;
|
||||
public class AvroKafkaSource extends AvroSource {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(AvroKafkaSource.class);
|
||||
|
||||
// these are native kafka's config. do not change the config names.
|
||||
private static final String NATIVE_KAFKA_KEY_DESERIALIZER_PROP = "key.deserializer";
|
||||
private static final String NATIVE_KAFKA_VALUE_DESERIALIZER_PROP = "value.deserializer";
|
||||
private final KafkaOffsetGen offsetGen;
|
||||
|
||||
private final HoodieDeltaStreamerMetrics metrics;
|
||||
|
||||
public AvroKafkaSource(TypedProperties props, JavaSparkContext sparkContext, SparkSession sparkSession,
|
||||
SchemaProvider schemaProvider, HoodieDeltaStreamerMetrics metrics) {
|
||||
super(props, sparkContext, sparkSession, schemaProvider);
|
||||
|
||||
props.put(NATIVE_KAFKA_KEY_DESERIALIZER_PROP, StringDeserializer.class);
|
||||
String deserializerClassName = props.getString(DataSourceWriteOptions.KAFKA_AVRO_VALUE_DESERIALIZER(), "");
|
||||
|
||||
if (deserializerClassName.isEmpty()) {
|
||||
props.put(NATIVE_KAFKA_VALUE_DESERIALIZER_PROP, KafkaAvroDeserializer.class);
|
||||
} else {
|
||||
try {
|
||||
if (schemaProvider == null) {
|
||||
throw new HoodieIOException("SchemaProvider has to be set to use custom Deserializer");
|
||||
}
|
||||
props.put(DataSourceWriteOptions.SCHEMA_PROVIDER_CLASS_PROP(), schemaProvider.getClass().getName());
|
||||
props.put(NATIVE_KAFKA_VALUE_DESERIALIZER_PROP, Class.forName(deserializerClassName));
|
||||
} catch (ClassNotFoundException e) {
|
||||
String error = "Could not load custom avro kafka deserializer: " + deserializerClassName;
|
||||
LOG.error(error);
|
||||
throw new HoodieException(error, e);
|
||||
}
|
||||
}
|
||||
|
||||
this.metrics = metrics;
|
||||
props.put("key.deserializer", StringDeserializer.class);
|
||||
props.put("value.deserializer", KafkaAvroDeserializer.class);
|
||||
offsetGen = new KafkaOffsetGen(props);
|
||||
}
|
||||
|
||||
|
||||
@@ -0,0 +1,141 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.utilities.deser;
|
||||
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.utilities.sources.helpers.SchemaTestProvider;
|
||||
import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
|
||||
|
||||
import io.confluent.kafka.schemaregistry.client.MockSchemaRegistryClient;
|
||||
import io.confluent.kafka.schemaregistry.client.SchemaRegistryClient;
|
||||
import io.confluent.kafka.serializers.KafkaAvroDeserializerConfig;
|
||||
import io.confluent.kafka.serializers.KafkaAvroSerializer;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
|
||||
/**
|
||||
* Tests {@link KafkaAvroSchemaDeserializer}.
|
||||
*/
|
||||
public class TestKafkaAvroSchemaDeserializer extends UtilitiesTestBase {
|
||||
|
||||
private final SchemaRegistryClient schemaRegistry;
|
||||
private final KafkaAvroSerializer avroSerializer;
|
||||
private final String topic;
|
||||
private final Schema origSchema = createUserSchema();
|
||||
private final Schema evolSchema = createExtendUserSchema();
|
||||
private Properties defaultConfig = new Properties();
|
||||
|
||||
public TestKafkaAvroSchemaDeserializer() {
|
||||
defaultConfig.put(KafkaAvroDeserializerConfig.SCHEMA_REGISTRY_URL_CONFIG, "bogus");
|
||||
defaultConfig.put("hoodie.deltastreamer.schemaprovider.class", SchemaTestProvider.class.getName());
|
||||
schemaRegistry = new MockSchemaRegistryClient();
|
||||
avroSerializer = new KafkaAvroSerializer(schemaRegistry, new HashMap(defaultConfig));
|
||||
topic = "test";
|
||||
}
|
||||
|
||||
private Schema createUserSchema() {
|
||||
String userSchema = "{\"namespace\": \"example.avro\", \"type\": \"record\", "
|
||||
+ "\"name\": \"User\","
|
||||
+ "\"fields\": [{\"name\": \"name\", \"type\": \"string\"}]}";
|
||||
Schema.Parser parser = new Schema.Parser();
|
||||
Schema schema = parser.parse(userSchema);
|
||||
return schema;
|
||||
}
|
||||
|
||||
private IndexedRecord createUserRecord() {
|
||||
Schema schema = createUserSchema();
|
||||
GenericRecord avroRecord = new GenericData.Record(schema);
|
||||
avroRecord.put("name", "testUser");
|
||||
return avroRecord;
|
||||
}
|
||||
|
||||
private Schema createExtendUserSchema() {
|
||||
String userSchema = "{\"namespace\": \"example.avro\", \"type\": \"record\", "
|
||||
+ "\"name\": \"User\","
|
||||
+ "\"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;
|
||||
}
|
||||
|
||||
private IndexedRecord createExtendUserRecord() {
|
||||
Schema schema = createExtendUserSchema();
|
||||
GenericRecord avroRecord = new GenericData.Record(schema);
|
||||
avroRecord.put("name", "testUser");
|
||||
avroRecord.put("age", 30);
|
||||
return avroRecord;
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests {@link KafkaAvroSchemaDeserializer#deserialize(boolean, String, Boolean, byte[], Schema)}.
|
||||
*/
|
||||
@Test
|
||||
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);
|
||||
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));
|
||||
|
||||
IndexedRecord avroRecordWithAllField = createExtendUserRecord();
|
||||
byte[] bytesExtendedRecord = avroSerializer.serialize(topic, avroRecordWithAllField);
|
||||
|
||||
SchemaTestProvider.schemaToReturn.set(evolSchema);
|
||||
avroDeserializer = new KafkaAvroSchemaDeserializer(schemaRegistry, new HashMap(defaultConfig));
|
||||
avroDeserializer.configure(new HashMap(defaultConfig), 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);
|
||||
assertEquals(avroRecordWithAllFieldActual.getSchema(), evolSchema);
|
||||
|
||||
// read old record w/ evolved schema.
|
||||
IndexedRecord actualRec = (IndexedRecord) avroDeserializer.deserialize(false, topic, false, bytesOrigRecord, origSchema);
|
||||
// record won't be equal to original record as we read w/ evolved schema. "age" will be added w/ default value of null
|
||||
assertNotEquals(avroRecord, actualRec);
|
||||
GenericRecord genericRecord = (GenericRecord) actualRec;
|
||||
GenericRecord origGenRec = (GenericRecord) avroRecord;
|
||||
assertEquals(genericRecord.get("name").toString(), origGenRec.get("name").toString());
|
||||
assertEquals(actualRec.getSchema(), evolSchema);
|
||||
assertNull(genericRecord.get("age"));
|
||||
}
|
||||
|
||||
protected TypedProperties getConvertToTypedProperties(Map<String, ?> configs) {
|
||||
TypedProperties typedProperties = new TypedProperties();
|
||||
for (Entry<String, ?> entry : configs.entrySet()) {
|
||||
typedProperties.put(entry.getKey(), entry.getValue());
|
||||
}
|
||||
return typedProperties;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,45 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.utilities.sources.helpers;
|
||||
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
* {@link SchemaProvider} for tests.
|
||||
*/
|
||||
public class SchemaTestProvider extends SchemaProvider {
|
||||
|
||||
public static AtomicReference<Schema> schemaToReturn = new AtomicReference<>(HoodieTestDataGenerator.AVRO_SCHEMA);
|
||||
|
||||
public SchemaTestProvider(TypedProperties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Schema getSourceSchema() {
|
||||
return schemaToReturn.get();
|
||||
}
|
||||
|
||||
}
|
||||
Reference in New Issue
Block a user