You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/07/08 03:58:46 UTC

[GitHub] [hudi] xushiyan commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

xushiyan commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r916351993


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java:
##########
@@ -84,8 +86,13 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute(String instantTime,
         dedupedKeys = keys.repartition(parallelism);
       }
 
-      HoodieData<HoodieRecord<T>> dedupedRecords =
-          dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
+      HoodieData dedupedRecords;
+      if (config.getRecordType() == HoodieRecordType.AVRO) {
+        dedupedRecords =
+            dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
+      } else {
+        dedupedRecords = dedupedKeys.map(key -> new HoodieEmptyRecord<>(key, config.getRecordType()));
+      }

Review Comment:
   so this is for BWC and avro type still using empty payload. it's helpful to add a note here to clarify the if else. in future we will remove avro record and empty payload completely, right?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java:
##########
@@ -84,21 +80,21 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
     Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf());
     HoodieBaseFile baseFile = mergeHandle.baseFileForMerge();
 
-    final GenericDatumWriter<GenericRecord> gWriter;
-    final GenericDatumReader<GenericRecord> gReader;
     Schema readSchema;
+    Schema readerSchema;
+    Schema writerSchema;
     if (externalSchemaTransformation || baseFile.getBootstrapBaseFile().isPresent()) {
-      readSchema = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), mergeHandle.getOldFilePath()).getSchema();
-      gWriter = new GenericDatumWriter<>(readSchema);
-      gReader = new GenericDatumReader<>(readSchema, mergeHandle.getWriterSchemaWithMetaFields());
+      readSchema = HoodieFileReaderFactory.getReaderFactory(table.getConfig().getRecordType()).getFileReader(table.getHadoopConf(), mergeHandle.getOldFilePath()).getSchema();
+      writerSchema = readSchema;
+      readerSchema = mergeHandle.getWriterSchemaWithMetaFields();
     } else {
-      gReader = null;
-      gWriter = null;
+      readerSchema = null;
+      writerSchema = null;
       readSchema = mergeHandle.getWriterSchemaWithMetaFields();
     }
 
     BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
-    HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath());
+    HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(table.getConfig().getRecordType()).getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath());

Review Comment:
   this FileReader creation can be unified with the one in the if block above?



##########
hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java:
##########
@@ -66,38 +62,39 @@ public void runMerge(HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List
     HoodieMergeHandle<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> mergeHandle = upsertHandle;
     HoodieBaseFile baseFile = mergeHandle.baseFileForMerge();
 
-    final GenericDatumWriter<GenericRecord> gWriter;
-    final GenericDatumReader<GenericRecord> gReader;
+    Schema readerSchema;
+    Schema writerSchema;

Review Comment:
   similar comments as made for HoodieMergeHelper



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);

Review Comment:
   i wonder why do we need getRecordColumnValues to be under the utils. it is the same signature as this instance method, and looks only specific to HoodieSparkRecord. Is it going to be used else where? if not, you can just keep it in the instance method here. Utils is supposed to have reusable and more generic APIs.



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    Option<SparkKeyGeneratorInterface> keyGeneratorOpt = Option.empty();
+    if (useKeygen && !Boolean.parseBoolean(prop.getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+      try {
+        keyGeneratorOpt = Option.of((SparkKeyGeneratorInterface) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(prop)));
+      } catch (IOException e) {
+        throw new HoodieException("Only SparkKeyGeneratorInterface are supported when meta columns are disabled ", e);
+      }
+    }
+    String key = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getRecordKey(data, structType)
+        : data.get(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), StringType).toString();
+    String partition = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getPartitionPath(data, structType)
+        : data.get(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), StringType).toString();
+    this.key = new HoodieKey(key, partition);
+
+    return this;
+  }
+
+  @Override
+  public Option<Map<String, String>> getMetadata() {
+    return Option.empty();
+  }
+
+  @Override
+  public boolean isPresent(Schema schema, Properties prop) throws IOException {
+    if (null == data) {
+      return false;
+    }
+    if (schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD) == null) {
+      return true;
+    }
+    Object deleteMarker = data.get(schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).pos(), BooleanType);
+    return !(deleteMarker instanceof Boolean && (boolean) deleteMarker);
+  }
+
+  @Override
+  public boolean shouldIgnore(Schema schema, Properties prop) throws IOException {
+    // TODO SENTINEL should refactor SENTINEL without Avro(GenericRecord)
+    if (null != data && data.equals(SENTINEL)) {

Review Comment:
   let's stick with style of `data != null` instead of `null != data`  throughout the codebase



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    Option<SparkKeyGeneratorInterface> keyGeneratorOpt = Option.empty();

Review Comment:
   use of Option here is redundant and make it less readable. you can just create this.key in the if else block, can't you?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java:
##########
@@ -135,21 +131,22 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
         readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation);
       } else {
         if (needToReWriteRecord) {
-          readerIterator = new RewriteIterator(reader.getRecordIterator(HoodieAvroIndexedRecord::new), readSchema, readSchema, table.getConfig().getProps(), renameCols);
+          readerIterator = new RewriteIterator(reader.getRecordIterator(), readSchema, readSchema, table.getConfig().getProps(), renameCols);
         } else {
-          readerIterator = reader.getRecordIterator(readSchema, HoodieAvroIndexedRecord::new);
+          readerIterator = reader.getRecordIterator(readSchema);
         }
       }
 
