You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by yu...@apache.org on 2022/09/22 03:17:42 UTC

[hudi] branch release-feature-rfc46 updated: [HUDI-4301] [HUDI-3384][HUDI-3385] Spark specific file reader/writer.(#5629)

This is an automated email from the ASF dual-hosted git repository.

yuzhaojing pushed a commit to branch release-feature-rfc46
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/release-feature-rfc46 by this push:
     new bd45932e04 [HUDI-4301] [HUDI-3384][HUDI-3385] Spark specific file reader/writer.(#5629)
bd45932e04 is described below

commit bd45932e043be724545dad4d85234d5ec9f23032
Author: Shawy Geng <ge...@gmail.com>
AuthorDate: Thu Sep 22 11:17:36 2022 +0800

    [HUDI-4301] [HUDI-3384][HUDI-3385] Spark specific file reader/writer.(#5629)
    
    * [HUDI-4301] [HUDI-3384][HUDI-3385] Spark specific file reader/writer.
    
    * add schema finger print
    
    * add benchmark
    
    * a new way to config the merger
    
    * fix
    
    Co-authored-by: wangzixuan.wzxuan <wa...@bytedance.com>
    Co-authored-by: gengxiaoyu <ge...@bytedance.com>
---
 .../hudi/cli/commands/ArchivedCommitsCommand.java  |   8 +-
 .../apache/hudi/cli/commands/ExportCommand.java    |   6 +-
 .../hudi/cli/commands/HoodieLogFileCommand.java    |  14 +-
 .../scala/org/apache/hudi/cli/SparkHelpers.scala   |   4 +-
 .../cli/commands/TestHoodieLogFileCommand.java     |   3 +
 .../apache/hudi/client/HoodieTimelineArchiver.java |   3 +-
 .../bootstrap/FullRecordBootstrapDataProvider.java |   4 +-
 .../common/table/log/HoodieFileSliceReader.java    |  46 +--
 .../apache/hudi/config/HoodieCompactionConfig.java |  12 -
 .../org/apache/hudi/config/HoodieWriteConfig.java  |  61 +++-
 .../org/apache/hudi/index/HoodieIndexUtils.java    |   5 +-
 .../org/apache/hudi/io/HoodieAppendHandle.java     |  35 ++-
 .../org/apache/hudi/io/HoodieConcatHandle.java     |   4 +-
 .../org/apache/hudi/io/HoodieCreateHandle.java     |  20 +-
 .../org/apache/hudi/io/HoodieKeyLookupHandle.java  |   4 +-
 .../java/org/apache/hudi/io/HoodieMergeHandle.java |  28 +-
 .../org/apache/hudi/io/HoodieRangeInfoHandle.java  |   4 +-
 .../java/org/apache/hudi/io/HoodieReadHandle.java  |   6 +-
 .../java/org/apache/hudi/io/HoodieWriteHandle.java |   9 +-
 .../hudi/io/storage/HoodieFileWriterFactory.java   | 113 --------
 .../strategy/ClusteringExecutionStrategy.java      |   3 +
 .../hudi/table/action/commit/BaseMergeHelper.java  |  50 +---
 .../hudi/table/action/commit/BaseWriteHelper.java  |  10 +-
 .../table/action/commit/HoodieDeleteHelper.java    |  13 +-
 .../table/action/commit/HoodieMergeHelper.java     |  39 ++-
 .../table/action/commit/HoodieWriteHelper.java     |  17 +-
 .../hudi/table/action/compact/HoodieCompactor.java |   1 +
 .../io/storage/TestHoodieHFileReaderWriter.java    |  26 +-
 .../hudi/io/storage/TestHoodieOrcReaderWriter.java |   5 +-
 .../io/storage/TestHoodieReaderWriterBase.java     |  36 +--
 .../hudi/testutils/HoodieWriteableTestTable.java   |   2 +-
 .../hudi/table/action/commit/FlinkMergeHelper.java |  41 +--
 .../hudi/table/action/commit/FlinkWriteHelper.java |  20 +-
 .../run/strategy/JavaExecutionStrategy.java        |  39 +--
 .../strategy/JavaSortAndSizeExecutionStrategy.java |   2 +-
 .../hudi/table/action/commit/JavaMergeHelper.java  |  37 +--
 .../hudi/table/action/commit/JavaWriteHelper.java  |  16 +-
 .../commit/TestJavaCopyOnWriteActionExecutor.java  |   2 +-
 .../MultipleSparkJobExecutionStrategy.java         |  61 ++--
 .../strategy/SingleSparkJobExecutionStrategy.java  |  51 ++--
 .../SparkSingleFileSortExecutionStrategy.java      |   2 +-
 .../SparkSortAndSizeExecutionStrategy.java         |   2 +-
 .../hudi/client/model/HoodieInternalRow.java       |   2 +-
 .../hudi/commmon/model/HoodieSparkRecord.java      | 283 ++++++++++++++++++
 .../bulkinsert/RDDConsistentBucketPartitioner.java |   5 +-
 .../RDDCustomColumnsSortPartitioner.java           |   4 +-
 .../bulkinsert/RDDSpatialCurveSortPartitioner.java |  83 ++++--
 .../hudi/io/storage/HoodieSparkFileReader.java     |  53 ++++
 .../io/storage/HoodieSparkFileReaderFactory.java   |  57 ++++
 .../hudi/io/storage/HoodieSparkFileWriter.java     |  64 +++++
 .../io/storage/HoodieSparkFileWriterFactory.java   | 113 ++++++++
 .../hudi/io/storage/HoodieSparkParquetReader.java  |  44 ++-
 .../io/storage/HoodieSparkParquetStreamWriter.java | 100 +++++++
 .../hudi/io/storage/HoodieSparkParquetWriter.java} |  53 ++--
 .../row/HoodieInternalRowFileWriterFactory.java    |   2 +-
 .../io/storage/row/HoodieRowParquetConfig.java     |  42 +++
 .../storage/row/HoodieRowParquetWriteSupport.java  |  14 +-
 .../apache/hudi/keygen/BuiltinKeyGenerator.java    |   2 +-
 .../bootstrap/OrcBootstrapMetadataHandler.java     |   5 +
 .../bootstrap/ParquetBootstrapMetadataHandler.java |  40 +--
 .../SparkBootstrapCommitActionExecutor.java        |   2 +-
 .../apache/hudi/util/HoodieSparkRecordUtils.java   | 111 ++++++++
 .../org/apache}/hudi/HoodieInternalRowUtils.scala  | 213 +++++++++-----
 .../org/apache/hudi/SparkConversionUtils.scala     |  30 +-
 .../scala/org/apache/spark/sql/DataFrameUtil.scala |  33 ++-
 .../spark/sql/HoodieCatalystExpressionUtils.scala  |   4 +
 .../spark/sql/hudi/SparkStructTypeSerializer.scala | 157 ++++++++++
 ...parkConsistentBucketClusteringPlanStrategy.java |   2 +-
 .../functional/TestConsistentBucketIndex.java      |   2 +-
 .../functional/TestHoodieBackedMetadata.java       |  14 +-
 .../functional/TestHoodieBackedTableMetadata.java  |   4 +-
 .../TestHoodieClientOnCopyOnWriteStorage.java      |  23 +-
 .../hudi/client/functional/TestHoodieIndex.java    |   2 +-
 .../client/functional/TestHoodieMetadataBase.java  |   2 +-
 .../TestSparkConsistentBucketClustering.java       |   2 +-
 .../index/hbase/TestHBaseQPSResourceAllocator.java |   2 +-
 .../index/hbase/TestSparkHoodieHBaseIndex.java     |   2 +-
 .../hudi/io/TestHoodieKeyLocationFetchHandle.java  |   2 +-
 .../org/apache/hudi/io/TestHoodieMergeHandle.java  |   2 +-
 .../storage/TestHoodieAvroFileWriterFactory.java   |   9 +-
 .../row/TestHoodieInternalRowParquetWriter.java    |   4 +-
 .../commit/TestCopyOnWriteActionExecutor.java      |   2 +-
 .../table/action/commit/TestUpsertPartitioner.java |   2 +-
 .../table/action/compact/CompactionTestBase.java   |   2 +-
 .../table/action/compact/TestHoodieCompactor.java  |   2 +-
 .../TestMergeOnReadRollbackActionExecutor.java     |   2 +-
 .../TestHoodieSparkMergeOnReadTableClustering.java |   2 +-
 .../TestHoodieSparkMergeOnReadTableCompaction.java |   2 +-
 .../TestHoodieSparkMergeOnReadTableRollback.java   |   2 +-
 .../hudi/testutils/HoodieClientTestBase.java       |   2 +-
 .../SparkClientFunctionalTestHarness.java          |   2 +-
 .../hudi/testutils/SparkDatasetTestUtils.java      |   2 +-
 .../java/org/apache/hudi/avro/HoodieAvroUtils.java |  36 ++-
 .../apache/hudi/common/config/HoodieConfig.java    |  30 +-
 .../hudi/common}/config/HoodieStorageConfig.java   |   7 +-
 .../hudi/common/model/HoodieAvroIndexedRecord.java | 137 +++++----
 .../apache/hudi/common/model/HoodieAvroRecord.java | 143 ++++------
 ...ecordMerge.java => HoodieAvroRecordMerger.java} |  43 ++-
 .../hudi/common/model/HoodieEmptyRecord.java       | 155 ++++++++++
 .../org/apache/hudi/common/model/HoodieRecord.java | 105 +++----
 ...ava => HoodieRecordCompatibilityInterface.java} |  31 +-
 .../{HoodieMerge.java => HoodieRecordMerger.java}  |  27 +-
 .../hudi/common/table/HoodieTableConfig.java       |  29 +-
 .../hudi/common/table/HoodieTableMetaClient.java   |  41 ++-
 .../table/log/AbstractHoodieLogRecordReader.java   |  96 +++----
 .../table/log/HoodieMergedLogRecordScanner.java    |  57 ++--
 .../table/log/HoodieUnMergedLogRecordScanner.java  |  22 +-
 .../table/log/block/HoodieAvroDataBlock.java       |  65 +++--
 .../common/table/log/block/HoodieDataBlock.java    |  45 +--
 .../table/log/block/HoodieHFileDataBlock.java      |  55 +---
 .../table/log/block/HoodieParquetDataBlock.java    | 101 ++++---
 .../table/timeline/HoodieArchivedTimeline.java     |   8 +-
 .../org/apache/hudi/common/util/ConfigUtils.java   |  55 ++++
 .../common/util/HoodieRecordSizeEstimator.java     |   5 +-
 .../apache/hudi/common/util/HoodieRecordUtils.java |  76 +++--
 .../hudi/common/util/ParquetReaderIterator.java    |  21 +-
 .../hudi/common/util/SerializationUtils.java       |  18 ++
 .../apache/hudi/common/util/SpillableMapUtils.java |   8 +-
 .../org/apache/hudi/common/util/StringUtils.java   |   2 +
 .../hudi/io/storage/HoodieAvroFileReader.java      |  78 +++--
 ...ctory.java => HoodieAvroFileReaderFactory.java} |  38 +--
 .../hudi/io/storage/HoodieAvroFileWriter.java      |   5 +-
 .../io/storage/HoodieAvroFileWriterFactory.java    | 119 ++++++++
 .../hudi/io/storage/HoodieAvroHFileReader.java     |  17 +-
 .../hudi/io/storage/HoodieAvroHFileWriter.java     |   0
 .../hudi/io/storage/HoodieAvroOrcReader.java       |   7 +-
 .../hudi/io/storage/HoodieAvroOrcWriter.java       |   0
 .../hudi/io/storage/HoodieAvroParquetReader.java   |  25 +-
 .../hudi/io/storage/HoodieAvroParquetWriter.java   |   0
 .../hudi/io/storage/HoodieBaseParquetWriter.java   |   0
 .../apache/hudi/io/storage/HoodieFileReader.java   |  28 +-
 .../hudi/io/storage/HoodieFileReaderFactory.java   |  44 ++-
 .../hudi/io/storage/HoodieFileWriterFactory.java   | 131 +++++++++
 .../apache/hudi/io/storage/HoodieHFileConfig.java  |   0
 .../apache/hudi/io/storage/HoodieOrcConfig.java    |   0
 .../hudi/metadata/HoodieBackedTableMetadata.java   |  40 +--
 .../HoodieMetadataMergedLogRecordReader.java       |  32 +--
 .../apache/hudi/metadata/HoodieTableMetadata.java  |   3 +
 .../hudi/metadata/HoodieTableMetadataUtil.java     |  11 +-
 .../common/functional/TestHoodieLogFormat.java     |  20 +-
 .../hudi/common/table/TestHoodieTableConfig.java   |  16 +-
 .../hudi/common/util/HoodieRecordUtilsTest.java    |  14 +-
 .../storage/TestHoodieAvroFileReaderFactory.java   |   8 +-
 .../examples/quickstart/TestQuickstartData.java    |   3 +
 .../apache/hudi/configuration/FlinkOptions.java    |  24 +-
 .../org/apache/hudi/sink/StreamWriteFunction.java  |  19 +-
 .../hudi/sink/clustering/ClusteringOperator.java   |  16 +-
 .../sink/partitioner/profile/WriteProfile.java     |   3 +-
 .../apache/hudi/streamer/FlinkStreamerConfig.java  |  15 +-
 .../org/apache/hudi/table/HoodieTableSource.java   |   3 +-
 .../org/apache/hudi/table/format/FormatUtils.java  |  13 +
 .../table/format/mor/MergeOnReadInputFormat.java   |  31 +-
 .../table/format/mor/MergeOnReadTableState.java    |   9 +-
 .../java/org/apache/hudi/util/StreamerUtil.java    |   6 +-
 .../apache/hudi/source/TestStreamReadOperator.java |   3 +-
 .../test/java/org/apache/hudi/utils/TestData.java  |   3 +
 .../hudi/hadoop/HoodieHFileRecordReader.java       |   2 +-
 .../org/apache/hudi/hadoop/InputSplitUtils.java    |   6 +-
 .../realtime/RealtimeCompactedRecordReader.java    |  16 +-
 .../realtime/RealtimeUnmergedRecordReader.java     |   5 +-
 .../utils/HoodieRealtimeRecordReaderUtils.java     |   7 +-
 .../apache/hudi/hadoop/TestInputPathHandler.java   |   2 +
 .../reader/DFSHoodieDatasetInputReader.java        |  12 +-
 .../main/java/org/apache/hudi/DataSourceUtils.java |   2 +-
 .../org/apache/hudi/HoodieSparkRecordMerger.java}  |  38 ++-
 .../scala/org/apache/hudi/DataSourceOptions.scala  |   9 +-
 .../scala/org/apache/hudi/HoodieBaseRelation.scala |  14 +-
 .../org/apache/hudi/HoodieMergeOnReadRDD.scala     |  92 ++++--
 .../org/apache/hudi/HoodieSparkSqlWriter.scala     | 114 ++++++--
 .../scala/org/apache/hudi/HoodieWriterUtils.scala  |   6 +-
 .../apache/spark/sql/hudi/DataSkippingUtils.scala  |   2 +-
 .../apache/spark/sql/hudi/HoodieOptionConfig.scala |  18 +-
 .../spark/sql/hudi/command/SqlKeyGenerator.scala   |   2 +-
 .../SparkFullBootstrapDataProviderBase.java        |  57 +++-
 .../apache/spark/sql/hudi/HoodieSparkRecord.java   | 190 -------------
 .../org/apache/spark/sql/hudi/SparkHelpers.scala   |   4 +-
 .../procedures/ExportInstantsProcedure.scala       |  11 +-
 .../ShowHoodieLogFileMetadataProcedure.scala       |  10 +-
 .../ShowHoodieLogFileRecordsProcedure.scala        |  12 +-
 .../java/org/apache/hudi/TestDataSourceUtils.java  |   2 +-
 .../org/apache/hudi/functional/TestBootstrap.java  |   2 +-
 .../apache/hudi/functional/TestOrcBootstrap.java   |   2 +-
 .../org/apache/hudi/TestDataSourceDefaults.scala   |   1 +
 .../org/apache/hudi/TestHoodieFileIndex.scala      |   7 +-
 .../apache/hudi/TestHoodieInternalRowUtils.scala   |   2 +-
 .../hudi/TestStructTypeSchemaEvolutionUtils.scala  |   3 +-
 .../apache/hudi/functional/TestCOWDataSource.scala | 315 +++++++++++++++------
 .../hudi/functional/TestColumnStatsIndex.scala     |   6 +-
 .../apache/hudi/functional/TestMORDataSource.scala | 301 +++++++++++++++-----
 .../TestMetadataTableWithSparkDataSource.scala     |   2 +-
 .../functional/TestParquetColumnProjection.scala   |   5 +-
 .../hudi/functional/TestStructuredStreaming.scala  |   8 +-
 .../ReadAndWriteWithoutAvroBenchmark.scala         | 154 ++++++++++
 .../spark/sql/hudi/TestHoodieOptionConfig.scala    |  10 +-
 .../org/apache/spark/sql/hudi/TestSqlConf.scala    |   3 +-
 .../utilities/HoodieMetadataTableValidator.java    |   3 +-
 .../hudi/utilities/deltastreamer/DeltaSync.java    |  13 +-
 .../deltastreamer/HoodieDeltaStreamer.java         |  12 +-
 198 files changed, 4107 insertions(+), 2051 deletions(-)

diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ArchivedCommitsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ArchivedCommitsCommand.java
index 3301c1402d..4fcde21087 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ArchivedCommitsCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ArchivedCommitsCommand.java
@@ -24,9 +24,9 @@ import org.apache.hudi.cli.HoodieCLI;
 import org.apache.hudi.cli.HoodiePrintHelper;
 import org.apache.hudi.cli.TableHeader;
 import org.apache.hudi.common.fs.FSUtils;
-import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieLogFile;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.log.HoodieLogFormat;
 import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
@@ -83,7 +83,7 @@ public class ArchivedCommitsCommand implements CommandMarker {
       // read the avro blocks
       while (reader.hasNext()) {
         HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
-        blk.getRecordIterator(HoodieAvroIndexedRecord::new).forEachRemaining(r -> readRecords.add((IndexedRecord) r.getData()));
+        blk.getRecordIterator(HoodieRecordType.AVRO).forEachRemaining(r -> readRecords.add((IndexedRecord) r.getData()));
       }
       List<Comparable[]> readCommits = readRecords.stream().map(r -> (GenericRecord) r)
           .filter(r -> r.get("actionType").toString().equals(HoodieTimeline.COMMIT_ACTION)
@@ -157,8 +157,8 @@ public class ArchivedCommitsCommand implements CommandMarker {
       // read the avro blocks
       while (reader.hasNext()) {
         HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
-        try (ClosableIterator<HoodieRecord> recordItr = blk.getRecordIterator(HoodieAvroIndexedRecord::new)) {
-          recordItr.forEachRemaining(r -> readRecords.add((IndexedRecord) r.getData()));
+        try (ClosableIterator<HoodieRecord<IndexedRecord>> recordItr = blk.getRecordIterator(HoodieRecordType.AVRO)) {
+          recordItr.forEachRemaining(r -> readRecords.add(r.getData()));
         }
       }
       List<Comparable[]> readCommits = readRecords.stream().map(r -> (GenericRecord) r)
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java
index 34fb0aab73..686a0dd068 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/ExportCommand.java
@@ -25,9 +25,9 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata;
 import org.apache.hudi.avro.model.HoodieSavepointMetadata;
 import org.apache.hudi.cli.HoodieCLI;
 import org.apache.hudi.common.fs.FSUtils;
-import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieLogFile;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.log.HoodieLogFormat;
 import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
@@ -125,9 +125,9 @@ public class ExportCommand implements CommandMarker {
       // read the avro blocks
       while (reader.hasNext() && copyCount < limit) {
         HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
-        try (ClosableIterator<HoodieRecord> recordItr = blk.getRecordIterator(HoodieAvroIndexedRecord::new)) {
+        try (ClosableIterator<HoodieRecord<IndexedRecord>> recordItr = blk.getRecordIterator(HoodieRecordType.AVRO)) {
           while (recordItr.hasNext()) {
-            IndexedRecord ir = (IndexedRecord) recordItr.next().getData();
+            IndexedRecord ir = recordItr.next().getData();
             // Archived instants are saved as arvo encoded HoodieArchivedMetaEntry records. We need to get the
             // metadata record from the entry and convert it to json.
             HoodieArchivedMetaEntry archiveEntryRecord = (HoodieArchivedMetaEntry) SpecificData.get()
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java
index 17d413fb1f..8068ea7a76 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java
@@ -25,8 +25,10 @@ import org.apache.hudi.cli.TableHeader;
 import org.apache.hudi.common.config.HoodieCommonConfig;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
+import org.apache.hudi.common.model.HoodieAvroRecordMerger;
 import org.apache.hudi.common.model.HoodieLogFile;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.TableSchemaResolver;
 import org.apache.hudi.common.table.log.HoodieLogFormat;
@@ -38,6 +40,7 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock;
 import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
 import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
 import org.apache.hudi.common.util.ClosableIterator;
+import org.apache.hudi.common.util.HoodieRecordUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieMemoryConfig;
@@ -125,7 +128,7 @@ public class HoodieLogFileCommand implements CommandMarker {
             instantTime = "dummy_instant_time_" + dummyInstantTimeCount;
           }
           if (n instanceof HoodieDataBlock) {
-            try (ClosableIterator<HoodieRecord> recordItr = ((HoodieDataBlock) n).getRecordIterator(HoodieAvroIndexedRecord::new)) {
+            try (ClosableIterator<HoodieRecord<IndexedRecord>> recordItr = ((HoodieDataBlock) n).getRecordIterator(HoodieRecordType.AVRO)) {
               recordItr.forEachRemaining(r -> recordCount.incrementAndGet());
             }
           }
@@ -221,11 +224,12 @@ public class HoodieLogFileCommand implements CommandMarker {
               .withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.defaultValue())
               .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue())
               .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue())
+              .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
               .build();
       for (HoodieRecord hoodieRecord : scanner) {
-        Option<IndexedRecord> record = hoodieRecord.toIndexedRecord(readerSchema, new Properties());
+        Option<HoodieAvroIndexedRecord> record = hoodieRecord.toIndexedRecord(readerSchema, new Properties());
         if (allRecords.size() < limit) {
-          allRecords.add(record.get());
+          allRecords.add(record.get().getData());
         }
       }
     } else {
@@ -239,10 +243,10 @@ public class HoodieLogFileCommand implements CommandMarker {
           HoodieLogBlock n = reader.next();
           if (n instanceof HoodieDataBlock) {
             HoodieDataBlock blk = (HoodieDataBlock) n;
-            try (ClosableIterator<HoodieRecord> recordItr = blk.getRecordIterator(HoodieAvroIndexedRecord::new)) {
+            try (ClosableIterator<HoodieRecord<IndexedRecord>> recordItr = blk.getRecordIterator(HoodieRecordType.AVRO)) {
               recordItr.forEachRemaining(record -> {
                 if (allRecords.size() < limit) {
-                  allRecords.add((IndexedRecord) record.getData());
+                  allRecords.add(record.getData());
                 }
               });
             }
diff --git a/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala b/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala
index ddb6b6cf2f..0181b0733c 100644
--- a/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala
+++ b/hudi-cli/src/main/scala/org/apache/hudi/cli/SparkHelpers.scala
@@ -23,14 +23,14 @@ import org.apache.hadoop.fs.{FileSystem, Path}
 import org.apache.hudi.avro.HoodieAvroWriteSupport
 import org.apache.hudi.client.SparkTaskContextSupplier
 import org.apache.hudi.common.bloom.{BloomFilter, BloomFilterFactory}
+import org.apache.hudi.common.config.HoodieStorageConfig
 import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord}
 import org.apache.hudi.common.util.BaseFileUtils
-import org.apache.hudi.config.{HoodieIndexConfig, HoodieStorageConfig}
+import org.apache.hudi.config.HoodieIndexConfig
 import org.apache.hudi.io.storage.{HoodieAvroParquetWriter, HoodieParquetConfig}
 import org.apache.parquet.avro.AvroSchemaConverter
 import org.apache.parquet.hadoop.metadata.CompressionCodecName
 import org.apache.spark.sql.{DataFrame, SQLContext}
-
 import scala.collection.JavaConversions._
 import scala.collection.mutable._
 
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java
index efe89b69ef..c26b7efcb2 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestHoodieLogFileCommand.java
@@ -29,6 +29,7 @@ import org.apache.hudi.common.config.HoodieCommonConfig;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieAvroRecord;
 import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
+import org.apache.hudi.common.model.HoodieAvroRecordMerger;
 import org.apache.hudi.common.model.HoodieLogFile;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieTableType;
@@ -38,6 +39,7 @@ import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
 import org.apache.hudi.common.table.log.block.HoodieLogBlock;
 import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
 import org.apache.hudi.common.testutils.SchemaTestUtil;
+import org.apache.hudi.common.util.HoodieRecordUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieMemoryConfig;
@@ -222,6 +224,7 @@ public class TestHoodieLogFileCommand extends CLIFunctionalTestHarness {
         .withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.defaultValue())
         .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue())
         .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue())
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
 
     Iterator<HoodieRecord> records = scanner.iterator();
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java
index 16db94bffd..f4ab9fe703 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java
@@ -32,6 +32,7 @@ import org.apache.hudi.common.model.HoodieAvroPayload;
 import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
 import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -344,7 +345,7 @@ public class HoodieTimelineArchiver<T extends HoodieAvroPayload, I, K, O> {
           // Read the avro blocks
           while (reader.hasNext()) {
             HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
-            blk.getRecordIterator(HoodieAvroIndexedRecord::new).forEachRemaining(r -> records.add((IndexedRecord) r.getData()));
+            blk.getRecordIterator(HoodieRecordType.AVRO).forEachRemaining(r -> records.add((IndexedRecord) r.getData()));
             if (records.size() >= this.config.getCommitArchivalBatchSize()) {
               writeToFile(wrapperSchema, records);
             }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java
index 1cf1702717..636e0e4b26 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/bootstrap/FullRecordBootstrapDataProvider.java
@@ -23,6 +23,7 @@ import org.apache.hudi.avro.model.HoodieFileStatus;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
 
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -49,8 +50,9 @@ public abstract class FullRecordBootstrapDataProvider<I> implements Serializable
    * @param tableName Hudi Table Name
    * @param sourceBasePath Source Base Path
    * @param partitionPaths Partition Paths
+   * @param config config
    * @return input records
    */
   public abstract I generateInputRecords(String tableName,
-      String sourceBasePath, List<Pair<String, List<HoodieFileStatus>>> partitionPaths);
+      String sourceBasePath, List<Pair<String, List<HoodieFileStatus>>> partitionPaths, HoodieWriteConfig config);
 }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java
index 52e411108f..35ca3d6d5a 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java
@@ -21,62 +21,32 @@ package org.apache.hudi.common.table.log;
 
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodiePayloadConfig;
-import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
 
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.stream.StreamSupport;
+import java.util.Properties;
 
 /**
  * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
  */
 public class HoodieFileSliceReader<T> implements Iterator<HoodieRecord<T>> {
+
   private final Iterator<HoodieRecord<T>> recordsIterator;
 
   public static HoodieFileSliceReader getFileSliceReader(
-      Option<HoodieAvroFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
-      String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
+      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, Properties props, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
     if (baseFileReader.isPresent()) {
-      Iterator baseIterator = baseFileReader.get().getRecordIterator(schema);
+      Iterator<HoodieRecord> baseIterator = baseFileReader.get().getRecordIterator(schema);
       while (baseIterator.hasNext()) {
-        GenericRecord record = (GenericRecord) baseIterator.next();
-        HoodieRecord hoodieRecord = transform(
-            record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
-        scanner.processNextRecord(hoodieRecord);
+        scanner.processNextRecord(baseIterator.next().wrapIntoHoodieRecordPayloadWithParams(schema, props,
+            simpleKeyGenFieldsOpt, scanner.isWithOperationField(), scanner.getPartitionName(), false));
       }
-      return new HoodieFileSliceReader(scanner.iterator());
-    } else {
-      Iterable<HoodieRecord> iterable = () -> scanner.iterator();
-      HoodiePayloadConfig payloadConfig = HoodiePayloadConfig.newBuilder().withPayloadOrderingField(preCombineField).build();
-      return new HoodieFileSliceReader(StreamSupport.stream(iterable.spliterator(), false)
-          .map(e -> {
-            try {
-              GenericRecord record = (GenericRecord) e.toIndexedRecord(schema, payloadConfig.getProps()).get();
-              return transform(record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
-            } catch (IOException io) {
-              throw new HoodieIOException("Error while creating reader for file slice with no base file.", io);
-            }
-          }).iterator());
     }
-  }
-
-  private static HoodieRecord transform(GenericRecord record,
-      HoodieMergedLogRecordScanner scanner,
-      String payloadClass,
-      String preCombineField,
-      Option<Pair<String, String>> simpleKeyGenFieldsOpt) {
-    return simpleKeyGenFieldsOpt.isPresent()
-        ? SpillableMapUtils.convertToHoodieRecordPayload(record,
-        payloadClass, preCombineField, simpleKeyGenFieldsOpt.get(), scanner.isWithOperationField(), Option.empty())
-        : SpillableMapUtils.convertToHoodieRecordPayload(record,
-        payloadClass, preCombineField, scanner.isWithOperationField(), scanner.getPartitionName());
+    return new HoodieFileSliceReader(scanner.iterator());
   }
 
   private HoodieFileSliceReader(Iterator<HoodieRecord<T>> recordsItr) {
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
index 8a3e7f3db2..3b61d49727 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieCompactionConfig.java
@@ -22,7 +22,6 @@ import org.apache.hudi.common.config.ConfigClassProperty;
 import org.apache.hudi.common.config.ConfigGroups;
 import org.apache.hudi.common.config.ConfigProperty;
 import org.apache.hudi.common.config.HoodieConfig;
-import org.apache.hudi.common.model.HoodieAvroRecordMerge;
 import org.apache.hudi.table.action.compact.CompactionTriggerStrategy;
 import org.apache.hudi.table.action.compact.strategy.CompactionStrategy;
 import org.apache.hudi.table.action.compact.strategy.LogFileSizeBasedCompactionStrategy;
@@ -120,12 +119,6 @@ public class HoodieCompactionConfig extends HoodieConfig {
           + "compaction during each compaction run. By default. Hudi picks the log file "
           + "with most accumulated unmerged data");
 
-  public static final ConfigProperty<String> MERGE_CLASS_NAME = ConfigProperty
-      .key("hoodie.compaction.merge.class")
-      .defaultValue(HoodieAvroRecordMerge.class.getName())
-      .withDocumentation("Merge class provide stateless component interface for merging records, and support various HoodieRecord "
-          + "types, such as Spark records or Flink records.");
-
   public static final ConfigProperty<String> COMPACTION_LAZY_BLOCK_READ_ENABLE = ConfigProperty
       .key("hoodie.compaction.lazy.block.read")
       .defaultValue("true")
@@ -359,11 +352,6 @@ public class HoodieCompactionConfig extends HoodieConfig {
       return this;
     }
 
-    public Builder withMergeClass(String mergeClass) {
-      compactionConfig.setValue(MERGE_CLASS_NAME, mergeClass);
-      return this;
-    }
-
     public Builder withTargetIOPerCompactionInMB(long targetIOPerCompactionInMB) {
       compactionConfig.setValue(TARGET_IO_PER_COMPACTION_IN_MB, String.valueOf(targetIOPerCompactionInMB));
       return this;
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
index cc5428f4e6..b04f57ada5 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java
@@ -29,14 +29,16 @@ import org.apache.hudi.common.config.HoodieCommonConfig;
 import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.common.config.HoodieMetadataConfig;
 import org.apache.hudi.common.config.HoodieMetastoreConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.engine.EngineType;
 import org.apache.hudi.common.fs.ConsistencyGuardConfig;
 import org.apache.hudi.common.fs.FileSystemRetryConfig;
-import org.apache.hudi.common.model.HoodieAvroRecordMerge;
+import org.apache.hudi.common.model.HoodieAvroRecordMerger;
 import org.apache.hudi.common.model.HoodieCleaningPolicy;
 import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
 import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.model.HoodieRecordMerger;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
 import org.apache.hudi.common.model.WriteConcurrencyMode;
@@ -45,6 +47,7 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock;
 import org.apache.hudi.common.table.marker.MarkerType;
 import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
 import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
+import org.apache.hudi.common.util.HoodieRecordUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ReflectionUtils;
 import org.apache.hudi.common.util.StringUtils;
@@ -126,11 +129,17 @@ public class HoodieWriteConfig extends HoodieConfig {
       .withDocumentation("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting. "
           + "This will render any value set for PRECOMBINE_FIELD_OPT_VAL in-effective");
 
-  public static final ConfigProperty<String> MERGE_CLASS_NAME = ConfigProperty
-      .key("hoodie.datasource.write.merge.class")
-      .defaultValue(HoodieAvroRecordMerge.class.getName())
-      .withDocumentation("Merge class provide stateless component interface for merging records, and support various HoodieRecord "
-          + "types, such as Spark records or Flink records.");
+  public static final ConfigProperty<String> MERGER_IMPLS = ConfigProperty
+      .key("hoodie.datasource.write.merger.impls")
+      .defaultValue(HoodieAvroRecordMerger.class.getName())
+      .withDocumentation("List of HoodieMerger implementations constituting Hudi's merging strategy -- based on the engine used. "
+          + "These merger impls will filter by hoodie.datasource.write.merger.strategy "
+          + "Hudi will pick most efficient implementation to perform merging/combining of the records (during update, reading MOR table, etc)");
+
+  public static final ConfigProperty<String> MERGER_STRATEGY = ConfigProperty
+      .key("hoodie.datasource.write.merger.strategy")
+      .defaultValue(StringUtils.DEFAULT_MERGER_STRATEGY_UUID)
+      .withDocumentation("Id of merger strategy.  Hudi will pick RecordMergers in hoodie.datasource.write.merger.impls which has the same merger strategy id");
 
   public static final ConfigProperty<String> KEYGENERATOR_CLASS_NAME = ConfigProperty
       .key("hoodie.datasource.write.keygenerator.class")
@@ -505,7 +514,9 @@ public class HoodieWriteConfig extends HoodieConfig {
   private HoodieMetadataConfig metadataConfig;
   private HoodieMetastoreConfig metastoreConfig;
   private HoodieCommonConfig commonConfig;
+  private HoodieStorageConfig storageConfig;
   private EngineType engineType;
+  private HoodieRecordMerger recordMerger;
 
   /**
    * @deprecated Use {@link #TBL_NAME} and its methods instead
@@ -882,6 +893,7 @@ public class HoodieWriteConfig extends HoodieConfig {
     super();
     this.engineType = EngineType.SPARK;
     this.clientSpecifiedViewStorageConfig = null;
+    applyMergerClass();
   }
 
   protected HoodieWriteConfig(EngineType engineType, Properties props) {
@@ -889,6 +901,7 @@ public class HoodieWriteConfig extends HoodieConfig {
     Properties newProps = new Properties();
     newProps.putAll(props);
     this.engineType = engineType;
+    applyMergerClass();
     this.consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().fromProperties(newProps).build();
     this.fileSystemRetryConfig = FileSystemRetryConfig.newBuilder().fromProperties(newProps).build();
     this.clientSpecifiedViewStorageConfig = FileSystemViewStorageConfig.newBuilder().fromProperties(newProps).build();
@@ -897,6 +910,16 @@ public class HoodieWriteConfig extends HoodieConfig {
     this.metadataConfig = HoodieMetadataConfig.newBuilder().fromProperties(props).build();
     this.metastoreConfig = HoodieMetastoreConfig.newBuilder().fromProperties(props).build();
     this.commonConfig = HoodieCommonConfig.newBuilder().fromProperties(props).build();
+    this.storageConfig = HoodieStorageConfig.newBuilder().fromProperties(props).build();
+  }
+
+  private void applyMergerClass() {
+    List<String> mergers = getSplitStringsOrDefault(MERGER_IMPLS).stream()
+        .map(String::trim)
+        .distinct()
+        .collect(Collectors.toList());
+    String mergerStrategy = getString(MERGER_STRATEGY);
+    this.recordMerger = HoodieRecordUtils.generateRecordMerger(getString(BASE_PATH), engineType, mergers, mergerStrategy);
   }
 
   public static HoodieWriteConfig.Builder newBuilder() {
@@ -910,6 +933,10 @@ public class HoodieWriteConfig extends HoodieConfig {
     return getString(BASE_PATH);
   }
 
+  public HoodieRecordMerger getRecordMerger() {
+    return recordMerger;
+  }
+
   public String getSchema() {
     return getString(AVRO_SCHEMA_STRING);
   }
@@ -918,6 +945,10 @@ public class HoodieWriteConfig extends HoodieConfig {
     setValue(AVRO_SCHEMA_STRING, schemaStr);
   }
 
+  public void setMergerClass(String mergerStrategy) {
+    setValue(MERGER_STRATEGY, mergerStrategy);
+  }
+
   public String getInternalSchema() {
     return getString(INTERNAL_SCHEMA_STRING);
   }
@@ -1339,10 +1370,6 @@ public class HoodieWriteConfig extends HoodieConfig {
     return getString(HoodiePayloadConfig.PAYLOAD_CLASS_NAME);
   }
 
-  public String getMergeClass() {
-    return getString(HoodieCompactionConfig.MERGE_CLASS_NAME);
-  }
-
   public int getTargetPartitionsPerDayBasedCompaction() {
     return getInt(HoodieCompactionConfig.TARGET_PARTITIONS_PER_DAYBASED_COMPACTION);
   }
@@ -1963,6 +1990,10 @@ public class HoodieWriteConfig extends HoodieConfig {
     return commonConfig;
   }
 
+  public HoodieStorageConfig getStorageConfig() {
+    return storageConfig;
+  }
+
   /**
    * Commit call back configs.
    */
@@ -2262,6 +2293,16 @@ public class HoodieWriteConfig extends HoodieConfig {
       return this;
     }
 
+    public Builder withMergerImpls(String mergerImpls) {
+      writeConfig.setValue(MERGER_IMPLS, mergerImpls);
+      return this;
+    }
+
+    public Builder withMergerStrategy(String mergerStrategy) {
+      writeConfig.setValue(MERGER_STRATEGY, mergerStrategy);
+      return this;
+    }
+
     public Builder withKeyGenerator(String keyGeneratorClass) {
       writeConfig.setValue(KEYGENERATOR_CLASS_NAME, keyGeneratorClass);
       return this;
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java
index e5f8d901c8..ad0cb61fd3 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java
@@ -24,6 +24,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.model.HoodieRecordLocation;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.util.HoodieTimer;
@@ -31,7 +32,7 @@ import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.exception.HoodieIndexException;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.log4j.LogManager;
@@ -132,7 +133,7 @@ public class HoodieIndexUtils {
       // Load all rowKeys from the file, to double-confirm
       if (!candidateRecordKeys.isEmpty()) {
         HoodieTimer timer = new HoodieTimer().startTimer();
-        HoodieAvroFileReader fileReader = HoodieFileReaderFactory.getFileReader(configuration, filePath);
+        HoodieFileReader fileReader = HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(configuration, filePath);
         Set<String> fileRowKeys = fileReader.filterRowKeys(new TreeSet<>(candidateRecordKeys));
         foundRecordKeys.addAll(fileRowKeys);
         LOG.info(String.format("Checked keys against file %s, in %d ms. #candidates (%d) #found (%d)", filePath,
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java
index b4f252db3a..3d2e830c46 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java
@@ -28,6 +28,7 @@ import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.BaseFile;
 import org.apache.hudi.common.model.DeleteRecord;
 import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieColumnRangeMetadata;
 import org.apache.hudi.common.model.HoodieDeltaWriteStat;
 import org.apache.hudi.common.model.HoodieLogFile;
@@ -36,7 +37,6 @@ import org.apache.hudi.common.model.HoodiePartitionMetadata;
 import org.apache.hudi.common.model.HoodiePayloadProps;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordLocation;
-import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
 import org.apache.hudi.common.model.IOType;
 import org.apache.hudi.common.table.log.AppendResult;
@@ -217,13 +217,18 @@ public class HoodieAppendHandle<T, I, K, O> extends HoodieWriteHandle<T, I, K, O
       boolean nullifyPayload = HoodieOperation.isDelete(hoodieRecord.getOperation()) && !config.allowOperationMetadataField();
       recordProperties.put(HoodiePayloadProps.PAYLOAD_IS_UPDATE_RECORD_FOR_MOR, String.valueOf(isUpdateRecord));
       Option<HoodieRecord> finalRecord = Option.empty();
-      if (!nullifyPayload && hoodieRecord.isPresent(tableSchema, recordProperties)) {
+      if (!nullifyPayload && !hoodieRecord.isDelete(tableSchema, recordProperties)) {
         if (hoodieRecord.shouldIgnore(tableSchema, recordProperties)) {
           return Option.of(hoodieRecord);
         }
         // Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
-        HoodieRecord rewrittenRecord = hoodieRecord.rewriteRecord(tableSchema, recordProperties, schemaOnReadEnabled, writeSchemaWithMetaFields);
-        HoodieRecord populatedRecord = populateMetadataFields(rewrittenRecord, tableSchema, recordProperties);
+        HoodieRecord rewrittenRecord;
+        if (schemaOnReadEnabled) {
+          rewrittenRecord = hoodieRecord.rewriteRecordWithNewSchema(tableSchema, recordProperties, writeSchemaWithMetaFields);
+        } else {
+          rewrittenRecord = hoodieRecord.rewriteRecord(tableSchema, recordProperties, writeSchemaWithMetaFields);
+        }
+        HoodieRecord populatedRecord = populateMetadataFields(rewrittenRecord, writeSchemaWithMetaFields, recordProperties);
         finalRecord = Option.of(populatedRecord);
         if (isUpdateRecord) {
           updatedRecordsWritten++;
@@ -249,21 +254,21 @@ public class HoodieAppendHandle<T, I, K, O> extends HoodieWriteHandle<T, I, K, O
   }
 
   private HoodieRecord populateMetadataFields(HoodieRecord<T> hoodieRecord, Schema schema, Properties prop) throws IOException {
-    Map<HoodieRecord.HoodieMetadataField, String> metadataValues = new HashMap<>();
+    Map<String, String> metadataValues = new HashMap<>();
     String seqId =
         HoodieRecord.generateSequenceId(instantTime, getPartitionId(), RECORD_COUNTER.getAndIncrement());
     if (config.populateMetaFields()) {
-      metadataValues.put(HoodieRecord.HoodieMetadataField.FILENAME_METADATA_FIELD, fileId);
-      metadataValues.put(HoodieRecord.HoodieMetadataField.PARTITION_PATH_METADATA_FIELD, partitionPath);
-      metadataValues.put(HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD, hoodieRecord.getRecordKey());
-      metadataValues.put(HoodieRecord.HoodieMetadataField.COMMIT_TIME_METADATA_FIELD, instantTime);
-      metadataValues.put(HoodieRecord.HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD, seqId);
+      metadataValues.put(HoodieRecord.HoodieMetadataField.FILENAME_METADATA_FIELD.getFieldName(), fileId);
+      metadataValues.put(HoodieRecord.HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.getFieldName(), partitionPath);
+      metadataValues.put(HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD.getFieldName(), hoodieRecord.getRecordKey());
+      metadataValues.put(HoodieRecord.HoodieMetadataField.COMMIT_TIME_METADATA_FIELD.getFieldName(), instantTime);
+      metadataValues.put(HoodieRecord.HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD.getFieldName(), seqId);
     }
     if (config.allowOperationMetadataField()) {
-      metadataValues.put(HoodieRecord.HoodieMetadataField.OPERATION_METADATA_FIELD, hoodieRecord.getOperation().getName());
+      metadataValues.put(HoodieRecord.HoodieMetadataField.OPERATION_METADATA_FIELD.getFieldName(), hoodieRecord.getOperation().getName());
     }
 
-    return hoodieRecord.addMetadataValues(schema, prop, metadataValues);
+    return hoodieRecord.updateValues(schema, prop, metadataValues);
   }
 
   private void initNewStatus() {
@@ -376,7 +381,7 @@ public class HoodieAppendHandle<T, I, K, O> extends HoodieWriteHandle<T, I, K, O
 
       List<IndexedRecord> indexedRecords = new LinkedList<>();
       for (HoodieRecord hoodieRecord : recordList) {
-        indexedRecords.add((IndexedRecord) hoodieRecord.toIndexedRecord(tableSchema, config.getProps()).get());
+        indexedRecords.add(((HoodieAvroIndexedRecord) hoodieRecord.toIndexedRecord(tableSchema, config.getProps()).get()).getData());
       }
 
       Map<String, HoodieColumnRangeMetadata<Comparable>> columnRangesMetadataMap =
@@ -439,7 +444,7 @@ public class HoodieAppendHandle<T, I, K, O> extends HoodieWriteHandle<T, I, K, O
 
   @Override
   protected void doWrite(HoodieRecord record, Schema schema, TypedProperties props) {
-    Option<Map<String, String>> recordMetadata = ((HoodieRecordPayload) record.getData()).getMetadata();
+    Option<Map<String, String>> recordMetadata = record.getMetadata();
     try {
       init(record);
       flushToDiskIfRequired(record);
@@ -524,7 +529,7 @@ public class HoodieAppendHandle<T, I, K, O> extends HoodieWriteHandle<T, I, K, O
       record.seal();
     }
     // fetch the ordering val first in case the record was deflated.
-    final Comparable<?> orderingVal = record.getOrderingValue();
+    final Comparable<?> orderingVal = record.getOrderingValue(config.getProps());
     Option<HoodieRecord> indexedRecord = prepareRecord(record);
     if (indexedRecord.isPresent()) {
       // Skip the ignored record.
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieConcatHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieConcatHandle.java
index c428ab0622..4f6b0428b2 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieConcatHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieConcatHandle.java
@@ -18,7 +18,6 @@
 
 package org.apache.hudi.io;
 
-import org.apache.avro.Schema;
 import org.apache.hudi.common.engine.TaskContextSupplier;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieKey;
@@ -30,6 +29,7 @@ import org.apache.hudi.exception.HoodieUpsertException;
 import org.apache.hudi.keygen.BaseKeyGenerator;
 import org.apache.hudi.table.HoodieTable;
 
+import org.apache.avro.Schema;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
@@ -97,7 +97,7 @@ public class HoodieConcatHandle<T, I, K, O> extends HoodieMergeHandle<T, I, K, O
     Schema schema = useWriterSchemaForCompaction ? tableSchemaWithMetaFields : tableSchema;
     try {
       // NOTE: We're enforcing preservation of the record metadata to keep existing semantic
-      writeToFile(new HoodieKey(key, partitionPath), oldRecord, schema, config.getProps(), true);
+      writeToFile(new HoodieKey(key, partitionPath), oldRecord, schema, config.getPayloadConfig().getProps(), true);
     } catch (IOException | RuntimeException e) {
       String errMsg = String.format("Failed to write old record into new file for key %s from old file %s to new file %s with writerSchema %s",
           key, getOldFilePath(), newFilePath, writeSchemaWithMetaFields.toString(true));
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java
index 7e55b84135..d6269a4fc9 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java
@@ -27,6 +27,7 @@ import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieOperation;
 import org.apache.hudi.common.model.HoodiePartitionMetadata;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField;
 import org.apache.hudi.common.model.HoodieRecordLocation;
 import org.apache.hudi.common.model.HoodieWriteStat;
 import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
@@ -99,8 +100,8 @@ public class HoodieCreateHandle<T, I, K, O> extends HoodieWriteHandle<T, I, K, O
           hoodieTable.getPartitionMetafileFormat());
       partitionMetadata.trySave(getPartitionId());
       createMarkerFile(partitionPath, FSUtils.makeBaseFileName(this.instantTime, this.writeToken, this.fileId, hoodieTable.getBaseFileExtension()));
-      this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config,
-        writeSchemaWithMetaFields, this.taskContextSupplier);
+      this.fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable.getHadoopConf(), config,
+        writeSchemaWithMetaFields, this.taskContextSupplier, config.getRecordMerger().getRecordType());
     } catch (IOException e) {
       throw new HoodieInsertException("Failed to initialize HoodieStorageWriter for path " + path, e);
     }
@@ -131,17 +132,22 @@ public class HoodieCreateHandle<T, I, K, O> extends HoodieWriteHandle<T, I, K, O
   protected void doWrite(HoodieRecord record, Schema schema, TypedProperties props) {
     Option<Map<String, String>> recordMetadata = record.getMetadata();
     try {
-      if (!HoodieOperation.isDelete(record.getOperation()) && record.isPresent(schema, config.getProps())) {
+      if (!HoodieOperation.isDelete(record.getOperation()) && !record.isDelete(schema, config.getProps())) {
         if (record.shouldIgnore(schema, config.getProps())) {
           return;
         }
         // Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
+        HoodieRecord rewriteRecord;
+        if (schemaOnReadEnabled) {
+          rewriteRecord = record.rewriteRecordWithNewSchema(schema, config.getProps(), writeSchemaWithMetaFields);
+        } else {
+          rewriteRecord = record.rewriteRecord(schema, config.getProps(), writeSchemaWithMetaFields);
+        }
+        rewriteRecord = rewriteRecord.updateValues(writeSchemaWithMetaFields, config.getProps(), Collections.singletonMap(HoodieMetadataField.FILENAME_METADATA_FIELD.getFieldName(), path.getName()));
         if (preserveMetadata) {
-          fileWriter.write(record.getRecordKey(), record.rewriteRecordWithMetadata(
-              schema, config.getProps(), schemaOnReadEnabled, writeSchemaWithMetaFields, path.getName()), writeSchemaWithMetaFields);
+          fileWriter.write(record.getRecordKey(), rewriteRecord, writeSchemaWithMetaFields);
         } else {
-          fileWriter.writeWithMetadata(record.getKey(), record.rewriteRecordWithMetadata(
-              schema, config.getProps(), schemaOnReadEnabled, writeSchemaWithMetaFields, path.getName()), writeSchemaWithMetaFields);
+          fileWriter.writeWithMetadata(record.getKey(), rewriteRecord, writeSchemaWithMetaFields);
         }
         // update the new location of record, so we know where to find it next
         record.unseal();
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java
index bc1da8c217..b55969d613 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieKeyLookupHandle.java
@@ -26,7 +26,7 @@ import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieIndexException;
 import org.apache.hudi.index.HoodieIndexUtils;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -66,7 +66,7 @@ public class HoodieKeyLookupHandle<T, I, K, O> extends HoodieReadHandle<T, I, K,
         bloomFilter = hoodieTable.getMetadataTable().getBloomFilter(partitionPathFileIDPair.getLeft(), partitionPathFileIDPair.getRight())
             .orElseThrow(() -> new HoodieIndexException("BloomFilter missing for " + partitionPathFileIDPair.getRight()));
       } else {
-        try (HoodieAvroFileReader reader = createNewFileReader()) {
+        try (HoodieFileReader reader = createNewFileReader()) {
           bloomFilter = reader.readBloomFilter();
         }
       }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
index fff4aa6d05..edbfa747f3 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
@@ -27,6 +27,7 @@ import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieOperation;
 import org.apache.hudi.common.model.HoodiePartitionMetadata;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField;
 import org.apache.hudi.common.model.HoodieRecordLocation;
 import org.apache.hudi.common.model.HoodieWriteStat;
 import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
@@ -40,7 +41,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieCorruptedDataException;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.exception.HoodieUpsertException;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 import org.apache.hudi.io.storage.HoodieFileWriter;
 import org.apache.hudi.keygen.BaseKeyGenerator;
@@ -279,7 +280,7 @@ public class HoodieMergeHandle<T, I, K, O> extends HoodieWriteHandle<T, I, K, O>
         return false;
       }
     }
-    return writeRecord(hoodieRecord, combineRecordOp, schema, config.getProps(), isDelete);
+    return writeRecord(hoodieRecord, combineRecordOp, schema, config.getPayloadConfig().getProps(), isDelete);
   }
 
   protected void writeInsertRecord(HoodieRecord<T> hoodieRecord) throws IOException {
@@ -288,7 +289,7 @@ public class HoodieMergeHandle<T, I, K, O> extends HoodieWriteHandle<T, I, K, O>
     if (hoodieRecord.shouldIgnore(schema, config.getProps())) {
       return;
     }
-    if (writeRecord(hoodieRecord, Option.of(hoodieRecord), schema, config.getProps(), HoodieOperation.isDelete(hoodieRecord.getOperation()))) {
+    if (writeRecord(hoodieRecord, Option.of(hoodieRecord), schema, config.getPayloadConfig().getProps(), HoodieOperation.isDelete(hoodieRecord.getOperation()))) {
       insertRecordsWritten++;
     }
   }
@@ -306,7 +307,7 @@ public class HoodieMergeHandle<T, I, K, O> extends HoodieWriteHandle<T, I, K, O>
       return false;
     }
     try {
-      if (combineRecord.isPresent() && combineRecord.get().isPresent(schema, config.getProps()) && !isDelete) {
+      if (combineRecord.isPresent() && !combineRecord.get().isDelete(schema, config.getProps()) && !isDelete) {
         writeToFile(hoodieRecord.getKey(), combineRecord.get(), schema, prop, preserveMetadata && useWriterSchemaForCompaction);
         recordsWritten++;
       } else {
@@ -329,16 +330,16 @@ public class HoodieMergeHandle<T, I, K, O> extends HoodieWriteHandle<T, I, K, O>
    * Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
    */
   public void write(HoodieRecord<T> oldRecord) {
-    String key = oldRecord.getRecordKey(keyGeneratorOpt);
-    boolean copyOldRecord = true;
     Schema schema = useWriterSchemaForCompaction ? tableSchemaWithMetaFields : tableSchema;
+    boolean copyOldRecord = true;
+    String key = oldRecord.getRecordKey(keyGeneratorOpt);
     TypedProperties props = config.getPayloadConfig().getProps();
     if (keyToNewRecords.containsKey(key)) {
       // If we have duplicate records that we are updating, then the hoodie record will be deflated after
       // writing the first record. So make a copy of the record to be merged
       HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key).newInstance();
       try {
-        Option<HoodieRecord> combinedRecord = merge.combineAndGetUpdateValue(oldRecord, hoodieRecord, schema, props);
+        Option<HoodieRecord> combinedRecord = recordMerger.merge(oldRecord, hoodieRecord, schema, props);
 
         if (combinedRecord.isPresent() && combinedRecord.get().shouldIgnore(schema, props)) {
           // If it is an IGNORE_RECORD, just copy the old record, and do not update the new record.
@@ -374,12 +375,19 @@ public class HoodieMergeHandle<T, I, K, O> extends HoodieWriteHandle<T, I, K, O>
   }
 
   protected void writeToFile(HoodieKey key, HoodieRecord<T> record, Schema schema, Properties prop, boolean shouldPreserveRecordMetadata) throws IOException {
+    HoodieRecord rewriteRecord;
+    if (schemaOnReadEnabled) {
+      rewriteRecord = record.rewriteRecordWithNewSchema(schema, prop, writeSchemaWithMetaFields);
+    } else {
+      rewriteRecord = record.rewriteRecord(schema, prop, writeSchemaWithMetaFields);
+    }
+    rewriteRecord = rewriteRecord.updateValues(writeSchemaWithMetaFields, prop, Collections.singletonMap(HoodieMetadataField.FILENAME_METADATA_FIELD.getFieldName(), newFilePath.getName()));
     if (shouldPreserveRecordMetadata) {
       // NOTE: `FILENAME_METADATA_FIELD` has to be rewritten to correctly point to the
       //       file holding this record even in cases when overall metadata is preserved
-      fileWriter.write(key.getRecordKey(), record.rewriteRecordWithMetadata(schema, prop, schemaOnReadEnabled, writeSchemaWithMetaFields, newFilePath.getName()), writeSchemaWithMetaFields);
+      fileWriter.write(key.getRecordKey(), rewriteRecord, writeSchemaWithMetaFields);
     } else {
-      fileWriter.writeWithMetadata(key, record.rewriteRecord(schema, prop, schemaOnReadEnabled, writeSchemaWithMetaFields), writeSchemaWithMetaFields);
+      fileWriter.writeWithMetadata(key, rewriteRecord, writeSchemaWithMetaFields);
     }
   }
 
@@ -444,7 +452,7 @@ public class HoodieMergeHandle<T, I, K, O> extends HoodieWriteHandle<T, I, K, O>
 
     long oldNumWrites = 0;
     try {
-      HoodieAvroFileReader reader = HoodieFileReaderFactory.getFileReader(hoodieTable.getHadoopConf(), oldFilePath);
+      HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(this.config.getRecordMerger().getRecordType()).getFileReader(hoodieTable.getHadoopConf(), oldFilePath);
       oldNumWrites = reader.getTotalRecords();
     } catch (IOException e) {
       throw new HoodieUpsertException("Failed to check for merge data validation", e);
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java
index 753d1db301..98092fd1e8 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieRangeInfoHandle.java
@@ -20,7 +20,7 @@ package org.apache.hudi.io;
 
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 import org.apache.hudi.table.HoodieTable;
 
 import java.io.IOException;
@@ -36,7 +36,7 @@ public class HoodieRangeInfoHandle<T, I, K, O> extends HoodieReadHandle<T, I, K,
   }
 
   public String[] getMinMaxKeys() throws IOException {
-    try (HoodieAvroFileReader reader = createNewFileReader()) {
+    try (HoodieFileReader reader = createNewFileReader()) {
       return reader.readMinMaxRecordKeys();
     }
   }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java
index b699e9700c..5f4c10bf96 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieReadHandle.java
@@ -24,7 +24,7 @@ import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 import org.apache.hudi.table.HoodieTable;
 
@@ -61,8 +61,8 @@ public abstract class HoodieReadHandle<T, I, K, O> extends HoodieIOHandle<T, I,
         .getLatestBaseFile(partitionPathFileIDPair.getLeft(), partitionPathFileIDPair.getRight()).get();
   }
 
-  protected HoodieAvroFileReader createNewFileReader() throws IOException {
-    return HoodieFileReaderFactory.getFileReader(hoodieTable.getHadoopConf(),
+  protected HoodieFileReader createNewFileReader() throws IOException {
+    return HoodieFileReaderFactory.getReaderFactory(this.config.getRecordMerger().getRecordType()).getFileReader(hoodieTable.getHadoopConf(),
         new Path(getLatestDataFile().getPath()));
   }
 }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java
index 9ee6e0884d..24d0b0bc07 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieWriteHandle.java
@@ -27,9 +27,8 @@ import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.engine.TaskContextSupplier;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieMerge;
+import org.apache.hudi.common.model.HoodieRecordMerger;
 import org.apache.hudi.common.model.IOType;
-import org.apache.hudi.common.util.HoodieRecordUtils;
 import org.apache.hudi.common.util.HoodieTimer;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ReflectionUtils;
@@ -61,7 +60,7 @@ public abstract class HoodieWriteHandle<T, I, K, O> extends HoodieIOHandle<T, I,
    */
   protected final Schema tableSchema;
   protected final Schema tableSchemaWithMetaFields;
-  protected final HoodieMerge merge;
+  protected final HoodieRecordMerger recordMerger;
 
   /**
    * The write schema. In most case the write schema is the same to the
@@ -106,7 +105,7 @@ public abstract class HoodieWriteHandle<T, I, K, O> extends HoodieIOHandle<T, I,
     this.taskContextSupplier = taskContextSupplier;
     this.writeToken = makeWriteToken();
     schemaOnReadEnabled = !isNullOrEmpty(hoodieTable.getConfig().getInternalSchema());
-    this.merge = HoodieRecordUtils.loadMerge(config.getMergeClass());
+    this.recordMerger = config.getRecordMerger();
   }
 
   /**
@@ -231,6 +230,6 @@ public abstract class HoodieWriteHandle<T, I, K, O> extends HoodieIOHandle<T, I,
 
   protected HoodieFileWriter createNewFileWriter(String instantTime, Path path, HoodieTable<T, I, K, O> hoodieTable,
                                                  HoodieWriteConfig config, Schema schema, TaskContextSupplier taskContextSupplier) throws IOException {
-    return HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable, config, schema, taskContextSupplier);
+    return HoodieFileWriterFactory.getFileWriter(instantTime, path, hoodieTable.getHadoopConf(), config, schema, taskContextSupplier, config.getRecordMerger().getRecordType());
   }
 }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java
deleted file mode 100644
index 5083ccf6c0..0000000000
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * 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.avro.HoodieAvroWriteSupport;
-import org.apache.hudi.common.bloom.BloomFilter;
-import org.apache.hudi.common.bloom.BloomFilterFactory;
-import org.apache.hudi.common.engine.TaskContextSupplier;
-import org.apache.hudi.common.fs.FSUtils;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.config.HoodieWriteConfig;
-import org.apache.hudi.table.HoodieTable;
-
-import org.apache.avro.Schema;
-import org.apache.avro.generic.IndexedRecord;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.parquet.avro.AvroSchemaConverter;
-
-import java.io.IOException;
-
-import static org.apache.hudi.common.model.HoodieFileFormat.HFILE;
-import static org.apache.hudi.common.model.HoodieFileFormat.ORC;
-import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
-import static org.apache.hudi.io.storage.HoodieHFileConfig.CACHE_DATA_IN_L1;
-import static org.apache.hudi.io.storage.HoodieHFileConfig.DROP_BEHIND_CACHE_COMPACTION;
-import static org.apache.hudi.io.storage.HoodieHFileConfig.HFILE_COMPARATOR;
-import static org.apache.hudi.io.storage.HoodieHFileConfig.PREFETCH_ON_OPEN;
-
-public class HoodieFileWriterFactory {
-
-  public static <T, I, K, O> HoodieFileWriter getFileWriter(
-      String instantTime, Path path, HoodieTable<T, I, K, O> hoodieTable, HoodieWriteConfig config, Schema schema,
-      TaskContextSupplier taskContextSupplier) throws IOException {
-    final String extension = FSUtils.getFileExtension(path.getName());
-    if (PARQUET.getFileExtension().equals(extension)) {
-      return newParquetFileWriter(instantTime, path, config, schema, hoodieTable, taskContextSupplier, config.populateMetaFields());
-    }
-    if (HFILE.getFileExtension().equals(extension)) {
-      return newHFileFileWriter(
-          instantTime, path, config, schema, hoodieTable.getHadoopConf(), taskContextSupplier);
-    }
-    if (ORC.getFileExtension().equals(extension)) {
-      return newOrcFileWriter(
-          instantTime, path, config, schema, hoodieTable.getHadoopConf(), taskContextSupplier);
-    }
-    throw new UnsupportedOperationException(extension + " format not supported yet.");
-  }
-
-  private static <T> HoodieAvroFileWriter newParquetFileWriter(
-      String instantTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable,
-      TaskContextSupplier taskContextSupplier, boolean populateMetaFields) throws IOException {
-    return newParquetFileWriter(instantTime, path, config, schema, hoodieTable.getHadoopConf(),
-        taskContextSupplier, populateMetaFields, populateMetaFields);
-  }
-
-  private static <T, R extends IndexedRecord> HoodieAvroFileWriter newParquetFileWriter(
-      String instantTime, Path path, HoodieWriteConfig config, Schema schema, Configuration conf,
-      TaskContextSupplier taskContextSupplier, boolean populateMetaFields, boolean enableBloomFilter) throws IOException {
-    Option<BloomFilter> filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty();
-    HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(conf).convert(schema), schema, filter);
-
-    HoodieParquetConfig<HoodieAvroWriteSupport> parquetConfig = new HoodieParquetConfig<>(writeSupport, config.getParquetCompressionCodec(),
-        config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetMaxFileSize(),
-        conf, config.getParquetCompressionRatio(), config.parquetDictionaryEnabled());
-
-    return new HoodieAvroParquetWriter(path, parquetConfig, instantTime, taskContextSupplier, populateMetaFields);
-  }
-
-  static <T, R extends IndexedRecord> HoodieAvroFileWriter newHFileFileWriter(
-      String instantTime, Path path, HoodieWriteConfig config, Schema schema, Configuration conf,
-      TaskContextSupplier taskContextSupplier) throws IOException {
-
-    BloomFilter filter = createBloomFilter(config);
-    HoodieHFileConfig hfileConfig = new HoodieHFileConfig(conf,
-        config.getHFileCompressionAlgorithm(), config.getHFileBlockSize(), config.getHFileMaxFileSize(),
-        HoodieAvroHFileReader.KEY_FIELD_NAME, PREFETCH_ON_OPEN, CACHE_DATA_IN_L1, DROP_BEHIND_CACHE_COMPACTION,
-        filter, HFILE_COMPARATOR);
-
-    return new HoodieAvroHFileWriter(instantTime, path, hfileConfig, schema, taskContextSupplier, config.populateMetaFields());
-  }
-
-  private static <T, R extends IndexedRecord> HoodieAvroFileWriter newOrcFileWriter(
-      String instantTime, Path path, HoodieWriteConfig config, Schema schema, Configuration conf,
-      TaskContextSupplier taskContextSupplier) throws IOException {
-    BloomFilter filter = createBloomFilter(config);
-    HoodieOrcConfig orcConfig = new HoodieOrcConfig(conf, config.getOrcCompressionCodec(),
-        config.getOrcStripeSize(), config.getOrcBlockSize(), config.getOrcMaxFileSize(), filter);
-    return new HoodieAvroOrcWriter(instantTime, path, orcConfig, schema, taskContextSupplier);
-  }
-
-  private static BloomFilter createBloomFilter(HoodieWriteConfig config) {
-    return BloomFilterFactory.createBloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP(),
-        config.getDynamicBloomFilterMaxNumEntries(),
-        config.getBloomFilterType());
-  }
-}
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java
index a4b09d006e..d2de068838 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/ClusteringExecutionStrategy.java
@@ -20,6 +20,7 @@ package org.apache.hudi.table.action.cluster.strategy;
 
 import org.apache.hudi.avro.model.HoodieClusteringPlan;
 import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.action.HoodieWriteMetadata;
@@ -39,11 +40,13 @@ public abstract class ClusteringExecutionStrategy<T, I, K, O> implements Seriali
   private final HoodieTable<T, I, K, O> hoodieTable;
   private final transient HoodieEngineContext engineContext;
   private final HoodieWriteConfig writeConfig;
+  protected final HoodieRecordType recordType;
 
   public ClusteringExecutionStrategy(HoodieTable table, HoodieEngineContext engineContext, HoodieWriteConfig writeConfig) {
     this.writeConfig = writeConfig;
     this.hoodieTable = table;
     this.engineContext = engineContext;
+    this.recordType = table.getConfig().getRecordMerger().getRecordType();
   }
 
   /**
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java
index 393ee9ddb6..a3a3188df9 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseMergeHelper.java
@@ -19,31 +19,22 @@
 package org.apache.hudi.table.action.commit;
 
 import org.apache.hudi.client.utils.MergingIterator;
-import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
-import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.io.HoodieMergeHandle;
 import org.apache.hudi.io.storage.HoodieFileReader;
 import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 import org.apache.hudi.table.HoodieTable;
 
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericDatumWriter;
-import org.apache.avro.io.BinaryDecoder;
-import org.apache.avro.io.BinaryEncoder;
-import org.apache.avro.io.DecoderFactory;
-import org.apache.avro.io.EncoderFactory;
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 
 import javax.annotation.Nonnull;
 
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.util.Iterator;
 
@@ -60,32 +51,6 @@ public abstract class BaseMergeHelper<T, I, K, O> {
    */
   public abstract void runMerge(HoodieTable<T, I, K, O> table, HoodieMergeHandle<T, I, K, O> upsertHandle) throws IOException;
 
-  protected HoodieRecord transformRecordBasedOnNewSchema(GenericDatumReader<GenericRecord> gReader, GenericDatumWriter<GenericRecord> gWriter,
-                                                               ThreadLocal<BinaryEncoder> encoderCache, ThreadLocal<BinaryDecoder> decoderCache,
-                                                               GenericRecord gRec) {
-    ByteArrayOutputStream inStream = null;
-    try {
-      inStream = new ByteArrayOutputStream();
-      BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(inStream, encoderCache.get());
-      encoderCache.set(encoder);
-      gWriter.write(gRec, encoder);
-      encoder.flush();
-
-      BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inStream.toByteArray(), decoderCache.get());
-      decoderCache.set(decoder);
-      GenericRecord transformedRec = gReader.read(null, decoder);
-      return new HoodieAvroIndexedRecord(transformedRec);
-    } catch (IOException e) {
-      throw new HoodieException(e);
-    } finally {
-      try {
-        inStream.close();
-      } catch (IOException ioe) {
-        throw new HoodieException(ioe.getMessage(), ioe);
-      }
-    }
-  }
-
   /**
    * Create Parquet record iterator that provides a stitched view of record read from skeleton and bootstrap file.
    * Skeleton file is a representation of the bootstrap file inside the table, with just the bare bone fields needed
@@ -99,7 +64,8 @@ public abstract class BaseMergeHelper<T, I, K, O> {
                                                       boolean externalSchemaTransformation) throws IOException {
     Path externalFilePath = new Path(baseFile.getBootstrapBaseFile().get().getPath());
     Configuration bootstrapFileConfig = new Configuration(table.getHadoopConf());
-    HoodieFileReader bootstrapReader = HoodieFileReaderFactory.getFileReader(bootstrapFileConfig, externalFilePath);
+    HoodieRecordType recordType = table.getConfig().getRecordMerger().getRecordType();
+    HoodieFileReader bootstrapReader = HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(bootstrapFileConfig, externalFilePath);
 
     Schema bootstrapReadSchema;
     if (externalSchemaTransformation) {
@@ -109,15 +75,15 @@ public abstract class BaseMergeHelper<T, I, K, O> {
     }
 
     return new MergingIterator<>(
-        reader.getRecordIterator(readerSchema, HoodieAvroIndexedRecord::new),
-        bootstrapReader.getRecordIterator(bootstrapReadSchema, HoodieAvroIndexedRecord::new),
-        (oneRecord, otherRecord) -> mergeRecords(oneRecord, otherRecord, readerSchema, mergeHandle.getWriterSchemaWithMetaFields()));
+        (Iterator<HoodieRecord>) reader.getRecordIterator(readerSchema),
+        (Iterator<HoodieRecord>) bootstrapReader.getRecordIterator(bootstrapReadSchema),
+        (oneRecord, otherRecord) -> mergeRecords(oneRecord, otherRecord, mergeHandle.getWriterSchemaWithMetaFields()));
   }
 
   @Nonnull
-  private static HoodieRecord mergeRecords(HoodieRecord one, HoodieRecord other, Schema readerSchema, Schema writerSchema) {
+  private static HoodieRecord mergeRecords(HoodieRecord left, HoodieRecord right, Schema targetSchema) {
     try {
-      return one.mergeWith(other, readerSchema, writerSchema);
+      return left.mergeWith(right, targetSchema);
     } catch (IOException e) {
       throw new HoodieIOException("Failed to merge records", e);
     }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java
index 1efe3d9641..9133358b7f 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseWriteHelper.java
@@ -19,9 +19,8 @@
 package org.apache.hudi.table.action.commit;
 
 import org.apache.hudi.common.engine.HoodieEngineContext;
-import org.apache.hudi.common.model.HoodieMerge;
+import org.apache.hudi.common.model.HoodieRecordMerger;
 import org.apache.hudi.common.model.WriteOperationType;
-import org.apache.hudi.common.util.HoodieRecordUtils;
 import org.apache.hudi.exception.HoodieUpsertException;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.table.HoodieTable;
@@ -30,6 +29,7 @@ import org.apache.hudi.table.action.HoodieWriteMetadata;
 
 import java.time.Duration;
 import java.time.Instant;
+import java.util.Properties;
 
 public abstract class BaseWriteHelper<T, I, K, O, R> {
 
@@ -83,10 +83,10 @@ public abstract class BaseWriteHelper<T, I, K, O, R> {
    */
   public I deduplicateRecords(
       I records, HoodieTable<T, I, K, O> table, int parallelism) {
-    HoodieMerge merge = HoodieRecordUtils.loadMerge(table.getConfig().getMergeClass());
-    return deduplicateRecords(records, table.getIndex(), parallelism, merge);
+    HoodieRecordMerger recordMerger = table.getConfig().getRecordMerger();
+    return deduplicateRecords(records, table.getIndex(), parallelism, recordMerger, table.getConfig().getProps());
   }
 
   public abstract I deduplicateRecords(
-      I records, HoodieIndex<?, ?> index, int parallelism, HoodieMerge merge);
+      I records, HoodieIndex<?, ?> index, int parallelism, HoodieRecordMerger recordMerger, Properties props);
 }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java
index f54184abb0..0d212555ab 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java
@@ -24,8 +24,10 @@ import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieEmptyRecord;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieUpsertException;
@@ -84,8 +86,15 @@ public class HoodieDeleteHelper<T, R> extends
         dedupedKeys = keys.repartition(parallelism);
       }
 
-      HoodieData<HoodieRecord<T>> dedupedRecords =
-          dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
+      HoodieData dedupedRecords;
+      HoodieRecordType recordType = config.getRecordMerger().getRecordType();
+      if (recordType == HoodieRecordType.AVRO) {
+        // For BWC, will remove when HoodieRecordPayload removed
+        dedupedRecords =
+            dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
+      } else {
+        dedupedRecords = dedupedKeys.map(key -> new HoodieEmptyRecord<>(key, recordType));
+      }
       Instant beginTag = Instant.now();
       // perform index loop up to get existing location of records
       HoodieData<HoodieRecord<T>> taggedRecords = table.getIndex().tagLocation(dedupedRecords, context, table);
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java
index 8f3eb67e9d..49edb981a5 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java
@@ -24,7 +24,6 @@ import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieBaseFile;
-import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.util.ClosableIterator;
@@ -44,11 +43,7 @@ import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 import org.apache.hudi.table.HoodieTable;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.BinaryDecoder;
-import org.apache.avro.io.BinaryEncoder;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -84,22 +79,25 @@ public class HoodieMergeHelper<T> extends
     Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf());
     HoodieBaseFile baseFile = mergeHandle.baseFileForMerge();
 
-    final GenericDatumWriter<GenericRecord> gWriter;
-    final GenericDatumReader<GenericRecord> gReader;
+    // Support schema evolution
     Schema readSchema;
+    // These two schema used to replace gWriter and gReader.
+    // In previous logic, avro record is serialized by gWriter and then is deserialized by gReader.
+    // Now we converge this logic in record#rewrite.
+    Schema readerSchema;
+    Schema writerSchema;
+    HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(table.getConfig().getRecordMerger().getRecordType()).getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath());
     if (externalSchemaTransformation || baseFile.getBootstrapBaseFile().isPresent()) {
-      readSchema = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), mergeHandle.getOldFilePath()).getSchema();
-      gWriter = new GenericDatumWriter<>(readSchema);
-      gReader = new GenericDatumReader<>(readSchema, mergeHandle.getWriterSchemaWithMetaFields());
+      readSchema = reader.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());
-
     Option<InternalSchema> querySchemaOpt = SerDeHelper.fromJson(table.getConfig().getInternalSchema());
     boolean needToReWriteRecord = false;
     Map<String, String> renameCols = new HashMap<>();
@@ -135,21 +133,22 @@ public class HoodieMergeHelper<T> extends
         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, new Properties(), readerSchema);
+        } catch (IOException e) {
+          throw new HoodieException(String.format("Failed to rewrite record. WriterSchema: %s; ReaderSchema: %s", writerSchema, readerSchema), e);
+        }
       }, table.getPreExecuteRunnable());
       wrapper.execute();
     } catch (Exception e) {
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java
index 57bb511c63..ad664606c7 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java
@@ -23,11 +23,15 @@ import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieMerge;
+import org.apache.hudi.common.model.HoodieRecordMerger;
 import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.table.HoodieTable;
 
+import java.io.IOException;
+import java.util.Properties;
+
 public class HoodieWriteHelper<T, R> extends BaseWriteHelper<T, HoodieData<HoodieRecord<T>>,
     HoodieData<HoodieKey>, HoodieData<WriteStatus>, R> {
   private HoodieWriteHelper() {
@@ -49,7 +53,7 @@ public class HoodieWriteHelper<T, R> extends BaseWriteHelper<T, HoodieData<Hoodi
 
   @Override
   public HoodieData<HoodieRecord<T>> deduplicateRecords(
-      HoodieData<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism, HoodieMerge merge) {
+      HoodieData<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism, HoodieRecordMerger recordMerger, Properties props) {
     boolean isIndexingGlobal = index.isGlobal();
     return records.mapToPair(record -> {
       HoodieKey hoodieKey = record.getKey();
@@ -57,8 +61,13 @@ public class HoodieWriteHelper<T, R> extends BaseWriteHelper<T, HoodieData<Hoodi
       Object key = isIndexingGlobal ? hoodieKey.getRecordKey() : hoodieKey;
       return Pair.of(key, record);
     }).reduceByKey((rec1, rec2) -> {
-      @SuppressWarnings("unchecked")
-      HoodieRecord<T> reducedRecord =  merge.preCombine(rec1, rec2);
+      HoodieRecord<T> reducedRecord;
+      try {
+        // Precombine do not need schema and do not return null
+        reducedRecord =  recordMerger.merge(rec1, rec2, null, props).get();
+      } catch (IOException e) {
+        throw new HoodieException(String.format("Error to merge two records, %s, %s", rec1, rec2), e);
+      }
       HoodieKey reducedKey = rec1.getData().equals(reducedRecord.getData()) ? rec1.getKey() : rec2.getKey();
       return reducedRecord.newInstance(reducedKey);
     }, parallelism).map(Pair::getRight);
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java
index 69c6b48fea..23eab31ab3 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java
@@ -194,6 +194,7 @@ public abstract class HoodieCompactor<T, I, K, O> implements Serializable {
         .withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
         .withOperationField(config.allowOperationMetadataField())
         .withPartition(operation.getPartitionPath())
+        .withRecordMerger(config.getRecordMerger())
         .build();
 
     Option<HoodieBaseFile> oldDataFileOpt =
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java
index 89e3a7738a..5fb7065a8a 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieHFileReaderWriter.java
@@ -33,7 +33,6 @@ import org.apache.hudi.common.engine.TaskContextSupplier;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieAvroRecord;
-import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.util.FileIOUtils;
@@ -105,8 +104,8 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
     when(mockTaskContextSupplier.getPartitionIdSupplier()).thenReturn(partitionSupplier);
     when(partitionSupplier.get()).thenReturn(10);
 
-    return (HoodieAvroHFileWriter)HoodieFileWriterFactory.newHFileFileWriter(
-        instantTime, getFilePath(), writeConfig, avroSchema, conf, mockTaskContextSupplier);
+    return (HoodieAvroHFileWriter)HoodieFileWriterFactory.getFileWriter(
+        instantTime, getFilePath(), conf, writeConfig.getStorageConfig(), avroSchema, mockTaskContextSupplier, writeConfig.getRecordMerger().getRecordType());
   }
 
   @Override
@@ -229,7 +228,7 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
         IntStream.concat(IntStream.range(40, NUM_RECORDS * 2), IntStream.range(10, 20))
             .mapToObj(i -> "key" + String.format("%02d", i)).collect(Collectors.toList());
     Schema avroSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, "/exampleSchema.avsc");
-    Iterator<IndexedRecord> iterator = hfileReader.getRecordsByKeysIterator(keys, avroSchema);
+    Iterator<IndexedRecord> iterator = hfileReader.getIndexedRecordsByKeysIterator(keys, avroSchema);
 
     List<Integer> expectedIds =
         IntStream.concat(IntStream.range(40, NUM_RECORDS), IntStream.range(10, 20))
@@ -255,18 +254,19 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
 
     List<String> keyPrefixes = Collections.singletonList("key");
     Iterator<IndexedRecord> iterator =
-        hfileReader.getRecordsByKeyPrefixIterator(keyPrefixes, avroSchema);
+        hfileReader.getIndexedRecordsByKeyPrefixIterator(keyPrefixes, avroSchema);
 
     List<GenericRecord> recordsByPrefix = toStream(iterator).map(r -> (GenericRecord)r).collect(Collectors.toList());
 
-    List<GenericRecord> allRecords = toStream(hfileReader.getRecordIterator(HoodieAvroIndexedRecord::new)).map(r -> (GenericRecord)r.getData()).collect(Collectors.toList());
+    List<GenericRecord> allRecords = toStream(hfileReader.getRecordIterator())
+        .map(r -> (GenericRecord) r.getData()).collect(Collectors.toList());
 
     assertEquals(allRecords, recordsByPrefix);
 
     // filter for "key1" : entries from key10 to key19 should be matched
     List<GenericRecord> expectedKey1s = allRecords.stream().filter(entry -> (entry.get("_row_key").toString()).contains("key1")).collect(Collectors.toList());
     iterator =
-        hfileReader.getRecordsByKeyPrefixIterator(Collections.singletonList("key1"), avroSchema);
+        hfileReader.getIndexedRecordsByKeyPrefixIterator(Collections.singletonList("key1"), avroSchema);
     recordsByPrefix =
         StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
             .map(r -> (GenericRecord)r)
@@ -276,7 +276,7 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
     // exact match
     List<GenericRecord> expectedKey25 = allRecords.stream().filter(entry -> (entry.get("_row_key").toString()).contains("key25")).collect(Collectors.toList());
     iterator =
-        hfileReader.getRecordsByKeyPrefixIterator(Collections.singletonList("key25"), avroSchema);
+        hfileReader.getIndexedRecordsByKeyPrefixIterator(Collections.singletonList("key25"), avroSchema);
     recordsByPrefix =
         StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
             .map(r -> (GenericRecord)r)
@@ -285,7 +285,7 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
 
     // no match. key prefix is beyond entries in file.
     iterator =
-        hfileReader.getRecordsByKeyPrefixIterator(Collections.singletonList("key99"), avroSchema);
+        hfileReader.getIndexedRecordsByKeyPrefixIterator(Collections.singletonList("key99"), avroSchema);
     recordsByPrefix =
         StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
             .map(r -> (GenericRecord)r)
@@ -294,7 +294,7 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
 
     // no match. but keyPrefix is in between the entries found in file.
     iterator =
-        hfileReader.getRecordsByKeyPrefixIterator(Collections.singletonList("key1234"), avroSchema);
+        hfileReader.getIndexedRecordsByKeyPrefixIterator(Collections.singletonList("key1234"), avroSchema);
     recordsByPrefix =
         StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
             .map(r -> (GenericRecord)r)
@@ -305,7 +305,7 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
     List<GenericRecord> expectedKey50and1s = allRecords.stream().filter(entry -> (entry.get("_row_key").toString()).contains("key1")
         || (entry.get("_row_key").toString()).contains("key50")).collect(Collectors.toList());
     iterator =
-        hfileReader.getRecordsByKeyPrefixIterator(Arrays.asList("key50", "key1"), avroSchema);
+        hfileReader.getIndexedRecordsByKeyPrefixIterator(Arrays.asList("key50", "key1"), avroSchema);
     recordsByPrefix =
         StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
             .map(r -> (GenericRecord)r)
@@ -316,7 +316,7 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
     List<GenericRecord> expectedKey50and0s = allRecords.stream().filter(entry -> (entry.get("_row_key").toString()).contains("key0")
         || (entry.get("_row_key").toString()).contains("key50")).collect(Collectors.toList());
     iterator =
-        hfileReader.getRecordsByKeyPrefixIterator(Arrays.asList("key50", "key0"), avroSchema);
+        hfileReader.getIndexedRecordsByKeyPrefixIterator(Arrays.asList("key50", "key0"), avroSchema);
     recordsByPrefix =
         StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
             .map(r -> (GenericRecord)r)
@@ -328,7 +328,7 @@ public class TestHoodieHFileReaderWriter extends TestHoodieReaderWriterBase {
         .filter(entry -> (entry.get("_row_key").toString()).contains("key1") || (entry.get("_row_key").toString()).contains("key0"))
         .collect(Collectors.toList());
     iterator =
-        hfileReader.getRecordsByKeyPrefixIterator(Arrays.asList("key1", "key0"), avroSchema);
+        hfileReader.getIndexedRecordsByKeyPrefixIterator(Arrays.asList("key1", "key0"), avroSchema);
     recordsByPrefix =
         StreamSupport.stream(Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED), false)
             .map(r -> (GenericRecord)r)
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java
index 10e0ce2256..1bd2a0cf93 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java
@@ -22,7 +22,8 @@ import org.apache.hudi.common.bloom.BloomFilter;
 import org.apache.hudi.common.bloom.BloomFilterFactory;
 import org.apache.hudi.common.bloom.BloomFilterTypeCode;
 import org.apache.hudi.common.engine.TaskContextSupplier;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 
 import org.apache.avro.Schema;
 import org.apache.hadoop.conf.Configuration;
@@ -70,7 +71,7 @@ public class TestHoodieOrcReaderWriter extends TestHoodieReaderWriterBase {
   @Override
   protected HoodieAvroFileReader createReader(
       Configuration conf) throws Exception {
-    return HoodieFileReaderFactory.getFileReader(conf, getFilePath());
+    return (HoodieAvroFileReader) HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(conf, getFilePath());
   }
 
   @Override
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java
index cf701bc017..fde1315a34 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieReaderWriterBase.java
@@ -113,7 +113,7 @@ public abstract class TestHoodieReaderWriterBase {
     Configuration conf = new Configuration();
     verifyMetadata(conf);
     verifySchema(conf, schemaPath);
-    verifySimpleRecords(new TransformIterator(createReader(conf).getRecordIterator(HoodieAvroIndexedRecord::new)));
+    verifySimpleRecords(createReader(conf).getRecordIterator());
   }
 
   @Test
@@ -140,7 +140,7 @@ public abstract class TestHoodieReaderWriterBase {
     Configuration conf = new Configuration();
     verifyMetadata(conf);
     verifySchema(conf, schemaPath);
-    verifyComplexRecords(new TransformIterator(createReader(conf).getRecordIterator(HoodieAvroIndexedRecord::new)));
+    verifyComplexRecords(createReader(conf).getRecordIterator());
   }
 
   @Test
@@ -197,10 +197,10 @@ public abstract class TestHoodieReaderWriterBase {
     writer.close();
   }
 
-  protected void verifySimpleRecords(Iterator<IndexedRecord> iterator) {
+  protected void verifySimpleRecords(Iterator<HoodieRecord<IndexedRecord>> iterator) {
     int index = 0;
     while (iterator.hasNext()) {
-      GenericRecord record = (GenericRecord) iterator.next();
+      GenericRecord record = (GenericRecord) iterator.next().getData();
       String key = "key" + String.format("%02d", index);
       assertEquals(key, record.get("_row_key").toString());
       assertEquals(Integer.toString(index), record.get("time").toString());
@@ -209,10 +209,10 @@ public abstract class TestHoodieReaderWriterBase {
     }
   }
 
-  protected void verifyComplexRecords(Iterator<IndexedRecord> iterator) {
+  protected void verifyComplexRecords(Iterator<HoodieRecord<IndexedRecord>> iterator) {
     int index = 0;
     while (iterator.hasNext()) {
-      GenericRecord record = (GenericRecord) iterator.next();
+      GenericRecord record = (GenericRecord) iterator.next().getData();
       String key = "key" + String.format("%02d", index);
       assertEquals(key, record.get("_row_key").toString());
       assertEquals(Integer.toString(index), record.get("time").toString());
@@ -247,10 +247,10 @@ public abstract class TestHoodieReaderWriterBase {
 
   private void verifyReaderWithSchema(String schemaPath, HoodieAvroFileReader hoodieReader) throws IOException {
     Schema evolvedSchema = getSchemaFromResource(TestHoodieReaderWriterBase.class, schemaPath);
-    Iterator<IndexedRecord> iter = hoodieReader.getRecordIterator(evolvedSchema);
+    Iterator<HoodieRecord<IndexedRecord>> iter = hoodieReader.getRecordIterator(evolvedSchema);
     int index = 0;
     while (iter.hasNext()) {
-      verifyRecord(schemaPath, (GenericRecord) iter.next(), index);
+      verifyRecord(schemaPath, (GenericRecord) iter.next().getData(), index);
       index++;
     }
   }
@@ -268,24 +268,4 @@ public abstract class TestHoodieReaderWriterBase {
     }
     assertNull(record.get("added_field"));
   }
-
-  class TransformIterator implements Iterator<IndexedRecord> {
-
-    private final Iterator<HoodieRecord> iter;
-
-    public TransformIterator(Iterator<HoodieRecord> iter) {
-      this.iter = iter;
-    }
-
-    @Override
-    public boolean hasNext() {
-      return iter.hasNext();
-    }
-
-    @Override
-    public IndexedRecord next() {
-      return (GenericRecord) iter.next().getData();
-
-    }
-  }
 }
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java
index a952576026..3ab7ccc9fb 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/testutils/HoodieWriteableTestTable.java
@@ -44,7 +44,7 @@ import org.apache.hudi.common.testutils.FileCreateUtils;
 import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.io.storage.HoodieAvroOrcWriter;
 import org.apache.hudi.io.storage.HoodieAvroParquetWriter;
 import org.apache.hudi.io.storage.HoodieOrcConfig;
diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java
index 7d778008ec..a15a438c7a 100644
--- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java
+++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java
@@ -20,7 +20,6 @@ package org.apache.hudi.table.action.commit;
 
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.model.HoodieBaseFile;
-import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.util.Option;
@@ -28,21 +27,18 @@ import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
 import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.io.HoodieMergeHandle;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 import org.apache.hudi.table.HoodieTable;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.BinaryDecoder;
-import org.apache.avro.io.BinaryEncoder;
 import org.apache.hadoop.conf.Configuration;
 
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Properties;
 
 /**
  * Flink merge helper.
@@ -64,42 +60,47 @@ public class FlinkMergeHelper<T> extends BaseMergeHelper<T, List<HoodieRecord<T>
   @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;
+    // Support schema evolution
     Schema readSchema;
+    // These two schema used to replace gWriter and gReader.
+    // In previous logic, avro record is serialized by gWriter and then is deserialized by gReader.
+    // Now we converge this logic in record#rewrite.
+    Schema readerSchema;
+    Schema writerSchema;
 
     final boolean externalSchemaTransformation = table.getConfig().shouldUseExternalSchemaTransformation();
     HoodieBaseFile baseFile = mergeHandle.baseFileForMerge();
+    Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf());
+    HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(table.getConfig().getRecordMerger().getRecordType()).getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath());
     if (externalSchemaTransformation || baseFile.getBootstrapBaseFile().isPresent()) {
-      readSchema = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), mergeHandle.getOldFilePath()).getSchema();
-      gWriter = new GenericDatumWriter<>(readSchema);
-      gReader = new GenericDatumReader<>(readSchema, mergeHandle.getWriterSchemaWithMetaFields());
+      readSchema = reader.getSchema();
+      writerSchema = readSchema;
+      readerSchema = mergeHandle.getWriterSchemaWithMetaFields();
     } else {
-      gReader = null;
-      gWriter = null;
+      readerSchema = null;
+      writerSchema = null;
       readSchema = mergeHandle.getWriterSchemaWithMetaFields();
     }
 
     BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
-    Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf());
-    HoodieAvroFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath());
     try {
       final Iterator<HoodieRecord> readerIterator;
       if (baseFile.getBootstrapBaseFile().isPresent()) {
         readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation);
       } 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(), new IteratorBasedQueueProducer<>(readerIterator),
           Option.of(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, new Properties(), readerSchema);
+        } catch (IOException e) {
+          throw new HoodieException(e);
+        }
       });
       wrapper.execute();
     } catch (Exception e) {
diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java
index ce6107714d..3fde3a54e8 100644
--- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java
+++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkWriteHelper.java
@@ -24,18 +24,21 @@ import org.apache.hudi.common.engine.HoodieEngineContext;
 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.model.HoodieMerge;
+import org.apache.hudi.common.model.HoodieRecordMerger;
 import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieUpsertException;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.action.HoodieWriteMetadata;
 
+import java.io.IOException;
 import java.time.Duration;
 import java.time.Instant;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Properties;
 import java.util.stream.Collectors;
 
 /**
@@ -87,21 +90,26 @@ public class FlinkWriteHelper<T, R> extends BaseWriteHelper<T, List<HoodieRecord
 
   @Override
   public List<HoodieRecord<T>> deduplicateRecords(
-      List<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism, HoodieMerge merge) {
+      List<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism, HoodieRecordMerger recordMerger, Properties props) {
     // If index used is global, then records are expected to differ in their partitionPath
     Map<Object, List<HoodieRecord<T>>> keyedRecords = records.stream()
         .collect(Collectors.groupingBy(record -> record.getKey().getRecordKey()));
 
     return keyedRecords.values().stream().map(x -> x.stream().reduce((rec1, rec2) -> {
-      @SuppressWarnings("unchecked")
-      final HoodieRecord reducedRec =  merge.preCombine(rec1, rec2);
+      HoodieRecord<T> reducedRecord;
+      try {
+        // Precombine do not need schema and do not return null
+        reducedRecord =  recordMerger.merge(rec1, rec2, null, props).get();
+      } catch (IOException e) {
+        throw new HoodieException(String.format("Error to merge two records, %s, %s", rec1, rec2), e);
+      }
       // we cannot allow the user to change the key or partitionPath, since that will affect
       // everything
       // so pick it from one of the records.
-      boolean choosePrev = rec1 == reducedRec;
+      boolean choosePrev = rec1 == reducedRecord;
       HoodieKey reducedKey = choosePrev ? rec1.getKey() : rec2.getKey();
       HoodieOperation operation = choosePrev ? rec1.getOperation() : rec2.getOperation();
-      HoodieRecord<T> hoodieRecord = reducedRec.newInstance(reducedKey, operation);
+      HoodieRecord<T> hoodieRecord = reducedRecord.newInstance(reducedKey, operation);
       // reuse the location from the first record.
       hoodieRecord.setCurrentLocation(rec1.getCurrentLocation());
       return hoodieRecord;
diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java
index 7db20bdf9b..fe398238da 100644
--- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java
+++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaExecutionStrategy.java
@@ -20,8 +20,6 @@
 package org.apache.hudi.client.clustering.run.strategy;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.generic.IndexedRecord;
 import org.apache.hadoop.fs.Path;
 import org.apache.hudi.avro.HoodieAvroUtils;
 import org.apache.hudi.avro.model.HoodieClusteringGroup;
@@ -32,12 +30,9 @@ import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.data.HoodieListData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.model.ClusteringOperation;
-import org.apache.hudi.common.model.HoodieAvroRecord;
 import org.apache.hudi.common.model.HoodieFileGroupId;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.model.RewriteAvroPayload;
 import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
 import org.apache.hudi.common.util.Option;
@@ -48,10 +43,8 @@ import org.apache.hudi.exception.HoodieClusteringException;
 import org.apache.hudi.execution.bulkinsert.JavaBulkInsertInternalPartitionerFactory;
 import org.apache.hudi.execution.bulkinsert.JavaCustomColumnsSortPartitioner;
 import org.apache.hudi.io.IOUtils;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 import org.apache.hudi.io.storage.HoodieFileReaderFactory;
-import org.apache.hudi.keygen.BaseKeyGenerator;
-import org.apache.hudi.keygen.KeyGenUtils;
 import org.apache.hudi.table.BulkInsertPartitioner;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.action.HoodieWriteMetadata;
@@ -64,6 +57,7 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 import java.util.stream.Collectors;
 
 import static org.apache.hudi.common.table.log.HoodieFileSliceReader.getFileSliceReader;
@@ -192,15 +186,15 @@ public abstract class JavaExecutionStrategy<T>
             .withPartition(clusteringOp.getPartitionPath())
             .withDiskMapType(config.getCommonConfig().getSpillableDiskMapType())
             .withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
+            .withRecordMerger(config.getRecordMerger())
             .build();
 
-        Option<HoodieAvroFileReader> baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath())
+        Option<HoodieFileReader> baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath())
             ? Option.empty()
-            : Option.of(HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())));
+            : Option.of(HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())));
         HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig();
         Iterator<HoodieRecord<T>> fileSliceReader = getFileSliceReader(baseFileReader, scanner, readerSchema,
-            tableConfig.getPayloadClass(),
-            tableConfig.getPreCombineField(),
+            tableConfig.getProps(),
             tableConfig.populateMetaFields() ? Option.empty() : Option.of(Pair.of(tableConfig.getRecordKeyFieldProp(),
                 tableConfig.getPartitionFieldProp())));
         fileSliceReader.forEachRemaining(records::add);
@@ -220,9 +214,9 @@ public abstract class JavaExecutionStrategy<T>
     clusteringOps.forEach(clusteringOp -> {
       try {
         Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema()));
-        HoodieAvroFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(getHoodieTable().getHadoopConf(), new Path(clusteringOp.getDataFilePath()));
-        Iterator<IndexedRecord> recordIterator = baseFileReader.getRecordIterator(readerSchema);
-        recordIterator.forEachRemaining(record -> records.add(transform(record)));
+        HoodieFileReader baseFileReader = HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(getHoodieTable().getHadoopConf(), new Path(clusteringOp.getDataFilePath()));
+        Iterator<HoodieRecord> recordIterator = baseFileReader.getRecordIterator(readerSchema);
+        recordIterator.forEachRemaining(record -> records.add(record.wrapIntoHoodieRecordPayloadWithKeyGen(new Properties(), Option.empty())));
       } catch (IOException e) {
         throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath()
             + " and " + clusteringOp.getDeltaFilePaths(), e);
@@ -230,19 +224,4 @@ public abstract class JavaExecutionStrategy<T>
     });
     return records;
   }
-
-  /**
-   * Transform IndexedRecord into HoodieRecord.
-   */
-  private HoodieRecord<T> transform(IndexedRecord indexedRecord) {
-    GenericRecord record = (GenericRecord) indexedRecord;
-    Option<BaseKeyGenerator> keyGeneratorOpt = Option.empty();
-    String key = KeyGenUtils.getRecordKeyFromGenericRecord(record, keyGeneratorOpt);
-    String partition = KeyGenUtils.getPartitionPathFromGenericRecord(record, keyGeneratorOpt);
-    HoodieKey hoodieKey = new HoodieKey(key, partition);
-
-    HoodieRecordPayload avroPayload = new RewriteAvroPayload(record);
-    HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, avroPayload);
-    return hoodieRecord;
-  }
 }
diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaSortAndSizeExecutionStrategy.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaSortAndSizeExecutionStrategy.java
index 8258c5e3e0..9e0a675fce 100644
--- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaSortAndSizeExecutionStrategy.java
+++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/JavaSortAndSizeExecutionStrategy.java
@@ -24,7 +24,7 @@ import org.apache.hudi.common.engine.EngineType;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieFileGroupId;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.io.CreateHandleFactory;
 import org.apache.hudi.table.HoodieTable;
diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java
index 4dcd0a6f7b..9593b5e72f 100644
--- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java
+++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java
@@ -20,7 +20,6 @@ package org.apache.hudi.table.action.commit;
 
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.model.HoodieBaseFile;
-import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.util.Option;
@@ -33,16 +32,13 @@ import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 import org.apache.hudi.table.HoodieTable;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericDatumReader;
-import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.io.BinaryDecoder;
-import org.apache.avro.io.BinaryEncoder;
 import org.apache.hadoop.conf.Configuration;
 
 import java.io.IOException;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Properties;
 
 public class JavaMergeHelper<T> extends BaseMergeHelper<T, List<HoodieRecord<T>>,
     List<HoodieKey>, List<WriteStatus>> {
@@ -66,38 +62,43 @@ public class JavaMergeHelper<T> extends BaseMergeHelper<T, List<HoodieRecord<T>>
     HoodieMergeHandle<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> mergeHandle = upsertHandle;
     HoodieBaseFile baseFile = mergeHandle.baseFileForMerge();
 
-    final GenericDatumWriter<GenericRecord> gWriter;
-    final GenericDatumReader<GenericRecord> gReader;
+    // Support schema evolution
     Schema readSchema;
+    // These two schema used to replace gWriter and gReader.
+    // In previous logic, avro record is serialized by gWriter and then is deserialized by gReader.
+    // Now we converge this logic in record#rewrite.
+    Schema readerSchema;
+    Schema writerSchema;
+    HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(table.getConfig().getRecordMerger().getRecordType()).getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath());
     if (externalSchemaTransformation || baseFile.getBootstrapBaseFile().isPresent()) {
-      readSchema = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), mergeHandle.getOldFilePath()).getSchema();
-      gWriter = new GenericDatumWriter<>(readSchema);
-      gReader = new GenericDatumReader<>(readSchema, mergeHandle.getWriterSchemaWithMetaFields());
+      readSchema = reader.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());
     try {
       final Iterator<HoodieRecord> readerIterator;
       if (baseFile.getBootstrapBaseFile().isPresent()) {
         readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation);
       } 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(), new IteratorBasedQueueProducer<>(readerIterator),
           Option.of(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, new Properties(), readerSchema);
+        } catch (IOException e) {
+          throw new HoodieException(e);
+        }
       });
       wrapper.execute();
     } catch (Exception e) {
diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java
index 39c60447c5..cc3d28fb20 100644
--- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java
+++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaWriteHelper.java
@@ -23,14 +23,17 @@ import org.apache.hudi.common.data.HoodieListData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieMerge;
+import org.apache.hudi.common.model.HoodieRecordMerger;
 import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.table.HoodieTable;
 
+import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Properties;
 import java.util.stream.Collectors;
 
 public class JavaWriteHelper<T,R> extends BaseWriteHelper<T, List<HoodieRecord<T>>,
@@ -54,7 +57,7 @@ public class JavaWriteHelper<T,R> extends BaseWriteHelper<T, List<HoodieRecord<T
 
   @Override
   public List<HoodieRecord<T>> deduplicateRecords(
-      List<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism, HoodieMerge merge) {
+      List<HoodieRecord<T>> records, HoodieIndex<?, ?> index, int parallelism, HoodieRecordMerger recordMerger, Properties props) {
     boolean isIndexingGlobal = index.isGlobal();
     Map<Object, List<Pair<Object, HoodieRecord<T>>>> keyedRecords = records.stream().map(record -> {
       HoodieKey hoodieKey = record.getKey();
@@ -64,8 +67,13 @@ public class JavaWriteHelper<T,R> extends BaseWriteHelper<T, List<HoodieRecord<T
     }).collect(Collectors.groupingBy(Pair::getLeft));
 
     return keyedRecords.values().stream().map(x -> x.stream().map(Pair::getRight).reduce((rec1, rec2) -> {
-      @SuppressWarnings("unchecked")
-      HoodieRecord<T> reducedRecord =  merge.preCombine(rec1,rec2);
+      HoodieRecord<T> reducedRecord;
+      try {
+        // Precombine do not need schema and do not return null
+        reducedRecord =  recordMerger.merge(rec1, rec2, null, props).get();
+      } catch (IOException e) {
+        throw new HoodieException(String.format("Error to merge two records, %s, %s", rec1, rec2), e);
+      }
       // we cannot allow the user to change the key or partitionPath, since that will affect
       // everything
       // so pick it from one of the records.
diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java
index 28d3ac6e9b..2c0409a502 100644
--- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java
+++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/table/action/commit/TestJavaCopyOnWriteActionExecutor.java
@@ -35,7 +35,7 @@ import org.apache.hudi.common.testutils.Transformations;
 import org.apache.hudi.common.util.BaseFileUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.hadoop.HoodieParquetInputFormat;
 import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java
index 9882709d37..25669efc65 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java
@@ -29,15 +29,14 @@ import org.apache.hudi.common.config.SerializableConfiguration;
 import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.model.ClusteringOperation;
-import org.apache.hudi.common.model.HoodieAvroRecord;
 import org.apache.hudi.common.model.HoodieFileGroupId;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.model.RewriteAvroPayload;
 import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
+import org.apache.hudi.common.util.ClosableIterator;
 import org.apache.hudi.common.util.FutureUtils;
+import org.apache.hudi.common.util.MappingIterator;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.StringUtils;
 import org.apache.hudi.common.util.collection.Pair;
@@ -45,15 +44,13 @@ import org.apache.hudi.config.HoodieClusteringConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.data.HoodieJavaRDD;
 import org.apache.hudi.exception.HoodieClusteringException;
-import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerFactory;
 import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner;
 import org.apache.hudi.execution.bulkinsert.RDDSpatialCurveSortPartitioner;
 import org.apache.hudi.io.IOUtils;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 import org.apache.hudi.keygen.BaseKeyGenerator;
-import org.apache.hudi.keygen.KeyGenUtils;
 import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
 import org.apache.hudi.table.BulkInsertPartitioner;
 import org.apache.hudi.table.HoodieTable;
@@ -61,8 +58,6 @@ import org.apache.hudi.table.action.HoodieWriteMetadata;
 import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.generic.IndexedRecord;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -78,6 +73,7 @@ import java.util.concurrent.CompletableFuture;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
 import static org.apache.hudi.common.table.log.HoodieFileSliceReader.getFileSliceReader;
 import static org.apache.hudi.config.HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS;
 
@@ -154,7 +150,7 @@ public abstract class MultipleSparkJobExecutionStrategy<T>
               orderByColumns,
               layoutOptStrategy,
               getWriteConfig().getLayoutOptimizationCurveBuildMethod(),
-              HoodieAvroUtils.addMetadataFields(schema));
+              HoodieAvroUtils.addMetadataFields(schema), recordType);
         case LINEAR:
           return new RDDCustomColumnsSortPartitioner(orderByColumns, HoodieAvroUtils.addMetadataFields(schema),
               getWriteConfig().isConsistentLogicalTimestampEnabled());
@@ -225,15 +221,15 @@ public abstract class MultipleSparkJobExecutionStrategy<T>
               .withPartition(clusteringOp.getPartitionPath())
               .withDiskMapType(config.getCommonConfig().getSpillableDiskMapType())
               .withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
+              .withRecordMerger(config.getRecordMerger())
               .build();
 
-          Option<HoodieAvroFileReader> baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath())
+          Option<HoodieFileReader> baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath())
               ? Option.empty()
-              : Option.of(HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())));
+              : Option.of(HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())));
           HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig();
           recordIterators.add(getFileSliceReader(baseFileReader, scanner, readerSchema,
-              tableConfig.getPayloadClass(),
-              tableConfig.getPreCombineField(),
+              tableConfig.getProps(),
               tableConfig.populateMetaFields() ? Option.empty() : Option.of(Pair.of(tableConfig.getRecordKeyFieldProp(),
                   tableConfig.getPartitionFieldProp()))));
         } catch (IOException e) {
@@ -258,12 +254,20 @@ public abstract class MultipleSparkJobExecutionStrategy<T>
     //       closure, as this might lead to issues attempting to serialize its nested fields
     return HoodieJavaRDD.of(jsc.parallelize(clusteringOps, clusteringOps.size())
         .mapPartitions(clusteringOpsPartition -> {
-          List<Iterator<IndexedRecord>> iteratorsForPartition = new ArrayList<>();
+          List<Iterator<HoodieRecord<T>>> iteratorsForPartition = new ArrayList<>();
           clusteringOpsPartition.forEachRemaining(clusteringOp -> {
             try {
               Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(writeConfig.getSchema()));
-              HoodieAvroFileReader baseFileReader = HoodieFileReaderFactory.getFileReader(hadoopConf.get(), new Path(clusteringOp.getDataFilePath()));
-              iteratorsForPartition.add(baseFileReader.getRecordIterator(readerSchema));
+              HoodieFileReader baseFileReader = HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(hadoopConf.get(), new Path(clusteringOp.getDataFilePath()));
+              Option<BaseKeyGenerator> keyGeneratorOp;
+              if (!Boolean.parseBoolean(writeConfig.getProps().getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+                keyGeneratorOp = Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(writeConfig.getProps()));
+              } else {
+                keyGeneratorOp = Option.empty();
+              }
+              MappingIterator mappingIterator = new MappingIterator((ClosableIterator<HoodieRecord>) baseFileReader.getRecordIterator(readerSchema),
+                  rec -> ((HoodieRecord) rec).wrapIntoHoodieRecordPayloadWithKeyGen(writeConfig.getProps(), keyGeneratorOp));
+              iteratorsForPartition.add(mappingIterator);
             } catch (IOException e) {
               throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath()
                   + " and " + clusteringOp.getDeltaFilePaths(), e);
@@ -271,8 +275,7 @@ public abstract class MultipleSparkJobExecutionStrategy<T>
           });
 
           return new ConcatenatingIterator<>(iteratorsForPartition);
-        })
-        .map(record -> transform(record, writeConfig)));
+        }));
   }
 
   /**
@@ -287,26 +290,4 @@ public abstract class MultipleSparkJobExecutionStrategy<T>
     }
     return writeStatusRDDArray;
   }
-
-  /**
-   * Transform IndexedRecord into HoodieRecord.
-   */
-  private static <T> HoodieRecord<T> transform(IndexedRecord indexedRecord, HoodieWriteConfig writeConfig) {
-    GenericRecord record = (GenericRecord) indexedRecord;
-    Option<BaseKeyGenerator> keyGeneratorOpt = Option.empty();
-    if (!writeConfig.populateMetaFields()) {
-      try {
-        keyGeneratorOpt = Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(writeConfig.getProps()));
-      } catch (IOException e) {
-        throw new HoodieIOException("Only BaseKeyGenerators are supported when meta columns are disabled ", e);
-      }
-    }
-    String key = KeyGenUtils.getRecordKeyFromGenericRecord(record, keyGeneratorOpt);
-    String partition = KeyGenUtils.getPartitionPathFromGenericRecord(record, keyGeneratorOpt);
-    HoodieKey hoodieKey = new HoodieKey(key, partition);
-
-    HoodieRecordPayload avroPayload = new RewriteAvroPayload(record);
-    HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, avroPayload);
-    return hoodieRecord;
-  }
 }
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SingleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SingleSparkJobExecutionStrategy.java
index c268992192..918542dbef 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SingleSparkJobExecutionStrategy.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SingleSparkJobExecutionStrategy.java
@@ -24,34 +24,29 @@ import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.client.utils.ConcatenatingIterator;
 import org.apache.hudi.common.config.SerializableSchema;
-import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.engine.TaskContextSupplier;
 import org.apache.hudi.common.model.ClusteringGroupInfo;
 import org.apache.hudi.common.model.ClusteringOperation;
-import org.apache.hudi.common.model.HoodieAvroRecord;
 import org.apache.hudi.common.model.HoodieFileGroupId;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.common.model.RewriteAvroPayload;
+import org.apache.hudi.common.util.ClosableIterator;
+import org.apache.hudi.common.util.MappingIterator;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.data.HoodieJavaRDD;
 import org.apache.hudi.exception.HoodieClusteringException;
-import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.io.storage.HoodieFileReader;
 import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 import org.apache.hudi.keygen.BaseKeyGenerator;
-import org.apache.hudi.keygen.KeyGenUtils;
 import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.action.HoodieWriteMetadata;
 import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.generic.IndexedRecord;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
@@ -68,6 +63,8 @@ import java.util.stream.Collectors;
 import java.util.stream.Stream;
 import java.util.stream.StreamSupport;
 
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+
 /**
  * Clustering strategy to submit single spark jobs.
  * MultipleSparkJobExecution strategy is not ideal for use cases that require large number of clustering groups
@@ -151,40 +148,28 @@ public abstract class SingleSparkJobExecutionStrategy<T>
     List<Iterator<HoodieRecord<T>>> iteratorsForPartition = clusteringOps.stream().map(clusteringOp -> {
 
       Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema()));
-      Iterable<IndexedRecord> indexedRecords = () -> {
+      Iterable<HoodieRecord<T>> indexedRecords = () -> {
         try {
-          return HoodieFileReaderFactory.getFileReader(getHoodieTable().getHadoopConf(), new Path(clusteringOp.getDataFilePath())).getRecordIterator(readerSchema);
+
+          HoodieFileReader baseFileReader = HoodieFileReaderFactory.getReaderFactory(recordType).getFileReader(getHoodieTable().getHadoopConf(), new Path(clusteringOp.getDataFilePath()));
+          Option<BaseKeyGenerator> keyGeneratorOp;
+          if (!Boolean.parseBoolean(getWriteConfig().getProps().getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+            keyGeneratorOp = Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(getWriteConfig().getProps()));
+          } else {
+            keyGeneratorOp = Option.empty();
+          }
+          MappingIterator mappingIterator = new MappingIterator((ClosableIterator<HoodieRecord>) baseFileReader.getRecordIterator(readerSchema),
+              rec -> ((HoodieRecord) rec).wrapIntoHoodieRecordPayloadWithKeyGen(getWriteConfig().getProps(), keyGeneratorOp));
+          return mappingIterator;
         } catch (IOException e) {
           throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath()
               + " and " + clusteringOp.getDeltaFilePaths(), e);
         }
       };
 
-      return StreamSupport.stream(indexedRecords.spliterator(), false).map(record -> transform(record)).iterator();
+      return StreamSupport.stream(indexedRecords.spliterator(), false).iterator();
     }).collect(Collectors.toList());
 
     return new ConcatenatingIterator<>(iteratorsForPartition);
   }
-
-  /**
-   * Transform IndexedRecord into HoodieRecord.
-   */
-  private HoodieRecord<T> transform(IndexedRecord indexedRecord) {
-    GenericRecord record = (GenericRecord) indexedRecord;
-    Option<BaseKeyGenerator> keyGeneratorOpt = Option.empty();
-    if (!getWriteConfig().populateMetaFields()) {
-      try {
-        keyGeneratorOpt = Option.of((BaseKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(getWriteConfig().getProps())));
-      } catch (IOException e) {
-        throw new HoodieIOException("Only BaseKeyGenerators are supported when meta columns are disabled ", e);
-      }
-    }
-    String key = KeyGenUtils.getRecordKeyFromGenericRecord(record, keyGeneratorOpt);
-    String partition = KeyGenUtils.getPartitionPathFromGenericRecord(record, keyGeneratorOpt);
-    HoodieKey hoodieKey = new HoodieKey(key, partition);
-
-    HoodieRecordPayload avroPayload = new RewriteAvroPayload(record);
-    HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, avroPayload);
-    return hoodieRecord;
-  }
 }
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java
index 0edef91210..8a65e49db4 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java
@@ -24,7 +24,7 @@ import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieFileGroupId;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieClusteringException;
 import org.apache.hudi.io.SingleFileHandleCreateFactory;
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java
index 33b7fad439..0f71f78a66 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java
@@ -23,7 +23,7 @@ import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.model.HoodieFileGroupId;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.io.CreateHandleFactory;
 import org.apache.hudi.table.HoodieTable;
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java
index 7425137665..2577f2601c 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/model/HoodieInternalRow.java
@@ -84,7 +84,7 @@ public class HoodieInternalRow extends InternalRow {
     this.sourceContainsMetaFields = sourceContainsMetaFields;
   }
 
-  private HoodieInternalRow(UTF8String[] metaFields,
+  public HoodieInternalRow(UTF8String[] metaFields,
                            InternalRow sourceRow,
                            boolean sourceContainsMetaFields) {
     this.metaFields = metaFields;
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java
new file mode 100644
index 0000000000..f866914b27
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java
@@ -0,0 +1,283 @@
+/*
+ * 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.client.model.HoodieInternalRow;
+import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
+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.ConfigUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.spark.sql.HoodieCatalystExpressionUtils$;
+import org.apache.spark.sql.HoodieUnsafeRowUtils;
+import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Properties;
+
+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> {
+
+  private StructType structType = null;
+  private Option<Long> schemaFingerPrint = Option.empty();
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    initSchema(schema);
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    initSchema(schema);
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation) {
+    super(key, data, operation);
+    initSchema(schema);
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    initSchema(record.getStructType());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, getStructType(), op);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, getStructType());
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get())
+        .getRecordKey(data, getStructType()).toString() : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    DataType dataType = getStructType().apply(keyFieldName).dataType();
+    int pos = getStructType().fieldIndex(keyFieldName);
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(data, columns, getStructType(), consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException {
+    StructType otherStructType = ((HoodieSparkRecord) other).getStructType();
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, getStructType(), (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    UTF8String[] metaFields = extractMetaField(targetStructType);
+    if (metaFields.length == 0) {
+      throw new UnsupportedOperationException();
+    }
+
+    InternalRow resultRow;
+    if (extractMetaField(getStructType()).length == 0) {
+      resultRow = new HoodieInternalRow(metaFields, data, false);
+    } else {
+      resultRow = new HoodieInternalRow(metaFields, data, true);
+    }
+
+    return new HoodieSparkRecord(getKey(), resultRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, getStructType(), newStructType, renameCols);
+    UnsafeProjection unsafeConvert = HoodieInternalRowUtils.getCachedUnsafeConvert(newStructType);
+    InternalRow resultRow = unsafeConvert.apply(rewriteRow);
+    UTF8String[] metaFields = extractMetaField(newStructType);
+    if (metaFields.length > 0) {
+      resultRow = new HoodieInternalRow(metaFields, data, true);
+    }
+
+    return new HoodieSparkRecord(getKey(), resultRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord updateValues(Schema recordSchema, Properties props, Map<String, String> metadataValues) throws IOException {
+    metadataValues.forEach((key, value) -> {
+      int pos = getStructType().fieldIndex(key);
+      if (value != null) {
+        data.update(pos, CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+
+    return new HoodieSparkRecord(getKey(), data, getStructType(), getOperation());
+  }
+
+  @Override
+  public boolean isDelete(Schema schema, Properties props) throws IOException {
+    if (null == data) {
+      return true;
+    }
+    if (schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD) == null) {
+      return false;
+    }
+    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 props) throws IOException {
+    if (data != null && data.equals(SENTINEL)) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public HoodieRecord wrapIntoHoodieRecordPayloadWithParams(
+      Schema schema, Properties props,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Boolean populateMetaFields) {
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(getStructType(), data, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(getStructType(), data, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(getStructType(), data, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord wrapIntoHoodieRecordPayloadWithKeyGen(Properties props, Option<BaseKeyGenerator> keyGen) {
+    String key;
+    String partition;
+    if (keyGen.isPresent() && !Boolean.parseBoolean(props.getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+      SparkKeyGeneratorInterface keyGenerator = (SparkKeyGeneratorInterface) keyGen.get();
+      key = keyGenerator.getRecordKey(data, getStructType()).toString();
+      partition = keyGenerator.getPartitionPath(data, getStructType()).toString();
+    } else {
+      key = data.get(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), StringType).toString();
+      partition = data.get(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), StringType).toString();
+    }
+    HoodieKey hoodieKey = new HoodieKey(key, partition);
+    return new HoodieSparkRecord(hoodieKey, data, getStructType(), getOperation());
+  }
+
+  @Override
+  public Option<Map<String, String>> getMetadata() {
+    return Option.empty();
+  }
+
+  @Override
+  public Option<HoodieAvroIndexedRecord> toIndexedRecord(Schema schema, Properties prop) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Comparable<?> getOrderingValue(Properties props) {
+    String orderingField = ConfigUtils.getOrderingField(props);
+    if (!HoodieCatalystExpressionUtils$.MODULE$.existField(getStructType(), orderingField)) {
+      return 0;
+    } else {
+      NestedFieldPath nestedFieldPath = HoodieInternalRowUtils.getCachedPosList(getStructType(),
+          orderingField);
+      Comparable<?> value = (Comparable<?>) HoodieUnsafeRowUtils.getNestedInternalRowValue(
+          data, nestedFieldPath);
+      return value;
+    }
+  }
+
+  public StructType getStructType() {
+    if (schemaFingerPrint.isPresent()) {
+      return HoodieInternalRowUtils.getCachedSchemaFromFingerPrint(schemaFingerPrint.get());
+    } else {
+      return structType;
+    }
+  }
+
+  private void initSchema(StructType structType) {
+    if (HoodieInternalRowUtils.containsCompressedSchema(structType)) {
+      HoodieInternalRowUtils.addCompressedSchema(structType);
+      this.schemaFingerPrint = Option.of(HoodieInternalRowUtils.getCachedFingerPrintFromSchema(structType));
+    } else {
+      this.structType = structType;
+    }
+  }
+
+  public void setStructType(StructType structType) {
+    if (structType != null) {
+      initSchema(structType);
+    }
+  }
+
+  private UTF8String[] extractMetaField(StructType structType) {
+    return HOODIE_META_COLUMNS_WITH_OPERATION.stream()
+        .filter(f -> HoodieCatalystExpressionUtils$.MODULE$.existField(structType, f))
+        .map(UTF8String::fromString)
+        .toArray(UTF8String[]::new);
+  }
+}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDConsistentBucketPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDConsistentBucketPartitioner.java
index 70258d7938..94e48c959c 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDConsistentBucketPartitioner.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDConsistentBucketPartitioner.java
@@ -22,7 +22,6 @@ import org.apache.hudi.avro.HoodieAvroUtils;
 import org.apache.hudi.common.config.SerializableSchema;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.ConsistentHashingNode;
-import org.apache.hudi.common.model.HoodieAvroRecord;
 import org.apache.hudi.common.model.HoodieConsistentHashingMetadata;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
@@ -235,8 +234,8 @@ public class RDDConsistentBucketPartitioner<T> extends RDDBucketIndexPartitioner
     final String[] sortColumns = sortColumnNames;
     final SerializableSchema schema = new SerializableSchema(HoodieAvroUtils.addMetadataFields((new Schema.Parser().parse(table.getConfig().getSchema()))));
     Comparator<HoodieRecord<T>> comparator = (Comparator<HoodieRecord<T>> & Serializable) (t1, t2) -> {
-      Object obj1 = HoodieAvroUtils.getRecordColumnValues((HoodieAvroRecord) t1, sortColumns, schema, consistentLogicalTimestampEnabled);
-      Object obj2 = HoodieAvroUtils.getRecordColumnValues((HoodieAvroRecord)t2, sortColumns, schema, consistentLogicalTimestampEnabled);
+      Object obj1 = t1.getRecordColumnValues(schema.get(), sortColumns, consistentLogicalTimestampEnabled);
+      Object obj2 = t2.getRecordColumnValues(schema.get(), sortColumns, consistentLogicalTimestampEnabled);
       return ((Comparable) obj1).compareTo(obj2);
     };
 
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java
index 120677f811..dbbcb22e90 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDCustomColumnsSortPartitioner.java
@@ -18,9 +18,7 @@
 
 package org.apache.hudi.execution.bulkinsert;
 
-import org.apache.hudi.avro.HoodieAvroUtils;
 import org.apache.hudi.common.config.SerializableSchema;
-import org.apache.hudi.common.model.HoodieAvroRecord;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.util.StringUtils;
 import org.apache.hudi.config.HoodieWriteConfig;
@@ -63,7 +61,7 @@ public class RDDCustomColumnsSortPartitioner<T>
     final boolean consistentLogicalTimestampEnabled = this.consistentLogicalTimestampEnabled;
     return records.sortBy(
         record -> {
-          Object recordValue = HoodieAvroUtils.getRecordColumnValues((HoodieAvroRecord)record, sortColumns, schema, consistentLogicalTimestampEnabled);
+          Object recordValue = record.getRecordColumnValues(schema.get(), sortColumns, consistentLogicalTimestampEnabled);
           // null values are replaced with empty string for null_first order
           if (recordValue == null) {
             return StringUtils.EMPTY_STRING;
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java
index a5921e5a46..1f74e82616 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java
@@ -19,23 +19,33 @@
 package org.apache.hudi.execution.bulkinsert;
 
 import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.HoodieInternalRowUtils;
 import org.apache.hudi.HoodieSparkUtils;
+import org.apache.hudi.SparkConversionUtils;
 import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
 import org.apache.hudi.common.config.SerializableSchema;
 import org.apache.hudi.common.model.HoodieAvroRecord;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.model.RewriteAvroPayload;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieClusteringConfig;
+import org.apache.hudi.config.HoodieClusteringConfig.LayoutOptimizationStrategy;
+import org.apache.hudi.config.HoodieClusteringConfig.SpatialCurveCompositionStrategyType;
 import org.apache.hudi.sort.SpaceCurveSortingHelper;
 import org.apache.hudi.table.BulkInsertPartitioner;
 
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.broadcast.Broadcast;
 import org.apache.spark.sql.Dataset;
 import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
 
 import java.util.Arrays;
 import java.util.List;
@@ -54,42 +64,65 @@ public class RDDSpatialCurveSortPartitioner<T>
   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().getData());
+
+      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());
+      Broadcast<StructType> structTypeBC = sparkEngineContext.getJavaSparkContext().broadcast(structType);
+      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();
+            String key = internalRow.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+            String partition = internalRow.getString(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal());
+            HoodieKey hoodieKey = new HoodieKey(key, partition);
+            HoodieRecord hoodieRecord = new HoodieSparkRecord(hoodieKey, internalRow, structTypeBC.value());
+            return hoodieRecord;
+          });
+    } else {
+      throw new UnsupportedOperationException(recordType.name());
+    }
   }
 
   private Dataset<Row> reorder(Dataset<Row> dataset, int numOutputGroups) {
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReader.java
new file mode 100644
index 0000000000..86134eda37
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReader.java
@@ -0,0 +1,53 @@
+/*
+ * 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.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+
+import static org.apache.hudi.common.util.TypeUtils.unsafeCast;
+
+public interface HoodieSparkFileReader extends HoodieFileReader<InternalRow> {
+
+  ClosableIterator<InternalRow> getInternalRowIterator(Schema readerSchema) throws IOException;
+
+  ClosableIterator<InternalRow> getInternalRowIterator(Schema readerSchema, Schema requestedSchema) throws IOException;
+
+  default ClosableIterator<HoodieRecord<InternalRow>> getRecordIterator(Schema readerSchema) throws IOException {
+    ClosableIterator<InternalRow> iterator = getInternalRowIterator(readerSchema);
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(readerSchema);
+    return new MappingIterator<>(iterator, data -> unsafeCast(new HoodieSparkRecord(data, structType)));
+  }
+
+  @Override
+  default ClosableIterator<HoodieRecord<InternalRow>> getRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException {
+    ClosableIterator<InternalRow> iterator = getInternalRowIterator(readerSchema, requestedSchema);
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(requestedSchema);
+    return new MappingIterator<>(iterator, data -> unsafeCast(new HoodieSparkRecord(data, structType)));
+  }
+}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java
new file mode 100644
index 0000000000..f0d650ecdc
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReaderFactory.java
@@ -0,0 +1,57 @@
+/*
+ * 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.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.exception.HoodieIOException;
+
+import java.io.IOException;
+import org.apache.spark.sql.internal.SQLConf;
+
+public class HoodieSparkFileReaderFactory extends HoodieFileReaderFactory  {
+
+  private static class SingletonHolder {
+    private static final HoodieSparkFileReaderFactory INSTANCE = new HoodieSparkFileReaderFactory();
+  }
+
+  private HoodieSparkFileReaderFactory() {
+  }
+
+  public static HoodieFileReaderFactory getFileReaderFactory() {
+    return SingletonHolder.INSTANCE;
+  }
+
+  protected HoodieFileReader newParquetFileReader(Configuration conf, Path path) {
+    conf.setIfUnset(SQLConf.PARQUET_BINARY_AS_STRING().key(),
+        SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString());
+    conf.setIfUnset(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(),
+        SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString());
+    conf.setIfUnset(SQLConf.CASE_SENSITIVE().key(), SQLConf.CASE_SENSITIVE().defaultValueString());
+    return new HoodieSparkParquetReader(conf, path);
+  }
+
+  protected HoodieFileReader newHFileFileReader(Configuration conf, Path path) throws IOException {
+    throw new HoodieIOException("Not support read HFile");
+  }
+
+  protected HoodieFileReader newOrcFileReader(Configuration conf, Path path) {
+    throw new HoodieIOException("Not support read orc file");
+  }
+}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriter.java
new file mode 100644
index 0000000000..58ee01182c
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriter.java
@@ -0,0 +1,64 @@
+/*
+ * 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;
+
+import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD;
+import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.COMMIT_TIME_METADATA_FIELD;
+import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.FILENAME_METADATA_FIELD;
+import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.PARTITION_PATH_METADATA_FIELD;
+import static org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD;
+
+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(COMMIT_TIME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(instantTime));
+    row.update(COMMIT_SEQNO_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(seqId));
+    row.update(RECORD_KEY_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getRecordKey()));
+    row.update(PARTITION_PATH_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getPartitionPath()));
+    row.update(FILENAME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(fileName));
+    return row;
+  }
+}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java
new file mode 100644
index 0000000000..f746bb0e8e
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java
@@ -0,0 +1,113 @@
+/*
+ * 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.common.bloom.BloomFilter;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.io.storage.row.HoodieRowParquetConfig;
+import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.io.IOException;
+
+public class HoodieSparkFileWriterFactory extends HoodieFileWriterFactory {
+
+  private static class SingletonHolder {
+
+    private static final HoodieSparkFileWriterFactory INSTANCE = new HoodieSparkFileWriterFactory();
+  }
+
+  public static HoodieFileWriterFactory getFileWriterFactory() {
+    return HoodieSparkFileWriterFactory.SingletonHolder.INSTANCE;
+  }
+
+  @Override
+  protected HoodieFileWriter newParquetFileWriter(
+      String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier) throws IOException {
+    boolean populateMetaFields = config.getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS);
+    boolean enableBloomFilter = populateMetaFields;
+    Option<BloomFilter> filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty();
+    String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME);
+    // Support PARQUET_COMPRESSION_CODEC_NAME is ""
+    if (compressionCodecName.isEmpty()) {
+      compressionCodecName = null;
+    }
+    HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport(conf,
+        HoodieInternalRowUtils.getCachedSchema(schema), filter,
+        HoodieStorageConfig.newBuilder().fromProperties(config.getProps()).build());
+    HoodieRowParquetConfig parquetConfig = new HoodieRowParquetConfig(writeSupport,
+        CompressionCodecName.fromConf(compressionCodecName),
+        config.getIntOrDefault(HoodieStorageConfig.PARQUET_BLOCK_SIZE),
+        config.getIntOrDefault(HoodieStorageConfig.PARQUET_PAGE_SIZE),
+        config.getLongOrDefault(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE),
+        conf,
+        config.getDoubleOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION),
+        config.getBooleanOrDefault(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED));
+    parquetConfig.getHadoopConf().addResource(writeSupport.getHadoopConf());
+
+    return new HoodieSparkParquetWriter(path, parquetConfig, instantTime, taskContextSupplier, populateMetaFields);
+  }
+
+  protected HoodieFileWriter newParquetFileWriter(
+      FSDataOutputStream outputStream, Configuration conf, HoodieConfig config, Schema schema) throws IOException {
+    boolean enableBloomFilter = false;
+    Option<BloomFilter> filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty();
+    HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport(conf,
+        HoodieInternalRowUtils.getCachedSchema(schema), filter,
+        HoodieStorageConfig.newBuilder().fromProperties(config.getProps()).build());
+    String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME);
+    // Support PARQUET_COMPRESSION_CODEC_NAME is ""
+    if (compressionCodecName.isEmpty()) {
+      compressionCodecName = null;
+    }
+    HoodieRowParquetConfig parquetConfig = new HoodieRowParquetConfig(writeSupport,
+        CompressionCodecName.fromConf(compressionCodecName),
+        config.getInt(HoodieStorageConfig.PARQUET_BLOCK_SIZE),
+        config.getInt(HoodieStorageConfig.PARQUET_PAGE_SIZE),
+        config.getLong(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE),
+        writeSupport.getHadoopConf(), config.getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION),
+        config.getBooleanOrDefault(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED));
+    parquetConfig.getHadoopConf().addResource(writeSupport.getHadoopConf());
+    return new HoodieSparkParquetStreamWriter(outputStream, parquetConfig);
+  }
+
+  @Override
+  protected HoodieFileWriter newHFileFileWriter(String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier) throws IOException {
+    throw new HoodieIOException("Not support write to HFile");
+  }
+
+  @Override
+  protected HoodieFileWriter newOrcFileWriter(String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier) throws IOException {
+    throw new HoodieIOException("Not support write to Orc file");
+  }
+}
diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java
similarity index 55%
copy from hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java
copy to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java
index 83ffaf589f..7078393d28 100644
--- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java
@@ -19,33 +19,38 @@
 package org.apache.hudi.io.storage;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.IndexedRecord;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HoodieInternalRowUtils;
 import org.apache.hudi.common.bloom.BloomFilter;
 import org.apache.hudi.common.model.HoodieFileFormat;
 import org.apache.hudi.common.util.BaseFileUtils;
 import org.apache.hudi.common.util.ClosableIterator;
 import org.apache.hudi.common.util.ParquetReaderIterator;
-import org.apache.parquet.avro.AvroParquetReader;
-import org.apache.parquet.avro.AvroReadSupport;
 import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.api.ReadSupport;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.io.InputFile;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
 
-public class HoodieAvroParquetReader implements HoodieAvroFileReader {
+public class HoodieSparkParquetReader implements HoodieSparkFileReader {
 
   private final Path path;
   private final Configuration conf;
   private final BaseFileUtils parquetUtils;
   private List<ParquetReaderIterator> readerIterators = new ArrayList<>();
 
-  public HoodieAvroParquetReader(Configuration configuration, Path path) {
-    this.conf = configuration;
+  public HoodieSparkParquetReader(Configuration conf, Path path) {
     this.path = path;
+    this.conf = conf;
     this.parquetUtils = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET);
   }
 
@@ -65,10 +70,29 @@ public class HoodieAvroParquetReader implements HoodieAvroFileReader {
   }
 
   @Override
-  public ClosableIterator<IndexedRecord> getRecordIterator(Schema schema) throws IOException {
-    AvroReadSupport.setAvroReadSchema(conf, schema);
-    ParquetReader<IndexedRecord> reader = AvroParquetReader.<IndexedRecord>builder(path).withConf(conf).build();
-    ParquetReaderIterator<IndexedRecord> parquetReaderIterator = new ParquetReaderIterator<>(reader);
+  public ClosableIterator<InternalRow> getInternalRowIterator(Schema schema) throws IOException {
+    return getInternalRowIterator(schema, null);
+  }
+
+  @Override
+  public ClosableIterator<InternalRow> getInternalRowIterator(Schema readerSchema, Schema requestedSchema) throws IOException {
+    if (requestedSchema == null) {
+      requestedSchema = readerSchema;
+    }
+    StructType readerStructType = HoodieInternalRowUtils.getCachedSchema(readerSchema);
+    StructType requestedStructType = HoodieInternalRowUtils.getCachedSchema(requestedSchema);
+    conf.set(ParquetReadSupport.PARQUET_READ_SCHEMA, readerStructType.json());
+    conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA(), requestedStructType.json());
+    conf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING().key(), (Boolean) SQLConf.get().getConf(SQLConf.PARQUET_BINARY_AS_STRING()));
+    conf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), (Boolean) SQLConf.get().getConf(SQLConf.PARQUET_INT96_AS_TIMESTAMP()));
+    InputFile inputFile = HadoopInputFile.fromPath(path, conf);
+    ParquetReader reader = new ParquetReader.Builder<InternalRow>(inputFile) {
+      @Override
+      protected ReadSupport getReadSupport() {
+        return new ParquetReadSupport();
+      }
+    }.withConf(conf).build();
+    ParquetReaderIterator<InternalRow> parquetReaderIterator = new ParquetReaderIterator<>(reader, InternalRow::copy);
     readerIterators.add(parquetReaderIterator);
     return parquetReaderIterator;
   }
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetStreamWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetStreamWriter.java
new file mode 100644
index 0000000000..5c6f85af29
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetStreamWriter.java
@@ -0,0 +1,100 @@
+/*
+ * 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 java.io.IOException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.io.storage.row.HoodieRowParquetConfig;
+import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport;
+import org.apache.hudi.parquet.io.OutputStreamBackedOutputFile;
+import org.apache.parquet.hadoop.ParquetFileWriter;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.api.WriteSupport;
+import org.apache.parquet.io.OutputFile;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.unsafe.types.UTF8String;
+
+public class HoodieSparkParquetStreamWriter implements HoodieSparkFileWriter, AutoCloseable {
+  private final ParquetWriter<InternalRow> writer;
+  private final HoodieRowParquetWriteSupport writeSupport;
+
+  public HoodieSparkParquetStreamWriter(FSDataOutputStream outputStream,
+      HoodieRowParquetConfig parquetConfig) throws IOException {
+    this.writeSupport = parquetConfig.getWriteSupport();
+    this.writer = new Builder<>(new OutputStreamBackedOutputFile(outputStream), writeSupport)
+        .withWriteMode(ParquetFileWriter.Mode.CREATE)
+        .withCompressionCodec(parquetConfig.getCompressionCodecName())
+        .withRowGroupSize(parquetConfig.getBlockSize())
+        .withPageSize(parquetConfig.getPageSize())
+        .withDictionaryPageSize(parquetConfig.getPageSize())
+        .withDictionaryEncoding(parquetConfig.dictionaryEnabled())
+        .withWriterVersion(ParquetWriter.DEFAULT_WRITER_VERSION)
+        .withConf(parquetConfig.getHadoopConf())
+        .build();
+  }
+
+  @Override
+  public boolean canWrite() {
+    return true;
+  }
+
+  @Override
+  public void writeRow(String key, InternalRow record) throws IOException {
+    writer.write(record);
+    writeSupport.add(UTF8String.fromString(key));
+  }
+
+  @Override
+  public void writeRowWithMetadata(HoodieKey key, InternalRow record) throws IOException {
+    // TODO support populating the metadata
+    this.writeRow(key.getRecordKey(), record);
+  }
+
+  @Override
+  public void close() throws IOException {
+    writer.close();
+  }
+
+  private static class Builder<T> extends ParquetWriter.Builder<T, Builder<T>> {
+    private final WriteSupport<T> writeSupport;
+
+    private Builder(Path file, WriteSupport<T> writeSupport) {
+      super(file);
+      this.writeSupport = writeSupport;
+    }
+
+    private Builder(OutputFile file, WriteSupport<T> writeSupport) {
+      super(file);
+      this.writeSupport = writeSupport;
+    }
+
+    @Override
+    protected Builder<T> self() {
+      return this;
+    }
+
+    @Override
+    protected WriteSupport<T> getWriteSupport(Configuration conf) {
+      return writeSupport;
+    }
+  }
+}
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetWriter.java
similarity index 53%
copy from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetWriter.java
copy to hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetWriter.java
index 36033d26b0..3b4a86502d 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetWriter.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetWriter.java
@@ -18,64 +18,55 @@
 
 package org.apache.hudi.io.storage;
 
-import org.apache.avro.generic.IndexedRecord;
 import org.apache.hadoop.fs.Path;
-import org.apache.hudi.avro.HoodieAvroWriteSupport;
 import org.apache.hudi.common.engine.TaskContextSupplier;
 import org.apache.hudi.common.model.HoodieKey;
-
-import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.hudi.io.storage.row.HoodieRowParquetConfig;
+import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.unsafe.types.UTF8String;
 
 import java.io.IOException;
 
-/**
- * HoodieParquetWriter extends the ParquetWriter to help limit the size of underlying file. Provides a way to check if
- * the current file can take more records with the <code>canWrite()</code>
- *
- * ATTENTION: HoodieParquetWriter is not thread safe and developer should take care of the order of write and close
- */
-@NotThreadSafe
-public class HoodieAvroParquetWriter
-    extends HoodieBaseParquetWriter<IndexedRecord>
-    implements HoodieAvroFileWriter {
+public class HoodieSparkParquetWriter extends HoodieBaseParquetWriter<InternalRow> implements HoodieSparkFileWriter {
 
+  // TODO: better code reuse
   private final String fileName;
   private final String instantTime;
   private final TaskContextSupplier taskContextSupplier;
   private final boolean populateMetaFields;
-  private final HoodieAvroWriteSupport writeSupport;
+  private final HoodieRowParquetWriteSupport writeSupport;
 
-  @SuppressWarnings({"unchecked", "rawtypes"})
-  public HoodieAvroParquetWriter(Path file,
-                                 HoodieParquetConfig<HoodieAvroWriteSupport> parquetConfig,
-                                 String instantTime,
-                                 TaskContextSupplier taskContextSupplier,
-                                 boolean populateMetaFields) throws IOException {
-    super(file, (HoodieParquetConfig) parquetConfig);
-    this.fileName = file.getName();
+  public HoodieSparkParquetWriter(Path file,
+                                  HoodieRowParquetConfig parquetConfig,
+                                  String instantTime,
+                                  TaskContextSupplier taskContextSupplier,
+                                  boolean populateMetaFields) throws IOException {
+    super(file, parquetConfig);
     this.writeSupport = parquetConfig.getWriteSupport();
+    this.fileName = file.getName();
     this.instantTime = instantTime;
     this.taskContextSupplier = taskContextSupplier;
     this.populateMetaFields = populateMetaFields;
   }
 
   @Override
-  public void writeAvroWithMetadata(HoodieKey key, IndexedRecord avroRecord) throws IOException {
+  public void writeRowWithMetadata(HoodieKey key, InternalRow row) throws IOException {
     if (populateMetaFields) {
-      prepRecordWithMetadata(key, avroRecord, instantTime,
+      prepRecordWithMetadata(key, row, instantTime,
           taskContextSupplier.getPartitionIdSupplier().get(), getWrittenRecordCount(), fileName);
-      super.write(avroRecord);
-      writeSupport.add(key.getRecordKey());
+      super.write(row);
+      writeSupport.add(UTF8String.fromString(key.getRecordKey()));
     } else {
-      super.write(avroRecord);
+      super.write(row);
     }
   }
 
   @Override
-  public void writeAvro(String key, IndexedRecord object) throws IOException {
-    super.write(object);
+  public void writeRow(String recordKey, InternalRow row) throws IOException {
+    super.write(row);
     if (populateMetaFields) {
-      writeSupport.add(key);
+      writeSupport.add(UTF8String.fromString(recordKey));
     }
   }
 
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java
index e68873f92e..8a61c7c44d 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieInternalRowFileWriterFactory.java
@@ -67,7 +67,7 @@ public class HoodieInternalRowFileWriterFactory {
   )
       throws IOException {
     HoodieRowParquetWriteSupport writeSupport =
-            new HoodieRowParquetWriteSupport(table.getHadoopConf(), structType, bloomFilterOpt, writeConfig);
+            new HoodieRowParquetWriteSupport(table.getHadoopConf(), structType, bloomFilterOpt, writeConfig.getStorageConfig());
 
     return new HoodieInternalRowParquetWriter(
         path,
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetConfig.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetConfig.java
new file mode 100644
index 0000000000..8a4a60b52f
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetConfig.java
@@ -0,0 +1,42 @@
+/*
+ * 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.row;
+
+import org.apache.hudi.io.storage.HoodieParquetConfig;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+/**
+ * ParquetConfig for datasource implementation with {@link org.apache.hudi.client.model.HoodieInternalRow}.
+ */
+public class HoodieRowParquetConfig extends HoodieParquetConfig<HoodieRowParquetWriteSupport> {
+
+  public HoodieRowParquetConfig(HoodieRowParquetWriteSupport writeSupport, CompressionCodecName compressionCodecName,
+                                int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf,
+                                double compressionRatio) {
+    super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio);
+  }
+
+  public HoodieRowParquetConfig(HoodieRowParquetWriteSupport writeSupport, CompressionCodecName compressionCodecName,
+      int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf,
+      double compressionRatio, boolean enableDictionary) {
+    super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio, enableDictionary);
+  }
+}
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java
index 28964ecc3f..b437501f42 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java
@@ -21,8 +21,8 @@ package org.apache.hudi.io.storage.row;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hudi.common.bloom.BloomFilter;
 import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.parquet.hadoop.api.WriteSupport;
 import org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport;
 import org.apache.spark.sql.types.StructType;
@@ -34,6 +34,7 @@ import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILT
 import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_BLOOM_FILTER_TYPE_CODE;
 import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER;
 import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER;
+import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_FIELD_ID_WRITE_ENABLED;
 
 /**
  * Hoodie Write Support for directly writing Row to Parquet.
@@ -46,11 +47,11 @@ public class HoodieRowParquetWriteSupport extends ParquetWriteSupport {
   private UTF8String minRecordKey;
   private UTF8String maxRecordKey;
 
-  public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, Option<BloomFilter> bloomFilterOpt, HoodieWriteConfig writeConfig) {
+  public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, Option<BloomFilter> bloomFilterOpt, HoodieStorageConfig config) {
     Configuration hadoopConf = new Configuration(conf);
-    hadoopConf.set("spark.sql.parquet.writeLegacyFormat", writeConfig.parquetWriteLegacyFormatEnabled());
-    hadoopConf.set("spark.sql.parquet.outputTimestampType", writeConfig.parquetOutputTimestampType());
-    hadoopConf.set("spark.sql.parquet.fieldId.write.enabled", writeConfig.parquetFieldIdWriteEnabled());
+    hadoopConf.set("spark.sql.parquet.writeLegacyFormat", config.getString(HoodieStorageConfig.PARQUET_WRITE_LEGACY_FORMAT_ENABLED));
+    hadoopConf.set("spark.sql.parquet.outputTimestampType", config.getString(HoodieStorageConfig.PARQUET_OUTPUT_TIMESTAMP_TYPE));
+    hadoopConf.set("spark.sql.parquet.fieldId.write.enabled", config.getString(PARQUET_FIELD_ID_WRITE_ENABLED));
     this.hadoopConf = hadoopConf;
     setSchema(structType, hadoopConf);
     this.bloomFilter = bloomFilterOpt.orElse(null);
@@ -77,6 +78,9 @@ public class HoodieRowParquetWriteSupport extends ParquetWriteSupport {
   }
 
   public void add(UTF8String recordKey) {
+    if (this.bloomFilter == null) {
+      return;
+    }
     this.bloomFilter.add(recordKey.getBytes());
 
     if (minRecordKey == null || minRecordKey.compareTo(recordKey) < 0) {
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java
index ad71b17ce7..60f3d4899d 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/BuiltinKeyGenerator.java
@@ -278,7 +278,7 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp
     // Avoid creating [[StringBuilder]] in case there's just one partition-path part,
     // and Hive-style of partitioning is not required
     if (!hiveStylePartitioning && partitionPathParts.length == 1) {
-      return emptyHandler.apply(converter.apply(partitionPathParts[0]));
+      return encoder.apply(emptyHandler.apply(converter.apply(partitionPathParts[0])));
     }
 
     StringBuilder<S> sb = builderFactory.get();
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java
index 96ac794dcb..d7a4a2b52a 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/OrcBootstrapMetadataHandler.java
@@ -24,6 +24,7 @@ import org.apache.hudi.client.bootstrap.BootstrapRecordPayload;
 import org.apache.hudi.common.model.HoodieAvroRecord;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.util.AvroOrcUtils;
 import org.apache.hudi.common.util.OrcReaderIterator;
 import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
@@ -63,6 +64,10 @@ class OrcBootstrapMetadataHandler extends BaseBootstrapMetadataHandler {
   @Override
   void executeBootstrap(HoodieBootstrapHandle<?, ?, ?, ?> bootstrapHandle, Path sourceFilePath, KeyGeneratorInterface keyGenerator,
                         String partitionPath, Schema avroSchema) throws Exception {
+    // TODO support spark orc reader
+    if (config.getRecordMerger().getRecordType() == HoodieRecordType.SPARK) {
+      throw new UnsupportedOperationException();
+    }
     BoundedInMemoryExecutor<GenericRecord, HoodieRecord, Void> wrapper = null;
     Reader orcReader = OrcFile.createReader(sourceFilePath, OrcFile.readerOptions(table.getHadoopConf()));
     TypeDescription orcSchema = orcReader.getSchema();
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java
index 5f45629ba8..b47a1c765d 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/ParquetBootstrapMetadataHandler.java
@@ -20,34 +20,32 @@ package org.apache.hudi.table.action.bootstrap;
 
 import org.apache.hudi.avro.HoodieAvroUtils;
 import org.apache.hudi.avro.model.HoodieFileStatus;
-import org.apache.hudi.client.bootstrap.BootstrapRecordPayload;
-import org.apache.hudi.common.model.HoodieAvroRecord;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.util.ParquetReaderIterator;
+import org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField;
+import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.io.HoodieBootstrapHandle;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 import org.apache.hudi.keygen.KeyGeneratorInterface;
 import org.apache.hudi.table.HoodieTable;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericData;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.generic.IndexedRecord;
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
-import org.apache.parquet.avro.AvroParquetReader;
 import org.apache.parquet.avro.AvroSchemaConverter;
 import org.apache.parquet.format.converter.ParquetMetadataConverter;
 import org.apache.parquet.hadoop.ParquetFileReader;
-import org.apache.parquet.hadoop.ParquetReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.schema.MessageType;
 
 import java.io.IOException;
+import java.util.Collections;
+import java.util.Properties;
 
 class ParquetBootstrapMetadataHandler extends BaseBootstrapMetadataHandler {
   private static final Logger LOG = LogManager.getLogger(ParquetBootstrapMetadataHandler.class);
@@ -67,18 +65,22 @@ class ParquetBootstrapMetadataHandler extends BaseBootstrapMetadataHandler {
   @Override
   void executeBootstrap(HoodieBootstrapHandle<?, ?, ?, ?> bootstrapHandle,
                         Path sourceFilePath, KeyGeneratorInterface keyGenerator, String partitionPath, Schema avroSchema) throws Exception {
-    BoundedInMemoryExecutor<GenericRecord, HoodieRecord, Void> wrapper = null;
-    ParquetReader<IndexedRecord> reader =
-        AvroParquetReader.<IndexedRecord>builder(sourceFilePath).withConf(table.getHadoopConf()).build();
+    BoundedInMemoryExecutor<HoodieRecord, HoodieRecord, Void> wrapper = null;
+    HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(table.getConfig().getRecordMerger().getRecordType())
+            .getFileReader(table.getHadoopConf(), sourceFilePath);
     try {
-      wrapper = new BoundedInMemoryExecutor<GenericRecord, HoodieRecord, Void>(config.getWriteBufferLimitBytes(),
-          new ParquetReaderIterator(reader), new BootstrapRecordConsumer(bootstrapHandle), inp -> {
-        String recKey = keyGenerator.getKey(inp).getRecordKey();
-        GenericRecord gr = new GenericData.Record(HoodieAvroUtils.RECORD_KEY_SCHEMA);
-        gr.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recKey);
-        BootstrapRecordPayload payload = new BootstrapRecordPayload(gr);
-        HoodieRecord rec = new HoodieAvroRecord(new HoodieKey(recKey, partitionPath), payload);
-        return rec;
+      wrapper = new BoundedInMemoryExecutor<HoodieRecord, HoodieRecord, Void>(config.getWriteBufferLimitBytes(),
+          reader.getRecordIterator(), new BootstrapRecordConsumer(bootstrapHandle), inp -> {
+        try {
+          String recKey = inp.getRecordKey(Option.of(keyGenerator));
+          HoodieRecord hoodieRecord = inp.rewriteRecord(reader.getSchema(), config.getProps(), HoodieAvroUtils.RECORD_KEY_SCHEMA);
+          return hoodieRecord
+              .updateValues(HoodieAvroUtils.RECORD_KEY_SCHEMA, new Properties(), Collections.singletonMap(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.getFieldName(), recKey))
+              .newInstance(new HoodieKey(recKey, partitionPath));
+        } catch (IOException e) {
+          LOG.error("Unable to overrideMetadataFieldValue", e);
+          return null;
+        }
       }, table.getPreExecuteRunnable());
       wrapper.execute();
     } catch (Exception e) {
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java
index 56e8f81405..e3524f4d70 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java
@@ -279,7 +279,7 @@ public class SparkBootstrapCommitActionExecutor<T>
             properties, context);
     JavaRDD<HoodieRecord> inputRecordsRDD =
         (JavaRDD<HoodieRecord>) inputProvider.generateInputRecords("bootstrap_source", config.getBootstrapSourceBasePath(),
-            partitionFilesList);
+            partitionFilesList, config);
     // Start Full Bootstrap
     String bootstrapInstantTime = HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS;
     final HoodieInstant requested = new HoodieInstant(
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java
new file mode 100644
index 0000000000..4779e8e05f
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java
@@ -0,0 +1,111 @@
+/*
+ * 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.util;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+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.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+
+import org.apache.spark.sql.HoodieCatalystExpressionUtils$;
+import org.apache.spark.sql.HoodieUnsafeRowUtils;
+import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+
+public class HoodieSparkRecordUtils {
+
+  /**
+   * Utility method to convert InternalRow to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, boolean withOperationField) {
+    return convertToHoodieSparkRecord(structType, data,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, Option.empty());
+  }
+
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, boolean withOperationField,
+      Option<String> partitionName) {
+    return convertToHoodieSparkRecord(structType, data,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, partitionName);
+  }
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, Pair<String, String> recordKeyPartitionPathFieldPair,
+      boolean withOperationField, Option<String> partitionName) {
+    final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), data).toString();
+    final String partitionPath = (partitionName.isPresent() ? partitionName.get() :
+        getValue(structType, recordKeyPartitionPathFieldPair.getRight(), data).toString());
+
+    HoodieOperation operation = withOperationField
+        ? HoodieOperation.fromName(getNullableValAsString(structType, data, HoodieRecord.OPERATION_METADATA_FIELD)) : null;
+    return new HoodieSparkRecord(new HoodieKey(recKey, partitionPath), data, structType, operation);
+  }
+
+  private static Object getValue(StructType structType, String fieldName, InternalRow row) {
+    NestedFieldPath posList = HoodieInternalRowUtils.getCachedPosList(structType, fieldName);
+    return HoodieUnsafeRowUtils.getNestedInternalRowValue(row, posList);
+  }
+
+  /**
+   * Returns the string value of the given record {@code rec} and field {@code fieldName}. The field and value both could be missing.
+   *
+   * @param row       The record
+   * @param fieldName The field name
+   * @return the string form of the field or empty if the schema does not contain the field name or the value is null
+   */
+  private static Option<String> getNullableValAsString(StructType structType, InternalRow row, String fieldName) {
+    String fieldVal = !HoodieCatalystExpressionUtils$.MODULE$.existField(structType, fieldName)
+        ? null : StringUtils.objToString(getValue(structType, fieldName, row));
+    return Option.ofNullable(fieldVal);
+  }
+
+  /**
+   * Gets record column values into one object.
+   *
+   * @param row  InternalRow.
+   * @param columns Names of the columns to get values.
+   * @param structType  {@link StructType} instance.
+   * @return Column value if a single column, or concatenated String values by comma.
+   */
+  public static Object getRecordColumnValues(InternalRow row,
+      String[] columns,
+      StructType structType, boolean consistentLogicalTimestampEnabled) {
+    if (columns.length == 1) {
+      NestedFieldPath posList = HoodieInternalRowUtils.getCachedPosList(structType, columns[0]);
+      return HoodieUnsafeRowUtils.getNestedInternalRowValue(row, posList);
+    } else {
+      // TODO this is inefficient, instead we can simply return array of Comparable
+      StringBuilder sb = new StringBuilder();
+      for (String col : columns) {
+        // TODO support consistentLogicalTimestampEnabled
+        NestedFieldPath posList = HoodieInternalRowUtils.getCachedPosList(structType, columns[0]);
+        return HoodieUnsafeRowUtils.getNestedInternalRowValue(row, posList);
+      }
+      return sb.toString();
+    }
+  }
+}
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieInternalRowUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieInternalRowUtils.scala
similarity index 51%
rename from hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieInternalRowUtils.scala
rename to hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieInternalRowUtils.scala
index 54e4010fe1..cd259974d9 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieInternalRowUtils.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieInternalRowUtils.scala
@@ -16,60 +16,74 @@
  * limitations under the License.
  */
 
-package org.apache.spark.sql.hudi
+package org.apache.hudi
 
 import java.nio.charset.StandardCharsets
-import java.util
+import java.util.HashMap
 import java.util.concurrent.ConcurrentHashMap
-import org.apache.avro.Schema
-import org.apache.hudi.AvroConversionUtils
-import org.apache.hudi.avro.HoodieAvroUtils
-import org.apache.hudi.avro.HoodieAvroUtils.{createFullName, fromJavaDate, toJavaDate}
+import org.apache.avro.{Schema, SchemaNormalization}
+import org.apache.hbase.thirdparty.com.google.common.base.Supplier
+import org.apache.hudi.avro.HoodieAvroUtils.{createFullName, toJavaDate}
 import org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField
-import org.apache.hudi.common.util.ValidationUtils
 import org.apache.hudi.exception.HoodieException
 import org.apache.spark.sql.HoodieCatalystExpressionUtils
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, MutableProjection, Projection}
+import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, Projection}
+import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath
+import org.apache.spark.sql.HoodieUnsafeRowUtils
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection
 import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData}
+import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
 import org.apache.spark.sql.types._
 import scala.collection.mutable
 
-/**
- * Helper class to do common stuff across Spark InternalRow.
- * Provides common methods similar to {@link HoodieAvroUtils}.
- */
 object HoodieInternalRowUtils {
 
-  val projectionMap = new ConcurrentHashMap[(StructType, StructType), MutableProjection]
+  // Projection are all thread local. Projection is not thread-safe
+  val unsafeProjectionThreadLocal: ThreadLocal[HashMap[(StructType, StructType), UnsafeProjection]] =
+    ThreadLocal.withInitial(new Supplier[HashMap[(StructType, StructType), UnsafeProjection]] {
+      override def get(): HashMap[(StructType, StructType), UnsafeProjection] = new HashMap[(StructType, StructType), UnsafeProjection]
+    })
+  val unsafeConvertThreadLocal: ThreadLocal[HashMap[StructType, UnsafeProjection]] =
+    ThreadLocal.withInitial(new Supplier[HashMap[StructType, UnsafeProjection]] {
+      override def get(): HashMap[StructType, UnsafeProjection] = new HashMap[StructType, UnsafeProjection]
+    })
   val schemaMap = new ConcurrentHashMap[Schema, StructType]
-  val SchemaPosMap = new ConcurrentHashMap[StructType, Map[String, (StructField, Int)]]
+  val schemaFingerPrintMap = new ConcurrentHashMap[Long, StructType]
+  val fingerPrintSchemaMap = new ConcurrentHashMap[StructType, Long]
+  val orderPosListMap = new ConcurrentHashMap[(StructType, String), NestedFieldPath]
 
+  /**
+   * @see org.apache.hudi.avro.HoodieAvroUtils#stitchRecords(org.apache.avro.generic.GenericRecord, org.apache.avro.generic.GenericRecord, org.apache.avro.Schema)
+   */
   def stitchRecords(left: InternalRow, leftSchema: StructType, right: InternalRow, rightSchema: StructType, stitchedSchema: StructType): InternalRow = {
     val mergeSchema = StructType(leftSchema.fields ++ rightSchema.fields)
     val row = new JoinedRow(left, right)
-    val projection = getCachedProjection(mergeSchema, stitchedSchema)
+    val projection = getCachedUnsafeProjection(mergeSchema, stitchedSchema)
     projection(row)
   }
 
+  /**
+   * @see org.apache.hudi.avro.HoodieAvroUtils#rewriteRecord(org.apache.avro.generic.GenericRecord, org.apache.avro.Schema)
+   */
   def rewriteRecord(oldRecord: InternalRow, oldSchema: StructType, newSchema: StructType): InternalRow = {
     val newRow = new GenericInternalRow(Array.fill(newSchema.fields.length)(null).asInstanceOf[Array[Any]])
 
-    val oldFieldMap = getCachedSchemaPosMap(oldSchema)
     for ((field, pos) <- newSchema.fields.zipWithIndex) {
       var oldValue: AnyRef = null
-      if (oldFieldMap.contains(field.name)) {
-        val (oldField, oldPos) = oldFieldMap(field.name)
+      if (HoodieCatalystExpressionUtils.existField(oldSchema, field.name)) {
+        val oldField = oldSchema(field.name)
+        val oldPos = oldSchema.fieldIndex(field.name)
         oldValue = oldRecord.get(oldPos, oldField.dataType)
       }
       if (oldValue != null) {
         field.dataType match {
           case structType: StructType =>
-            val oldField = oldFieldMap(field.name)._1.asInstanceOf[StructType]
-            rewriteRecord(oldValue.asInstanceOf[InternalRow], oldField, structType)
+            val oldType = oldSchema(field.name).dataType.asInstanceOf[StructType]
+            val newValue = rewriteRecord(oldValue.asInstanceOf[InternalRow], oldType, structType)
+            newRow.update(pos, newValue)
           case decimalType: DecimalType =>
-            val oldField = oldFieldMap(field.name)._1.asInstanceOf[DecimalType]
-            if (decimalType.scale != oldField.scale || decimalType.precision != oldField.precision) {
+            val oldFieldSchema = oldSchema(field.name).dataType.asInstanceOf[DecimalType]
+            if (decimalType.scale != oldFieldSchema.scale || decimalType.precision != oldFieldSchema.precision) {
               newRow.update(pos, Decimal.fromDecimal(oldValue.asInstanceOf[Decimal].toBigDecimal.setScale(newSchema.asInstanceOf[DecimalType].scale))
               )
             } else {
@@ -86,34 +100,44 @@ object HoodieInternalRowUtils {
     newRow
   }
 
-  def rewriteRecordWithNewSchema(oldRecord: InternalRow, oldSchema: StructType, newSchema: StructType, renameCols: util.Map[String, String]): InternalRow = {
-    rewriteRecordWithNewSchema(oldRecord, oldSchema, newSchema, renameCols, new util.LinkedList[String]).asInstanceOf[InternalRow]
+  /**
+   * @see org.apache.hudi.avro.HoodieAvroUtils#rewriteRecordWithNewSchema(org.apache.avro.generic.IndexedRecord, org.apache.avro.Schema, java.util.Map)
+   */
+  def rewriteRecordWithNewSchema(oldRecord: InternalRow, oldSchema: StructType, newSchema: StructType, renameCols: java.util.Map[String, String]): InternalRow = {
+    rewriteRecordWithNewSchema(oldRecord, oldSchema, newSchema, renameCols, new java.util.LinkedList[String]).asInstanceOf[InternalRow]
   }
 
-  private def rewriteRecordWithNewSchema(oldRecord: Any, oldSchema: DataType, newSchema: DataType, renameCols: util.Map[String, String], fieldNames: util.Deque[String]): Any = {
+  /**
+   * @see org.apache.hudi.avro.HoodieAvroUtils#rewriteRecordWithNewSchema(java.lang.Object, org.apache.avro.Schema, org.apache.avro.Schema, java.util.Map, java.util.Deque)
+   */
+  private def rewriteRecordWithNewSchema(oldRecord: Any, oldSchema: DataType, newSchema: DataType, renameCols: java.util.Map[String, String], fieldNames: java.util.Deque[String]): Any = {
     if (oldRecord == null) {
       null
     } else {
       newSchema match {
         case targetSchema: StructType =>
-          ValidationUtils.checkArgument(oldRecord.isInstanceOf[InternalRow], "cannot rewrite record with different type")
+          if (!oldRecord.isInstanceOf[InternalRow]) {
+            throw new IllegalArgumentException("cannot rewrite record with different type")
+          }
           val oldRow = oldRecord.asInstanceOf[InternalRow]
           val helper = mutable.Map[Integer, Any]()
 
-          val oldSchemaPos = getCachedSchemaPosMap(oldSchema.asInstanceOf[StructType])
+          val oldStrucType = oldSchema.asInstanceOf[StructType]
           targetSchema.fields.zipWithIndex.foreach { case (field, i) =>
             fieldNames.push(field.name)
-            if (oldSchemaPos.contains(field.name)) {
-              val (oldField, oldPos) = oldSchemaPos(field.name)
+            if (HoodieCatalystExpressionUtils.existField(oldStrucType, field.name)) {
+              val oldField = oldStrucType(field.name)
+              val oldPos = oldStrucType.fieldIndex(field.name)
               helper(i) = rewriteRecordWithNewSchema(oldRow.get(oldPos, oldField.dataType), oldField.dataType, field.dataType, renameCols, fieldNames)
             } else {
               val fieldFullName = createFullName(fieldNames)
               val colNamePartsFromOldSchema = renameCols.getOrDefault(fieldFullName, "").split("\\.")
               val lastColNameFromOldSchema = colNamePartsFromOldSchema(colNamePartsFromOldSchema.length - 1)
               // deal with rename
-              if (!oldSchemaPos.contains(field.name) && oldSchemaPos.contains(lastColNameFromOldSchema)) {
+              if (!HoodieCatalystExpressionUtils.existField(oldStrucType, field.name) && HoodieCatalystExpressionUtils.existField(oldStrucType, lastColNameFromOldSchema)) {
                 // find rename
-                val (oldField, oldPos) = oldSchemaPos(lastColNameFromOldSchema)
+                val oldField = oldStrucType(lastColNameFromOldSchema)
+                val oldPos = oldStrucType.fieldIndex(lastColNameFromOldSchema)
                 helper(i) = rewriteRecordWithNewSchema(oldRow.get(oldPos, oldField.dataType), oldField.dataType, field.dataType, renameCols, fieldNames)
               }
             }
@@ -131,7 +155,9 @@ object HoodieInternalRowUtils {
 
           newRow
         case targetSchema: ArrayType =>
-          ValidationUtils.checkArgument(oldRecord.isInstanceOf[ArrayData], "cannot rewrite record with different type")
+          if (!oldRecord.isInstanceOf[ArrayData]) {
+            throw new IllegalArgumentException("cannot rewrite record with different type")
+          }
           val oldElementType = oldSchema.asInstanceOf[ArrayType].elementType
           val oldArray = oldRecord.asInstanceOf[ArrayData]
           val newElementType = targetSchema.elementType
@@ -142,7 +168,9 @@ object HoodieInternalRowUtils {
 
           newArray
         case targetSchema: MapType =>
-          ValidationUtils.checkArgument(oldRecord.isInstanceOf[MapData], "cannot rewrite record with different type")
+          if (!oldRecord.isInstanceOf[MapData]) {
+            throw new IllegalArgumentException("cannot rewrite record with different type")
+          }
           val oldValueType = oldSchema.asInstanceOf[MapType].valueType
           val oldKeyType = oldSchema.asInstanceOf[MapType].keyType
           val oldMap = oldRecord.asInstanceOf[MapData]
@@ -161,55 +189,86 @@ object HoodieInternalRowUtils {
     }
   }
 
+  /**
+   * @see org.apache.hudi.avro.HoodieAvroUtils#rewriteRecordWithMetadata(org.apache.avro.generic.GenericRecord, org.apache.avro.Schema, java.lang.String)
+   */
   def rewriteRecordWithMetadata(record: InternalRow, oldSchema: StructType, newSchema: StructType, fileName: String): InternalRow = {
     val newRecord = rewriteRecord(record, oldSchema, newSchema)
-    newRecord.update(HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal, fileName)
+    newRecord.update(HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal, CatalystTypeConverters.convertToCatalyst(fileName))
 
     newRecord
   }
 
+  /**
+   * @see org.apache.hudi.avro.HoodieAvroUtils#rewriteEvolutionRecordWithMetadata(org.apache.avro.generic.GenericRecord, org.apache.avro.Schema, java.lang.String)
+   */
   def rewriteEvolutionRecordWithMetadata(record: InternalRow, oldSchema: StructType, newSchema: StructType, fileName: String): InternalRow = {
-    val newRecord = rewriteRecordWithNewSchema(record, oldSchema, newSchema, new util.HashMap[String, String]())
-    newRecord.update(HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal, fileName)
+    val newRecord = rewriteRecordWithNewSchema(record, oldSchema, newSchema, new java.util.HashMap[String, String]())
+    newRecord.update(HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal, CatalystTypeConverters.convertToCatalyst(fileName))
 
     newRecord
   }
 
+  def getCachedPosList(structType: StructType, field: String): NestedFieldPath = {
+    val schemaPair = (structType, field)
+    if (!orderPosListMap.containsKey(schemaPair)) {
+      val posList = HoodieUnsafeRowUtils.composeNestedFieldPath(structType, field)
+      orderPosListMap.put(schemaPair, posList)
+    }
+    orderPosListMap.get(schemaPair)
+  }
+
+  def getCachedUnsafeConvert(structType: StructType): UnsafeProjection = {
+    val map = unsafeConvertThreadLocal.get()
+    if (!map.containsKey(structType)) {
+      val projection = UnsafeProjection.create(structType)
+      map.put(structType, projection)
+    }
+    map.get(structType)
+  }
+
+  def getCachedUnsafeProjection(from: StructType, to: StructType): Projection = {
+    val schemaPair = (from, to)
+    val map = unsafeProjectionThreadLocal.get()
+    if (!map.containsKey(schemaPair)) {
+      val projection = HoodieCatalystExpressionUtils.generateUnsafeProjection(from, to)
+      map.put(schemaPair, projection)
+    }
+    map.get(schemaPair)
+  }
+
   def getCachedSchema(schema: Schema): StructType = {
-    if (!schemaMap.contains(schema)) {
-      schemaMap.synchronized {
-        if (!schemaMap.contains(schema)) {
-          val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
-          schemaMap.put(schema, structType)
-        }
-      }
+    if (!schemaMap.containsKey(schema)) {
+      val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
+      schemaMap.put(schema, structType)
     }
     schemaMap.get(schema)
   }
 
-  private def getCachedProjection(from: StructType, to: StructType): Projection = {
-    val schemaPair = (from, to)
-    if (!projectionMap.contains(schemaPair)) {
-      projectionMap.synchronized {
-        if (!projectionMap.contains(schemaPair)) {
-          val projection = HoodieCatalystExpressionUtils.generateMutableProjection(from, to)
-          projectionMap.put(schemaPair, projection)
-        }
-      }
+  def getCachedSchemaFromFingerPrint(fingerPrint: Long): StructType = {
+    if (!schemaFingerPrintMap.containsKey(fingerPrint)) {
+      throw new IllegalArgumentException("Not exist " + fingerPrint)
     }
-    projectionMap.get(schemaPair)
+    schemaFingerPrintMap.get(fingerPrint)
   }
 
-  def getCachedSchemaPosMap(schema: StructType): Map[String, (StructField, Int)] = {
-    if (!SchemaPosMap.contains(schema)) {
-      SchemaPosMap.synchronized {
-        if (!SchemaPosMap.contains(schema)) {
-          val fieldMap = schema.fields.zipWithIndex.map { case (field, i) => (field.name, (field, i)) }.toMap
-          SchemaPosMap.put(schema, fieldMap)
-        }
-      }
+  def getCachedFingerPrintFromSchema(schema: StructType): Long = {
+    if (!fingerPrintSchemaMap.containsKey(schema)) {
+      throw new IllegalArgumentException("Not exist " + schema)
     }
-    SchemaPosMap.get(schema)
+    fingerPrintSchemaMap.get(schema)
+  }
+
+  def addCompressedSchema(schema: StructType): Unit ={
+    if (!fingerPrintSchemaMap.containsKey(schema)) {
+      val fingerPrint = SchemaNormalization.fingerprint64(schema.json.getBytes(StandardCharsets.UTF_8))
+      schemaFingerPrintMap.put(fingerPrint, schema)
+      fingerPrintSchemaMap.put(schema, fingerPrint)
+    }
+  }
+
+  def containsCompressedSchema(schema: StructType): Boolean = {
+    fingerPrintSchemaMap.containsKey(schema)
   }
 
   private def rewritePrimaryType(oldValue: Any, oldSchema: DataType, newSchema: DataType): Any = {
@@ -231,35 +290,35 @@ object HoodieInternalRowUtils {
     val value = newSchema match {
       case NullType | BooleanType =>
       case DateType if oldSchema.equals(StringType) =>
-        fromJavaDate(java.sql.Date.valueOf(oldValue.toString))
+        CatalystTypeConverters.convertToCatalyst(java.sql.Date.valueOf(oldValue.toString))
       case LongType =>
         oldSchema match {
-          case IntegerType => oldValue.asInstanceOf[Int].longValue()
+          case IntegerType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[Int].longValue())
           case _ =>
         }
       case FloatType =>
         oldSchema match {
-          case IntegerType => oldValue.asInstanceOf[Int].floatValue()
-          case LongType => oldValue.asInstanceOf[Long].floatValue()
+          case IntegerType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[Int].floatValue())
+          case LongType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[Long].floatValue())
           case _ =>
         }
       case DoubleType =>
         oldSchema match {
-          case IntegerType => oldValue.asInstanceOf[Int].doubleValue()
-          case LongType => oldValue.asInstanceOf[Long].doubleValue()
-          case FloatType => java.lang.Double.valueOf(oldValue.asInstanceOf[Float] + "")
+          case IntegerType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[Int].doubleValue())
+          case LongType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[Long].doubleValue())
+          case FloatType => CatalystTypeConverters.convertToCatalyst(java.lang.Double.valueOf(oldValue.asInstanceOf[Float] + ""))
           case _ =>
         }
       case BinaryType =>
         oldSchema match {
-          case StringType => oldValue.asInstanceOf[String].getBytes(StandardCharsets.UTF_8)
+          case StringType => CatalystTypeConverters.convertToCatalyst(oldValue.asInstanceOf[String].getBytes(StandardCharsets.UTF_8))
           case _ =>
         }
       case StringType =>
         oldSchema match {
-          case BinaryType => new String(oldValue.asInstanceOf[Array[Byte]])
-          case DateType => toJavaDate(oldValue.asInstanceOf[Integer]).toString
-          case IntegerType | LongType | FloatType | DoubleType | DecimalType() => oldValue.toString
+          case BinaryType => CatalystTypeConverters.convertToCatalyst(new String(oldValue.asInstanceOf[Array[Byte]]))
+          case DateType => CatalystTypeConverters.convertToCatalyst(toJavaDate(oldValue.asInstanceOf[Integer]).toString)
+          case IntegerType | LongType | FloatType | DoubleType | DecimalType() => CatalystTypeConverters.convertToCatalyst(oldValue.toString)
           case _ =>
         }
       case DecimalType() =>
@@ -275,7 +334,11 @@ object HoodieInternalRowUtils {
     if (value == None) {
       throw new HoodieException(String.format("cannot support rewrite value for schema type: %s since the old schema type is: %s", newSchema, oldSchema))
     } else {
-      value
+      CatalystTypeConverters.convertToCatalyst(value)
     }
   }
+
+  def removeFields(schema: StructType, fieldsToRemove: java.util.List[String]): StructType = {
+    StructType(schema.fields.filter(field => !fieldsToRemove.contains(field.name)))
+  }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieMerge.java b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkConversionUtils.scala
similarity index 52%
copy from hudi-common/src/main/java/org/apache/hudi/common/model/HoodieMerge.java
copy to hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkConversionUtils.scala
index 6becf35591..799bda2f8e 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieMerge.java
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkConversionUtils.scala
@@ -16,23 +16,21 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.common.model;
+package org.apache.hudi
 
-import org.apache.avro.Schema;
-import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.model.HoodieRecord
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.sql.{DataFrameUtil, Dataset, Row, SparkSession}
 
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Properties;
+object SparkConversionUtils {
 
-/**
- * HoodieMerge defines how to merge two records. It is a stateless component.
- * It can implement the merging logic of HoodieRecord of different engines
- * and avoid the performance consumption caused by the serialization/deserialization of Avro payload.
- */
-public interface HoodieMerge extends Serializable {
-  
-  HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer);
-
-  Option<HoodieRecord> combineAndGetUpdateValue(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;
+  def createDataFrame[T](rdd: RDD[HoodieRecord[T]], ss: SparkSession, structType: StructType): Dataset[Row] = {
+    if (rdd.isEmpty()) {
+      ss.emptyDataFrame
+    } else {
+      DataFrameUtil.createFromInternalRows(ss, structType, rdd.map(_.getData.asInstanceOf[InternalRow]))
+    }
+  }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieMerge.java b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/DataFrameUtil.scala
similarity index 52%
copy from hudi-common/src/main/java/org/apache/hudi/common/model/HoodieMerge.java
copy to hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/DataFrameUtil.scala
index 6becf35591..f05da8a0ce 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieMerge.java
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/DataFrameUtil.scala
@@ -16,23 +16,22 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.common.model;
+package org.apache.spark.sql
 
-import org.apache.avro.Schema;
-import org.apache.hudi.common.util.Option;
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.LogicalRDD
+import org.apache.spark.sql.types.StructType
 
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Properties;
+object DataFrameUtil {
 
-/**
- * HoodieMerge defines how to merge two records. It is a stateless component.
- * It can implement the merging logic of HoodieRecord of different engines
- * and avoid the performance consumption caused by the serialization/deserialization of Avro payload.
- */
-public interface HoodieMerge extends Serializable {
-  
-  HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer);
-
-  Option<HoodieRecord> combineAndGetUpdateValue(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;
-}
+  /**
+   * Creates a DataFrame out of RDD[InternalRow] that you can get
+   * using `df.queryExection.toRdd`
+   */
+  def createFromInternalRows(sparkSession: SparkSession, schema:
+  StructType, rdd: RDD[InternalRow]): DataFrame = {
+    val logicalPlan = LogicalRDD(schema.toAttributes, rdd)(sparkSession)
+    Dataset.ofRows(sparkSession, logicalPlan)
+  }
+}
\ No newline at end of file
diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystExpressionUtils.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystExpressionUtils.scala
index c047730a32..86f10d0806 100644
--- a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystExpressionUtils.scala
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/HoodieCatalystExpressionUtils.scala
@@ -226,6 +226,10 @@ object HoodieCatalystExpressionUtils {
     )
   }
 
+  def existField(structType: StructType, name: String): Boolean = {
+    structType.getFieldIndex(name).isDefined
+  }
+
   private def hasUnresolvedRefs(resolvedExpr: Expression): Boolean =
     resolvedExpr.collectFirst {
       case _: UnresolvedAttribute | _: UnresolvedFunction => true
diff --git a/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkStructTypeSerializer.scala b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkStructTypeSerializer.scala
new file mode 100644
index 0000000000..e3407d19b6
--- /dev/null
+++ b/hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkStructTypeSerializer.scala
@@ -0,0 +1,157 @@
+/*
+ * 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.spark.sql.hudi
+
+import com.esotericsoftware.kryo.Kryo
+import com.esotericsoftware.kryo.io.{Input, Output}
+import com.twitter.chill.KSerializer
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+import java.nio.ByteBuffer
+import java.nio.charset.StandardCharsets
+import org.apache.avro.SchemaNormalization
+import org.apache.commons.io.IOUtils
+import org.apache.hudi.commmon.model.HoodieSparkRecord
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.Utils
+import org.apache.spark.{SparkEnv, SparkException}
+import scala.collection.mutable
+
+/**
+ * Custom serializer used for generic spark records. If the user registers the schemas
+ * ahead of time, then the schema's fingerprint will be sent with each message instead of the actual
+ * schema, as to reduce network IO.
+ * Actions like parsing or compressing schemas are computationally expensive so the serializer
+ * caches all previously seen values as to reduce the amount of work needed to do.
+ * @param schemas a map where the keys are unique IDs for spark schemas and the values are the
+ *                string representation of the Avro schema, used to decrease the amount of data
+ *                that needs to be serialized.
+ */
+class SparkStructTypeSerializer(schemas: Map[Long, StructType]) extends KSerializer[HoodieSparkRecord] {
+  /** Used to reduce the amount of effort to compress the schema */
+  private val compressCache = new mutable.HashMap[StructType, Array[Byte]]()
+  private val decompressCache = new mutable.HashMap[ByteBuffer, StructType]()
+
+  /** Fingerprinting is very expensive so this alleviates most of the work */
+  private val fingerprintCache = new mutable.HashMap[StructType, Long]()
+  private val schemaCache = new mutable.HashMap[Long, StructType]()
+
+  // GenericAvroSerializer can't take a SparkConf in the constructor b/c then it would become
+  // a member of KryoSerializer, which would make KryoSerializer not Serializable.  We make
+  // the codec lazy here just b/c in some unit tests, we use a KryoSerializer w/out having
+  // the SparkEnv set (note those tests would fail if they tried to serialize avro data).
+  private lazy val codec = CompressionCodec.createCodec(SparkEnv.get.conf)
+
+  /**
+   * Used to compress Schemas when they are being sent over the wire.
+   * The compression results are memoized to reduce the compression time since the
+   * same schema is compressed many times over
+   */
+  def compress(schema: StructType): Array[Byte] = compressCache.getOrElseUpdate(schema, {
+    val bos = new ByteArrayOutputStream()
+    val out = codec.compressedOutputStream(bos)
+    Utils.tryWithSafeFinally {
+      out.write(schema.json.getBytes(StandardCharsets.UTF_8))
+    } {
+      out.close()
+    }
+    bos.toByteArray
+  })
+
+  /**
+   * Decompresses the schema into the actual in-memory object. Keeps an internal cache of already
+   * seen values so to limit the number of times that decompression has to be done.
+   */
+  def decompress(schemaBytes: ByteBuffer): StructType = decompressCache.getOrElseUpdate(schemaBytes, {
+    val bis = new ByteArrayInputStream(
+      schemaBytes.array(),
+      schemaBytes.arrayOffset() + schemaBytes.position(),
+      schemaBytes.remaining())
+    val in = codec.compressedInputStream(bis)
+    val bytes = Utils.tryWithSafeFinally {
+      IOUtils.toByteArray(in)
+    } {
+      in.close()
+    }
+    StructType.fromString(new String(bytes, StandardCharsets.UTF_8))
+  })
+
+  /**
+   * Serializes a record to the given output stream. It caches a lot of the internal data as
+   * to not redo work
+   */
+  def serializeDatum(datum: HoodieSparkRecord, output: Output): Unit = {
+    val schema = datum.getStructType
+    val fingerprint = fingerprintCache.getOrElseUpdate(schema, {
+      SchemaNormalization.fingerprint64(schema.json.getBytes(StandardCharsets.UTF_8))
+    })
+    schemas.get(fingerprint) match {
+      case Some(_) =>
+        output.writeBoolean(true)
+        output.writeLong(fingerprint)
+      case None =>
+        output.writeBoolean(false)
+        val compressedSchema = compress(schema)
+        output.writeInt(compressedSchema.length)
+        output.writeBytes(compressedSchema)
+    }
+
+    val record = datum.newInstance().asInstanceOf[HoodieSparkRecord]
+    record.setStructType(null)
+    val stream = new ObjectOutputStream(output)
+    stream.writeObject(record)
+    stream.close()
+  }
+
+  /**
+   * Deserializes generic records into their in-memory form. There is internal
+   * state to keep a cache of already seen schemas and datum readers.
+   */
+  def deserializeDatum(input: Input): HoodieSparkRecord = {
+    val schema = {
+      if (input.readBoolean()) {
+        val fingerprint = input.readLong()
+        schemaCache.getOrElseUpdate(fingerprint, {
+          schemas.get(fingerprint) match {
+            case Some(s) => s
+            case None =>
+              throw new SparkException(
+                "Error reading attempting to read spark data -- encountered an unknown " +
+                  s"fingerprint: $fingerprint, not sure what schema to use.  This could happen " +
+                  "if you registered additional schemas after starting your spark context.")
+          }
+        })
+      } else {
+        val length = input.readInt()
+        decompress(ByteBuffer.wrap(input.readBytes(length)))
+      }
+    }
+    val stream = new ObjectInputStream(input)
+    val record = stream.readObject().asInstanceOf[HoodieSparkRecord]
+    stream.close()
+    record.setStructType(schema)
+
+    record
+  }
+
+  override def write(kryo: Kryo, output: Output, datum: HoodieSparkRecord): Unit =
+    serializeDatum(datum, output)
+
+  override def read(kryo: Kryo, input: Input, datumClass: Class[HoodieSparkRecord]): HoodieSparkRecord =
+    deserializeDatum(input)
+}
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/clustering/plan/strategy/TestSparkConsistentBucketClusteringPlanStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/clustering/plan/strategy/TestSparkConsistentBucketClusteringPlanStrategy.java
index 810dbc5bc7..694fc77205 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/clustering/plan/strategy/TestSparkConsistentBucketClusteringPlanStrategy.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/clustering/plan/strategy/TestSparkConsistentBucketClusteringPlanStrategy.java
@@ -29,7 +29,7 @@ import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.testutils.HoodieTestUtils;
 import org.apache.hudi.common.util.collection.Triple;
 import org.apache.hudi.config.HoodieIndexConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.index.bucket.ConsistentBucketIdentifier;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java
index d970d593d0..a58c2ce8b5 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestConsistentBucketIndex.java
@@ -33,7 +33,7 @@ import org.apache.hudi.common.testutils.HoodieTestUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.hadoop.HoodieParquetInputFormat;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java
index e3a90becdd..db8afa2ce1 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java
@@ -41,10 +41,10 @@ import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
 import org.apache.hudi.common.model.HoodieFileFormat;
 import org.apache.hudi.common.model.HoodieFileGroup;
 import org.apache.hudi.common.model.HoodieFileGroupId;
-import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieLogFile;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.model.HoodieWriteStat;
@@ -80,7 +80,7 @@ import org.apache.hudi.config.HoodieClusteringConfig;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
 import org.apache.hudi.config.HoodieLockConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieMetadataException;
 import org.apache.hudi.index.HoodieIndex;
@@ -358,7 +358,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
 
     assertTrue(metadataWriter.isPresent());
     HoodieTableConfig hoodieTableConfig =
-        new HoodieTableConfig(this.fs, metaClient.getMetaPath(), writeConfig.getPayloadClass());
+        new HoodieTableConfig(this.fs, metaClient.getMetaPath(), writeConfig.getPayloadClass(), writeConfig.getStringOrDefault(HoodieWriteConfig.MERGER_IMPLS));
     assertFalse(hoodieTableConfig.getMetadataPartitions().isEmpty());
 
     // Turn off metadata table
@@ -375,7 +375,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
     assertFalse(metadataWriter2.isPresent());
 
     HoodieTableConfig hoodieTableConfig2 =
-        new HoodieTableConfig(this.fs, metaClient.getMetaPath(), writeConfig2.getPayloadClass());
+        new HoodieTableConfig(this.fs, metaClient.getMetaPath(), writeConfig2.getPayloadClass(), writeConfig.getStringOrDefault(HoodieWriteConfig.MERGER_IMPLS));
     assertEquals(Collections.emptySet(), hoodieTableConfig2.getMetadataPartitions());
     // Assert metadata table folder is deleted
     assertFalse(metaClient.getFs().exists(
@@ -397,7 +397,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
     validateMetadata(testTable, true);
     assertTrue(metadataWriter3.isPresent());
     HoodieTableConfig hoodieTableConfig3 =
-        new HoodieTableConfig(this.fs, metaClient.getMetaPath(), writeConfig.getPayloadClass());
+        new HoodieTableConfig(this.fs, metaClient.getMetaPath(), writeConfig.getPayloadClass(), writeConfig.getStringOrDefault(HoodieWriteConfig.MERGER_IMPLS));
     assertFalse(hoodieTableConfig3.getMetadataPartitions().isEmpty());
   }
 
@@ -1012,7 +1012,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
         while (logFileReader.hasNext()) {
           HoodieLogBlock logBlock = logFileReader.next();
           if (logBlock instanceof HoodieDataBlock) {
-            try (ClosableIterator<HoodieRecord> recordItr = ((HoodieDataBlock) logBlock).getRecordIterator(HoodieAvroIndexedRecord::new)) {
+            try (ClosableIterator<HoodieRecord<IndexedRecord>> recordItr = ((HoodieDataBlock) logBlock).getRecordIterator(HoodieRecordType.AVRO)) {
               recordItr.forEachRemaining(indexRecord -> {
                 final GenericRecord record = (GenericRecord) indexRecord.getData();
                 if (enableMetaFields) {
@@ -2540,7 +2540,7 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
         while (logFileReader.hasNext()) {
           HoodieLogBlock logBlock = logFileReader.next();
           if (logBlock instanceof HoodieDataBlock) {
-            try (ClosableIterator<HoodieRecord> recordItr = ((HoodieDataBlock) logBlock).getRecordIterator(HoodieAvroIndexedRecord::new)) {
+            try (ClosableIterator<HoodieRecord<IndexedRecord>> recordItr = ((HoodieDataBlock) logBlock).getRecordIterator(HoodieRecordType.AVRO)) {
               recordItr.forEachRemaining(indexRecord -> {
                 final GenericRecord record = (GenericRecord) indexRecord.getData();
                 final GenericRecord colStatsRecord = (GenericRecord) record.get(HoodieMetadataPayload.SCHEMA_FIELD_ID_COLUMN_STATS);
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java
index 432aa83d6a..5efed8a461 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java
@@ -23,9 +23,9 @@ import org.apache.hudi.avro.model.HoodieMetadataRecord;
 import org.apache.hudi.common.config.HoodieMetadataConfig;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieBaseFile;
-import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieLogFile;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.TableSchemaResolver;
@@ -353,7 +353,7 @@ public class TestHoodieBackedTableMetadata extends TestHoodieMetadataBase {
         while (logFileReader.hasNext()) {
           HoodieLogBlock logBlock = logFileReader.next();
           if (logBlock instanceof HoodieDataBlock) {
-            try (ClosableIterator<HoodieRecord> recordItr = ((HoodieDataBlock) logBlock).getRecordIterator(HoodieAvroIndexedRecord::new)) {
+            try (ClosableIterator<HoodieRecord<IndexedRecord>> recordItr = ((HoodieDataBlock) logBlock).getRecordIterator(HoodieRecordType.AVRO)) {
               recordItr.forEachRemaining(indexRecord -> {
                 final GenericRecord record = (GenericRecord) indexRecord.getData();
                 assertNull(record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD));
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java
index eb950a9a77..d342f92011 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java
@@ -31,6 +31,7 @@ import org.apache.hudi.client.clustering.run.strategy.SparkSingleFileSortExecuti
 import org.apache.hudi.client.validator.SparkPreCommitValidator;
 import org.apache.hudi.client.validator.SqlQueryEqualityPreCommitValidator;
 import org.apache.hudi.client.validator.SqlQuerySingleResultPreCommitValidator;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.common.config.TypedProperties;
 import org.apache.hudi.common.data.HoodieData;
 import org.apache.hudi.common.engine.HoodieEngineContext;
@@ -38,14 +39,14 @@ import org.apache.hudi.common.fs.ConsistencyGuardConfig;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieAvroRecord;
-import org.apache.hudi.common.model.HoodieAvroRecordMerge;
+import org.apache.hudi.common.model.HoodieAvroRecordMerger;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
 import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
 import org.apache.hudi.common.model.HoodieFileGroupId;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieMerge;
+import org.apache.hudi.common.model.HoodieRecordMerger;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieWriteStat;
 import org.apache.hudi.common.model.IOType;
@@ -69,6 +70,7 @@ import org.apache.hudi.common.util.BaseFileUtils;
 import org.apache.hudi.common.util.ClusteringUtils;
 import org.apache.hudi.common.util.CollectionUtils;
 import org.apache.hudi.common.util.FileIOUtils;
+import org.apache.hudi.common.util.HoodieRecordUtils;
 import org.apache.hudi.common.util.MarkerUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.StringUtils;
@@ -79,7 +81,6 @@ import org.apache.hudi.config.HoodieClusteringConfig;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
 import org.apache.hudi.config.HoodiePreCommitValidatorConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.data.HoodieJavaRDD;
 import org.apache.hudi.exception.HoodieCommitException;
@@ -443,6 +444,11 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
    */
   private void testDeduplication(
       Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn, boolean populateMetaFields) throws Exception {
+    HoodieWriteConfig.Builder configBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY)
+        .combineInput(true, true);
+    addConfigsForPopulateMetaFields(configBuilder, populateMetaFields);
+    HoodieWriteConfig writeConfig = configBuilder.build();
+
     String newCommitTime = "001";
 
     String recordKey = UUID.randomUUID().toString();
@@ -464,8 +470,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
     // Global dedup should be done based on recordKey only
     HoodieIndex index = mock(HoodieIndex.class);
     when(index.isGlobal()).thenReturn(true);
-    HoodieMerge merge = new HoodieAvroRecordMerge();
-    List<HoodieRecord<RawTripTestPayload>> dedupedRecs = HoodieWriteHelper.newInstance().deduplicateRecords(records, index, 1, merge).collectAsList();
+    HoodieRecordMerger recordMerger = HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName());
+    List<HoodieRecord<RawTripTestPayload>> dedupedRecs = HoodieWriteHelper.newInstance().deduplicateRecords(records, index, 1, recordMerger, writeConfig.getProps()).collectAsList();
     assertEquals(1, dedupedRecs.size());
     assertEquals(dedupedRecs.get(0).getPartitionPath(), recordThree.getPartitionPath());
     assertNodupesWithinPartition(dedupedRecs);
@@ -473,17 +479,14 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
     // non-Global dedup should be done based on both recordKey and partitionPath
     index = mock(HoodieIndex.class);
     when(index.isGlobal()).thenReturn(false);
-    dedupedRecs = HoodieWriteHelper.newInstance().deduplicateRecords(records, index, 1, merge).collectAsList();
+    dedupedRecs = HoodieWriteHelper.newInstance().deduplicateRecords(records, index, 1, recordMerger, writeConfig.getProps()).collectAsList();
     assertEquals(2, dedupedRecs.size());
     assertNodupesWithinPartition(dedupedRecs);
 
     // Perform write-action and check
     JavaRDD<HoodieRecord> recordList = jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1);
-    HoodieWriteConfig.Builder configBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY)
-        .combineInput(true, true);
-    addConfigsForPopulateMetaFields(configBuilder, populateMetaFields);
 
-    try (SparkRDDWriteClient client = getHoodieWriteClient(configBuilder.build());) {
+    try (SparkRDDWriteClient client = getHoodieWriteClient(writeConfig);) {
       client.startCommitWithTime(newCommitTime);
       List<WriteStatus> statuses = writeFn.apply(client, recordList, newCommitTime).collect();
       assertNoWriteErrors(statuses);
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java
index 8c92f8189f..d6d5b2495a 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java
@@ -38,7 +38,7 @@ import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
 import org.apache.hudi.config.HoodieLayoutConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.index.HoodieIndex.IndexType;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java
index 29c653daee..6bb9502852 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java
@@ -21,6 +21,7 @@ package org.apache.hudi.client.functional;
 import org.apache.hudi.avro.model.HoodieMetadataRecord;
 import org.apache.hudi.client.HoodieTimelineArchiver;
 import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.common.fs.ConsistencyGuardConfig;
 import org.apache.hudi.common.model.HoodieCleaningPolicy;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
@@ -39,7 +40,6 @@ import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieCleanConfig;
 import org.apache.hudi.config.HoodieArchivalConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.config.metrics.HoodieMetricsConfig;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java
index 6763cb154f..9dcd1d876a 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSparkConsistentBucketClustering.java
@@ -36,7 +36,7 @@ import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieClusteringConfig;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java
index be80e9d378..a60ecee1a9 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseQPSResourceAllocator.java
@@ -23,7 +23,7 @@ import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieHBaseIndexConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.index.HoodieIndex;
 
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java
index 407fb8de0e..43c45d00d0 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java
@@ -37,7 +37,7 @@ import org.apache.hudi.config.HoodieArchivalConfig;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieHBaseIndexConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.table.HoodieSparkTable;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java
index e02e613642..2151b13284 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieKeyLocationFetchHandle.java
@@ -33,7 +33,7 @@ import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.index.HoodieIndexUtils;
 import org.apache.hudi.keygen.BaseKeyGenerator;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java
index 72749160e6..05a4de483c 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java
@@ -33,7 +33,7 @@ import org.apache.hudi.common.testutils.HoodieTestTable;
 import org.apache.hudi.common.util.collection.ExternalSpillableMap;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieAvroFileWriterFactory.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieAvroFileWriterFactory.java
index d9377fc4e8..7789254bc7 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieAvroFileWriterFactory.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/TestHoodieAvroFileWriterFactory.java
@@ -20,6 +20,7 @@ package org.apache.hudi.io.storage;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hudi.client.SparkTaskContextSupplier;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.table.HoodieSparkTable;
@@ -46,26 +47,26 @@ public class TestHoodieAvroFileWriterFactory extends HoodieClientTestBase {
     HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
     SparkTaskContextSupplier supplier = new SparkTaskContextSupplier();
     HoodieFileWriter parquetWriter = HoodieFileWriterFactory.getFileWriter(instantTime,
-        parquetPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier);
+        parquetPath, table.getHadoopConf(), cfg.getStorageConfig(), HoodieTestDataGenerator.AVRO_SCHEMA, supplier, HoodieRecordType.AVRO);
     assertTrue(parquetWriter instanceof HoodieAvroParquetWriter);
 
     // hfile format.
     final Path hfilePath = new Path(basePath + "/partition/path/f1_1-0-1_000.hfile");
     HoodieFileWriter hfileWriter = HoodieFileWriterFactory.getFileWriter(instantTime,
-        hfilePath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier);
+        hfilePath, table.getHadoopConf(), cfg.getStorageConfig(), HoodieTestDataGenerator.AVRO_SCHEMA, supplier, HoodieRecordType.AVRO);
     assertTrue(hfileWriter instanceof HoodieAvroHFileWriter);
 
     // orc file format.
     final Path orcPath = new Path(basePath + "/partition/path/f1_1-0-1_000.orc");
     HoodieFileWriter orcFileWriter = HoodieFileWriterFactory.getFileWriter(instantTime,
-        orcPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier);
+        orcPath, table.getHadoopConf(), cfg.getStorageConfig(), HoodieTestDataGenerator.AVRO_SCHEMA, supplier, HoodieRecordType.AVRO);
     assertTrue(orcFileWriter instanceof HoodieAvroOrcWriter);
 
     // other file format exception.
     final Path logPath = new Path(basePath + "/partition/path/f.b51192a8-574b-4a85-b246-bcfec03ac8bf_100.log.2_1-0-1");
     final Throwable thrown = assertThrows(UnsupportedOperationException.class, () -> {
       HoodieFileWriter logWriter = HoodieFileWriterFactory.getFileWriter(instantTime, logPath,
-          table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier);
+          table.getHadoopConf(), cfg.getStorageConfig(), HoodieTestDataGenerator.AVRO_SCHEMA, supplier, HoodieRecordType.AVRO);
     }, "should fail since log storage writer is not supported yet.");
     assertTrue(thrown.getMessage().contains("format not supported yet."));
   }
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java
index 481cda00d6..6d43782ee3 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/storage/row/TestHoodieInternalRowParquetWriter.java
@@ -20,9 +20,9 @@ package org.apache.hudi.io.storage.row;
 
 import org.apache.hudi.common.bloom.BloomFilter;
 import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.io.storage.HoodieParquetConfig;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
@@ -116,6 +116,6 @@ public class TestHoodieInternalRowParquetWriter extends HoodieClientTestHarness
         writeConfig.getBloomFilterFPP(),
         writeConfig.getDynamicBloomFilterMaxNumEntries(),
         writeConfig.getBloomFilterType());
-    return new HoodieRowParquetWriteSupport(hadoopConf, SparkDatasetTestUtils.STRUCT_TYPE, Option.of(filter), writeConfig);
+    return new HoodieRowParquetWriteSupport(hadoopConf, SparkDatasetTestUtils.STRUCT_TYPE, Option.of(filter), writeConfig.getStorageConfig());
   }
 }
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java
index f165b48db0..a186070e00 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestCopyOnWriteActionExecutor.java
@@ -40,7 +40,7 @@ import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieIndexConfig;
 import org.apache.hudi.config.HoodieLayoutConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.data.HoodieJavaRDD;
 import org.apache.hudi.hadoop.HoodieParquetInputFormat;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java
index 3039eb3bd9..9d3fcce2d6 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java
@@ -38,7 +38,7 @@ import org.apache.hudi.config.HoodieClusteringConfig;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieHBaseIndexConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java
index a571a6f473..b99abcadea 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/CompactionTestBase.java
@@ -42,7 +42,7 @@ import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.table.HoodieTable;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java
index 59174a9371..c6cd554e28 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java
@@ -37,7 +37,7 @@ import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
 import org.apache.hudi.config.HoodieMemoryConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieNotSupportedException;
 import org.apache.hudi.index.HoodieIndex;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java
index 1c4de34e5e..707cf3da50 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java
@@ -39,7 +39,7 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
 import org.apache.hudi.common.util.StringUtils;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.table.HoodieTable;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java
index 5438fbcfc0..1b5d29094d 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableClustering.java
@@ -32,7 +32,7 @@ import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieClusteringConfig;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.table.HoodieSparkTable;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java
index f959a8f0d9..5c9021a5b0 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java
@@ -33,7 +33,7 @@ import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
 import org.apache.hudi.config.HoodieLayoutConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.table.action.commit.SparkBucketIndexPartitioner;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java
index 0a11425ec5..a5a5c31de2 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java
@@ -46,7 +46,7 @@ import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieCleanConfig;
 import org.apache.hudi.config.HoodieCompactionConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.index.HoodieIndex;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java
index 900674a677..204654dc5d 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java
@@ -20,6 +20,7 @@ package org.apache.hudi.testutils;
 
 import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.common.HoodieCleanStat;
 import org.apache.hudi.common.fs.ConsistencyGuardConfig;
 import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
@@ -41,7 +42,6 @@ import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieCleanConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.config.HoodieCompactionConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.index.HoodieIndex.IndexType;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java
index cb7b2e6b3c..c452f413bc 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java
@@ -47,7 +47,7 @@ import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieClusteringConfig;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.data.HoodieJavaRDD;
 import org.apache.hudi.exception.HoodieIOException;
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java
index 918462ac0a..09e6bd699b 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkDatasetTestUtils.java
@@ -23,7 +23,7 @@ import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
 import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.index.HoodieIndex;
 
diff --git a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java
index 1c638c5d63..2af3f8f3a1 100644
--- a/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java
@@ -19,10 +19,12 @@
 package org.apache.hudi.avro;
 
 import org.apache.hudi.common.config.SerializableSchema;
+import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieAvroRecord;
 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.SpillableMapUtils;
 import org.apache.hudi.common.util.StringUtils;
 import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.hudi.common.util.collection.Pair;
@@ -113,7 +115,7 @@ public class HoodieAvroUtils {
    * TODO serialize other type of record.
    */
   public static Option<byte[]> recordToBytes(HoodieRecord record, Schema schema) throws IOException {
-    return Option.of(HoodieAvroUtils.indexedRecordToBytes((IndexedRecord) record.toIndexedRecord(schema, new Properties()).get()));
+    return Option.of(HoodieAvroUtils.indexedRecordToBytes(((HoodieAvroIndexedRecord) record.toIndexedRecord(schema, new Properties()).get()).getData()));
   }
 
   /**
@@ -434,14 +436,18 @@ public class HoodieAvroUtils {
 
   private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field field) {
     Schema oldSchema = oldRecord.getSchema();
-    Object fieldValue = oldSchema.getField(field.name()) == null ? null : oldRecord.get(field.name());
+    Field oldSchemaField = oldSchema.getField(field.name());
+    Object fieldValue = oldSchemaField == null ? null : oldRecord.get(field.name());
 
     if (fieldValue != null) {
       // In case field's value is a nested record, we have to rewrite it as well
       Object newFieldValue;
       if (fieldValue instanceof GenericRecord) {
         GenericRecord record = (GenericRecord) fieldValue;
-        newFieldValue = rewriteRecord(record, resolveUnionSchema(field.schema(), record.getSchema().getFullName()));
+        // May return null when use rewrite
+        String recordFullName = record.getSchema().getFullName();
+        String fullName = recordFullName != null ? recordFullName : oldSchemaField.name();
+        newFieldValue = rewriteRecord(record, resolveUnionSchema(field.schema(), fullName));
       } else {
         newFieldValue = fieldValue;
       }
@@ -696,7 +702,7 @@ public class HoodieAvroUtils {
                                              String[] columns,
                                              Schema schema, boolean consistentLogicalTimestampEnabled) {
     try {
-      GenericRecord genericRecord = (GenericRecord) record.toIndexedRecord(schema, new Properties()).get();
+      GenericRecord genericRecord = (GenericRecord) ((HoodieAvroIndexedRecord) record.toIndexedRecord(schema, new Properties()).get()).getData();
       if (columns.length == 1) {
         return HoodieAvroUtils.getNestedFieldVal(genericRecord, columns[0], true, consistentLogicalTimestampEnabled);
       } else {
@@ -1033,6 +1039,28 @@ public class HoodieAvroUtils {
     };
   }
 
+  public static HoodieRecord createHoodieRecordFromAvro(
+      IndexedRecord data,
+      String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Boolean populateMetaFields) {
+    if (populateMetaFields) {
+      return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) data,
+          payloadClass, preCombineField, withOperation);
+      // Support HoodieFileSliceReader
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      // TODO in HoodieFileSliceReader may partitionName=option#empty
+      return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) data,
+          payloadClass, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, partitionNameOp);
+    } else {
+      return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) data,
+          payloadClass, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
   public static GenericRecord rewriteRecordDeep(GenericRecord oldRecord, Schema newSchema) {
     return rewriteRecordWithNewSchema(oldRecord, newSchema, Collections.EMPTY_MAP);
   }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java
index 6d4236b048..8aa50faaab 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java
@@ -19,10 +19,12 @@
 package org.apache.hudi.common.config;
 
 import org.apache.hadoop.fs.FSDataInputStream;
+
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ReflectionUtils;
 import org.apache.hudi.common.util.StringUtils;
 import org.apache.hudi.exception.HoodieException;
+
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
@@ -143,6 +145,14 @@ public class HoodieConfig implements Serializable {
     return StringUtils.split(getString(configProperty), delimiter);
   }
 
+  public <T> List<String> getSplitStringsOrDefault(ConfigProperty<T> configProperty) {
+    return getSplitStringsOrDefault(configProperty, ",");
+  }
+
+  public <T> List<String> getSplitStringsOrDefault(ConfigProperty<T> configProperty, String delimiter) {
+    return StringUtils.split(getStringOrDefault(configProperty), delimiter);
+  }
+
   public String getString(String key) {
     return props.getProperty(key);
   }
@@ -155,7 +165,7 @@ public class HoodieConfig implements Serializable {
   public <T> Integer getIntOrDefault(ConfigProperty<T> configProperty) {
     Option<Object> rawValue = getRawValue(configProperty);
     return rawValue.map(v -> Integer.parseInt(v.toString()))
-        .orElse((Integer) configProperty.defaultValue());
+        .orElse(Integer.parseInt(configProperty.defaultValue().toString()));
   }
 
   public <T> Boolean getBoolean(ConfigProperty<T> configProperty) {
@@ -182,16 +192,34 @@ public class HoodieConfig implements Serializable {
     return rawValue.map(v -> Long.parseLong(v.toString())).orElse(null);
   }
 
+  public <T> Long getLongOrDefault(ConfigProperty<T> configProperty) {
+    Option<Object> rawValue = getRawValue(configProperty);
+    return rawValue.map(v -> Long.parseLong(v.toString()))
+            .orElseGet(() -> Long.parseLong(configProperty.defaultValue().toString()));
+  }
+
   public <T> Float getFloat(ConfigProperty<T> configProperty) {
     Option<Object> rawValue = getRawValue(configProperty);
     return rawValue.map(v -> Float.parseFloat(v.toString())).orElse(null);
   }
 
+  public <T> Float getFloatOrDefault(ConfigProperty<T> configProperty) {
+    Option<Object> rawValue = getRawValue(configProperty);
+    return rawValue.map(v -> Float.parseFloat(v.toString()))
+            .orElseGet(() -> Float.parseFloat(configProperty.defaultValue().toString()));
+  }
+
   public <T> Double getDouble(ConfigProperty<T> configProperty) {
     Option<Object> rawValue = getRawValue(configProperty);
     return rawValue.map(v -> Double.parseDouble(v.toString())).orElse(null);
   }
 
+  public <T> Double getDoubleOrDefault(ConfigProperty<T> configProperty) {
+    Option<Object> rawValue = getRawValue(configProperty);
+    return rawValue.map(v -> Double.parseDouble(v.toString()))
+            .orElseGet(() -> Double.parseDouble(configProperty.defaultValue().toString()));
+  }
+
   public <T> String getStringOrDefault(ConfigProperty<T> configProperty) {
     return getStringOrDefault(configProperty, configProperty.defaultValue().toString());
   }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java
similarity index 98%
rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java
rename to hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java
index 40c53fae96..89841ed2fa 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieStorageConfig.java
@@ -16,12 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.config;
-
-import org.apache.hudi.common.config.ConfigClassProperty;
-import org.apache.hudi.common.config.ConfigGroups;
-import org.apache.hudi.common.config.ConfigProperty;
-import org.apache.hudi.common.config.HoodieConfig;
+package org.apache.hudi.common.config;
 
 import javax.annotation.concurrent.Immutable;
 
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java
index daec2fee03..28265bbbf4 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java
@@ -19,21 +19,22 @@
 package org.apache.hudi.common.model;
 
 import org.apache.hudi.avro.HoodieAvroUtils;
-import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.util.ConfigUtils;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
 
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.generic.IndexedRecord;
 
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
 
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+
 /**
  * This only use by reader returning.
  */
@@ -48,7 +49,7 @@ public class HoodieAvroIndexedRecord extends HoodieRecord<IndexedRecord> {
   }
 
   public HoodieAvroIndexedRecord(HoodieKey key, IndexedRecord data, HoodieOperation operation) {
-    super(key, data, operation, null);
+    super(key, data, operation);
   }
 
   public HoodieAvroIndexedRecord(HoodieRecord<IndexedRecord> record) {
@@ -58,28 +59,19 @@ public class HoodieAvroIndexedRecord extends HoodieRecord<IndexedRecord> {
   public HoodieAvroIndexedRecord() {
   }
 
-  @Override
-  public Option<IndexedRecord> toIndexedRecord(Schema schema, Properties prop) {
-    return Option.of(data);
-  }
-
-  public Option<IndexedRecord> toIndexedRecord() {
-    return Option.of(data);
-  }
-
   @Override
   public HoodieRecord newInstance() {
-    throw new UnsupportedOperationException();
+    return new HoodieAvroIndexedRecord(this);
   }
 
   @Override
   public HoodieRecord<IndexedRecord> newInstance(HoodieKey key, HoodieOperation op) {
-    throw new UnsupportedOperationException();
+    return new HoodieAvroIndexedRecord(key, data, op);
   }
 
   @Override
   public HoodieRecord<IndexedRecord> newInstance(HoodieKey key) {
-    throw new UnsupportedOperationException();
+    return new HoodieAvroIndexedRecord(key, data);
   }
 
   @Override
@@ -87,6 +79,11 @@ public class HoodieAvroIndexedRecord extends HoodieRecord<IndexedRecord> {
     return keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getRecordKey((GenericRecord) data) : ((GenericRecord) data).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
   }
 
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.AVRO;
+  }
+
   @Override
   public String getRecordKey(String keyFieldName) {
     return Option.ofNullable(data.getSchema().getField(keyFieldName))
@@ -95,74 +92,80 @@ public class HoodieAvroIndexedRecord extends HoodieRecord<IndexedRecord> {
   }
 
   @Override
-  public HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException {
-    ValidationUtils.checkState(other instanceof HoodieAvroIndexedRecord);
-    GenericRecord record = HoodieAvroUtils.stitchRecords((GenericRecord) data, (GenericRecord) other.getData(), writerSchema);
-    return new HoodieAvroIndexedRecord(record);
+  public Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled) {
+    throw new UnsupportedOperationException();
   }
 
   @Override
-  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
-    GenericRecord avroPayloadInNewSchema =
-        HoodieAvroUtils.rewriteRecord((GenericRecord) data, targetSchema);
-    return new HoodieAvroIndexedRecord(avroPayloadInNewSchema);
+  public HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException {
+    GenericRecord record = HoodieAvroUtils.stitchRecords((GenericRecord) data, (GenericRecord) other.getData(), targetSchema);
+    return new HoodieAvroIndexedRecord(record);
   }
 
   @Override
-  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
-    GenericRecord rewriteRecord = schemaOnReadEnabled ? HoodieAvroUtils.rewriteRecordWithNewSchema(data, writeSchemaWithMetaFields, new HashMap<>())
-        : HoodieAvroUtils.rewriteRecord((GenericRecord) data, writeSchemaWithMetaFields);
-    return new HoodieAvroIndexedRecord(rewriteRecord);
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException {
+    GenericRecord genericRecord = HoodieAvroUtils.rewriteRecord((GenericRecord) data, targetSchema);
+    return new HoodieAvroIndexedRecord(genericRecord);
   }
 
   @Override
-  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
-    GenericRecord rewriteRecord = schemaOnReadEnabled ? HoodieAvroUtils.rewriteEvolutionRecordWithMetadata((GenericRecord) data, writeSchemaWithMetaFields, fileName)
-        : HoodieAvroUtils.rewriteRecordWithMetadata((GenericRecord) data, writeSchemaWithMetaFields, fileName);
-    return new HoodieAvroIndexedRecord(rewriteRecord);
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    GenericRecord genericRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(data, newSchema, renameCols);
+    return new HoodieAvroIndexedRecord(genericRecord);
   }
 
   @Override
-  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
-    GenericRecord rewriteRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(data, newSchema, renameCols);
-    return new HoodieAvroIndexedRecord(rewriteRecord);
-  }
+  public HoodieRecord updateValues(Schema recordSchema, Properties props, Map<String, String> metadataValues) throws IOException {
+    metadataValues.forEach((key, value) -> {
+      if (value != null) {
+        ((GenericRecord) data).put(key, value);
+      }
+    });
 
-  @Override
-  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols, Mapper mapper) throws IOException {
-    GenericRecord oldRecord = (GenericRecord) getData();
-    GenericRecord rewriteRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(oldRecord, newSchema, renameCols);
-    return mapper.apply(rewriteRecord);
+    return new HoodieAvroIndexedRecord(data);
   }
 
   @Override
-  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
-    GenericRecord oldRecord = (GenericRecord) data;
-    GenericRecord rewriteRecord = HoodieAvroUtils.rewriteRecord(oldRecord, newSchema);
-    return new HoodieAvroIndexedRecord(rewriteRecord);
+  public boolean isDelete(Schema schema, Properties props) {
+    return false;
   }
 
   @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) {
-        ((GenericRecord) data).put(metadataField.getFieldName(), value);
-      }
-    });
-
-    return new HoodieAvroIndexedRecord(data);
+  public boolean shouldIgnore(Schema schema, Properties props) throws IOException {
+    return getData().equals(SENTINEL);
   }
 
   @Override
-  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
-    data.put(pos, newValue);
-    return this;
+  public HoodieRecord wrapIntoHoodieRecordPayloadWithParams(
+      Schema schema,
+      Properties props,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Boolean populateMetaFields) {
+    String payloadClass = ConfigUtils.getPayloadClass(props);
+    String preCombineField = ConfigUtils.getOrderingField(props);
+    return HoodieAvroUtils.createHoodieRecordFromAvro(data, payloadClass, preCombineField, simpleKeyGenFieldsOpt, withOperation, partitionNameOp, populateMetaFields);
   }
 
   @Override
-  public boolean shouldIgnore(Schema schema, Properties prop) throws IOException {
-    return getData().equals(SENTINEL);
+  public HoodieRecord wrapIntoHoodieRecordPayloadWithKeyGen(Properties props, Option<BaseKeyGenerator> keyGen) {
+    GenericRecord record = (GenericRecord) data;
+    String key;
+    String partition;
+    if (keyGen.isPresent() && !Boolean.parseBoolean(props.getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+      BaseKeyGenerator keyGeneratorOpt = keyGen.get();
+      key = keyGeneratorOpt.getRecordKey(record);
+      partition = keyGeneratorOpt.getPartitionPath(record);
+    } else {
+      key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+      partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
+    }
+    HoodieKey hoodieKey = new HoodieKey(key, partition);
+
+    HoodieRecordPayload avroPayload = new RewriteAvroPayload(record);
+    HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, avroPayload);
+    return hoodieRecord;
   }
 
   @Override
@@ -171,7 +174,17 @@ public class HoodieAvroIndexedRecord extends HoodieRecord<IndexedRecord> {
   }
 
   @Override
-  public boolean isPresent(Schema schema, Properties prop) {
-    return true;
+  public Comparable<?> getOrderingValue(Properties props) {
+    boolean consistentLogicalTimestampEnabled = Boolean.parseBoolean(props.getProperty(
+        KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(),
+        KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()));
+    return (Comparable<?>) HoodieAvroUtils.getNestedFieldVal((GenericRecord) data,
+        ConfigUtils.getOrderingField(props),
+        true, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public Option<HoodieAvroIndexedRecord> toIndexedRecord(Schema schema, Properties props) {
+    return Option.of(this);
   }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java
index 7bc6af89f8..65f15ca6a4 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java
@@ -20,33 +20,27 @@
 package org.apache.hudi.common.model;
 
 import org.apache.hudi.avro.HoodieAvroUtils;
-import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.util.HoodieRecordUtils;
+import org.apache.hudi.common.util.ConfigUtils;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.keygen.BaseKeyGenerator;
 
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.generic.IndexedRecord;
 
-import javax.annotation.Nonnull;
-
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
 
-import static org.apache.hudi.common.util.TypeUtils.unsafeCast;
-
 public class HoodieAvroRecord<T extends HoodieRecordPayload> extends HoodieRecord<T> {
+
   public HoodieAvroRecord(HoodieKey key, T data) {
     super(key, data);
   }
 
   public HoodieAvroRecord(HoodieKey key, T data, HoodieOperation operation) {
-    super(key, data, operation, null);
+    super(key, data, operation);
   }
 
   public HoodieAvroRecord(HoodieRecord<T> record) {
@@ -79,6 +73,11 @@ public class HoodieAvroRecord<T extends HoodieRecordPayload> extends HoodieRecor
     return data;
   }
 
+  @Override
+  public Comparable<?> getOrderingValue(Properties props) {
+    return this.getData().getOrderingValue();
+  }
+
   @Override
   public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
     return getRecordKey();
@@ -90,33 +89,22 @@ public class HoodieAvroRecord<T extends HoodieRecordPayload> extends HoodieRecor
   }
 
   @Override
-  public Comparable<?> getOrderingValue() {
-    return data.getOrderingValue();
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.AVRO;
   }
 
   @Override
-  public Option<IndexedRecord> toIndexedRecord(Schema schema, Properties prop) throws IOException {
-    return getData().getInsertValue(schema, prop);
+  public Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled) {
+    return HoodieAvroUtils.getRecordColumnValues(this, columns, recordSchema, consistentLogicalTimestampEnabled);
   }
 
-  //////////////////////////////////////////////////////////////////////////////
-
-  //
-  // NOTE: This method duplicates those ones of the HoodieRecordPayload and are placed here
-  //       for the duration of RFC-46 implementation, until migration off `HoodieRecordPayload`
-  //       is complete
   @Override
-  public HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException {
-    ValidationUtils.checkState(other instanceof HoodieAvroRecord);
-    GenericRecord mergedPayload = HoodieAvroUtils.stitchRecords(
-        (GenericRecord) toIndexedRecord(readerSchema, new Properties()).get(),
-        (GenericRecord) other.toIndexedRecord(readerSchema, new Properties()).get(),
-        writerSchema);
-    return new HoodieAvroRecord(getKey(), instantiateRecordPayloadWrapper(mergedPayload, getOrderingValue()), getOperation());
+  public HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException {
+    throw new UnsupportedOperationException();
   }
 
   @Override
-  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException {
     Option<IndexedRecord> avroRecordPayloadOpt = getData().getInsertValue(recordSchema, props);
     GenericRecord avroPayloadInNewSchema =
         HoodieAvroUtils.rewriteRecord((GenericRecord) avroRecordPayloadOpt.get(), targetSchema);
@@ -124,62 +112,57 @@ public class HoodieAvroRecord<T extends HoodieRecordPayload> extends HoodieRecor
   }
 
   @Override
-  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
-    GenericRecord record = (GenericRecord) getData().getInsertValue(recordSchema, prop).get();
-    GenericRecord rewriteRecord = schemaOnReadEnabled ? HoodieAvroUtils.rewriteRecordWithNewSchema(record, writeSchemaWithMetaFields, new HashMap<>())
-        : HoodieAvroUtils.rewriteRecord(record, writeSchemaWithMetaFields);
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    GenericRecord oldRecord = (GenericRecord) getData().getInsertValue(recordSchema, props).get();
+    GenericRecord rewriteRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(oldRecord, newSchema, renameCols);
     return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(rewriteRecord), getOperation());
   }
 
   @Override
-  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
-    GenericRecord record = (GenericRecord) getData().getInsertValue(recordSchema, prop).get();
-    GenericRecord rewriteRecord =  schemaOnReadEnabled ? HoodieAvroUtils.rewriteEvolutionRecordWithMetadata(record, writeSchemaWithMetaFields, fileName)
-        : HoodieAvroUtils.rewriteRecordWithMetadata(record, writeSchemaWithMetaFields, fileName);
-    return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(rewriteRecord), getOperation());
-  }
+  public HoodieRecord updateValues(Schema recordSchema, Properties props, Map<String, String> metadataValues) throws IOException {
+    GenericRecord avroRecordPayload = (GenericRecord) getData().getInsertValue(recordSchema, props).get();
 
-  @Override
-  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
-    GenericRecord oldRecord = (GenericRecord) getData().getInsertValue(recordSchema, prop).get();
-    GenericRecord rewriteRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(oldRecord, newSchema, renameCols);
-    return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(rewriteRecord), getOperation());
+    metadataValues.forEach((key, value) -> {
+      if (value != null) {
+        avroRecordPayload.put(key, value);
+      }
+    });
+
+    return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(avroRecordPayload), getOperation());
   }
 
   @Override
-  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
-    GenericRecord oldRecord = (GenericRecord) getData().getInsertValue(recordSchema, prop).get();
-    GenericRecord rewriteRecord = HoodieAvroUtils.rewriteRecord(oldRecord, newSchema);
-    return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(rewriteRecord), getOperation());
+  public boolean isDelete(Schema schema, Properties props) throws IOException {
+    return !getData().getInsertValue(schema, props).isPresent();
   }
 
   @Override
-  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols, Mapper mapper) throws IOException {
-    GenericRecord oldRecord = (GenericRecord) getData().getInsertValue(recordSchema, prop).get();
-    GenericRecord rewriteRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(oldRecord, newSchema, renameCols);
-    return mapper.apply(rewriteRecord);
+  public boolean shouldIgnore(Schema schema, Properties props) throws IOException {
+    Option<IndexedRecord> insertRecord = getData().getInsertValue(schema, props);
+    // just skip the ignored record
+    if (insertRecord.isPresent() && insertRecord.get().equals(SENTINEL)) {
+      return true;
+    } else {
+      return false;
+    }
   }
 
   @Override
-  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
-    IndexedRecord record = (IndexedRecord) data.getInsertValue(recordSchema, prop).get();
-    record.put(pos, newValue);
-    return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload((GenericRecord) record), getOperation());
+  public HoodieRecord wrapIntoHoodieRecordPayloadWithParams(
+      Schema schema, Properties props,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Boolean populateMetaFields) throws IOException {
+    IndexedRecord indexedRecord = (IndexedRecord) data.getInsertValue(schema, props).get();
+    String payloadClass = ConfigUtils.getPayloadClass(props);
+    String preCombineField = ConfigUtils.getOrderingField(props);
+    return HoodieAvroUtils.createHoodieRecordFromAvro(indexedRecord, payloadClass, preCombineField, simpleKeyGenFieldsOpt, withOperation, partitionNameOp, populateMetaFields);
   }
 
   @Override
-  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
-    // NOTE: RewriteAvroPayload is expected here
-    GenericRecord avroRecordPayload = (GenericRecord) getData().getInsertValue(recordSchema, prop).get();
-
-    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
-      String value = metadataValues.get(metadataField);
-      if (value != null) {
-        avroRecordPayload.put(metadataField.getFieldName(), value);
-      }
-    });
-
-    return new HoodieAvroRecord<>(getKey(), new RewriteAvroPayload(avroRecordPayload), getOperation());
+  public HoodieRecord wrapIntoHoodieRecordPayloadWithKeyGen(Properties props, Option<BaseKeyGenerator> keyGen) {
+    throw new UnsupportedOperationException();
   }
 
   public Option<Map<String, String>> getMetadata() {
@@ -187,28 +170,12 @@ public class HoodieAvroRecord<T extends HoodieRecordPayload> extends HoodieRecor
   }
 
   @Override
-  public boolean isPresent(Schema schema, Properties prop) throws IOException {
-    return getData().getInsertValue(schema, prop).isPresent();
-  }
-
-  @Override
-  public boolean shouldIgnore(Schema schema, Properties prop) throws IOException {
-    Option<IndexedRecord> insertRecord = getData().getInsertValue(schema, prop);
-    // just skip the ignored record
-    if (insertRecord.isPresent() && insertRecord.get().equals(SENTINEL)) {
-      return true;
+  public Option<HoodieAvroIndexedRecord> toIndexedRecord(Schema schema, Properties props) throws IOException {
+    Option<IndexedRecord> avroData = getData().getInsertValue(schema, props);
+    if (avroData.isPresent()) {
+      return Option.of(new HoodieAvroIndexedRecord(avroData.get()));
     } else {
-      return false;
+      return Option.empty();
     }
   }
-
-  @Nonnull
-  private T instantiateRecordPayloadWrapper(Object combinedAvroPayload, Comparable newPreCombineVal) {
-    return unsafeCast(
-        HoodieRecordUtils.loadPayload(
-            getData().getClass().getCanonicalName(),
-            new Object[]{combinedAvroPayload, newPreCombineVal},
-            GenericRecord.class,
-            Comparable.class));
-  }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecordMerge.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecordMerger.java
similarity index 55%
rename from hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecordMerge.java
rename to hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecordMerger.java
index bea89efdc1..9fa70a3719 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecordMerge.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecordMerger.java
@@ -20,7 +20,11 @@ package org.apache.hudi.common.model;
 
 import org.apache.avro.Schema;
 import org.apache.avro.generic.IndexedRecord;
+
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.hudi.metadata.HoodieMetadataPayload;
 
 import java.io.IOException;
@@ -28,9 +32,32 @@ import java.util.Properties;
 
 import static org.apache.hudi.common.util.TypeUtils.unsafeCast;
 
-public class HoodieAvroRecordMerge implements HoodieMerge {
+public class HoodieAvroRecordMerger implements HoodieRecordMerger {
+
+  @Override
+  public String getMergingStrategy() {
+    return StringUtils.DEFAULT_MERGER_STRATEGY_UUID;
+  }
+
   @Override
-  public HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer) {
+  public Option<HoodieRecord> merge(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException {
+    ValidationUtils.checkArgument(older.getRecordType() == HoodieRecordType.AVRO);
+    ValidationUtils.checkArgument(newer.getRecordType() == HoodieRecordType.AVRO);
+    if (older instanceof HoodieAvroRecord && newer instanceof HoodieAvroRecord) {
+      return Option.of(preCombine(older, newer));
+    } else if (older instanceof HoodieAvroIndexedRecord && newer instanceof HoodieAvroRecord) {
+      return combineAndGetUpdateValue(older, newer, schema, props);
+    } else {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.AVRO;
+  }
+
+  private HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer) {
     HoodieRecordPayload picked = unsafeCast(((HoodieAvroRecord) newer).getData().preCombine(((HoodieAvroRecord) older).getData()));
     if (picked instanceof HoodieMetadataPayload) {
       // NOTE: HoodieMetadataPayload return a new payload
@@ -39,19 +66,13 @@ public class HoodieAvroRecordMerge implements HoodieMerge {
     return picked.equals(((HoodieAvroRecord) newer).getData()) ? newer : older;
   }
 
-  @Override
-  public Option<HoodieRecord> combineAndGetUpdateValue(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException {
-    Option<IndexedRecord> previousRecordAvroPayload;
-    if (older instanceof HoodieAvroIndexedRecord) {
-      previousRecordAvroPayload = Option.ofNullable(((HoodieAvroIndexedRecord) older).getData());
-    } else {
-      previousRecordAvroPayload = ((HoodieRecordPayload)older.getData()).getInsertValue(schema, props);
-    }
+  private Option<HoodieRecord> combineAndGetUpdateValue(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException {
+    Option<HoodieAvroIndexedRecord> previousRecordAvroPayload = older.toIndexedRecord(schema, props);
     if (!previousRecordAvroPayload.isPresent()) {
       return Option.empty();
     }
 
-    return ((HoodieAvroRecord) newer).getData().combineAndGetUpdateValue(previousRecordAvroPayload.get(), schema, props)
+    return ((HoodieAvroRecord) newer).getData().combineAndGetUpdateValue(previousRecordAvroPayload.get().getData(), schema, props)
         .map(combinedAvroPayload -> new HoodieAvroIndexedRecord((IndexedRecord) combinedAvroPayload));
   }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieEmptyRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieEmptyRecord.java
new file mode 100644
index 0000000000..f2167bb15e
--- /dev/null
+++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieEmptyRecord.java
@@ -0,0 +1,155 @@
+/*
+ * 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.common.model;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+
+import org.apache.avro.Schema;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Properties;
+
+public class HoodieEmptyRecord<T> extends HoodieRecord<T> {
+
+  private final HoodieRecordType type;
+  private final Comparable<?> orderingVal;
+
+  public HoodieEmptyRecord(HoodieKey key, HoodieRecordType type) {
+    super(key, null);
+    this.type = type;
+    this.orderingVal = null;
+  }
+
+  public HoodieEmptyRecord(HoodieKey key, HoodieOperation operation, Comparable<?> orderingVal, HoodieRecordType type) {
+    super(key, null, operation);
+    this.type = type;
+    this.orderingVal = orderingVal;
+  }
+
+  public HoodieEmptyRecord(HoodieRecord<T> record, HoodieRecordType type) {
+    super(record);
+    this.type = type;
+    this.orderingVal = record.getOrderingValue(new Properties());
+  }
+
+  public HoodieEmptyRecord(HoodieRecordType type) {
+    this.type = type;
+    this.orderingVal = null;
+  }
+
+  @Override
+  public T getData() {
+    return null;
+  }
+
+  @Override
+  public Comparable<?> getOrderingValue(Properties props) {
+    return orderingVal;
+  }
+
+  @Override
+  public HoodieRecord<T> newInstance() {
+    return this;
+  }
+
+  @Override
+  public HoodieRecord<T> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieEmptyRecord<>(key, op, orderingVal, type);
+  }
+
+  @Override
+  public HoodieRecord<T> newInstance(HoodieKey key) {
+    return new HoodieEmptyRecord<>(key, type);
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return type;
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    return key.getRecordKey();
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    return key.getRecordKey();
+  }
+
+  @Override
+  public Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public HoodieRecord updateValues(Schema recordSchema, Properties props, Map<String, String> metadataValues) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean isDelete(Schema schema, Properties props) throws IOException {
+    return true;
+  }
+
+  @Override
+  public boolean shouldIgnore(Schema schema, Properties props) throws IOException {
+    return false;
+  }
+
+  @Override
+  public HoodieRecord wrapIntoHoodieRecordPayloadWithParams(Schema schema, Properties props, Option<Pair<String, String>> simpleKeyGenFieldsOpt, Boolean withOperation, Option<String> partitionNameOp,
+      Boolean populateMetaFieldsOp)
+      throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public HoodieRecord wrapIntoHoodieRecordPayloadWithKeyGen(Properties props, Option<BaseKeyGenerator> keyGen) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Option<HoodieAvroIndexedRecord> toIndexedRecord(Schema schema, Properties props) throws IOException {
+    return Option.empty();
+  }
+
+  @Override
+  public Option<Map<String, String>> getMetadata() {
+    return Option.empty();
+  }
+}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java
index 8ab6a245c4..22807d927e 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java
@@ -18,10 +18,10 @@
 
 package org.apache.hudi.common.model;
 
+import java.util.Collections;
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.generic.IndexedRecord;
-import org.apache.hudi.common.config.TypedProperties;
+
 import org.apache.hudi.common.util.CollectionUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.Pair;
@@ -40,7 +40,7 @@ import java.util.stream.IntStream;
 /**
  * A Single Record managed by Hoodie.
  */
-public abstract class HoodieRecord<T> implements Serializable {
+public abstract class HoodieRecord<T> implements HoodieRecordCompatibilityInterface, Serializable {
 
   public static final String COMMIT_TIME_METADATA_FIELD = HoodieMetadataField.COMMIT_TIME_METADATA_FIELD.getFieldName();
   public static final String COMMIT_SEQNO_METADATA_FIELD = HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD.getFieldName();
@@ -109,7 +109,7 @@ public abstract class HoodieRecord<T> implements Serializable {
   /**
    * Identifies the record across the table.
    */
-  private HoodieKey key;
+  protected HoodieKey key;
 
   /**
    * Actual payload of the record.
@@ -136,32 +136,21 @@ public abstract class HoodieRecord<T> implements Serializable {
    */
   private HoodieOperation operation;
 
-  /**
-   * For purposes of preCombining.
-   */
-  private Comparable<?> orderingVal;
-
   public HoodieRecord(HoodieKey key, T data) {
-    this(key, data, null, null);
+    this(key, data, null);
   }
 
-  public HoodieRecord(HoodieKey key, T data, Comparable<?> orderingVal) {
-    this(key, data, null, orderingVal);
-  }
-
-  public HoodieRecord(HoodieKey key, T data, HoodieOperation operation, Comparable<?> orderingVal) {
+  public HoodieRecord(HoodieKey key, T data, HoodieOperation operation) {
     this.key = key;
     this.data = data;
     this.currentLocation = null;
     this.newLocation = null;
     this.sealed = false;
     this.operation = operation;
-    // default natural order is 0
-    this.orderingVal = orderingVal == null ? 0 : orderingVal;
   }
 
   public HoodieRecord(HoodieRecord<T> record) {
-    this(record.key, record.data, record.operation, record.orderingVal);
+    this(record.key, record.data, record.operation);
     this.currentLocation = record.currentLocation;
     this.newLocation = record.newLocation;
     this.sealed = record.sealed;
@@ -184,13 +173,11 @@ public abstract class HoodieRecord<T> implements Serializable {
     return operation;
   }
 
-  public Comparable<?> getOrderingValue() {
-    return orderingVal;
-  }
+  public abstract Comparable<?> getOrderingValue(Properties props);
 
   public T getData() {
     if (data == null) {
-      throw new IllegalStateException("HoodieRecord already deflated for record.");
+      throw new IllegalStateException("Payload already deflated for record.");
     }
     return data;
   }
@@ -213,6 +200,10 @@ public abstract class HoodieRecord<T> implements Serializable {
     return this;
   }
 
+  public void setData(T data) {
+    this.data = data;
+  }
+
   public HoodieRecordLocation getCurrentLocation() {
     return currentLocation;
   }
@@ -273,6 +264,8 @@ public abstract class HoodieRecord<T> implements Serializable {
     return key.getRecordKey();
   }
 
+  public abstract HoodieRecordType getRecordType();
+
   public abstract String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt);
 
   public abstract String getRecordKey(String keyFieldName);
@@ -291,59 +284,51 @@ public abstract class HoodieRecord<T> implements Serializable {
     }
   }
 
-  //////////////////////////////////////////////////////////////////////////////
-
-  //
-  // NOTE: This method duplicates those ones of the HoodieRecordPayload and are placed here
-  //       for the duration of RFC-46 implementation, until migration off `HoodieRecordPayload`
-  //       is complete
-  //
-  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException;
+  /**
+   * Get column in record to support RDDCustomColumnsSortPartitioner
+   */
+  public abstract Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled);
 
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException;
+  /**
+   * Support bootstrap.
+   */
+  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException;
 
   /**
-   * Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields.
+   * Rewrite record into new schema(add meta columns)
    */
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException;
+  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException;
 
-  public abstract HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException;
+  /**
+   * Support schema evolution.
+   */
+  public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map<String, String> renameCols) throws IOException;
 
-  public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException;
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema) throws IOException {
+    return rewriteRecordWithNewSchema(recordSchema, props, newSchema, Collections.emptyMap());
+  }
 
-  public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols, Mapper mapper) throws IOException;
+  /**
+   * This method could change in the future.
+   * @temporary
+   */
+  public abstract HoodieRecord updateValues(Schema recordSchema, Properties props, Map<String, String> metadataValues) throws IOException;
 
-  public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException;
+  public abstract boolean isDelete(Schema schema, Properties props) throws IOException;
 
-  public abstract HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException;
+  /**
+   * Is EmptyRecord. Generated by ExpressionPayload.
+   */
+  public abstract boolean shouldIgnore(Schema schema, Properties props) throws IOException;
 
-  public abstract HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException;
+  public abstract Option<HoodieAvroIndexedRecord> toIndexedRecord(Schema schema, Properties props) throws IOException;
 
   public abstract Option<Map<String, String>> getMetadata();
 
-  public abstract boolean isPresent(Schema schema, Properties prop) throws IOException;
-
-  public abstract boolean shouldIgnore(Schema schema, Properties prop) throws IOException;
-
-  public abstract Option<IndexedRecord> toIndexedRecord(Schema schema, Properties prop) throws IOException;
-
-  //////////////////////////////////////////////////////////////////////////////
-
   public static String generateSequenceId(String instantTime, int partitionId, long recordIndex) {
     return instantTime + "_" + partitionId + "_" + recordIndex;
   }
 
-  /**
-   * NOTE: This is temporary transition construct to be able to construct
-   *       HoodieRecord instances w/o excessive wiring into a lot of components
-   *       a lot of details that are irrelevant for these
-   * TODO remove
-   */
-  @FunctionalInterface
-  public interface Mapper {
-    HoodieRecord apply(IndexedRecord avroPayload);
-  }
-
   /**
    * A special record returned by {@link HoodieRecordPayload}, which means we should just skip this record.
    * This record is only used for {@link HoodieRecordPayload} currently, so it should not
@@ -384,4 +369,8 @@ public abstract class HoodieRecord<T> implements Serializable {
       return null;
     }
   }
+
+  public enum HoodieRecordType {
+    AVRO, SPARK
+  }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieMerge.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordCompatibilityInterface.java
similarity index 56%
copy from hudi-common/src/main/java/org/apache/hudi/common/model/HoodieMerge.java
copy to hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordCompatibilityInterface.java
index 6becf35591..c2eb164dca 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieMerge.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordCompatibilityInterface.java
@@ -18,21 +18,28 @@
 
 package org.apache.hudi.common.model;
 
+import java.io.IOException;
+import java.util.Properties;
 import org.apache.avro.Schema;
 import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.keygen.BaseKeyGenerator;
 
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Properties;
+public interface HoodieRecordCompatibilityInterface {
 
-/**
- * HoodieMerge defines how to merge two records. It is a stateless component.
- * It can implement the merging logic of HoodieRecord of different engines
- * and avoid the performance consumption caused by the serialization/deserialization of Avro payload.
- */
-public interface HoodieMerge extends Serializable {
-  
-  HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer);
+  /**
+   * This method used to extract HoodieKey not through keyGenerator.
+   */
+  HoodieRecord wrapIntoHoodieRecordPayloadWithParams(
+      Schema schema,
+      Properties props,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Boolean populateMetaFieldsOp) throws IOException;
 
-  Option<HoodieRecord> combineAndGetUpdateValue(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;
+  /**
+   * This method used to extract HoodieKey through keyGenerator. This method used in ClusteringExecutionStrategy.
+   */
+  HoodieRecord wrapIntoHoodieRecordPayloadWithKeyGen(Properties props, Option<BaseKeyGenerator> keyGen);
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieMerge.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java
similarity index 56%
rename from hudi-common/src/main/java/org/apache/hudi/common/model/HoodieMerge.java
rename to hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java
index 6becf35591..43665b5711 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieMerge.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java
@@ -19,6 +19,10 @@
 package org.apache.hudi.common.model;
 
 import org.apache.avro.Schema;
+
+import org.apache.hudi.ApiMaturityLevel;
+import org.apache.hudi.PublicAPIClass;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.util.Option;
 
 import java.io.IOException;
@@ -30,9 +34,24 @@ import java.util.Properties;
  * It can implement the merging logic of HoodieRecord of different engines
  * and avoid the performance consumption caused by the serialization/deserialization of Avro payload.
  */
-public interface HoodieMerge extends Serializable {
-  
-  HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer);
+@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
+public interface HoodieRecordMerger extends Serializable {
+
+  /**
+   * This method converges combineAndGetUpdateValue and precombine from HoodiePayload.
+   * It'd be associative operation: f(a, f(b, c)) = f(f(a, b), c) (which we can translate as having 3 versions A, B, C
+   * of the single record, both orders of operations applications have to yield the same result)
+   */
+  Option<HoodieRecord> merge(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;
+
+  /**
+   * The record type handled by the current merger.
+   * SPARK, AVRO, FLINK
+   */
+  HoodieRecordType getRecordType();
 
-  Option<HoodieRecord> combineAndGetUpdateValue(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;
+  /**
+   * The kind of merging strategy this recordMerger belongs to. An UUID represents merging strategy.
+   */
+  String getMergingStrategy();
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java
index aae29a21bb..4b1a0bb0cb 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java
@@ -26,7 +26,6 @@ import org.apache.hudi.common.config.ConfigProperty;
 import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.common.config.OrderedProperties;
 import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.HoodieAvroRecordMerge;
 import org.apache.hudi.common.model.HoodieFileFormat;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieTableType;
@@ -156,11 +155,10 @@ public class HoodieTableConfig extends HoodieConfig {
       .withDocumentation("Payload class to use for performing compactions, i.e merge delta logs with current base file and then "
           + " produce a new base file.");
 
-  public static final ConfigProperty<String> MERGE_CLASS_NAME = ConfigProperty
-      .key("hoodie.compaction.merge.class")
-      .defaultValue(HoodieAvroRecordMerge.class.getName())
-      .withDocumentation("Merge class provide stateless component interface for merging records, and support various HoodieRecord "
-          + "types, such as Spark records or Flink records.");
+  public static final ConfigProperty<String> MERGER_STRATEGY = ConfigProperty
+      .key("hoodie.compaction.merger.strategy")
+      .defaultValue(StringUtils.DEFAULT_MERGER_STRATEGY_UUID)
+      .withDocumentation("Id of merger strategy.  Hudi will pick RecordMergers in hoodie.datasource.write.merger.impls which has the same merger strategy id");
 
   public static final ConfigProperty<String> ARCHIVELOG_FOLDER = ConfigProperty
       .key("hoodie.archivelog.folder")
@@ -244,15 +242,24 @@ public class HoodieTableConfig extends HoodieConfig {
 
   private static final String TABLE_CHECKSUM_FORMAT = "%s.%s"; // <database_name>.<table_name>
 
-  public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName) {
+  public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName, String mergerStrategy) {
     super();
     Path propertyPath = new Path(metaPath, HOODIE_PROPERTIES_FILE);
     LOG.info("Loading table properties from " + propertyPath);
     try {
       fetchConfigs(fs, metaPath);
+      boolean needStore = false;
       if (contains(PAYLOAD_CLASS_NAME) && payloadClassName != null
           && !getString(PAYLOAD_CLASS_NAME).equals(payloadClassName)) {
         setValue(PAYLOAD_CLASS_NAME, payloadClassName);
+        needStore = true;
+      }
+      if (contains(MERGER_STRATEGY) && payloadClassName != null
+          && !getString(MERGER_STRATEGY).equals(mergerStrategy)) {
+        setValue(MERGER_STRATEGY, mergerStrategy);
+        needStore = true;
+      }
+      if (needStore) {
         // FIXME(vc): wonder if this can be removed. Need to look into history.
         try (FSDataOutputStream outputStream = fs.create(propertyPath)) {
           storeProperties(props, outputStream);
@@ -419,6 +426,7 @@ public class HoodieTableConfig extends HoodieConfig {
       hoodieConfig.setDefaultValue(TYPE);
       if (hoodieConfig.getString(TYPE).equals(HoodieTableType.MERGE_ON_READ.name())) {
         hoodieConfig.setDefaultValue(PAYLOAD_CLASS_NAME);
+        hoodieConfig.setDefaultValue(MERGER_STRATEGY);
       }
       hoodieConfig.setDefaultValue(ARCHIVELOG_FOLDER);
       if (!hoodieConfig.contains(TIMELINE_LAYOUT_VERSION)) {
@@ -489,11 +497,10 @@ public class HoodieTableConfig extends HoodieConfig {
   }
 
   /**
-   * Read the hoodie merge class for HoodieRecords from the table properties.
+   * Read the payload class for HoodieRecords from the table properties.
    */
-  public String getMergeClass() {
-    return getStringOrDefault(MERGE_CLASS_NAME).replace("com.uber.hoodie",
-        "org.apache.hudi");
+  public String getMergerStrategy() {
+    return getStringOrDefault(MERGER_STRATEGY);
   }
 
   public String getPreCombineField() {
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
index e58e89b36b..d6b556e463 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java
@@ -102,7 +102,6 @@ public class HoodieTableMetaClient implements Serializable {
   //       computations secured by its immutability
   protected SerializablePath basePath;
   protected SerializablePath metaPath;
-
   private transient HoodieWrapperFileSystem fs;
   private boolean loadActiveTimelineOnLoad;
   protected SerializableConfiguration hadoopConf;
@@ -117,7 +116,7 @@ public class HoodieTableMetaClient implements Serializable {
 
   protected HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad,
                                 ConsistencyGuardConfig consistencyGuardConfig, Option<TimelineLayoutVersion> layoutVersion,
-                                String payloadClassName, FileSystemRetryConfig fileSystemRetryConfig) {
+                                String payloadClassName, String mergerStrategy, FileSystemRetryConfig fileSystemRetryConfig) {
     LOG.info("Loading HoodieTableMetaClient from " + basePath);
     this.consistencyGuardConfig = consistencyGuardConfig;
     this.fileSystemRetryConfig = fileSystemRetryConfig;
@@ -126,7 +125,7 @@ public class HoodieTableMetaClient implements Serializable {
     this.metaPath = new SerializablePath(new CachingPath(basePath, METAFOLDER_NAME));
     this.fs = getFs();
     TableNotFoundException.checkTableValidity(fs, this.basePath.get(), metaPath.get());
-    this.tableConfig = new HoodieTableConfig(fs, metaPath.toString(), payloadClassName);
+    this.tableConfig = new HoodieTableConfig(fs, metaPath.toString(), payloadClassName, mergerStrategy);
     this.tableType = tableConfig.getTableType();
     Option<TimelineLayoutVersion> tableConfigVersion = tableConfig.getTimelineLayoutVersion();
     if (layoutVersion.isPresent() && tableConfigVersion.isPresent()) {
@@ -161,6 +160,7 @@ public class HoodieTableMetaClient implements Serializable {
         .setConsistencyGuardConfig(oldMetaClient.consistencyGuardConfig)
         .setLayoutVersion(Option.of(oldMetaClient.timelineLayoutVersion))
         .setPayloadClassName(null)
+        .setMergerStrategy(null)
         .setFileSystemRetryConfig(oldMetaClient.fileSystemRetryConfig).build();
   }
 
@@ -634,7 +634,7 @@ public class HoodieTableMetaClient implements Serializable {
 
   private static HoodieTableMetaClient newMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad,
       ConsistencyGuardConfig consistencyGuardConfig, Option<TimelineLayoutVersion> layoutVersion,
-      String payloadClassName, FileSystemRetryConfig fileSystemRetryConfig, Properties props) {
+      String payloadClassName, String mergerStrategy, FileSystemRetryConfig fileSystemRetryConfig, Properties props) {
     HoodieMetastoreConfig metastoreConfig = null == props
         ? new HoodieMetastoreConfig.Builder().build()
         : new HoodieMetastoreConfig.Builder().fromProperties(props).build();
@@ -644,7 +644,7 @@ public class HoodieTableMetaClient implements Serializable {
             conf, consistencyGuardConfig, fileSystemRetryConfig,
             props.getProperty(HoodieTableConfig.DATABASE_NAME.key()), props.getProperty(HoodieTableConfig.NAME.key()), metastoreConfig)
         : new HoodieTableMetaClient(conf, basePath,
-        loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName, fileSystemRetryConfig);
+        loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName, mergerStrategy, fileSystemRetryConfig);
   }
 
   public static Builder builder() {
@@ -660,6 +660,7 @@ public class HoodieTableMetaClient implements Serializable {
     private String basePath;
     private boolean loadActiveTimelineOnLoad = false;
     private String payloadClassName = null;
+    private String mergerStrategy = null;
     private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build();
     private FileSystemRetryConfig fileSystemRetryConfig = FileSystemRetryConfig.newBuilder().build();
     private Option<TimelineLayoutVersion> layoutVersion = Option.of(TimelineLayoutVersion.CURR_LAYOUT_VERSION);
@@ -685,6 +686,11 @@ public class HoodieTableMetaClient implements Serializable {
       return this;
     }
 
+    public Builder setMergerStrategy(String mergerStrategy) {
+      this.mergerStrategy = mergerStrategy;
+      return this;
+    }
+
     public Builder setConsistencyGuardConfig(ConsistencyGuardConfig consistencyGuardConfig) {
       this.consistencyGuardConfig = consistencyGuardConfig;
       return this;
@@ -709,7 +715,8 @@ public class HoodieTableMetaClient implements Serializable {
       ValidationUtils.checkArgument(conf != null, "Configuration needs to be set to init HoodieTableMetaClient");
       ValidationUtils.checkArgument(basePath != null, "basePath needs to be set to init HoodieTableMetaClient");
       return newMetaClient(conf, basePath,
-          loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName, fileSystemRetryConfig, props);
+          loadActiveTimelineOnLoad, consistencyGuardConfig, layoutVersion, payloadClassName,
+          mergerStrategy, fileSystemRetryConfig, props);
     }
   }
 
@@ -726,7 +733,7 @@ public class HoodieTableMetaClient implements Serializable {
     private String recordKeyFields;
     private String archiveLogFolder;
     private String payloadClassName;
-    private String mergeClassName;
+    private String mergerStrategy;
     private Integer timelineLayoutVersion;
     private String baseFileFormat;
     private String preCombineField;
@@ -793,8 +800,8 @@ public class HoodieTableMetaClient implements Serializable {
       return this;
     }
 
-    public PropertyBuilder setMergeClassName(String mergeClassName) {
-      this.mergeClassName = mergeClassName;
+    public PropertyBuilder setMergerStrategy(String mergerStrategy) {
+      this.mergerStrategy = mergerStrategy;
       return this;
     }
 
@@ -900,9 +907,10 @@ public class HoodieTableMetaClient implements Serializable {
 
     public PropertyBuilder fromMetaClient(HoodieTableMetaClient metaClient) {
       return setTableType(metaClient.getTableType())
-        .setTableName(metaClient.getTableConfig().getTableName())
-        .setArchiveLogFolder(metaClient.getArchivePath())
-        .setPayloadClassName(metaClient.getTableConfig().getPayloadClass());
+          .setTableName(metaClient.getTableConfig().getTableName())
+          .setArchiveLogFolder(metaClient.getArchivePath())
+          .setPayloadClassName(metaClient.getTableConfig().getPayloadClass())
+          .setMergerStrategy(metaClient.getTableConfig().getMergerStrategy());
     }
 
     public PropertyBuilder fromProperties(Properties properties) {
@@ -932,6 +940,10 @@ public class HoodieTableMetaClient implements Serializable {
         setPayloadClassName(
             hoodieConfig.getString(HoodieTableConfig.PAYLOAD_CLASS_NAME));
       }
+      if (hoodieConfig.contains(HoodieTableConfig.MERGER_STRATEGY)) {
+        setMergerStrategy(
+            hoodieConfig.getString(HoodieTableConfig.MERGER_STRATEGY));
+      }
       if (hoodieConfig.contains(HoodieTableConfig.TIMELINE_LAYOUT_VERSION)) {
         setTimelineLayoutVersion(hoodieConfig.getInt(HoodieTableConfig.TIMELINE_LAYOUT_VERSION));
       }
@@ -1009,9 +1021,8 @@ public class HoodieTableMetaClient implements Serializable {
       if (tableType == HoodieTableType.MERGE_ON_READ && payloadClassName != null) {
         tableConfig.setValue(HoodieTableConfig.PAYLOAD_CLASS_NAME, payloadClassName);
       }
-
-      if (mergeClassName != null) {
-        tableConfig.setValue(HoodieTableConfig.MERGE_CLASS_NAME, mergeClassName);
+      if (tableType == HoodieTableType.MERGE_ON_READ && mergerStrategy != null) {
+        tableConfig.setValue(HoodieTableConfig.MERGER_STRATEGY, mergerStrategy);
       }
 
       if (null != tableCreateSchema) {
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java
index a41e5a1906..65defdc2bd 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java
@@ -19,14 +19,13 @@
 package org.apache.hudi.common.table.log;
 
 import org.apache.hudi.common.model.DeleteRecord;
-import org.apache.hudi.common.model.HoodieAvroRecord;
 import org.apache.hudi.common.model.HoodieLogFile;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
+import org.apache.hudi.common.model.HoodieRecordMerger;
 import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
-import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock.RecordIterator;
 import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
 import org.apache.hudi.common.table.log.block.HoodieDataBlock;
 import org.apache.hudi.common.table.log.block.HoodieDeleteBlock;
@@ -36,7 +35,6 @@ import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
 import org.apache.hudi.common.util.ClosableIterator;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.InternalSchemaCache;
 import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.hudi.common.util.collection.Pair;
@@ -44,8 +42,6 @@ import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIOException;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.generic.IndexedRecord;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hudi.internal.schema.InternalSchema;
@@ -57,7 +53,6 @@ import org.apache.log4j.Logger;
 import java.io.IOException;
 import java.util.ArrayDeque;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.Deque;
 import java.util.HashSet;
 import java.util.List;
@@ -66,6 +61,7 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.stream.Collectors;
 
+import static org.apache.hudi.common.util.TypeUtils.unsafeCast;
 import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME;
 import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.COMMAND_BLOCK;
 import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK;
@@ -91,13 +87,13 @@ public abstract class AbstractHoodieLogRecordReader {
   // Latest valid instant time
   // Log-Blocks belonging to inflight delta-instants are filtered-out using this high-watermark.
   private final String latestInstantTime;
-  private final HoodieTableMetaClient hoodieTableMetaClient;
+  protected final HoodieTableMetaClient hoodieTableMetaClient;
   // Merge strategy to use when combining records from log
   private final String payloadClassFQN;
   // preCombine field
-  private final String preCombineField;
+  protected final String preCombineField;
   // Stateless component for merging records
-  private final String mergeClassFQN;
+  protected final HoodieRecordMerger recordMerger;
   // simple key gen fields
   private Option<Pair<String, String>> simpleKeyGenFields = Option.empty();
   // Log File Paths
@@ -140,21 +136,23 @@ public abstract class AbstractHoodieLogRecordReader {
   private Option<String> partitionName;
   // Populate meta fields for the records
   private boolean populateMetaFields = true;
+  // Record type read from log block
+  protected final HoodieRecordType recordType;
 
   protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List<String> logFilePaths,
                                           Schema readerSchema,
                                           String latestInstantTime, boolean readBlocksLazily, boolean reverseReader,
                                           int bufferSize, Option<InstantRange> instantRange,
-                                          boolean withOperationField) {
+                                          boolean withOperationField, HoodieRecordMerger recordMerger) {
     this(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize,
-        instantRange, withOperationField, true, Option.empty(), InternalSchema.getEmptyInternalSchema());
+        instantRange, withOperationField, true, Option.empty(), InternalSchema.getEmptyInternalSchema(), recordMerger);
   }
 
   protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List<String> logFilePaths,
                                           Schema readerSchema, String latestInstantTime, boolean readBlocksLazily,
                                           boolean reverseReader, int bufferSize, Option<InstantRange> instantRange,
                                           boolean withOperationField, boolean forceFullScan,
-                                          Option<String> partitionName, InternalSchema internalSchema) {
+                                          Option<String> partitionName, InternalSchema internalSchema, HoodieRecordMerger recordMerger) {
     this.readerSchema = readerSchema;
     this.latestInstantTime = latestInstantTime;
     this.hoodieTableMetaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(basePath).build();
@@ -162,7 +160,7 @@ public abstract class AbstractHoodieLogRecordReader {
     HoodieTableConfig tableConfig = this.hoodieTableMetaClient.getTableConfig();
     this.payloadClassFQN = tableConfig.getPayloadClass();
     this.preCombineField = tableConfig.getPreCombineField();
-    this.mergeClassFQN = tableConfig.getMergeClass();
+    this.recordMerger = recordMerger;
     this.totalLogFiles.addAndGet(logFilePaths.size());
     this.logFilePaths = logFilePaths;
     this.reverseReader = reverseReader;
@@ -182,6 +180,7 @@ public abstract class AbstractHoodieLogRecordReader {
           Pair.of(tableConfig.getRecordKeyFieldProp(), tableConfig.getPartitionFieldProp()));
     }
     this.partitionName = partitionName;
+    this.recordType = recordMerger.getRecordType();
   }
 
   protected String getKeyField() {
@@ -380,27 +379,24 @@ public abstract class AbstractHoodieLogRecordReader {
    * handle it.
    */
   private void processDataBlock(HoodieDataBlock dataBlock, Option<KeySpec> keySpecOpt) throws Exception {
-    HoodieRecord.Mapper mapper = (rec) -> createHoodieRecord(rec, this.hoodieTableMetaClient.getTableConfig(),
-        this.payloadClassFQN, this.preCombineField, this.withOperationField, this.simpleKeyGenFields, this.partitionName);
-
-    try (ClosableIterator<HoodieRecord> recordIterator = getRecordsIterator(dataBlock, keySpecOpt, mapper)) {
+    try (ClosableIterator<HoodieRecord> recordIterator = getRecordsIterator(dataBlock, keySpecOpt, recordType)) {
       Option<Schema> schemaOption = getMergedSchema(dataBlock);
-      Schema finalReadSchema;
-      if (recordIterator instanceof RecordIterator) {
-        finalReadSchema = ((RecordIterator) recordIterator).getFinalReadSchema();
-      } else {
-        finalReadSchema = dataBlock.getSchema();
-      }
       while (recordIterator.hasNext()) {
         HoodieRecord currentRecord = recordIterator.next();
-        HoodieRecord record = schemaOption.isPresent()
-            ? currentRecord.rewriteRecordWithNewSchema(finalReadSchema, new Properties(), schemaOption.get(), Collections.emptyMap(), mapper) : currentRecord;
-        processNextRecord(record);
+        Schema schema = schemaOption.isPresent() ? schemaOption.get() : dataBlock.getSchema();
+        HoodieRecord record = schemaOption.isPresent() ? currentRecord.rewriteRecordWithNewSchema(dataBlock.getSchema(), new Properties(), schemaOption.get()) : currentRecord;
+        HoodieRecord completedRecord = record.wrapIntoHoodieRecordPayloadWithParams(schema, hoodieTableMetaClient.getTableConfig().getProps(), this.simpleKeyGenFields,
+            this.withOperationField, this.partitionName, getPopulateMetaFields());
+        processNextRecord(completedRecord);
         totalLogRecords.incrementAndGet();
       }
     }
   }
 
+  protected boolean getPopulateMetaFields() {
+    return this.populateMetaFields;
+  }
+
   /**
    * Get final Read Schema for support evolution.
    * step1: find the fileSchema for current dataBlock.
@@ -423,38 +419,12 @@ public abstract class AbstractHoodieLogRecordReader {
     return result;
   }
 
-  /**
-   * Create @{@link HoodieRecord} from the @{@link IndexedRecord}.
-   *
-   * @param rec                - IndexedRecord to create the HoodieRecord from
-   * @param hoodieTableConfig  - Table config
-   * @param payloadClassFQN    - Payload class fully qualified name
-   * @param preCombineField    - PreCombine field
-   * @param withOperationField - Whether operation field is enabled
-   * @param simpleKeyGenFields - Key generator fields when populate meta fields is tuened off
-   * @param partitionName      - Partition name
-   * @return HoodieRecord created from the IndexedRecord
-   */
-  protected HoodieAvroRecord<?> createHoodieRecord(final IndexedRecord rec, final HoodieTableConfig hoodieTableConfig,
-                                               final String payloadClassFQN, final String preCombineField,
-                                               final boolean withOperationField,
-                                               final Option<Pair<String, String>> simpleKeyGenFields,
-                                               final Option<String> partitionName) {
-    if (this.populateMetaFields) {
-      return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, payloadClassFQN,
-          preCombineField, withOperationField);
-    } else {
-      return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, payloadClassFQN,
-          preCombineField, simpleKeyGenFields.get(), withOperationField, partitionName);
-    }
-  }
-
   /**
    * Process next record.
    *
    * @param hoodieRecord Hoodie Record to process
    */
-  protected abstract void processNextRecord(HoodieRecord<? extends HoodieRecordPayload> hoodieRecord) throws Exception;
+  protected abstract <T> void processNextRecord(HoodieRecord<T> hoodieRecord) throws Exception;
 
   /**
    * Process next deleted record.
@@ -496,13 +466,15 @@ public abstract class AbstractHoodieLogRecordReader {
     progress = (numLogFilesSeen - 1) / logFilePaths.size();
   }
 
-  private ClosableIterator<HoodieRecord> getRecordsIterator(HoodieDataBlock dataBlock, Option<KeySpec> keySpecOpt, HoodieRecord.Mapper mapper) throws IOException {
+  private ClosableIterator<HoodieRecord> getRecordsIterator(HoodieDataBlock dataBlock, Option<KeySpec> keySpecOpt, HoodieRecordType type) throws IOException {
+    ClosableIterator<HoodieRecord> iter;
     if (keySpecOpt.isPresent()) {
       KeySpec keySpec = keySpecOpt.get();
-      return dataBlock.getRecordIterator(keySpec.keys, keySpec.fullKey, mapper);
+      iter =  unsafeCast(dataBlock.getRecordIterator(keySpec.keys, keySpec.fullKey, type));
+    } else {
+      iter = unsafeCast(dataBlock.getRecordIterator(type));
     }
-
-    return dataBlock.getRecordIterator(mapper);
+    return iter;
   }
 
   /**
@@ -528,10 +500,6 @@ public abstract class AbstractHoodieLogRecordReader {
     return payloadClassFQN;
   }
 
-  protected String getMergeClassFQN() {
-    return mergeClassFQN;
-  }
-
   public Option<String> getPartitionName() {
     return partitionName;
   }
@@ -591,6 +559,10 @@ public abstract class AbstractHoodieLogRecordReader {
       throw new UnsupportedOperationException();
     }
 
+    public Builder withRecordMerger(HoodieRecordMerger recordMerger) {
+      throw new UnsupportedOperationException();
+    }
+
     public abstract AbstractHoodieLogRecordReader build();
   }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java
index dfc3c14b5b..f39cb1fc02 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java
@@ -20,27 +20,25 @@ package org.apache.hudi.common.table.log;
 
 import org.apache.hudi.common.config.HoodieCommonConfig;
 import org.apache.hudi.common.model.DeleteRecord;
-import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieEmptyRecord;
 import org.apache.hudi.common.model.HoodieKey;
-import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecordMerger;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieMerge;
-import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.util.CollectionUtils;
 import org.apache.hudi.common.util.DefaultSizeEstimator;
 import org.apache.hudi.common.util.HoodieRecordSizeEstimator;
-import org.apache.hudi.common.util.HoodieRecordUtils;
 import org.apache.hudi.common.util.HoodieTimer;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ReflectionUtils;
 import org.apache.hudi.common.util.SpillableMapUtils;
+import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.hudi.common.util.collection.ExternalSpillableMap;
 import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.internal.schema.InternalSchema;
 
 import org.apache.avro.Schema;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hudi.internal.schema.InternalSchema;
-
 import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -80,8 +78,6 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
   // Stores the total time taken to perform reading and merging of log blocks
   private long totalTimeTakenToReadAndMergeBlocks;
 
-  private final HoodieMerge merge;
-
   @SuppressWarnings("unchecked")
   protected HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths, Schema readerSchema,
                                          String latestInstantTime, Long maxMemorySizeInBytes, boolean readBlocksLazily,
@@ -90,16 +86,14 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
                                          ExternalSpillableMap.DiskMapType diskMapType,
                                          boolean isBitCaskDiskMapCompressionEnabled,
                                          boolean withOperationField, boolean forceFullScan,
-                                         Option<String> partitionName, InternalSchema internalSchema) {
+                                         Option<String> partitionName, InternalSchema internalSchema, HoodieRecordMerger recordMerger) {
     super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize,
-        instantRange, withOperationField,
-        forceFullScan, partitionName, internalSchema);
+        instantRange, withOperationField, forceFullScan, partitionName, internalSchema, recordMerger);
     try {
       // Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize
       this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator(),
           new HoodieRecordSizeEstimator(readerSchema), diskMapType, isBitCaskDiskMapCompressionEnabled);
       this.maxMemorySizeInBytes = maxMemorySizeInBytes;
-      this.merge = HoodieRecordUtils.loadMerge(getMergeClassFQN());
     } catch (IOException e) {
       throw new HoodieIOException("IOException when creating ExternalSpillableMap at " + spillableMapBasePath, e);
     }
@@ -147,19 +141,19 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
   }
 
   @Override
-  protected void processNextRecord(HoodieRecord hoodieRecord) throws IOException {
+  protected <T> void processNextRecord(HoodieRecord<T> hoodieRecord) throws IOException {
     String key = hoodieRecord.getRecordKey();
     if (records.containsKey(key)) {
       // Merge and store the merged record. The HoodieRecordPayload implementation is free to decide what should be
       // done when a DELETE (empty payload) is encountered before or after an insert/update.
 
-      HoodieRecord<? extends HoodieRecordPayload> oldRecord = records.get(key);
-      HoodieRecordPayload oldValue = oldRecord.getData();
-      HoodieRecordPayload combinedValue = (HoodieRecordPayload) merge.preCombine(oldRecord, hoodieRecord).getData();
+      HoodieRecord<T> oldRecord = records.get(key);
+      T oldValue = oldRecord.getData();
+      T combinedValue = ((HoodieRecord<T>) recordMerger.merge(oldRecord, hoodieRecord, readerSchema, this.hoodieTableMetaClient.getTableConfig().getProps()).get()).getData();
       // If combinedValue is oldValue, no need rePut oldRecord
       if (combinedValue != oldValue) {
-        HoodieOperation operation = hoodieRecord.getOperation();
-        records.put(key, new HoodieAvroRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue, operation));
+        hoodieRecord.setData(combinedValue);
+        records.put(key, hoodieRecord);
       }
     } else {
       // Put the record as is
@@ -170,13 +164,14 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
   @Override
   protected void processNextDeletedRecord(DeleteRecord deleteRecord) {
     String key = deleteRecord.getRecordKey();
-    HoodieRecord<? extends HoodieRecordPayload> oldRecord = records.get(key);
+    HoodieRecord oldRecord = records.get(key);
     if (oldRecord != null) {
       // Merge and store the merged record. The ordering val is taken to decide whether the same key record
       // should be deleted or be kept. The old record is kept only if the DELETE record has smaller ordering val.
       // For same ordering values, uses the natural order(arrival time semantics).
 
-      Comparable curOrderingVal = oldRecord.getOrderingValue();
+      Comparable curOrderingVal = oldRecord.getOrderingValue(
+          this.hoodieTableMetaClient.getTableConfig().getProps());
       Comparable deleteOrderingVal = deleteRecord.getOrderingValue();
       // Checks the ordering value does not equal to 0
       // because we use 0 as the default value which means natural order
@@ -189,8 +184,13 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
       }
     }
     // Put the DELETE record
-    records.put(key, SpillableMapUtils.generateEmptyPayload(key,
-        deleteRecord.getPartitionPath(), deleteRecord.getOrderingValue(), getPayloadClassFQN()));
+    if (recordType == HoodieRecordType.AVRO) {
+      records.put(key, SpillableMapUtils.generateEmptyPayload(key,
+          deleteRecord.getPartitionPath(), deleteRecord.getOrderingValue(), getPayloadClassFQN()));
+    } else {
+      HoodieEmptyRecord record = new HoodieEmptyRecord<>(new HoodieKey(key, deleteRecord.getPartitionPath()), null, deleteRecord.getOrderingValue(), recordType);
+      records.put(key, record);
+    }
   }
 
   public long getTotalTimeTakenToReadAndMergeBlocks() {
@@ -226,6 +226,7 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
     protected String partitionName;
     // operation field default false
     private boolean withOperationField = false;
+    private HoodieRecordMerger recordMerger;
 
     @Override
     public Builder withFileSystem(FileSystem fs) {
@@ -317,16 +318,24 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
       return this;
     }
 
+    @Override
+    public Builder withRecordMerger(HoodieRecordMerger recordMerger) {
+      this.recordMerger = recordMerger;
+      return this;
+    }
+
     @Override
     public HoodieMergedLogRecordScanner build() {
       if (this.partitionName == null && CollectionUtils.nonEmpty(this.logFilePaths)) {
         this.partitionName = getRelativePartitionPath(new Path(basePath), new Path(this.logFilePaths.get(0)).getParent());
       }
+      ValidationUtils.checkArgument(recordMerger != null);
+
       return new HoodieMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema,
           latestInstantTime, maxMemorySizeInBytes, readBlocksLazily, reverseReader,
           bufferSize, spillableMapBasePath, instantRange,
           diskMapType, isBitCaskDiskMapCompressionEnabled, withOperationField, true,
-          Option.ofNullable(partitionName), internalSchema);
+          Option.ofNullable(partitionName), internalSchema, recordMerger);
     }
   }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java
index 8ea34d6f2f..c29f231921 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieUnMergedLogRecordScanner.java
@@ -20,8 +20,9 @@ package org.apache.hudi.common.table.log;
 
 import org.apache.hudi.common.model.DeleteRecord;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.model.HoodieRecordMerger;
 import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
 
 import org.apache.avro.Schema;
 import org.apache.hadoop.fs.FileSystem;
@@ -37,8 +38,8 @@ public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordReade
 
   private HoodieUnMergedLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths, Schema readerSchema,
                                          String latestInstantTime, boolean readBlocksLazily, boolean reverseReader, int bufferSize,
-                                         LogRecordScannerCallback callback, Option<InstantRange> instantRange) {
-    super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange, false);
+                                         LogRecordScannerCallback callback, Option<InstantRange> instantRange, HoodieRecordMerger recordMerger) {
+    super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize, instantRange, false, recordMerger);
     this.callback = callback;
   }
 
@@ -50,7 +51,7 @@ public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordReade
   }
 
   @Override
-  protected void processNextRecord(HoodieRecord<? extends HoodieRecordPayload> hoodieRecord) throws Exception {
+  protected <T> void processNextRecord(HoodieRecord<T> hoodieRecord) throws Exception {
     // Just call callback without merging
     callback.apply(hoodieRecord);
   }
@@ -66,7 +67,7 @@ public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordReade
   @FunctionalInterface
   public interface LogRecordScannerCallback {
 
-    void apply(HoodieRecord<? extends HoodieRecordPayload> record) throws Exception;
+    void apply(HoodieRecord<?> record) throws Exception;
   }
 
   /**
@@ -84,6 +85,7 @@ public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordReade
     private Option<InstantRange> instantRange = Option.empty();
     // specific configurations
     private LogRecordScannerCallback callback;
+    private HoodieRecordMerger recordMerger;
 
     public Builder withFileSystem(FileSystem fs) {
       this.fs = fs;
@@ -135,10 +137,18 @@ public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordReade
       return this;
     }
 
+    @Override
+    public Builder withRecordMerger(HoodieRecordMerger recordMerger) {
+      this.recordMerger = recordMerger;
+      return this;
+    }
+
     @Override
     public HoodieUnMergedLogRecordScanner build() {
+      ValidationUtils.checkArgument(recordMerger != null);
+
       return new HoodieUnMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema,
-          latestInstantTime, readBlocksLazily, reverseReader, bufferSize, callback, instantRange);
+          latestInstantTime, readBlocksLazily, reverseReader, bufferSize, callback, instantRange, recordMerger);
     }
   }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java
index 2712d000ac..9c6135dd28 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java
@@ -18,6 +18,16 @@
 
 package org.apache.hudi.common.table.log.block;
 
+import org.apache.hudi.common.fs.SizeAwareDataInputStream;
+import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
+import org.apache.hudi.common.util.ClosableIterator;
+import org.apache.hudi.common.util.MappingIterator;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.internal.schema.InternalSchema;
+
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericDatumReader;
 import org.apache.avro.generic.GenericDatumWriter;
@@ -29,13 +39,6 @@ import org.apache.avro.io.DecoderFactory;
 import org.apache.avro.io.Encoder;
 import org.apache.avro.io.EncoderFactory;
 import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hudi.common.fs.SizeAwareDataInputStream;
-import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
-import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.util.ClosableIterator;
-import org.apache.hudi.common.util.Option;
-import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.internal.schema.InternalSchema;
 
 import javax.annotation.Nonnull;
 
@@ -54,10 +57,10 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
-import java.util.stream.Collectors;
 import java.util.zip.DeflaterOutputStream;
 import java.util.zip.InflaterInputStream;
 
+import static org.apache.hudi.common.util.ValidationUtils.checkArgument;
 import static org.apache.hudi.common.util.ValidationUtils.checkState;
 
 /**
@@ -115,13 +118,14 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
     output.writeInt(records.size());
 
     // 3. Write the records
-    for (HoodieRecord s : records) {
+    for (HoodieRecord<?> s : records) {
       ByteArrayOutputStream temp = new ByteArrayOutputStream();
       BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(temp, encoderCache.get());
       encoderCache.set(encoder);
       try {
         // Encode the record into bytes
-        IndexedRecord data = (IndexedRecord) s.toIndexedRecord(schema, new Properties()).get();
+        // Spark Record not support write avro log
+        IndexedRecord data = s.toIndexedRecord(schema, new Properties()).get().getData();
         writer.write(data, encoder);
         encoder.flush();
 
@@ -142,25 +146,25 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
 
   // TODO (na) - Break down content into smaller chunks of byte [] to be GC as they are used
   @Override
-  protected ClosableIterator<HoodieRecord> deserializeRecords(byte[] content, HoodieRecord.Mapper mapper) throws IOException {
+  protected <T> ClosableIterator<HoodieRecord<T>> deserializeRecords(byte[] content, HoodieRecordType type) throws IOException {
     checkState(this.readerSchema != null, "Reader's schema has to be non-null");
-    return RecordIterator.getInstance(this, content, internalSchema, mapper);
+    checkArgument(type != HoodieRecordType.SPARK, "Not support read avro to spark record");
+    // TODO AvroSparkReader need
+    RecordIterator iterator = RecordIterator.getInstance(this, content, internalSchema);
+    return new MappingIterator<>(iterator, data -> (HoodieRecord<T>) new HoodieAvroIndexedRecord(data));
   }
 
-  public static class RecordIterator implements ClosableIterator<HoodieRecord> {
+  private static class RecordIterator implements ClosableIterator<IndexedRecord> {
     private byte[] content;
     private final SizeAwareDataInputStream dis;
     private final GenericDatumReader<IndexedRecord> reader;
     private final ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
-    private final HoodieRecord.Mapper mapper;
-    private final Schema finalReadSchema;
 
     private int totalRecords = 0;
     private int readRecords = 0;
 
-    private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content, InternalSchema internalSchema, HoodieRecord.Mapper mapper) throws IOException {
+    private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content, InternalSchema internalSchema) throws IOException {
       this.content = content;
-      this.mapper = mapper;
 
       this.dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(this.content)));
 
@@ -177,7 +181,6 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
         finalReadSchema = writerSchema;
       }
 
-      this.finalReadSchema = finalReadSchema;
       this.reader = new GenericDatumReader<>(writerSchema, finalReadSchema);
 
       if (logBlockVersion.hasRecordCount()) {
@@ -185,14 +188,10 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
       }
     }
 
-    public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content, InternalSchema internalSchema, HoodieRecord.Mapper mapper) throws IOException {
+    public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content, InternalSchema internalSchema) throws IOException {
       // Get schema from the header
       Schema writerSchema = new Schema.Parser().parse(dataBlock.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
-      return new RecordIterator(dataBlock.readerSchema, writerSchema, content, internalSchema, mapper);
-    }
-
-    public Schema getFinalReadSchema() {
-      return finalReadSchema;
+      return new RecordIterator(dataBlock.readerSchema, writerSchema, content, internalSchema);
     }
 
     @Override
@@ -212,7 +211,7 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
     }
 
     @Override
-    public HoodieRecord next() {
+    public IndexedRecord next() {
       try {
         int recordLength = this.dis.readInt();
         BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(this.content, this.dis.getNumberOfBytesRead(),
@@ -221,7 +220,7 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
         IndexedRecord record = this.reader.read(null, decoder);
         this.dis.skipBytes(recordLength);
         this.readRecords++;
-        return mapper.apply(record);
+        return record;
       } catch (IOException e) {
         throw new HoodieIOException("Unable to convert bytes to record.", e);
       }
@@ -274,18 +273,18 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
     GenericDatumReader<IndexedRecord> reader = new GenericDatumReader<>(writerSchema, readerSchema);
     // 2. Get the total records
     int totalRecords = dis.readInt();
-    List<IndexedRecord> records = new ArrayList<>(totalRecords);
+    List<HoodieRecord> records = new ArrayList<>(totalRecords);
 
     // 3. Read the content
     for (int i = 0; i < totalRecords; i++) {
       int recordLength = dis.readInt();
       Decoder decoder = DecoderFactory.get().binaryDecoder(content, dis.getNumberOfBytesRead(), recordLength, null);
       IndexedRecord record = reader.read(null, decoder);
-      records.add(record);
+      records.add(new HoodieAvroIndexedRecord(record));
       dis.skipBytes(recordLength);
     }
     dis.close();
-    return new HoodieAvroDataBlock(records.stream().map(HoodieAvroIndexedRecord::new).collect(Collectors.toList()), readerSchema);
+    return new HoodieAvroDataBlock(records, readerSchema);
   }
 
   private static byte[] compress(String text) {
@@ -327,8 +326,8 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
     output.writeInt(schemaContent.length);
     output.write(schemaContent);
 
-    List<HoodieRecord> records = new ArrayList<>();
-    try (ClosableIterator<HoodieRecord> recordItr = getRecordIterator(HoodieAvroIndexedRecord::new)) {
+    List<HoodieRecord<?>> records = new ArrayList<>();
+    try (ClosableIterator<HoodieRecord<Object>> recordItr = getRecordIterator(HoodieRecordType.AVRO)) {
       recordItr.forEachRemaining(records::add);
     }
 
@@ -336,9 +335,9 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
     output.writeInt(records.size());
 
     // 3. Write the records
-    Iterator<HoodieRecord> itr = records.iterator();
+    Iterator<HoodieRecord<?>> itr = records.iterator();
     while (itr.hasNext()) {
-      IndexedRecord s = (IndexedRecord)itr.next().getData();
+      IndexedRecord s = itr.next().toIndexedRecord(schema, new Properties()).get().getData();
       ByteArrayOutputStream temp = new ByteArrayOutputStream();
       Encoder encoder = EncoderFactory.get().binaryEncoder(temp, null);
       try {
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java
index d88fff750f..ab669730f6 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieDataBlock.java
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.common.table.log.block;
 
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.util.ClosableIterator;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.exception.HoodieIOException;
@@ -35,6 +36,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.function.Function;
 
+import static org.apache.hudi.common.util.TypeUtils.unsafeCast;
 import static org.apache.hudi.common.util.ValidationUtils.checkState;
 
 /**
@@ -138,13 +140,14 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
   /**
    * Returns all the records iterator contained w/in this block.
    */
-  public final ClosableIterator<HoodieRecord> getRecordIterator(HoodieRecord.Mapper mapper) {
+  public final <T> ClosableIterator<HoodieRecord<T>> getRecordIterator(HoodieRecordType type) {
     if (records.isPresent()) {
-      return list2Iterator(records.get());
+      // TODO need convert record type
+      return list2Iterator(unsafeCast(records.get()));
     }
     try {
       // in case records are absent, read content lazily and then convert to IndexedRecords
-      return readRecordsFromBlockPayload(mapper);
+      return readRecordsFromBlockPayload(type);
     } catch (IOException io) {
       throw new HoodieIOException("Unable to convert content bytes to records", io);
     }
@@ -162,38 +165,38 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
    * @return List of IndexedRecords for the keys of interest.
    * @throws IOException in case of failures encountered when reading/parsing records
    */
-  public final ClosableIterator<HoodieRecord> getRecordIterator(List<String> keys, boolean fullKey, HoodieRecord.Mapper mapper) throws IOException {
+  public final <T> ClosableIterator<HoodieRecord<T>> getRecordIterator(List<String> keys, boolean fullKey, HoodieRecordType type) throws IOException {
     boolean fullScan = keys.isEmpty();
     if (enablePointLookups && !fullScan) {
-      return lookupRecords(keys, fullKey, mapper);
+      return lookupRecords(keys, fullKey);
     }
 
     // Otherwise, we fetch all the records and filter out all the records, but the
     // ones requested
-    ClosableIterator<HoodieRecord> allRecords = getRecordIterator(mapper);
+    ClosableIterator<HoodieRecord<T>> allRecords = getRecordIterator(type);
     if (fullScan) {
       return allRecords;
     }
 
     HashSet<String> keySet = new HashSet<>(keys);
-    return FilteringIterator.getInstance(allRecords, keySet, fullKey, this::getRecordKey);
+    return FilteringIterator.getInstance(allRecords, keySet, fullKey, record -> getRecordKey(record));
   }
 
-  protected ClosableIterator<HoodieRecord> readRecordsFromBlockPayload(HoodieRecord.Mapper mapper) throws IOException {
+  protected <T> ClosableIterator<HoodieRecord<T>> readRecordsFromBlockPayload(HoodieRecordType type) throws IOException {
     if (readBlockLazily && !getContent().isPresent()) {
       // read log block contents from disk
       inflate();
     }
 
     try {
-      return deserializeRecords(getContent().get(), mapper);
+      return deserializeRecords(getContent().get(), type);
     } finally {
       // Free up content to be GC'd by deflating the block
       deflate();
     }
   }
 
-  protected ClosableIterator<HoodieRecord> lookupRecords(List<String> keys, boolean fullKey, HoodieRecord.Mapper mapper) throws IOException {
+  protected <T> ClosableIterator<HoodieRecord<T>> lookupRecords(List<String> keys, boolean fullKey) throws IOException {
     throw new UnsupportedOperationException(
         String.format("Point lookups are not supported by this Data block type (%s)", getBlockType())
     );
@@ -201,7 +204,7 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
 
   protected abstract byte[] serializeRecords(List<HoodieRecord> records) throws IOException;
 
-  protected abstract ClosableIterator<HoodieRecord> deserializeRecords(byte[] content, HoodieRecord.Mapper mapper) throws IOException;
+  protected abstract <T> ClosableIterator<HoodieRecord<T>> deserializeRecords(byte[] content, HoodieRecordType type) throws IOException;
 
   public abstract HoodieLogBlockType getBlockType();
 
@@ -244,29 +247,29 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
    * A {@link ClosableIterator} that supports filtering strategy with given keys.
    * User should supply the key extraction function for fetching string format keys.
    */
-  private static class FilteringIterator implements ClosableIterator<HoodieRecord> {
-    private final ClosableIterator<HoodieRecord> nested; // nested iterator
+  private static class FilteringIterator<T> implements ClosableIterator<HoodieRecord<T>> {
+    private final ClosableIterator<HoodieRecord<T>> nested; // nested iterator
 
     private final Set<String> keys; // the filtering keys
     private final boolean fullKey;
 
-    private final Function<HoodieRecord, Option<String>> keyExtract; // function to extract the key
+    private final Function<HoodieRecord<T>, Option<String>> keyExtract; // function to extract the key
 
-    private HoodieRecord next;
+    private HoodieRecord<T> next;
 
-    private FilteringIterator(ClosableIterator<HoodieRecord> nested, Set<String> keys, boolean fullKey, Function<HoodieRecord, Option<String>> keyExtract) {
+    private FilteringIterator(ClosableIterator<HoodieRecord<T>> nested, Set<String> keys, boolean fullKey, Function<HoodieRecord<T>, Option<String>> keyExtract) {
       this.nested = nested;
       this.keys = keys;
       this.fullKey = fullKey;
       this.keyExtract = keyExtract;
     }
 
-    public static FilteringIterator getInstance(
-        ClosableIterator<HoodieRecord> nested,
+    public static <T> FilteringIterator<T> getInstance(
+        ClosableIterator<HoodieRecord<T>> nested,
         Set<String> keys,
         boolean fullKey,
-        Function<HoodieRecord, Option<String>> keyExtract) {
-      return new FilteringIterator(nested, keys, fullKey, keyExtract);
+        Function<HoodieRecord<T>, Option<String>> keyExtract) {
+      return new FilteringIterator<>(nested, keys, fullKey, keyExtract);
     }
 
     @Override
@@ -292,7 +295,7 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
     }
 
     @Override
-    public HoodieRecord next() {
+    public HoodieRecord<T> next() {
       return this.next;
     }
   }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java
index ca2c46022c..37db0caa9c 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java
@@ -33,8 +33,14 @@ import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
 import org.apache.hudi.avro.HoodieAvroUtils;
 import org.apache.hudi.common.fs.inline.InLineFSUtils;
 import org.apache.hudi.common.fs.inline.InLineFileSystem;
+import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
+import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
+import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockContentLocation;
+import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
 import org.apache.hudi.common.util.ClosableIterator;
+import org.apache.hudi.common.util.MappingIterator;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.StringUtils;
 import org.apache.hudi.common.util.ValidationUtils;
@@ -56,6 +62,7 @@ import java.util.Properties;
 import java.util.TreeMap;
 
 import static org.apache.hudi.common.util.ValidationUtils.checkState;
+import static org.apache.hudi.common.util.TypeUtils.unsafeCast;
 
 /**
  * HoodieHFileDataBlock contains a list of records stored inside an HFile format. It is used with the HFile
@@ -125,7 +132,7 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
     Iterator<HoodieRecord> itr = records.iterator();
     int id = 0;
     while (itr.hasNext()) {
-      HoodieRecord record = itr.next();
+      HoodieRecord<?> record = itr.next();
       String recordKey;
       if (useIntegerKey) {
         recordKey = String.format("%" + keyWidth + "s", id++);
@@ -162,7 +169,7 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
   }
 
   @Override
-  protected ClosableIterator<HoodieRecord> deserializeRecords(byte[] content, HoodieRecord.Mapper mapper) throws IOException {
+  protected <T> ClosableIterator<HoodieRecord<T>> deserializeRecords(byte[] content, HoodieRecordType type) throws IOException {
     checkState(readerSchema != null, "Reader's schema has to be non-null");
 
     // Get schema from the header
@@ -170,28 +177,12 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
 
     // Read the content
     HoodieAvroHFileReader reader = new HoodieAvroHFileReader(null, pathForReader, content, Option.of(writerSchema));
-    Iterator<IndexedRecord> recordIterator = reader.getRecordIterator(readerSchema);
-    return new ClosableIterator<HoodieRecord>() {
-      @Override
-      public void close() {
-        reader.close();
-      }
-
-      @Override
-      public boolean hasNext() {
-        return recordIterator.hasNext();
-      }
-
-      @Override
-      public HoodieRecord next() {
-        return mapper.apply(recordIterator.next());
-      }
-    };
+    return unsafeCast(reader.getRecordIterator(readerSchema));
   }
 
   // TODO abstract this w/in HoodieDataBlock
   @Override
-  protected ClosableIterator<HoodieRecord> lookupRecords(List<String> keys, boolean fullKey, HoodieRecord.Mapper mapper) throws IOException {
+  protected <T> ClosableIterator<HoodieRecord<T>> lookupRecords(List<String> keys, boolean fullKey) throws IOException {
     HoodieLogBlockContentLocation blockContentLoc = getBlockContentLocation().get();
 
     // NOTE: It's important to extend Hadoop configuration here to make sure configuration
@@ -216,32 +207,16 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
 
     // Get writer's schema from the header
     final ClosableIterator<IndexedRecord> recordIterator =
-        fullKey ? reader.getRecordsByKeysIterator(sortedKeys, readerSchema) : reader.getRecordsByKeyPrefixIterator(sortedKeys, readerSchema);
+        fullKey ? reader.getIndexedRecordsByKeysIterator(sortedKeys, readerSchema) : reader.getIndexedRecordsByKeyPrefixIterator(sortedKeys, readerSchema);
 
-    return new ClosableIterator<HoodieRecord>() {
-      @Override
-      public boolean hasNext() {
-        return recordIterator.hasNext();
-      }
-
-      @Override
-      public HoodieRecord next() {
-        return mapper.apply(recordIterator.next());
-      }
-
-      @Override
-      public void close() {
-        recordIterator.close();
-        reader.close();
-      }
-    };
+    return new MappingIterator<>(recordIterator, data -> (HoodieRecord<T>) new HoodieAvroIndexedRecord((data)));
   }
 
-  private byte[] serializeRecord(HoodieRecord record, Schema schema) throws IOException {
+  private byte[] serializeRecord(HoodieRecord<?> record, Schema schema) throws IOException {
     Option<Schema.Field> keyField = getKeyField(schema);
     // Reset key value w/in the record to avoid duplicating the key w/in payload
     if (keyField.isPresent()) {
-      record.overrideMetadataFieldValue(schema, new Properties(), keyField.get().pos(), StringUtils.EMPTY_STRING);
+      record.updateValues(schema, new Properties(), Collections.singletonMap(keyField.get().name(), StringUtils.EMPTY_STRING));
     }
     return HoodieAvroUtils.recordToBytes(record, schema).get();
   }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java
index 98f5dcf3a0..79d3b3ec86 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java
@@ -18,30 +18,25 @@
 
 package org.apache.hudi.common.table.log.block;
 
-import org.apache.avro.Schema;
-import org.apache.avro.generic.IndexedRecord;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.Path;
-import org.apache.hudi.avro.HoodieAvroWriteSupport;
+import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.common.fs.inline.InLineFSUtils;
 import org.apache.hudi.common.fs.inline.InLineFileSystem;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.util.ClosableIterator;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.ParquetReaderIterator;
-import org.apache.hudi.io.storage.HoodieAvroFileReader.HoodieRecordTransformIterator;
-import org.apache.hudi.io.storage.HoodieParquetConfig;
-import org.apache.hudi.io.storage.HoodieParquetStreamWriter;
-import org.apache.parquet.avro.AvroParquetReader;
-import org.apache.parquet.avro.AvroReadSupport;
-import org.apache.parquet.avro.AvroSchemaConverter;
-import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.hudi.io.storage.HoodieFileReaderFactory;
+import org.apache.hudi.io.storage.HoodieFileWriter;
+import org.apache.hudi.io.storage.HoodieFileWriterFactory;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
 import org.apache.parquet.hadoop.ParquetWriter;
 import org.apache.parquet.hadoop.metadata.CompressionCodecName;
-import org.apache.parquet.hadoop.util.HadoopInputFile;
-import org.apache.parquet.io.InputFile;
 
 import javax.annotation.Nonnull;
 import java.io.ByteArrayOutputStream;
@@ -50,6 +45,14 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_BLOCK_SIZE;
+import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME;
+import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION;
+import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_MAX_FILE_SIZE;
+import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_PAGE_SIZE;
+import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+
 /**
  * HoodieParquetDataBlock contains a list of records serialized using Parquet.
  */
@@ -93,53 +96,47 @@ public class HoodieParquetDataBlock extends HoodieDataBlock {
     }
 
     Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
-
-    HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
-        new AvroSchemaConverter().convert(writerSchema), writerSchema, Option.empty());
-
-    HoodieParquetConfig<HoodieAvroWriteSupport> avroParquetConfig =
-        new HoodieParquetConfig<>(
-            writeSupport,
-            compressionCodecName.get(),
-            ParquetWriter.DEFAULT_BLOCK_SIZE,
-            ParquetWriter.DEFAULT_PAGE_SIZE,
-            1024 * 1024 * 1024,
-            new Configuration(),
-            Double.parseDouble(String.valueOf(0.1)));//HoodieStorageConfig.PARQUET_COMPRESSION_RATIO.defaultValue()));
-
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
-    try (FSDataOutputStream outputStream = new FSDataOutputStream(baos)) {
-      try (HoodieParquetStreamWriter parquetWriter = new HoodieParquetStreamWriter(outputStream, avroParquetConfig)) {
-        for (HoodieRecord record : records) {
+    try (FSDataOutputStream outputStream = new FSDataOutputStream(baos, null)) {
+      HoodieFileWriter parquetWriter = null;
+      HoodieConfig config = new HoodieConfig();
+      config.setValue(PARQUET_COMPRESSION_CODEC_NAME.key(), compressionCodecName.get().name());
+      config.setValue(PARQUET_BLOCK_SIZE.key(), String.valueOf(ParquetWriter.DEFAULT_BLOCK_SIZE));
+      config.setValue(PARQUET_PAGE_SIZE.key(), String.valueOf(ParquetWriter.DEFAULT_PAGE_SIZE));
+      config.setValue(PARQUET_MAX_FILE_SIZE.key(), String.valueOf(1024 * 1024 * 1024));
+      config.setValue(PARQUET_COMPRESSION_RATIO_FRACTION.key(), String.valueOf(0.1));
+      config.setValue(POPULATE_META_FIELDS.key(), "false");
+      HoodieRecordType recordType = records.iterator().next().getRecordType();
+      try {
+        parquetWriter = HoodieFileWriterFactory.getFileWriter(
+            HoodieFileFormat.PARQUET,
+            outputStream,
+            new Configuration(),
+            config,
+            writerSchema,
+            recordType);
+        for (HoodieRecord<?> record : records) {
           String recordKey = getRecordKey(record).orElse(null);
           parquetWriter.write(recordKey, record, writerSchema);
         }
         outputStream.flush();
+      } finally {
+        if (parquetWriter != null) {
+          parquetWriter.close();
+        }
       }
     }
 
     return baos.toByteArray();
   }
 
-  public static ClosableIterator<IndexedRecord> getProjectedParquetRecordsIterator(Configuration conf,
-                                                                                   Schema readerSchema,
-                                                                                   InputFile inputFile) throws IOException {
-    AvroReadSupport.setAvroReadSchema(conf, readerSchema);
-    AvroReadSupport.setRequestedProjection(conf, readerSchema);
-
-    ParquetReader<IndexedRecord> reader =
-        AvroParquetReader.<IndexedRecord>builder(inputFile).withConf(conf).build();
-    return new ParquetReaderIterator<>(reader);
-  }
-
   /**
    * NOTE: We're overriding the whole reading sequence to make sure we properly respect
    *       the requested Reader's schema and only fetch the columns that have been explicitly
    *       requested by the caller (providing projected Reader's schema)
    */
   @Override
-  protected ClosableIterator<HoodieRecord> readRecordsFromBlockPayload(HoodieRecord.Mapper mapper) throws IOException {
+  protected <T> ClosableIterator<HoodieRecord<T>> readRecordsFromBlockPayload(HoodieRecordType type) throws IOException {
     HoodieLogBlockContentLocation blockContentLoc = getBlockContentLocation().get();
 
     // NOTE: It's important to extend Hadoop configuration here to make sure configuration
@@ -153,15 +150,13 @@ public class HoodieParquetDataBlock extends HoodieDataBlock {
         blockContentLoc.getContentPositionInLogFile(),
         blockContentLoc.getBlockSize());
 
-    ClosableIterator<IndexedRecord> iterator = getProjectedParquetRecordsIterator(
-        inlineConf,
-        readerSchema,
-        HadoopInputFile.fromPath(inlineLogFilePath, inlineConf));
-    return new HoodieRecordTransformIterator(iterator, mapper);
+    ClosableIterator<HoodieRecord<T>> iterator = HoodieFileReaderFactory.getReaderFactory(type).getFileReader(inlineConf, inlineLogFilePath, PARQUET)
+        .getRecordIterator(readerSchema, readerSchema);
+    return iterator;
   }
 
   @Override
-  protected ClosableIterator<HoodieRecord> deserializeRecords(byte[] content, HoodieRecord.Mapper mapper) throws IOException {
+  protected <T> ClosableIterator<HoodieRecord<T>> deserializeRecords(byte[] content, HoodieRecordType type) throws IOException {
     throw new UnsupportedOperationException("Should not be invoked");
   }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java
index 22dd2b7ee3..3ed6e8d2d5 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java
@@ -22,10 +22,10 @@ import org.apache.hudi.avro.HoodieAvroUtils;
 import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
 import org.apache.hudi.avro.model.HoodieMergeArchiveFilePlan;
 import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
-import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieLogFile;
 import org.apache.hudi.common.model.HoodiePartitionMetadata;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.log.HoodieLogFormat;
 import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
@@ -259,12 +259,12 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
               HoodieAvroDataBlock avroBlock = (HoodieAvroDataBlock) block;
               // TODO If we can store additional metadata in datablock, we can skip parsing records
               // (such as startTime, endTime of records in the block)
-              try (ClosableIterator<HoodieRecord> itr = avroBlock.getRecordIterator(HoodieAvroIndexedRecord::new)) {
+              try (ClosableIterator<HoodieRecord<IndexedRecord>> itr = avroBlock.getRecordIterator(HoodieRecordType.AVRO)) {
                 StreamSupport.stream(Spliterators.spliteratorUnknownSize(itr, Spliterator.IMMUTABLE), true)
                     // Filter blocks in desired time window
-                    .map(r -> (GenericRecord) ((HoodieAvroIndexedRecord) r).toIndexedRecord().get())
+                    .map(r -> (GenericRecord) r.getData())
                     .filter(commitsFilter::apply)
-                    .map(r -> readCommit((GenericRecord) r, loadInstantDetails))
+                    .map(r -> readCommit(r, loadInstantDetails))
                     .filter(c -> filter == null || filter.isInRange(c))
                     .forEach(instantsInRange::add);
               }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ConfigUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ConfigUtils.java
new file mode 100644
index 0000000000..60aa6c43a9
--- /dev/null
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ConfigUtils.java
@@ -0,0 +1,55 @@
+/*
+ * 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.common.util;
+
+import org.apache.hudi.common.model.HoodiePayloadProps;
+import org.apache.hudi.common.table.HoodieTableConfig;
+
+import java.util.Properties;
+
+public class ConfigUtils {
+
+  /**
+   * Get ordering field.
+   */
+  public static String getOrderingField(Properties properties) {
+    String orderField = null;
+    if (properties.containsKey(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY)) {
+      orderField = properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY);
+    } else if (properties.containsKey("hoodie.datasource.write.precombine.field")) {
+      orderField = properties.getProperty("hoodie.datasource.write.precombine.field");
+    } else if (properties.containsKey(HoodieTableConfig.PRECOMBINE_FIELD.key())) {
+      orderField = properties.getProperty(HoodieTableConfig.PRECOMBINE_FIELD.key());
+    }
+    return orderField;
+  }
+
+  /**
+   * Get payload class.
+   */
+  public static String getPayloadClass(Properties properties) {
+    String payloadClass = null;
+    if (properties.containsKey(HoodieTableConfig.PAYLOAD_CLASS_NAME.key())) {
+      payloadClass = properties.getProperty(HoodieTableConfig.PAYLOAD_CLASS_NAME.key());
+    } else if (properties.containsKey("hoodie.datasource.write.payload.class")) {
+      payloadClass = properties.getProperty("hoodie.datasource.write.payload.class");
+    }
+    return payloadClass;
+  }
+}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordSizeEstimator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordSizeEstimator.java
index feebf6c482..32955abbf0 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordSizeEstimator.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordSizeEstimator.java
@@ -19,21 +19,18 @@
 package org.apache.hudi.common.util;
 
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.util.collection.ExternalSpillableMap;
 
 import org.apache.avro.Schema;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
-import java.io.Serializable;
-
 /**
  * Size Estimator for Hoodie record payload.
  * 
  * @param <T>
  */
-public class HoodieRecordSizeEstimator<T extends HoodieRecordPayload> implements SizeEstimator<HoodieRecord<T>>, Serializable {
+public class HoodieRecordSizeEstimator<T> implements SizeEstimator<HoodieRecord<T>> {
 
   private static final Logger LOG = LogManager.getLogger(HoodieRecordSizeEstimator.class);
 
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordUtils.java
index 075d117fe2..92b14ff340 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/HoodieRecordUtils.java
@@ -18,13 +18,20 @@
 
 package org.apache.hudi.common.util;
 
-import org.apache.hudi.common.model.HoodieMerge;
-import org.apache.hudi.common.model.HoodieRecordPayload;
-import org.apache.hudi.exception.HoodieException;
-
 import java.lang.reflect.InvocationTargetException;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
+import java.util.Objects;
+import org.apache.hudi.common.engine.EngineType;
+import org.apache.hudi.common.model.HoodieAvroRecordMerger;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
+import org.apache.hudi.common.model.HoodieRecordMerger;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.metadata.HoodieTableMetadata;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
 
 /**
  * A utility class for HoodieRecord.
@@ -32,37 +39,74 @@ import java.util.Map;
 public class HoodieRecordUtils {
 
   private static final Map<String, Object> INSTANCE_CACHE = new HashMap<>();
+  private static final Logger LOG = LogManager.getLogger(HoodieRecordUtils.class);
 
   /**
    * Instantiate a given class with a record merge.
    */
-  public static HoodieMerge loadMerge(String mergeClass) {
+  public static HoodieRecordMerger loadRecordMerger(String mergerClass) {
     try {
-      HoodieMerge merge = (HoodieMerge) INSTANCE_CACHE.get(mergeClass);
-      if (null == merge) {
-        synchronized (HoodieMerge.class) {
-          merge = (HoodieMerge) INSTANCE_CACHE.get(mergeClass);
-          if (null == merge) {
-            merge = (HoodieMerge)ReflectionUtils.loadClass(mergeClass, new Object[]{});
-            INSTANCE_CACHE.put(mergeClass, merge);
+      HoodieRecordMerger recordMerger = (HoodieRecordMerger) INSTANCE_CACHE.get(mergerClass);
+      if (null == recordMerger) {
+        synchronized (HoodieRecordMerger.class) {
+          recordMerger = (HoodieRecordMerger) INSTANCE_CACHE.get(mergerClass);
+          if (null == recordMerger) {
+            recordMerger = (HoodieRecordMerger) ReflectionUtils.loadClass(mergerClass,
+                new Object[]{});
+            INSTANCE_CACHE.put(mergerClass, recordMerger);
           }
         }
       }
-      return merge;
+      return recordMerger;
     } catch (HoodieException e) {
       throw new HoodieException("Unable to instantiate hoodie merge class ", e);
     }
   }
 
+  /**
+   * Instantiate a given class with a record merge.
+   */
+  public static HoodieRecordMerger generateRecordMerger(String basePath, EngineType engineType,
+      List<String> mergerClassList, String mergerStrategy) {
+    if (mergerClassList.isEmpty() || HoodieTableMetadata.isMetadataTable(basePath)) {
+      return HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName());
+    } else {
+      return mergerClassList.stream()
+          .map(clazz -> {
+            try {
+              return loadRecordMerger(clazz);
+            } catch (HoodieException e) {
+              LOG.warn(String.format("Unable to init %s", clazz), e);
+              return null;
+            }
+          })
+          .filter(Objects::nonNull)
+          .filter(merger -> merger.getMergingStrategy().equals(mergerStrategy))
+          .filter(merger -> recordTypeCompatibleEngine(merger.getRecordType(), engineType))
+          .findFirst()
+          .orElse(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()));
+    }
+  }
+
   /**
    * Instantiate a given class with an avro record payload.
    */
-  public static <T extends HoodieRecordPayload> T loadPayload(String recordPayloadClass, Object[] payloadArgs,
-                                                              Class<?>... constructorArgTypes) {
+  public static <T extends HoodieRecordPayload> T loadPayload(String recordPayloadClass,
+      Object[] payloadArgs,
+      Class<?>... constructorArgTypes) {
     try {
-      return (T) ReflectionUtils.getClass(recordPayloadClass).getConstructor(constructorArgTypes).newInstance(payloadArgs);
+      return (T) ReflectionUtils.getClass(recordPayloadClass).getConstructor(constructorArgTypes)
+          .newInstance(payloadArgs);
     } catch (InstantiationException | IllegalAccessException | InvocationTargetException | NoSuchMethodException e) {
       throw new HoodieException("Unable to instantiate payload class ", e);
     }
   }
+
+  public static boolean recordTypeCompatibleEngine(HoodieRecordType recordType, EngineType engineType) {
+    if (engineType == EngineType.SPARK && recordType == HoodieRecordType.SPARK) {
+      return true;
+    } else {
+      return false;
+    }
+  }
 }
\ No newline at end of file
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetReaderIterator.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetReaderIterator.java
index 03bd471b60..ca8c3ab428 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetReaderIterator.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ParquetReaderIterator.java
@@ -24,6 +24,7 @@ import org.apache.hudi.exception.HoodieException;
 import org.apache.parquet.hadoop.ParquetReader;
 
 import java.io.IOException;
+import java.util.function.Function;
 
 /**
  * This class wraps a parquet reader and provides an iterator based api to read from a parquet file. This is used in
@@ -35,17 +36,24 @@ public class ParquetReaderIterator<T> implements ClosableIterator<T> {
   private final ParquetReader<T> parquetReader;
   // Holds the next entry returned by the parquet reader
   private T next;
+  // For directly use InternalRow
+  private Function<T, T> mapper;
 
   public ParquetReaderIterator(ParquetReader<T> parquetReader) {
     this.parquetReader = parquetReader;
   }
 
+  public ParquetReaderIterator(ParquetReader<T> parquetReader, Function<T, T> mapper) {
+    this.parquetReader = parquetReader;
+    this.mapper = mapper;
+  }
+
   @Override
   public boolean hasNext() {
     try {
       // To handle when hasNext() is called multiple times for idempotency and/or the first time
       if (this.next == null) {
-        this.next = parquetReader.read();
+        this.next = read();
       }
       return this.next != null;
     } catch (Exception e) {
@@ -64,7 +72,7 @@ public class ParquetReaderIterator<T> implements ClosableIterator<T> {
         }
       }
       T retVal = this.next;
-      this.next = parquetReader.read();
+      this.next = read();
       return retVal;
     } catch (Exception e) {
       FileIOUtils.closeQuietly(parquetReader);
@@ -72,6 +80,15 @@ public class ParquetReaderIterator<T> implements ClosableIterator<T> {
     }
   }
 
+  private T read() throws IOException {
+    T record = parquetReader.read();
+    if (mapper == null || record == null) {
+      return record;
+    } else {
+      return mapper.apply(record);
+    }
+  }
+
   public void close() {
     try {
       parquetReader.close();
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/SerializationUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/SerializationUtils.java
index 9041db5144..6f3b3ef928 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/SerializationUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/SerializationUtils.java
@@ -19,8 +19,11 @@
 package org.apache.hudi.common.util;
 
 import com.esotericsoftware.kryo.Kryo;
+import com.esotericsoftware.kryo.Serializer;
 import com.esotericsoftware.kryo.io.Input;
 import com.esotericsoftware.kryo.io.Output;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
 import org.objenesis.strategy.StdInstantiatorStrategy;
 
 import java.io.ByteArrayOutputStream;
@@ -36,6 +39,14 @@ public class SerializationUtils {
   private static final ThreadLocal<KryoSerializerInstance> SERIALIZER_REF =
       ThreadLocal.withInitial(KryoSerializerInstance::new);
 
+  private static Pair<String, Serializer<?>> SERIALIZER_REGISTER = null;
+
+  public static void setOverallRegister(String className, Serializer<?> serializer) {
+    if (SERIALIZER_REGISTER == null) {
+      SERIALIZER_REGISTER = Pair.of(className, serializer);
+    }
+  }
+
   // Serialize
   // -----------------------------------------------------------------------
 
@@ -121,6 +132,13 @@ public class SerializationUtils {
       // Handle cases where we may have an odd classloader setup like with libjars
       // for hadoop
       kryo.setClassLoader(Thread.currentThread().getContextClassLoader());
+      if (SERIALIZER_REGISTER != null) {
+        try {
+          kryo.register(Class.forName(SERIALIZER_REGISTER.getLeft()), SERIALIZER_REGISTER.getRight());
+        } catch (ClassNotFoundException e) {
+          throw new HoodieException(e);
+        }
+      }
       return kryo;
     }
 
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java
index d2d91bbfb6..d8cf85cfd4 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/SpillableMapUtils.java
@@ -106,13 +106,13 @@ public class SpillableMapUtils {
   /**
    * Utility method to convert bytes to HoodieRecord using schema and payload class.
    */
-  public static <R> R convertToHoodieRecordPayload(GenericRecord rec, String payloadClazz, String preCombineField, boolean withOperationField) {
+  public static <R> HoodieRecord<R> convertToHoodieRecordPayload(GenericRecord rec, String payloadClazz, String preCombineField, boolean withOperationField) {
     return convertToHoodieRecordPayload(rec, payloadClazz, preCombineField,
         Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
         withOperationField, Option.empty());
   }
 
-  public static <R> R convertToHoodieRecordPayload(GenericRecord record, String payloadClazz,
+  public static <R> HoodieRecord<R> convertToHoodieRecordPayload(GenericRecord record, String payloadClazz,
                                                    String preCombineField,
                                                    boolean withOperationField,
                                                    Option<String> partitionName) {
@@ -124,7 +124,7 @@ public class SpillableMapUtils {
   /**
    * Utility method to convert bytes to HoodieRecord using schema and payload class.
    */
-  public static <R> R convertToHoodieRecordPayload(GenericRecord record, String payloadClazz,
+  public static <R> HoodieRecord<R> convertToHoodieRecordPayload(GenericRecord record, String payloadClazz,
                                                    String preCombineField,
                                                    Pair<String, String> recordKeyPartitionPathFieldPair,
                                                    boolean withOperationField,
@@ -140,7 +140,7 @@ public class SpillableMapUtils {
         HoodieRecordUtils.loadPayload(payloadClazz, new Object[]{record, preCombineVal}, GenericRecord.class,
             Comparable.class), operation);
 
-    return (R) hoodieRecord;
+    return (HoodieRecord<R>) hoodieRecord;
   }
 
   /**
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java
index a4f2c62437..9499954911 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java
@@ -33,6 +33,8 @@ public class StringUtils {
 
   public static final String EMPTY_STRING = "";
 
+  public static final String DEFAULT_MERGER_STRATEGY_UUID = "eeb8d96f-b1e4-49fd-bbf8-28ac514178e5";
+
   /**
    * <p>
    * Joins the elements of the provided array into a single String containing the provided list of elements.
diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReader.java
index f454fa28f1..c2ddfa319f 100644
--- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReader.java
+++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReader.java
@@ -18,85 +18,71 @@
 
 package org.apache.hudi.io.storage;
 
-import org.apache.avro.Schema;
-import org.apache.avro.generic.IndexedRecord;
-
+import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecord.Mapper;
 import org.apache.hudi.common.util.ClosableIterator;
 import org.apache.hudi.common.util.MappingIterator;
 import org.apache.hudi.common.util.Option;
 
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.util.List;
 
-public interface HoodieAvroFileReader extends HoodieFileReader, AutoCloseable {
+import static org.apache.hudi.common.util.TypeUtils.unsafeCast;
+
+public interface HoodieAvroFileReader extends HoodieFileReader<IndexedRecord>, AutoCloseable {
+
+  ClosableIterator<IndexedRecord> getIndexedRecordIterator(Schema readerSchema) throws IOException;
 
-  ClosableIterator<IndexedRecord> getRecordIterator(Schema readerSchema) throws IOException;
+  ClosableIterator<IndexedRecord> getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException;
 
-  default Option<IndexedRecord> getRecordByKey(String key, Schema readerSchema) throws IOException {
+  default Option<IndexedRecord> getIndexedRecordByKey(String key, Schema readerSchema) throws IOException {
     throw new UnsupportedOperationException();
   }
 
-  default ClosableIterator<IndexedRecord> getRecordsByKeysIterator(List<String> keys, Schema schema) throws IOException {
+  default ClosableIterator<IndexedRecord> getIndexedRecordsByKeysIterator(List<String> keys, Schema schema) throws IOException {
     throw new UnsupportedOperationException();
   }
 
-  default ClosableIterator<IndexedRecord> getRecordsByKeysIterator(List<String> keys) throws IOException {
-    return getRecordsByKeysIterator(keys, getSchema());
+  default ClosableIterator<IndexedRecord> getIndexedRecordsByKeysIterator(List<String> keys) throws IOException {
+    return getIndexedRecordsByKeysIterator(keys, getSchema());
   }
 
-  default ClosableIterator<IndexedRecord> getRecordsByKeyPrefixIterator(List<String> keyPrefixes, Schema schema) throws IOException {
+  default ClosableIterator<IndexedRecord> getIndexedRecordsByKeyPrefixIterator(List<String> keyPrefixes, Schema schema) throws IOException {
     throw new UnsupportedEncodingException();
   }
 
-  default ClosableIterator<IndexedRecord> getRecordsByKeyPrefixIterator(List<String> keyPrefixes) throws IOException {
-    return getRecordsByKeyPrefixIterator(keyPrefixes, getSchema());
+  default ClosableIterator<IndexedRecord> getIndexedRecordsByKeyPrefixIterator(List<String> keyPrefixes) throws IOException {
+    return getIndexedRecordsByKeyPrefixIterator(keyPrefixes, getSchema());
   }
 
-  default ClosableIterator<HoodieRecord> getRecordsByKeysIterator(List<String> keys, Schema schema, HoodieRecord.Mapper mapper) throws IOException {
-    ClosableIterator<IndexedRecord> iterator = getRecordsByKeysIterator(keys, schema);
-    return new HoodieRecordTransformIterator(iterator, mapper);
+  default ClosableIterator<HoodieRecord<IndexedRecord>> getRecordsByKeysIterator(List<String> keys, Schema schema) throws IOException {
+    ClosableIterator<IndexedRecord> iterator = getIndexedRecordsByKeysIterator(keys, schema);
+    return new MappingIterator<>(iterator, data -> unsafeCast(new HoodieAvroIndexedRecord(data)));
   }
 
-  default ClosableIterator<HoodieRecord> getRecordsByKeyPrefixIterator(List<String> keyPrefixes, Schema schema, HoodieRecord.Mapper mapper) throws IOException {
-    ClosableIterator<IndexedRecord> iterator = getRecordsByKeyPrefixIterator(keyPrefixes, schema);
-    return new HoodieRecordTransformIterator(iterator, mapper);
+  default ClosableIterator<HoodieRecord<IndexedRecord>> getRecordsByKeyPrefixIterator(List<String> keyPrefixes, Schema schema) throws IOException {
+    ClosableIterator<IndexedRecord> iterator = getIndexedRecordsByKeyPrefixIterator(keyPrefixes, schema);
+    return new MappingIterator<>(iterator, data -> unsafeCast(new HoodieAvroIndexedRecord(data)));
   }
 
   @Override
-  default ClosableIterator<HoodieRecord> getRecordIterator(Schema schema, HoodieRecord.Mapper mapper) throws IOException {
-    return new MappingIterator<>(getRecordIterator(schema), mapper::apply);
+  default ClosableIterator<HoodieRecord<IndexedRecord>> getRecordIterator(Schema schema) throws IOException {
+    ClosableIterator<IndexedRecord> iterator = getIndexedRecordIterator(schema);
+    return new MappingIterator<>(iterator, data -> unsafeCast(new HoodieAvroIndexedRecord(data)));
   }
 
   @Override
-  default Option<HoodieRecord> getRecordByKey(String key, Schema readerSchema, HoodieRecord.Mapper mapper) throws IOException {
-    return getRecordByKey(key, readerSchema).map(mapper::apply);
+  default ClosableIterator<HoodieRecord<IndexedRecord>> getRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException {
+    ClosableIterator<IndexedRecord> iterator = getIndexedRecordIterator(readerSchema, requestedSchema);
+    return new MappingIterator<>(iterator, data -> unsafeCast(new HoodieAvroIndexedRecord(data)));
   }
 
-  class HoodieRecordTransformIterator implements ClosableIterator<HoodieRecord> {
-    private final ClosableIterator<IndexedRecord> dataIterator;
-    private final HoodieRecord.Mapper mapper;
-
-    public HoodieRecordTransformIterator(ClosableIterator<IndexedRecord> dataIterator, Mapper mapper) {
-      this.dataIterator = dataIterator;
-      this.mapper = mapper;
-    }
-
-    @Override
-    public boolean hasNext() {
-      return dataIterator.hasNext();
-    }
-
-    @Override
-    public HoodieRecord next() {
-      return mapper.apply(dataIterator.next());
-    }
-
-    @Override
-    public void close() {
-      dataIterator.close();
-    }
+  default Option<HoodieRecord<IndexedRecord>> getRecordByKey(String key, Schema readerSchema) throws IOException {
+    return getIndexedRecordByKey(key, readerSchema)
+        .map(data -> unsafeCast(new HoodieAvroIndexedRecord(data)));
   }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderFactory.java
similarity index 52%
copy from hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java
copy to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderFactory.java
index d2e35eb450..a194e2fc6a 100644
--- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java
+++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderFactory.java
@@ -21,41 +21,33 @@ package org.apache.hudi.io.storage;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hudi.common.fs.FSUtils;
 
 import java.io.IOException;
 
-import static org.apache.hudi.common.model.HoodieFileFormat.HFILE;
-import static org.apache.hudi.common.model.HoodieFileFormat.ORC;
-import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
-
-public class HoodieFileReaderFactory {
-
-  public static HoodieAvroFileReader getFileReader(Configuration conf, Path path) throws IOException {
-    final String extension = FSUtils.getFileExtension(path.toString());
-    if (PARQUET.getFileExtension().equals(extension)) {
-      return newParquetFileReader(conf, path);
-    }
-    if (HFILE.getFileExtension().equals(extension)) {
-      return newHFileFileReader(conf, path);
-    }
-    if (ORC.getFileExtension().equals(extension)) {
-      return newOrcFileReader(conf, path);
-    }
-
-    throw new UnsupportedOperationException(extension + " format not supported yet.");
+public class HoodieAvroFileReaderFactory extends HoodieFileReaderFactory {
+
+  private static class SingletonHolder {
+    private static final HoodieAvroFileReaderFactory INSTANCE = new HoodieAvroFileReaderFactory();
+  }
+
+  private HoodieAvroFileReaderFactory() {
+  }
+
+  public static HoodieFileReaderFactory getFileReaderFactory() {
+    return SingletonHolder.INSTANCE;
   }
 
-  private static HoodieAvroFileReader newParquetFileReader(Configuration conf, Path path) {
+  protected HoodieFileReader newParquetFileReader(Configuration conf, Path path) {
     return new HoodieAvroParquetReader(conf, path);
   }
 
-  private static HoodieAvroFileReader newHFileFileReader(Configuration conf, Path path) throws IOException {
+  protected HoodieFileReader newHFileFileReader(Configuration conf, Path path) throws IOException {
     CacheConfig cacheConfig = new CacheConfig(conf);
     return new HoodieAvroHFileReader(conf, path, cacheConfig);
   }
 
-  private static HoodieAvroFileReader newOrcFileReader(Configuration conf, Path path) {
+  @Override
+  protected HoodieFileReader newOrcFileReader(Configuration conf, Path path) {
     return new HoodieAvroOrcReader(conf, path);
   }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriter.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriter.java
index 947b83fe86..bb046c2395 100644
--- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriter.java
+++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriter.java
@@ -19,6 +19,7 @@
 package org.apache.hudi.io.storage;
 
 import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieKey;
 import org.apache.hudi.common.model.HoodieRecord;
 
@@ -41,13 +42,13 @@ public interface HoodieAvroFileWriter extends HoodieFileWriter {
 
   @Override
   default void writeWithMetadata(HoodieKey key, HoodieRecord record, Schema schema, Properties props) throws IOException {
-    IndexedRecord avroPayload = (IndexedRecord)record.toIndexedRecord(schema, props).get();
+    IndexedRecord avroPayload = ((HoodieAvroIndexedRecord) record.toIndexedRecord(schema, props).get()).getData();
     writeAvroWithMetadata(key, avroPayload);
   }
 
   @Override
   default void write(String recordKey, HoodieRecord record, Schema schema, Properties props) throws IOException {
-    IndexedRecord avroPayload = (IndexedRecord)record.toIndexedRecord(schema, props).get();
+    IndexedRecord avroPayload = ((HoodieAvroIndexedRecord) record.toIndexedRecord(schema, props).get()).getData();
     writeAvro(recordKey, avroPayload);
   }
 
diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java
new file mode 100644
index 0000000000..0ab052a877
--- /dev/null
+++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileWriterFactory.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.io.storage;
+
+import org.apache.hudi.avro.HoodieAvroWriteSupport;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.Option;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.io.compress.Compression;
+import org.apache.orc.CompressionKind;
+import org.apache.parquet.avro.AvroSchemaConverter;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.io.IOException;
+
+import static org.apache.hudi.io.storage.HoodieHFileConfig.CACHE_DATA_IN_L1;
+import static org.apache.hudi.io.storage.HoodieHFileConfig.DROP_BEHIND_CACHE_COMPACTION;
+import static org.apache.hudi.io.storage.HoodieHFileConfig.HFILE_COMPARATOR;
+import static org.apache.hudi.io.storage.HoodieHFileConfig.PREFETCH_ON_OPEN;
+
+public class HoodieAvroFileWriterFactory extends HoodieFileWriterFactory {
+
+  private static class SingletonHolder {
+    private static final HoodieAvroFileWriterFactory INSTANCE = new HoodieAvroFileWriterFactory();
+  }
+
+  private HoodieAvroFileWriterFactory() {
+  }
+
+  public static HoodieFileWriterFactory getFileReaderFactory() {
+    return HoodieAvroFileWriterFactory.SingletonHolder.INSTANCE;
+  }
+
+  protected HoodieFileWriter newParquetFileWriter(
+      String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier) throws IOException {
+    boolean populateMetaFields = config.getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS);
+    boolean enableBloomFilter = populateMetaFields;
+    Option<BloomFilter> filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty();
+    HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(conf).convert(schema), schema, filter);
+    String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME);
+    // Support PARQUET_COMPRESSION_CODEC_NAME is ""
+    if (compressionCodecName.isEmpty()) {
+      compressionCodecName = null;
+    }
+    HoodieParquetConfig<HoodieAvroWriteSupport> parquetConfig = new HoodieParquetConfig(writeSupport,
+        CompressionCodecName.fromConf(compressionCodecName),
+        config.getIntOrDefault(HoodieStorageConfig.PARQUET_BLOCK_SIZE),
+        config.getIntOrDefault(HoodieStorageConfig.PARQUET_PAGE_SIZE),
+        config.getLongOrDefault(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE),
+        conf, config.getDoubleOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION),
+        config.getBooleanOrDefault(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED));
+    return new HoodieAvroParquetWriter(path, parquetConfig, instantTime, taskContextSupplier, populateMetaFields);
+  }
+
+  protected HoodieFileWriter newParquetFileWriter(
+      FSDataOutputStream outputStream, Configuration conf, HoodieConfig config, Schema schema) throws IOException {
+    boolean enableBloomFilter = false;
+    Option<BloomFilter> filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty();
+    HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(conf).convert(schema), schema, filter);
+    HoodieParquetConfig<HoodieAvroWriteSupport> parquetConfig = new HoodieParquetConfig<>(writeSupport,
+        CompressionCodecName.fromConf(config.getString(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME)),
+        config.getInt(HoodieStorageConfig.PARQUET_BLOCK_SIZE),
+        config.getInt(HoodieStorageConfig.PARQUET_PAGE_SIZE),
+        config.getLong(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE), // todo: 1024*1024*1024
+        conf, config.getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION));
+    return new HoodieParquetStreamWriter(outputStream, parquetConfig);
+  }
+
+  protected HoodieFileWriter newHFileFileWriter(
+      String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier) throws IOException {
+    BloomFilter filter = createBloomFilter(config);
+    HoodieHFileConfig hfileConfig = new HoodieHFileConfig(conf,
+        Compression.Algorithm.valueOf(config.getString(HoodieStorageConfig.HFILE_COMPRESSION_ALGORITHM_NAME)),
+        config.getInt(HoodieStorageConfig.HFILE_BLOCK_SIZE),
+        config.getLong(HoodieStorageConfig.HFILE_MAX_FILE_SIZE),
+        HoodieAvroHFileReader.KEY_FIELD_NAME, PREFETCH_ON_OPEN, CACHE_DATA_IN_L1, DROP_BEHIND_CACHE_COMPACTION,
+        filter, HFILE_COMPARATOR);
+
+    return new HoodieAvroHFileWriter(instantTime, path, hfileConfig, schema, taskContextSupplier, config.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS));
+  }
+
+  protected HoodieFileWriter newOrcFileWriter(
+      String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier) throws IOException {
+    BloomFilter filter = createBloomFilter(config);
+    HoodieOrcConfig orcConfig = new HoodieOrcConfig(conf,
+        CompressionKind.valueOf(config.getString(HoodieStorageConfig.ORC_COMPRESSION_CODEC_NAME)),
+        config.getInt(HoodieStorageConfig.ORC_STRIPE_SIZE),
+        config.getInt(HoodieStorageConfig.ORC_BLOCK_SIZE),
+        config.getLong(HoodieStorageConfig.ORC_FILE_MAX_SIZE), filter);
+    return new HoodieAvroOrcWriter(instantTime, path, orcConfig, schema, taskContextSupplier);
+  }
+}
diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java
index 6ac2a94932..90323d8883 100644
--- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java
+++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java
@@ -171,21 +171,26 @@ public class HoodieAvroHFileReader implements HoodieAvroFileReader {
 
   @SuppressWarnings("unchecked")
   @Override
-  public Option<IndexedRecord> getRecordByKey(String key, Schema readerSchema) throws IOException {
+  public Option<IndexedRecord> getIndexedRecordByKey(String key, Schema readerSchema) throws IOException {
     synchronized (sharedScannerLock) {
       return fetchRecordByKeyInternal(sharedScanner, key, getSchema(), readerSchema);
     }
   }
 
-  public ClosableIterator<IndexedRecord> getRecordIterator(Schema readerSchema) throws IOException {
+  public ClosableIterator<IndexedRecord> getIndexedRecordIterator(Schema readerSchema) throws IOException {
     // TODO eval whether seeking scanner would be faster than pread
     HFileScanner scanner = getHFileScanner(reader, false);
     return new RecordIterator(scanner, getSchema(), readerSchema);
   }
 
+  @Override
+  public ClosableIterator<IndexedRecord> getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
   @SuppressWarnings("unchecked")
   @Override
-  public ClosableIterator<IndexedRecord> getRecordsByKeysIterator(List<String> keys, Schema readerSchema) throws IOException {
+  public ClosableIterator<IndexedRecord> getIndexedRecordsByKeysIterator(List<String> keys, Schema readerSchema) throws IOException {
     // We're caching blocks for this scanner to minimize amount of traffic
     // to the underlying storage as we fetched (potentially) sparsely distributed
     // keys
@@ -195,7 +200,7 @@ public class HoodieAvroHFileReader implements HoodieAvroFileReader {
 
   @SuppressWarnings("unchecked")
   @Override
-  public ClosableIterator<IndexedRecord> getRecordsByKeyPrefixIterator(List<String> keyPrefixes, Schema readerSchema) throws IOException {
+  public ClosableIterator<IndexedRecord> getIndexedRecordsByKeyPrefixIterator(List<String> keyPrefixes, Schema readerSchema) throws IOException {
     // We're caching blocks for this scanner to minimize amount of traffic
     // to the underlying storage as we fetched (potentially) sparsely distributed
     // keys
@@ -372,7 +377,7 @@ public class HoodieAvroHFileReader implements HoodieAvroFileReader {
    */
   public static List<IndexedRecord> readAllRecords(HoodieAvroHFileReader reader) throws IOException {
     Schema schema = reader.getSchema();
-    return toStream(reader.getRecordIterator(schema))
+    return toStream(reader.getIndexedRecordIterator(schema))
         .collect(Collectors.toList());
   }
 
@@ -395,7 +400,7 @@ public class HoodieAvroHFileReader implements HoodieAvroFileReader {
                                                               List<String> keys,
                                                               Schema schema) throws IOException {
     Collections.sort(keys);
-    return toStream(reader.getRecordsByKeysIterator(keys, schema))
+    return toStream(reader.getIndexedRecordsByKeysIterator(keys, schema))
         .collect(Collectors.toList());
   }
 
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileWriter.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileWriter.java
similarity index 100%
rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileWriter.java
rename to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileWriter.java
diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcReader.java
index 8f953d25b5..051add1102 100644
--- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcReader.java
+++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcReader.java
@@ -65,7 +65,7 @@ public class HoodieAvroOrcReader implements HoodieAvroFileReader {
   }
 
   @Override
-  public ClosableIterator<IndexedRecord> getRecordIterator(Schema schema) throws IOException {
+  public ClosableIterator<IndexedRecord> getIndexedRecordIterator(Schema schema) throws IOException {
     try {
       Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf));
       TypeDescription orcSchema = AvroOrcUtils.createOrcSchema(schema);
@@ -76,6 +76,11 @@ public class HoodieAvroOrcReader implements HoodieAvroFileReader {
     }
   }
 
+  @Override
+  public ClosableIterator<IndexedRecord> getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
   @Override
   public Schema getSchema() {
     return orcUtils.readAvroSchema(conf, path);
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcWriter.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcWriter.java
similarity index 100%
rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcWriter.java
rename to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcWriter.java
diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java
index 83ffaf589f..769ef391b4 100644
--- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java
+++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java
@@ -26,6 +26,7 @@ import org.apache.hudi.common.bloom.BloomFilter;
 import org.apache.hudi.common.model.HoodieFileFormat;
 import org.apache.hudi.common.util.BaseFileUtils;
 import org.apache.hudi.common.util.ClosableIterator;
+import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ParquetReaderIterator;
 import org.apache.parquet.avro.AvroParquetReader;
 import org.apache.parquet.avro.AvroReadSupport;
@@ -65,12 +66,13 @@ public class HoodieAvroParquetReader implements HoodieAvroFileReader {
   }
 
   @Override
-  public ClosableIterator<IndexedRecord> getRecordIterator(Schema schema) throws IOException {
-    AvroReadSupport.setAvroReadSchema(conf, schema);
-    ParquetReader<IndexedRecord> reader = AvroParquetReader.<IndexedRecord>builder(path).withConf(conf).build();
-    ParquetReaderIterator<IndexedRecord> parquetReaderIterator = new ParquetReaderIterator<>(reader);
-    readerIterators.add(parquetReaderIterator);
-    return parquetReaderIterator;
+  public ClosableIterator<IndexedRecord> getIndexedRecordIterator(Schema schema) throws IOException {
+    return getIndexedRecordIteratorInternal(schema, Option.empty());
+  }
+
+  @Override
+  public ClosableIterator<IndexedRecord> getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException {
+    return getIndexedRecordIteratorInternal(readerSchema, Option.of(requestedSchema));
   }
 
   @Override
@@ -87,4 +89,15 @@ public class HoodieAvroParquetReader implements HoodieAvroFileReader {
   public long getTotalRecords() {
     return parquetUtils.getRowCount(conf, path);
   }
+
+  private ClosableIterator<IndexedRecord> getIndexedRecordIteratorInternal(Schema schema, Option<Schema> requestedSchema) throws IOException {
+    AvroReadSupport.setAvroReadSchema(conf, schema);
+    if (requestedSchema.isPresent()) {
+      AvroReadSupport.setRequestedProjection(conf, requestedSchema.get());
+    }
+    ParquetReader<IndexedRecord> reader = AvroParquetReader.<IndexedRecord>builder(path).withConf(conf).build();
+    ParquetReaderIterator<IndexedRecord> parquetReaderIterator = new ParquetReaderIterator<>(reader);
+    readerIterators.add(parquetReaderIterator);
+    return parquetReaderIterator;
+  }
 }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetWriter.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetWriter.java
similarity index 100%
rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetWriter.java
rename to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetWriter.java
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetWriter.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetWriter.java
similarity index 100%
rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetWriter.java
rename to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetWriter.java
diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java
index 175722bb9b..8152a176a0 100644
--- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java
+++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java
@@ -29,7 +29,7 @@ import java.io.UnsupportedEncodingException;
 import java.util.List;
 import java.util.Set;
 
-public interface HoodieFileReader extends AutoCloseable {
+public interface HoodieFileReader<T> extends AutoCloseable {
 
   String[] readMinMaxRecordKeys();
 
@@ -37,34 +37,36 @@ public interface HoodieFileReader extends AutoCloseable {
 
   Set<String> filterRowKeys(Set<String> candidateRowKeys);
 
-  ClosableIterator<HoodieRecord> getRecordIterator(Schema readerSchema, HoodieRecord.Mapper mapper) throws IOException;
+  ClosableIterator<HoodieRecord<T>> getRecordIterator(Schema readerSchema) throws IOException;
 
-  default ClosableIterator<HoodieRecord> getRecordIterator(HoodieRecord.Mapper mapper) throws IOException {
-    return getRecordIterator(getSchema(), mapper);
+  ClosableIterator<HoodieRecord<T>> getRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException;
+
+  default ClosableIterator<HoodieRecord<T>> getRecordIterator() throws IOException {
+    return getRecordIterator(getSchema());
   }
 
-  default Option<HoodieRecord> getRecordByKey(String key, Schema readerSchema, HoodieRecord.Mapper mapper) throws IOException {
+  default Option<HoodieRecord<T>> getRecordByKey(String key, Schema readerSchema) throws IOException {
     throw new UnsupportedOperationException();
   }
 
-  default Option<HoodieRecord> getRecordByKey(String key, HoodieRecord.Mapper mapper) throws IOException {
-    return getRecordByKey(key, getSchema(), mapper);
+  default Option<HoodieRecord<T>> getRecordByKey(String key) throws IOException {
+    return getRecordByKey(key, getSchema());
   }
 
-  default ClosableIterator<HoodieRecord> getRecordsByKeysIterator(List<String> keys, Schema schema, HoodieRecord.Mapper mapper) throws IOException {
+  default ClosableIterator<HoodieRecord<T>> getRecordsByKeysIterator(List<String> keys, Schema schema) throws IOException {
     throw new UnsupportedOperationException();
   }
 
-  default ClosableIterator<HoodieRecord> getRecordsByKeysIterator(List<String> keys, HoodieRecord.Mapper mapper) throws IOException {
-    return getRecordsByKeysIterator(keys, getSchema(), mapper);
+  default ClosableIterator<HoodieRecord<T>> getRecordsByKeysIterator(List<String> keys) throws IOException {
+    return getRecordsByKeysIterator(keys, getSchema());
   }
 
-  default ClosableIterator<HoodieRecord> getRecordsByKeyPrefixIterator(List<String> keyPrefixes, Schema schema, HoodieRecord.Mapper mapper) throws IOException {
+  default ClosableIterator<HoodieRecord<T>> getRecordsByKeyPrefixIterator(List<String> keyPrefixes, Schema schema) throws IOException {
     throw new UnsupportedEncodingException();
   }
 
-  default ClosableIterator<HoodieRecord> getRecordsByKeyPrefixIterator(List<String> keyPrefixes, HoodieRecord.Mapper mapper) throws IOException {
-    return getRecordsByKeyPrefixIterator(keyPrefixes, getSchema(), mapper);
+  default ClosableIterator<HoodieRecord<T>> getRecordsByKeyPrefixIterator(List<String> keyPrefixes) throws IOException {
+    return getRecordsByKeyPrefixIterator(keyPrefixes, getSchema());
   }
 
   Schema getSchema();
diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java
index d2e35eb450..f84ec530ad 100644
--- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java
+++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java
@@ -20,10 +20,16 @@ package org.apache.hudi.io.storage;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+
 import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.exception.HoodieException;
 
 import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
 
 import static org.apache.hudi.common.model.HoodieFileFormat.HFILE;
 import static org.apache.hudi.common.model.HoodieFileFormat.ORC;
@@ -31,7 +37,24 @@ import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
 
 public class HoodieFileReaderFactory {
 
-  public static HoodieAvroFileReader getFileReader(Configuration conf, Path path) throws IOException {
+  public static HoodieFileReaderFactory getReaderFactory(HoodieRecord.HoodieRecordType recordType) {
+    switch (recordType) {
+      case AVRO:
+        return HoodieAvroFileReaderFactory.getFileReaderFactory();
+      case SPARK:
+        try {
+          Class<?> clazz = ReflectionUtils.getClass("org.apache.hudi.io.storage.HoodieSparkFileReaderFactory");
+          Method method = clazz.getMethod("getFileReaderFactory", null);
+          return (HoodieFileReaderFactory) method.invoke(null,null);
+        } catch (NoSuchMethodException | IllegalArgumentException | InvocationTargetException | IllegalAccessException e) {
+          throw new HoodieException("Unable to create hoodie spark file writer factory", e);
+        }
+      default:
+        throw new UnsupportedOperationException(recordType + " record type not supported yet.");
+    }
+  }
+
+  public HoodieFileReader getFileReader(Configuration conf, Path path) throws IOException {
     final String extension = FSUtils.getFileExtension(path.toString());
     if (PARQUET.getFileExtension().equals(extension)) {
       return newParquetFileReader(conf, path);
@@ -46,16 +69,19 @@ public class HoodieFileReaderFactory {
     throw new UnsupportedOperationException(extension + " format not supported yet.");
   }
 
-  private static HoodieAvroFileReader newParquetFileReader(Configuration conf, Path path) {
-    return new HoodieAvroParquetReader(conf, path);
+  public HoodieFileReader getFileReader(Configuration conf, Path path, HoodieFileFormat format) throws IOException {
+    return this.newParquetFileReader(conf, path);
+  }
+
+  protected HoodieFileReader newParquetFileReader(Configuration conf, Path path) {
+    throw new UnsupportedOperationException();
   }
 
-  private static HoodieAvroFileReader newHFileFileReader(Configuration conf, Path path) throws IOException {
-    CacheConfig cacheConfig = new CacheConfig(conf);
-    return new HoodieAvroHFileReader(conf, path, cacheConfig);
+  protected HoodieFileReader newHFileFileReader(Configuration conf, Path path) throws IOException {
+    throw new UnsupportedOperationException();
   }
 
-  private static HoodieAvroFileReader newOrcFileReader(Configuration conf, Path path) {
-    return new HoodieAvroOrcReader(conf, path);
+  protected HoodieFileReader newOrcFileReader(Configuration conf, Path path) {
+    throw new UnsupportedOperationException();
   }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java
new file mode 100644
index 0000000000..99f35d7a0f
--- /dev/null
+++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriterFactory.java
@@ -0,0 +1,131 @@
+/*
+ * 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.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.bloom.BloomFilterTypeCode;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+
+import static org.apache.hudi.common.model.HoodieFileFormat.HFILE;
+import static org.apache.hudi.common.model.HoodieFileFormat.ORC;
+import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
+
+public class HoodieFileWriterFactory {
+
+  private static HoodieFileWriterFactory getWriterFactory(HoodieRecord.HoodieRecordType recordType) {
+    switch (recordType) {
+      case AVRO:
+        return HoodieAvroFileWriterFactory.getFileReaderFactory();
+      case SPARK:
+        try {
+          Class<?> clazz = ReflectionUtils.getClass("org.apache.hudi.io.storage.HoodieSparkFileWriterFactory");
+          Method method = clazz.getMethod("getFileWriterFactory", null);
+          return (HoodieFileWriterFactory) method.invoke(null, null);
+        } catch (NoSuchMethodException | IllegalAccessException | IllegalArgumentException | InvocationTargetException e) {
+          throw new HoodieException("Unable to create hoodie spark file writer factory", e);
+        }
+      default:
+        throw new UnsupportedOperationException(recordType + " record type not supported yet.");
+    }
+  }
+
+  public static <T, I, K, O> HoodieFileWriter getFileWriter(
+      String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier, HoodieRecordType recordType) throws IOException {
+    final String extension = FSUtils.getFileExtension(path.getName());
+    HoodieFileWriterFactory factory = getWriterFactory(recordType);
+    return factory.getFileWriterByFormat(extension, instantTime, path, conf, config, schema, taskContextSupplier);
+  }
+
+  public static <T, I, K, O> HoodieFileWriter getFileWriter(HoodieFileFormat format,
+      FSDataOutputStream outputStream, Configuration conf, HoodieConfig config, Schema schema, HoodieRecordType recordType) throws IOException {
+    HoodieFileWriterFactory factory = getWriterFactory(recordType);
+    return factory.getFileWriterByFormat(format, outputStream, conf, config, schema);
+  }
+
+  protected <T, I, K, O> HoodieFileWriter getFileWriterByFormat(
+      String extension, String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier) throws IOException {
+    if (PARQUET.getFileExtension().equals(extension)) {
+      return newParquetFileWriter(instantTime, path, conf, config, schema, taskContextSupplier);
+    }
+    if (HFILE.getFileExtension().equals(extension)) {
+      return newHFileFileWriter(instantTime, path, conf, config, schema, taskContextSupplier);
+    }
+    if (ORC.getFileExtension().equals(extension)) {
+      return newOrcFileWriter(instantTime, path, conf, config, schema, taskContextSupplier);
+    }
+    throw new UnsupportedOperationException(extension + " format not supported yet.");
+  }
+
+  protected <T, I, K, O> HoodieFileWriter getFileWriterByFormat(HoodieFileFormat format,
+      FSDataOutputStream outputStream, Configuration conf, HoodieConfig config, Schema schema) throws IOException {
+    switch (format) {
+      case PARQUET:
+        return newParquetFileWriter(outputStream, conf, config, schema);
+      default:
+        throw new UnsupportedOperationException(format + " format not supported yet.");
+    }
+  }
+
+  protected HoodieFileWriter newParquetFileWriter(
+      String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  protected HoodieFileWriter newParquetFileWriter(
+      FSDataOutputStream outputStream, Configuration conf, HoodieConfig config, Schema schema) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  protected HoodieFileWriter newHFileFileWriter(
+      String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  protected HoodieFileWriter newOrcFileWriter(
+      String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  protected BloomFilter createBloomFilter(HoodieConfig config) {
+    return BloomFilterFactory.createBloomFilter(60000, 0.000000001, 100000,
+        BloomFilterTypeCode.DYNAMIC_V0.name());
+  }
+}
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java
similarity index 100%
rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java
rename to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcConfig.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcConfig.java
similarity index 100%
rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcConfig.java
rename to hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcConfig.java
diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java
index 8b5bad3233..f777c55e89 100644
--- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java
+++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java
@@ -32,9 +32,9 @@ import org.apache.hudi.common.function.SerializableFunction;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieAvroRecord;
 import org.apache.hudi.common.model.HoodieBaseFile;
-import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieLogFile;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
@@ -50,7 +50,7 @@ import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.exception.HoodieMetadataException;
 import org.apache.hudi.exception.TableNotFoundException;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 
 import org.apache.avro.Schema;
@@ -97,7 +97,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
   private final boolean reuse;
 
   // Readers for the latest file slice corresponding to file groups in the metadata partition
-  private Map<Pair<String, String>, Pair<HoodieAvroFileReader, HoodieMetadataMergedLogRecordReader>> partitionReaders =
+  private Map<Pair<String, String>, Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader>> partitionReaders =
       new ConcurrentHashMap<>();
 
   public HoodieBackedTableMetadata(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
@@ -164,12 +164,12 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
             (SerializableFunction<FileSlice, Iterator<HoodieRecord<HoodieMetadataPayload>>>) fileSlice -> {
               // NOTE: Since this will be executed by executors, we can't access previously cached
               //       readers, and therefore have to always open new ones
-              Pair<HoodieAvroFileReader, HoodieMetadataMergedLogRecordReader> readers =
+              Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> readers =
                   openReaders(partitionName, fileSlice);
               try {
                 List<Long> timings = new ArrayList<>();
 
-                HoodieAvroFileReader baseFileReader = readers.getKey();
+                HoodieFileReader baseFileReader = readers.getKey();
                 HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight();
 
                 if (baseFileReader == null && logRecordScanner == null) {
@@ -210,11 +210,11 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
     List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> result = new ArrayList<>();
     AtomicInteger fileSlicesKeysCount = new AtomicInteger();
     partitionFileSliceToKeysMap.forEach((partitionFileSlicePair, fileSliceKeys) -> {
-      Pair<HoodieAvroFileReader, HoodieMetadataMergedLogRecordReader> readers =
+      Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> readers =
           getOrCreateReaders(partitionName, partitionFileSlicePair.getRight());
       try {
         List<Long> timings = new ArrayList<>();
-        HoodieAvroFileReader baseFileReader = readers.getKey();
+        HoodieFileReader baseFileReader = readers.getKey();
         HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight();
         if (baseFileReader == null && logRecordScanner == null) {
           return;
@@ -281,7 +281,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
     return logRecords;
   }
 
-  private List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> readFromBaseAndMergeWithLogRecords(HoodieAvroFileReader baseFileReader,
+  private List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> readFromBaseAndMergeWithLogRecords(HoodieFileReader baseFileReader,
                                                                                                              List<String> keys,
                                                                                                              boolean fullKeys,
                                                                                                              Map<String, Option<HoodieRecord<HoodieMetadataPayload>>> logRecords,
@@ -342,12 +342,12 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
     }
   }
 
-  private Map<String, HoodieRecord<HoodieMetadataPayload>> fetchBaseFileRecordsByKeys(HoodieAvroFileReader baseFileReader,
+  private Map<String, HoodieRecord<HoodieMetadataPayload>> fetchBaseFileRecordsByKeys(HoodieFileReader baseFileReader,
                                                                                       List<String> keys,
                                                                                       boolean fullKeys,
                                                                                       String partitionName) throws IOException {
-    ClosableIterator<HoodieRecord> records = fullKeys ? baseFileReader.getRecordsByKeysIterator(keys, HoodieAvroIndexedRecord::new)
-        : baseFileReader.getRecordsByKeyPrefixIterator(keys, HoodieAvroIndexedRecord::new);
+    ClosableIterator<HoodieRecord> records = fullKeys ? baseFileReader.getRecordsByKeysIterator(keys)
+        : baseFileReader.getRecordsByKeyPrefixIterator(keys);
 
     return toStream(records)
         .map(record -> {
@@ -402,7 +402,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
    * @param slice            - The file slice to open readers for
    * @return File reader and the record scanner pair for the requested file slice
    */
-  private Pair<HoodieAvroFileReader, HoodieMetadataMergedLogRecordReader> getOrCreateReaders(String partitionName, FileSlice slice) {
+  private Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> getOrCreateReaders(String partitionName, FileSlice slice) {
     if (reuse) {
       return partitionReaders.computeIfAbsent(Pair.of(partitionName, slice.getFileId()), k -> {
         return openReaders(partitionName, slice); });
@@ -411,12 +411,12 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
     }
   }
 
-  private Pair<HoodieAvroFileReader, HoodieMetadataMergedLogRecordReader> openReaders(String partitionName, FileSlice slice) {
+  private Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> openReaders(String partitionName, FileSlice slice) {
     try {
       HoodieTimer timer = new HoodieTimer().startTimer();
       // Open base file reader
-      Pair<HoodieAvroFileReader, Long> baseFileReaderOpenTimePair = getBaseFileReader(slice, timer);
-      HoodieAvroFileReader baseFileReader = baseFileReaderOpenTimePair.getKey();
+      Pair<HoodieFileReader, Long> baseFileReaderOpenTimePair = getBaseFileReader(slice, timer);
+      HoodieFileReader baseFileReader = baseFileReaderOpenTimePair.getKey();
       final long baseFileOpenMs = baseFileReaderOpenTimePair.getValue();
 
       // Open the log record scanner using the log files from the latest file slice
@@ -434,14 +434,14 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
     }
   }
 
-  private Pair<HoodieAvroFileReader, Long> getBaseFileReader(FileSlice slice, HoodieTimer timer) throws IOException {
-    HoodieAvroFileReader baseFileReader = null;
+  private Pair<HoodieFileReader, Long> getBaseFileReader(FileSlice slice, HoodieTimer timer) throws IOException {
+    HoodieFileReader baseFileReader = null;
     Long baseFileOpenMs;
     // If the base file is present then create a reader
     Option<HoodieBaseFile> basefile = slice.getBaseFile();
     if (basefile.isPresent()) {
       String basefilePath = basefile.get().getPath();
-      baseFileReader = HoodieFileReaderFactory.getFileReader(hadoopConf.get(), new Path(basefilePath));
+      baseFileReader = HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(hadoopConf.get(), new Path(basefilePath));
       baseFileOpenMs = timer.endTimer();
       LOG.info(String.format("Opened metadata base file from %s at instant %s in %d ms", basefilePath,
           basefile.get().getCommitTime(), baseFileOpenMs));
@@ -572,7 +572,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
    * @param partitionFileSlicePair - Partition and FileSlice
    */
   private synchronized void close(Pair<String, String> partitionFileSlicePair) {
-    Pair<HoodieAvroFileReader, HoodieMetadataMergedLogRecordReader> readers =
+    Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> readers =
         partitionReaders.remove(partitionFileSlicePair);
     closeReader(readers);
   }
@@ -587,7 +587,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
     partitionReaders.clear();
   }
 
-  private void closeReader(Pair<HoodieAvroFileReader, HoodieMetadataMergedLogRecordReader> readers) {
+  private void closeReader(Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> readers) {
     if (readers != null) {
       try {
         if (readers.getKey() != null) {
diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java
index cbd7e6c175..1d21ceabdc 100644
--- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java
+++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataMergedLogRecordReader.java
@@ -19,17 +19,14 @@
 package org.apache.hudi.metadata;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
-import org.apache.avro.generic.IndexedRecord;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hudi.common.config.HoodieMetadataConfig;
-import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieAvroRecordMerger;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
 import org.apache.hudi.common.table.log.InstantRange;
+import org.apache.hudi.common.util.HoodieRecordUtils;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.collection.ExternalSpillableMap;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.internal.schema.InternalSchema;
@@ -61,24 +58,8 @@ public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc
                                               boolean isBitCaskDiskMapCompressionEnabled,
                                               Option<InstantRange> instantRange, boolean allowFullScan) {
     super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, true, false, bufferSize,
-        spillableMapBasePath, instantRange, diskMapType, isBitCaskDiskMapCompressionEnabled, false, allowFullScan, Option.of(partitionName), InternalSchema.getEmptyInternalSchema());
-  }
-
-  @Override
-  protected HoodieAvroRecord<?> createHoodieRecord(final IndexedRecord rec, final HoodieTableConfig hoodieTableConfig,
-                                               final String payloadClassFQN, final String preCombineField,
-                                               final boolean withOperationField,
-                                               final Option<Pair<String, String>> simpleKeyGenFields,
-                                               final Option<String> partitionName) {
-    if (hoodieTableConfig.populateMetaFields()) {
-      return super.createHoodieRecord(rec, hoodieTableConfig, payloadClassFQN, preCombineField, withOperationField,
-          simpleKeyGenFields, partitionName);
-    }
-
-    // When meta fields are not available, create the record using the
-    // preset key field and the known partition name
-    return SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, payloadClassFQN,
-        preCombineField, simpleKeyGenFields.get(), withOperationField, partitionName);
+        spillableMapBasePath, instantRange, diskMapType, isBitCaskDiskMapCompressionEnabled, false, allowFullScan, Option.of(partitionName),
+        InternalSchema.getEmptyInternalSchema(), HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()));
   }
 
   /**
@@ -128,6 +109,11 @@ public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc
     }
   }
 
+  @Override
+  protected boolean getPopulateMetaFields() {
+    return this.hoodieTableMetaClient.getTableConfig().populateMetaFields() && super.getPopulateMetaFields();
+  }
+
   @Override
   protected String getKeyField() {
     return HoodieMetadataPayload.KEY_FIELD_NAME;
diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java
index 349c0efb48..cb78ad8dd1 100644
--- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java
+++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadata.java
@@ -92,6 +92,9 @@ public interface HoodieTableMetadata extends Serializable, AutoCloseable {
    * @param basePath The base path to check
    */
   static boolean isMetadataTable(String basePath) {
+    if (basePath == null || basePath.isEmpty()) {
+      return false;
+    }
     if (basePath.endsWith(Path.SEPARATOR)) {
       basePath = basePath.substring(0, basePath.length() - 1);
     }
diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java
index 3604572368..d932661bb0 100644
--- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java
+++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java
@@ -34,6 +34,7 @@ import org.apache.hudi.common.model.HoodieCommitMetadata;
 import org.apache.hudi.common.model.HoodieDeltaWriteStat;
 import org.apache.hudi.common.model.HoodieFileFormat;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
 import org.apache.hudi.common.model.HoodieWriteStat;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -53,7 +54,7 @@ import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.exception.HoodieMetadataException;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 import org.apache.hudi.util.Lazy;
 
@@ -413,8 +414,8 @@ public class HoodieTableMetadataUtil {
       }
 
       final Path writeFilePath = new Path(recordsGenerationParams.getDataMetaClient().getBasePath(), pathWithPartition);
-      try (HoodieAvroFileReader fileReader =
-               HoodieFileReaderFactory.getFileReader(recordsGenerationParams.getDataMetaClient().getHadoopConf(), writeFilePath)) {
+      try (HoodieFileReader fileReader =
+               HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(recordsGenerationParams.getDataMetaClient().getHadoopConf(), writeFilePath)) {
         try {
           final BloomFilter fileBloomFilter = fileReader.readBloomFilter();
           if (fileBloomFilter == null) {
@@ -873,8 +874,8 @@ public class HoodieTableMetadataUtil {
         }
         final String pathWithPartition = partitionName + "/" + appendedFile;
         final Path appendedFilePath = new Path(recordsGenerationParams.getDataMetaClient().getBasePath(), pathWithPartition);
-        try (HoodieAvroFileReader fileReader =
-                 HoodieFileReaderFactory.getFileReader(recordsGenerationParams.getDataMetaClient().getHadoopConf(), appendedFilePath)) {
+        try (HoodieFileReader fileReader =
+                 HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(recordsGenerationParams.getDataMetaClient().getHadoopConf(), appendedFilePath)) {
           final BloomFilter fileBloomFilter = fileReader.readBloomFilter();
           if (fileBloomFilter == null) {
             LOG.error("Failed to read bloom filter for " + appendedFilePath);
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java
index 289180580b..596a109108 100755
--- a/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/functional/TestHoodieLogFormat.java
@@ -24,8 +24,10 @@ import org.apache.hudi.common.model.DeleteRecord;
 import org.apache.hudi.common.model.HoodieArchivedLogFile;
 import org.apache.hudi.common.model.HoodieAvroRecord;
 import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
+import org.apache.hudi.common.model.HoodieAvroRecordMerger;
 import org.apache.hudi.common.model.HoodieLogFile;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.table.log.AppendResult;
@@ -50,6 +52,7 @@ import org.apache.hudi.common.testutils.HoodieTestUtils;
 import org.apache.hudi.common.testutils.SchemaTestUtil;
 import org.apache.hudi.common.testutils.minicluster.MiniClusterUtil;
 import org.apache.hudi.common.util.ClosableIterator;
+import org.apache.hudi.common.util.HoodieRecordUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.collection.ExternalSpillableMap;
 import org.apache.hudi.exception.CorruptedLogFileException;
@@ -591,6 +594,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
         .withSpillableMapBasePath(BASE_OUTPUT_PATH)
         .withDiskMapType(diskMapType)
         .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
 
     List<IndexedRecord> scannedRecords = new ArrayList<>();
@@ -805,6 +809,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
         .withSpillableMapBasePath(BASE_OUTPUT_PATH)
         .withDiskMapType(diskMapType)
         .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
     assertEquals(200, scanner.getTotalLogRecords());
     Set<String> readKeys = new HashSet<>(200);
@@ -883,6 +888,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
         .withSpillableMapBasePath(BASE_OUTPUT_PATH)
         .withDiskMapType(diskMapType)
         .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
     assertEquals(200, scanner.getTotalLogRecords(), "We read 200 records from 2 write batches");
     Set<String> readKeys = new HashSet<>(200);
@@ -970,6 +976,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
         .withSpillableMapBasePath(BASE_OUTPUT_PATH)
         .withDiskMapType(diskMapType)
         .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
     assertEquals(200, scanner.getTotalLogRecords(), "We would read 200 records");
     Set<String> readKeys = new HashSet<>(200);
@@ -1048,6 +1055,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
         .withSpillableMapBasePath(BASE_OUTPUT_PATH)
         .withDiskMapType(diskMapType)
         .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
 
     assertEquals(200, scanner.getTotalLogRecords(), "We still would read 200 records");
@@ -1094,6 +1102,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
         .withSpillableMapBasePath(BASE_OUTPUT_PATH)
         .withDiskMapType(diskMapType)
         .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
     scanner.forEach(s -> readKeys.add(s.getKey().getRecordKey()));
     assertEquals(200, readKeys.size(), "Stream collect should return all 200 records after rollback of delete");
@@ -1189,6 +1198,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
         .withSpillableMapBasePath(BASE_OUTPUT_PATH)
         .withDiskMapType(diskMapType)
         .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
 
     assertEquals(200, scanner.getTotalLogRecords(), "We still would read 200 records");
@@ -1292,6 +1302,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
         .withSpillableMapBasePath(BASE_OUTPUT_PATH)
         .withDiskMapType(diskMapType)
         .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
     assertEquals(0, scanner.getTotalLogRecords(), "We would have scanned 0 records because of rollback");
 
@@ -1360,6 +1371,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
         .withSpillableMapBasePath(BASE_OUTPUT_PATH)
         .withDiskMapType(diskMapType)
         .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
     assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records");
     FileCreateUtils.deleteDeltaCommit(basePath, "100", fs);
@@ -1411,6 +1423,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
         .withSpillableMapBasePath(BASE_OUTPUT_PATH)
         .withDiskMapType(diskMapType)
         .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
     assertEquals(100, scanner.getTotalLogRecords(), "We still would read 100 records");
     final List<String> readKeys = new ArrayList<>(100);
@@ -1481,6 +1494,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
         .withSpillableMapBasePath(BASE_OUTPUT_PATH)
         .withDiskMapType(diskMapType)
         .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
     assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records");
   }
@@ -1587,6 +1601,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
         .withSpillableMapBasePath(BASE_OUTPUT_PATH)
         .withDiskMapType(diskMapType)
         .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
     assertEquals(0, scanner.getTotalLogRecords(), "We would read 0 records");
     FileCreateUtils.deleteDeltaCommit(basePath, "100", fs);
@@ -1661,6 +1676,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
           .withSpillableMapBasePath(BASE_OUTPUT_PATH)
           .withDiskMapType(diskMapType)
           .withBitCaskDiskMapCompressionEnabled(isCompressionEnabled)
+          .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
           .build();
 
       assertEquals(Math.max(numRecordsInLog1, numRecordsInLog2), scanner.getNumMergedRecordsInLog(),
@@ -2040,10 +2056,10 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
    * Utility to convert the given iterator to a List.
    */
   private static List<IndexedRecord> getRecords(HoodieDataBlock dataBlock) {
-    ClosableIterator<HoodieRecord> itr = dataBlock.getRecordIterator(HoodieAvroIndexedRecord::new);
+    ClosableIterator<HoodieRecord<IndexedRecord>> itr = dataBlock.getRecordIterator(HoodieRecordType.AVRO);
 
     List<IndexedRecord> elements = new ArrayList<>();
-    itr.forEachRemaining(r -> elements.add((IndexedRecord) r.getData()));
+    itr.forEachRemaining(r -> elements.add(r.getData()));
     return elements;
   }
 }
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java b/hudi-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java
index 0defefe2ea..69b183fa09 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableConfig.java
@@ -63,7 +63,7 @@ public class TestHoodieTableConfig extends HoodieCommonTestHarness {
   @Test
   public void testCreate() throws IOException {
     assertTrue(fs.exists(new Path(metaPath, HoodieTableConfig.HOODIE_PROPERTIES_FILE)));
-    HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null);
+    HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null, null);
     assertEquals(6, config.getProps().size());
   }
 
@@ -76,7 +76,7 @@ public class TestHoodieTableConfig extends HoodieCommonTestHarness {
 
     assertTrue(fs.exists(cfgPath));
     assertFalse(fs.exists(backupCfgPath));
-    HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null);
+    HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null, null);
     assertEquals(7, config.getProps().size());
     assertEquals("test-table2", config.getTableName());
     assertEquals("new_field", config.getPreCombineField());
@@ -89,7 +89,7 @@ public class TestHoodieTableConfig extends HoodieCommonTestHarness {
 
     assertTrue(fs.exists(cfgPath));
     assertFalse(fs.exists(backupCfgPath));
-    HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null);
+    HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null, null);
     assertEquals(5, config.getProps().size());
     assertNull(config.getProps().getProperty("hoodie.invalid.config"));
     assertFalse(config.getProps().contains(HoodieTableConfig.ARCHIVELOG_FOLDER.key()));
@@ -99,13 +99,13 @@ public class TestHoodieTableConfig extends HoodieCommonTestHarness {
   public void testReadsWhenPropsFileDoesNotExist() throws IOException {
     fs.delete(cfgPath, false);
     assertThrows(HoodieIOException.class, () -> {
-      new HoodieTableConfig(fs, metaPath.toString(), null);
+      new HoodieTableConfig(fs, metaPath.toString(), null, null);
     });
   }
 
   @Test
   public void testReadsWithUpdateFailures() throws IOException {
-    HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null);
+    HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null, null);
     fs.delete(cfgPath, false);
     try (FSDataOutputStream out = fs.create(backupCfgPath)) {
       config.getProps().store(out, "");
@@ -113,14 +113,14 @@ public class TestHoodieTableConfig extends HoodieCommonTestHarness {
 
     assertFalse(fs.exists(cfgPath));
     assertTrue(fs.exists(backupCfgPath));
-    config = new HoodieTableConfig(fs, metaPath.toString(), null);
+    config = new HoodieTableConfig(fs, metaPath.toString(), null, null);
     assertEquals(6, config.getProps().size());
   }
 
   @ParameterizedTest
   @ValueSource(booleans = {true, false})
   public void testUpdateRecovery(boolean shouldPropsFileExist) throws IOException {
-    HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null);
+    HoodieTableConfig config = new HoodieTableConfig(fs, metaPath.toString(), null, null);
     if (!shouldPropsFileExist) {
       fs.delete(cfgPath, false);
     }
@@ -131,7 +131,7 @@ public class TestHoodieTableConfig extends HoodieCommonTestHarness {
     HoodieTableConfig.recoverIfNeeded(fs, cfgPath, backupCfgPath);
     assertTrue(fs.exists(cfgPath));
     assertFalse(fs.exists(backupCfgPath));
-    config = new HoodieTableConfig(fs, metaPath.toString(), null);
+    config = new HoodieTableConfig(fs, metaPath.toString(), null, null);
     assertEquals(6, config.getProps().size());
   }
 }
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/HoodieRecordUtilsTest.java b/hudi-common/src/test/java/org/apache/hudi/common/util/HoodieRecordUtilsTest.java
index 0c51571c9e..4dcabecccb 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/util/HoodieRecordUtilsTest.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/util/HoodieRecordUtilsTest.java
@@ -20,8 +20,8 @@ package org.apache.hudi.common.util;
 
 import org.apache.avro.generic.GenericRecord;
 import org.apache.hudi.common.model.DefaultHoodieRecordPayload;
-import org.apache.hudi.common.model.HoodieAvroRecordMerge;
-import org.apache.hudi.common.model.HoodieMerge;
+import org.apache.hudi.common.model.HoodieAvroRecordMerger;
+import org.apache.hudi.common.model.HoodieRecordMerger;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.junit.jupiter.api.Test;
 
@@ -31,11 +31,11 @@ class HoodieRecordUtilsTest {
 
   @Test
   void loadHoodieMerge() {
-    String mergeClassName = HoodieAvroRecordMerge.class.getName();
-    HoodieMerge merge1 = HoodieRecordUtils.loadMerge(mergeClassName);
-    HoodieMerge merge2 = HoodieRecordUtils.loadMerge(mergeClassName);
-    assertEquals(merge1.getClass().getName(), mergeClassName);
-    assertEquals(merge1, merge2);
+    String mergeClassName = HoodieAvroRecordMerger.class.getName();
+    HoodieRecordMerger recordMerger1 = HoodieRecordUtils.loadRecordMerger(mergeClassName);
+    HoodieRecordMerger recordMerger2 = HoodieRecordUtils.loadRecordMerger(mergeClassName);
+    assertEquals(recordMerger1.getClass().getName(), mergeClassName);
+    assertEquals(recordMerger1, recordMerger2);
   }
 
   @Test
diff --git a/hudi-common/src/test/java/org/apache/hudi/io/storage/TestHoodieAvroFileReaderFactory.java b/hudi-common/src/test/java/org/apache/hudi/io/storage/TestHoodieAvroFileReaderFactory.java
index f049033688..c306bab384 100644
--- a/hudi-common/src/test/java/org/apache/hudi/io/storage/TestHoodieAvroFileReaderFactory.java
+++ b/hudi-common/src/test/java/org/apache/hudi/io/storage/TestHoodieAvroFileReaderFactory.java
@@ -18,6 +18,8 @@
 
 package org.apache.hudi.io.storage;
 
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.junit.jupiter.api.Test;
@@ -40,19 +42,19 @@ public class TestHoodieAvroFileReaderFactory {
     // parquet file format.
     final Configuration hadoopConf = new Configuration();
     final Path parquetPath = new Path("/partition/path/f1_1-0-1_000.parquet");
-    HoodieAvroFileReader parquetReader = HoodieFileReaderFactory.getFileReader(hadoopConf, parquetPath);
+    HoodieFileReader parquetReader = HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(hadoopConf, parquetPath);
     assertTrue(parquetReader instanceof HoodieAvroParquetReader);
 
     // log file format.
     final Path logPath = new Path("/partition/path/f.b51192a8-574b-4a85-b246-bcfec03ac8bf_100.log.2_1-0-1");
     final Throwable thrown = assertThrows(UnsupportedOperationException.class, () -> {
-      HoodieAvroFileReader logWriter = HoodieFileReaderFactory.getFileReader(hadoopConf, logPath);
+      HoodieFileReader logWriter = HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(hadoopConf, logPath);
     }, "should fail since log storage reader is not supported yet.");
     assertTrue(thrown.getMessage().contains("format not supported yet."));
 
     // Orc file format.
     final Path orcPath = new Path("/partition/path/f1_1-0-1_000.orc");
-    HoodieAvroFileReader orcReader = HoodieFileReaderFactory.getFileReader(hadoopConf, orcPath);
+    HoodieFileReader orcReader = HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(hadoopConf, orcPath);
     assertTrue(orcReader instanceof HoodieAvroOrcReader);
   }
 }
diff --git a/hudi-examples/hudi-examples-flink/src/test/java/org/apache/hudi/examples/quickstart/TestQuickstartData.java b/hudi-examples/hudi-examples-flink/src/test/java/org/apache/hudi/examples/quickstart/TestQuickstartData.java
index 7fc93c776f..ab92990718 100644
--- a/hudi-examples/hudi-examples-flink/src/test/java/org/apache/hudi/examples/quickstart/TestQuickstartData.java
+++ b/hudi-examples/hudi-examples-flink/src/test/java/org/apache/hudi/examples/quickstart/TestQuickstartData.java
@@ -38,7 +38,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hudi.common.config.HoodieCommonConfig;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieAvroRecordMerger;
 import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
+import org.apache.hudi.common.util.HoodieRecordUtils;
 import org.apache.hudi.examples.quickstart.utils.QuickstartConfigurations;
 import org.apache.parquet.Strings;
 import org.apache.parquet.avro.AvroParquetReader;
@@ -360,6 +362,7 @@ public class TestQuickstartData {
         .withSpillableMapBasePath("/tmp/")
         .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue())
         .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue())
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
   }
 
diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
index 9957e0c980..b969074f60 100644
--- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
+++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
@@ -23,10 +23,11 @@ import org.apache.hudi.common.config.ConfigClassProperty;
 import org.apache.hudi.common.config.ConfigGroups;
 import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.common.model.EventTimeAvroPayload;
-import org.apache.hudi.common.model.HoodieAvroRecordMerge;
+import org.apache.hudi.common.model.HoodieAvroRecordMerger;
 import org.apache.hudi.common.model.HoodieCleaningPolicy;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.model.WriteOperationType;
+import org.apache.hudi.common.util.StringUtils;
 import org.apache.hudi.config.HoodieIndexConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieException;
@@ -116,6 +117,20 @@ public class FlinkOptions extends HoodieConfig {
       .withDescription("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.\n"
           + "This will render any value set for the option in-effective");
 
+  public static final ConfigOption<String> RECORD_MERGER_IMPLS = ConfigOptions
+      .key("record.merger.impls")
+      .stringType()
+      .defaultValue(HoodieAvroRecordMerger.class.getName())
+      .withDescription("List of HoodieMerger implementations constituting Hudi's merging strategy -- based on the engine used. "
+          + "These merger impls will filter by record.merger.strategy. "
+          + "Hudi will pick most efficient implementation to perform merging/combining of the records (during update, reading MOR table, etc)");
+
+  public static final ConfigOption<String> RECORD_MERGER_STRATEGY = ConfigOptions
+      .key("record.merger.strategy")
+      .stringType()
+      .defaultValue(StringUtils.DEFAULT_MERGER_STRATEGY_UUID)
+      .withDescription("Id of merger strategy.  Hudi will pick RecordMergers in record.merger.impls which has the same merger strategy id");
+
   public static final ConfigOption<String> PARTITION_DEFAULT_NAME = ConfigOptions
       .key("partition.default_name")
       .stringType()
@@ -301,13 +316,6 @@ public class FlinkOptions extends HoodieConfig {
       .defaultValue(WriteOperationType.UPSERT.value())
       .withDescription("The write operation, that this write should do");
 
-  public static final ConfigOption<String> MERGE_CLASS_NAME = ConfigOptions
-      .key("write.merge.class")
-      .stringType()
-      .defaultValue(HoodieAvroRecordMerge.class.getName())
-      .withDescription("Merge class provide stateless component interface for merging records, and support various HoodieRecord "
-          + "types, such as Spark records or Flink records.");
-
   /**
    * Flag to indicate whether to drop duplicates before insert/upsert.
    * By default false to gain extra performance.
diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java
index b6b65bb8f1..a9f88cd028 100644
--- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java
+++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteFunction.java
@@ -23,11 +23,10 @@ import org.apache.hudi.common.model.HoodieAvroRecord;
 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.model.HoodieMerge;
+import org.apache.hudi.common.model.HoodieRecordMerger;
 import org.apache.hudi.common.model.HoodieRecordLocation;
 import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.model.WriteOperationType;
-import org.apache.hudi.common.util.HoodieRecordUtils;
 import org.apache.hudi.common.util.ObjectSizeCalculator;
 import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.hudi.configuration.FlinkOptions;
@@ -52,6 +51,7 @@ import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Properties;
 import java.util.Random;
 import java.util.function.BiFunction;
 import java.util.stream.Collectors;
@@ -104,7 +104,7 @@ public class StreamWriteFunction<I> extends AbstractStreamWriteFunction<I> {
 
   private transient BiFunction<List<HoodieRecord>, String, List<WriteStatus>> writeFunction;
 
-  private transient HoodieMerge merge;
+  private transient HoodieRecordMerger recordMerger;
 
   /**
    * Total size tracer.
@@ -201,9 +201,8 @@ public class StreamWriteFunction<I> extends AbstractStreamWriteFunction<I> {
   }
 
   private void initMergeClass() {
-    String mergeClassName = metaClient.getTableConfig().getMergeClass();
-    LOG.info("init hoodie merge with class [{}]", mergeClassName);
-    merge = HoodieRecordUtils.loadMerge(mergeClassName);
+    recordMerger = writeClient.getConfig().getRecordMerger();
+    LOG.info("init hoodie merge with class [{}]", recordMerger.getClass().getName());
   }
 
   /**
@@ -432,7 +431,9 @@ public class StreamWriteFunction<I> extends AbstractStreamWriteFunction<I> {
     List<HoodieRecord> records = bucket.writeBuffer();
     ValidationUtils.checkState(records.size() > 0, "Data bucket to flush has no buffering records");
     if (config.getBoolean(FlinkOptions.PRE_COMBINE)) {
-      records = FlinkWriteHelper.newInstance().deduplicateRecords(records, (HoodieIndex) null, -1, merge);
+      Properties props = new Properties();
+      config.addAllToProperties(props);
+      records = FlinkWriteHelper.newInstance().deduplicateRecords(records, (HoodieIndex) null, -1, recordMerger, props);
     }
     bucket.preWrite(records);
     final List<WriteStatus> writeStatus = new ArrayList<>(writeFunction.apply(records, instant));
@@ -467,7 +468,9 @@ public class StreamWriteFunction<I> extends AbstractStreamWriteFunction<I> {
             List<HoodieRecord> records = bucket.writeBuffer();
             if (records.size() > 0) {
               if (config.getBoolean(FlinkOptions.PRE_COMBINE)) {
-                records = FlinkWriteHelper.newInstance().deduplicateRecords(records, (HoodieIndex) null, -1, merge);
+                Properties props = new Properties();
+                config.addAllToProperties(props);
+                records = FlinkWriteHelper.newInstance().deduplicateRecords(records, (HoodieIndex) null, -1, recordMerger, props);
               }
               bucket.preWrite(records);
               writeStatus.addAll(writeFunction.apply(records, currentInstant));
diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java
index 5fd2510fd5..da4504c8e6 100644
--- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java
+++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringOperator.java
@@ -31,7 +31,7 @@ import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.StringUtils;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodieStorageConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.configuration.FlinkOptions;
 import org.apache.hudi.configuration.OptionsResolver;
@@ -39,6 +39,7 @@ import org.apache.hudi.exception.HoodieClusteringException;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.io.IOUtils;
 import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 import org.apache.hudi.sink.bulk.BulkInsertWriterHelper;
 import org.apache.hudi.sink.bulk.sort.SortOperatorGen;
@@ -262,9 +263,9 @@ public class ClusteringOperator extends TableStreamOperator<ClusteringCommitEven
 
     for (ClusteringOperation clusteringOp : clusteringOps) {
       try {
-        Option<HoodieAvroFileReader> baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath())
+        Option<HoodieFileReader> baseFileReader = StringUtils.isNullOrEmpty(clusteringOp.getDataFilePath())
             ? Option.empty()
-            : Option.of(HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())));
+            : Option.of(HoodieFileReaderFactory.getReaderFactory(table.getConfig().getRecordMerger().getRecordType()).getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())));
         HoodieMergedLogRecordScanner scanner = HoodieMergedLogRecordScanner.newBuilder()
             .withFileSystem(table.getMetaClient().getFs())
             .withBasePath(table.getMetaClient().getBasePath())
@@ -278,18 +279,18 @@ public class ClusteringOperator extends TableStreamOperator<ClusteringCommitEven
             .withSpillableMapBasePath(writeConfig.getSpillableMapBasePath())
             .withDiskMapType(writeConfig.getCommonConfig().getSpillableDiskMapType())
             .withBitCaskDiskMapCompressionEnabled(writeConfig.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
+            .withRecordMerger(writeConfig.getRecordMerger())
             .build();
 
         HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig();
         HoodieFileSliceReader<? extends IndexedRecord> hoodieFileSliceReader = HoodieFileSliceReader.getFileSliceReader(baseFileReader, scanner, readerSchema,
-            tableConfig.getPayloadClass(),
-            tableConfig.getPreCombineField(),
+            tableConfig.getProps(),
             tableConfig.populateMetaFields() ? Option.empty() : Option.of(Pair.of(tableConfig.getRecordKeyFieldProp(),
                 tableConfig.getPartitionFieldProp())));
 
         recordIterators.add(StreamSupport.stream(Spliterators.spliteratorUnknownSize(hoodieFileSliceReader, Spliterator.NONNULL), false).map(hoodieRecord -> {
           try {
-            return this.transform(hoodieRecord.toIndexedRecord(readerSchema, new Properties()).get());
+            return this.transform(hoodieRecord.toIndexedRecord(readerSchema, new Properties()).get().getData());
           } catch (IOException e) {
             throw new HoodieIOException("Failed to read next record", e);
           }
@@ -310,7 +311,8 @@ public class ClusteringOperator extends TableStreamOperator<ClusteringCommitEven
     List<Iterator<RowData>> iteratorsForPartition = clusteringOps.stream().map(clusteringOp -> {
       Iterable<IndexedRecord> indexedRecords = () -> {
         try {
-          return HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath())).getRecordIterator(readerSchema);
+          return ((HoodieAvroFileReader)HoodieFileReaderFactory.getReaderFactory(table.getConfig().getRecordMerger().getRecordType())
+              .getFileReader(table.getHadoopConf(), new Path(clusteringOp.getDataFilePath()))).getIndexedRecordIterator(readerSchema);
         } catch (IOException e) {
           throw new HoodieClusteringException("Error reading input data for " + clusteringOp.getDataFilePath()
               + " and " + clusteringOp.getDeltaFilePaths(), e);
diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java
index db13a6c2ae..2dd89e96da 100644
--- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java
+++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java
@@ -19,6 +19,7 @@
 package org.apache.hudi.sink.partitioner.profile;
 
 import org.apache.hudi.client.common.HoodieFlinkEngineContext;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieBaseFile;
 import org.apache.hudi.common.model.HoodieCommitMetadata;
@@ -79,7 +80,7 @@ public class WriteProfile {
 
   /**
    * Total records to write for each bucket based on
-   * the config option {@link org.apache.hudi.config.HoodieStorageConfig#PARQUET_MAX_FILE_SIZE}.
+   * the config option {@link HoodieStorageConfig#PARQUET_MAX_FILE_SIZE}.
    */
   private long recordsPerBucket;
 
diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java
index 3143f55c63..517e57be26 100644
--- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java
+++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java
@@ -19,7 +19,7 @@
 package org.apache.hudi.streamer;
 
 import org.apache.hudi.client.utils.OperationConverter;
-import org.apache.hudi.common.model.HoodieAvroRecordMerge;
+import org.apache.hudi.common.model.HoodieAvroRecordMerger;
 import org.apache.hudi.common.model.HoodieCleaningPolicy;
 import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
 import org.apache.hudi.common.model.WriteOperationType;
@@ -119,9 +119,13 @@ public class FlinkStreamerConfig extends Configuration {
       + "a GenericRecord. Implement your own, if you want to do something other than overwriting existing value.")
   public String payloadClassName = OverwriteWithLatestAvroPayload.class.getName();
 
-  @Parameter(names = {"--merge-class"}, description = "Implements of HoodieMerge, that defines how to merge two records."
-      + "Implement your own, if you want to implement specific record merge logic.")
-  public String mergeClassName = HoodieAvroRecordMerge.class.getName();
+  @Parameter(names = {"--merger-impls"}, description = "List of HoodieMerger implementations constituting Hudi's merging strategy -- based on the engine used. "
+      + "These merger impls will filter by merger-strategy "
+      + "Hudi will pick most efficient implementation to perform merging/combining of the records (during update, reading MOR table, etc)")
+  public String mergerImpls = HoodieAvroRecordMerger.class.getName();
+
+  @Parameter(names = {"--merger-strategy"}, description = "Id of merger strategy. Hudi will pick RecordMergers in merger-impls which has the same merger strategy id")
+  public String mergerStrategy = StringUtils.DEFAULT_MERGER_STRATEGY_UUID;
 
   @Parameter(names = {"--op"}, description = "Takes one of these values : UPSERT (default), INSERT (use when input "
       + "is purely new data/inserts to gain speed).", converter = OperationConverter.class)
@@ -368,7 +372,8 @@ public class FlinkStreamerConfig extends Configuration {
     conf.setString(FlinkOptions.OPERATION, config.operation.value());
     conf.setString(FlinkOptions.PRECOMBINE_FIELD, config.sourceOrderingField);
     conf.setString(FlinkOptions.PAYLOAD_CLASS_NAME, config.payloadClassName);
-    conf.setString(FlinkOptions.MERGE_CLASS_NAME, config.mergeClassName);
+    conf.setString(FlinkOptions.RECORD_MERGER_IMPLS, config.mergerImpls);
+    conf.setString(FlinkOptions.RECORD_MERGER_STRATEGY, config.mergerStrategy);
     conf.setBoolean(FlinkOptions.PRE_COMBINE, config.preCombine);
     conf.setInteger(FlinkOptions.RETRY_TIMES, Integer.parseInt(config.instantRetryTimes));
     conf.setLong(FlinkOptions.RETRY_INTERVAL_MS, Long.parseLong(config.instantRetryInterval));
diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java
index 5e8239cf6d..6fac5e4b88 100644
--- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java
+++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java
@@ -423,8 +423,7 @@ public class HoodieTableSource implements
         tableAvroSchema.toString(),
         AvroSchemaConverter.convertToSchema(requiredRowType).toString(),
         inputSplits,
-        conf.getString(FlinkOptions.RECORD_KEY_FIELD).split(","),
-        conf.getString(FlinkOptions.MERGE_CLASS_NAME));
+        conf.getString(FlinkOptions.RECORD_KEY_FIELD).split(","));
     return MergeOnReadInputFormat.builder()
         .config(this.conf)
         .tableState(hoodieTableState)
diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java
index 8adbde355c..00fa1b34d8 100644
--- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java
+++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/FormatUtils.java
@@ -18,13 +18,17 @@
 
 package org.apache.hudi.table.format;
 
+import java.util.stream.Collectors;
+import org.apache.hudi.common.engine.EngineType;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieOperation;
 import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordMerger;
 import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
 import org.apache.hudi.common.table.log.HoodieUnMergedLogRecordScanner;
 import org.apache.hudi.common.util.DefaultSizeEstimator;
 import org.apache.hudi.common.util.Functions;
+import org.apache.hudi.common.util.HoodieRecordUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
 import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer;
@@ -140,6 +144,7 @@ public class FormatUtils {
         .withSpillableMapBasePath(writeConfig.getSpillableMapBasePath())
         .withInstantRange(split.getInstantRange())
         .withOperationField(flinkConf.getBoolean(FlinkOptions.CHANGELOG_ENABLED))
+        .withRecordMerger(writeConfig.getRecordMerger())
         .build();
   }
 
@@ -149,6 +154,12 @@ public class FormatUtils {
       org.apache.flink.configuration.Configuration flinkConf,
       Configuration hadoopConf,
       HoodieUnMergedLogRecordScanner.LogRecordScannerCallback callback) {
+    List<String> mergers = Arrays.stream(flinkConf.getString(FlinkOptions.RECORD_MERGER_IMPLS).split(","))
+        .map(String::trim)
+        .distinct()
+        .collect(Collectors.toList());
+    HoodieRecordMerger merger = HoodieRecordUtils.generateRecordMerger(
+        split.getTablePath(), EngineType.FLINK, mergers, flinkConf.getString(FlinkOptions.RECORD_MERGER_STRATEGY));
     FileSystem fs = FSUtils.getFs(split.getTablePath(), hadoopConf);
     return HoodieUnMergedLogRecordScanner.newBuilder()
         .withFileSystem(fs)
@@ -166,6 +177,7 @@ public class FormatUtils {
                 HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE))
         .withInstantRange(split.getInstantRange())
         .withLogRecordScannerCallback(callback)
+        .withRecordMerger(merger)
         .build();
   }
 
@@ -243,6 +255,7 @@ public class FormatUtils {
         .withSpillableMapBasePath(writeConfig.getSpillableMapBasePath())
         .withDiskMapType(writeConfig.getCommonConfig().getSpillableDiskMapType())
         .withBitCaskDiskMapCompressionEnabled(writeConfig.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
+        .withRecordMerger(writeConfig.getRecordMerger())
         .build();
   }
 
diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java
index 09f4984c77..776b20fde2 100644
--- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java
+++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputFormat.java
@@ -18,11 +18,13 @@
 
 package org.apache.hudi.table.format.mor;
 
+import java.util.stream.Collectors;
+import org.apache.hudi.common.engine.EngineType;
 import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieAvroRecord;
 import org.apache.hudi.common.model.HoodieOperation;
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieMerge;
+import org.apache.hudi.common.model.HoodieRecordMerger;
 import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
 import org.apache.hudi.common.table.log.InstantRange;
 import org.apache.hudi.common.util.ClosableIterator;
@@ -205,8 +207,7 @@ public class MergeOnReadInputFormat
           this.requiredPos,
           this.emitDelete,
           this.tableState.getOperationPos(),
-          getFullSchemaReader(split.getBasePath().get()),
-          tableState.getMergeClass());
+          getFullSchemaReader(split.getBasePath().get()));
     } else {
       throw new HoodieException("Unable to select an Iterator to read the Hoodie MOR File Split for "
           + "file path: " + split.getBasePath()
@@ -644,7 +645,7 @@ public class MergeOnReadInputFormat
 
     private final InstantRange instantRange;
 
-    private final HoodieMerge merge;
+    private final HoodieRecordMerger recordMerger;
 
     // add the flag because the flink ParquetColumnarRowSplitReader is buggy:
     // method #reachedEnd() returns false after it returns true.
@@ -668,8 +669,7 @@ public class MergeOnReadInputFormat
         int[] requiredPos,
         boolean emitDelete,
         int operationPos,
-        ParquetColumnarRowSplitReader reader, // the reader should be with full schema
-        String mergeClass) {
+        ParquetColumnarRowSplitReader reader) { // the reader should be with full schema
       this.tableSchema = tableSchema;
       this.reader = reader;
       this.scanner = FormatUtils.logScanner(split, tableSchema, flinkConf, hadoopConf);
@@ -684,7 +684,11 @@ public class MergeOnReadInputFormat
       this.avroToRowDataConverter = AvroToRowDataConverters.createRowConverter(requiredRowType);
       this.projection = RowDataProjection.instance(requiredRowType, requiredPos);
       this.instantRange = split.getInstantRange().orElse(null);
-      this.merge = HoodieRecordUtils.loadMerge(mergeClass);
+      List<String> mergers = Arrays.stream(flinkConf.getString(FlinkOptions.RECORD_MERGER_IMPLS).split(","))
+          .map(String::trim)
+          .distinct()
+          .collect(Collectors.toList());
+      this.recordMerger = HoodieRecordUtils.generateRecordMerger(split.getTablePath(), EngineType.FLINK, mergers, flinkConf.getString(FlinkOptions.RECORD_MERGER_STRATEGY));
     }
 
     @Override
@@ -701,18 +705,18 @@ public class MergeOnReadInputFormat
         final String curKey = currentRecord.getString(HOODIE_RECORD_KEY_COL_POS).toString();
         if (scanner.getRecords().containsKey(curKey)) {
           keyToSkip.add(curKey);
-          Option<IndexedRecord> mergedAvroRecord = mergeRowWithLog(currentRecord, curKey);
+          Option<HoodieAvroIndexedRecord> mergedAvroRecord = mergeRowWithLog(currentRecord, curKey);
           if (!mergedAvroRecord.isPresent()) {
             // deleted
             continue;
           } else {
-            final RowKind rowKind = FormatUtils.getRowKindSafely(mergedAvroRecord.get(), this.operationPos);
+            final RowKind rowKind = FormatUtils.getRowKindSafely(mergedAvroRecord.get().getData(), this.operationPos);
             if (!emitDelete && rowKind == RowKind.DELETE) {
               // deleted
               continue;
             }
             GenericRecord avroRecord = buildAvroRecordBySchema(
-                mergedAvroRecord.get(),
+                mergedAvroRecord.get().getData(),
                 requiredSchema,
                 requiredPos,
                 recordBuilder);
@@ -770,13 +774,14 @@ public class MergeOnReadInputFormat
       }
     }
 
-    private Option<IndexedRecord> mergeRowWithLog(
+    private Option<HoodieAvroIndexedRecord> mergeRowWithLog(
         RowData curRow,
         String curKey) throws IOException {
       final HoodieAvroRecord<?> record = (HoodieAvroRecord) scanner.getRecords().get(curKey);
       GenericRecord historyAvroRecord = (GenericRecord) rowDataToAvroConverter.convert(tableSchema, curRow);
-      Option<HoodieRecord> resultRecord = merge.combineAndGetUpdateValue(new HoodieAvroIndexedRecord(historyAvroRecord), record, tableSchema, payloadProps);
-      return ((HoodieAvroIndexedRecord) resultRecord.get()).toIndexedRecord();
+      HoodieAvroIndexedRecord hoodieAvroIndexedRecord = new HoodieAvroIndexedRecord(historyAvroRecord);
+      Option<HoodieRecord> resultRecord = recordMerger.merge(hoodieAvroIndexedRecord, record, tableSchema, payloadProps);
+      return resultRecord.get().toIndexedRecord(tableSchema, new Properties());
     }
   }
 
diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadTableState.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadTableState.java
index bbb21db7f8..36dfecbb79 100644
--- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadTableState.java
+++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadTableState.java
@@ -41,7 +41,6 @@ public class MergeOnReadTableState implements Serializable {
   private final List<MergeOnReadInputSplit> inputSplits;
   private final String[] pkFields;
   private final int operationPos;
-  private final String mergeClass;
 
   public MergeOnReadTableState(
       RowType rowType,
@@ -49,8 +48,7 @@ public class MergeOnReadTableState implements Serializable {
       String avroSchema,
       String requiredAvroSchema,
       List<MergeOnReadInputSplit> inputSplits,
-      String[] pkFields,
-      String mergeClass) {
+      String[] pkFields) {
     this.rowType = rowType;
     this.requiredRowType = requiredRowType;
     this.avroSchema = avroSchema;
@@ -58,7 +56,6 @@ public class MergeOnReadTableState implements Serializable {
     this.inputSplits = inputSplits;
     this.pkFields = pkFields;
     this.operationPos = rowType.getFieldIndex(HoodieRecord.OPERATION_METADATA_FIELD);
-    this.mergeClass = mergeClass;
   }
 
   public RowType getRowType() {
@@ -85,10 +82,6 @@ public class MergeOnReadTableState implements Serializable {
     return operationPos;
   }
 
-  public String getMergeClass() {
-    return mergeClass;
-  }
-
   public int[] getRequiredPositions() {
     final List<String> fieldNames = rowType.getFieldNames();
     return requiredRowType.getFieldNames().stream()
diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
index 85ea097c64..f844f08c67 100644
--- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
+++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
@@ -26,6 +26,7 @@ import org.apache.hudi.common.config.DFSPropertiesConfiguration;
 import org.apache.hudi.common.config.HoodieMetadataConfig;
 import org.apache.hudi.common.config.SerializableConfiguration;
 import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.config.HoodieStorageConfig;
 import org.apache.hudi.common.engine.EngineType;
 import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.HoodieCleaningPolicy;
@@ -47,7 +48,6 @@ import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieLockConfig;
 import org.apache.hudi.config.HoodieMemoryConfig;
 import org.apache.hudi.config.HoodiePayloadConfig;
-import org.apache.hudi.config.HoodieStorageConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.configuration.FlinkOptions;
 import org.apache.hudi.configuration.HadoopConfigurations;
@@ -195,7 +195,6 @@ public class StreamerUtil {
                 .archiveCommitsWith(conf.getInteger(FlinkOptions.ARCHIVE_MIN_COMMITS), conf.getInteger(FlinkOptions.ARCHIVE_MAX_COMMITS))
                 .build())
             .withCompactionConfig(HoodieCompactionConfig.newBuilder()
-                .withMergeClass(conf.getString(FlinkOptions.MERGE_CLASS_NAME))
                 .withTargetIOPerCompactionInMB(conf.getLong(FlinkOptions.COMPACTION_TARGET_IO))
                 .withInlineCompactionTriggerStrategy(
                     CompactionTriggerStrategy.valueOf(conf.getString(FlinkOptions.COMPACTION_TRIGGER_STRATEGY).toUpperCase(Locale.ROOT)))
@@ -232,6 +231,8 @@ public class StreamerUtil {
             .withEmbeddedTimelineServerReuseEnabled(true) // make write client embedded timeline service singleton
             .withAutoCommit(false)
             .withAllowOperationMetadataField(conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED))
+            .withMergerImpls(conf.getString(FlinkOptions.RECORD_MERGER_IMPLS))
+            .withMergerStrategy(conf.getString(FlinkOptions.RECORD_MERGER_STRATEGY))
             .withProps(flinkConf2TypedProperties(conf))
             .withSchema(getSourceSchema(conf).toString());
 
@@ -310,7 +311,6 @@ public class StreamerUtil {
           .setTableName(conf.getString(FlinkOptions.TABLE_NAME))
           .setRecordKeyFields(conf.getString(FlinkOptions.RECORD_KEY_FIELD, null))
           .setPayloadClassName(conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME))
-          .setMergeClassName(conf.getString(FlinkOptions.MERGE_CLASS_NAME))
           .setPreCombineField(OptionsResolver.getPreCombineField(conf))
           .setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue())
           .setPartitionFields(conf.getString(FlinkOptions.PARTITION_PATH_FIELD, null))
diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadOperator.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadOperator.java
index f2095c1844..63d5c1f6bd 100644
--- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadOperator.java
+++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadOperator.java
@@ -263,8 +263,7 @@ public class TestStreamReadOperator {
         tableAvroSchema.toString(),
         AvroSchemaConverter.convertToSchema(TestConfigurations.ROW_TYPE).toString(),
         Collections.emptyList(),
-        new String[0],
-        metaClient.getTableConfig().getMergeClass());
+        new String[0]);
     MergeOnReadInputFormat inputFormat = MergeOnReadInputFormat.builder()
         .config(conf)
         .tableState(hoodieTableState)
diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java
index 0338895c14..88d488a5bc 100644
--- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java
+++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java
@@ -24,6 +24,7 @@ import org.apache.hudi.common.fs.FSUtils;
 import org.apache.hudi.common.model.BaseFile;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieAvroRecordMerger;
 import org.apache.hudi.common.model.HoodieLogFile;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
@@ -31,6 +32,7 @@ import org.apache.hudi.common.table.TableSchemaResolver;
 import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.testutils.HoodieTestUtils;
+import org.apache.hudi.common.util.HoodieRecordUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.configuration.FlinkOptions;
@@ -771,6 +773,7 @@ public class TestData {
         .withSpillableMapBasePath("/tmp/")
         .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue())
         .withBitCaskDiskMapCompressionEnabled(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue())
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
   }
 
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileRecordReader.java
index 4fa62e79ea..5b7e7fbc67 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileRecordReader.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileRecordReader.java
@@ -56,7 +56,7 @@ public class HoodieHFileRecordReader implements RecordReader<NullWritable, Array
   @Override
   public boolean next(NullWritable key, ArrayWritable value) throws IOException {
     if (recordIterator == null) {
-      recordIterator = reader.getRecordIterator(schema);
+      recordIterator = reader.getIndexedRecordIterator(schema);
     }
 
     if (!recordIterator.hasNext()) {
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputSplitUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputSplitUtils.java
index ae550cb335..f2437eb6f6 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputSplitUtils.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/InputSplitUtils.java
@@ -22,9 +22,10 @@ import org.apache.avro.Schema;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.FileSplit;
 import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 
 import java.io.DataInput;
@@ -64,7 +65,8 @@ public class InputSplitUtils {
   public static Schema getBaseFileSchema(FileSplit split, Configuration conf) {
     try {
       if (split instanceof BootstrapBaseFileSplit) {
-        HoodieAvroFileReader storageReader = HoodieFileReaderFactory.getFileReader(conf,
+        // TODO mr support other record type
+        HoodieFileReader storageReader = HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(conf,
             ((BootstrapBaseFileSplit)(split)).getBootstrapFileSplit().getPath());
         return HoodieAvroUtils.addMetadataFields(storageReader.getSchema());
       }
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java
index 4fa3781aea..c2b8864bc3 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java
@@ -27,8 +27,11 @@ import org.apache.hadoop.mapred.RecordReader;
 import org.apache.hudi.avro.HoodieAvroUtils;
 import org.apache.hudi.common.config.HoodieCommonConfig;
 import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
+import org.apache.hudi.common.model.HoodieAvroRecordMerger;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
+import org.apache.hudi.common.util.HoodieRecordUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
 import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
@@ -89,10 +92,11 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader
         .withDiskMapType(jobConf.getEnum(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key(), HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue()))
         .withBitCaskDiskMapCompressionEnabled(jobConf.getBoolean(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(),
             HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue()))
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
   }
 
-  private Option<GenericRecord> buildGenericRecordwithCustomPayload(HoodieRecord record) throws IOException {
+  private Option<HoodieAvroIndexedRecord> buildGenericRecordwithCustomPayload(HoodieRecord record) throws IOException {
     if (usesCustomPayload) {
       return record.toIndexedRecord(getWriterSchema(), payloadProps);
     } else {
@@ -112,7 +116,7 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader
           this.deltaRecordKeys.remove(key);
           // TODO(NA): Invoke preCombine here by converting arrayWritable to Avro. This is required since the
           // deltaRecord may not be a full record and needs values of columns from the parquet
-          Option<GenericRecord> rec = buildGenericRecordwithCustomPayload(deltaRecordMap.get(key));
+          Option<HoodieAvroIndexedRecord> rec = buildGenericRecordwithCustomPayload(deltaRecordMap.get(key));
           // If the record is not present, this is a delete record using an empty payload so skip this base record
           // and move to the next record
           if (!rec.isPresent()) {
@@ -129,7 +133,7 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader
     }
     while (this.deltaItr.hasNext()) {
       final String key = this.deltaItr.next();
-      Option<GenericRecord> rec = buildGenericRecordwithCustomPayload(deltaRecordMap.get(key));
+      Option<HoodieAvroIndexedRecord> rec = buildGenericRecordwithCustomPayload(deltaRecordMap.get(key));
       if (rec.isPresent()) {
         setUpWritable(rec, arrayWritable, key);
         return true;
@@ -138,12 +142,12 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader
     return false;
   }
 
-  private void setUpWritable(Option<GenericRecord> rec, ArrayWritable arrayWritable, String key) {
-    GenericRecord recordToReturn = rec.get();
+  private void setUpWritable(Option<HoodieAvroIndexedRecord> rec, ArrayWritable arrayWritable, String key) {
+    GenericRecord recordToReturn = (GenericRecord) rec.get().getData();
     if (usesCustomPayload) {
       // If using a custom payload, return only the projection fields. The readerSchema is a schema derived from
       // the writerSchema with only the projection fields
-      recordToReturn = HoodieAvroUtils.rewriteRecord(rec.get(), getReaderSchema());
+      recordToReturn = HoodieAvroUtils.rewriteRecord((GenericRecord) rec.get().getData(), getReaderSchema());
     }
     // we assume, a later safe record in the log, is newer than what we have in the map &
     // replace it. Since we want to return an arrayWritable which is the same length as the elements in the latest
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java
index e418bebb50..db187f18cf 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeUnmergedRecordReader.java
@@ -19,9 +19,11 @@
 package org.apache.hudi.hadoop.realtime;
 
 import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieAvroRecordMerger;
 import org.apache.hudi.common.table.log.HoodieUnMergedLogRecordScanner;
 import org.apache.hudi.common.util.DefaultSizeEstimator;
 import org.apache.hudi.common.util.Functions;
+import org.apache.hudi.common.util.HoodieRecordUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
 import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer;
@@ -92,10 +94,11 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader
         .withBufferSize(this.jobConf.getInt(HoodieRealtimeConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP, HoodieRealtimeConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE))
         .withLogRecordScannerCallback(record -> {
           // convert Hoodie log record to Hadoop AvroWritable and buffer
-          GenericRecord rec = (GenericRecord) record.toIndexedRecord(getReaderSchema(), payloadProps).get();
+          GenericRecord rec = (GenericRecord) record.toIndexedRecord(getReaderSchema(), payloadProps).get().getData();
           ArrayWritable aWritable = (ArrayWritable) HoodieRealtimeRecordReaderUtils.avroToArrayWritable(rec, getHiveSchema());
           this.executor.getQueue().insertRecord(aWritable);
         })
+        .withRecordMerger(HoodieRecordUtils.loadRecordMerger(HoodieAvroRecordMerger.class.getName()))
         .build();
     // Start reading and buffering
     this.executor.startProducers();
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java
index 0e7dce2b0f..0a3b86f109 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieRealtimeRecordReaderUtils.java
@@ -41,11 +41,13 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapred.JobConf;
+
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -72,7 +74,8 @@ public class HoodieRealtimeRecordReaderUtils {
    */
   public static Schema readSchema(Configuration conf, Path filePath) {
     try {
-      HoodieAvroFileReader storageReader = HoodieFileReaderFactory.getFileReader(conf, filePath);
+      // TODO mr support other record type
+      HoodieFileReader storageReader = HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(conf, filePath);
       return storageReader.getSchema();
     } catch (IOException e) {
       throw new HoodieIOException("Failed to read schema from " + filePath, e);
diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java
index 0be736b2a0..7e571720c1 100644
--- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java
+++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestInputPathHandler.java
@@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.testutils.minicluster.HdfsTestService;
... 3488 lines suppressed ...