vburenin commented on a change in pull request #2598:
URL: https://github.com/apache/hudi/pull/2598#discussion_r583777748



##########
File path: 
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/AvroKafkaSourceHelpers.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.avro.Schema;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
+public class AvroKafkaSourceHelpers {
+
+  public static final String INJECT_KAFKA_FIELDS = 
"hoodie.deltastreamer.source.inject_kafka_fields";
+
+  public static final String KAFKA_PARTITION = "_hudi_kafka_partition";

Review comment:
       +1, we also can make it configurable.

##########
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:
       There is a trick that I am utilizing such as injection of kafka meta 
data into incoming message and source schema when it is decoded. So it is part 
of the source schema. The target schema gets modified appropriately, or gets 
inferred if the target schema is "null".
   ```java
   if (injectKafkaData) {
         return KafkaUtils.createRDD(sparkContext, offsetGen.getKafkaParams(), 
offsetRanges,
             
LocationStrategies.PreferConsistent()).map(AvroKafkaSourceHelpers::addKafkaFields);
   }
   ```
   
   AvroKafkaSourceHelpers::addKafkaFields is responsible for that:
   ```java
     public static GenericRecord addKafkaFields(ConsumerRecord<Object, Object> 
obj) {
       GenericRecord record = (GenericRecord) obj.value();
       record.put(AvroKafkaSourceHelpers.KAFKA_OFFSET, obj.offset());
       record.put(AvroKafkaSourceHelpers.KAFKA_PARTITION, obj.partition());
       record.put(AvroKafkaSourceHelpers.KAFKA_TOPIC, obj.topic());
       record.put(AvroKafkaSourceHelpers.KAFKA_KEY, obj.key());
       return record;
     }
   ```
       

##########
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);
     } catch (IOException ioe) {
       throw new HoodieIOException("Error reading source schema from registry 
:" + registryUrl, ioe);
     }
   }
 
-  @Override
-  public Schema getTargetSchema() {
-    String registryUrl = config.getString(Config.SRC_SCHEMA_REGISTRY_URL_PROP);
-    String targetRegistryUrl = 
config.getString(Config.TARGET_SCHEMA_REGISTRY_URL_PROP, registryUrl);

Review comment:
       This logic has been moved into constructor:
   ```java
    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");
   ```
   




----------------------------------------------------------------
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]


Reply via email to