diff --git a/hudi-utilities/pom.xml b/hudi-utilities/pom.xml
index c4a1606fd52f1..1297f40f852ae 100644
--- a/hudi-utilities/pom.xml
+++ b/hudi-utilities/pom.xml
@@ -287,7 +287,6 @@
org.apache.kafkakafka_${scala.binary.version}${kafka.version}
- test
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/FilebasedSchemaProvider.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/FilebasedSchemaProvider.java
index 43f2ff27d2eff..604a267ccb61d 100644
--- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/FilebasedSchemaProvider.java
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/schema/FilebasedSchemaProvider.java
@@ -40,8 +40,10 @@ public class FilebasedSchemaProvider extends SchemaProvider {
* Configs supported.
*/
public static class Config {
- private static final String SOURCE_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider.source.schema.file";
+
+ public static final String SOURCE_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider.source.schema.file";
private static final String TARGET_SCHEMA_FILE_PROP = "hoodie.deltastreamer.schemaprovider.target.schema.file";
+ // public static final String SOURCE_SCHEMA_PROP = "hoodie.deltastreamer.schemaprovider.source.schema";
}
private final FileSystem fs;
@@ -49,14 +51,17 @@ public static class Config {
private final Schema sourceSchema;
private Schema targetSchema;
+ private TypedProperties props;
public FilebasedSchemaProvider(TypedProperties props, JavaSparkContext jssc) {
super(props, jssc);
+ this.props = props;
DataSourceUtils.checkRequiredProperties(props, Collections.singletonList(Config.SOURCE_SCHEMA_FILE_PROP));
String sourceFile = props.getString(Config.SOURCE_SCHEMA_FILE_PROP);
this.fs = FSUtils.getFs(sourceFile, jssc.hadoopConfiguration(), true);
try {
this.sourceSchema = new Schema.Parser().parse(this.fs.open(new Path(sourceFile)));
+ // props.setProperty(Config.SOURCE_SCHEMA_PROP, sourceSchema.toString());
if (props.containsKey(Config.TARGET_SCHEMA_FILE_PROP)) {
this.targetSchema =
new Schema.Parser().parse(fs.open(new Path(props.getString(Config.TARGET_SCHEMA_FILE_PROP))));
diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/serde/AbstractHoodieKafkaAvroDeserializer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/serde/AbstractHoodieKafkaAvroDeserializer.java
new file mode 100644
index 0000000000000..aaae059b37070
--- /dev/null
+++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/serde/AbstractHoodieKafkaAvroDeserializer.java
@@ -0,0 +1,130 @@
+/*
+ * 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.serde;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.utilities.UtilHelpers;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+import org.apache.hudi.utilities.serde.config.HoodieKafkaAvroDeserializationConfig;
+
+import kafka.utils.VerifiableProperties;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.specific.SpecificDatumReader;
+import org.apache.kafka.common.errors.SerializationException;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Properties;
+
+import static org.apache.hudi.utilities.serde.config.HoodieKafkaAvroDeserializationConfig.SCHEMA_PROVIDER_CLASS_PROP;
+
+public class AbstractHoodieKafkaAvroDeserializer {
+
+ private final DecoderFactory decoderFactory = DecoderFactory.get();
+ private boolean useSpecificAvroReader = false;
+ private Schema sourceSchema;
+ private Schema targetSchema;
+
+ public AbstractHoodieKafkaAvroDeserializer(VerifiableProperties properties) {
+ // this.sourceSchema = new Schema.Parser().parse(properties.props().getProperty(FilebasedSchemaProvider.Config.SOURCE_SCHEMA_PROP));
+ TypedProperties typedProperties = new TypedProperties();
+ copyProperties(typedProperties, properties.props());
+ try {
+ SchemaProvider schemaProvider = UtilHelpers.createSchemaProvider(
+ typedProperties.getString(SCHEMA_PROVIDER_CLASS_PROP), typedProperties, null);
+ this.sourceSchema = Objects.requireNonNull(schemaProvider).getSourceSchema();
+ this.targetSchema = Objects.requireNonNull(schemaProvider).getTargetSchema();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ private void copyProperties(TypedProperties typedProperties, Properties properties) {
+ for (Map.Entry