-      ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
-      ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
       wrapper = new BoundedInMemoryExecutor(table.getConfig().getWriteBufferLimitBytes(), readerIterator,
           new UpdateHandler(mergeHandle), record -> {
         if (!externalSchemaTransformation) {
           return record;
         }
-        // TODO Other type of record need to change
-        return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) ((HoodieRecord)record).getData());
+        try {
+          return ((HoodieRecord) record).rewriteRecord(writerSchema, readerSchema, new TypedProperties());
+        } catch (IOException e) {
+          throw new HoodieException(e);

Review Comment:
   can you add some message when wrap an exception to help troubleshoot?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java:
##########
@@ -84,21 +80,21 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
     Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf());
     HoodieBaseFile baseFile = mergeHandle.baseFileForMerge();
 
-    final GenericDatumWriter<GenericRecord> gWriter;
-    final GenericDatumReader<GenericRecord> gReader;
     Schema readSchema;
+    Schema readerSchema;
+    Schema writerSchema;

Review Comment:
   please use better names to distinguish these 3 vars. people will confuse: what is the difference btw readSchema and readerSchema? why readSchema assign to writerSchema? even with better names, you can add inline comments to explain the usage & scenarios further for these vars.



##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java:
##########
@@ -61,42 +57,43 @@ public static FlinkMergeHelper newInstance() {
   @Override
   public void runMerge(HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table,
                        HoodieMergeHandle<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> mergeHandle) throws IOException {
-    final GenericDatumWriter<GenericRecord> gWriter;
-    final GenericDatumReader<GenericRecord> gReader;
     Schema readSchema;
+    Schema readerSchema;
+    Schema writerSchema;

Review Comment:
   similar comments as made for HoodieMergeHelper



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java:
##########
@@ -135,21 +131,22 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
         readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation);
       } else {
         if (needToReWriteRecord) {
-          readerIterator = new RewriteIterator(reader.getRecordIterator(HoodieAvroIndexedRecord::new), readSchema, readSchema, table.getConfig().getProps(), renameCols);
+          readerIterator = new RewriteIterator(reader.getRecordIterator(), readSchema, readSchema, table.getConfig().getProps(), renameCols);
         } else {
-          readerIterator = reader.getRecordIterator(readSchema, HoodieAvroIndexedRecord::new);
+          readerIterator = reader.getRecordIterator(readSchema);
         }
       }
 
-      ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
-      ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
       wrapper = new BoundedInMemoryExecutor(table.getConfig().getWriteBufferLimitBytes(), readerIterator,
           new UpdateHandler(mergeHandle), record -> {
         if (!externalSchemaTransformation) {
           return record;
         }
-        // TODO Other type of record need to change
-        return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) ((HoodieRecord)record).getData());
+        try {
+          return ((HoodieRecord) record).rewriteRecord(writerSchema, readerSchema, new TypedProperties());

Review Comment:
   > new TypedProperties()
   
   no need to pass any properties from `table` ?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);

