nsivabalan commented on a change in pull request #2598:
URL: https://github.com/apache/hudi/pull/2598#discussion_r583833111
##########
File path:
hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/SchemaRegistryProvider.java
##########
@@ -58,30 +66,67 @@ private static String fetchSchemaFromRegistry(String
registryUrl) throws IOExcep
public SchemaRegistryProvider(TypedProperties props, JavaSparkContext jssc) {
super(props, jssc);
DataSourceUtils.checkRequiredProperties(props,
Collections.singletonList(Config.SRC_SCHEMA_REGISTRY_URL_PROP));
+ this.cacheDisabled = !props.getBoolean(Config.CACHE_SCHEMAS, false);
+ this.injectKafkaFieldSchema =
props.getBoolean(AvroKafkaSourceHelpers.INJECT_KAFKA_FIELDS, false);
+ this.registryUrl = config.getString(Config.SRC_SCHEMA_REGISTRY_URL_PROP);
+ this.targetRegistryUrl =
config.getString(Config.TARGET_SCHEMA_REGISTRY_URL_PROP, registryUrl);
+ this.noTargetSchema = targetRegistryUrl.equals("null");
}
- private static Schema getSchema(String registryUrl) throws IOException {
- return new Schema.Parser().parse(fetchSchemaFromRegistry(registryUrl));
+ private static Schema getSchema(String registryUrl, boolean
injectKafkaFieldSchema) throws IOException {
+ Schema schema = new
Schema.Parser().parse(fetchSchemaFromRegistry(registryUrl));
+ if (injectKafkaFieldSchema) {
+ return AvroKafkaSourceHelpers.addKafkaMetadataFields(schema);
+ }
+ return schema;
}
@Override
public Schema getSourceSchema() {
- String registryUrl = config.getString(Config.SRC_SCHEMA_REGISTRY_URL_PROP);
+ if (cacheDisabled) {
+ return getSourceSchemaFromRegistry();
+ }
+ if (sourceSchema == null) {
+ synchronized (this) {
+ if (sourceSchema == null) {
+ sourceSchema = getSourceSchemaFromRegistry();
+ }
+ }
+ }
+ return sourceSchema;
+ }
+
+ @Override
+ public Schema getTargetSchema() {
+ if (noTargetSchema) {
+ return null;
+ }
+ if (cacheDisabled) {
+ return getTargetSchemaFromRegistry();
+ }
+ if (targetSchema == null) {
+ synchronized (this) {
+ if (targetSchema == null) {
+ targetSchema = getTargetSchemaFromRegistry();
+ }
+ }
+ }
+ return targetSchema;
+ }
+
+ private Schema getSourceSchemaFromRegistry() {
try {
- return getSchema(registryUrl);
+ return getSchema(registryUrl, injectKafkaFieldSchema);
Review comment:
I get it now. But clarify me something.
```
return KafkaUtils.createRDD(sparkContext, offsetGen.getKafkaParams(),
offsetRanges,
LocationStrategies.PreferConsistent()).map(AvroKafkaSourceHelpers::addKafkaFields);
```
What I infer from this is, first we read data from kafka source (using
custom deser class) and then we do mapping to Generic object by adding kafka
meta fields.
To Custom deser, we pass in source schema which has meta fields added. but
actual kafka source data won't have these meta fields right?
Once we return from this method, within sourceFormatAdaptor, I see we
leverage sourceSchema (in which we are expected to have meta fields). Here I
agree we need to have kafka meta fields in source schema.
But not sure, if the source schema used in deser class can have kafka meta
fields.
Can you help me comprehend.
Or can you help me understand where/when exactly
AbstractKafkaAvroDeserializer.deserialize() is called in this flow.
sorry I haven't worked w/ kafka source in hudi. all my understanding is just
by reading the code :)
----------------------------------------------------------------
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]