Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[HUDI-7108] Fix schema refresh for KafkaAvroSchemaDeserializer #10118

Merged
merged 1 commit into from
Nov 20, 2023
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.hudi.utilities.sources;

import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.utilities.UtilHelpers;
import org.apache.hudi.utilities.deser.KafkaAvroSchemaDeserializer;
import org.apache.hudi.utilities.exception.HoodieReadFromSourceException;
Expand Down Expand Up @@ -78,18 +79,25 @@ public AvroKafkaSource(TypedProperties props, JavaSparkContext sparkContext, Spa

try {
props.put(NATIVE_KAFKA_VALUE_DESERIALIZER_PROP, Class.forName(deserializerClassName).getName());
if (deserializerClassName.equals(KafkaAvroSchemaDeserializer.class.getName())) {
if (schemaProvider == null) {
throw new HoodieReadFromSourceException("SchemaProvider has to be set to use KafkaAvroSchemaDeserializer");
}
props.put(KAFKA_VALUE_DESERIALIZER_SCHEMA.key(), schemaProvider.getSourceSchema().toString());
}
} catch (ClassNotFoundException e) {
String error = "Could not load custom avro kafka deserializer: " + deserializerClassName;
LOG.error(error);
throw new HoodieReadFromSourceException(error, e);
}
this.offsetGen = new KafkaOffsetGen(props);

if (deserializerClassName.equals(KafkaAvroSchemaDeserializer.class.getName())) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we move all these to a private method named refreshPropsAndOffsetGen

if (deserializerClassName.equals(KafkaAvroSchemaDeserializer.class.getName())) {
      configureSchemaDeserializer();
    }
    offsetGen = new KafkaOffsetGen(props);

so that we can call here in L88 and in L 96.

configureSchemaDeserializer();
}
offsetGen = new KafkaOffsetGen(props);
}

@Override
protected InputBatch<JavaRDD<GenericRecord>> fetchNewData(Option<String> lastCheckpointStr, long sourceLimit) {
if (deserializerClassName.equals(KafkaAvroSchemaDeserializer.class.getName())) {
configureSchemaDeserializer();
offsetGen = new KafkaOffsetGen(props);
}
return super.fetchNewData(lastCheckpointStr, sourceLimit);
}

@Override
Expand Down Expand Up @@ -121,4 +129,11 @@ protected JavaRDD<GenericRecord> maybeAppendKafkaOffsets(JavaRDD<ConsumerRecord<
return kafkaRDD.map(consumerRecord -> (GenericRecord) consumerRecord.value());
}
}

private void configureSchemaDeserializer() {
if (schemaProvider == null) {
throw new HoodieReadFromSourceException("SchemaProvider has to be set to use KafkaAvroSchemaDeserializer");
}
props.put(KAFKA_VALUE_DESERIALIZER_SCHEMA.key(), schemaProvider.getSourceSchema().toString());
}
}
Loading