nsivabalan commented on a change in pull request #2619: URL: https://github.com/apache/hudi/pull/2619#discussion_r590421462
########## File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deser/KafkaAvroSchemaDeserializer.java ########## @@ -0,0 +1,84 @@ +/* + * 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.serializers.KafkaAvroDeserializer; +import org.apache.avro.Schema; +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 static final String SCHEMA_PROVIDER_CLASS_PROP = "hoodie.deltastreamer.schemaprovider.class"; + protected Schema sourceSchema; + + public KafkaAvroSchemaDeserializer() {} + + @Override + public void configure(Map<String, ?> configs, boolean isKey) { + super.configure(configs, isKey); + try { + TypedProperties props = getConvertToTypedProperties(configs); + String className = props.getString(SCHEMA_PROVIDER_CLASS_PROP); + SchemaProvider schemaProvider = (SchemaProvider) ReflectionUtils.loadClass(className, props); Review comment: may I know if this will work for any schemaProvider? or are we making any assumptions? Bcoz, as of now, guess we instantiate schemaProvider for DeltaStreamer in UtilHelpers.createSchemaProvider(className, props, jsc). ########## File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/deser/KafkaAvroSchemaDeserializer.java ########## @@ -0,0 +1,84 @@ +/* + * 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.serializers.KafkaAvroDeserializer; +import org.apache.avro.Schema; +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 static final String SCHEMA_PROVIDER_CLASS_PROP = "hoodie.deltastreamer.schemaprovider.class"; Review comment: May be we should try to re-use HoodieDeltaStreamer.Config.schemaProviderClassName. ########## File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java ########## @@ -42,18 +43,31 @@ */ public class AvroKafkaSource extends AvroSource { + private static final String KAFKA_AVRO_VALUE_DESERIALIZER = "hoodie.deltastreamer.source.kafka.value.deserializer.class"; private static final Logger LOG = LogManager.getLogger(AvroKafkaSource.class); - 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); - this.metrics = metrics; + props.put("key.deserializer", StringDeserializer.class); - props.put("value.deserializer", KafkaAvroDeserializer.class); + String deserializerClassName = props.getString(KAFKA_AVRO_VALUE_DESERIALIZER, ""); Review comment: may be we should add default value to this config similar to others configs in HoodieWriteClientConfig. ########## File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java ########## @@ -42,18 +43,31 @@ */ public class AvroKafkaSource extends AvroSource { + private static final String KAFKA_AVRO_VALUE_DESERIALIZER = "hoodie.deltastreamer.source.kafka.value.deserializer.class"; private static final Logger LOG = LogManager.getLogger(AvroKafkaSource.class); - 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); - this.metrics = metrics; Review comment: nit. why moved this later. ########## File path: hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/AvroKafkaSource.java ########## @@ -42,18 +43,31 @@ */ public class AvroKafkaSource extends AvroSource { + private static final String KAFKA_AVRO_VALUE_DESERIALIZER = "hoodie.deltastreamer.source.kafka.value.deserializer.class"; Review comment: @n3nash : do you know whats the right place to store these configs for sources? ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