Review Comment:
   similar to getRecordColumnValues



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {

Review Comment:
   not making use of the props at all ?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {

Review Comment:
   why boolean as an option? also a general rule of practice is have <=5 args. A public API with >5 args is really hard to use.



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReader.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.io.storage;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.ClosableIterator;
+import org.apache.hudi.common.util.MappingIterator;
+
+import org.apache.avro.Schema;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+
+import static org.apache.hudi.TypeUtils.unsafeCast;
+
+public interface HoodieSparkFileReader extends HoodieFileReader<InternalRow> {
+
+  Logger LOG = LogManager.getLogger(HoodieSparkFileReader.class);

Review Comment:
   dont think Logger needs here at interface level. the implementation should create its own logger



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    Option<SparkKeyGeneratorInterface> keyGeneratorOpt = Option.empty();
+    if (useKeygen && !Boolean.parseBoolean(prop.getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+      try {
+        keyGeneratorOpt = Option.of((SparkKeyGeneratorInterface) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(prop)));
+      } catch (IOException e) {
+        throw new HoodieException("Only SparkKeyGeneratorInterface are supported when meta columns are disabled ", e);
+      }
+    }
+    String key = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getRecordKey(data, structType)
+        : data.get(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), StringType).toString();
+    String partition = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getPartitionPath(data, structType)
+        : data.get(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), StringType).toString();
+    this.key = new HoodieKey(key, partition);
+
+    return this;
+  }
+
+  @Override
+  public Option<Map<String, String>> getMetadata() {
+    return Option.empty();
+  }
+
+  @Override
+  public boolean isPresent(Schema schema, Properties prop) throws IOException {
+    if (null == data) {
+      return false;
+    }
+    if (schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD) == null) {
+      return true;
+    }
+    Object deleteMarker = data.get(schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).pos(), BooleanType);
+    return !(deleteMarker instanceof Boolean && (boolean) deleteMarker);
+  }
+
+  @Override
+  public boolean shouldIgnore(Schema schema, Properties prop) throws IOException {
+    // TODO SENTINEL should refactor SENTINEL without Avro(GenericRecord)
+    if (null != data && data.equals(SENTINEL)) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Option<IndexedRecord> toIndexedRecord(Schema schema, Properties prop) throws IOException {

Review Comment:
   Why return an option? An API like `toXYZ` should always return non-null XYZ. 



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java:
##########
@@ -54,42 +63,64 @@
   private final SerializableSchema schema;
   private final HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy;
   private final HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType;
+  private final HoodieRecordType recordType;
 
   public RDDSpatialCurveSortPartitioner(HoodieSparkEngineContext sparkEngineContext,
-                                        String[] orderByColumns,
-                                        HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy,
-                                        HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType,
-                                        Schema schema) {
+      String[] orderByColumns,
+      LayoutOptimizationStrategy layoutOptStrategy,
+      SpatialCurveCompositionStrategyType curveCompositionStrategyType,
+      Schema schema, HoodieRecordType recordType) {
     this.sparkEngineContext = sparkEngineContext;
     this.orderByColumns = orderByColumns;
     this.layoutOptStrategy = layoutOptStrategy;
     this.curveCompositionStrategyType = curveCompositionStrategyType;
     this.schema = new SerializableSchema(schema);
+    this.recordType = recordType;
   }
 
   @Override
   public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records, int outputSparkPartitions) {
-    JavaRDD<GenericRecord> genericRecordsRDD =
-        records.map(f -> (GenericRecord) f.toIndexedRecord(schema.get(), new Properties()).get());
-
-    Dataset<Row> sourceDataset =
-        AvroConversionUtils.createDataFrame(
-            genericRecordsRDD.rdd(),
-            schema.toString(),
-            sparkEngineContext.getSqlContext().sparkSession()
-        );
-
-    Dataset<Row> sortedDataset = reorder(sourceDataset, outputSparkPartitions);
-
-    return HoodieSparkUtils.createRdd(sortedDataset, schema.get().getName(), schema.get().getNamespace(), false, Option.empty())
-        .toJavaRDD()
-        .map(record -> {
-          String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
-          String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
-          HoodieKey hoodieKey = new HoodieKey(key, partition);
-          HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, new RewriteAvroPayload(record));
-          return hoodieRecord;
-        });
+    if (recordType == HoodieRecordType.AVRO) {
+      JavaRDD<GenericRecord> genericRecordsRDD =
+          records.map(f -> (GenericRecord) f.toIndexedRecord(schema.get(), new Properties()).get());
+
+      Dataset<Row> sourceDataset =
+          AvroConversionUtils.createDataFrame(
+              genericRecordsRDD.rdd(),
+              schema.toString(),
+              sparkEngineContext.getSqlContext().sparkSession()
+          );
+
+      Dataset<Row> sortedDataset = reorder(sourceDataset, outputSparkPartitions);
+
+      return HoodieSparkUtils.createRdd(sortedDataset, schema.get().getName(), schema.get().getNamespace(), false, Option.empty())
+          .toJavaRDD()
+          .map(record -> {
+            String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+            String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
+            HoodieKey hoodieKey = new HoodieKey(key, partition);
+            HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, new RewriteAvroPayload(record));
+            return hoodieRecord;
+          });
+    } else if (recordType == HoodieRecordType.SPARK) {
+      StructType structType = HoodieInternalRowUtils.getCachedSchema(schema.get());
+      Dataset<Row> sourceDataset = SparkConversionUtils.createDataFrame(records.rdd(), sparkEngineContext.getSqlContext().sparkSession(), structType);
+
+      Dataset<Row> sortedDataset = reorder(sourceDataset, outputSparkPartitions);
+
+      return sortedDataset.queryExecution().toRdd()
+          .toJavaRDD()
+          .map(row -> {
+            InternalRow internalRow = row.copy();

Review Comment:
   can you help clarify why copy() here?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {

Review Comment:
   Properties props



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriter.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.io.storage;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+
+import java.io.IOException;
+import java.util.Properties;
+
+public interface HoodieSparkFileWriter extends HoodieFileWriter {
+  boolean canWrite();
+
+  void close() throws IOException;
+
+  void writeRowWithMetadata(HoodieKey recordKey, InternalRow row) throws IOException;
+
+  void writeRow(String recordKey, InternalRow row) throws IOException;
+
+  @Override
+  default void write(String recordKey, HoodieRecord record, Schema schema, Properties props) throws IOException {
+    writeRow(recordKey, (InternalRow) record.getData());
+  }
+
+  @Override
+  default void writeWithMetadata(HoodieKey key, HoodieRecord record, Schema schema, Properties props) throws IOException {
+    writeRowWithMetadata(key, (InternalRow) record.getData());
+  }
+
+  default InternalRow prepRecordWithMetadata(HoodieKey key, InternalRow row, String instantTime, Integer partitionId, long recordIndex, String fileName)  {
+    String seqId = HoodieRecord.generateSequenceId(instantTime, partitionId, recordIndex);
+    row.update(HoodieRecord.HoodieMetadataField.COMMIT_TIME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(instantTime));
+    row.update(HoodieRecord.HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(seqId));
+    row.update(HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getRecordKey()));
+    row.update(HoodieRecord.HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getPartitionPath()));
+    row.update(HoodieRecord.HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(fileName));
+    return row;
+    // Object[] metadata = {instantTime, seqId, key.getRecordKey(), key.getPartitionPath(), fileName};
+    // InternalRow metadataRow = new GenericInternalRow(Arrays.stream(metadata)
+    //    .map(o -> CatalystTypeConverters.convertToCatalyst(o)).toArray());
+    // return new JoinedRow(metadataRow, row);

Review Comment:
   clean up?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriter.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.io.storage;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+
+import java.io.IOException;
+import java.util.Properties;
+
+public interface HoodieSparkFileWriter extends HoodieFileWriter {
+  boolean canWrite();
+
+  void close() throws IOException;
+
+  void writeRowWithMetadata(HoodieKey recordKey, InternalRow row) throws IOException;
+
+  void writeRow(String recordKey, InternalRow row) throws IOException;
+
+  @Override
+  default void write(String recordKey, HoodieRecord record, Schema schema, Properties props) throws IOException {
+    writeRow(recordKey, (InternalRow) record.getData());
+  }
+
+  @Override
+  default void writeWithMetadata(HoodieKey key, HoodieRecord record, Schema schema, Properties props) throws IOException {
+    writeRowWithMetadata(key, (InternalRow) record.getData());
+  }
+
+  default InternalRow prepRecordWithMetadata(HoodieKey key, InternalRow row, String instantTime, Integer partitionId, long recordIndex, String fileName)  {
+    String seqId = HoodieRecord.generateSequenceId(instantTime, partitionId, recordIndex);
+    row.update(HoodieRecord.HoodieMetadataField.COMMIT_TIME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(instantTime));
+    row.update(HoodieRecord.HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(seqId));
+    row.update(HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getRecordKey()));
+    row.update(HoodieRecord.HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getPartitionPath()));
+    row.update(HoodieRecord.HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(fileName));

Review Comment:
   use static imports to help reduce verbosity



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    Option<SparkKeyGeneratorInterface> keyGeneratorOpt = Option.empty();
+    if (useKeygen && !Boolean.parseBoolean(prop.getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+      try {
+        keyGeneratorOpt = Option.of((SparkKeyGeneratorInterface) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(prop)));
+      } catch (IOException e) {
+        throw new HoodieException("Only SparkKeyGeneratorInterface are supported when meta columns are disabled ", e);
+      }
+    }
+    String key = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getRecordKey(data, structType)
+        : data.get(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), StringType).toString();
+    String partition = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getPartitionPath(data, structType)
+        : data.get(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), StringType).toString();
+    this.key = new HoodieKey(key, partition);
+
+    return this;
+  }
+
+  @Override
+  public Option<Map<String, String>> getMetadata() {
+    return Option.empty();
+  }
+
+  @Override
+  public boolean isPresent(Schema schema, Properties prop) throws IOException {

Review Comment:
   Properties props



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {

Review Comment:
   Properties props



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

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org