You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by ud...@apache.org on 2021/08/19 20:36:48 UTC

[hudi] branch master updated: Restore 0.8.0 config keys with deprecated annotation (#3506)

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

uditme pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
     new c350d05  Restore 0.8.0 config keys with deprecated annotation (#3506)
c350d05 is described below

commit c350d05dd3301f14fa9d688746c9de2416db3f11
Author: Udit Mehrotra <ud...@gmail.com>
AuthorDate: Thu Aug 19 13:36:40 2021 -0700

    Restore 0.8.0 config keys with deprecated annotation (#3506)
    
    Co-authored-by: Sagar Sumit <sa...@gmail.com>
    Co-authored-by: Vinoth Chandar <vi...@apache.org>
---
 docker/demo/sparksql-incremental.commands          |   4 +-
 .../hudi/cli/commands/HoodieLogFileCommand.java    |   4 +-
 .../org/apache/hudi/cli/commands/SparkMain.java    |  16 +-
 .../scala/org/apache/hudi/cli/SparkHelpers.scala   |   9 +-
 .../cli/commands/TestHoodieLogFileCommand.java     |   4 +-
 .../hudi/cli/commands/TestRepairsCommand.java      |   2 +-
 .../cli/commands/TestUpgradeDowngradeCommand.java  |   2 +-
 .../http/HoodieWriteCommitHttpCallbackClient.java  |   4 +-
 .../callback/util/HoodieCommitCallbackFactory.java |   2 +-
 .../apache/hudi/config/HoodieBootstrapConfig.java  | 118 ++-
 .../apache/hudi/config/HoodieClusteringConfig.java | 184 ++++-
 .../apache/hudi/config/HoodieCompactionConfig.java | 297 ++++++--
 .../apache/hudi/config/HoodieHBaseIndexConfig.java | 271 +++++--
 .../org/apache/hudi/config/HoodieIndexConfig.java  | 247 ++++--
 .../org/apache/hudi/config/HoodieLockConfig.java   |  55 +-
 .../org/apache/hudi/config/HoodieMemoryConfig.java |  47 +-
 .../apache/hudi/config/HoodieMetricsConfig.java    | 119 ++-
 .../hudi/config/HoodieMetricsDatadogConfig.java    |  97 ++-
 .../hudi/config/HoodieMetricsPrometheusConfig.java |  85 ++-
 .../apache/hudi/config/HoodiePayloadConfig.java    |  12 +-
 .../config/HoodiePreCommitValidatorConfig.java     |  33 +-
 .../apache/hudi/config/HoodieStorageConfig.java    | 165 +++-
 .../config/HoodieWriteCommitCallbackConfig.java    |  66 +-
 .../org/apache/hudi/config/HoodieWriteConfig.java  | 837 +++++++++++++++------
 .../java/org/apache/hudi/keygen/KeyGenUtils.java   |   2 +-
 .../factory/HoodieAvroKeyGeneratorFactory.java     |   2 +-
 .../metadata/HoodieBackedTableMetadataWriter.java  |   8 +-
 .../java/org/apache/hudi/table/HoodieTable.java    |  10 +-
 .../hudi/table/HoodieTimelineArchiveLog.java       |   8 +-
 .../table/upgrade/BaseOneToTwoUpgradeHandler.java  |   6 +-
 .../hudi/io/storage/TestHoodieOrcReaderWriter.java |   2 +-
 .../factory/TestHoodieAvroKeyGeneratorFactory.java |   4 +-
 .../datadog/TestHoodieMetricsDatadogConfig.java    |   2 +-
 .../hudi/testutils/HoodieWriteableTestTable.java   |   8 +-
 .../SparkSizeBasedClusteringPlanStrategy.java      |   5 +-
 .../SparkSortAndSizeExecutionStrategy.java         |  17 +-
 .../apache/hudi/client/utils/SparkMemoryUtils.java |   8 +-
 .../SqlQueryEqualityPreCommitValidator.java        |   3 +-
 .../SqlQueryInequalityPreCommitValidator.java      |   3 +-
 .../SqlQuerySingleResultPreCommitValidator.java    |   3 +-
 .../hudi/index/hbase/SparkHoodieHBaseIndex.java    |   6 +-
 .../factory/HoodieSparkKeyGeneratorFactory.java    |   2 +-
 .../functional/TestHoodieBackedMetadata.java       |  45 +-
 .../TestHoodieClientOnCopyOnWriteStorage.java      |  17 +-
 .../index/hbase/TestHBaseQPSResourceAllocator.java |   2 +-
 .../apache/hudi/keygen/TestCustomKeyGenerator.java |   4 +-
 .../TestHoodieSparkKeyGeneratorFactory.java        |   4 +-
 .../apache/hudi/table/TestConsistencyGuard.java    |   2 +-
 .../hudi/table/TestHoodieMergeOnReadTable.java     |  20 +-
 .../strategy/TestHoodieCompactionStrategy.java     |   3 +-
 .../table/action/rollback/TestRollbackUtils.java   |  11 +-
 .../hudi/table/upgrade/TestUpgradeDowngrade.java   |  38 +-
 .../hudi/testutils/HoodieClientTestHarness.java    |  32 +-
 .../hudi/testutils/HoodieClientTestUtils.java      |   4 +-
 .../hudi/common/config/HoodieMetadataConfig.java   | 203 ++++-
 .../hudi/common/config/LockConfiguration.java      |  54 ++
 .../hudi/common/fs/ConsistencyGuardConfig.java     |  97 ++-
 .../java/org/apache/hudi/common/fs/FSUtils.java    |  25 +-
 .../hudi/common/model/HoodiePayloadProps.java      |  11 +
 .../hudi/common/table/HoodieTableConfig.java       | 219 ++++--
 .../hudi/common/table/HoodieTableMetaClient.java   | 100 +--
 .../table/view/FileSystemViewStorageConfig.java    | 224 ++++--
 .../hudi/keygen/constant/KeyGeneratorOptions.java  |  31 +
 .../metadata/HoodieMetadataFileSystemView.java     |   2 +-
 .../hudi/common/bootstrap/TestBootstrapIndex.java  |   9 +-
 .../org/apache/hudi/common/fs/TestFSUtils.java     |   7 +-
 .../common/model/TestHoodieDeltaWriteStat.java     |   3 +-
 .../view/TestPriorityBasedFileSystemView.java      |   2 +-
 .../hudi/common/testutils/FileCreateUtils.java     |   4 +-
 .../hudi/common/testutils/HoodieTestTable.java     |   2 +-
 .../hudi/common/testutils/HoodieTestUtils.java     |   6 +-
 .../hudi/common/util/TestTablePathUtils.java       |   9 +-
 .../examples/spark/HoodieDataSourceExample.scala   |  32 +-
 .../examples/spark/HoodieMorCompactionJob.scala    |   6 +-
 .../apache/hudi/configuration/FlinkOptions.java    |  10 +-
 .../sink/partitioner/profile/WriteProfile.java     |   2 +-
 .../apache/hudi/sink/utils/HiveSyncContext.java    |   2 +-
 .../apache/hudi/sink/utils/PayloadCreation.java    |   2 +-
 .../apache/hudi/streamer/FlinkStreamerConfig.java  |   6 +-
 .../org/apache/hudi/table/HoodieTableFactory.java  |  14 +-
 .../java/org/apache/hudi/util/StreamerUtil.java    |   8 +-
 .../hudi/sink/partitioner/TestBucketAssigner.java  |   2 +-
 .../apache/hudi/table/TestHoodieTableFactory.java  |  32 +-
 .../hudi/hadoop/utils/HoodieInputFormatUtils.java  |  14 +-
 .../apache/hudi/hadoop/TestInputPathHandler.java   |   8 +-
 .../realtime/TestHoodieRealtimeRecordReader.java   |  14 +-
 .../hudi/integ/testsuite/HoodieTestSuiteJob.java   |   4 +-
 .../reader/DFSHoodieDatasetInputReader.java        |   6 +-
 .../testsuite/dag/nodes/SparkBulkInsertNode.scala  |   2 +-
 .../testsuite/dag/nodes/SparkInsertNode.scala      |   2 +-
 .../testsuite/dag/nodes/SparkUpsertNode.scala      |   2 +-
 .../testsuite/job/TestHoodieTestSuiteJob.java      |   2 +-
 .../main/java/org/apache/hudi/DataSourceUtils.java |  10 +-
 .../BulkInsertDataInternalWriterHelper.java        |   2 +-
 .../scala/org/apache/hudi/DataSourceOptions.scala  |  57 +-
 .../HoodieBulkInsertInternalWriterTestBase.java    |   4 +-
 .../apache/hudi/HoodieDatasetBulkInsertHelper.java |   2 +-
 .../scala/org/apache/hudi/HoodieFileIndex.scala    |  18 +-
 .../org/apache/hudi/HoodieSparkSqlWriter.scala     |  67 +-
 .../scala/org/apache/hudi/HoodieWriterUtils.scala  |  15 +-
 .../apache/spark/sql/hudi/HoodieOptionConfig.scala |  10 +-
 .../hudi/command/CreateHoodieTableCommand.scala    |  56 +-
 .../hudi/command/DeleteHoodieTableCommand.scala    |  17 +-
 .../command/InsertIntoHoodieTableCommand.scala     |  33 +-
 .../hudi/command/MergeIntoHoodieTableCommand.scala |  27 +-
 .../spark/sql/hudi/command/SqlKeyGenerator.scala   |   8 +-
 .../hudi/command/UpdateHoodieTableCommand.scala    |  14 +-
 .../hudi-spark/src/test/java/HoodieJavaApp.java    |  14 +-
 .../src/test/java/HoodieJavaGenerateApp.java       |   4 +-
 .../src/test/java/HoodieJavaStreamingApp.java      |   6 +-
 .../org/apache/hudi/functional/TestBootstrap.java  |  27 +-
 .../TestHoodieDatasetBulkInsertHelper.java         |   4 +-
 .../apache/hudi/HoodieSparkSqlWriterSuite.scala    |  57 +-
 .../org/apache/hudi/TestHoodieFileIndex.scala      |  22 +-
 .../apache/hudi/functional/TestCOWDataSource.scala |  30 +-
 .../functional/TestDataSourceForBootstrap.scala    |  34 +-
 .../apache/hudi/functional/TestEmptyCommit.scala   |   6 +-
 .../apache/hudi/functional/TestMORDataSource.scala |  40 +-
 .../hudi/functional/TestStreamingSource.scala      |  14 +-
 .../hudi/functional/TestStructuredStreaming.scala  |   8 +-
 .../hudi/functional/TestTimeTravelQuery.scala      |  16 +-
 .../apache/spark/sql/hudi/TestCreateTable.scala    |  46 +-
 .../org/apache/hudi/internal/DefaultSource.java    |  10 +-
 .../apache/hudi/spark3/internal/DefaultSource.java |   8 +-
 .../java/org/apache/hudi/dla/DLASyncConfig.java    |   6 +-
 .../java/org/apache/hudi/hive/HiveSyncConfig.java  |   2 +-
 .../hudi/timeline/service/TimelineService.java     |  11 +-
 .../hudi/utilities/HoodieSnapshotCopier.java       |   2 +-
 .../kafka/HoodieWriteCommitKafkaCallback.java      |  30 +-
 .../HoodieWriteCommitKafkaCallbackConfig.java      |  49 +-
 .../utilities/deltastreamer/BootstrapExecutor.java |  14 +-
 .../hudi/utilities/deltastreamer/DeltaSync.java    |  22 +-
 .../hudi/utilities/perf/TimelineServerPerf.java    |   6 +-
 .../TestKafkaConnectHdfsProvider.java              |   2 +-
 .../functional/TestHoodieDeltaStreamer.java        |   8 +-
 .../TestHoodieMultiTableDeltaStreamer.java         |   6 +-
 .../functional/TestHoodieSnapshotCopier.java       |   4 +-
 137 files changed, 3460 insertions(+), 1527 deletions(-)

diff --git a/docker/demo/sparksql-incremental.commands b/docker/demo/sparksql-incremental.commands
index e53203b..da61347 100644
--- a/docker/demo/sparksql-incremental.commands
+++ b/docker/demo/sparksql-incremental.commands
@@ -42,7 +42,7 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl
     option(DataSourceWriteOptions.RECORDKEY_FIELD.key(), "key").
     option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key(), "datestr").
     option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), "ts").
-    option(HoodieWriteConfig.TABLE_NAME.key(), "stock_ticks_derived_mor").
+    option(HoodieWriteConfig.TBL_NAME.key(), "stock_ticks_derived_mor").
     option(DataSourceWriteOptions.HIVE_TABLE.key(), "stock_ticks_derived_mor").
     option(DataSourceWriteOptions.HIVE_DATABASE.key(), "default").
     option(DataSourceWriteOptions.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000").
@@ -74,7 +74,7 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl
     option(DataSourceWriteOptions.RECORDKEY_FIELD.key(), "key").
     option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key(), "datestr").
     option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), "ts").
-    option(HoodieWriteConfig.TABLE_NAME.key(), "stock_ticks_derived_mor_bs").
+    option(HoodieWriteConfig.TBL_NAME.key(), "stock_ticks_derived_mor_bs").
     option(DataSourceWriteOptions.HIVE_TABLE.key(), "stock_ticks_derived_mor_bs").
     option(DataSourceWriteOptions.HIVE_DATABASE.key(), "default").
     option(DataSourceWriteOptions.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000").
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 1c747ab..27bcd81 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
@@ -204,10 +204,10 @@ public class HoodieLogFileCommand implements CommandMarker {
                       .getCommitTimeline().lastInstant().get().getTimestamp())
               .withReadBlocksLazily(
                   Boolean.parseBoolean(
-                      HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED.defaultValue()))
+                      HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLE.defaultValue()))
               .withReverseReader(
                   Boolean.parseBoolean(
-                      HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED.defaultValue()))
+                      HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLE.defaultValue()))
               .withBufferSize(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE.defaultValue())
               .withMaxMemorySizeInBytes(
                   HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES)
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java
index ccf0327..f86937d 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java
@@ -18,8 +18,8 @@
 
 package org.apache.hudi.cli.commands;
 
-import org.apache.hudi.cli.DeDupeType;
 import org.apache.hudi.DataSourceWriteOptions;
+import org.apache.hudi.cli.DeDupeType;
 import org.apache.hudi.cli.DedupeSparkJob;
 import org.apache.hudi.cli.utils.SparkUtil;
 import org.apache.hudi.client.SparkRDDWriteClient;
@@ -361,17 +361,17 @@ public class SparkMain {
     TypedProperties properties = propsFilePath == null ? UtilHelpers.buildProperties(configs)
         : UtilHelpers.readConfig(FSUtils.getFs(propsFilePath, jsc.hadoopConfiguration()), new Path(propsFilePath), configs).getConfig();
 
-    properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH.key(), sourcePath);
+    properties.setProperty(HoodieBootstrapConfig.BASE_PATH.key(), sourcePath);
 
-    if (!StringUtils.isNullOrEmpty(keyGenerator) &&  KeyGeneratorType.getNames().contains(keyGenerator.toUpperCase(Locale.ROOT))) {
-      properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_TYPE.key(), keyGenerator.toUpperCase(Locale.ROOT));
+    if (!StringUtils.isNullOrEmpty(keyGenerator) && KeyGeneratorType.getNames().contains(keyGenerator.toUpperCase(Locale.ROOT))) {
+      properties.setProperty(HoodieBootstrapConfig.KEYGEN_TYPE.key(), keyGenerator.toUpperCase(Locale.ROOT));
     } else {
-      properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS.key(), keyGenerator);
+      properties.setProperty(HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key(), keyGenerator);
     }
 
-    properties.setProperty(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER.key(), fullBootstrapInputProvider);
-    properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM.key(), parallelism);
-    properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR.key(), selectorClass);
+    properties.setProperty(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER_CLASS_NAME.key(), fullBootstrapInputProvider);
+    properties.setProperty(HoodieBootstrapConfig.PARALLELISM_VALUE.key(), parallelism);
+    properties.setProperty(HoodieBootstrapConfig.MODE_SELECTOR_CLASS_NAME.key(), selectorClass);
     properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD().key(), recordKeyCols);
     properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), partitionFields);
 
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 33d0145..3802bb4 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
@@ -25,8 +25,7 @@ import org.apache.hudi.avro.HoodieAvroWriteSupport
 import org.apache.hudi.client.SparkTaskContextSupplier
 import org.apache.hudi.common.HoodieJsonPayload
 import org.apache.hudi.common.bloom.{BloomFilter, BloomFilterFactory}
-import org.apache.hudi.common.model.HoodieFileFormat
-import org.apache.hudi.common.model.HoodieRecord
+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.io.storage.{HoodieAvroParquetConfig, HoodieParquetWriter}
@@ -43,10 +42,10 @@ object SparkHelpers {
   def skipKeysAndWriteNewFile(instantTime: String, fs: FileSystem, sourceFile: Path, destinationFile: Path, keysToSkip: Set[String]) {
     val sourceRecords = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET).readAvroRecords(fs.getConf, sourceFile)
     val schema: Schema = sourceRecords.get(0).getSchema
-    val filter: BloomFilter = BloomFilterFactory.createBloomFilter(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES.defaultValue.toInt, HoodieIndexConfig.BLOOM_FILTER_FPP.defaultValue.toDouble,
-      HoodieIndexConfig.HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.defaultValue.toInt, HoodieIndexConfig.BLOOM_INDEX_FILTER_TYPE.defaultValue);
+    val filter: BloomFilter = BloomFilterFactory.createBloomFilter(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES_VALUE.defaultValue.toInt, HoodieIndexConfig.BLOOM_FILTER_FPP_VALUE.defaultValue.toDouble,
+      HoodieIndexConfig.BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.defaultValue.toInt, HoodieIndexConfig.BLOOM_FILTER_TYPE.defaultValue);
     val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(fs.getConf).convert(schema), schema, org.apache.hudi.common.util.Option.of(filter))
-    val parquetConfig: HoodieAvroParquetConfig = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.PARQUET_BLOCK_SIZE_BYTES.defaultValue.toInt, HoodieStorageConfig.PARQUET_PAGE_SIZE_BYTES.defaultValue.toInt, HoodieStorageConfig.PARQUET_FILE_MAX_BYTES.defaultValue.toInt, fs.getConf, HoodieStorageConfig.PARQUET_COMPRESSION_RATIO.defaultValue.toDouble)
+    val parquetConfig: HoodieAvroParquetConfig = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.PARQUET_BLOCK_SIZE.defaultValue.toInt, HoodieStorageConfig.PARQUET_PAGE_SIZE.defaultValue.toInt, HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.defaultValue.toInt, fs.getConf, HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION.defaultValue.toDouble)
 
     // Add current classLoad for config, if not will throw classNotFound of 'HoodieWrapperFileSystem'.
     parquetConfig.getHadoopConf().setClassLoader(Thread.currentThread.getContextClassLoader)
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 a44601d..c03077a 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
@@ -208,10 +208,10 @@ public class TestHoodieLogFileCommand extends AbstractShellIntegrationTest {
             HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES)
         .withReadBlocksLazily(
             Boolean.parseBoolean(
-                HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED.defaultValue()))
+                HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLE.defaultValue()))
         .withReverseReader(
             Boolean.parseBoolean(
-                HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED.defaultValue()))
+                HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLE.defaultValue()))
         .withBufferSize(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE.defaultValue())
         .withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH.defaultValue())
         .withDiskMapType(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue())
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRepairsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRepairsCommand.java
index f500f30..4cc7691 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRepairsCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRepairsCommand.java
@@ -67,7 +67,7 @@ public class TestRepairsCommand extends AbstractShellIntegrationTest {
     // Create table and connect
     new TableCommand().createTable(
         tablePath, tableName, HoodieTableType.COPY_ON_WRITE.name(),
-        HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue(), TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload");
+        HoodieTableConfig.ARCHIVELOG_FOLDER.defaultValue(), TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload");
   }
 
   /**
diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestUpgradeDowngradeCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestUpgradeDowngradeCommand.java
index 5fb9191..b3a31dc 100644
--- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestUpgradeDowngradeCommand.java
+++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestUpgradeDowngradeCommand.java
@@ -112,6 +112,6 @@ public class TestUpgradeDowngradeCommand extends AbstractShellIntegrationTest {
     HoodieConfig hoodieConfig = HoodieConfig.create(fsDataInputStream);
     fsDataInputStream.close();
     assertEquals(Integer.toString(HoodieTableVersion.ZERO.versionCode()), hoodieConfig
-        .getString(HoodieTableConfig.HOODIE_TABLE_VERSION_PROP));
+        .getString(HoodieTableConfig.VERSION));
   }
 }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/client/http/HoodieWriteCommitHttpCallbackClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/client/http/HoodieWriteCommitHttpCallbackClient.java
index 08ed283..6d1059c 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/client/http/HoodieWriteCommitHttpCallbackClient.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/client/http/HoodieWriteCommitHttpCallbackClient.java
@@ -79,7 +79,7 @@ public class HoodieWriteCommitHttpCallbackClient implements Closeable {
   }
 
   private String getApiKey() {
-    return writeConfig.getString(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_API_KEY);
+    return writeConfig.getString(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_API_KEY_VALUE);
   }
 
   private String getUrl() {
@@ -97,7 +97,7 @@ public class HoodieWriteCommitHttpCallbackClient implements Closeable {
   }
 
   private Integer getHttpTimeoutSeconds() {
-    return writeConfig.getInt(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_TIMEOUT_SECONDS);
+    return writeConfig.getInt(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_TIMEOUT_IN_SECONDS);
   }
 
   @Override
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/util/HoodieCommitCallbackFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/util/HoodieCommitCallbackFactory.java
index 66166d4..aaa5eed 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/util/HoodieCommitCallbackFactory.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/callback/util/HoodieCommitCallbackFactory.java
@@ -39,7 +39,7 @@ public class HoodieCommitCallbackFactory {
       return (HoodieWriteCommitCallback) instance;
     } else {
       throw new HoodieCommitCallbackException(String.format("The value of the config option %s can not be null or "
-          + "empty", HoodieWriteCommitCallbackConfig.CALLBACK_CLASS.key()));
+          + "empty", HoodieWriteCommitCallbackConfig.CALLBACK_CLASS_NAME.key()));
     }
   }
 
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java
index 836b0c1..94bb783 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java
@@ -44,57 +44,55 @@ import java.util.Properties;
         + " writers and new hudi writers in parallel, to validate the migration.")
 public class HoodieBootstrapConfig extends HoodieConfig {
 
-  public static final ConfigProperty<String> BOOTSTRAP_BASE_PATH = ConfigProperty
+  public static final ConfigProperty<String> BASE_PATH = ConfigProperty
       .key("hoodie.bootstrap.base.path")
       .noDefaultValue()
       .sinceVersion("0.6.0")
       .withDocumentation("Base path of the dataset that needs to be bootstrapped as a Hudi table");
-  @Deprecated
-  public static final String BOOTSTRAP_BASE_PATH_PROP = BOOTSTRAP_BASE_PATH.key();
 
-  public static final ConfigProperty<String> BOOTSTRAP_MODE_SELECTOR = ConfigProperty
+  public static final ConfigProperty<String> MODE_SELECTOR_CLASS_NAME = ConfigProperty
       .key("hoodie.bootstrap.mode.selector")
       .defaultValue(MetadataOnlyBootstrapModeSelector.class.getCanonicalName())
       .sinceVersion("0.6.0")
       .withDocumentation("Selects the mode in which each file/partition in the bootstrapped dataset gets bootstrapped");
 
-  public static final ConfigProperty<String> FULL_BOOTSTRAP_INPUT_PROVIDER = ConfigProperty
+  public static final ConfigProperty<String> FULL_BOOTSTRAP_INPUT_PROVIDER_CLASS_NAME = ConfigProperty
       .key("hoodie.bootstrap.full.input.provider")
       .defaultValue("org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider")
       .sinceVersion("0.6.0")
       .withDocumentation("Class to use for reading the bootstrap dataset partitions/files, for Bootstrap mode FULL_RECORD");
 
-  public static final ConfigProperty<String> BOOTSTRAP_KEYGEN_CLASS = ConfigProperty
+  public static final ConfigProperty<String> KEYGEN_CLASS_NAME = ConfigProperty
       .key("hoodie.bootstrap.keygen.class")
       .noDefaultValue()
       .sinceVersion("0.6.0")
       .withDocumentation("Key generator implementation to be used for generating keys from the bootstrapped dataset");
 
-  public static final ConfigProperty<String> BOOTSTRAP_KEYGEN_TYPE = ConfigProperty
+  public static final ConfigProperty<String> KEYGEN_TYPE = ConfigProperty
       .key("hoodie.bootstrap.keygen.type")
       .defaultValue(KeyGeneratorType.SIMPLE.name())
       .sinceVersion("0.9.0")
       .withDocumentation("Type of build-in key generator, currently support SIMPLE, COMPLEX, TIMESTAMP, CUSTOM, NON_PARTITION, GLOBAL_DELETE");
 
-  public static final ConfigProperty<String> BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS = ConfigProperty
+  public static final ConfigProperty<String> PARTITION_PATH_TRANSLATOR_CLASS_NAME = ConfigProperty
       .key("hoodie.bootstrap.partitionpath.translator.class")
       .defaultValue(IdentityBootstrapPartitionPathTranslator.class.getName())
       .sinceVersion("0.6.0")
       .withDocumentation("Translates the partition paths from the bootstrapped data into how is laid out as a Hudi table.");
 
-  public static final ConfigProperty<String> BOOTSTRAP_PARALLELISM = ConfigProperty
+  public static final ConfigProperty<String> PARALLELISM_VALUE = ConfigProperty
       .key("hoodie.bootstrap.parallelism")
       .defaultValue("1500")
       .sinceVersion("0.6.0")
       .withDocumentation("Parallelism value to be used to bootstrap data into hudi");
 
-  public static final ConfigProperty<String> BOOTSTRAP_MODE_SELECTOR_REGEX = ConfigProperty
+  public static final ConfigProperty<String> PARTITION_SELECTOR_REGEX_PATTERN = ConfigProperty
       .key("hoodie.bootstrap.mode.selector.regex")
       .defaultValue(".*")
       .sinceVersion("0.6.0")
       .withDocumentation("Matches each bootstrap dataset partition against this regex and applies the mode below to it.");
 
-  public static final ConfigProperty<String> BOOTSTRAP_MODE_SELECTOR_REGEX_MODE = ConfigProperty
+  public static final ConfigProperty<String> PARTITION_SELECTOR_REGEX_MODE = ConfigProperty
       .key("hoodie.bootstrap.mode.selector.regex.mode")
       .defaultValue(BootstrapMode.METADATA_ONLY.name())
       .sinceVersion("0.6.0")
@@ -102,13 +100,87 @@ public class HoodieBootstrapConfig extends HoodieConfig {
           + "METADATA_ONLY will generate just skeleton base files with keys/footers, avoiding full cost of rewriting the dataset. "
           + "FULL_RECORD will perform a full copy/rewrite of the data as a Hudi table.");
 
-  public static final ConfigProperty<String> BOOTSTRAP_INDEX_CLASS = ConfigProperty
+  public static final ConfigProperty<String> INDEX_CLASS_NAME = ConfigProperty
       .key("hoodie.bootstrap.index.class")
       .defaultValue(HFileBootstrapIndex.class.getName())
       .sinceVersion("0.6.0")
       .withDocumentation("Implementation to use, for mapping a skeleton base file to a boostrap base file.");
+
+  /**
+   * @deprecated Use {@link #BASE_PATH} and its methods instead
+   */
+  @Deprecated
+  public static final String BOOTSTRAP_BASE_PATH_PROP = BASE_PATH.key();
+  /**
+   * @deprecated Use {@link #INDEX_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String BOOTSTRAP_INDEX_CLASS_PROP = INDEX_CLASS_NAME.key();
+  /**
+   * @deprecated Use {@link #INDEX_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_BOOTSTRAP_INDEX_CLASS = INDEX_CLASS_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #MODE_SELECTOR_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String BOOTSTRAP_MODE_SELECTOR = MODE_SELECTOR_CLASS_NAME.key();
+  /**
+   * @deprecated Use {@link #FULL_BOOTSTRAP_INPUT_PROVIDER_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String FULL_BOOTSTRAP_INPUT_PROVIDER = FULL_BOOTSTRAP_INPUT_PROVIDER_CLASS_NAME.key();
+  /**
+   * @deprecated Use {@link #FULL_BOOTSTRAP_INPUT_PROVIDER_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_FULL_BOOTSTRAP_INPUT_PROVIDER = FULL_BOOTSTRAP_INPUT_PROVIDER_CLASS_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #KEYGEN_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String BOOTSTRAP_KEYGEN_CLASS = KEYGEN_CLASS_NAME.key();
+  /**
+   * @deprecated Use {@link #PARTITION_PATH_TRANSLATOR_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS = PARTITION_PATH_TRANSLATOR_CLASS_NAME.key();
+  /**
+   * @deprecated Use {@link #PARTITION_PATH_TRANSLATOR_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS = PARTITION_PATH_TRANSLATOR_CLASS_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #PARALLELISM_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String BOOTSTRAP_PARALLELISM = PARALLELISM_VALUE.key();
+  /**
+   * @deprecated Use {@link #PARALLELISM_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_BOOTSTRAP_PARALLELISM = PARALLELISM_VALUE.defaultValue();
+  /**
+   * @deprecated Use {@link #PARTITION_SELECTOR_REGEX_PATTERN} and its methods instead
+   */
+  @Deprecated
+  public static final String BOOTSTRAP_MODE_SELECTOR_REGEX = PARTITION_SELECTOR_REGEX_PATTERN.key();
+  /**
+   * @deprecated Use {@link #PARTITION_SELECTOR_REGEX_MODE} and its methods instead
+   */
+  @Deprecated
+  public static final String BOOTSTRAP_MODE_SELECTOR_REGEX_MODE = PARTITION_SELECTOR_REGEX_MODE.key();
+  /**
+   * @deprecated Use {@link #PARTITION_SELECTOR_REGEX_PATTERN} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX = PARTITION_SELECTOR_REGEX_PATTERN.defaultValue();
+  /**
+   * @deprecated Use {@link #PARTITION_SELECTOR_REGEX_MODE} and its methods instead
+   */
   @Deprecated
-  public static final String BOOTSTRAP_INDEX_CLASS_PROP = BOOTSTRAP_INDEX_CLASS.key();
+  public static final String DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX_MODE = PARTITION_SELECTOR_REGEX_MODE.defaultValue();
 
   private HoodieBootstrapConfig() {
     super();
@@ -130,48 +202,48 @@ public class HoodieBootstrapConfig extends HoodieConfig {
     }
 
     public Builder withBootstrapBasePath(String basePath) {
-      bootstrapConfig.setValue(BOOTSTRAP_BASE_PATH, basePath);
+      bootstrapConfig.setValue(BASE_PATH, basePath);
       return this;
     }
 
     public Builder withBootstrapModeSelector(String partitionSelectorClass) {
-      bootstrapConfig.setValue(BOOTSTRAP_MODE_SELECTOR, partitionSelectorClass);
+      bootstrapConfig.setValue(MODE_SELECTOR_CLASS_NAME, partitionSelectorClass);
       return this;
     }
 
     public Builder withFullBootstrapInputProvider(String partitionSelectorClass) {
-      bootstrapConfig.setValue(FULL_BOOTSTRAP_INPUT_PROVIDER, partitionSelectorClass);
+      bootstrapConfig.setValue(FULL_BOOTSTRAP_INPUT_PROVIDER_CLASS_NAME, partitionSelectorClass);
       return this;
     }
 
     public Builder withBootstrapKeyGenClass(String keyGenClass) {
-      bootstrapConfig.setValue(BOOTSTRAP_KEYGEN_CLASS, keyGenClass);
+      bootstrapConfig.setValue(KEYGEN_CLASS_NAME, keyGenClass);
       return this;
     }
 
     public Builder withBootstrapKeyGenType(String keyGenType) {
-      bootstrapConfig.setValue(BOOTSTRAP_KEYGEN_TYPE, keyGenType);
+      bootstrapConfig.setValue(KEYGEN_TYPE, keyGenType);
       return this;
     }
 
     public Builder withBootstrapPartitionPathTranslatorClass(String partitionPathTranslatorClass) {
       bootstrapConfig
-          .setValue(BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS, partitionPathTranslatorClass);
+          .setValue(PARTITION_PATH_TRANSLATOR_CLASS_NAME, partitionPathTranslatorClass);
       return this;
     }
 
     public Builder withBootstrapParallelism(int parallelism) {
-      bootstrapConfig.setValue(BOOTSTRAP_PARALLELISM, String.valueOf(parallelism));
+      bootstrapConfig.setValue(PARALLELISM_VALUE, String.valueOf(parallelism));
       return this;
     }
 
     public Builder withBootstrapModeSelectorRegex(String regex) {
-      bootstrapConfig.setValue(BOOTSTRAP_MODE_SELECTOR_REGEX, regex);
+      bootstrapConfig.setValue(PARTITION_SELECTOR_REGEX_PATTERN, regex);
       return this;
     }
 
     public Builder withBootstrapModeForRegexMatch(BootstrapMode modeForRegexMatch) {
-      bootstrapConfig.setValue(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE, modeForRegexMatch.name());
+      bootstrapConfig.setValue(PARTITION_SELECTOR_REGEX_MODE, modeForRegexMatch.name());
       return this;
     }
 
@@ -182,7 +254,7 @@ public class HoodieBootstrapConfig extends HoodieConfig {
 
     public HoodieBootstrapConfig build() {
       // TODO: use infer function instead
-      bootstrapConfig.setDefaultValue(BOOTSTRAP_INDEX_CLASS, HoodieTableConfig.getDefaultBootstrapIndexClass(
+      bootstrapConfig.setDefaultValue(INDEX_CLASS_NAME, HoodieTableConfig.getDefaultBootstrapIndexClass(
           bootstrapConfig.getProps()));
       bootstrapConfig.setDefaults(HoodieBootstrapConfig.class.getName());
       return bootstrapConfig;
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java
index 4c68ec4..f1e930b 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieClusteringConfig.java
@@ -40,21 +40,21 @@ public class HoodieClusteringConfig extends HoodieConfig {
   // Any strategy specific params can be saved with this prefix
   public static final String CLUSTERING_STRATEGY_PARAM_PREFIX = "hoodie.clustering.plan.strategy.";
 
-  public static final ConfigProperty<String> CLUSTERING_TARGET_PARTITIONS = ConfigProperty
+  public static final ConfigProperty<String> DAYBASED_LOOKBACK_PARTITIONS = ConfigProperty
       .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "daybased.lookback.partitions")
       .defaultValue("2")
       .sinceVersion("0.7.0")
       .withDocumentation("Number of partitions to list to create ClusteringPlan");
 
-  public static final ConfigProperty<String> CLUSTERING_PLAN_STRATEGY_CLASS = ConfigProperty
+  public static final ConfigProperty<String> PLAN_STRATEGY_CLASS_NAME = ConfigProperty
       .key("hoodie.clustering.plan.strategy.class")
       .defaultValue("org.apache.hudi.client.clustering.plan.strategy.SparkRecentDaysClusteringPlanStrategy")
       .sinceVersion("0.7.0")
       .withDocumentation("Config to provide a strategy class (subclass of ClusteringPlanStrategy) to create clustering plan "
           + "i.e select what file groups are being clustered. Default strategy, looks at the last N (determined by "
-          + CLUSTERING_TARGET_PARTITIONS.key() + ") day based partitions picks the small file slices within those partitions.");
+          + DAYBASED_LOOKBACK_PARTITIONS.key() + ") day based partitions picks the small file slices within those partitions.");
 
-  public static final ConfigProperty<String> CLUSTERING_EXECUTION_STRATEGY_CLASS = ConfigProperty
+  public static final ConfigProperty<String> EXECUTION_STRATEGY_CLASS_NAME = ConfigProperty
       .key("hoodie.clustering.execution.strategy.class")
       .defaultValue("org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy")
       .sinceVersion("0.7.0")
@@ -67,36 +67,32 @@ public class HoodieClusteringConfig extends HoodieConfig {
       .defaultValue("false")
       .sinceVersion("0.7.0")
       .withDocumentation("Turn on inline clustering - clustering will be run after each write operation is complete");
-  @Deprecated
-  public static final String INLINE_CLUSTERING_PROP = INLINE_CLUSTERING.key();
 
-  public static final ConfigProperty<String> INLINE_CLUSTERING_MAX_COMMIT = ConfigProperty
+  public static final ConfigProperty<String> INLINE_CLUSTERING_MAX_COMMITS = ConfigProperty
       .key("hoodie.clustering.inline.max.commits")
       .defaultValue("4")
       .sinceVersion("0.7.0")
       .withDocumentation("Config to control frequency of clustering planning");
-  @Deprecated
-  public static final String INLINE_CLUSTERING_MAX_COMMIT_PROP = INLINE_CLUSTERING_MAX_COMMIT.key();
 
-  public static final ConfigProperty<String> ASYNC_CLUSTERING_MAX_COMMIT_PROP = ConfigProperty
+  public static final ConfigProperty<String> ASYNC_CLUSTERING_MAX_COMMITS = ConfigProperty
       .key("hoodie.clustering.async.max.commits")
       .defaultValue("4")
       .sinceVersion("0.9.0")
       .withDocumentation("Config to control frequency of async clustering");
 
-  public static final ConfigProperty<String> CLUSTERING_SKIP_PARTITIONS_FROM_LATEST = ConfigProperty
-          .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "daybased.skipfromlatest.partitions")
-          .defaultValue("0")
-          .sinceVersion("0.9.0")
-          .withDocumentation("Number of partitions to skip from latest when choosing partitions to create ClusteringPlan");
+  public static final ConfigProperty<String> PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST = ConfigProperty
+      .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "daybased.skipfromlatest.partitions")
+      .defaultValue("0")
+      .sinceVersion("0.9.0")
+      .withDocumentation("Number of partitions to skip from latest when choosing partitions to create ClusteringPlan");
 
-  public static final ConfigProperty<String> CLUSTERING_PLAN_SMALL_FILE_LIMIT = ConfigProperty
+  public static final ConfigProperty<String> PLAN_STRATEGY_SMALL_FILE_LIMIT = ConfigProperty
       .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "small.file.limit")
       .defaultValue(String.valueOf(600 * 1024 * 1024L))
       .sinceVersion("0.7.0")
       .withDocumentation("Files smaller than the size specified here are candidates for clustering");
 
-  public static final ConfigProperty<String> CLUSTERING_MAX_BYTES_PER_GROUP = ConfigProperty
+  public static final ConfigProperty<String> PLAN_STRATEGY_MAX_BYTES_PER_OUTPUT_FILEGROUP = ConfigProperty
       .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "max.bytes.per.group")
       .defaultValue(String.valueOf(2 * 1024 * 1024 * 1024L))
       .sinceVersion("0.7.0")
@@ -104,46 +100,156 @@ public class HoodieClusteringConfig extends HoodieConfig {
           + " is defined by below two properties (CLUSTERING_MAX_BYTES_PER_GROUP * CLUSTERING_MAX_NUM_GROUPS)."
           + " Max amount of data to be included in one group");
 
-  public static final ConfigProperty<String> CLUSTERING_MAX_NUM_GROUPS = ConfigProperty
+  public static final ConfigProperty<String> PLAN_STRATEGY_MAX_GROUPS = ConfigProperty
       .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "max.num.groups")
       .defaultValue("30")
       .sinceVersion("0.7.0")
       .withDocumentation("Maximum number of groups to create as part of ClusteringPlan. Increasing groups will increase parallelism");
 
-  public static final ConfigProperty<String> CLUSTERING_TARGET_FILE_MAX_BYTES = ConfigProperty
+  public static final ConfigProperty<String> PLAN_STRATEGY_TARGET_FILE_MAX_BYTES = ConfigProperty
       .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "target.file.max.bytes")
       .defaultValue(String.valueOf(1024 * 1024 * 1024L))
       .sinceVersion("0.7.0")
       .withDocumentation("Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups");
 
-  public static final ConfigProperty<String> CLUSTERING_SORT_COLUMNS_PROPERTY = ConfigProperty
+  public static final ConfigProperty<String> PLAN_STRATEGY_SORT_COLUMNS = ConfigProperty
       .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "sort.columns")
       .noDefaultValue()
       .sinceVersion("0.7.0")
       .withDocumentation("Columns to sort the data by when clustering");
 
-  public static final ConfigProperty<String> CLUSTERING_UPDATES_STRATEGY = ConfigProperty
+  public static final ConfigProperty<String> UPDATES_STRATEGY = ConfigProperty
       .key("hoodie.clustering.updates.strategy")
       .defaultValue("org.apache.hudi.client.clustering.update.strategy.SparkRejectUpdateStrategy")
       .sinceVersion("0.7.0")
       .withDocumentation("Determines how to handle updates, deletes to file groups that are under clustering."
           + " Default strategy just rejects the update");
-  @Deprecated
-  public static final String CLUSTERING_UPDATES_STRATEGY_PROP = CLUSTERING_UPDATES_STRATEGY.key();
 
   public static final ConfigProperty<String> ASYNC_CLUSTERING_ENABLE = ConfigProperty
       .key("hoodie.clustering.async.enabled")
       .defaultValue("false")
       .sinceVersion("0.7.0")
       .withDocumentation("Enable running of clustering service, asynchronously as inserts happen on the table.");
-  @Deprecated
-  public static final String ASYNC_CLUSTERING_ENABLE_OPT_KEY = "hoodie.clustering.async.enabled";
 
-  public static final ConfigProperty<Boolean> CLUSTERING_PRESERVE_HOODIE_COMMIT_METADATA = ConfigProperty
+  public static final ConfigProperty<Boolean> PRESERVE_COMMIT_METADATA = ConfigProperty
       .key("hoodie.clustering.preserve.commit.metadata")
       .defaultValue(false)
       .sinceVersion("0.9.0")
       .withDocumentation("When rewriting data, preserves existing hoodie_commit_time");
+
+  /**
+   * @deprecated Use {@link #PLAN_STRATEGY_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String CLUSTERING_PLAN_STRATEGY_CLASS = PLAN_STRATEGY_CLASS_NAME.key();
+  /**
+   * @deprecated Use {@link #PLAN_STRATEGY_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_CLUSTERING_PLAN_STRATEGY_CLASS = PLAN_STRATEGY_CLASS_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #EXECUTION_STRATEGY_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String CLUSTERING_EXECUTION_STRATEGY_CLASS = EXECUTION_STRATEGY_CLASS_NAME.key();
+  /**
+   * @deprecated Use {@link #EXECUTION_STRATEGY_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS = EXECUTION_STRATEGY_CLASS_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #INLINE_CLUSTERING} and its methods instead
+   */
+  @Deprecated
+  public static final String INLINE_CLUSTERING_PROP = INLINE_CLUSTERING.key();
+  /**
+   * @deprecated Use {@link #INLINE_CLUSTERING} and its methods instead
+   */
+  @Deprecated
+  private static final String DEFAULT_INLINE_CLUSTERING = INLINE_CLUSTERING.defaultValue();
+  /**
+   * @deprecated Use {@link #INLINE_CLUSTERING_MAX_COMMITS} and its methods instead
+   */
+  @Deprecated
+  public static final String INLINE_CLUSTERING_MAX_COMMIT_PROP = INLINE_CLUSTERING_MAX_COMMITS.key();
+  /**
+   * @deprecated Use {@link #INLINE_CLUSTERING_MAX_COMMITS} and its methods instead
+   */
+  @Deprecated
+  private static final String DEFAULT_INLINE_CLUSTERING_NUM_COMMITS = INLINE_CLUSTERING_MAX_COMMITS.defaultValue();
+  /**
+   * @deprecated Use {@link #DAYBASED_LOOKBACK_PARTITIONS} and its methods instead
+   */
+  @Deprecated
+  public static final String CLUSTERING_TARGET_PARTITIONS = DAYBASED_LOOKBACK_PARTITIONS.key();
+  /**
+   * @deprecated Use {@link #DAYBASED_LOOKBACK_PARTITIONS} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_CLUSTERING_TARGET_PARTITIONS = DAYBASED_LOOKBACK_PARTITIONS.defaultValue();
+  /**
+   * @deprecated Use {@link #PLAN_STRATEGY_SMALL_FILE_LIMIT} and its methods instead
+   */
+  @Deprecated
+  public static final String CLUSTERING_PLAN_SMALL_FILE_LIMIT = PLAN_STRATEGY_SMALL_FILE_LIMIT.key();
+  /**
+   * @deprecated Use {@link #PLAN_STRATEGY_SMALL_FILE_LIMIT} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_CLUSTERING_PLAN_SMALL_FILE_LIMIT = PLAN_STRATEGY_SMALL_FILE_LIMIT.defaultValue();
+  /**
+   * @deprecated Use {@link #PLAN_STRATEGY_MAX_BYTES_PER_OUTPUT_FILEGROUP} and its methods instead
+   */
+  @Deprecated
+  public static final String CLUSTERING_MAX_BYTES_PER_GROUP = PLAN_STRATEGY_MAX_BYTES_PER_OUTPUT_FILEGROUP.key();
+  /**
+   * @deprecated Use {@link #PLAN_STRATEGY_MAX_BYTES_PER_OUTPUT_FILEGROUP} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_CLUSTERING_MAX_GROUP_SIZE = PLAN_STRATEGY_MAX_BYTES_PER_OUTPUT_FILEGROUP.defaultValue();
+  /**
+   * @deprecated Use {@link #PLAN_STRATEGY_MAX_GROUPS} and its methods instead
+   */
+  @Deprecated
+  public static final String CLUSTERING_MAX_NUM_GROUPS = PLAN_STRATEGY_MAX_GROUPS.key();
+  /**
+   * @deprecated Use {@link #PLAN_STRATEGY_MAX_GROUPS} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_CLUSTERING_MAX_NUM_GROUPS = PLAN_STRATEGY_MAX_GROUPS.defaultValue();
+  /**
+   * @deprecated Use {@link #PLAN_STRATEGY_TARGET_FILE_MAX_BYTES} and its methods instead
+   */
+  @Deprecated
+  public static final String CLUSTERING_TARGET_FILE_MAX_BYTES = PLAN_STRATEGY_TARGET_FILE_MAX_BYTES.key();
+  /**
+   * @deprecated Use {@link #PLAN_STRATEGY_TARGET_FILE_MAX_BYTES} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_CLUSTERING_TARGET_FILE_MAX_BYTES = PLAN_STRATEGY_TARGET_FILE_MAX_BYTES.defaultValue();
+  /**
+   * @deprecated Use {@link #PLAN_STRATEGY_SORT_COLUMNS} and its methods instead
+   */
+  @Deprecated
+  public static final String CLUSTERING_SORT_COLUMNS_PROPERTY = PLAN_STRATEGY_SORT_COLUMNS.key();
+  /**
+   * @deprecated Use {@link #UPDATES_STRATEGY} and its methods instead
+   */
+  @Deprecated
+  public static final String CLUSTERING_UPDATES_STRATEGY_PROP = UPDATES_STRATEGY.key();
+  /**
+   * @deprecated Use {@link #UPDATES_STRATEGY} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_CLUSTERING_UPDATES_STRATEGY = UPDATES_STRATEGY.defaultValue();
+  /**
+   * @deprecated Use {@link #ASYNC_CLUSTERING_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String ASYNC_CLUSTERING_ENABLE_OPT_KEY = ASYNC_CLUSTERING_ENABLE.key();
+  /** @deprecated Use {@link #ASYNC_CLUSTERING_ENABLE} and its methods instead */
+  @Deprecated
+  public static final String DEFAULT_ASYNC_CLUSTERING_ENABLE_OPT_VAL = ASYNC_CLUSTERING_ENABLE.defaultValue();
   
   public HoodieClusteringConfig() {
     super();
@@ -165,47 +271,47 @@ public class HoodieClusteringConfig extends HoodieConfig {
     }
 
     public Builder withClusteringPlanStrategyClass(String clusteringStrategyClass) {
-      clusteringConfig.setValue(CLUSTERING_PLAN_STRATEGY_CLASS, clusteringStrategyClass);
+      clusteringConfig.setValue(PLAN_STRATEGY_CLASS_NAME, clusteringStrategyClass);
       return this;
     }
 
     public Builder withClusteringExecutionStrategyClass(String runClusteringStrategyClass) {
-      clusteringConfig.setValue(CLUSTERING_EXECUTION_STRATEGY_CLASS, runClusteringStrategyClass);
+      clusteringConfig.setValue(EXECUTION_STRATEGY_CLASS_NAME, runClusteringStrategyClass);
       return this;
     }
 
     public Builder withClusteringTargetPartitions(int clusteringTargetPartitions) {
-      clusteringConfig.setValue(CLUSTERING_TARGET_PARTITIONS, String.valueOf(clusteringTargetPartitions));
+      clusteringConfig.setValue(DAYBASED_LOOKBACK_PARTITIONS, String.valueOf(clusteringTargetPartitions));
       return this;
     }
 
     public Builder withClusteringSkipPartitionsFromLatest(int clusteringSkipPartitionsFromLatest) {
-      clusteringConfig.setValue(CLUSTERING_SKIP_PARTITIONS_FROM_LATEST, String.valueOf(clusteringSkipPartitionsFromLatest));
+      clusteringConfig.setValue(PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST, String.valueOf(clusteringSkipPartitionsFromLatest));
       return this;
     }
 
     public Builder withClusteringPlanSmallFileLimit(long clusteringSmallFileLimit) {
-      clusteringConfig.setValue(CLUSTERING_PLAN_SMALL_FILE_LIMIT, String.valueOf(clusteringSmallFileLimit));
+      clusteringConfig.setValue(PLAN_STRATEGY_SMALL_FILE_LIMIT, String.valueOf(clusteringSmallFileLimit));
       return this;
     }
     
     public Builder withClusteringSortColumns(String sortColumns) {
-      clusteringConfig.setValue(CLUSTERING_SORT_COLUMNS_PROPERTY, sortColumns);
+      clusteringConfig.setValue(PLAN_STRATEGY_SORT_COLUMNS, sortColumns);
       return this;
     }
 
     public Builder withClusteringMaxBytesInGroup(long clusteringMaxGroupSize) {
-      clusteringConfig.setValue(CLUSTERING_MAX_BYTES_PER_GROUP, String.valueOf(clusteringMaxGroupSize));
+      clusteringConfig.setValue(PLAN_STRATEGY_MAX_BYTES_PER_OUTPUT_FILEGROUP, String.valueOf(clusteringMaxGroupSize));
       return this;
     }
 
     public Builder withClusteringMaxNumGroups(int maxNumGroups) {
-      clusteringConfig.setValue(CLUSTERING_MAX_NUM_GROUPS, String.valueOf(maxNumGroups));
+      clusteringConfig.setValue(PLAN_STRATEGY_MAX_GROUPS, String.valueOf(maxNumGroups));
       return this;
     }
 
     public Builder withClusteringTargetFileMaxBytes(long targetFileSize) {
-      clusteringConfig.setValue(CLUSTERING_TARGET_FILE_MAX_BYTES, String.valueOf(targetFileSize));
+      clusteringConfig.setValue(PLAN_STRATEGY_TARGET_FILE_MAX_BYTES, String.valueOf(targetFileSize));
       return this;
     }
 
@@ -215,12 +321,12 @@ public class HoodieClusteringConfig extends HoodieConfig {
     }
 
     public Builder withInlineClusteringNumCommits(int numCommits) {
-      clusteringConfig.setValue(INLINE_CLUSTERING_MAX_COMMIT, String.valueOf(numCommits));
+      clusteringConfig.setValue(INLINE_CLUSTERING_MAX_COMMITS, String.valueOf(numCommits));
       return this;
     }
 
     public Builder withAsyncClusteringMaxCommits(int numCommits) {
-      clusteringConfig.setValue(ASYNC_CLUSTERING_MAX_COMMIT_PROP, String.valueOf(numCommits));
+      clusteringConfig.setValue(ASYNC_CLUSTERING_MAX_COMMITS, String.valueOf(numCommits));
       return this;
     }
 
@@ -230,7 +336,7 @@ public class HoodieClusteringConfig extends HoodieConfig {
     }
 
     public Builder withClusteringUpdatesStrategy(String updatesStrategyClass) {
-      clusteringConfig.setValue(CLUSTERING_UPDATES_STRATEGY, updatesStrategyClass);
+      clusteringConfig.setValue(UPDATES_STRATEGY, updatesStrategyClass);
       return this;
     }
 
@@ -240,7 +346,7 @@ public class HoodieClusteringConfig extends HoodieConfig {
     }
 
     public Builder withPreserveHoodieCommitMetadata(Boolean preserveHoodieCommitMetadata) {
-      clusteringConfig.setValue(CLUSTERING_PRESERVE_HOODIE_COMMIT_METADATA, String.valueOf(preserveHoodieCommitMetadata));
+      clusteringConfig.setValue(PRESERVE_COMMIT_METADATA, String.valueOf(preserveHoodieCommitMetadata));
       return this;
     }
 
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 1ae8495..ce74aad 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
@@ -56,24 +56,18 @@ public class HoodieCompactionConfig extends HoodieConfig {
       .withDocumentation("When enabled, the cleaner table service is invoked immediately after each commit,"
           + " to delete older file slices. It's recommended to enable this, to ensure metadata and data storage"
           + " growth is bounded.");
-  @Deprecated
-  public static final String AUTO_CLEAN_PROP = AUTO_CLEAN.key();
 
   public static final ConfigProperty<String> ASYNC_CLEAN = ConfigProperty
       .key("hoodie.clean.async")
       .defaultValue("false")
       .withDocumentation("Only applies when " + AUTO_CLEAN.key() + " is turned on. "
           + "When turned on runs cleaner async with writing, which can speed up overall write performance.");
-  @Deprecated
-  public static final String ASYNC_CLEAN_PROP = ASYNC_CLEAN.key();;
 
   public static final ConfigProperty<String> CLEANER_COMMITS_RETAINED = ConfigProperty
       .key("hoodie.cleaner.commits.retained")
       .defaultValue("10")
       .withDocumentation("Number of commits to retain, without cleaning. This will be retained for num_of_commits * time_between_commits "
           + "(scheduled). This also directly translates into how much data retention the table supports for incremental queries.");
-  @Deprecated
-  public static final String CLEANER_COMMITS_RETAINED_PROP = CLEANER_COMMITS_RETAINED.key();;
 
   public static final ConfigProperty<String> CLEANER_POLICY = ConfigProperty
       .key("hoodie.cleaner.policy")
@@ -82,48 +76,36 @@ public class HoodieCompactionConfig extends HoodieConfig {
           + " By default, cleaner spares the file slices written by the last N commits, determined by  " + CLEANER_COMMITS_RETAINED.key()
           + " Long running query plans may often refer to older file slices and will break if those are cleaned, before the query has had"
           + "   a chance to run. So, it is good to make sure that the data is retained for more than the maximum query execution time");
-  @Deprecated
-  public static final String CLEANER_POLICY_PROP = CLEANER_POLICY.key();
 
   public static final ConfigProperty<String> INLINE_COMPACT = ConfigProperty
       .key("hoodie.compact.inline")
       .defaultValue("false")
       .withDocumentation("When set to true, compaction service is triggered after each write. While being "
           + " simpler operationally, this adds extra latency on the write path.");
-  @Deprecated
-  public static final String INLINE_COMPACT_PROP = INLINE_COMPACT.key();
 
   public static final ConfigProperty<String> INLINE_COMPACT_NUM_DELTA_COMMITS = ConfigProperty
       .key("hoodie.compact.inline.max.delta.commits")
       .defaultValue("5")
       .withDocumentation("Number of delta commits after the last compaction, before scheduling of a new compaction is attempted.");
-  @Deprecated
-  public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = INLINE_COMPACT_NUM_DELTA_COMMITS.key();
 
   public static final ConfigProperty<String> INLINE_COMPACT_TIME_DELTA_SECONDS = ConfigProperty
       .key("hoodie.compact.inline.max.delta.seconds")
       .defaultValue(String.valueOf(60 * 60))
       .withDocumentation("Number of elapsed seconds after the last compaction, before scheduling a new one.");
-  @Deprecated
-  public static final String INLINE_COMPACT_TIME_DELTA_SECONDS_PROP = INLINE_COMPACT_TIME_DELTA_SECONDS.key();
 
   public static final ConfigProperty<String> INLINE_COMPACT_TRIGGER_STRATEGY = ConfigProperty
       .key("hoodie.compact.inline.trigger.strategy")
       .defaultValue(CompactionTriggerStrategy.NUM_COMMITS.name())
       .withDocumentation("Controls how compaction scheduling is triggered, by time or num delta commits or combination of both. "
           + "Valid options: " + Arrays.stream(CompactionTriggerStrategy.values()).map(Enum::name).collect(Collectors.joining(",")));
-  @Deprecated
-  public static final String INLINE_COMPACT_TRIGGER_STRATEGY_PROP = INLINE_COMPACT_TRIGGER_STRATEGY.key();
 
   public static final ConfigProperty<String> CLEANER_FILE_VERSIONS_RETAINED = ConfigProperty
       .key("hoodie.cleaner.fileversions.retained")
       .defaultValue("3")
       .withDocumentation("When " + HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS.name() + " cleaning policy is used, "
           + " the minimum number of file slices to retain in each file group, during cleaning.");
-  @Deprecated
-  public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP = CLEANER_COMMITS_RETAINED.key();
 
-  public static final ConfigProperty<String> CLEANER_INCREMENTAL_MODE = ConfigProperty
+  public static final ConfigProperty<String> CLEANER_INCREMENTAL_MODE_ENABLE = ConfigProperty
       .key("hoodie.cleaner.incremental.mode")
       .defaultValue("true")
       .withDocumentation("When enabled, the plans for each cleaner service run is computed incrementally off the events "
@@ -136,26 +118,20 @@ public class HoodieCompactionConfig extends HoodieConfig {
       .withDocumentation("Archiving service moves older entries from timeline into an archived log after each write, to "
           + " keep the metadata overhead constant, even as the table size grows."
           + "This config controls the maximum number of instants to retain in the active timeline. ");
-  @Deprecated
-  public static final String MAX_COMMITS_TO_KEEP_PROP = MAX_COMMITS_TO_KEEP.key();
 
   public static final ConfigProperty<String> MIN_COMMITS_TO_KEEP = ConfigProperty
       .key("hoodie.keep.min.commits")
       .defaultValue("20")
       .withDocumentation("Similar to " + MAX_COMMITS_TO_KEEP.key() + ", but controls the minimum number of"
           + "instants to retain in the active timeline.");
-  @Deprecated
-  public static final String MIN_COMMITS_TO_KEEP_PROP = MIN_COMMITS_TO_KEEP.key();
 
   public static final ConfigProperty<String> COMMITS_ARCHIVAL_BATCH_SIZE = ConfigProperty
       .key("hoodie.commits.archival.batch")
       .defaultValue(String.valueOf(10))
       .withDocumentation("Archiving of instants is batched in best-effort manner, to pack more instants into a single"
           + " archive log. This config controls such archival batch size.");
-  @Deprecated
-  public static final String COMMITS_ARCHIVAL_BATCH_SIZE_PROP = COMMITS_ARCHIVAL_BATCH_SIZE.key();
 
-  public static final ConfigProperty<String> CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = ConfigProperty
+  public static final ConfigProperty<String> CLEANER_BOOTSTRAP_BASE_FILE_ENABLE = ConfigProperty
       .key("hoodie.cleaner.delete.bootstrap.base.file")
       .defaultValue("false")
       .withDocumentation("When set to true, cleaner also deletes the bootstrap base file when it's skeleton base file is "
@@ -163,7 +139,7 @@ public class HoodieCompactionConfig extends HoodieConfig {
           + " table receives updates/deletes. Another reason to turn this on, would be to ensure data residing in bootstrap "
           + " base files are also physically deleted, to comply with data privacy enforcement processes.");
 
-  public static final ConfigProperty<String> PARQUET_SMALL_FILE_LIMIT_BYTES = ConfigProperty
+  public static final ConfigProperty<String> PARQUET_SMALL_FILE_LIMIT = ConfigProperty
       .key("hoodie.parquet.small.file.limit")
       .defaultValue(String.valueOf(104857600))
       .withDocumentation("During upsert operation, we opportunistically expand existing small files on storage, instead of writing"
@@ -177,10 +153,8 @@ public class HoodieCompactionConfig extends HoodieConfig {
           + " to bin pack records into partitions. If the previous commit is too small to make an accurate estimation, "
           + " Hudi will search commits in the reverse order, until we find a commit that has totalBytesWritten "
           + " larger than (PARQUET_SMALL_FILE_LIMIT_BYTES * this_threshold)");
-  @Deprecated
-  public static final String RECORD_SIZE_ESTIMATION_THRESHOLD_PROP = RECORD_SIZE_ESTIMATION_THRESHOLD.key();
 
-  public static final ConfigProperty<String> CLEANER_PARALLELISM = ConfigProperty
+  public static final ConfigProperty<String> CLEANER_PARALLELISM_VALUE = ConfigProperty
       .key("hoodie.cleaner.parallelism")
       .defaultValue("200")
       .withDocumentation("Parallelism for the cleaning operation. Increase this if cleaning becomes slow.");
@@ -191,8 +165,6 @@ public class HoodieCompactionConfig extends HoodieConfig {
       .defaultValue(String.valueOf(500 * 1024))
       .withDocumentation("Amount of MBs to spend during compaction run for the LogFileSizeBasedCompactionStrategy. "
           + "This value helps bound ingestion latency while compaction is run inline mode.");
-  @Deprecated
-  public static final String TARGET_IO_PER_COMPACTION_IN_MB_PROP = TARGET_IO_PER_COMPACTION_IN_MB.key();
 
   public static final ConfigProperty<String> COMPACTION_STRATEGY = ConfigProperty
       .key("hoodie.compaction.strategy")
@@ -200,34 +172,26 @@ public class HoodieCompactionConfig extends HoodieConfig {
       .withDocumentation("Compaction strategy decides which file groups are picked up for "
           + "compaction during each compaction run. By default. Hudi picks the log file "
           + "with most accumulated unmerged data");
-  @Deprecated
-  public static final String COMPACTION_STRATEGY_PROP = COMPACTION_STRATEGY.key();
 
-  public static final ConfigProperty<String> PAYLOAD_CLASS = ConfigProperty
+  public static final ConfigProperty<String> PAYLOAD_CLASS_NAME = ConfigProperty
       .key("hoodie.compaction.payload.class")
       .defaultValue(OverwriteWithLatestAvroPayload.class.getName())
       .withDocumentation("This needs to be same as class used during insert/upserts. Just like writing, compaction also uses "
           + "the record payload class to merge records in the log against each other, merge again with the base file and "
           + "produce the final record to be written after compaction.");
-  @Deprecated
-  public static final String PAYLOAD_CLASS_PROP = PAYLOAD_CLASS.key();
 
-  public static final ConfigProperty<String> COMPACTION_LAZY_BLOCK_READ_ENABLED = ConfigProperty
+  public static final ConfigProperty<String> COMPACTION_LAZY_BLOCK_READ_ENABLE = ConfigProperty
       .key("hoodie.compaction.lazy.block.read")
       .defaultValue("false")
       .withDocumentation("When merging the delta log files, this config helps to choose whether the log blocks "
           + "should be read lazily or not. Choose true to use lazy block reading (low memory usage, but incurs seeks to each block"
           + " header) or false for immediate block read (higher memory usage)");
-  @Deprecated
-  public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = COMPACTION_LAZY_BLOCK_READ_ENABLED.key();
 
-  public static final ConfigProperty<String> COMPACTION_REVERSE_LOG_READ_ENABLED = ConfigProperty
+  public static final ConfigProperty<String> COMPACTION_REVERSE_LOG_READ_ENABLE = ConfigProperty
       .key("hoodie.compaction.reverse.log.read")
       .defaultValue("false")
       .withDocumentation("HoodieLogFormatReader reads a logfile in the forward direction starting from pos=0 to pos=file_length. "
           + "If this config is set to true, the reader reads the logfile in reverse direction, from pos=file_length to pos=0");
-  @Deprecated
-  public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP = COMPACTION_REVERSE_LOG_READ_ENABLED.key();
 
   public static final ConfigProperty<String> FAILED_WRITES_CLEANER_POLICY = ConfigProperty
       .key("hoodie.cleaner.policy.failed.writes")
@@ -235,21 +199,17 @@ public class HoodieCompactionConfig extends HoodieConfig {
       .withDocumentation("Cleaning policy for failed writes to be used. Hudi will delete any files written by "
           + "failed writes to re-claim space. Choose to perform this rollback of failed writes eagerly before "
           + "every writer starts (only supported for single writer) or lazily by the cleaner (required for multi-writers)");
-  @Deprecated
-  public static final String FAILED_WRITES_CLEANER_POLICY_PROP = FAILED_WRITES_CLEANER_POLICY.key();
 
   public static final ConfigProperty<String> TARGET_PARTITIONS_PER_DAYBASED_COMPACTION = ConfigProperty
       .key("hoodie.compaction.daybased.target.partitions")
       .defaultValue("10")
       .withDocumentation("Used by org.apache.hudi.io.compact.strategy.DayBasedCompactionStrategy to denote the number of "
           + "latest partitions to compact during a compaction run.");
-  @Deprecated
-  public static final String TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP = TARGET_PARTITIONS_PER_DAYBASED_COMPACTION.key();
 
   /**
    * Configs related to specific table types.
    */
-  public static final ConfigProperty<String> COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = ConfigProperty
+  public static final ConfigProperty<String> COPY_ON_WRITE_INSERT_SPLIT_SIZE = ConfigProperty
       .key("hoodie.copyonwrite.insert.split.size")
       .defaultValue(String.valueOf(500000))
       .withDocumentation("Number of inserts assigned for each partition/bucket for writing. "
@@ -257,20 +217,237 @@ public class HoodieCompactionConfig extends HoodieConfig {
           + "  over provision to 500K. As long as auto-tuning of splits is turned on, this only affects the first "
           + "  write, where there is no history to learn record sizes from.");
 
-  public static final ConfigProperty<String> COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = ConfigProperty
+  public static final ConfigProperty<String> COPY_ON_WRITE_AUTO_SPLIT_INSERTS = ConfigProperty
       .key("hoodie.copyonwrite.insert.auto.split")
       .defaultValue("true")
       .withDocumentation("Config to control whether we control insert split sizes automatically based on average"
           + " record sizes. It's recommended to keep this turned on, since hand tuning is otherwise extremely"
           + " cumbersome.");
 
-  public static final ConfigProperty<String> COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = ConfigProperty
+  public static final ConfigProperty<String> COPY_ON_WRITE_RECORD_SIZE_ESTIMATE = ConfigProperty
       .key("hoodie.copyonwrite.record.size.estimate")
       .defaultValue(String.valueOf(1024))
       .withDocumentation("The average record size. If not explicitly specified, hudi will compute the "
           + "record size estimate compute dynamically based on commit metadata. "
           + " This is critical in computing the insert parallelism and bin-packing inserts into small files.");
 
+  /** @deprecated Use {@link #CLEANER_POLICY} and its methods instead */
+  @Deprecated
+  public static final String CLEANER_POLICY_PROP = CLEANER_POLICY.key();
+  /** @deprecated Use {@link #AUTO_CLEAN} and its methods instead */
+  @Deprecated
+  public static final String AUTO_CLEAN_PROP = AUTO_CLEAN.key();
+  /** @deprecated Use {@link #ASYNC_CLEAN} and its methods instead */
+  @Deprecated
+  public static final String ASYNC_CLEAN_PROP = ASYNC_CLEAN.key();
+  /** @deprecated Use {@link #INLINE_COMPACT} and its methods instead */
+  @Deprecated
+  public static final String INLINE_COMPACT_PROP = INLINE_COMPACT.key();
+  /** @deprecated Use {@link #INLINE_COMPACT_NUM_DELTA_COMMITS} and its methods instead */
+  @Deprecated
+  public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = INLINE_COMPACT_NUM_DELTA_COMMITS.key();
+  /** @deprecated Use {@link #INLINE_COMPACT_TIME_DELTA_SECONDS} and its methods instead */
+  @Deprecated
+  public static final String INLINE_COMPACT_TIME_DELTA_SECONDS_PROP = INLINE_COMPACT_TIME_DELTA_SECONDS.key();
+  /** @deprecated Use {@link #INLINE_COMPACT_TRIGGER_STRATEGY} and its methods instead */
+  @Deprecated
+  public static final String INLINE_COMPACT_TRIGGER_STRATEGY_PROP = INLINE_COMPACT_TRIGGER_STRATEGY.key();
+  /** @deprecated Use {@link #CLEANER_FILE_VERSIONS_RETAINED} and its methods instead */
+  @Deprecated
+  public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP = CLEANER_FILE_VERSIONS_RETAINED.key();
+  /**
+   * @deprecated Use {@link #CLEANER_COMMITS_RETAINED} and its methods instead
+   */
+  @Deprecated
+  public static final String CLEANER_COMMITS_RETAINED_PROP = CLEANER_COMMITS_RETAINED.key();
+  /**
+   * @deprecated Use {@link #CLEANER_INCREMENTAL_MODE_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String CLEANER_INCREMENTAL_MODE = CLEANER_INCREMENTAL_MODE_ENABLE.key();
+  /**
+   * @deprecated Use {@link #MAX_COMMITS_TO_KEEP} and its methods instead
+   */
+  @Deprecated
+  public static final String MAX_COMMITS_TO_KEEP_PROP = MAX_COMMITS_TO_KEEP.key();
+  /**
+   * @deprecated Use {@link #MIN_COMMITS_TO_KEEP} and its methods instead
+   */
+  @Deprecated
+  public static final String MIN_COMMITS_TO_KEEP_PROP = MIN_COMMITS_TO_KEEP.key();
+  /**
+   * @deprecated Use {@link #COMMITS_ARCHIVAL_BATCH_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String COMMITS_ARCHIVAL_BATCH_SIZE_PROP = COMMITS_ARCHIVAL_BATCH_SIZE.key();
+  /**
+   * @deprecated Use {@link #CLEANER_BOOTSTRAP_BASE_FILE_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = CLEANER_BOOTSTRAP_BASE_FILE_ENABLE.key();
+  /**
+   * @deprecated Use {@link #PARQUET_SMALL_FILE_LIMIT} and its methods instead
+   */
+  @Deprecated
+  public static final String PARQUET_SMALL_FILE_LIMIT_BYTES = PARQUET_SMALL_FILE_LIMIT.key();
+  /**
+   * @deprecated Use {@link #PARQUET_SMALL_FILE_LIMIT} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES = PARQUET_SMALL_FILE_LIMIT.defaultValue();
+  /**
+   * @deprecated Use {@link #RECORD_SIZE_ESTIMATION_THRESHOLD} and its methods instead
+   */
+  @Deprecated
+  public static final String RECORD_SIZE_ESTIMATION_THRESHOLD_PROP = RECORD_SIZE_ESTIMATION_THRESHOLD.key();
+  /**
+   * @deprecated Use {@link #RECORD_SIZE_ESTIMATION_THRESHOLD} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_RECORD_SIZE_ESTIMATION_THRESHOLD = RECORD_SIZE_ESTIMATION_THRESHOLD.defaultValue();
+  /**
+   * @deprecated Use {@link #COPY_ON_WRITE_INSERT_SPLIT_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = COPY_ON_WRITE_INSERT_SPLIT_SIZE.key();
+  /**
+   * @deprecated Use {@link #COPY_ON_WRITE_INSERT_SPLIT_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = COPY_ON_WRITE_INSERT_SPLIT_SIZE.defaultValue();
+  /**
+   * @deprecated Use {@link #COPY_ON_WRITE_AUTO_SPLIT_INSERTS} and its methods instead
+   */
+  @Deprecated
+  public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = COPY_ON_WRITE_AUTO_SPLIT_INSERTS.key();
+  /**
+   * @deprecated Use {@link #COPY_ON_WRITE_AUTO_SPLIT_INSERTS} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = COPY_ON_WRITE_AUTO_SPLIT_INSERTS.defaultValue();
+  /**
+   * @deprecated Use {@link #COPY_ON_WRITE_RECORD_SIZE_ESTIMATE} and its methods instead
+   */
+  @Deprecated
+  public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = COPY_ON_WRITE_RECORD_SIZE_ESTIMATE.key();
+  /**
+   * @deprecated Use {@link #COPY_ON_WRITE_RECORD_SIZE_ESTIMATE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = COPY_ON_WRITE_RECORD_SIZE_ESTIMATE.defaultValue();
+  /**
+   * @deprecated Use {@link #CLEANER_PARALLELISM_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String CLEANER_PARALLELISM = CLEANER_PARALLELISM_VALUE.key();
+  /**
+   * @deprecated Use {@link #CLEANER_PARALLELISM_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_CLEANER_PARALLELISM = CLEANER_PARALLELISM_VALUE.defaultValue();
+  /**
+   * @deprecated Use {@link #TARGET_IO_PER_COMPACTION_IN_MB} and its methods instead
+   */
+  @Deprecated
+  public static final String TARGET_IO_PER_COMPACTION_IN_MB_PROP = TARGET_IO_PER_COMPACTION_IN_MB.key();
+  /**
+   * @deprecated Use {@link #TARGET_IO_PER_COMPACTION_IN_MB} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB = TARGET_IO_PER_COMPACTION_IN_MB.defaultValue();
+  /**
+   * @deprecated Use {@link #COMPACTION_STRATEGY} and its methods instead
+   */
+  @Deprecated
+  public static final String COMPACTION_STRATEGY_PROP = COMPACTION_STRATEGY.key();
+  /** @deprecated Use {@link #COMPACTION_STRATEGY} and its methods instead */
+  @Deprecated
+  public static final String DEFAULT_COMPACTION_STRATEGY = COMPACTION_STRATEGY.defaultValue();
+  /** @deprecated Use {@link #PAYLOAD_CLASS_NAME} and its methods instead */
+  @Deprecated
+  public static final String DEFAULT_PAYLOAD_CLASS = PAYLOAD_CLASS_NAME.defaultValue();
+  /** @deprecated Use {@link #PAYLOAD_CLASS_NAME} and its methods instead */
+  @Deprecated
+  public static final String PAYLOAD_CLASS_PROP = PAYLOAD_CLASS_NAME.key();
+  /** @deprecated Use {@link #COMPACTION_LAZY_BLOCK_READ_ENABLE} and its methods instead */
+  @Deprecated
+  public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = COMPACTION_LAZY_BLOCK_READ_ENABLE.key();
+  /** @deprecated Use {@link #COMPACTION_LAZY_BLOCK_READ_ENABLE} and its methods instead */
+  @Deprecated
+  public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = COMPACTION_REVERSE_LOG_READ_ENABLE.defaultValue();
+  /** @deprecated Use {@link #COMPACTION_REVERSE_LOG_READ_ENABLE} and its methods instead */
+  @Deprecated
+  public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP = COMPACTION_REVERSE_LOG_READ_ENABLE.key();
+  /** @deprecated Use {@link #COMPACTION_REVERSE_LOG_READ_ENABLE} and its methods instead */
+  @Deprecated
+  public static final String DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED = COMPACTION_REVERSE_LOG_READ_ENABLE.defaultValue();
+  /** @deprecated Use {@link #CLEANER_POLICY} and its methods instead */
+  @Deprecated
+  private static final String DEFAULT_CLEANER_POLICY = CLEANER_POLICY.defaultValue();
+  /** @deprecated Use {@link #FAILED_WRITES_CLEANER_POLICY} and its methods instead */
+  @Deprecated
+  public static final String FAILED_WRITES_CLEANER_POLICY_PROP = FAILED_WRITES_CLEANER_POLICY.key();
+  /** @deprecated Use {@link #FAILED_WRITES_CLEANER_POLICY} and its methods instead */
+  @Deprecated
+  private  static final String DEFAULT_FAILED_WRITES_CLEANER_POLICY = FAILED_WRITES_CLEANER_POLICY.defaultValue();
+  /** @deprecated Use {@link #AUTO_CLEAN} and its methods instead */
+  @Deprecated
+  private static final String DEFAULT_AUTO_CLEAN = AUTO_CLEAN.defaultValue();
+  /**
+   * @deprecated Use {@link #ASYNC_CLEAN} and its methods instead
+   */
+  @Deprecated
+  private static final String DEFAULT_ASYNC_CLEAN = ASYNC_CLEAN.defaultValue();
+  /**
+   * @deprecated Use {@link #INLINE_COMPACT} and its methods instead
+   */
+  @Deprecated
+  private static final String DEFAULT_INLINE_COMPACT = INLINE_COMPACT.defaultValue();
+  /**
+   * @deprecated Use {@link #CLEANER_INCREMENTAL_MODE_ENABLE} and its methods instead
+   */
+  @Deprecated
+  private static final String DEFAULT_INCREMENTAL_CLEANER = CLEANER_INCREMENTAL_MODE_ENABLE.defaultValue();
+  /** @deprecated Use {@link #INLINE_COMPACT_NUM_DELTA_COMMITS} and its methods instead */
+  @Deprecated
+  private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = INLINE_COMPACT_NUM_DELTA_COMMITS.defaultValue();
+  /** @deprecated Use {@link #INLINE_COMPACT_TIME_DELTA_SECONDS} and its methods instead */
+  @Deprecated
+  private static final String DEFAULT_INLINE_COMPACT_TIME_DELTA_SECONDS = INLINE_COMPACT_TIME_DELTA_SECONDS.defaultValue();
+  /** @deprecated Use {@link #INLINE_COMPACT_TRIGGER_STRATEGY} and its methods instead */
+  @Deprecated
+  private static final String DEFAULT_INLINE_COMPACT_TRIGGER_STRATEGY = INLINE_COMPACT_TRIGGER_STRATEGY.defaultValue();
+  /** @deprecated Use {@link #CLEANER_FILE_VERSIONS_RETAINED} and its methods instead */
+  @Deprecated
+  private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = CLEANER_FILE_VERSIONS_RETAINED.defaultValue();
+  /** @deprecated Use {@link #CLEANER_COMMITS_RETAINED} and its methods instead */
+  @Deprecated
+  private static final String DEFAULT_CLEANER_COMMITS_RETAINED = CLEANER_COMMITS_RETAINED.defaultValue();
+  /** @deprecated Use {@link #MAX_COMMITS_TO_KEEP} and its methods instead */
+  @Deprecated
+  private static final String DEFAULT_MAX_COMMITS_TO_KEEP = MAX_COMMITS_TO_KEEP.defaultValue();
+  /**
+   * @deprecated Use {@link #MIN_COMMITS_TO_KEEP} and its methods instead
+   */
+  @Deprecated
+  private static final String DEFAULT_MIN_COMMITS_TO_KEEP = MIN_COMMITS_TO_KEEP.defaultValue();
+  /**
+   * @deprecated Use {@link #COMMITS_ARCHIVAL_BATCH_SIZE} and its methods instead
+   */
+  @Deprecated
+  private static final String DEFAULT_COMMITS_ARCHIVAL_BATCH_SIZE = COMMITS_ARCHIVAL_BATCH_SIZE.defaultValue();
+  /**
+   * @deprecated Use {@link #CLEANER_BOOTSTRAP_BASE_FILE_ENABLE} and its methods instead
+   */
+  @Deprecated
+  private static final String DEFAULT_CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = CLEANER_BOOTSTRAP_BASE_FILE_ENABLE.defaultValue();
+  /** @deprecated Use {@link #TARGET_PARTITIONS_PER_DAYBASED_COMPACTION} and its methods instead */
+  @Deprecated
+  public static final String TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP = TARGET_PARTITIONS_PER_DAYBASED_COMPACTION.key();
+  /** @deprecated Use {@link #TARGET_PARTITIONS_PER_DAYBASED_COMPACTION} and its methods instead */
+  @Deprecated
+  public static final String DEFAULT_TARGET_PARTITIONS_PER_DAYBASED_COMPACTION = TARGET_PARTITIONS_PER_DAYBASED_COMPACTION.defaultValue();
+
   private HoodieCompactionConfig() {
     super();
   }
@@ -306,7 +483,7 @@ public class HoodieCompactionConfig extends HoodieConfig {
     }
 
     public Builder withIncrementalCleaningMode(Boolean incrementalCleaningMode) {
-      compactionConfig.setValue(CLEANER_INCREMENTAL_MODE, String.valueOf(incrementalCleaningMode));
+      compactionConfig.setValue(CLEANER_INCREMENTAL_MODE_ENABLE, String.valueOf(incrementalCleaningMode));
       return this;
     }
 
@@ -342,7 +519,7 @@ public class HoodieCompactionConfig extends HoodieConfig {
     }
 
     public Builder compactionSmallFileSize(long smallFileLimitBytes) {
-      compactionConfig.setValue(PARQUET_SMALL_FILE_LIMIT_BYTES, String.valueOf(smallFileLimitBytes));
+      compactionConfig.setValue(PARQUET_SMALL_FILE_LIMIT, String.valueOf(smallFileLimitBytes));
       return this;
     }
 
@@ -352,22 +529,22 @@ public class HoodieCompactionConfig extends HoodieConfig {
     }
 
     public Builder insertSplitSize(int insertSplitSize) {
-      compactionConfig.setValue(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, String.valueOf(insertSplitSize));
+      compactionConfig.setValue(COPY_ON_WRITE_INSERT_SPLIT_SIZE, String.valueOf(insertSplitSize));
       return this;
     }
 
     public Builder autoTuneInsertSplits(boolean autoTuneInsertSplits) {
-      compactionConfig.setValue(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, String.valueOf(autoTuneInsertSplits));
+      compactionConfig.setValue(COPY_ON_WRITE_AUTO_SPLIT_INSERTS, String.valueOf(autoTuneInsertSplits));
       return this;
     }
 
     public Builder approxRecordSize(int recordSizeEstimate) {
-      compactionConfig.setValue(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE, String.valueOf(recordSizeEstimate));
+      compactionConfig.setValue(COPY_ON_WRITE_RECORD_SIZE_ESTIMATE, String.valueOf(recordSizeEstimate));
       return this;
     }
 
     public Builder withCleanerParallelism(int cleanerParallelism) {
-      compactionConfig.setValue(CLEANER_PARALLELISM, String.valueOf(cleanerParallelism));
+      compactionConfig.setValue(CLEANER_PARALLELISM_VALUE, String.valueOf(cleanerParallelism));
       return this;
     }
 
@@ -377,7 +554,7 @@ public class HoodieCompactionConfig extends HoodieConfig {
     }
 
     public Builder withPayloadClass(String payloadClassName) {
-      compactionConfig.setValue(PAYLOAD_CLASS, payloadClassName);
+      compactionConfig.setValue(PAYLOAD_CLASS_NAME, payloadClassName);
       return this;
     }
 
@@ -397,12 +574,12 @@ public class HoodieCompactionConfig extends HoodieConfig {
     }
 
     public Builder withCompactionLazyBlockReadEnabled(Boolean compactionLazyBlockReadEnabled) {
-      compactionConfig.setValue(COMPACTION_LAZY_BLOCK_READ_ENABLED, String.valueOf(compactionLazyBlockReadEnabled));
+      compactionConfig.setValue(COMPACTION_LAZY_BLOCK_READ_ENABLE, String.valueOf(compactionLazyBlockReadEnabled));
       return this;
     }
 
     public Builder withCompactionReverseLogReadEnabled(Boolean compactionReverseLogReadEnabled) {
-      compactionConfig.setValue(COMPACTION_REVERSE_LOG_READ_ENABLED, String.valueOf(compactionReverseLogReadEnabled));
+      compactionConfig.setValue(COMPACTION_REVERSE_LOG_READ_ENABLE, String.valueOf(compactionReverseLogReadEnabled));
       return this;
     }
 
@@ -417,7 +594,7 @@ public class HoodieCompactionConfig extends HoodieConfig {
     }
 
     public Builder withCleanBootstrapBaseFileEnabled(Boolean cleanBootstrapSourceFileEnabled) {
-      compactionConfig.setValue(CLEANER_BOOTSTRAP_BASE_FILE_ENABLED, String.valueOf(cleanBootstrapSourceFileEnabled));
+      compactionConfig.setValue(CLEANER_BOOTSTRAP_BASE_FILE_ENABLE, String.valueOf(cleanBootstrapSourceFileEnabled));
       return this;
     }
 
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java
index 2c8da8b..7d048d5 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieHBaseIndexConfig.java
@@ -36,138 +36,114 @@ import java.util.Properties;
         + "records as either inserts or updates to older records.")
 public class HoodieHBaseIndexConfig extends HoodieConfig {
 
-  public static final ConfigProperty<String> HBASE_ZKQUORUM = ConfigProperty
+  public static final ConfigProperty<String> ZKQUORUM = ConfigProperty
       .key("hoodie.index.hbase.zkquorum")
       .noDefaultValue()
       .withDocumentation("Only applies if index type is HBASE. HBase ZK Quorum url to connect to");
-  @Deprecated
-  public static final String HBASE_ZKQUORUM_PROP = HBASE_ZKQUORUM.key();
 
-  public static final ConfigProperty<String> HBASE_ZKPORT = ConfigProperty
+  public static final ConfigProperty<String> ZKPORT = ConfigProperty
       .key("hoodie.index.hbase.zkport")
       .noDefaultValue()
       .withDocumentation("Only applies if index type is HBASE. HBase ZK Quorum port to connect to");
-  @Deprecated
-  public static final String HBASE_ZKPORT_PROP = HBASE_ZKPORT.key();
 
-  public static final ConfigProperty<String> HBASE_TABLENAME = ConfigProperty
+  public static final ConfigProperty<String> TABLENAME = ConfigProperty
       .key("hoodie.index.hbase.table")
       .noDefaultValue()
       .withDocumentation("Only applies if index type is HBASE. HBase Table name to use as the index. "
           + "Hudi stores the row_key and [partition_path, fileID, commitTime] mapping in the table");
-  @Deprecated
-  public static final String HBASE_TABLENAME_PROP = HBASE_TABLENAME.key();
 
-  public static final ConfigProperty<Integer> HBASE_GET_BATCH_SIZE = ConfigProperty
+  public static final ConfigProperty<Integer> GET_BATCH_SIZE = ConfigProperty
       .key("hoodie.index.hbase.get.batch.size")
       .defaultValue(100)
       .withDocumentation("Controls the batch size for performing gets against HBase. "
           + "Batching improves throughput, by saving round trips.");
-  @Deprecated
-  public static final String HBASE_GET_BATCH_SIZE_PROP = HBASE_GET_BATCH_SIZE.key();
 
-  public static final ConfigProperty<String> HBASE_ZK_ZNODEPARENT = ConfigProperty
+  public static final ConfigProperty<String> ZK_NODE_PATH = ConfigProperty
       .key("hoodie.index.hbase.zknode.path")
       .noDefaultValue()
       .withDocumentation("Only applies if index type is HBASE. This is the root znode that will contain "
           + "all the znodes created/used by HBase");
 
-  public static final ConfigProperty<Integer> HBASE_PUT_BATCH_SIZE = ConfigProperty
+  public static final ConfigProperty<Integer> PUT_BATCH_SIZE = ConfigProperty
       .key("hoodie.index.hbase.put.batch.size")
       .defaultValue(100)
       .withDocumentation("Controls the batch size for performing puts against HBase. "
           + "Batching improves throughput, by saving round trips.");
-  @Deprecated
-  public static final String HBASE_PUT_BATCH_SIZE_PROP = HBASE_PUT_BATCH_SIZE.key();
 
-  public static final ConfigProperty<String> HBASE_INDEX_QPS_ALLOCATOR_CLASS = ConfigProperty
+  public static final ConfigProperty<String> QPS_ALLOCATOR_CLASS_NAME = ConfigProperty
       .key("hoodie.index.hbase.qps.allocator.class")
       .defaultValue(DefaultHBaseQPSResourceAllocator.class.getName())
       .withDocumentation("Property to set which implementation of HBase QPS resource allocator to be used, which"
           + "controls the batching rate dynamically.");
 
-  public static final ConfigProperty<String> HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE = ConfigProperty
+  public static final ConfigProperty<String> PUT_BATCH_SIZE_AUTO_COMPUTE = ConfigProperty
       .key("hoodie.index.hbase.put.batch.size.autocompute")
       .defaultValue("false")
       .withDocumentation("Property to set to enable auto computation of put batch size");
-  @Deprecated
-  public static final String HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP = HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE.key();
 
-  public static final ConfigProperty<Float> HBASE_QPS_FRACTION = ConfigProperty
+  public static final ConfigProperty<Float> QPS_FRACTION = ConfigProperty
       .key("hoodie.index.hbase.qps.fraction")
       .defaultValue(0.5f)
       .withDocumentation("Property to set the fraction of the global share of QPS that should be allocated to this job. Let's say there are 3"
           + " jobs which have input size in terms of number of rows required for HbaseIndexing as x, 2x, 3x respectively. Then"
           + " this fraction for the jobs would be (0.17) 1/6, 0.33 (2/6) and 0.5 (3/6) respectively."
           + " Default is 50%, which means a total of 2 jobs can run using HbaseIndex without overwhelming Region Servers.");
-  @Deprecated
-  public static final String HBASE_QPS_FRACTION_PROP = HBASE_QPS_FRACTION.key();
 
-  public static final ConfigProperty<Integer> HBASE_MAX_QPS_PER_REGION_SERVER = ConfigProperty
+  public static final ConfigProperty<Integer> MAX_QPS_PER_REGION_SERVER = ConfigProperty
       .key("hoodie.index.hbase.max.qps.per.region.server")
       .defaultValue(1000)
       .withDocumentation("Property to set maximum QPS allowed per Region Server. This should be same across various jobs. This is intended to\n"
           + " limit the aggregate QPS generated across various jobs to an Hbase Region Server. It is recommended to set this\n"
           + " value based on global indexing throughput needs and most importantly, how much the HBase installation in use is\n"
           + " able to tolerate without Region Servers going down.");
-  @Deprecated
-  public static final String HBASE_MAX_QPS_PER_REGION_SERVER_PROP = HBASE_MAX_QPS_PER_REGION_SERVER.key();
 
-  public static final ConfigProperty<Boolean> HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY = ConfigProperty
+  public static final ConfigProperty<Boolean> COMPUTE_QPS_DYNAMICALLY = ConfigProperty
       .key("hoodie.index.hbase.dynamic_qps")
       .defaultValue(false)
       .withDocumentation("Property to decide if HBASE_QPS_FRACTION_PROP is dynamically calculated based on write volume.");
 
-  public static final ConfigProperty<String> HBASE_MIN_QPS_FRACTION = ConfigProperty
+  public static final ConfigProperty<String> MIN_QPS_FRACTION = ConfigProperty
       .key("hoodie.index.hbase.min.qps.fraction")
       .noDefaultValue()
       .withDocumentation("Minimum for HBASE_QPS_FRACTION_PROP to stabilize skewed write workloads");
-  @Deprecated
-  public static final String HBASE_MIN_QPS_FRACTION_PROP = HBASE_MIN_QPS_FRACTION.key();
 
-  public static final ConfigProperty<String> HBASE_MAX_QPS_FRACTION = ConfigProperty
+  public static final ConfigProperty<String> MAX_QPS_FRACTION = ConfigProperty
       .key("hoodie.index.hbase.max.qps.fraction")
       .noDefaultValue()
       .withDocumentation("Maximum for HBASE_QPS_FRACTION_PROP to stabilize skewed write workloads");
-  @Deprecated
-  public static final String HBASE_MAX_QPS_FRACTION_PROP = HBASE_MAX_QPS_FRACTION.key();
 
-  public static final ConfigProperty<Integer> HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS = ConfigProperty
+  public static final ConfigProperty<Integer> DESIRED_PUTS_TIME_IN_SECONDS = ConfigProperty
       .key("hoodie.index.hbase.desired_puts_time_in_secs")
       .defaultValue(600)
       .withDocumentation("");
 
-  public static final ConfigProperty<String> HBASE_SLEEP_MS_PUT_BATCH = ConfigProperty
+  public static final ConfigProperty<String> SLEEP_MS_FOR_PUT_BATCH = ConfigProperty
       .key("hoodie.index.hbase.sleep.ms.for.put.batch")
       .noDefaultValue()
       .withDocumentation("");
-  @Deprecated
-  public static final String HBASE_SLEEP_MS_PUT_BATCH_PROP = HBASE_SLEEP_MS_PUT_BATCH.key();
 
-  public static final ConfigProperty<String> HBASE_SLEEP_MS_GET_BATCH = ConfigProperty
+  public static final ConfigProperty<String> SLEEP_MS_FOR_GET_BATCH = ConfigProperty
       .key("hoodie.index.hbase.sleep.ms.for.get.batch")
       .noDefaultValue()
       .withDocumentation("");
-  @Deprecated
-  public static final String HBASE_SLEEP_MS_GET_BATCH_PROP = HBASE_SLEEP_MS_GET_BATCH.key();;
 
-  public static final ConfigProperty<Integer> HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS = ConfigProperty
+  public static final ConfigProperty<Integer> ZK_SESSION_TIMEOUT_MS = ConfigProperty
       .key("hoodie.index.hbase.zk.session_timeout_ms")
       .defaultValue(60 * 1000)
       .withDocumentation("Session timeout value to use for Zookeeper failure detection, for the HBase client."
           + "Lower this value, if you want to fail faster.");
 
-  public static final ConfigProperty<Integer> HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS = ConfigProperty
+  public static final ConfigProperty<Integer> ZK_CONNECTION_TIMEOUT_MS = ConfigProperty
       .key("hoodie.index.hbase.zk.connection_timeout_ms")
       .defaultValue(15 * 1000)
       .withDocumentation("Timeout to use for establishing connection with zookeeper, from HBase client.");
 
-  public static final ConfigProperty<String> HBASE_ZK_PATH_QPS_ROOT = ConfigProperty
+  public static final ConfigProperty<String> ZKPATH_QPS_ROOT = ConfigProperty
       .key("hoodie.index.hbase.zkpath.qps_root")
       .defaultValue("/QPS_ROOT")
       .withDocumentation("chroot in zookeeper, to use for all qps allocation co-ordination.");
 
-  public static final ConfigProperty<Boolean> HBASE_INDEX_UPDATE_PARTITION_PATH = ConfigProperty
+  public static final ConfigProperty<Boolean> UPDATE_PARTITION_PATH_ENABLE = ConfigProperty
       .key("hoodie.hbase.index.update.partition.path")
       .defaultValue(false)
       .withDocumentation("Only applies if index type is HBASE. "
@@ -175,12 +151,175 @@ public class HoodieHBaseIndexConfig extends HoodieConfig {
           + "this config when set, will delete old record in old paritition "
           + "and will insert it as new record in new partition.");
 
-  public static final ConfigProperty<Boolean> HBASE_INDEX_ROLLBACK_SYNC = ConfigProperty
+  public static final ConfigProperty<Boolean> ROLLBACK_SYNC_ENABLE = ConfigProperty
       .key("hoodie.index.hbase.rollback.sync")
       .defaultValue(false)
       .withDocumentation("When set to true, the rollback method will delete the last failed task index. "
           + "The default value is false. Because deleting the index will add extra load on the Hbase cluster for each rollback");
 
+  /**
+   * @deprecated Use {@link #ZKQUORUM} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_ZKQUORUM_PROP = ZKQUORUM.key();
+  /**
+   * @deprecated Use {@link #ZKPORT} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_ZKPORT_PROP = ZKPORT.key();
+  /**
+   * @deprecated Use {@link #TABLENAME} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_TABLENAME_PROP = TABLENAME.key();
+  /**
+   * @deprecated Use {@link #GET_BATCH_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_GET_BATCH_SIZE_PROP = GET_BATCH_SIZE.key();
+  /**
+   * @deprecated Use {@link #ZK_NODE_PATH} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_ZK_ZNODEPARENT = ZK_NODE_PATH.key();
+  /**
+   * @deprecated Use {@link #PUT_BATCH_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_PUT_BATCH_SIZE_PROP = PUT_BATCH_SIZE.key();
+  /**
+   * @deprecated Use {@link #QPS_ALLOCATOR_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_INDEX_QPS_ALLOCATOR_CLASS = QPS_ALLOCATOR_CLASS_NAME.key();
+  /**
+   * @deprecated Use {@link #QPS_ALLOCATOR_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS = QPS_ALLOCATOR_CLASS_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #PUT_BATCH_SIZE_AUTO_COMPUTE} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP = PUT_BATCH_SIZE_AUTO_COMPUTE.key();
+  /**
+   * @deprecated Use {@link #PUT_BATCH_SIZE_AUTO_COMPUTE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE = PUT_BATCH_SIZE_AUTO_COMPUTE.defaultValue();
+  /**
+   * @deprecated Use {@link #MAX_QPS_FRACTION} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_QPS_FRACTION_PROP = QPS_FRACTION.key();
+  /**
+   * @deprecated Use {@link #MAX_QPS_PER_REGION_SERVER} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_MAX_QPS_PER_REGION_SERVER_PROP = MAX_QPS_PER_REGION_SERVER.key();
+  @Deprecated
+  public static final int DEFAULT_HBASE_BATCH_SIZE = 100;
+  /**
+   * @deprecated Use {@link #MAX_QPS_PER_REGION_SERVER} and its methods instead
+   */
+  @Deprecated
+  public static final int DEFAULT_HBASE_MAX_QPS_PER_REGION_SERVER = MAX_QPS_PER_REGION_SERVER.defaultValue();
+  /**
+   * @deprecated Use {@link #QPS_FRACTION} and its methods instead
+   */
+  @Deprecated
+  public static final float DEFAULT_HBASE_QPS_FRACTION = QPS_FRACTION.defaultValue();
+  /**
+   * @deprecated Use {@link #COMPUTE_QPS_DYNAMICALLY} and its methods instead
+   */
+  @Deprecated
+  public static final String HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY = COMPUTE_QPS_DYNAMICALLY.key();
+  /**
+   * @deprecated Use {@link #COMPUTE_QPS_DYNAMICALLY} and its methods instead
+   */
+  @Deprecated
+  public static final boolean DEFAULT_HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY = COMPUTE_QPS_DYNAMICALLY.defaultValue();
+  /**
+   * @deprecated Use {@link #MIN_QPS_FRACTION} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_MIN_QPS_FRACTION_PROP = MIN_QPS_FRACTION.key();
+  /**
+   * @deprecated Use {@link #MAX_QPS_FRACTION} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_MAX_QPS_FRACTION_PROP = MAX_QPS_FRACTION.key();
+  /**
+   * @deprecated Use {@link #DESIRED_PUTS_TIME_IN_SECONDS} and its methods instead
+   */
+  @Deprecated
+  public static final String HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS = DESIRED_PUTS_TIME_IN_SECONDS.key();
+  /**
+   * @deprecated Use {@link #DESIRED_PUTS_TIME_IN_SECONDS} and its methods instead
+   */
+  @Deprecated
+  public static final int DEFAULT_HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS = DESIRED_PUTS_TIME_IN_SECONDS.defaultValue();
+  /**
+   * @deprecated Use {@link #SLEEP_MS_FOR_PUT_BATCH} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_SLEEP_MS_PUT_BATCH_PROP = SLEEP_MS_FOR_PUT_BATCH.key();
+  /**
+   * @deprecated Use {@link #SLEEP_MS_FOR_GET_BATCH} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_SLEEP_MS_GET_BATCH_PROP = SLEEP_MS_FOR_GET_BATCH.key();
+  /**
+   * @deprecated Use {@link #ZK_SESSION_TIMEOUT_MS} and its methods instead
+   */
+  @Deprecated
+  public static final String HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS = ZK_SESSION_TIMEOUT_MS.key();
+  /**
+   * @deprecated Use {@link #ZK_SESSION_TIMEOUT_MS} and its methods instead
+   */
+  @Deprecated
+  public static final int DEFAULT_ZK_SESSION_TIMEOUT_MS = ZK_SESSION_TIMEOUT_MS.defaultValue();
+  /**
+   * @deprecated Use {@link #ZK_CONNECTION_TIMEOUT_MS} and its methods instead
+   */
+  @Deprecated
+  public static final String HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS = ZK_CONNECTION_TIMEOUT_MS.key();
+  /**
+   * @deprecated Use {@link #ZK_CONNECTION_TIMEOUT_MS} and its methods instead
+   */
+  @Deprecated
+  public static final int DEFAULT_ZK_CONNECTION_TIMEOUT_MS = ZK_CONNECTION_TIMEOUT_MS.defaultValue();
+  /**
+   * @deprecated Use {@link #ZKPATH_QPS_ROOT} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_ZK_PATH_QPS_ROOT = ZKPATH_QPS_ROOT.key();
+  /**
+   * @deprecated Use {@link #ZKPATH_QPS_ROOT} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_HBASE_ZK_PATH_QPS_ROOT = ZKPATH_QPS_ROOT.defaultValue();
+  /**
+   * @deprecated Use {@link #UPDATE_PARTITION_PATH_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_INDEX_UPDATE_PARTITION_PATH = UPDATE_PARTITION_PATH_ENABLE.key();
+  /**
+   * @deprecated Use {@link #UPDATE_PARTITION_PATH_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final Boolean DEFAULT_HBASE_INDEX_UPDATE_PARTITION_PATH = UPDATE_PARTITION_PATH_ENABLE.defaultValue();
+  /**
+   * @deprecated Use {@link #ROLLBACK_SYNC_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String HBASE_INDEX_ROLLBACK_SYNC = ROLLBACK_SYNC_ENABLE.key();
+  /**
+   * @deprecated Use {@link #ROLLBACK_SYNC_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final Boolean DEFAULT_HBASE_INDEX_ROLLBACK_SYNC = ROLLBACK_SYNC_ENABLE.defaultValue();
+
   private HoodieHBaseIndexConfig() {
     super();
   }
@@ -206,102 +345,102 @@ public class HoodieHBaseIndexConfig extends HoodieConfig {
     }
 
     public HoodieHBaseIndexConfig.Builder hbaseZkQuorum(String zkString) {
-      hBaseIndexConfig.setValue(HBASE_ZKQUORUM, zkString);
+      hBaseIndexConfig.setValue(ZKQUORUM, zkString);
       return this;
     }
 
     public HoodieHBaseIndexConfig.Builder hbaseZkPort(int port) {
-      hBaseIndexConfig.setValue(HBASE_ZKPORT, String.valueOf(port));
+      hBaseIndexConfig.setValue(ZKPORT, String.valueOf(port));
       return this;
     }
 
     public HoodieHBaseIndexConfig.Builder hbaseTableName(String tableName) {
-      hBaseIndexConfig.setValue(HBASE_TABLENAME, tableName);
+      hBaseIndexConfig.setValue(TABLENAME, tableName);
       return this;
     }
 
     public Builder hbaseZkZnodeQPSPath(String zkZnodeQPSPath) {
-      hBaseIndexConfig.setValue(HBASE_ZK_PATH_QPS_ROOT, zkZnodeQPSPath);
+      hBaseIndexConfig.setValue(ZKPATH_QPS_ROOT, zkZnodeQPSPath);
       return this;
     }
 
     public Builder hbaseIndexGetBatchSize(int getBatchSize) {
-      hBaseIndexConfig.setValue(HBASE_GET_BATCH_SIZE, String.valueOf(getBatchSize));
+      hBaseIndexConfig.setValue(GET_BATCH_SIZE, String.valueOf(getBatchSize));
       return this;
     }
 
     public Builder hbaseIndexPutBatchSize(int putBatchSize) {
-      hBaseIndexConfig.setValue(HBASE_PUT_BATCH_SIZE, String.valueOf(putBatchSize));
+      hBaseIndexConfig.setValue(PUT_BATCH_SIZE, String.valueOf(putBatchSize));
       return this;
     }
 
     public Builder hbaseIndexPutBatchSizeAutoCompute(boolean putBatchSizeAutoCompute) {
-      hBaseIndexConfig.setValue(HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE, String.valueOf(putBatchSizeAutoCompute));
+      hBaseIndexConfig.setValue(PUT_BATCH_SIZE_AUTO_COMPUTE, String.valueOf(putBatchSizeAutoCompute));
       return this;
     }
 
     public Builder hbaseIndexDesiredPutsTime(int desiredPutsTime) {
-      hBaseIndexConfig.setValue(HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS, String.valueOf(desiredPutsTime));
+      hBaseIndexConfig.setValue(DESIRED_PUTS_TIME_IN_SECONDS, String.valueOf(desiredPutsTime));
       return this;
     }
 
     public Builder hbaseIndexShouldComputeQPSDynamically(boolean shouldComputeQPsDynamically) {
-      hBaseIndexConfig.setValue(HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY, String.valueOf(shouldComputeQPsDynamically));
+      hBaseIndexConfig.setValue(COMPUTE_QPS_DYNAMICALLY, String.valueOf(shouldComputeQPsDynamically));
       return this;
     }
 
     public Builder hbaseIndexQPSFraction(float qpsFraction) {
-      hBaseIndexConfig.setValue(HBASE_QPS_FRACTION, String.valueOf(qpsFraction));
+      hBaseIndexConfig.setValue(QPS_FRACTION, String.valueOf(qpsFraction));
       return this;
     }
 
     public Builder hbaseIndexMinQPSFraction(float minQPSFraction) {
-      hBaseIndexConfig.setValue(HBASE_MIN_QPS_FRACTION, String.valueOf(minQPSFraction));
+      hBaseIndexConfig.setValue(MIN_QPS_FRACTION, String.valueOf(minQPSFraction));
       return this;
     }
 
     public Builder hbaseIndexMaxQPSFraction(float maxQPSFraction) {
-      hBaseIndexConfig.setValue(HBASE_MAX_QPS_FRACTION, String.valueOf(maxQPSFraction));
+      hBaseIndexConfig.setValue(MAX_QPS_FRACTION, String.valueOf(maxQPSFraction));
       return this;
     }
 
     public Builder hbaseIndexSleepMsBetweenPutBatch(int sleepMsBetweenPutBatch) {
-      hBaseIndexConfig.setValue(HBASE_SLEEP_MS_PUT_BATCH, String.valueOf(sleepMsBetweenPutBatch));
+      hBaseIndexConfig.setValue(SLEEP_MS_FOR_PUT_BATCH, String.valueOf(sleepMsBetweenPutBatch));
       return this;
     }
 
     public Builder hbaseIndexSleepMsBetweenGetBatch(int sleepMsBetweenGetBatch) {
-      hBaseIndexConfig.setValue(HBASE_SLEEP_MS_GET_BATCH, String.valueOf(sleepMsBetweenGetBatch));
+      hBaseIndexConfig.setValue(SLEEP_MS_FOR_GET_BATCH, String.valueOf(sleepMsBetweenGetBatch));
       return this;
     }
 
     public Builder hbaseIndexUpdatePartitionPath(boolean updatePartitionPath) {
-      hBaseIndexConfig.setValue(HBASE_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
+      hBaseIndexConfig.setValue(UPDATE_PARTITION_PATH_ENABLE, String.valueOf(updatePartitionPath));
       return this;
     }
 
     public Builder hbaseIndexRollbackSync(boolean rollbackSync) {
-      hBaseIndexConfig.setValue(HBASE_INDEX_ROLLBACK_SYNC, String.valueOf(rollbackSync));
+      hBaseIndexConfig.setValue(ROLLBACK_SYNC_ENABLE, String.valueOf(rollbackSync));
       return this;
     }
 
     public Builder withQPSResourceAllocatorType(String qpsResourceAllocatorClass) {
-      hBaseIndexConfig.setValue(HBASE_INDEX_QPS_ALLOCATOR_CLASS, qpsResourceAllocatorClass);
+      hBaseIndexConfig.setValue(QPS_ALLOCATOR_CLASS_NAME, qpsResourceAllocatorClass);
       return this;
     }
 
     public Builder hbaseIndexZkSessionTimeout(int zkSessionTimeout) {
-      hBaseIndexConfig.setValue(HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS, String.valueOf(zkSessionTimeout));
+      hBaseIndexConfig.setValue(ZK_SESSION_TIMEOUT_MS, String.valueOf(zkSessionTimeout));
       return this;
     }
 
     public Builder hbaseIndexZkConnectionTimeout(int zkConnectionTimeout) {
-      hBaseIndexConfig.setValue(HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS, String.valueOf(zkConnectionTimeout));
+      hBaseIndexConfig.setValue(ZK_CONNECTION_TIMEOUT_MS, String.valueOf(zkConnectionTimeout));
       return this;
     }
 
     public Builder hbaseZkZnodeParent(String zkZnodeParent) {
-      hBaseIndexConfig.setValue(HBASE_ZK_ZNODEPARENT, zkZnodeParent);
+      hBaseIndexConfig.setValue(ZK_NODE_PATH, zkZnodeParent);
       return this;
     }
 
@@ -317,7 +456,7 @@ public class HoodieHBaseIndexConfig extends HoodieConfig {
      */
     public HoodieHBaseIndexConfig.Builder hbaseIndexMaxQPSPerRegionServer(int maxQPSPerRegionServer) {
       // This should be same across various jobs
-      hBaseIndexConfig.setValue(HoodieHBaseIndexConfig.HBASE_MAX_QPS_PER_REGION_SERVER,
+      hBaseIndexConfig.setValue(HoodieHBaseIndexConfig.MAX_QPS_PER_REGION_SERVER,
           String.valueOf(maxQPSPerRegionServer));
       return this;
     }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java
index ca3e58e..4f9b9fc 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieIndexConfig.java
@@ -34,11 +34,11 @@ import java.io.FileReader;
 import java.io.IOException;
 import java.util.Properties;
 
-import static org.apache.hudi.config.HoodieHBaseIndexConfig.HBASE_ZKQUORUM;
-import static org.apache.hudi.config.HoodieHBaseIndexConfig.HBASE_ZKPORT;
-import static org.apache.hudi.config.HoodieHBaseIndexConfig.HBASE_TABLENAME;
-import static org.apache.hudi.config.HoodieHBaseIndexConfig.HBASE_GET_BATCH_SIZE;
-import static org.apache.hudi.config.HoodieHBaseIndexConfig.HBASE_PUT_BATCH_SIZE;
+import static org.apache.hudi.config.HoodieHBaseIndexConfig.GET_BATCH_SIZE;
+import static org.apache.hudi.config.HoodieHBaseIndexConfig.PUT_BATCH_SIZE;
+import static org.apache.hudi.config.HoodieHBaseIndexConfig.TABLENAME;
+import static org.apache.hudi.config.HoodieHBaseIndexConfig.ZKPORT;
+import static org.apache.hudi.config.HoodieHBaseIndexConfig.ZKQUORUM;
 
 /**
  * Indexing related config.
@@ -57,19 +57,15 @@ public class HoodieIndexConfig extends HoodieConfig {
           + "Possible options are [BLOOM | GLOBAL_BLOOM |SIMPLE | GLOBAL_SIMPLE | INMEMORY | HBASE]. "
           + "Bloom filters removes the dependency on a external system "
           + "and is stored in the footer of the Parquet Data Files");
-  @Deprecated
-  public static final String INDEX_TYPE_PROP = INDEX_TYPE.key();
 
-  public static final ConfigProperty<String> INDEX_CLASS = ConfigProperty
+  public static final ConfigProperty<String> INDEX_CLASS_NAME = ConfigProperty
       .key("hoodie.index.class")
       .defaultValue("")
       .withDocumentation("Full path of user-defined index class and must be a subclass of HoodieIndex class. "
           + "It will take precedence over the hoodie.index.type configuration if specified");
-  @Deprecated
-  public static final String INDEX_CLASS_PROP = INDEX_CLASS.key();
 
   // ***** Bloom Index configs *****
-  public static final ConfigProperty<String> BLOOM_FILTER_NUM_ENTRIES = ConfigProperty
+  public static final ConfigProperty<String> BLOOM_FILTER_NUM_ENTRIES_VALUE = ConfigProperty
       .key("hoodie.index.bloom.num_entries")
       .defaultValue("60000")
       .withDocumentation("Only applies if index type is BLOOM. "
@@ -81,7 +77,7 @@ public class HoodieIndexConfig extends HoodieConfig {
           + "increase the size every base file linearly (roughly 4KB for every 50000 entries). "
           + "This config is also used with DYNAMIC bloom filter which determines the initial size for the bloom.");
 
-  public static final ConfigProperty<String> BLOOM_FILTER_FPP = ConfigProperty
+  public static final ConfigProperty<String> BLOOM_FILTER_FPP_VALUE = ConfigProperty
       .key("hoodie.index.bloom.fpp")
       .defaultValue("0.000000001")
       .withDocumentation("Only applies if index type is BLOOM. "
@@ -97,8 +93,6 @@ public class HoodieIndexConfig extends HoodieConfig {
       .withDocumentation("Only applies if index type is BLOOM. "
           + "This is the amount of parallelism for index lookup, which involves a shuffle. "
           + "By default, this is auto computed based on input workload characteristics.");
-  @Deprecated
-  public static final String BLOOM_INDEX_PARALLELISM_PROP = BLOOM_INDEX_PARALLELISM.key();
 
   public static final ConfigProperty<String> BLOOM_INDEX_PRUNE_BY_RANGES = ConfigProperty
       .key("hoodie.bloom.index.prune.by.ranges")
@@ -108,8 +102,6 @@ public class HoodieIndexConfig extends HoodieConfig {
           + "if the key has a monotonously increasing prefix, such as timestamp. "
           + "If the record key is completely random, it is better to turn this off, since range pruning will only "
           + " add extra overhead to the index lookup.");
-  @Deprecated
-  public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP = BLOOM_INDEX_PRUNE_BY_RANGES.key();
 
   public static final ConfigProperty<String> BLOOM_INDEX_USE_CACHING = ConfigProperty
       .key("hoodie.bloom.index.use.caching")
@@ -117,8 +109,6 @@ public class HoodieIndexConfig extends HoodieConfig {
       .withDocumentation("Only applies if index type is BLOOM."
           + "When true, the input RDD will cached to speed up index lookup by reducing IO "
           + "for computing parallelism or affected partitions");
-  @Deprecated
-  public static final String BLOOM_INDEX_USE_CACHING_PROP = BLOOM_INDEX_USE_CACHING.key();
 
   public static final ConfigProperty<String> BLOOM_INDEX_TREE_BASED_FILTER = ConfigProperty
       .key("hoodie.bloom.index.use.treebased.filter")
@@ -126,8 +116,6 @@ public class HoodieIndexConfig extends HoodieConfig {
       .withDocumentation("Only applies if index type is BLOOM. "
           + "When true, interval tree based file pruning optimization is enabled. "
           + "This mode speeds-up file-pruning based on key ranges when compared with the brute-force mode");
-  @Deprecated
-  public static final String BLOOM_INDEX_TREE_BASED_FILTER_PROP = BLOOM_INDEX_TREE_BASED_FILTER.key();
 
   // TODO: On by default. Once stable, we will remove the other mode.
   public static final ConfigProperty<String> BLOOM_INDEX_BUCKETIZED_CHECKING = ConfigProperty
@@ -136,17 +124,15 @@ public class HoodieIndexConfig extends HoodieConfig {
       .withDocumentation("Only applies if index type is BLOOM. "
           + "When true, bucketized bloom filtering is enabled. "
           + "This reduces skew seen in sort based bloom index lookup");
-  @Deprecated
-  public static final String BLOOM_INDEX_BUCKETIZED_CHECKING_PROP = BLOOM_INDEX_BUCKETIZED_CHECKING.key();
 
-  public static final ConfigProperty<String> BLOOM_INDEX_FILTER_TYPE = ConfigProperty
+  public static final ConfigProperty<String> BLOOM_FILTER_TYPE = ConfigProperty
       .key("hoodie.bloom.index.filter.type")
       .defaultValue(BloomFilterTypeCode.DYNAMIC_V0.name())
       .withDocumentation("Filter type used. Default is BloomFilterTypeCode.DYNAMIC_V0. "
           + "Available values are [BloomFilterTypeCode.SIMPLE , BloomFilterTypeCode.DYNAMIC_V0]. "
           + "Dynamic bloom filters auto size themselves based on number of keys.");
 
-  public static final ConfigProperty<String> HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES = ConfigProperty
+  public static final ConfigProperty<String> BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES = ConfigProperty
       .key("hoodie.bloom.index.filter.dynamic.max.entries")
       .defaultValue("100000")
       .withDocumentation("The threshold for the maximum number of keys to record in a dynamic Bloom filter row. "
@@ -158,24 +144,18 @@ public class HoodieIndexConfig extends HoodieConfig {
       .withDocumentation("Only applies if index type is SIMPLE. "
           + "When true, the incoming writes will cached to speed up index lookup by reducing IO "
           + "for computing parallelism or affected partitions");
-  @Deprecated
-  public static final String SIMPLE_INDEX_USE_CACHING_PROP = SIMPLE_INDEX_USE_CACHING.key();
 
   public static final ConfigProperty<String> SIMPLE_INDEX_PARALLELISM = ConfigProperty
       .key("hoodie.simple.index.parallelism")
       .defaultValue("50")
       .withDocumentation("Only applies if index type is SIMPLE. "
           + "This is the amount of parallelism for index lookup, which involves a Spark Shuffle");
-  @Deprecated
-  public static final String SIMPLE_INDEX_PARALLELISM_PROP = SIMPLE_INDEX_PARALLELISM.key();
 
   public static final ConfigProperty<String> GLOBAL_SIMPLE_INDEX_PARALLELISM = ConfigProperty
       .key("hoodie.global.simple.index.parallelism")
       .defaultValue("100")
       .withDocumentation("Only applies if index type is GLOBAL_SIMPLE. "
           + "This is the amount of parallelism for index lookup, which involves a Spark Shuffle");
-  @Deprecated
-  public static final String GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP = GLOBAL_SIMPLE_INDEX_PARALLELISM.key();
 
   // 1B bloom filter checks happen in 250 seconds. 500ms to read a bloom filter.
   // 10M checks in 2500ms, thus amortizing the cost of reading bloom filter across partitions.
@@ -186,16 +166,14 @@ public class HoodieIndexConfig extends HoodieConfig {
           + "This configuration controls the “bucket” size which tracks the number of record-key checks made against "
           + "a single file and is the unit of work allocated to each partition performing bloom filter lookup. "
           + "A higher value would amortize the fixed cost of reading a bloom filter to memory.");
-  @Deprecated
-  public static final String BLOOM_INDEX_KEYS_PER_BUCKET_PROP = BLOOM_INDEX_KEYS_PER_BUCKET.key();
 
-  public static final ConfigProperty<String> BLOOM_INDEX_INPUT_STORAGE_LEVEL = ConfigProperty
+  public static final ConfigProperty<String> BLOOM_INDEX_INPUT_STORAGE_LEVEL_VALUE = ConfigProperty
       .key("hoodie.bloom.index.input.storage.level")
       .defaultValue("MEMORY_AND_DISK_SER")
       .withDocumentation("Only applies when #bloomIndexUseCaching is set. Determine what level of persistence is used to cache input RDDs. "
           + "Refer to org.apache.spark.storage.StorageLevel for different values");
 
-  public static final ConfigProperty<String> SIMPLE_INDEX_INPUT_STORAGE_LEVEL = ConfigProperty
+  public static final ConfigProperty<String> SIMPLE_INDEX_INPUT_STORAGE_LEVEL_VALUE = ConfigProperty
       .key("hoodie.simple.index.input.storage.level")
       .defaultValue("MEMORY_AND_DISK_SER")
       .withDocumentation("Only applies when #simpleIndexUseCaching is set. Determine what level of persistence is used to cache input RDDs. "
@@ -209,7 +187,7 @@ public class HoodieIndexConfig extends HoodieConfig {
    * <p>
    * When set to false, a record will be updated to the old partition.
    */
-  public static final ConfigProperty<String> BLOOM_INDEX_UPDATE_PARTITION_PATH = ConfigProperty
+  public static final ConfigProperty<String> BLOOM_INDEX_UPDATE_PARTITION_PATH_ENABLE = ConfigProperty
       .key("hoodie.bloom.index.update.partition.path")
       .defaultValue("true")
       .withDocumentation("Only applies if index type is GLOBAL_BLOOM. "
@@ -217,26 +195,195 @@ public class HoodieIndexConfig extends HoodieConfig {
           + "inserting the incoming record into the new partition and deleting the original record in the old partition. "
           + "When set to false, the original record will only be updated in the old partition");
 
-  public static final ConfigProperty<String> SIMPLE_INDEX_UPDATE_PARTITION_PATH = ConfigProperty
+  public static final ConfigProperty<String> SIMPLE_INDEX_UPDATE_PARTITION_PATH_ENABLE = ConfigProperty
       .key("hoodie.simple.index.update.partition.path")
       .defaultValue("true")
-      .withDocumentation("Similar to " + BLOOM_INDEX_UPDATE_PARTITION_PATH + ", but for simple index.");
+      .withDocumentation("Similar to " + BLOOM_INDEX_UPDATE_PARTITION_PATH_ENABLE + ", but for simple index.");
 
   /**
    * Deprecated configs. These are now part of {@link HoodieHBaseIndexConfig}.
    */
   @Deprecated
-  public static final String HBASE_ZKQUORUM_PROP = HBASE_ZKQUORUM.key();
+  public static final String HBASE_ZKQUORUM_PROP = ZKQUORUM.key();
+  @Deprecated
+  public static final String HBASE_ZKPORT_PROP = ZKPORT.key();
+  @Deprecated
+  public static final String HBASE_ZK_ZNODEPARENT = HoodieHBaseIndexConfig.ZK_NODE_PATH.key();
+  @Deprecated
+  public static final String HBASE_TABLENAME_PROP = TABLENAME.key();
   @Deprecated
-  public static final String HBASE_ZKPORT_PROP = HBASE_ZKPORT.key();
+  public static final String HBASE_GET_BATCH_SIZE_PROP = GET_BATCH_SIZE.key();
   @Deprecated
-  public static final String HBASE_ZK_ZNODEPARENT = HoodieHBaseIndexConfig.HBASE_ZK_ZNODEPARENT.key();
+  public static final String HBASE_PUT_BATCH_SIZE_PROP = PUT_BATCH_SIZE.key();
   @Deprecated
-  public static final String HBASE_TABLENAME_PROP = HBASE_TABLENAME.key();
+  public static final String DEFAULT_HBASE_BATCH_SIZE = "100";
+  /** @deprecated Use {@link #INDEX_TYPE} and its methods instead */
   @Deprecated
-  public static final String HBASE_GET_BATCH_SIZE_PROP = HBASE_GET_BATCH_SIZE.key();
+  public static final String INDEX_TYPE_PROP = INDEX_TYPE.key();
+  /**
+   * @deprecated Use {@link #INDEX_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String INDEX_CLASS_PROP = INDEX_CLASS_NAME.key();
+  /**
+   * @deprecated Use {@link #INDEX_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_INDEX_CLASS = INDEX_CLASS_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #BLOOM_FILTER_NUM_ENTRIES_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String BLOOM_FILTER_NUM_ENTRIES = BLOOM_FILTER_NUM_ENTRIES_VALUE.key();
+  /**
+   * @deprecated Use {@link #BLOOM_FILTER_NUM_ENTRIES_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_BLOOM_FILTER_NUM_ENTRIES = BLOOM_FILTER_NUM_ENTRIES_VALUE.defaultValue();
+  /**
+   * @deprecated Use {@link #BLOOM_FILTER_FPP_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String BLOOM_FILTER_FPP = BLOOM_FILTER_FPP_VALUE.key();
+  /**
+   * @deprecated Use {@link #BLOOM_FILTER_FPP_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_BLOOM_FILTER_FPP = BLOOM_FILTER_FPP_VALUE.defaultValue();
+  /**
+   * @deprecated Use {@link #BLOOM_INDEX_PARALLELISM} and its methods instead
+   */
+  @Deprecated
+  public static final String BLOOM_INDEX_PARALLELISM_PROP = BLOOM_INDEX_PARALLELISM.key();
+  /**
+   * @deprecated Use {@link #BLOOM_INDEX_PARALLELISM} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_BLOOM_INDEX_PARALLELISM = BLOOM_INDEX_PARALLELISM.defaultValue();
+  /**
+   * @deprecated Use {@link #BLOOM_INDEX_PRUNE_BY_RANGES} and its methods instead
+   */
+  @Deprecated
+  public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP = BLOOM_INDEX_PRUNE_BY_RANGES.key();
+  /** @deprecated Use {@link #BLOOM_INDEX_PRUNE_BY_RANGES} and its methods instead */
+  @Deprecated
+  public static final String DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES = BLOOM_INDEX_PRUNE_BY_RANGES.defaultValue();
+  /** @deprecated Use {@link #BLOOM_INDEX_USE_CACHING} and its methods instead */
+  @Deprecated
+  public static final String BLOOM_INDEX_USE_CACHING_PROP = BLOOM_INDEX_USE_CACHING.key();
+  /** @deprecated Use {@link #BLOOM_INDEX_USE_CACHING} and its methods instead */
+  @Deprecated
+  public static final String DEFAULT_BLOOM_INDEX_USE_CACHING = BLOOM_INDEX_USE_CACHING.defaultValue();
+  /** @deprecated Use {@link #BLOOM_INDEX_TREE_BASED_FILTER} and its methods instead */
+  @Deprecated
+  public static final String BLOOM_INDEX_TREE_BASED_FILTER_PROP = BLOOM_INDEX_TREE_BASED_FILTER.key();
+  /** @deprecated Use {@link #BLOOM_INDEX_TREE_BASED_FILTER} and its methods instead */
+  @Deprecated
+  public static final String DEFAULT_BLOOM_INDEX_TREE_BASED_FILTER = BLOOM_INDEX_TREE_BASED_FILTER.defaultValue();
+  /**
+   * @deprecated Use {@link #BLOOM_INDEX_BUCKETIZED_CHECKING} and its methods instead
+   */
+  @Deprecated
+  public static final String BLOOM_INDEX_BUCKETIZED_CHECKING_PROP = BLOOM_INDEX_BUCKETIZED_CHECKING.key();
+  /**
+   * @deprecated Use {@link #BLOOM_INDEX_BUCKETIZED_CHECKING} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_BLOOM_INDEX_BUCKETIZED_CHECKING = BLOOM_INDEX_BUCKETIZED_CHECKING.defaultValue();
+  /**
+   * @deprecated Use {@link #BLOOM_FILTER_TYPE} and its methods instead
+   */
+  @Deprecated
+  public static final String BLOOM_INDEX_FILTER_TYPE = BLOOM_FILTER_TYPE.key();
+  /**
+   * @deprecated Use {@link #BLOOM_FILTER_TYPE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_BLOOM_INDEX_FILTER_TYPE = BLOOM_FILTER_TYPE.defaultValue();
+  /**
+   * @deprecated Use {@link #BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES} and its methods instead
+   */
+  @Deprecated
+  public static final String HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES = BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.key();
+  /**
+   * @deprecated Use {@link #BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES = BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.defaultValue();
+  /**
+   * @deprecated Use {@link #SIMPLE_INDEX_USE_CACHING} and its methods instead
+   */
+  @Deprecated
+  public static final String SIMPLE_INDEX_USE_CACHING_PROP = SIMPLE_INDEX_USE_CACHING.key();
+  /**
+   * @deprecated Use {@link #SIMPLE_INDEX_USE_CACHING} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_SIMPLE_INDEX_USE_CACHING = SIMPLE_INDEX_USE_CACHING.defaultValue();
+  /** @deprecated Use {@link #SIMPLE_INDEX_PARALLELISM} and its methods instead */
+  @Deprecated
+  public static final String SIMPLE_INDEX_PARALLELISM_PROP = SIMPLE_INDEX_PARALLELISM.key();
+  /** @deprecated Use {@link #SIMPLE_INDEX_PARALLELISM} and its methods instead */
+  @Deprecated
+  public static final String DEFAULT_SIMPLE_INDEX_PARALLELISM = SIMPLE_INDEX_PARALLELISM.defaultValue();
+  /** @deprecated Use {@link #GLOBAL_SIMPLE_INDEX_PARALLELISM} and its methods instead */
+  @Deprecated
+  public static final String GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP = GLOBAL_SIMPLE_INDEX_PARALLELISM.key();
+  /**
+   * @deprecated Use {@link #GLOBAL_SIMPLE_INDEX_PARALLELISM} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_GLOBAL_SIMPLE_INDEX_PARALLELISM = GLOBAL_SIMPLE_INDEX_PARALLELISM.defaultValue();
+  /**
+   * @deprecated Use {@link #BLOOM_INDEX_KEYS_PER_BUCKET} and its methods instead
+   */
+  @Deprecated
+  public static final String BLOOM_INDEX_KEYS_PER_BUCKET_PROP = BLOOM_INDEX_KEYS_PER_BUCKET.key();
+  /**
+   * @deprecated Use {@link #BLOOM_INDEX_KEYS_PER_BUCKET} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_BLOOM_INDEX_KEYS_PER_BUCKET = BLOOM_INDEX_KEYS_PER_BUCKET.defaultValue();
+  /**
+   * @deprecated Use {@link #BLOOM_INDEX_INPUT_STORAGE_LEVEL_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String BLOOM_INDEX_INPUT_STORAGE_LEVEL = BLOOM_INDEX_INPUT_STORAGE_LEVEL_VALUE.key();
+  /**
+   * @deprecated Use {@link #BLOOM_INDEX_INPUT_STORAGE_LEVEL_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL = BLOOM_INDEX_INPUT_STORAGE_LEVEL_VALUE.defaultValue();
+  /**
+   * @deprecated Use {@link #SIMPLE_INDEX_INPUT_STORAGE_LEVEL_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String SIMPLE_INDEX_INPUT_STORAGE_LEVEL = SIMPLE_INDEX_INPUT_STORAGE_LEVEL_VALUE.key();
+  /**
+   * @deprecated Use {@link #SIMPLE_INDEX_INPUT_STORAGE_LEVEL_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_SIMPLE_INDEX_INPUT_STORAGE_LEVEL = SIMPLE_INDEX_INPUT_STORAGE_LEVEL_VALUE.defaultValue();
+  /**
+   * @deprecated Use {@link #BLOOM_INDEX_UPDATE_PARTITION_PATH_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String BLOOM_INDEX_UPDATE_PARTITION_PATH = BLOOM_INDEX_UPDATE_PARTITION_PATH_ENABLE.key();
+  /**
+   * @deprecated Use {@link #BLOOM_INDEX_UPDATE_PARTITION_PATH_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_BLOOM_INDEX_UPDATE_PARTITION_PATH = BLOOM_INDEX_UPDATE_PARTITION_PATH_ENABLE.defaultValue();
+  /**
+   * @deprecated Use {@link #SIMPLE_INDEX_UPDATE_PARTITION_PATH_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String SIMPLE_INDEX_UPDATE_PARTITION_PATH = SIMPLE_INDEX_UPDATE_PARTITION_PATH_ENABLE.key();
+  /**
+   * @deprecated Use {@link #SIMPLE_INDEX_UPDATE_PARTITION_PATH_ENABLE} and its methods instead
+   */
   @Deprecated
-  public static final String HBASE_PUT_BATCH_SIZE_PROP = HBASE_PUT_BATCH_SIZE.key();
+  public static final String DEFAULT_SIMPLE_INDEX_UPDATE_PARTITION_PATH = SIMPLE_INDEX_UPDATE_PARTITION_PATH_ENABLE.defaultValue();
 
   private EngineType engineType;
 
@@ -280,7 +427,7 @@ public class HoodieIndexConfig extends HoodieConfig {
     }
 
     public Builder withIndexClass(String indexClass) {
-      hoodieIndexConfig.setValue(INDEX_CLASS, indexClass);
+      hoodieIndexConfig.setValue(INDEX_CLASS_NAME, indexClass);
       return this;
     }
 
@@ -290,12 +437,12 @@ public class HoodieIndexConfig extends HoodieConfig {
     }
 
     public Builder bloomFilterNumEntries(int numEntries) {
-      hoodieIndexConfig.setValue(BLOOM_FILTER_NUM_ENTRIES, String.valueOf(numEntries));
+      hoodieIndexConfig.setValue(BLOOM_FILTER_NUM_ENTRIES_VALUE, String.valueOf(numEntries));
       return this;
     }
 
     public Builder bloomFilterFPP(double fpp) {
-      hoodieIndexConfig.setValue(BLOOM_FILTER_FPP, String.valueOf(fpp));
+      hoodieIndexConfig.setValue(BLOOM_FILTER_FPP_VALUE, String.valueOf(fpp));
       return this;
     }
 
@@ -330,12 +477,12 @@ public class HoodieIndexConfig extends HoodieConfig {
     }
 
     public Builder withBloomIndexInputStorageLevel(String level) {
-      hoodieIndexConfig.setValue(BLOOM_INDEX_INPUT_STORAGE_LEVEL, level);
+      hoodieIndexConfig.setValue(BLOOM_INDEX_INPUT_STORAGE_LEVEL_VALUE, level);
       return this;
     }
 
     public Builder withBloomIndexUpdatePartitionPath(boolean updatePartitionPath) {
-      hoodieIndexConfig.setValue(BLOOM_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
+      hoodieIndexConfig.setValue(BLOOM_INDEX_UPDATE_PARTITION_PATH_ENABLE, String.valueOf(updatePartitionPath));
       return this;
     }
 
@@ -350,7 +497,7 @@ public class HoodieIndexConfig extends HoodieConfig {
     }
 
     public Builder withSimpleIndexInputStorageLevel(String level) {
-      hoodieIndexConfig.setValue(SIMPLE_INDEX_INPUT_STORAGE_LEVEL, level);
+      hoodieIndexConfig.setValue(SIMPLE_INDEX_INPUT_STORAGE_LEVEL_VALUE, level);
       return this;
     }
 
@@ -360,7 +507,7 @@ public class HoodieIndexConfig extends HoodieConfig {
     }
 
     public Builder withGlobalSimpleIndexUpdatePartitionPath(boolean updatePartitionPath) {
-      hoodieIndexConfig.setValue(SIMPLE_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
+      hoodieIndexConfig.setValue(SIMPLE_INDEX_UPDATE_PARTITION_PATH_ENABLE, String.valueOf(updatePartitionPath));
       return this;
     }
 
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java
index 41ad4e5..fd7b310 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java
@@ -70,8 +70,6 @@ public class HoodieLockConfig extends HoodieConfig {
       .sinceVersion("0.8.0")
       .withDocumentation("Initial amount of time to wait between retries to acquire locks, "
           + " subsequent retries will exponentially backoff.");
-  @Deprecated
-  public static final String LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP = LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS.key();
 
   public static final ConfigProperty<String> LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS = ConfigProperty
       .key(LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP_KEY)
@@ -79,72 +77,54 @@ public class HoodieLockConfig extends HoodieConfig {
       .sinceVersion("0.8.0")
       .withDocumentation("Maximum amount of time to wait between retries by lock provider client. This bounds"
           + " the maximum delay from the exponential backoff. Currently used by ZK based lock provider only.");
-  @Deprecated
-  public static final String LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP = LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS.key();
 
   public static final ConfigProperty<String> LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS = ConfigProperty
       .key(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY)
       .defaultValue(String.valueOf(10000L))
       .sinceVersion("0.8.0")
       .withDocumentation("Amount of time to wait between retries on the lock provider by the lock manager");
-  @Deprecated
-  public static final String LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP = LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS.key();
 
   public static final ConfigProperty<String> LOCK_ACQUIRE_NUM_RETRIES = ConfigProperty
       .key(LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY)
       .defaultValue(DEFAULT_LOCK_ACQUIRE_NUM_RETRIES)
       .sinceVersion("0.8.0")
       .withDocumentation("Maximum number of times to retry lock acquire, at each lock provider");
-  @Deprecated
-  public static final String LOCK_ACQUIRE_NUM_RETRIES_PROP = LOCK_ACQUIRE_NUM_RETRIES.key();
 
   public static final ConfigProperty<String> LOCK_ACQUIRE_CLIENT_NUM_RETRIES = ConfigProperty
       .key(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY)
       .defaultValue(String.valueOf(0))
       .sinceVersion("0.8.0")
       .withDocumentation("Maximum number of times to retry to acquire lock additionally from the lock manager.");
-  @Deprecated
-  public static final String LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP = LOCK_ACQUIRE_CLIENT_NUM_RETRIES.key();
 
   public static final ConfigProperty<Integer> LOCK_ACQUIRE_WAIT_TIMEOUT_MS = ConfigProperty
       .key(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY)
       .defaultValue(60 * 1000)
       .sinceVersion("0.8.0")
       .withDocumentation("Timeout in ms, to wait on an individual lock acquire() call, at the lock provider.");
-  @Deprecated
-  public static final String LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP = LOCK_ACQUIRE_WAIT_TIMEOUT_MS.key();
 
   public static final ConfigProperty<String> FILESYSTEM_LOCK_PATH = ConfigProperty
       .key(FILESYSTEM_LOCK_PATH_PROP_KEY)
       .noDefaultValue()
       .sinceVersion("0.8.0")
       .withDocumentation("For DFS based lock providers, path to store the locks under.");
-  @Deprecated
-  public static final String FILESYSTEM_LOCK_PATH_PROP = FILESYSTEM_LOCK_PATH.key();
 
   public static final ConfigProperty<String> HIVE_DATABASE_NAME = ConfigProperty
       .key(HIVE_DATABASE_NAME_PROP_KEY)
       .noDefaultValue()
       .sinceVersion("0.8.0")
       .withDocumentation("For Hive based lock provider, the Hive database to acquire lock against");
-  @Deprecated
-  public static final String HIVE_DATABASE_NAME_PROP = HIVE_DATABASE_NAME.key();
 
   public static final ConfigProperty<String> HIVE_TABLE_NAME = ConfigProperty
       .key(HIVE_TABLE_NAME_PROP_KEY)
       .noDefaultValue()
       .sinceVersion("0.8.0")
       .withDocumentation("For Hive based lock provider, the Hive table to acquire lock against");
-  @Deprecated
-  public static final String HIVE_TABLE_NAME_PROP = HIVE_TABLE_NAME.key();
 
   public static final ConfigProperty<String> HIVE_METASTORE_URI = ConfigProperty
       .key(HIVE_METASTORE_URI_PROP_KEY)
       .noDefaultValue()
       .sinceVersion("0.8.0")
       .withDocumentation("For Hive based lock provider, the Hive metastore URI to acquire locks against.");
-  @Deprecated
-  public static final String HIVE_METASTORE_URI_PROP = HIVE_METASTORE_URI.key();
 
   public static final ConfigProperty<String> ZK_BASE_PATH = ConfigProperty
       .key(ZK_BASE_PATH_PROP_KEY)
@@ -152,40 +132,30 @@ public class HoodieLockConfig extends HoodieConfig {
       .sinceVersion("0.8.0")
       .withDocumentation("The base path on Zookeeper under which to create lock related ZNodes. "
           + "This should be same for all concurrent writers to the same table");
-  @Deprecated
-  public static final String ZK_BASE_PATH_PROP = ZK_BASE_PATH.key();
 
   public static final ConfigProperty<Integer> ZK_SESSION_TIMEOUT_MS = ConfigProperty
       .key(ZK_SESSION_TIMEOUT_MS_PROP_KEY)
       .defaultValue(DEFAULT_ZK_SESSION_TIMEOUT_MS)
       .sinceVersion("0.8.0")
       .withDocumentation("Timeout in ms, to wait after losing connection to ZooKeeper, before the session is expired");
-  @Deprecated
-  public static final String ZK_SESSION_TIMEOUT_MS_PROP = ZK_SESSION_TIMEOUT_MS.key();
 
   public static final ConfigProperty<Integer> ZK_CONNECTION_TIMEOUT_MS = ConfigProperty
       .key(ZK_CONNECTION_TIMEOUT_MS_PROP_KEY)
       .defaultValue(DEFAULT_ZK_CONNECTION_TIMEOUT_MS)
       .sinceVersion("0.8.0")
       .withDocumentation("Timeout in ms, to wait for establishing connection with Zookeeper.");
-  @Deprecated
-  public static final String ZK_CONNECTION_TIMEOUT_MS_PROP = ZK_CONNECTION_TIMEOUT_MS.key();
 
   public static final ConfigProperty<String> ZK_CONNECT_URL = ConfigProperty
       .key(ZK_CONNECT_URL_PROP_KEY)
       .noDefaultValue()
       .sinceVersion("0.8.0")
       .withDocumentation("Zookeeper URL to connect to.");
-  @Deprecated
-  public static final String ZK_CONNECT_URL_PROP = ZK_CONNECT_URL.key();
 
   public static final ConfigProperty<String> ZK_PORT = ConfigProperty
       .key(ZK_PORT_PROP_KEY)
       .noDefaultValue()
       .sinceVersion("0.8.0")
       .withDocumentation("Zookeeper port to connect to.");
-  @Deprecated
-  public static final String ZK_PORT_PROP = ZK_PORT.key();
 
   public static final ConfigProperty<String> ZK_LOCK_KEY = ConfigProperty
       .key(ZK_LOCK_KEY_PROP_KEY)
@@ -193,28 +163,35 @@ public class HoodieLockConfig extends HoodieConfig {
       .sinceVersion("0.8.0")
       .withDocumentation("Key name under base_path at which to create a ZNode and acquire lock. "
           + "Final path on zk will look like base_path/lock_key. We recommend setting this to the table name");
-  @Deprecated
-  public static final String ZK_LOCK_KEY_PROP = ZK_LOCK_KEY.key();
 
   // Pluggable type of lock provider
-  public static final ConfigProperty<String> LOCK_PROVIDER_CLASS = ConfigProperty
+  public static final ConfigProperty<String> LOCK_PROVIDER_CLASS_NAME = ConfigProperty
       .key(LOCK_PREFIX + "provider")
       .defaultValue(ZookeeperBasedLockProvider.class.getName())
       .sinceVersion("0.8.0")
       .withDocumentation("Lock provider class name, user can provide their own implementation of LockProvider "
           + "which should be subclass of org.apache.hudi.common.lock.LockProvider");
-  @Deprecated
-  public static final String LOCK_PROVIDER_CLASS_PROP = LOCK_PROVIDER_CLASS.key();
 
   // Pluggable strategies to use when resolving conflicts
-  public static final ConfigProperty<String> WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS = ConfigProperty
+  public static final ConfigProperty<String> WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME = ConfigProperty
       .key(LOCK_PREFIX + "conflict.resolution.strategy")
       .defaultValue(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName())
       .sinceVersion("0.8.0")
       .withDocumentation("Lock provider class name, this should be subclass of "
           + "org.apache.hudi.client.transaction.ConflictResolutionStrategy");
+
+  /** @deprecated Use {@link #WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME} and its methods instead */
+  @Deprecated
+  public static final String WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP = WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME.key();
+  /** @deprecated Use {@link #WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME} and its methods instead */
+  @Deprecated
+  public static final String DEFAULT_WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS = WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME.defaultValue();
+  /** @deprecated Use {@link #LOCK_PROVIDER_CLASS_NAME} and its methods instead */
+  @Deprecated
+  public static final String LOCK_PROVIDER_CLASS_PROP = LOCK_PROVIDER_CLASS_NAME.key();
+  /** @deprecated Use {@link #LOCK_PROVIDER_CLASS_NAME} and its methods instead */
   @Deprecated
-  public static final String WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP = WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS.key();
+  public static final String DEFAULT_LOCK_PROVIDER_CLASS = LOCK_PROVIDER_CLASS_NAME.defaultValue();
 
   private HoodieLockConfig() {
     super();
@@ -241,7 +218,7 @@ public class HoodieLockConfig extends HoodieConfig {
     }
 
     public HoodieLockConfig.Builder withLockProvider(Class<? extends LockProvider> lockProvider) {
-      lockConfig.setValue(LOCK_PROVIDER_CLASS, lockProvider.getName());
+      lockConfig.setValue(LOCK_PROVIDER_CLASS_NAME, lockProvider.getName());
       return this;
     }
 
@@ -321,7 +298,7 @@ public class HoodieLockConfig extends HoodieConfig {
     }
 
     public HoodieLockConfig.Builder withConflictResolutionStrategy(ConflictResolutionStrategy conflictResolutionStrategy) {
-      lockConfig.setValue(WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS, conflictResolutionStrategy.getClass().getName());
+      lockConfig.setValue(WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME, conflictResolutionStrategy.getClass().getName());
       return this;
     }
 
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java
index b8f3410..8845ccb 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMemoryConfig.java
@@ -46,8 +46,6 @@ public class HoodieMemoryConfig extends HoodieConfig {
       .defaultValue(String.valueOf(0.6))
       .withDocumentation("This fraction is multiplied with the user memory fraction (1 - spark.memory.fraction) "
           + "to get a final fraction of heap space to use during merge");
-  @Deprecated
-  public static final String MAX_MEMORY_FRACTION_FOR_MERGE_PROP = MAX_MEMORY_FRACTION_FOR_MERGE.key();
 
   // Default max memory fraction during compaction, excess spills to disk
   public static final ConfigProperty<String> MAX_MEMORY_FRACTION_FOR_COMPACTION = ConfigProperty
@@ -58,8 +56,6 @@ public class HoodieMemoryConfig extends HoodieConfig {
           + "less than or equal to the number of entries in the corresponding parquet file. This can lead to "
           + "OOM in the Scanner. Hence, a spillable map helps alleviate the memory pressure. Use this config to "
           + "set the max allowable inMemory footprint of the spillable map");
-  @Deprecated
-  public static final String MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP = MAX_MEMORY_FRACTION_FOR_COMPACTION.key();
 
   // Default memory size (1GB) per compaction (used if SparkEnv is absent), excess spills to disk
   public static final long DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES = 1024 * 1024 * 1024L;
@@ -70,29 +66,21 @@ public class HoodieMemoryConfig extends HoodieConfig {
       .key("hoodie.memory.merge.max.size")
       .defaultValue(DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES)
       .withDocumentation("Maximum amount of memory used for merge operations, before spilling to local storage.");
-  @Deprecated
-  public static final String MAX_MEMORY_FOR_MERGE_PROP = MAX_MEMORY_FOR_MERGE.key();
 
   public static final ConfigProperty<String> MAX_MEMORY_FOR_COMPACTION = ConfigProperty
       .key("hoodie.memory.compaction.max.size")
       .noDefaultValue()
       .withDocumentation("Maximum amount of memory used for compaction operations, before spilling to local storage.");
-  @Deprecated
-  public static final String MAX_MEMORY_FOR_COMPACTION_PROP = MAX_MEMORY_FOR_COMPACTION.key();
 
   public static final ConfigProperty<Integer> MAX_DFS_STREAM_BUFFER_SIZE = ConfigProperty
       .key("hoodie.memory.dfs.buffer.max.size")
       .defaultValue(16 * 1024 * 1024)
       .withDocumentation("Property to control the max memory for dfs input stream buffer size");
-  @Deprecated
-  public static final String MAX_DFS_STREAM_BUFFER_SIZE_PROP = MAX_DFS_STREAM_BUFFER_SIZE.key();
 
   public static final ConfigProperty<String> SPILLABLE_MAP_BASE_PATH = ConfigProperty
       .key("hoodie.memory.spillable.map.path")
       .defaultValue("/tmp/")
       .withDocumentation("Default file path prefix for spillable map");
-  @Deprecated
-  public static final String SPILLABLE_MAP_BASE_PATH_PROP = SPILLABLE_MAP_BASE_PATH.key();
 
   public static final ConfigProperty<Double> WRITESTATUS_FAILURE_FRACTION = ConfigProperty
       .key("hoodie.memory.writestatus.failure.fraction")
@@ -100,8 +88,43 @@ public class HoodieMemoryConfig extends HoodieConfig {
       .withDocumentation("Property to control how what fraction of the failed record, exceptions we report back to driver. "
           + "Default is 10%. If set to 100%, with lot of failures, this can cause memory pressure, cause OOMs and "
           + "mask actual data errors.");
+
+  /** @deprecated Use {@link #MAX_MEMORY_FRACTION_FOR_MERGE} and its methods instead */
+  @Deprecated
+  public static final String MAX_MEMORY_FRACTION_FOR_MERGE_PROP = MAX_MEMORY_FRACTION_FOR_MERGE.key();
+  /** @deprecated Use {@link #MAX_MEMORY_FRACTION_FOR_MERGE} and its methods instead */
+  @Deprecated
+  public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE = MAX_MEMORY_FRACTION_FOR_MERGE.defaultValue();
+  /** @deprecated Use {@link #MAX_MEMORY_FRACTION_FOR_COMPACTION} and its methods instead */
+  @Deprecated
+  public static final String MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP = MAX_MEMORY_FRACTION_FOR_COMPACTION.key();
+  /** @deprecated Use {@link #MAX_MEMORY_FRACTION_FOR_COMPACTION} and its methods instead */
+  @Deprecated
+  public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION = MAX_MEMORY_FRACTION_FOR_COMPACTION.defaultValue();
+  /** @deprecated Use {@link #MAX_MEMORY_FOR_MERGE} and its methods instead */
+  @Deprecated
+  public static final String MAX_MEMORY_FOR_MERGE_PROP = MAX_MEMORY_FOR_MERGE.key();
+  /** @deprecated Use {@link #MAX_MEMORY_FOR_COMPACTION} and its methods instead */
+  @Deprecated
+  public static final String MAX_MEMORY_FOR_COMPACTION_PROP = MAX_MEMORY_FOR_COMPACTION.key();
+  /** @deprecated Use {@link #MAX_DFS_STREAM_BUFFER_SIZE} and its methods instead */
+  @Deprecated
+  public static final String MAX_DFS_STREAM_BUFFER_SIZE_PROP = MAX_DFS_STREAM_BUFFER_SIZE.key();
+  /** @deprecated Use {@link #MAX_DFS_STREAM_BUFFER_SIZE} and its methods instead */
+  @Deprecated
+  public static final int DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE = MAX_DFS_STREAM_BUFFER_SIZE.defaultValue();
+  /** @deprecated Use {@link #SPILLABLE_MAP_BASE_PATH} and its methods instead */
+  @Deprecated
+  public static final String SPILLABLE_MAP_BASE_PATH_PROP = SPILLABLE_MAP_BASE_PATH.key();
+  /** @deprecated Use {@link #SPILLABLE_MAP_BASE_PATH} and its methods instead */
+  @Deprecated
+  public static final String DEFAULT_SPILLABLE_MAP_BASE_PATH = SPILLABLE_MAP_BASE_PATH.defaultValue();
+  /** @deprecated Use {@link #WRITESTATUS_FAILURE_FRACTION} and its methods instead */
   @Deprecated
   public static final String WRITESTATUS_FAILURE_FRACTION_PROP = WRITESTATUS_FAILURE_FRACTION.key();
+  /** @deprecated Use {@link #WRITESTATUS_FAILURE_FRACTION} and its methods instead */
+  @Deprecated
+  public static final double DEFAULT_WRITESTATUS_FAILURE_FRACTION = WRITESTATUS_FAILURE_FRACTION.defaultValue();
 
   private HoodieMemoryConfig() {
     super();
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java
index b5434b0..e1d0b63 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsConfig.java
@@ -43,13 +43,13 @@ public class HoodieMetricsConfig extends HoodieConfig {
 
   public static final String METRIC_PREFIX = "hoodie.metrics";
 
-  public static final ConfigProperty<Boolean> METRICS_ON = ConfigProperty
+  public static final ConfigProperty<Boolean> TURN_METRICS_ON = ConfigProperty
       .key(METRIC_PREFIX + ".on")
       .defaultValue(false)
       .sinceVersion("0.5.0")
       .withDocumentation("Turn on/off metrics reporting. off by default.");
 
-  public static final ConfigProperty<MetricsReporterType> METRICS_REPORTER_TYPE = ConfigProperty
+  public static final ConfigProperty<MetricsReporterType> METRICS_REPORTER_TYPE_VALUE = ConfigProperty
       .key(METRIC_PREFIX + ".reporter.type")
       .defaultValue(MetricsReporterType.GRAPHITE)
       .sinceVersion("0.5.0")
@@ -58,13 +58,13 @@ public class HoodieMetricsConfig extends HoodieConfig {
   // Graphite
   public static final String GRAPHITE_PREFIX = METRIC_PREFIX + ".graphite";
 
-  public static final ConfigProperty<String> GRAPHITE_SERVER_HOST = ConfigProperty
+  public static final ConfigProperty<String> GRAPHITE_SERVER_HOST_NAME = ConfigProperty
       .key(GRAPHITE_PREFIX + ".host")
       .defaultValue("localhost")
       .sinceVersion("0.5.0")
       .withDocumentation("Graphite host to connect to");
 
-  public static final ConfigProperty<Integer> GRAPHITE_SERVER_PORT = ConfigProperty
+  public static final ConfigProperty<Integer> GRAPHITE_SERVER_PORT_NUM = ConfigProperty
       .key(GRAPHITE_PREFIX + ".port")
       .defaultValue(4756)
       .sinceVersion("0.5.0")
@@ -73,38 +73,119 @@ public class HoodieMetricsConfig extends HoodieConfig {
   // Jmx
   public static final String JMX_PREFIX = METRIC_PREFIX + ".jmx";
 
-  public static final ConfigProperty<String> JMX_HOST = ConfigProperty
+  public static final ConfigProperty<String> JMX_HOST_NAME = ConfigProperty
       .key(JMX_PREFIX + ".host")
       .defaultValue("localhost")
       .sinceVersion("0.5.1")
       .withDocumentation("Jmx host to connect to");
 
-  public static final ConfigProperty<Integer> JMX_PORT = ConfigProperty
+  public static final ConfigProperty<Integer> JMX_PORT_NUM = ConfigProperty
       .key(JMX_PREFIX + ".port")
       .defaultValue(9889)
       .sinceVersion("0.5.1")
       .withDocumentation("Jmx port to connect to");
 
-  public static final ConfigProperty<String> GRAPHITE_METRIC_PREFIX = ConfigProperty
+  public static final ConfigProperty<String> GRAPHITE_METRIC_PREFIX_VALUE = ConfigProperty
       .key(GRAPHITE_PREFIX + ".metric.prefix")
       .noDefaultValue()
       .sinceVersion("0.5.1")
       .withDocumentation("Standard prefix applied to all metrics. This helps to add datacenter, environment information for e.g");
 
   // User defined
-  public static final ConfigProperty<String> METRICS_REPORTER_CLASS = ConfigProperty
+  public static final ConfigProperty<String> METRICS_REPORTER_CLASS_NAME = ConfigProperty
       .key(METRIC_PREFIX + ".reporter.class")
       .defaultValue("")
       .sinceVersion("0.6.0")
       .withDocumentation("");
 
   // Enable metrics collection from executors
-  public static final ConfigProperty<String> ENABLE_EXECUTOR_METRICS = ConfigProperty
+  public static final ConfigProperty<String> EXECUTOR_METRICS_ENABLE = ConfigProperty
       .key(METRIC_PREFIX + ".executor.enable")
       .noDefaultValue()
       .sinceVersion("0.7.0")
       .withDocumentation("");
 
+  /**
+   * @deprecated Use {@link #TURN_METRICS_ON} and its methods instead
+   */
+  @Deprecated
+  public static final String METRICS_ON = TURN_METRICS_ON.key();
+  /**
+   * @deprecated Use {@link #TURN_METRICS_ON} and its methods instead
+   */
+  @Deprecated
+  public static final boolean DEFAULT_METRICS_ON = TURN_METRICS_ON.defaultValue();
+  /**
+   * @deprecated Use {@link #METRICS_REPORTER_TYPE_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String METRICS_REPORTER_TYPE = METRICS_REPORTER_TYPE_VALUE.key();
+  /**
+   * @deprecated Use {@link #METRICS_REPORTER_TYPE_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE = METRICS_REPORTER_TYPE_VALUE.defaultValue();
+  /**
+   * @deprecated Use {@link #GRAPHITE_SERVER_HOST_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String GRAPHITE_SERVER_HOST = GRAPHITE_SERVER_HOST_NAME.key();
+  /**
+   * @deprecated Use {@link #GRAPHITE_SERVER_HOST_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_GRAPHITE_SERVER_HOST = GRAPHITE_SERVER_HOST_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #GRAPHITE_SERVER_PORT_NUM} and its methods instead
+   */
+  @Deprecated
+  public static final String GRAPHITE_SERVER_PORT = GRAPHITE_SERVER_PORT_NUM.key();
+  /**
+   * @deprecated Use {@link #GRAPHITE_SERVER_PORT_NUM} and its methods instead
+   */
+  @Deprecated
+  public static final int DEFAULT_GRAPHITE_SERVER_PORT = GRAPHITE_SERVER_PORT_NUM.defaultValue();
+  /**
+   * @deprecated Use {@link #JMX_HOST_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String JMX_HOST = JMX_HOST_NAME.key();
+  /**
+   * @deprecated Use {@link #JMX_HOST_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_JMX_HOST = JMX_HOST_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #JMX_PORT_NUM} and its methods instead
+   */
+  @Deprecated
+  public static final String JMX_PORT = JMX_PORT_NUM.key();
+  /**
+   * @deprecated Use {@link #JMX_PORT_NUM} and its methods instead
+   */
+  @Deprecated
+  public static final int DEFAULT_JMX_PORT = JMX_PORT_NUM.defaultValue();
+  /**
+   * @deprecated Use {@link #GRAPHITE_METRIC_PREFIX_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String GRAPHITE_METRIC_PREFIX = GRAPHITE_METRIC_PREFIX_VALUE.key();
+  /**
+   * @deprecated Use {@link #METRICS_REPORTER_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String METRICS_REPORTER_CLASS = METRICS_REPORTER_CLASS_NAME.key();
+  /**
+   * @deprecated Use {@link #METRICS_REPORTER_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_METRICS_REPORTER_CLASS = METRICS_REPORTER_CLASS_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #EXECUTOR_METRICS_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String ENABLE_EXECUTOR_METRICS = EXECUTOR_METRICS_ENABLE.key();
+
   private HoodieMetricsConfig() {
     super();
   }
@@ -130,47 +211,47 @@ public class HoodieMetricsConfig extends HoodieConfig {
     }
 
     public Builder on(boolean metricsOn) {
-      hoodieMetricsConfig.setValue(METRICS_ON, String.valueOf(metricsOn));
+      hoodieMetricsConfig.setValue(TURN_METRICS_ON, String.valueOf(metricsOn));
       return this;
     }
 
     public Builder withReporterType(String reporterType) {
-      hoodieMetricsConfig.setValue(METRICS_REPORTER_TYPE, reporterType);
+      hoodieMetricsConfig.setValue(METRICS_REPORTER_TYPE_VALUE, reporterType);
       return this;
     }
 
     public Builder toGraphiteHost(String host) {
-      hoodieMetricsConfig.setValue(GRAPHITE_SERVER_HOST, host);
+      hoodieMetricsConfig.setValue(GRAPHITE_SERVER_HOST_NAME, host);
       return this;
     }
 
     public Builder onGraphitePort(int port) {
-      hoodieMetricsConfig.setValue(GRAPHITE_SERVER_PORT, String.valueOf(port));
+      hoodieMetricsConfig.setValue(GRAPHITE_SERVER_PORT_NUM, String.valueOf(port));
       return this;
     }
 
     public Builder toJmxHost(String host) {
-      hoodieMetricsConfig.setValue(JMX_HOST, host);
+      hoodieMetricsConfig.setValue(JMX_HOST_NAME, host);
       return this;
     }
 
     public Builder onJmxPort(String port) {
-      hoodieMetricsConfig.setValue(JMX_PORT, port);
+      hoodieMetricsConfig.setValue(JMX_PORT_NUM, port);
       return this;
     }
 
     public Builder usePrefix(String prefix) {
-      hoodieMetricsConfig.setValue(GRAPHITE_METRIC_PREFIX, prefix);
+      hoodieMetricsConfig.setValue(GRAPHITE_METRIC_PREFIX_VALUE, prefix);
       return this;
     }
 
     public Builder withReporterClass(String className) {
-      hoodieMetricsConfig.setValue(METRICS_REPORTER_CLASS, className);
+      hoodieMetricsConfig.setValue(METRICS_REPORTER_CLASS_NAME, className);
       return this;
     }
 
     public Builder withExecutorMetrics(boolean enable) {
-      hoodieMetricsConfig.setValue(ENABLE_EXECUTOR_METRICS, String.valueOf(enable));
+      hoodieMetricsConfig.setValue(EXECUTOR_METRICS_ENABLE, String.valueOf(enable));
       return this;
     }
 
@@ -178,7 +259,7 @@ public class HoodieMetricsConfig extends HoodieConfig {
 
       hoodieMetricsConfig.setDefaults(HoodieMetricsConfig.class.getName());
 
-      MetricsReporterType reporterType = MetricsReporterType.valueOf(hoodieMetricsConfig.getString(METRICS_REPORTER_TYPE));
+      MetricsReporterType reporterType = MetricsReporterType.valueOf(hoodieMetricsConfig.getString(METRICS_REPORTER_TYPE_VALUE));
 
       hoodieMetricsConfig.setDefaultOnCondition(reporterType == MetricsReporterType.DATADOG,
           HoodieMetricsDatadogConfig.newBuilder().fromProperties(hoodieMetricsConfig.getProps()).build());
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsDatadogConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsDatadogConfig.java
index df70ce3..d63cb0f 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsDatadogConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsDatadogConfig.java
@@ -43,63 +43,124 @@ public class HoodieMetricsDatadogConfig extends HoodieConfig {
 
   public static final String DATADOG_PREFIX = METRIC_PREFIX + ".datadog";
 
-  public static final ConfigProperty<Integer> DATADOG_REPORT_PERIOD_SECONDS = ConfigProperty
+  public static final ConfigProperty<Integer> REPORT_PERIOD_IN_SECONDS = ConfigProperty
       .key(DATADOG_PREFIX + ".report.period.seconds")
       .defaultValue(30)
       .sinceVersion("0.6.0")
       .withDocumentation("Datadog reporting period in seconds. Default to 30.");
 
-  public static final ConfigProperty<String> DATADOG_API_SITE = ConfigProperty
+  public static final ConfigProperty<String> API_SITE_VALUE = ConfigProperty
       .key(DATADOG_PREFIX + ".api.site")
       .noDefaultValue()
       .sinceVersion("0.6.0")
       .withDocumentation("Datadog API site: EU or US");
 
-  public static final ConfigProperty<String> DATADOG_API_KEY = ConfigProperty
+  public static final ConfigProperty<String> API_KEY = ConfigProperty
       .key(DATADOG_PREFIX + ".api.key")
       .noDefaultValue()
       .sinceVersion("0.6.0")
       .withDocumentation("Datadog API key");
 
-  public static final ConfigProperty<Boolean> DATADOG_API_KEY_SKIP_VALIDATION = ConfigProperty
+  public static final ConfigProperty<Boolean> API_KEY_SKIP_VALIDATION = ConfigProperty
       .key(DATADOG_PREFIX + ".api.key.skip.validation")
       .defaultValue(false)
       .sinceVersion("0.6.0")
       .withDocumentation("Before sending metrics via Datadog API, whether to skip validating Datadog API key or not. "
           + "Default to false.");
 
-  public static final ConfigProperty<String> DATADOG_API_KEY_SUPPLIER = ConfigProperty
+  public static final ConfigProperty<String> API_KEY_SUPPLIER = ConfigProperty
       .key(DATADOG_PREFIX + ".api.key.supplier")
       .noDefaultValue()
       .sinceVersion("0.6.0")
       .withDocumentation("Datadog API key supplier to supply the API key at runtime. "
           + "This will take effect if hoodie.metrics.datadog.api.key is not set.");
 
-  public static final ConfigProperty<Integer> DATADOG_API_TIMEOUT_SECONDS = ConfigProperty
+  public static final ConfigProperty<Integer> API_TIMEOUT_IN_SECONDS = ConfigProperty
       .key(DATADOG_PREFIX + ".api.timeout.seconds")
       .defaultValue(3)
       .sinceVersion("0.6.0")
       .withDocumentation("Datadog API timeout in seconds. Default to 3.");
 
-  public static final ConfigProperty<String> DATADOG_METRIC_PREFIX = ConfigProperty
+  public static final ConfigProperty<String> METRIC_PREFIX_VALUE = ConfigProperty
       .key(DATADOG_PREFIX + ".metric.prefix")
       .noDefaultValue()
       .sinceVersion("0.6.0")
       .withDocumentation("Datadog metric prefix to be prepended to each metric name with a dot as delimiter. "
           + "For example, if it is set to foo, foo. will be prepended.");
 
-  public static final ConfigProperty<String> DATADOG_METRIC_HOST = ConfigProperty
+  public static final ConfigProperty<String> METRIC_HOST_NAME = ConfigProperty
       .key(DATADOG_PREFIX + ".metric.host")
       .noDefaultValue()
       .sinceVersion("0.6.0")
       .withDocumentation("Datadog metric host to be sent along with metrics data.");
 
-  public static final ConfigProperty<String> DATADOG_METRIC_TAGS = ConfigProperty
+  public static final ConfigProperty<String> METRIC_TAG_VALUES = ConfigProperty
       .key(DATADOG_PREFIX + ".metric.tags")
       .noDefaultValue()
       .sinceVersion("0.6.0")
       .withDocumentation("Datadog metric tags (comma-delimited) to be sent along with metrics data.");
 
+  /**
+   * @deprecated Use {@link #REPORT_PERIOD_IN_SECONDS} and its methods instead
+   */
+  @Deprecated
+  public static final String DATADOG_REPORT_PERIOD_SECONDS = REPORT_PERIOD_IN_SECONDS.key();
+  /**
+   * @deprecated Use {@link #REPORT_PERIOD_IN_SECONDS} and its methods instead
+   */
+  @Deprecated
+  public static final int DEFAULT_DATADOG_REPORT_PERIOD_SECONDS = REPORT_PERIOD_IN_SECONDS.defaultValue();
+  /**
+   * @deprecated Use {@link #API_SITE_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String DATADOG_API_SITE = API_SITE_VALUE.key();
+  /**
+   * @deprecated Use {@link #API_KEY} and its methods instead
+   */
+  @Deprecated
+  public static final String DATADOG_API_KEY = API_KEY.key();
+  /**
+   * @deprecated Use {@link #API_KEY_SKIP_VALIDATION} and its methods instead
+   */
+  @Deprecated
+  public static final String DATADOG_API_KEY_SKIP_VALIDATION = API_KEY_SKIP_VALIDATION.key();
+  /**
+   * @deprecated Use {@link #API_KEY_SKIP_VALIDATION} and its methods instead
+   */
+  @Deprecated
+  public static final boolean DEFAULT_DATADOG_API_KEY_SKIP_VALIDATION = API_KEY_SKIP_VALIDATION.defaultValue();
+  /**
+   * @deprecated Use {@link #API_KEY_SUPPLIER} and its methods instead
+   */
+  @Deprecated
+  public static final String DATADOG_API_KEY_SUPPLIER = API_KEY_SUPPLIER.key();
+  /**
+   * @deprecated Use {@link #API_TIMEOUT_IN_SECONDS} and its methods instead
+   */
+  @Deprecated
+  public static final String DATADOG_API_TIMEOUT_SECONDS = API_TIMEOUT_IN_SECONDS.key();
+  /**
+   * @deprecated Use {@link #API_TIMEOUT_IN_SECONDS} and its methods instead
+   */
+  @Deprecated
+  public static final int DEFAULT_DATADOG_API_TIMEOUT_SECONDS = API_TIMEOUT_IN_SECONDS.defaultValue();
+  /**
+   * @deprecated Use {@link #METRIC_PREFIX_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String DATADOG_METRIC_PREFIX = METRIC_PREFIX_VALUE.key();
+  /**
+   * @deprecated Use {@link #METRIC_HOST_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DATADOG_METRIC_HOST = METRIC_HOST_NAME.key();
+  /**
+   * @deprecated Use {@link #METRIC_TAG_VALUES} and its methods instead
+   */
+  @Deprecated
+  public static final String DATADOG_METRIC_TAGS = METRIC_TAG_VALUES.key();
+
   private HoodieMetricsDatadogConfig() {
     super();
   }
@@ -118,47 +179,47 @@ public class HoodieMetricsDatadogConfig extends HoodieConfig {
     }
 
     public Builder withDatadogReportPeriodSeconds(int period) {
-      metricsDatadogConfig.setValue(DATADOG_REPORT_PERIOD_SECONDS, String.valueOf(period));
+      metricsDatadogConfig.setValue(REPORT_PERIOD_IN_SECONDS, String.valueOf(period));
       return this;
     }
 
     public Builder withDatadogApiSite(String apiSite) {
-      metricsDatadogConfig.setValue(DATADOG_API_SITE, apiSite);
+      metricsDatadogConfig.setValue(API_SITE_VALUE, apiSite);
       return this;
     }
 
     public Builder withDatadogApiKey(String apiKey) {
-      metricsDatadogConfig.setValue(DATADOG_API_KEY, apiKey);
+      metricsDatadogConfig.setValue(API_KEY, apiKey);
       return this;
     }
 
     public Builder withDatadogApiKeySkipValidation(boolean skip) {
-      metricsDatadogConfig.setValue(DATADOG_API_KEY_SKIP_VALIDATION, String.valueOf(skip));
+      metricsDatadogConfig.setValue(API_KEY_SKIP_VALIDATION, String.valueOf(skip));
       return this;
     }
 
     public Builder withDatadogApiKeySupplier(String apiKeySupplier) {
-      metricsDatadogConfig.setValue(DATADOG_API_KEY_SUPPLIER, apiKeySupplier);
+      metricsDatadogConfig.setValue(API_KEY_SUPPLIER, apiKeySupplier);
       return this;
     }
 
     public Builder withDatadogApiTimeoutSeconds(int timeout) {
-      metricsDatadogConfig.setValue(DATADOG_API_TIMEOUT_SECONDS, String.valueOf(timeout));
+      metricsDatadogConfig.setValue(API_TIMEOUT_IN_SECONDS, String.valueOf(timeout));
       return this;
     }
 
     public Builder withDatadogPrefix(String prefix) {
-      metricsDatadogConfig.setValue(DATADOG_METRIC_PREFIX, prefix);
+      metricsDatadogConfig.setValue(METRIC_PREFIX_VALUE, prefix);
       return this;
     }
 
     public Builder withDatadogHost(String host) {
-      metricsDatadogConfig.setValue(DATADOG_METRIC_HOST, host);
+      metricsDatadogConfig.setValue(METRIC_HOST_NAME, host);
       return this;
     }
 
     public Builder withDatadogTags(String tags) {
-      metricsDatadogConfig.setValue(DATADOG_METRIC_TAGS, tags);
+      metricsDatadogConfig.setValue(METRIC_TAG_VALUES, tags);
       return this;
     }
 
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsPrometheusConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsPrometheusConfig.java
index a8eac62..615222f 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsPrometheusConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieMetricsPrometheusConfig.java
@@ -36,37 +36,37 @@ public class HoodieMetricsPrometheusConfig extends HoodieConfig {
   // Prometheus PushGateWay
   public static final String PUSHGATEWAY_PREFIX = METRIC_PREFIX + ".pushgateway";
 
-  public static final ConfigProperty<String> PUSHGATEWAY_HOST = ConfigProperty
+  public static final ConfigProperty<String> PUSHGATEWAY_HOST_NAME = ConfigProperty
       .key(PUSHGATEWAY_PREFIX + ".host")
       .defaultValue("localhost")
       .sinceVersion("0.6.0")
       .withDocumentation("Hostname of the prometheus push gateway");
 
-  public static final ConfigProperty<Integer> PUSHGATEWAY_PORT = ConfigProperty
+  public static final ConfigProperty<Integer> PUSHGATEWAY_PORT_NUM = ConfigProperty
       .key(PUSHGATEWAY_PREFIX + ".port")
       .defaultValue(9091)
       .sinceVersion("0.6.0")
       .withDocumentation("Port for the push gateway.");
 
-  public static final ConfigProperty<Integer> PUSHGATEWAY_REPORT_PERIOD_SECONDS = ConfigProperty
+  public static final ConfigProperty<Integer> PUSHGATEWAY_REPORT_PERIOD_IN_SECONDS = ConfigProperty
       .key(PUSHGATEWAY_PREFIX + ".report.period.seconds")
       .defaultValue(30)
       .sinceVersion("0.6.0")
       .withDocumentation("Reporting interval in seconds.");
 
-  public static final ConfigProperty<Boolean> PUSHGATEWAY_DELETE_ON_SHUTDOWN = ConfigProperty
+  public static final ConfigProperty<Boolean> PUSHGATEWAY_DELETE_ON_SHUTDOWN_ENABLE = ConfigProperty
       .key(PUSHGATEWAY_PREFIX + ".delete.on.shutdown")
       .defaultValue(true)
       .sinceVersion("0.6.0")
       .withDocumentation("");
 
-  public static final ConfigProperty<String> PUSHGATEWAY_JOB_NAME = ConfigProperty
+  public static final ConfigProperty<String> PUSHGATEWAY_JOBNAME = ConfigProperty
       .key(PUSHGATEWAY_PREFIX + ".job.name")
       .defaultValue("")
       .sinceVersion("0.6.0")
       .withDocumentation("Name of the push gateway job.");
 
-  public static final ConfigProperty<Boolean> PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX = ConfigProperty
+  public static final ConfigProperty<Boolean> PUSHGATEWAY_RANDOM_JOBNAME_SUFFIX = ConfigProperty
       .key(PUSHGATEWAY_PREFIX + ".random.job.name.suffix")
       .defaultValue(true)
       .sinceVersion("0.6.0")
@@ -75,12 +75,83 @@ public class HoodieMetricsPrometheusConfig extends HoodieConfig {
   // Prometheus HttpServer
   public static final String PROMETHEUS_PREFIX = METRIC_PREFIX + ".prometheus";
 
-  public static final ConfigProperty<Integer> PROMETHEUS_PORT = ConfigProperty
+  public static final ConfigProperty<Integer> PROMETHEUS_PORT_NUM = ConfigProperty
       .key(PROMETHEUS_PREFIX + ".port")
       .defaultValue(9090)
       .sinceVersion("0.6.0")
       .withDocumentation("Port for prometheus server.");
 
+  /**
+   * @deprecated Use {@link #PUSHGATEWAY_HOST_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String PUSHGATEWAY_HOST = PUSHGATEWAY_HOST_NAME.key();
+  /**
+   * @deprecated Use {@link #PUSHGATEWAY_HOST_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_PUSHGATEWAY_HOST = PUSHGATEWAY_HOST_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #PUSHGATEWAY_PORT_NUM} and its methods instead
+   */
+  @Deprecated
+  public static final String PUSHGATEWAY_PORT = PUSHGATEWAY_PORT_NUM.key();
+  /**
+   * @deprecated Use {@link #PUSHGATEWAY_PORT_NUM} and its methods instead
+   */
+  @Deprecated
+  public static final int DEFAULT_PUSHGATEWAY_PORT = PUSHGATEWAY_PORT_NUM.defaultValue();
+  /**
+   * @deprecated Use {@link #PUSHGATEWAY_REPORT_PERIOD_IN_SECONDS} and its methods instead
+   */
+  @Deprecated
+  public static final String PUSHGATEWAY_REPORT_PERIOD_SECONDS = PUSHGATEWAY_REPORT_PERIOD_IN_SECONDS.key();
+  /**
+   * @deprecated Use {@link #PUSHGATEWAY_REPORT_PERIOD_IN_SECONDS} and its methods instead
+   */
+  @Deprecated
+  public static final int DEFAULT_PUSHGATEWAY_REPORT_PERIOD_SECONDS = PUSHGATEWAY_REPORT_PERIOD_IN_SECONDS.defaultValue();
+  /**
+   * @deprecated Use {@link #PUSHGATEWAY_DELETE_ON_SHUTDOWN_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String PUSHGATEWAY_DELETE_ON_SHUTDOWN = PUSHGATEWAY_DELETE_ON_SHUTDOWN_ENABLE.key();
+  /**
+   * @deprecated Use {@link #PUSHGATEWAY_DELETE_ON_SHUTDOWN_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final boolean DEFAULT_PUSHGATEWAY_DELETE_ON_SHUTDOWN = PUSHGATEWAY_DELETE_ON_SHUTDOWN_ENABLE.defaultValue();
+  /**
+   * @deprecated Use {@link #PUSHGATEWAY_JOBNAME} and its methods instead
+   */
+  @Deprecated
+  public static final String PUSHGATEWAY_JOB_NAME = PUSHGATEWAY_JOBNAME.key();
+  /**
+   * @deprecated Use {@link #PUSHGATEWAY_JOBNAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_PUSHGATEWAY_JOB_NAME = PUSHGATEWAY_JOBNAME.defaultValue();
+  /**
+   * @deprecated Use {@link #PUSHGATEWAY_RANDOM_JOBNAME_SUFFIX} and its methods instead
+   */
+  @Deprecated
+  public static final String PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX = PUSHGATEWAY_RANDOM_JOBNAME_SUFFIX.key();
+  /**
+   * @deprecated Use {@link #PUSHGATEWAY_RANDOM_JOBNAME_SUFFIX} and its methods instead
+   */
+  @Deprecated
+  public static final boolean DEFAULT_PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX = PUSHGATEWAY_RANDOM_JOBNAME_SUFFIX.defaultValue();
+  /**
+   * @deprecated Use {@link #PROMETHEUS_PORT_NUM} and its methods instead
+   */
+  @Deprecated
+  public static final String PROMETHEUS_PORT = PROMETHEUS_PORT_NUM.key();
+  /**
+   * @deprecated Use {@link #PROMETHEUS_PORT_NUM} and its methods instead
+   */
+  @Deprecated
+  public static final int DEFAULT_PROMETHEUS_PORT = PROMETHEUS_PORT_NUM.defaultValue();
+
   private HoodieMetricsPrometheusConfig() {
     super();
   }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePayloadConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePayloadConfig.java
index 95d4fe3..2989d8c 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePayloadConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePayloadConfig.java
@@ -40,21 +40,17 @@ import static org.apache.hudi.common.model.HoodiePayloadProps.PAYLOAD_ORDERING_F
         + "control merges based on specific business fields in the data.")
 public class HoodiePayloadConfig extends HoodieConfig {
 
-  public static final ConfigProperty<String> PAYLOAD_ORDERING_FIELD = ConfigProperty
+  public static final ConfigProperty<String> ORDERING_FIELD = ConfigProperty
       .key(PAYLOAD_ORDERING_FIELD_PROP_KEY)
       .defaultValue("ts")
       .withDocumentation("Table column/field name to order records that have the same key, before "
           + "merging and writing to storage.");
-  @Deprecated
-  public static final String PAYLOAD_ORDERING_FIELD_PROP = PAYLOAD_ORDERING_FIELD.key();
 
-  public static final ConfigProperty<String> PAYLOAD_EVENT_TIME_FIELD = ConfigProperty
+  public static final ConfigProperty<String> EVENT_TIME_FIELD = ConfigProperty
       .key(PAYLOAD_EVENT_TIME_FIELD_PROP_KEY)
       .defaultValue("ts")
       .withDocumentation("Table column/field name to derive timestamp associated with the records. This can"
           + "be useful for e.g, determining the freshness of the table.");
-  @Deprecated
-  public static final String PAYLOAD_EVENT_TIME_FIELD_PROP = PAYLOAD_EVENT_TIME_FIELD.key();
 
   private HoodiePayloadConfig() {
     super();
@@ -81,12 +77,12 @@ public class HoodiePayloadConfig extends HoodieConfig {
     }
 
     public Builder withPayloadOrderingField(String payloadOrderingField) {
-      payloadConfig.setValue(PAYLOAD_ORDERING_FIELD, String.valueOf(payloadOrderingField));
+      payloadConfig.setValue(ORDERING_FIELD, String.valueOf(payloadOrderingField));
       return this;
     }
 
     public Builder withPayloadEventTimeField(String payloadEventTimeField) {
-      payloadConfig.setValue(PAYLOAD_EVENT_TIME_FIELD, String.valueOf(payloadEventTimeField));
+      payloadConfig.setValue(EVENT_TIME_FIELD, String.valueOf(payloadEventTimeField));
       return this;
     }
 
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePreCommitValidatorConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePreCommitValidatorConfig.java
index 1373c13..8ea0858 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePreCommitValidatorConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodiePreCommitValidatorConfig.java
@@ -22,6 +22,7 @@ import org.apache.hudi.common.config.ConfigProperty;
 import org.apache.hudi.common.config.HoodieConfig;
 
 import javax.annotation.concurrent.Immutable;
+
 import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
@@ -33,26 +34,26 @@ import java.util.Properties;
 @Immutable
 public class HoodiePreCommitValidatorConfig extends HoodieConfig {
 
-  public static final ConfigProperty<String> PRE_COMMIT_VALIDATORS = ConfigProperty
+  public static final ConfigProperty<String> VALIDATOR_CLASS_NAMES = ConfigProperty
       .key("hoodie.precommit.validators")
       .defaultValue("")
       .withDocumentation("Comma separated list of class names that can be invoked to validate commit");
   public static final String VALIDATOR_TABLE_VARIABLE = "<TABLE_NAME>";
 
-  public static final ConfigProperty<String> PRE_COMMIT_VALIDATORS_EQUALITY_SQL_QUERIES = ConfigProperty
+  public static final ConfigProperty<String> EQUALITY_SQL_QUERIES = ConfigProperty
       .key("hoodie.precommit.validators.equality.sql.queries")
       .defaultValue("")
       .withDocumentation("Spark SQL queries to run on table before committing new data to validate state before and after commit."
-          + " Multiple queries separated by ';' delimiter are supported." 
-          + " Example: \"select count(*) from \\<TABLE_NAME\\>" 
+          + " Multiple queries separated by ';' delimiter are supported."
+          + " Example: \"select count(*) from \\<TABLE_NAME\\>"
           + " Note \\<TABLE_NAME\\> is replaced by table state before and after commit.");
-  
-  public static final ConfigProperty<String> PRE_COMMIT_VALIDATORS_SINGLE_VALUE_SQL_QUERIES = ConfigProperty
+
+  public static final ConfigProperty<String> SINGLE_VALUE_SQL_QUERIES = ConfigProperty
       .key("hoodie.precommit.validators.single.value.sql.queries")
       .defaultValue("")
-      .withDocumentation("Spark SQL queries to run on table before committing new data to validate state after commit." 
-          + "Multiple queries separated by ';' delimiter are supported." 
-          + "Expected result is included as part of query separated by '#'. Example query: 'query1#result1:query2#result2'" 
+      .withDocumentation("Spark SQL queries to run on table before committing new data to validate state after commit."
+          + "Multiple queries separated by ';' delimiter are supported."
+          + "Expected result is included as part of query separated by '#'. Example query: 'query1#result1:query2#result2'"
           + "Note \\<TABLE_NAME\\> variable is expected to be present in query.");
 
   /**
@@ -61,11 +62,11 @@ public class HoodiePreCommitValidatorConfig extends HoodieConfig {
    * Example query: 'select count(*) from \<TABLE_NAME\> where col=null'
    * Note \<TABLE_NAME\> variable is expected to be present in query.
    */
-  public static final ConfigProperty<String> PRE_COMMIT_VALIDATORS_INEQUALITY_SQL_QUERIES = ConfigProperty
+  public static final ConfigProperty<String> INEQUALITY_SQL_QUERIES = ConfigProperty
       .key("hoodie.precommit.validators.inequality.sql.queries")
       .defaultValue("")
-      .withDocumentation("Spark SQL queries to run on table before committing new data to validate state before and after commit." 
-          + "Multiple queries separated by ';' delimiter are supported." 
+      .withDocumentation("Spark SQL queries to run on table before committing new data to validate state before and after commit."
+          + "Multiple queries separated by ';' delimiter are supported."
           + "Example query: 'select count(*) from \\<TABLE_NAME\\> where col=null'"
           + "Note \\<TABLE_NAME\\> variable is expected to be present in query.");
 
@@ -94,22 +95,22 @@ public class HoodiePreCommitValidatorConfig extends HoodieConfig {
     }
 
     public Builder withPreCommitValidator(String preCommitValidators) {
-      preCommitValidatorConfig.setValue(PRE_COMMIT_VALIDATORS, preCommitValidators);
+      preCommitValidatorConfig.setValue(VALIDATOR_CLASS_NAMES, preCommitValidators);
       return this;
     }
 
     public Builder withPrecommitValidatorEqualitySqlQueries(String preCommitValidators) {
-      preCommitValidatorConfig.setValue(PRE_COMMIT_VALIDATORS_EQUALITY_SQL_QUERIES, preCommitValidators);
+      preCommitValidatorConfig.setValue(EQUALITY_SQL_QUERIES, preCommitValidators);
       return this;
     }
 
     public Builder withPrecommitValidatorSingleResultSqlQueries(String preCommitValidators) {
-      preCommitValidatorConfig.setValue(PRE_COMMIT_VALIDATORS_SINGLE_VALUE_SQL_QUERIES, preCommitValidators);
+      preCommitValidatorConfig.setValue(SINGLE_VALUE_SQL_QUERIES, preCommitValidators);
       return this;
     }
 
     public Builder withPrecommitValidatorInequalitySqlQueries(String preCommitValidators) {
-      preCommitValidatorConfig.setValue(PRE_COMMIT_VALIDATORS_INEQUALITY_SQL_QUERIES, preCommitValidators);
+      preCommitValidatorConfig.setValue(INEQUALITY_SQL_QUERIES, preCommitValidators);
       return this;
     }
 
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java
index 4015371..9dcde92 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieStorageConfig.java
@@ -39,25 +39,25 @@ import java.util.Properties;
     description = "Configurations that control aspects around writing, sizing, reading base and log files.")
 public class HoodieStorageConfig extends HoodieConfig {
 
-  public static final ConfigProperty<String> PARQUET_FILE_MAX_BYTES = ConfigProperty
+  public static final ConfigProperty<String> PARQUET_MAX_FILE_SIZE = ConfigProperty
       .key("hoodie.parquet.max.file.size")
       .defaultValue(String.valueOf(120 * 1024 * 1024))
       .withDocumentation("Target size for parquet files produced by Hudi write phases. "
           + "For DFS, this needs to be aligned with the underlying filesystem block size for optimal performance.");
 
-  public static final ConfigProperty<String> PARQUET_BLOCK_SIZE_BYTES = ConfigProperty
+  public static final ConfigProperty<String> PARQUET_BLOCK_SIZE = ConfigProperty
       .key("hoodie.parquet.block.size")
       .defaultValue(String.valueOf(120 * 1024 * 1024))
       .withDocumentation("Parquet RowGroup size. It's recommended to make this large enough that scan costs can be"
           + " amortized by packing enough column values into a single row group.");
 
-  public static final ConfigProperty<String> PARQUET_PAGE_SIZE_BYTES = ConfigProperty
+  public static final ConfigProperty<String> PARQUET_PAGE_SIZE = ConfigProperty
       .key("hoodie.parquet.page.size")
       .defaultValue(String.valueOf(1 * 1024 * 1024))
       .withDocumentation("Parquet page size. Page is the unit of read within a parquet file. "
           + "Within a block, pages are compressed separately.");
 
-  public static final ConfigProperty<String> ORC_FILE_MAX_BYTES = ConfigProperty
+  public static final ConfigProperty<String> ORC_FILE_MAX_SIZE = ConfigProperty
       .key("hoodie.orc.max.file.size")
       .defaultValue(String.valueOf(120 * 1024 * 1024))
       .withDocumentation("Target file size for ORC base files.");
@@ -69,64 +69,175 @@ public class HoodieStorageConfig extends HoodieConfig {
 
   public static final ConfigProperty<String> ORC_BLOCK_SIZE = ConfigProperty
       .key("hoodie.orc.block.size")
-      .defaultValue(ORC_FILE_MAX_BYTES.defaultValue())
+      .defaultValue(ORC_FILE_MAX_SIZE.defaultValue())
       .withDocumentation("ORC block size, recommended to be aligned with the target file size.");
 
-  public static final ConfigProperty<String> HFILE_FILE_MAX_BYTES = ConfigProperty
+  public static final ConfigProperty<String> HFILE_MAX_FILE_SIZE = ConfigProperty
       .key("hoodie.hfile.max.file.size")
       .defaultValue(String.valueOf(120 * 1024 * 1024))
       .withDocumentation("Target file size for HFile base files.");
 
-  public static final ConfigProperty<String> HFILE_BLOCK_SIZE_BYTES = ConfigProperty
+  public static final ConfigProperty<String> HFILE_BLOCK_SIZE = ConfigProperty
       .key("hoodie.hfile.block.size")
       .defaultValue(String.valueOf(1024 * 1024))
       .withDocumentation("Lower values increase the size of metadata tracked within HFile, but can offer potentially "
           + "faster lookup times.");
 
   // used to size log files
-  public static final ConfigProperty<String> LOGFILE_SIZE_MAX_BYTES = ConfigProperty
+  public static final ConfigProperty<String> LOGFILE_MAX_SIZE = ConfigProperty
       .key("hoodie.logfile.max.size")
       .defaultValue(String.valueOf(1024 * 1024 * 1024)) // 1 GB
       .withDocumentation("LogFile max size. This is the maximum size allowed for a log file "
           + "before it is rolled over to the next version.");
 
   // used to size data blocks in log file
-  public static final ConfigProperty<String> LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = ConfigProperty
+  public static final ConfigProperty<String> LOGFILE_DATA_BLOCK_MAX_SIZE = ConfigProperty
       .key("hoodie.logfile.data.block.max.size")
       .defaultValue(String.valueOf(256 * 1024 * 1024))
       .withDocumentation("LogFile Data block max size. This is the maximum size allowed for a single data block "
           + "to be appended to a log file. This helps to make sure the data appended to the log file is broken up "
           + "into sizable blocks to prevent from OOM errors. This size should be greater than the JVM memory.");
 
-  public static final ConfigProperty<String> PARQUET_COMPRESSION_RATIO = ConfigProperty
+  public static final ConfigProperty<String> PARQUET_COMPRESSION_RATIO_FRACTION = ConfigProperty
       .key("hoodie.parquet.compression.ratio")
       .defaultValue(String.valueOf(0.1))
       .withDocumentation("Expected compression of parquet data used by Hudi, when it tries to size new parquet files. "
           + "Increase this value, if bulk_insert is producing smaller than expected sized files");
 
   // Default compression codec for parquet
-  public static final ConfigProperty<String> PARQUET_COMPRESSION_CODEC = ConfigProperty
+  public static final ConfigProperty<String> PARQUET_COMPRESSION_CODEC_NAME = ConfigProperty
       .key("hoodie.parquet.compression.codec")
       .defaultValue("gzip")
       .withDocumentation("Compression Codec for parquet files");
 
-  public static final ConfigProperty<String> HFILE_COMPRESSION_ALGORITHM = ConfigProperty
+  public static final ConfigProperty<String> HFILE_COMPRESSION_ALGORITHM_NAME = ConfigProperty
       .key("hoodie.hfile.compression.algorithm")
       .defaultValue("GZ")
       .withDocumentation("Compression codec to use for hfile base files.");
 
-  public static final ConfigProperty<String> ORC_COMPRESSION_CODEC = ConfigProperty
+  public static final ConfigProperty<String> ORC_COMPRESSION_CODEC_NAME = ConfigProperty
       .key("hoodie.orc.compression.codec")
       .defaultValue("ZLIB")
       .withDocumentation("Compression codec to use for ORC base files.");
 
   // Default compression ratio for log file to parquet, general 3x
-  public static final ConfigProperty<String> LOGFILE_TO_PARQUET_COMPRESSION_RATIO = ConfigProperty
+  public static final ConfigProperty<String> LOGFILE_TO_PARQUET_COMPRESSION_RATIO_FRACTION = ConfigProperty
       .key("hoodie.logfile.to.parquet.compression.ratio")
       .defaultValue(String.valueOf(0.35))
       .withDocumentation("Expected additional compression as records move from log files to parquet. Used for merge_on_read "
           + "table to send inserts into log files & control the size of compacted parquet file.");
 
+  /**
+   * @deprecated Use {@link #PARQUET_MAX_FILE_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String PARQUET_FILE_MAX_BYTES = PARQUET_MAX_FILE_SIZE.key();
+  /**
+   * @deprecated Use {@link #PARQUET_MAX_FILE_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_PARQUET_FILE_MAX_BYTES = PARQUET_MAX_FILE_SIZE.defaultValue();
+  /**
+   * @deprecated Use {@link #PARQUET_BLOCK_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String PARQUET_BLOCK_SIZE_BYTES = PARQUET_BLOCK_SIZE.key();
+  /**
+   * @deprecated Use {@link #PARQUET_BLOCK_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_PARQUET_BLOCK_SIZE_BYTES = PARQUET_BLOCK_SIZE.defaultValue();
+  /**
+   * @deprecated Use {@link #PARQUET_PAGE_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String PARQUET_PAGE_SIZE_BYTES = PARQUET_PAGE_SIZE.key();
+  /**
+   * @deprecated Use {@link #PARQUET_PAGE_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_PARQUET_PAGE_SIZE_BYTES = PARQUET_PAGE_SIZE.defaultValue();
+  /**
+   * @deprecated Use {@link #HFILE_MAX_FILE_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String HFILE_FILE_MAX_BYTES = HFILE_MAX_FILE_SIZE.key();
+  /**
+   * @deprecated Use {@link #HFILE_MAX_FILE_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_HFILE_FILE_MAX_BYTES = HFILE_MAX_FILE_SIZE.defaultValue();
+  /**
+   * @deprecated Use {@link #HFILE_BLOCK_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String HFILE_BLOCK_SIZE_BYTES = HFILE_BLOCK_SIZE.defaultValue();
+  /**
+   * @deprecated Use {@link #HFILE_BLOCK_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_HFILE_BLOCK_SIZE_BYTES = HFILE_BLOCK_SIZE.defaultValue();
+  /**
+   * @deprecated Use {@link #LOGFILE_MAX_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String LOGFILE_SIZE_MAX_BYTES = LOGFILE_MAX_SIZE.key();
+  /**
+   * @deprecated Use {@link #LOGFILE_MAX_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_LOGFILE_SIZE_MAX_BYTES = LOGFILE_MAX_SIZE.defaultValue();
+  /**
+   * @deprecated Use {@link #LOGFILE_DATA_BLOCK_MAX_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = LOGFILE_DATA_BLOCK_MAX_SIZE.key();
+  /**
+   * @deprecated Use {@link #LOGFILE_DATA_BLOCK_MAX_SIZE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = LOGFILE_DATA_BLOCK_MAX_SIZE.defaultValue();
+  /**
+   * @deprecated Use {@link #PARQUET_COMPRESSION_RATIO_FRACTION} and its methods instead
+   */
+  @Deprecated
+  public static final String PARQUET_COMPRESSION_RATIO = PARQUET_COMPRESSION_RATIO_FRACTION.key();
+  /**
+   * @deprecated Use {@link #PARQUET_COMPRESSION_RATIO_FRACTION} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_STREAM_COMPRESSION_RATIO = PARQUET_COMPRESSION_RATIO_FRACTION.defaultValue();
+  /**
+   * @deprecated Use {@link #PARQUET_COMPRESSION_CODEC_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String PARQUET_COMPRESSION_CODEC = PARQUET_COMPRESSION_CODEC_NAME.key();
+  /**
+   * @deprecated Use {@link #HFILE_COMPRESSION_ALGORITHM_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String HFILE_COMPRESSION_ALGORITHM = HFILE_COMPRESSION_ALGORITHM_NAME.key();
+  /**
+   * @deprecated Use {@link #PARQUET_COMPRESSION_CODEC_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_PARQUET_COMPRESSION_CODEC = PARQUET_COMPRESSION_CODEC_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #HFILE_COMPRESSION_ALGORITHM_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_HFILE_COMPRESSION_ALGORITHM = HFILE_COMPRESSION_ALGORITHM_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #LOGFILE_TO_PARQUET_COMPRESSION_RATIO_FRACTION} and its methods instead
+   */
+  @Deprecated
+  public static final String LOGFILE_TO_PARQUET_COMPRESSION_RATIO = LOGFILE_TO_PARQUET_COMPRESSION_RATIO_FRACTION.key();
+  /**
+   * @deprecated Use {@link #LOGFILE_TO_PARQUET_COMPRESSION_RATIO_FRACTION} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO = LOGFILE_TO_PARQUET_COMPRESSION_RATIO_FRACTION.defaultValue();
+
   private HoodieStorageConfig() {
     super();
   }
@@ -152,62 +263,62 @@ public class HoodieStorageConfig extends HoodieConfig {
     }
 
     public Builder parquetMaxFileSize(long maxFileSize) {
-      storageConfig.setValue(PARQUET_FILE_MAX_BYTES, String.valueOf(maxFileSize));
+      storageConfig.setValue(PARQUET_MAX_FILE_SIZE, String.valueOf(maxFileSize));
       return this;
     }
 
     public Builder parquetBlockSize(int blockSize) {
-      storageConfig.setValue(PARQUET_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
+      storageConfig.setValue(PARQUET_BLOCK_SIZE, String.valueOf(blockSize));
       return this;
     }
 
     public Builder parquetPageSize(int pageSize) {
-      storageConfig.setValue(PARQUET_PAGE_SIZE_BYTES, String.valueOf(pageSize));
+      storageConfig.setValue(PARQUET_PAGE_SIZE, String.valueOf(pageSize));
       return this;
     }
 
     public Builder hfileMaxFileSize(long maxFileSize) {
-      storageConfig.setValue(HFILE_FILE_MAX_BYTES, String.valueOf(maxFileSize));
+      storageConfig.setValue(HFILE_MAX_FILE_SIZE, String.valueOf(maxFileSize));
       return this;
     }
 
     public Builder hfileBlockSize(int blockSize) {
-      storageConfig.setValue(HFILE_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
+      storageConfig.setValue(HFILE_BLOCK_SIZE, String.valueOf(blockSize));
       return this;
     }
 
     public Builder logFileDataBlockMaxSize(int dataBlockSize) {
-      storageConfig.setValue(LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES, String.valueOf(dataBlockSize));
+      storageConfig.setValue(LOGFILE_DATA_BLOCK_MAX_SIZE, String.valueOf(dataBlockSize));
       return this;
     }
 
     public Builder logFileMaxSize(int logFileSize) {
-      storageConfig.setValue(LOGFILE_SIZE_MAX_BYTES, String.valueOf(logFileSize));
+      storageConfig.setValue(LOGFILE_MAX_SIZE, String.valueOf(logFileSize));
       return this;
     }
 
     public Builder parquetCompressionRatio(double parquetCompressionRatio) {
-      storageConfig.setValue(PARQUET_COMPRESSION_RATIO, String.valueOf(parquetCompressionRatio));
+      storageConfig.setValue(PARQUET_COMPRESSION_RATIO_FRACTION, String.valueOf(parquetCompressionRatio));
       return this;
     }
 
     public Builder parquetCompressionCodec(String parquetCompressionCodec) {
-      storageConfig.setValue(PARQUET_COMPRESSION_CODEC, parquetCompressionCodec);
+      storageConfig.setValue(PARQUET_COMPRESSION_CODEC_NAME, parquetCompressionCodec);
       return this;
     }
 
     public Builder hfileCompressionAlgorithm(String hfileCompressionAlgorithm) {
-      storageConfig.setValue(HFILE_COMPRESSION_ALGORITHM, hfileCompressionAlgorithm);
+      storageConfig.setValue(HFILE_COMPRESSION_ALGORITHM_NAME, hfileCompressionAlgorithm);
       return this;
     }
 
     public Builder logFileToParquetCompressionRatio(double logFileToParquetCompressionRatio) {
-      storageConfig.setValue(LOGFILE_TO_PARQUET_COMPRESSION_RATIO, String.valueOf(logFileToParquetCompressionRatio));
+      storageConfig.setValue(LOGFILE_TO_PARQUET_COMPRESSION_RATIO_FRACTION, String.valueOf(logFileToParquetCompressionRatio));
       return this;
     }
 
     public Builder orcMaxFileSize(long maxFileSize) {
-      storageConfig.setValue(ORC_FILE_MAX_BYTES, String.valueOf(maxFileSize));
+      storageConfig.setValue(ORC_FILE_MAX_SIZE, String.valueOf(maxFileSize));
       return this;
     }
 
@@ -222,7 +333,7 @@ public class HoodieStorageConfig extends HoodieConfig {
     }
 
     public Builder orcCompressionCodec(String orcCompressionCodec) {
-      storageConfig.setValue(ORC_COMPRESSION_CODEC, orcCompressionCodec);
+      storageConfig.setValue(ORC_COMPRESSION_CODEC_NAME, orcCompressionCodec);
       return this;
     }
 
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java
index d313ea3..09c2b09 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteCommitCallbackConfig.java
@@ -38,20 +38,18 @@ public class HoodieWriteCommitCallbackConfig extends HoodieConfig {
 
   public static final String CALLBACK_PREFIX = "hoodie.write.commit.callback.";
 
-  public static final ConfigProperty<Boolean> CALLBACK_ON = ConfigProperty
+  public static final ConfigProperty<Boolean> TURN_CALLBACK_ON = ConfigProperty
       .key(CALLBACK_PREFIX + "on")
       .defaultValue(false)
       .sinceVersion("0.6.0")
       .withDocumentation("Turn commit callback on/off. off by default.");
 
-  public static final ConfigProperty<String> CALLBACK_CLASS = ConfigProperty
+  public static final ConfigProperty<String> CALLBACK_CLASS_NAME = ConfigProperty
       .key(CALLBACK_PREFIX + "class")
       .defaultValue("org.apache.hudi.callback.impl.HoodieWriteCommitHttpCallback")
       .sinceVersion("0.6.0")
       .withDocumentation("Full path of callback class and must be a subclass of HoodieWriteCommitCallback class, "
           + "org.apache.hudi.callback.impl.HoodieWriteCommitHttpCallback by default");
-  @Deprecated
-  public static final String CALLBACK_CLASS_PROP = CALLBACK_CLASS.key();
 
   // ***** HTTP callback configs *****
   public static final ConfigProperty<String> CALLBACK_HTTP_URL = ConfigProperty
@@ -59,21 +57,65 @@ public class HoodieWriteCommitCallbackConfig extends HoodieConfig {
       .noDefaultValue()
       .sinceVersion("0.6.0")
       .withDocumentation("Callback host to be sent along with callback messages");
-  @Deprecated
-  public static final String CALLBACK_HTTP_URL_PROP = CALLBACK_HTTP_URL.key();
 
-  public static final ConfigProperty<String> CALLBACK_HTTP_API_KEY = ConfigProperty
+  public static final ConfigProperty<String> CALLBACK_HTTP_API_KEY_VALUE = ConfigProperty
       .key(CALLBACK_PREFIX + "http.api.key")
       .defaultValue("hudi_write_commit_http_callback")
       .sinceVersion("0.6.0")
       .withDocumentation("Http callback API key. hudi_write_commit_http_callback by default");
 
-  public static final ConfigProperty<Integer> CALLBACK_HTTP_TIMEOUT_SECONDS = ConfigProperty
+  public static final ConfigProperty<Integer> CALLBACK_HTTP_TIMEOUT_IN_SECONDS = ConfigProperty
       .key(CALLBACK_PREFIX + "http.timeout.seconds")
       .defaultValue(3)
       .sinceVersion("0.6.0")
       .withDocumentation("Callback timeout in seconds. 3 by default");
 
+  /**
+   * @deprecated Use {@link #TURN_CALLBACK_ON} and its methods instead
+   */
+  @Deprecated
+  public static final String CALLBACK_ON = TURN_CALLBACK_ON.key();
+  /**
+   * @deprecated Use {@link #TURN_CALLBACK_ON} and its methods instead
+   */
+  @Deprecated
+  public static final boolean DEFAULT_CALLBACK_ON = TURN_CALLBACK_ON.defaultValue();
+  /**
+   * @deprecated Use {@link #CALLBACK_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String CALLBACK_CLASS_PROP = CALLBACK_CLASS_NAME.key();
+  /**
+   * @deprecated Use {@link #CALLBACK_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_CALLBACK_CLASS_PROP = CALLBACK_CLASS_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #CALLBACK_HTTP_URL} and its methods instead
+   */
+  @Deprecated
+  public static final String CALLBACK_HTTP_URL_PROP = CALLBACK_HTTP_URL.key();
+  /**
+   * @deprecated Use {@link #CALLBACK_HTTP_API_KEY_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String CALLBACK_HTTP_API_KEY = CALLBACK_HTTP_API_KEY_VALUE.key();
+  /**
+   * @deprecated Use {@link #CALLBACK_HTTP_API_KEY_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_CALLBACK_HTTP_API_KEY = CALLBACK_HTTP_API_KEY_VALUE.defaultValue();
+  /**
+   * @deprecated Use {@link #CALLBACK_HTTP_TIMEOUT_IN_SECONDS} and its methods instead
+   */
+  @Deprecated
+  public static final String CALLBACK_HTTP_TIMEOUT_SECONDS = CALLBACK_HTTP_TIMEOUT_IN_SECONDS.key();
+  /**
+   * @deprecated Use {@link #CALLBACK_HTTP_TIMEOUT_IN_SECONDS} and its methods instead
+   */
+  @Deprecated
+  public static final int DEFAULT_CALLBACK_HTTP_TIMEOUT_SECONDS = CALLBACK_HTTP_TIMEOUT_IN_SECONDS.defaultValue();
+
   private HoodieWriteCommitCallbackConfig() {
     super();
   }
@@ -99,12 +141,12 @@ public class HoodieWriteCommitCallbackConfig extends HoodieConfig {
     }
 
     public HoodieWriteCommitCallbackConfig.Builder writeCommitCallbackOn(String callbackOn) {
-      writeCommitCallbackConfig.setValue(CALLBACK_ON, callbackOn);
+      writeCommitCallbackConfig.setValue(TURN_CALLBACK_ON, callbackOn);
       return this;
     }
 
     public HoodieWriteCommitCallbackConfig.Builder withCallbackClass(String callbackClass) {
-      writeCommitCallbackConfig.setValue(CALLBACK_CLASS, callbackClass);
+      writeCommitCallbackConfig.setValue(CALLBACK_CLASS_NAME, callbackClass);
       return this;
     }
 
@@ -114,12 +156,12 @@ public class HoodieWriteCommitCallbackConfig extends HoodieConfig {
     }
 
     public Builder withCallbackHttpTimeoutSeconds(String timeoutSeconds) {
-      writeCommitCallbackConfig.setValue(CALLBACK_HTTP_TIMEOUT_SECONDS, timeoutSeconds);
+      writeCommitCallbackConfig.setValue(CALLBACK_HTTP_TIMEOUT_IN_SECONDS, timeoutSeconds);
       return this;
     }
 
     public Builder withCallbackHttpApiKey(String apiKey) {
-      writeCommitCallbackConfig.setValue(CALLBACK_HTTP_API_KEY, apiKey);
+      writeCommitCallbackConfig.setValue(CALLBACK_HTTP_API_KEY_VALUE, apiKey);
       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 122b90f..448ce9f 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
@@ -42,6 +42,7 @@ import org.apache.hudi.common.util.ReflectionUtils;
 import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode;
 import org.apache.hudi.index.HoodieIndex;
+import org.apache.hudi.keygen.SimpleAvroKeyGenerator;
 import org.apache.hudi.keygen.constant.KeyGeneratorType;
 import org.apache.hudi.metrics.MetricsReporterType;
 import org.apache.hudi.metrics.datadog.DatadogHttpClient.ApiSite;
@@ -78,7 +79,7 @@ public class HoodieWriteConfig extends HoodieConfig {
 
   private static final long serialVersionUID = 0L;
 
-  public static final ConfigProperty<String> TABLE_NAME = ConfigProperty
+  public static final ConfigProperty<String> TBL_NAME = ConfigProperty
       .key("hoodie.table.name")
       .noDefaultValue()
       .withDocumentation("Table name that will be used for registering with metastores like HMS. Needs to be same across runs.");
@@ -88,22 +89,18 @@ public class HoodieWriteConfig extends HoodieConfig {
       .defaultValue("ts")
       .withDocumentation("Field used in preCombining before actual write. When two records have the same key value, "
           + "we will pick the one with the largest value for the precombine field, determined by Object.compareTo(..)");
-  @Deprecated
-  public static final String PRECOMBINE_FIELD_PROP = PRECOMBINE_FIELD.key();
 
-  public static final ConfigProperty<String> WRITE_PAYLOAD_CLASS = ConfigProperty
+  public static final ConfigProperty<String> WRITE_PAYLOAD_CLASS_NAME = ConfigProperty
       .key("hoodie.datasource.write.payload.class")
       .defaultValue(OverwriteWithLatestAvroPayload.class.getName())
       .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> KEYGENERATOR_CLASS = ConfigProperty
+  public static final ConfigProperty<String> KEYGENERATOR_CLASS_NAME = ConfigProperty
       .key("hoodie.datasource.write.keygenerator.class")
       .noDefaultValue()
       .withDocumentation("Key generator class, that implements `org.apache.hudi.keygen.KeyGenerator` "
           + "extract a key out of incoming records.");
-  @Deprecated
-  public static final String KEYGENERATOR_CLASS_PROP = KEYGENERATOR_CLASS.key();
 
   public static final ConfigProperty<String> KEYGENERATOR_TYPE = ConfigProperty
       .key("hoodie.datasource.write.keygenerator.type")
@@ -111,13 +108,13 @@ public class HoodieWriteConfig extends HoodieConfig {
       .withDocumentation("Easily configure one the built-in key generators, instead of specifying the key generator class."
           + "Currently supports SIMPLE, COMPLEX, TIMESTAMP, CUSTOM, NON_PARTITION, GLOBAL_DELETE");
 
-  public static final ConfigProperty<String> ROLLBACK_USING_MARKERS = ConfigProperty
+  public static final ConfigProperty<String> ROLLBACK_USING_MARKERS_ENABLE = ConfigProperty
       .key("hoodie.rollback.using.markers")
       .defaultValue("false")
       .withDocumentation("Enables a more efficient mechanism for rollbacks based on the marker files generated "
           + "during the writes. Turned off by default.");
 
-  public static final ConfigProperty<String> TIMELINE_LAYOUT_VERSION = ConfigProperty
+  public static final ConfigProperty<String> TIMELINE_LAYOUT_VERSION_NUM = ConfigProperty
       .key("hoodie.timeline.layout.version")
       .defaultValue(Integer.toString(TimelineLayoutVersion.VERSION_1))
       .sinceVersion("0.5.1")
@@ -137,55 +134,53 @@ public class HoodieWriteConfig extends HoodieConfig {
           + "Always prefix it explicitly with the storage scheme (e.g hdfs://, s3:// etc). "
           + "Hudi stores all the main meta-data about commits, savepoints, cleaning audit logs "
           + "etc in .hoodie directory under this base path directory.");
-  @Deprecated
-  public static final String BASE_PATH_PROP = BASE_PATH.key();
 
-  public static final ConfigProperty<String> AVRO_SCHEMA = ConfigProperty
+  public static final ConfigProperty<String> AVRO_SCHEMA_STRING = ConfigProperty
       .key("hoodie.avro.schema")
       .noDefaultValue()
       .withDocumentation("Schema string representing the current write schema of the table. Hudi passes this to "
           + "implementations of HoodieRecordPayload to convert incoming records to avro. This is also used as the write schema "
           + "evolving records during an update.");
 
-  public static final ConfigProperty<String> AVRO_SCHEMA_VALIDATE = ConfigProperty
+  public static final ConfigProperty<String> AVRO_SCHEMA_VALIDATE_ENABLE = ConfigProperty
       .key("hoodie.avro.schema.validate")
       .defaultValue("false")
       .withDocumentation("Validate the schema used for the write against the latest schema, for backwards compatibility.");
 
-  public static final ConfigProperty<String> INSERT_PARALLELISM = ConfigProperty
+  public static final ConfigProperty<String> INSERT_PARALLELISM_VALUE = ConfigProperty
       .key("hoodie.insert.shuffle.parallelism")
       .defaultValue("1500")
       .withDocumentation("Parallelism for inserting records into the table. Inserts can shuffle data before writing to tune file sizes and optimize the storage layout.");
 
-  public static final ConfigProperty<String> BULKINSERT_PARALLELISM = ConfigProperty
+  public static final ConfigProperty<String> BULKINSERT_PARALLELISM_VALUE = ConfigProperty
       .key("hoodie.bulkinsert.shuffle.parallelism")
       .defaultValue("1500")
       .withDocumentation("For large initial imports using bulk_insert operation, controls the parallelism to use for sort modes or custom partitioning done"
           + "before writing records to the table.");
 
-  public static final ConfigProperty<String> BULKINSERT_USER_DEFINED_PARTITIONER_CLASS = ConfigProperty
+  public static final ConfigProperty<String> BULKINSERT_USER_DEFINED_PARTITIONER_CLASS_NAME = ConfigProperty
       .key("hoodie.bulkinsert.user.defined.partitioner.class")
       .noDefaultValue()
       .withDocumentation("If specified, this class will be used to re-partition records before they are bulk inserted. This can be used to sort, pack, cluster data"
           + " optimally for common query patterns.");
 
-  public static final ConfigProperty<String> UPSERT_PARALLELISM = ConfigProperty
+  public static final ConfigProperty<String> UPSERT_PARALLELISM_VALUE = ConfigProperty
       .key("hoodie.upsert.shuffle.parallelism")
       .defaultValue("1500")
       .withDocumentation("Parallelism to use for upsert operation on the table. Upserts can shuffle data to perform index lookups, file sizing, bin packing records optimally"
           + "into file groups.");
 
-  public static final ConfigProperty<String> DELETE_PARALLELISM = ConfigProperty
+  public static final ConfigProperty<String> DELETE_PARALLELISM_VALUE = ConfigProperty
       .key("hoodie.delete.shuffle.parallelism")
       .defaultValue("1500")
       .withDocumentation("Parallelism used for “delete” operation. Delete operations also performs shuffles, similar to upsert operation.");
 
-  public static final ConfigProperty<String> ROLLBACK_PARALLELISM = ConfigProperty
+  public static final ConfigProperty<String> ROLLBACK_PARALLELISM_VALUE = ConfigProperty
       .key("hoodie.rollback.parallelism")
       .defaultValue("100")
       .withDocumentation("Parallelism for rollback of commits. Rollbacks perform delete of files or logging delete blocks to file groups on storage in parallel.");
 
-  public static final ConfigProperty<String> WRITE_BUFFER_LIMIT_BYTES = ConfigProperty
+  public static final ConfigProperty<String> WRITE_BUFFER_LIMIT_BYTES_VALUE = ConfigProperty
       .key("hoodie.write.buffer.limit.bytes")
       .defaultValue(String.valueOf(4 * 1024 * 1024))
       .withDocumentation("Size of in-memory buffer used for parallelizing network reads and lake storage writes.");
@@ -195,8 +190,6 @@ public class HoodieWriteConfig extends HoodieConfig {
       .defaultValue("false")
       .withDocumentation("When inserted records share same key, controls whether they should be first combined (i.e de-duplicated) before"
           + " writing to storage.");
-  @Deprecated
-  public static final String COMBINE_BEFORE_INSERT_PROP = COMBINE_BEFORE_INSERT.key();
 
   public static final ConfigProperty<String> COMBINE_BEFORE_UPSERT = ConfigProperty
       .key("hoodie.combine.before.upsert")
@@ -204,40 +197,32 @@ public class HoodieWriteConfig extends HoodieConfig {
       .withDocumentation("When upserted records share same key, controls whether they should be first combined (i.e de-duplicated) before"
           + " writing to storage. This should be turned off only if you are absolutely certain that there are no duplicates incoming, "
           + " otherwise it can lead to duplicate keys and violate the uniqueness guarantees.");
-  @Deprecated
-  public static final String COMBINE_BEFORE_UPSERT_PROP = COMBINE_BEFORE_UPSERT.key();
 
   public static final ConfigProperty<String> COMBINE_BEFORE_DELETE = ConfigProperty
       .key("hoodie.combine.before.delete")
       .defaultValue("true")
       .withDocumentation("During delete operations, controls whether we should combine deletes (and potentially also upserts) before "
           + " writing to storage.");
-  @Deprecated
-  public static final String COMBINE_BEFORE_DELETE_PROP = COMBINE_BEFORE_DELETE.key();
 
-  public static final ConfigProperty<String> WRITE_STATUS_STORAGE_LEVEL = ConfigProperty
+  public static final ConfigProperty<String> WRITE_STATUS_STORAGE_LEVEL_VALUE = ConfigProperty
       .key("hoodie.write.status.storage.level")
       .defaultValue("MEMORY_AND_DISK_SER")
       .withDocumentation("Write status objects hold metadata about a write (stats, errors), that is not yet committed to storage. "
           + "This controls the how that information is cached for inspection by clients. We rarely expect this to be changed.");
 
-  public static final ConfigProperty<String> HOODIE_AUTO_COMMIT = ConfigProperty
+  public static final ConfigProperty<String> AUTO_COMMIT_ENABLE = ConfigProperty
       .key("hoodie.auto.commit")
       .defaultValue("true")
       .withDocumentation("Controls whether a write operation should auto commit. This can be turned off to perform inspection"
           + " of the uncommitted write before deciding to commit.");
-  @Deprecated
-  public static final String HOODIE_AUTO_COMMIT_PROP = HOODIE_AUTO_COMMIT.key();
 
-  public static final ConfigProperty<String> HOODIE_WRITE_STATUS_CLASS = ConfigProperty
+  public static final ConfigProperty<String> WRITE_STATUS_CLASS_NAME = ConfigProperty
       .key("hoodie.writestatus.class")
       .defaultValue(WriteStatus.class.getName())
       .withDocumentation("Subclass of " + WriteStatus.class.getName() + " to be used to collect information about a write. Can be "
           + "overridden to collection additional metrics/statistics about the data if needed.");
-  @Deprecated
-  public static final String HOODIE_WRITE_STATUS_CLASS_PROP = HOODIE_WRITE_STATUS_CLASS.key();
 
-  public static final ConfigProperty<String> FINALIZE_WRITE_PARALLELISM = ConfigProperty
+  public static final ConfigProperty<String> FINALIZE_WRITE_PARALLELISM_VALUE = ConfigProperty
       .key("hoodie.finalize.write.parallelism")
       .defaultValue("1500")
       .withDocumentation("Parallelism for the write finalization internal operation, which involves removing any partially written "
@@ -268,23 +253,23 @@ public class HoodieWriteConfig extends HoodieConfig {
       .sinceVersion("0.9.0")
       .withDocumentation("The batch interval in milliseconds for marker creation batch processing");
 
-  public static final ConfigProperty<String> MARKERS_DELETE_PARALLELISM = ConfigProperty
+  public static final ConfigProperty<String> MARKERS_DELETE_PARALLELISM_VALUE = ConfigProperty
       .key("hoodie.markers.delete.parallelism")
       .defaultValue("100")
       .withDocumentation("Determines the parallelism for deleting marker files, which are used to track all files (valid or invalid/partial) written during "
           + "a write operation. Increase this value if delays are observed, with large batch writes.");
 
-  public static final ConfigProperty<String> BULKINSERT_SORT_MODE = ConfigProperty
+  public static final ConfigProperty<String> BULK_INSERT_SORT_MODE = ConfigProperty
       .key("hoodie.bulkinsert.sort.mode")
       .defaultValue(BulkInsertSortMode.GLOBAL_SORT.toString())
       .withDocumentation("Sorting modes to use for sorting records for bulk insert. This is user when user "
-          + BULKINSERT_USER_DEFINED_PARTITIONER_CLASS.key() + "is not configured. Available values are - "
+          + BULKINSERT_USER_DEFINED_PARTITIONER_CLASS_NAME.key() + "is not configured. Available values are - "
           + "GLOBAL_SORT: this ensures best file sizes, with lowest memory overhead at cost of sorting. "
           + "PARTITION_SORT: Strikes a balance by only sorting within a partition, still keeping the memory overhead of writing "
           + "lowest and best effort file sizing. "
           + "NONE: No sorting. Fastest and matches `spark.write.parquet()` in terms of number of files, overheads");
 
-  public static final ConfigProperty<String> EMBEDDED_TIMELINE_SERVER_ENABLED = ConfigProperty
+  public static final ConfigProperty<String> EMBEDDED_TIMELINE_SERVER_ENABLE = ConfigProperty
       .key("hoodie.embed.timeline.server")
       .defaultValue("true")
       .withDocumentation("When true, spins up an instance of the timeline server (meta server that serves cached file listings, statistics),"
@@ -296,65 +281,57 @@ public class HoodieWriteConfig extends HoodieConfig {
       .withDocumentation("Controls whether the timeline server instance should be cached and reused across the JVM (across task lifecycles)"
           + "to avoid startup costs. This should rarely be changed.");
 
-  public static final ConfigProperty<String> EMBEDDED_TIMELINE_SERVER_PORT = ConfigProperty
+  public static final ConfigProperty<String> EMBEDDED_TIMELINE_SERVER_PORT_NUM = ConfigProperty
       .key("hoodie.embed.timeline.server.port")
       .defaultValue("0")
       .withDocumentation("Port at which the timeline server listens for requests. When running embedded in each writer, it picks "
           + "a free port and communicates to all the executors. This should rarely be changed.");
 
-  public static final ConfigProperty<String> EMBEDDED_TIMELINE_SERVER_THREADS = ConfigProperty
+  public static final ConfigProperty<String> EMBEDDED_TIMELINE_NUM_SERVER_THREADS = ConfigProperty
       .key("hoodie.embed.timeline.server.threads")
       .defaultValue("-1")
       .withDocumentation("Number of threads to serve requests in the timeline server. By default, auto configured based on the number of underlying cores.");
 
-  public static final ConfigProperty<String> EMBEDDED_TIMELINE_SERVER_COMPRESS_OUTPUT = ConfigProperty
+  public static final ConfigProperty<String> EMBEDDED_TIMELINE_SERVER_COMPRESS_ENABLE = ConfigProperty
       .key("hoodie.embed.timeline.server.gzip")
       .defaultValue("true")
       .withDocumentation("Controls whether gzip compression is used, for large responses from the timeline server, to improve latency.");
 
-  public static final ConfigProperty<String> EMBEDDED_TIMELINE_SERVER_USE_ASYNC = ConfigProperty
+  public static final ConfigProperty<String> EMBEDDED_TIMELINE_SERVER_USE_ASYNC_ENABLE = ConfigProperty
       .key("hoodie.embed.timeline.server.async")
       .defaultValue("false")
       .withDocumentation("Controls whether or not, the requests to the timeline server are processed in asynchronous fashion, "
           + "potentially improving throughput.");
 
-  public static final ConfigProperty<String> FAIL_ON_TIMELINE_ARCHIVING_ENABLED = ConfigProperty
+  public static final ConfigProperty<String> FAIL_ON_TIMELINE_ARCHIVING_ENABLE = ConfigProperty
       .key("hoodie.fail.on.timeline.archiving")
       .defaultValue("true")
       .withDocumentation("Timeline archiving removes older instants from the timeline, after each write operation, to minimize metadata overhead. "
           + "Controls whether or not, the write should be failed as well, if such archiving fails.");
-  @Deprecated
-  public static final String FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP = FAIL_ON_TIMELINE_ARCHIVING_ENABLED.key();
 
   public static final ConfigProperty<Long> INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = ConfigProperty
       .key("hoodie.consistency.check.initial_interval_ms")
       .defaultValue(2000L)
       .withDocumentation("Initial time between successive attempts to ensure written data's metadata is consistent on storage. Grows with exponential"
           + " backoff after the initial value.");
-  @Deprecated
-  public static final String INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP = INITIAL_CONSISTENCY_CHECK_INTERVAL_MS.key();
 
   public static final ConfigProperty<Long> MAX_CONSISTENCY_CHECK_INTERVAL_MS = ConfigProperty
       .key("hoodie.consistency.check.max_interval_ms")
       .defaultValue(300000L)
       .withDocumentation("Max time to wait between successive attempts at performing consistency checks");
-  @Deprecated
-  public static final String MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = MAX_CONSISTENCY_CHECK_INTERVAL_MS.key();
 
   public static final ConfigProperty<Integer> MAX_CONSISTENCY_CHECKS = ConfigProperty
       .key("hoodie.consistency.check.max_checks")
       .defaultValue(7)
       .withDocumentation("Maximum number of checks, for consistency of written data.");
-  @Deprecated
-  public static final String MAX_CONSISTENCY_CHECKS_PROP = MAX_CONSISTENCY_CHECKS.key();
 
-  public static final ConfigProperty<String> MERGE_DATA_VALIDATION_CHECK_ENABLED = ConfigProperty
+  public static final ConfigProperty<String> MERGE_DATA_VALIDATION_CHECK_ENABLE = ConfigProperty
       .key("hoodie.merge.data.validation.enabled")
       .defaultValue("false")
       .withDocumentation("When enabled, data validation checks are performed during merges to ensure expected "
           + "number of records after merge operation.");
 
-  public static final ConfigProperty<String> MERGE_ALLOW_DUPLICATE_ON_INSERTS = ConfigProperty
+  public static final ConfigProperty<String> MERGE_ALLOW_DUPLICATE_ON_INSERTS_ENABLE = ConfigProperty
       .key("hoodie.merge.allow.duplicate.on.inserts")
       .defaultValue("false")
       .withDocumentation("When enabled, we allow duplicate keys even if inserts are routed to merge with an existing file (for ensuring file sizing)."
@@ -364,33 +341,25 @@ public class HoodieWriteConfig extends HoodieConfig {
       .key("hoodie.client.heartbeat.interval_in_ms")
       .defaultValue(60 * 1000)
       .withDocumentation("Writers perform heartbeats to indicate liveness. Controls how often (in ms), such heartbeats are registered to lake storage.");
-  @Deprecated
-  public static final String CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP = CLIENT_HEARTBEAT_INTERVAL_IN_MS.key();
 
   public static final ConfigProperty<Integer> CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES = ConfigProperty
       .key("hoodie.client.heartbeat.tolerable.misses")
       .defaultValue(2)
       .withDocumentation("Number of heartbeat misses, before a writer is deemed not alive and all pending writes are aborted.");
-  @Deprecated
-  public static final String CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP = CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES.key();
 
-  public static final ConfigProperty<String> WRITE_CONCURRENCY_MOD = ConfigProperty
+  public static final ConfigProperty<String> WRITE_CONCURRENCY_MODE = ConfigProperty
       .key("hoodie.write.concurrency.mode")
       .defaultValue(WriteConcurrencyMode.SINGLE_WRITER.name())
       .withDocumentation("Enable different concurrency modes. Options are "
           + "SINGLE_WRITER: Only one active writer to the table. Maximizes throughput"
           + "OPTIMISTIC_CONCURRENCY_CONTROL: Multiple writers can operate on the table and exactly one of them succeed "
           + "if a conflict (writes affect the same file group) is detected.");
-  @Deprecated
-  public static final String WRITE_CONCURRENCY_MODE_PROP = WRITE_CONCURRENCY_MOD.key();
 
   public static final ConfigProperty<String> WRITE_META_KEY_PREFIXES = ConfigProperty
       .key("hoodie.write.meta.key.prefixes")
       .defaultValue("")
       .withDocumentation("Comma separated metadata key prefixes to override from latest commit "
           + "during overlapping commits via multi writing");
-  @Deprecated
-  public static final String WRITE_META_KEY_PREFIXES_PROP = WRITE_META_KEY_PREFIXES.key();
 
   /**
    * Currently the  use this to specify the write schema.
@@ -412,22 +381,22 @@ public class HoodieWriteConfig extends HoodieConfig {
    * Given the importance of supporting such cases for the user's migration to 0.5.x, we are proposing a safety flag
    * (disabled by default) which will allow this old behavior.
    */
-  public static final ConfigProperty<String> ALLOW_MULTI_WRITE_ON_SAME_INSTANT = ConfigProperty
+  public static final ConfigProperty<String> ALLOW_MULTI_WRITE_ON_SAME_INSTANT_ENABLE = ConfigProperty
       .key("_.hoodie.allow.multi.write.on.same.instant")
       .defaultValue("false")
       .withDocumentation("");
 
-  public static final ConfigProperty<String> EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION = ConfigProperty
-      .key(AVRO_SCHEMA.key() + ".external.transformation")
+  public static final ConfigProperty<String> AVRO_EXTERNAL_SCHEMA_TRANSFORMATION_ENABLE = ConfigProperty
+      .key(AVRO_SCHEMA_STRING.key() + ".external.transformation")
       .defaultValue("false")
-      .withAlternatives(AVRO_SCHEMA.key() + ".externalTransformation")
+      .withAlternatives(AVRO_SCHEMA_STRING.key() + ".externalTransformation")
       .withDocumentation("When enabled, records in older schema are rewritten into newer schema during upsert,delete and background"
           + " compaction,clustering operations.");
 
   public static final ConfigProperty<Boolean> ALLOW_EMPTY_COMMIT = ConfigProperty
-       .key("hoodie.allow.empty.commit")
-       .defaultValue(true)
-       .withDocumentation("Whether to allow generation of empty commits, even if no data was written in the commit. "
+      .key("hoodie.allow.empty.commit")
+      .defaultValue(true)
+      .withDocumentation("Whether to allow generation of empty commits, even if no data was written in the commit. "
           + "It's useful in cases where extra metadata needs to be published regardless e.g tracking source offsets when ingesting data");
 
   public static final ConfigProperty<Boolean> ALLOW_OPERATION_METADATA_FIELD = ConfigProperty
@@ -449,6 +418,384 @@ public class HoodieWriteConfig extends HoodieConfig {
   private EngineType engineType;
 
   /**
+   * @deprecated Use {@link #TBL_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String TABLE_NAME = TBL_NAME.key();
+  /**
+   * @deprecated Use {@link #PRECOMBINE_FIELD} and its methods instead
+   */
+  @Deprecated
+  public static final String PRECOMBINE_FIELD_PROP = PRECOMBINE_FIELD.key();
+  /**
+   * @deprecated Use {@link #WRITE_PAYLOAD_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String WRITE_PAYLOAD_CLASS = WRITE_PAYLOAD_CLASS_NAME.key();
+  /**
+   * @deprecated Use {@link #WRITE_PAYLOAD_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_WRITE_PAYLOAD_CLASS = WRITE_PAYLOAD_CLASS_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #KEYGENERATOR_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String KEYGENERATOR_CLASS_PROP = KEYGENERATOR_CLASS_NAME.key();
+  /**
+   * @deprecated Use {@link #KEYGENERATOR_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_KEYGENERATOR_CLASS = SimpleAvroKeyGenerator.class.getName();
+  /**
+   * @deprecated Use {@link #ROLLBACK_USING_MARKERS_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_ROLLBACK_USING_MARKERS = ROLLBACK_USING_MARKERS_ENABLE.defaultValue();
+  /**
+   * @deprecated Use {@link #ROLLBACK_USING_MARKERS_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String ROLLBACK_USING_MARKERS = ROLLBACK_USING_MARKERS_ENABLE.key();
+  /**
+   * @deprecated Use {@link #TIMELINE_LAYOUT_VERSION_NUM} and its methods instead
+   */
+  @Deprecated
+  public static final String TIMELINE_LAYOUT_VERSION = TIMELINE_LAYOUT_VERSION_NUM.key();
+  /**
+   * @deprecated Use {@link #BASE_PATH} and its methods instead
+   */
+  @Deprecated
+  public static final String BASE_PATH_PROP = BASE_PATH.key();
+  /**
+   * @deprecated Use {@link #AVRO_SCHEMA_STRING} and its methods instead
+   */
+  @Deprecated
+  public static final String AVRO_SCHEMA = AVRO_SCHEMA_STRING.key();
+  /**
+   * @deprecated Use {@link #AVRO_SCHEMA_VALIDATE_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String AVRO_SCHEMA_VALIDATE = AVRO_SCHEMA_VALIDATE_ENABLE.key();
+  /**
+   * @deprecated Use {@link #AVRO_SCHEMA_VALIDATE_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_AVRO_SCHEMA_VALIDATE = AVRO_SCHEMA_VALIDATE_ENABLE.defaultValue();
+  /**
+   * @deprecated Use {@link #INSERT_PARALLELISM_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_PARALLELISM = INSERT_PARALLELISM_VALUE.defaultValue();
+  /**
+   * @deprecated Use {@link #INSERT_PARALLELISM_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String INSERT_PARALLELISM = INSERT_PARALLELISM_VALUE.key();
+  /**
+   * @deprecated Use {@link #BULKINSERT_PARALLELISM_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String BULKINSERT_PARALLELISM = BULKINSERT_PARALLELISM_VALUE.key();
+  /**
+   * @deprecated Use {@link #BULKINSERT_USER_DEFINED_PARTITIONER_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String BULKINSERT_USER_DEFINED_PARTITIONER_CLASS = BULKINSERT_USER_DEFINED_PARTITIONER_CLASS_NAME.key();
+  @Deprecated
+  public static final String BULKINSERT_INPUT_DATA_SCHEMA_DDL = "hoodie.bulkinsert.schema.ddl";
+  /**
+   * @deprecated Use {@link #UPSERT_PARALLELISM_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String UPSERT_PARALLELISM = UPSERT_PARALLELISM_VALUE.key();
+  /**
+   * @deprecated Use {@link #DELETE_PARALLELISM_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String DELETE_PARALLELISM = DELETE_PARALLELISM_VALUE.key();
+  /**
+   * @deprecated Use {@link #ROLLBACK_PARALLELISM_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_ROLLBACK_PARALLELISM = ROLLBACK_PARALLELISM_VALUE.defaultValue();
+  /**
+   * @deprecated Use {@link #ROLLBACK_PARALLELISM_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String ROLLBACK_PARALLELISM = ROLLBACK_PARALLELISM_VALUE.key();
+  /**
+   * @deprecated Use {@link #WRITE_BUFFER_LIMIT_BYTES_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String WRITE_BUFFER_LIMIT_BYTES = WRITE_BUFFER_LIMIT_BYTES_VALUE.key();
+  /**
+   * @deprecated Use {@link #WRITE_BUFFER_LIMIT_BYTES_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_WRITE_BUFFER_LIMIT_BYTES = WRITE_BUFFER_LIMIT_BYTES_VALUE.defaultValue();
+  /**
+   * @deprecated Use {@link #COMBINE_BEFORE_INSERT} and its methods instead
+   */
+  @Deprecated
+  public static final String COMBINE_BEFORE_INSERT_PROP = COMBINE_BEFORE_INSERT.key();
+  /**
+   * @deprecated Use {@link #COMBINE_BEFORE_INSERT} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_COMBINE_BEFORE_INSERT = COMBINE_BEFORE_INSERT.defaultValue();
+  /**
+   * @deprecated Use {@link #COMBINE_BEFORE_UPSERT} and its methods instead
+   */
+  @Deprecated
+  public static final String COMBINE_BEFORE_UPSERT_PROP = COMBINE_BEFORE_UPSERT.key();
+  /**
+   * @deprecated Use {@link #COMBINE_BEFORE_UPSERT} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_COMBINE_BEFORE_UPSERT = COMBINE_BEFORE_UPSERT.defaultValue();
+  /**
+   * @deprecated Use {@link #COMBINE_BEFORE_DELETE} and its methods instead
+   */
+  @Deprecated
+  public static final String COMBINE_BEFORE_DELETE_PROP = COMBINE_BEFORE_DELETE.key();
+  /**
+   * @deprecated Use {@link #COMBINE_BEFORE_DELETE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_COMBINE_BEFORE_DELETE = COMBINE_BEFORE_DELETE.defaultValue();
+  /**
+   * @deprecated Use {@link #WRITE_STATUS_STORAGE_LEVEL_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String WRITE_STATUS_STORAGE_LEVEL = WRITE_STATUS_STORAGE_LEVEL_VALUE.key();
+  /**
+   * @deprecated Use {@link #WRITE_STATUS_STORAGE_LEVEL_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_WRITE_STATUS_STORAGE_LEVEL = WRITE_STATUS_STORAGE_LEVEL_VALUE.defaultValue();
+  /**
+   * @deprecated Use {@link #AUTO_COMMIT_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String HOODIE_AUTO_COMMIT_PROP = AUTO_COMMIT_ENABLE.key();
+  /**
+   * @deprecated Use {@link #AUTO_COMMIT_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_HOODIE_AUTO_COMMIT = AUTO_COMMIT_ENABLE.defaultValue();
+  /**
+   * @deprecated Use {@link #WRITE_STATUS_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String HOODIE_WRITE_STATUS_CLASS_PROP = WRITE_STATUS_CLASS_NAME.key();
+  /**
+   * @deprecated Use {@link #WRITE_STATUS_CLASS_NAME} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WRITE_STATUS_CLASS_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #FINALIZE_WRITE_PARALLELISM_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String FINALIZE_WRITE_PARALLELISM = FINALIZE_WRITE_PARALLELISM_VALUE.key();
+  /**
+   * @deprecated Use {@link #FINALIZE_WRITE_PARALLELISM_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = FINALIZE_WRITE_PARALLELISM_VALUE.defaultValue();
+  /**
+   * @deprecated Use {@link #MARKERS_DELETE_PARALLELISM_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String MARKERS_DELETE_PARALLELISM = MARKERS_DELETE_PARALLELISM_VALUE.key();
+  /**
+   * @deprecated Use {@link #MARKERS_DELETE_PARALLELISM_VALUE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_MARKERS_DELETE_PARALLELISM = MARKERS_DELETE_PARALLELISM_VALUE.defaultValue();
+  /**
+   * @deprecated Use {@link #BULK_INSERT_SORT_MODE} and its methods instead
+   */
+  @Deprecated
+  public static final String BULKINSERT_SORT_MODE = BULK_INSERT_SORT_MODE.key();
+  /**
+   * @deprecated Use {@link #BULK_INSERT_SORT_MODE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_BULKINSERT_SORT_MODE = BULK_INSERT_SORT_MODE.defaultValue();
+  /**
+   * @deprecated Use {@link #EMBEDDED_TIMELINE_SERVER_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String EMBEDDED_TIMELINE_SERVER_ENABLED = EMBEDDED_TIMELINE_SERVER_ENABLE.key();
+  /**
+   * @deprecated Use {@link #EMBEDDED_TIMELINE_SERVER_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_EMBEDDED_TIMELINE_SERVER_ENABLED = EMBEDDED_TIMELINE_SERVER_ENABLE.defaultValue();
+  /**
+   * @deprecated Use {@link #EMBEDDED_TIMELINE_SERVER_PORT_NUM} and its methods instead
+   */
+  @Deprecated
+  public static final String EMBEDDED_TIMELINE_SERVER_PORT = EMBEDDED_TIMELINE_SERVER_PORT_NUM.key();
+  /**
+   * @deprecated Use {@link #EMBEDDED_TIMELINE_SERVER_PORT_NUM} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_EMBEDDED_TIMELINE_SERVER_PORT = EMBEDDED_TIMELINE_SERVER_PORT_NUM.defaultValue();
+  /**
+   * @deprecated Use {@link #EMBEDDED_TIMELINE_NUM_SERVER_THREADS} and its methods instead
+   */
+  @Deprecated
+  public static final String EMBEDDED_TIMELINE_SERVER_THREADS = EMBEDDED_TIMELINE_NUM_SERVER_THREADS.key();
+  /**
+   * @deprecated Use {@link #EMBEDDED_TIMELINE_NUM_SERVER_THREADS} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_EMBEDDED_TIMELINE_SERVER_THREADS = EMBEDDED_TIMELINE_NUM_SERVER_THREADS.defaultValue();
+  /**
+   * @deprecated Use {@link #EMBEDDED_TIMELINE_SERVER_COMPRESS_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String EMBEDDED_TIMELINE_SERVER_COMPRESS_OUTPUT = EMBEDDED_TIMELINE_SERVER_COMPRESS_ENABLE.key();
+  /**
+   * @deprecated Use {@link #EMBEDDED_TIMELINE_SERVER_COMPRESS_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_EMBEDDED_TIMELINE_COMPRESS_OUTPUT = EMBEDDED_TIMELINE_SERVER_COMPRESS_ENABLE.defaultValue();
+  /**
+   * @deprecated Use {@link #EMBEDDED_TIMELINE_SERVER_USE_ASYNC_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String EMBEDDED_TIMELINE_SERVER_USE_ASYNC = EMBEDDED_TIMELINE_SERVER_USE_ASYNC_ENABLE.key();
+  /**
+   * @deprecated Use {@link #EMBEDDED_TIMELINE_SERVER_USE_ASYNC_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_EMBEDDED_TIMELINE_SERVER_ASYNC = EMBEDDED_TIMELINE_SERVER_USE_ASYNC_ENABLE.defaultValue();
+  /**
+   * @deprecated Use {@link #FAIL_ON_TIMELINE_ARCHIVING_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP = FAIL_ON_TIMELINE_ARCHIVING_ENABLE.key();
+  /**
+   * @deprecated Use {@link #FAIL_ON_TIMELINE_ARCHIVING_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_FAIL_ON_TIMELINE_ARCHIVING_ENABLED = FAIL_ON_TIMELINE_ARCHIVING_ENABLE.defaultValue();
+  /**
+   * @deprecated Use {@link #INITIAL_CONSISTENCY_CHECK_INTERVAL_MS} and its methods instead
+   */
+  @Deprecated
+  public static final String INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP = INITIAL_CONSISTENCY_CHECK_INTERVAL_MS.key();
+  /**
+   * @deprecated Use {@link #INITIAL_CONSISTENCY_CHECK_INTERVAL_MS} and its methods instead
+   */
+  @Deprecated
+  public static long DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = INITIAL_CONSISTENCY_CHECK_INTERVAL_MS.defaultValue();
+  /**
+   * @deprecated Use {@link #MAX_CONSISTENCY_CHECK_INTERVAL_MS} and its methods instead
+   */
+  @Deprecated
+  public static final String MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = MAX_CONSISTENCY_CHECK_INTERVAL_MS.key();
+  /**
+   * @deprecated Use {@link #MAX_CONSISTENCY_CHECK_INTERVAL_MS} and its methods instead
+   */
+  @Deprecated
+  public static long DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS = MAX_CONSISTENCY_CHECK_INTERVAL_MS.defaultValue();
+  /**
+   * @deprecated Use {@link #MAX_CONSISTENCY_CHECKS} and its methods instead
+   */
+  @Deprecated
+  public static final String MAX_CONSISTENCY_CHECKS_PROP = MAX_CONSISTENCY_CHECKS.key();
+  /**
+   * @deprecated Use {@link #MAX_CONSISTENCY_CHECKS} and its methods instead
+   */
+  @Deprecated
+  public static int DEFAULT_MAX_CONSISTENCY_CHECKS = MAX_CONSISTENCY_CHECKS.defaultValue();
+  /**
+   * @deprecated Use {@link #MERGE_DATA_VALIDATION_CHECK_ENABLE} and its methods instead
+   */
+  @Deprecated
+  private static final String MERGE_DATA_VALIDATION_CHECK_ENABLED = MERGE_DATA_VALIDATION_CHECK_ENABLE.key();
+  /**
+   * @deprecated Use {@link #MERGE_DATA_VALIDATION_CHECK_ENABLE} and its methods instead
+   */
+  @Deprecated
+  private static final String DEFAULT_MERGE_DATA_VALIDATION_CHECK_ENABLED = MERGE_DATA_VALIDATION_CHECK_ENABLE.defaultValue();
+  /**
+   * @deprecated Use {@link #MERGE_ALLOW_DUPLICATE_ON_INSERTS_ENABLE} and its methods instead
+   */
+  @Deprecated
+  private static final String MERGE_ALLOW_DUPLICATE_ON_INSERTS = MERGE_ALLOW_DUPLICATE_ON_INSERTS_ENABLE.key();
+  /**
+   * @deprecated Use {@link #MERGE_ALLOW_DUPLICATE_ON_INSERTS_ENABLE} and its methods instead
+   */
+  @Deprecated
+  private static final String DEFAULT_MERGE_ALLOW_DUPLICATE_ON_INSERTS = MERGE_ALLOW_DUPLICATE_ON_INSERTS_ENABLE.defaultValue();
+  /**
+   * @deprecated Use {@link #CLIENT_HEARTBEAT_INTERVAL_IN_MS} and its methods instead
+   */
+  @Deprecated
+  public static final String CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP = CLIENT_HEARTBEAT_INTERVAL_IN_MS.key();
+  /**
+   * @deprecated Use {@link #CLIENT_HEARTBEAT_INTERVAL_IN_MS} and its methods instead
+   */
+  @Deprecated
+  public static final Integer DEFAULT_CLIENT_HEARTBEAT_INTERVAL_IN_MS = CLIENT_HEARTBEAT_INTERVAL_IN_MS.defaultValue();
+  /**
+   * @deprecated Use {@link #CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES} and its methods instead
+   */
+  @Deprecated
+  public static final String CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP = CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES.key();
+  /**
+   * @deprecated Use {@link #CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES} and its methods instead
+   */
+  @Deprecated
+  public static final Integer DEFAULT_CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES = CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES.defaultValue();
+  /**
+   * @deprecated Use {@link #WRITE_CONCURRENCY_MODE} and its methods instead
+   */
+  @Deprecated
+  public static final String WRITE_CONCURRENCY_MODE_PROP = WRITE_CONCURRENCY_MODE.key();
+  /**
+   * @deprecated Use {@link #WRITE_CONCURRENCY_MODE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_WRITE_CONCURRENCY_MODE = WRITE_CONCURRENCY_MODE.defaultValue();
+  /**
+   * @deprecated Use {@link #WRITE_META_KEY_PREFIXES} and its methods instead
+   */
+  @Deprecated
+  public static final String WRITE_META_KEY_PREFIXES_PROP = WRITE_META_KEY_PREFIXES.key();
+  /**
+   * @deprecated Use {@link #WRITE_META_KEY_PREFIXES} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_WRITE_META_KEY_PREFIXES = WRITE_META_KEY_PREFIXES.defaultValue();
+  /**
+   * @deprecated Use {@link #ALLOW_MULTI_WRITE_ON_SAME_INSTANT_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String ALLOW_MULTI_WRITE_ON_SAME_INSTANT = ALLOW_MULTI_WRITE_ON_SAME_INSTANT_ENABLE.key();
+  /**
+   * @deprecated Use {@link #ALLOW_MULTI_WRITE_ON_SAME_INSTANT_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_ALLOW_MULTI_WRITE_ON_SAME_INSTANT = ALLOW_MULTI_WRITE_ON_SAME_INSTANT_ENABLE.defaultValue();
+  /**
+   * @deprecated Use {@link #AVRO_EXTERNAL_SCHEMA_TRANSFORMATION_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION = AVRO_EXTERNAL_SCHEMA_TRANSFORMATION_ENABLE.key();
+  /**
+   * @deprecated Use {@link #AVRO_EXTERNAL_SCHEMA_TRANSFORMATION_ENABLE} and its methods instead
+   */
+  @Deprecated
+  public static final String DEFAULT_EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION = AVRO_EXTERNAL_SCHEMA_TRANSFORMATION_ENABLE.defaultValue();
+
+  /**
    * Use Spark engine by default.
    */
   protected HoodieWriteConfig() {
@@ -482,11 +829,11 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public String getSchema() {
-    return getString(AVRO_SCHEMA);
+    return getString(AVRO_SCHEMA_STRING);
   }
 
   public void setSchema(String schemaStr) {
-    setValue(AVRO_SCHEMA, schemaStr);
+    setValue(AVRO_SCHEMA_STRING, schemaStr);
   }
 
   /**
@@ -504,11 +851,11 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public boolean getAvroSchemaValidate() {
-    return getBoolean(AVRO_SCHEMA_VALIDATE);
+    return getBoolean(AVRO_SCHEMA_VALIDATE_ENABLE);
   }
 
   public String getTableName() {
-    return getString(TABLE_NAME);
+    return getString(TBL_NAME);
   }
 
   public String getPreCombineField() {
@@ -516,15 +863,15 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public String getWritePayloadClass() {
-    return getString(WRITE_PAYLOAD_CLASS);
+    return getString(WRITE_PAYLOAD_CLASS_NAME);
   }
 
   public String getKeyGeneratorClass() {
-    return getString(KEYGENERATOR_CLASS);
+    return getString(KEYGENERATOR_CLASS_NAME);
   }
 
   public Boolean shouldAutoCommit() {
-    return getBoolean(HOODIE_AUTO_COMMIT);
+    return getBoolean(AUTO_COMMIT_ENABLE);
   }
 
   public Boolean shouldAssumeDatePartitioning() {
@@ -532,35 +879,35 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public boolean shouldUseExternalSchemaTransformation() {
-    return getBoolean(EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION);
+    return getBoolean(AVRO_EXTERNAL_SCHEMA_TRANSFORMATION_ENABLE);
   }
 
   public Integer getTimelineLayoutVersion() {
-    return getInt(TIMELINE_LAYOUT_VERSION);
+    return getInt(TIMELINE_LAYOUT_VERSION_NUM);
   }
 
   public int getBulkInsertShuffleParallelism() {
-    return getInt(BULKINSERT_PARALLELISM);
+    return getInt(BULKINSERT_PARALLELISM_VALUE);
   }
 
   public String getUserDefinedBulkInsertPartitionerClass() {
-    return getString(BULKINSERT_USER_DEFINED_PARTITIONER_CLASS);
+    return getString(BULKINSERT_USER_DEFINED_PARTITIONER_CLASS_NAME);
   }
 
   public int getInsertShuffleParallelism() {
-    return getInt(INSERT_PARALLELISM);
+    return getInt(INSERT_PARALLELISM_VALUE);
   }
 
   public int getUpsertShuffleParallelism() {
-    return getInt(UPSERT_PARALLELISM);
+    return getInt(UPSERT_PARALLELISM_VALUE);
   }
 
   public int getDeleteShuffleParallelism() {
-    return Math.max(getInt(DELETE_PARALLELISM), 1);
+    return Math.max(getInt(DELETE_PARALLELISM_VALUE), 1);
   }
 
   public int getRollbackParallelism() {
-    return getInt(ROLLBACK_PARALLELISM);
+    return getInt(ROLLBACK_PARALLELISM_VALUE);
   }
 
   public int getFileListingParallelism() {
@@ -568,11 +915,11 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public boolean shouldRollbackUsingMarkers() {
-    return getBoolean(ROLLBACK_USING_MARKERS);
+    return getBoolean(ROLLBACK_USING_MARKERS_ENABLE);
   }
 
   public int getWriteBufferLimitBytes() {
-    return Integer.parseInt(getStringOrDefault(WRITE_BUFFER_LIMIT_BYTES));
+    return Integer.parseInt(getStringOrDefault(WRITE_BUFFER_LIMIT_BYTES_VALUE));
   }
 
   public boolean shouldCombineBeforeInsert() {
@@ -588,15 +935,15 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public boolean shouldAllowMultiWriteOnSameInstant() {
-    return getBoolean(ALLOW_MULTI_WRITE_ON_SAME_INSTANT);
+    return getBoolean(ALLOW_MULTI_WRITE_ON_SAME_INSTANT_ENABLE);
   }
 
   public String getWriteStatusClassName() {
-    return getString(HOODIE_WRITE_STATUS_CLASS);
+    return getString(WRITE_STATUS_CLASS_NAME);
   }
 
   public int getFinalizeWriteParallelism() {
-    return getInt(FINALIZE_WRITE_PARALLELISM);
+    return getInt(FINALIZE_WRITE_PARALLELISM_VALUE);
   }
 
   public MarkerType getMarkersType() {
@@ -613,11 +960,11 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public int getMarkersDeleteParallelism() {
-    return getInt(MARKERS_DELETE_PARALLELISM);
+    return getInt(MARKERS_DELETE_PARALLELISM_VALUE);
   }
 
   public boolean isEmbeddedTimelineServerEnabled() {
-    return getBoolean(EMBEDDED_TIMELINE_SERVER_ENABLED);
+    return getBoolean(EMBEDDED_TIMELINE_SERVER_ENABLE);
   }
 
   public boolean isEmbeddedTimelineServerReuseEnabled() {
@@ -625,23 +972,23 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public int getEmbeddedTimelineServerPort() {
-    return Integer.parseInt(getStringOrDefault(EMBEDDED_TIMELINE_SERVER_PORT));
+    return Integer.parseInt(getStringOrDefault(EMBEDDED_TIMELINE_SERVER_PORT_NUM));
   }
 
   public int getEmbeddedTimelineServerThreads() {
-    return Integer.parseInt(getStringOrDefault(EMBEDDED_TIMELINE_SERVER_THREADS));
+    return Integer.parseInt(getStringOrDefault(EMBEDDED_TIMELINE_NUM_SERVER_THREADS));
   }
 
   public boolean getEmbeddedTimelineServerCompressOutput() {
-    return Boolean.parseBoolean(getStringOrDefault(EMBEDDED_TIMELINE_SERVER_COMPRESS_OUTPUT));
+    return Boolean.parseBoolean(getStringOrDefault(EMBEDDED_TIMELINE_SERVER_COMPRESS_ENABLE));
   }
 
   public boolean getEmbeddedTimelineServerUseAsync() {
-    return Boolean.parseBoolean(getStringOrDefault(EMBEDDED_TIMELINE_SERVER_USE_ASYNC));
+    return Boolean.parseBoolean(getStringOrDefault(EMBEDDED_TIMELINE_SERVER_USE_ASYNC_ENABLE));
   }
 
   public boolean isFailOnTimelineArchivingEnabled() {
-    return getBoolean(FAIL_ON_TIMELINE_ARCHIVING_ENABLED);
+    return getBoolean(FAIL_ON_TIMELINE_ARCHIVING_ENABLE);
   }
 
   public int getMaxConsistencyChecks() {
@@ -657,16 +1004,16 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public BulkInsertSortMode getBulkInsertSortMode() {
-    String sortMode = getString(BULKINSERT_SORT_MODE);
+    String sortMode = getString(BULK_INSERT_SORT_MODE);
     return BulkInsertSortMode.valueOf(sortMode.toUpperCase());
   }
 
   public boolean isMergeDataValidationCheckEnabled() {
-    return getBoolean(MERGE_DATA_VALIDATION_CHECK_ENABLED);
+    return getBoolean(MERGE_DATA_VALIDATION_CHECK_ENABLE);
   }
 
   public boolean allowDuplicateInserts() {
-    return getBoolean(MERGE_ALLOW_DUPLICATE_ON_INSERTS);
+    return getBoolean(MERGE_ALLOW_DUPLICATE_ON_INSERTS_ENABLE);
   }
 
   public EngineType getEngineType() {
@@ -674,8 +1021,8 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public boolean populateMetaFields() {
-    return Boolean.parseBoolean(getStringOrDefault(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS,
-        HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()));
+    return Boolean.parseBoolean(getStringOrDefault(HoodieTableConfig.POPULATE_META_FIELDS,
+        HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()));
   }
 
   /**
@@ -702,7 +1049,7 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public int getParquetSmallFileLimit() {
-    return getInt(HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT_BYTES);
+    return getInt(HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT);
   }
 
   public double getRecordSizeEstimationThreshold() {
@@ -710,19 +1057,19 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public int getCopyOnWriteInsertSplitSize() {
-    return getInt(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE);
+    return getInt(HoodieCompactionConfig.COPY_ON_WRITE_INSERT_SPLIT_SIZE);
   }
 
   public int getCopyOnWriteRecordSizeEstimate() {
-    return getInt(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE);
+    return getInt(HoodieCompactionConfig.COPY_ON_WRITE_RECORD_SIZE_ESTIMATE);
   }
 
   public boolean shouldAutoTuneInsertSplits() {
-    return getBoolean(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS);
+    return getBoolean(HoodieCompactionConfig.COPY_ON_WRITE_AUTO_SPLIT_INSERTS);
   }
 
   public int getCleanerParallelism() {
-    return getInt(HoodieCompactionConfig.CLEANER_PARALLELISM);
+    return getInt(HoodieCompactionConfig.CLEANER_PARALLELISM_VALUE);
   }
 
   public boolean isAutoClean() {
@@ -734,7 +1081,7 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public boolean incrementalCleanerModeEnabled() {
-    return getBoolean(HoodieCompactionConfig.CLEANER_INCREMENTAL_MODE);
+    return getBoolean(HoodieCompactionConfig.CLEANER_INCREMENTAL_MODE_ENABLE);
   }
 
   public boolean inlineCompactionEnabled() {
@@ -762,11 +1109,11 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public Boolean getCompactionLazyBlockReadEnabled() {
-    return getBoolean(HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED);
+    return getBoolean(HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLE);
   }
 
   public Boolean getCompactionReverseLogReadEnabled() {
-    return getBoolean(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED);
+    return getBoolean(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLE);
   }
 
   public boolean inlineClusteringEnabled() {
@@ -778,7 +1125,7 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public boolean isPreserveHoodieCommitMetadata() {
-    return getBoolean(HoodieClusteringConfig.CLUSTERING_PRESERVE_HOODIE_COMMIT_METADATA);
+    return getBoolean(HoodieClusteringConfig.PRESERVE_COMMIT_METADATA);
   }
 
   public boolean isClusteringEnabled() {
@@ -787,15 +1134,15 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public int getInlineClusterMaxCommits() {
-    return getInt(HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT);
+    return getInt(HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMITS);
   }
 
   public int getAsyncClusterMaxCommits() {
-    return getInt(HoodieClusteringConfig.ASYNC_CLUSTERING_MAX_COMMIT_PROP);
+    return getInt(HoodieClusteringConfig.ASYNC_CLUSTERING_MAX_COMMITS);
   }
 
   public String getPayloadClass() {
-    return getString(HoodieCompactionConfig.PAYLOAD_CLASS);
+    return getString(HoodieCompactionConfig.PAYLOAD_CLASS_NAME);
   }
 
   public int getTargetPartitionsPerDayBasedCompaction() {
@@ -807,11 +1154,11 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public Boolean shouldCleanBootstrapBaseFile() {
-    return getBoolean(HoodieCompactionConfig.CLEANER_BOOTSTRAP_BASE_FILE_ENABLED);
+    return getBoolean(HoodieCompactionConfig.CLEANER_BOOTSTRAP_BASE_FILE_ENABLE);
   }
 
   public String getClusteringUpdatesStrategyClass() {
-    return getString(HoodieClusteringConfig.CLUSTERING_UPDATES_STRATEGY);
+    return getString(HoodieClusteringConfig.UPDATES_STRATEGY);
   }
 
   public HoodieFailedWritesCleaningPolicy getFailedWritesCleanPolicy() {
@@ -823,39 +1170,39 @@ public class HoodieWriteConfig extends HoodieConfig {
    * Clustering properties.
    */
   public String getClusteringPlanStrategyClass() {
-    return getString(HoodieClusteringConfig.CLUSTERING_PLAN_STRATEGY_CLASS);
+    return getString(HoodieClusteringConfig.PLAN_STRATEGY_CLASS_NAME);
   }
 
   public String getClusteringExecutionStrategyClass() {
-    return getString(HoodieClusteringConfig.CLUSTERING_EXECUTION_STRATEGY_CLASS);
+    return getString(HoodieClusteringConfig.EXECUTION_STRATEGY_CLASS_NAME);
   }
 
   public long getClusteringMaxBytesInGroup() {
-    return getLong(HoodieClusteringConfig.CLUSTERING_MAX_BYTES_PER_GROUP);
+    return getLong(HoodieClusteringConfig.PLAN_STRATEGY_MAX_BYTES_PER_OUTPUT_FILEGROUP);
   }
 
   public long getClusteringSmallFileLimit() {
-    return getLong(HoodieClusteringConfig.CLUSTERING_PLAN_SMALL_FILE_LIMIT);
+    return getLong(HoodieClusteringConfig.PLAN_STRATEGY_SMALL_FILE_LIMIT);
   }
 
   public int getClusteringMaxNumGroups() {
-    return getInt(HoodieClusteringConfig.CLUSTERING_MAX_NUM_GROUPS);
+    return getInt(HoodieClusteringConfig.PLAN_STRATEGY_MAX_GROUPS);
   }
 
   public long getClusteringTargetFileMaxBytes() {
-    return getLong(HoodieClusteringConfig.CLUSTERING_TARGET_FILE_MAX_BYTES);
+    return getLong(HoodieClusteringConfig.PLAN_STRATEGY_TARGET_FILE_MAX_BYTES);
   }
 
   public int getTargetPartitionsForClustering() {
-    return getInt(HoodieClusteringConfig.CLUSTERING_TARGET_PARTITIONS);
+    return getInt(HoodieClusteringConfig.DAYBASED_LOOKBACK_PARTITIONS);
   }
 
   public int getSkipPartitionsFromLatestForClustering() {
-    return getInt(HoodieClusteringConfig.CLUSTERING_SKIP_PARTITIONS_FROM_LATEST);
+    return getInt(HoodieClusteringConfig.PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST);
   }
 
   public String getClusteringSortColumns() {
-    return getString(HoodieClusteringConfig.CLUSTERING_SORT_COLUMNS_PROPERTY);
+    return getString(HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS);
   }
 
   /**
@@ -866,79 +1213,79 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public String getIndexClass() {
-    return getString(HoodieIndexConfig.INDEX_CLASS);
+    return getString(HoodieIndexConfig.INDEX_CLASS_NAME);
   }
 
   public int getBloomFilterNumEntries() {
-    return getInt(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES);
+    return getInt(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES_VALUE);
   }
 
   public double getBloomFilterFPP() {
-    return getDouble(HoodieIndexConfig.BLOOM_FILTER_FPP);
+    return getDouble(HoodieIndexConfig.BLOOM_FILTER_FPP_VALUE);
   }
 
   public String getHbaseZkQuorum() {
-    return getString(HoodieHBaseIndexConfig.HBASE_ZKQUORUM);
+    return getString(HoodieHBaseIndexConfig.ZKQUORUM);
   }
 
   public int getHbaseZkPort() {
-    return getInt(HoodieHBaseIndexConfig.HBASE_ZKPORT);
+    return getInt(HoodieHBaseIndexConfig.ZKPORT);
   }
 
   public String getHBaseZkZnodeParent() {
-    return getString(HoodieHBaseIndexConfig.HBASE_ZK_ZNODEPARENT);
+    return getString(HoodieHBaseIndexConfig.ZK_NODE_PATH);
   }
 
   public String getHbaseTableName() {
-    return getString(HoodieHBaseIndexConfig.HBASE_TABLENAME);
+    return getString(HoodieHBaseIndexConfig.TABLENAME);
   }
 
   public int getHbaseIndexGetBatchSize() {
-    return getInt(HoodieHBaseIndexConfig.HBASE_GET_BATCH_SIZE);
+    return getInt(HoodieHBaseIndexConfig.GET_BATCH_SIZE);
   }
 
   public Boolean getHBaseIndexRollbackSync() {
-    return getBoolean(HoodieHBaseIndexConfig.HBASE_INDEX_ROLLBACK_SYNC);
+    return getBoolean(HoodieHBaseIndexConfig.ROLLBACK_SYNC_ENABLE);
   }
 
   public int getHbaseIndexPutBatchSize() {
-    return getInt(HoodieHBaseIndexConfig.HBASE_PUT_BATCH_SIZE);
+    return getInt(HoodieHBaseIndexConfig.PUT_BATCH_SIZE);
   }
 
   public Boolean getHbaseIndexPutBatchSizeAutoCompute() {
-    return getBoolean(HoodieHBaseIndexConfig.HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE);
+    return getBoolean(HoodieHBaseIndexConfig.PUT_BATCH_SIZE_AUTO_COMPUTE);
   }
 
   public String getHBaseQPSResourceAllocatorClass() {
-    return getString(HoodieHBaseIndexConfig.HBASE_INDEX_QPS_ALLOCATOR_CLASS);
+    return getString(HoodieHBaseIndexConfig.QPS_ALLOCATOR_CLASS_NAME);
   }
 
   public String getHBaseQPSZKnodePath() {
-    return getString(HoodieHBaseIndexConfig.HBASE_ZK_PATH_QPS_ROOT);
+    return getString(HoodieHBaseIndexConfig.ZKPATH_QPS_ROOT);
   }
 
   public String getHBaseZkZnodeSessionTimeout() {
-    return getString(HoodieHBaseIndexConfig.HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS);
+    return getString(HoodieHBaseIndexConfig.ZK_SESSION_TIMEOUT_MS);
   }
 
   public String getHBaseZkZnodeConnectionTimeout() {
-    return getString(HoodieHBaseIndexConfig.HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS);
+    return getString(HoodieHBaseIndexConfig.ZK_CONNECTION_TIMEOUT_MS);
   }
 
   public boolean getHBaseIndexShouldComputeQPSDynamically() {
-    return getBoolean(HoodieHBaseIndexConfig.HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY);
+    return getBoolean(HoodieHBaseIndexConfig.COMPUTE_QPS_DYNAMICALLY);
   }
 
   public int getHBaseIndexDesiredPutsTime() {
-    return getInt(HoodieHBaseIndexConfig.HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS);
+    return getInt(HoodieHBaseIndexConfig.DESIRED_PUTS_TIME_IN_SECONDS);
   }
 
   public String getBloomFilterType() {
-    return getString(HoodieIndexConfig.BLOOM_INDEX_FILTER_TYPE);
+    return getString(HoodieIndexConfig.BLOOM_FILTER_TYPE);
   }
 
   public int getDynamicBloomFilterMaxNumEntries() {
-    return getInt(HoodieIndexConfig.HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES);
+    return getInt(HoodieIndexConfig.BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES);
   }
 
   /**
@@ -947,15 +1294,15 @@ public class HoodieWriteConfig extends HoodieConfig {
    * the jobs would be (0.17) 1/6, 0.33 (2/6) and 0.5 (3/6) respectively.
    */
   public float getHbaseIndexQPSFraction() {
-    return getFloat(HoodieHBaseIndexConfig.HBASE_QPS_FRACTION);
+    return getFloat(HoodieHBaseIndexConfig.QPS_FRACTION);
   }
 
   public float getHBaseIndexMinQPSFraction() {
-    return getFloat(HoodieHBaseIndexConfig.HBASE_MIN_QPS_FRACTION);
+    return getFloat(HoodieHBaseIndexConfig.MIN_QPS_FRACTION);
   }
 
   public float getHBaseIndexMaxQPSFraction() {
-    return getFloat(HoodieHBaseIndexConfig.HBASE_MAX_QPS_FRACTION);
+    return getFloat(HoodieHBaseIndexConfig.MAX_QPS_FRACTION);
   }
 
   /**
@@ -963,11 +1310,11 @@ public class HoodieWriteConfig extends HoodieConfig {
    * Hoodie jobs to an Hbase Region Server
    */
   public int getHbaseIndexMaxQPSPerRegionServer() {
-    return getInt(HoodieHBaseIndexConfig.HBASE_MAX_QPS_PER_REGION_SERVER);
+    return getInt(HoodieHBaseIndexConfig.MAX_QPS_PER_REGION_SERVER);
   }
 
   public boolean getHbaseIndexUpdatePartitionPath() {
-    return getBoolean(HoodieHBaseIndexConfig.HBASE_INDEX_UPDATE_PARTITION_PATH);
+    return getBoolean(HoodieHBaseIndexConfig.UPDATE_PARTITION_PATH_ENABLE);
   }
 
   public int getBloomIndexParallelism() {
@@ -995,7 +1342,7 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public boolean getBloomIndexUpdatePartitionPath() {
-    return getBoolean(HoodieIndexConfig.BLOOM_INDEX_UPDATE_PARTITION_PATH);
+    return getBoolean(HoodieIndexConfig.BLOOM_INDEX_UPDATE_PARTITION_PATH_ENABLE);
   }
 
   public int getSimpleIndexParallelism() {
@@ -1011,58 +1358,58 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public boolean getGlobalSimpleIndexUpdatePartitionPath() {
-    return getBoolean(HoodieIndexConfig.SIMPLE_INDEX_UPDATE_PARTITION_PATH);
+    return getBoolean(HoodieIndexConfig.SIMPLE_INDEX_UPDATE_PARTITION_PATH_ENABLE);
   }
 
   /**
    * storage properties.
    */
   public long getParquetMaxFileSize() {
-    return getLong(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES);
+    return getLong(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE);
   }
 
   public int getParquetBlockSize() {
-    return getInt(HoodieStorageConfig.PARQUET_BLOCK_SIZE_BYTES);
+    return getInt(HoodieStorageConfig.PARQUET_BLOCK_SIZE);
   }
 
   public int getParquetPageSize() {
-    return getInt(HoodieStorageConfig.PARQUET_PAGE_SIZE_BYTES);
+    return getInt(HoodieStorageConfig.PARQUET_PAGE_SIZE);
   }
 
   public int getLogFileDataBlockMaxSize() {
-    return getInt(HoodieStorageConfig.LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES);
+    return getInt(HoodieStorageConfig.LOGFILE_DATA_BLOCK_MAX_SIZE);
   }
 
   public int getLogFileMaxSize() {
-    return getInt(HoodieStorageConfig.LOGFILE_SIZE_MAX_BYTES);
+    return getInt(HoodieStorageConfig.LOGFILE_MAX_SIZE);
   }
 
   public double getParquetCompressionRatio() {
-    return getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO);
+    return getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION);
   }
 
   public CompressionCodecName getParquetCompressionCodec() {
-    return CompressionCodecName.fromConf(getString(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC));
+    return CompressionCodecName.fromConf(getString(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME));
   }
 
   public double getLogFileToParquetCompressionRatio() {
-    return getDouble(HoodieStorageConfig.LOGFILE_TO_PARQUET_COMPRESSION_RATIO);
+    return getDouble(HoodieStorageConfig.LOGFILE_TO_PARQUET_COMPRESSION_RATIO_FRACTION);
   }
 
   public long getHFileMaxFileSize() {
-    return getLong(HoodieStorageConfig.HFILE_FILE_MAX_BYTES);
+    return getLong(HoodieStorageConfig.HFILE_MAX_FILE_SIZE);
   }
 
   public int getHFileBlockSize() {
-    return getInt(HoodieStorageConfig.HFILE_BLOCK_SIZE_BYTES);
+    return getInt(HoodieStorageConfig.HFILE_BLOCK_SIZE);
   }
 
   public Compression.Algorithm getHFileCompressionAlgorithm() {
-    return Compression.Algorithm.valueOf(getString(HoodieStorageConfig.HFILE_COMPRESSION_ALGORITHM));
+    return Compression.Algorithm.valueOf(getString(HoodieStorageConfig.HFILE_COMPRESSION_ALGORITHM_NAME));
   }
 
   public long getOrcMaxFileSize() {
-    return getLong(HoodieStorageConfig.ORC_FILE_MAX_BYTES);
+    return getLong(HoodieStorageConfig.ORC_FILE_MAX_SIZE);
   }
 
   public int getOrcStripeSize() {
@@ -1074,114 +1421,114 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public CompressionKind getOrcCompressionCodec() {
-    return CompressionKind.valueOf(getString(HoodieStorageConfig.ORC_COMPRESSION_CODEC));
+    return CompressionKind.valueOf(getString(HoodieStorageConfig.ORC_COMPRESSION_CODEC_NAME));
   }
 
   /**
    * metrics properties.
    */
   public boolean isMetricsOn() {
-    return getBoolean(HoodieMetricsConfig.METRICS_ON);
+    return getBoolean(HoodieMetricsConfig.TURN_METRICS_ON);
   }
 
   public boolean isExecutorMetricsEnabled() {
     return Boolean.parseBoolean(
-        getStringOrDefault(HoodieMetricsConfig.ENABLE_EXECUTOR_METRICS, "false"));
+        getStringOrDefault(HoodieMetricsConfig.EXECUTOR_METRICS_ENABLE, "false"));
   }
 
   public MetricsReporterType getMetricsReporterType() {
-    return MetricsReporterType.valueOf(getString(HoodieMetricsConfig.METRICS_REPORTER_TYPE));
+    return MetricsReporterType.valueOf(getString(HoodieMetricsConfig.METRICS_REPORTER_TYPE_VALUE));
   }
 
   public String getGraphiteServerHost() {
-    return getString(HoodieMetricsConfig.GRAPHITE_SERVER_HOST);
+    return getString(HoodieMetricsConfig.GRAPHITE_SERVER_HOST_NAME);
   }
 
   public int getGraphiteServerPort() {
-    return getInt(HoodieMetricsConfig.GRAPHITE_SERVER_PORT);
+    return getInt(HoodieMetricsConfig.GRAPHITE_SERVER_PORT_NUM);
   }
 
   public String getGraphiteMetricPrefix() {
-    return getString(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX);
+    return getString(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX_VALUE);
   }
 
   public String getJmxHost() {
-    return getString(HoodieMetricsConfig.JMX_HOST);
+    return getString(HoodieMetricsConfig.JMX_HOST_NAME);
   }
 
   public String getJmxPort() {
-    return getString(HoodieMetricsConfig.JMX_PORT);
+    return getString(HoodieMetricsConfig.JMX_PORT_NUM);
   }
 
   public int getDatadogReportPeriodSeconds() {
-    return getInt(HoodieMetricsDatadogConfig.DATADOG_REPORT_PERIOD_SECONDS);
+    return getInt(HoodieMetricsDatadogConfig.REPORT_PERIOD_IN_SECONDS);
   }
 
   public ApiSite getDatadogApiSite() {
-    return ApiSite.valueOf(getString(HoodieMetricsDatadogConfig.DATADOG_API_SITE));
+    return ApiSite.valueOf(getString(HoodieMetricsDatadogConfig.API_SITE_VALUE));
   }
 
   public String getDatadogApiKey() {
-    if (props.containsKey(HoodieMetricsDatadogConfig.DATADOG_API_KEY.key())) {
-      return getString(HoodieMetricsDatadogConfig.DATADOG_API_KEY);
+    if (props.containsKey(HoodieMetricsDatadogConfig.API_KEY.key())) {
+      return getString(HoodieMetricsDatadogConfig.API_KEY);
     } else {
       Supplier<String> apiKeySupplier = ReflectionUtils.loadClass(
-          getString(HoodieMetricsDatadogConfig.DATADOG_API_KEY_SUPPLIER));
+          getString(HoodieMetricsDatadogConfig.API_KEY_SUPPLIER));
       return apiKeySupplier.get();
     }
   }
 
   public boolean getDatadogApiKeySkipValidation() {
-    return getBoolean(HoodieMetricsDatadogConfig.DATADOG_API_KEY_SKIP_VALIDATION);
+    return getBoolean(HoodieMetricsDatadogConfig.API_KEY_SKIP_VALIDATION);
   }
 
   public int getDatadogApiTimeoutSeconds() {
-    return getInt(HoodieMetricsDatadogConfig.DATADOG_API_TIMEOUT_SECONDS);
+    return getInt(HoodieMetricsDatadogConfig.API_TIMEOUT_IN_SECONDS);
   }
 
   public String getDatadogMetricPrefix() {
-    return getString(HoodieMetricsDatadogConfig.DATADOG_METRIC_PREFIX);
+    return getString(HoodieMetricsDatadogConfig.METRIC_PREFIX_VALUE);
   }
 
   public String getDatadogMetricHost() {
-    return getString(HoodieMetricsDatadogConfig.DATADOG_METRIC_HOST);
+    return getString(HoodieMetricsDatadogConfig.METRIC_HOST_NAME);
   }
 
   public List<String> getDatadogMetricTags() {
     return Arrays.stream(getStringOrDefault(
-        HoodieMetricsDatadogConfig.DATADOG_METRIC_TAGS, ",").split("\\s*,\\s*")).collect(Collectors.toList());
+        HoodieMetricsDatadogConfig.METRIC_TAG_VALUES, ",").split("\\s*,\\s*")).collect(Collectors.toList());
   }
 
   public String getMetricReporterClassName() {
-    return getString(HoodieMetricsConfig.METRICS_REPORTER_CLASS);
+    return getString(HoodieMetricsConfig.METRICS_REPORTER_CLASS_NAME);
   }
 
   public int getPrometheusPort() {
-    return getInt(HoodieMetricsPrometheusConfig.PROMETHEUS_PORT);
+    return getInt(HoodieMetricsPrometheusConfig.PROMETHEUS_PORT_NUM);
   }
 
   public String getPushGatewayHost() {
-    return getString(HoodieMetricsPrometheusConfig.PUSHGATEWAY_HOST);
+    return getString(HoodieMetricsPrometheusConfig.PUSHGATEWAY_HOST_NAME);
   }
 
   public int getPushGatewayPort() {
-    return getInt(HoodieMetricsPrometheusConfig.PUSHGATEWAY_PORT);
+    return getInt(HoodieMetricsPrometheusConfig.PUSHGATEWAY_PORT_NUM);
   }
 
   public int getPushGatewayReportPeriodSeconds() {
-    return getInt(HoodieMetricsPrometheusConfig.PUSHGATEWAY_REPORT_PERIOD_SECONDS);
+    return getInt(HoodieMetricsPrometheusConfig.PUSHGATEWAY_REPORT_PERIOD_IN_SECONDS);
   }
 
   public boolean getPushGatewayDeleteOnShutdown() {
-    return getBoolean(HoodieMetricsPrometheusConfig.PUSHGATEWAY_DELETE_ON_SHUTDOWN);
+    return getBoolean(HoodieMetricsPrometheusConfig.PUSHGATEWAY_DELETE_ON_SHUTDOWN_ENABLE);
   }
 
   public String getPushGatewayJobName() {
-    return getString(HoodieMetricsPrometheusConfig.PUSHGATEWAY_JOB_NAME);
+    return getString(HoodieMetricsPrometheusConfig.PUSHGATEWAY_JOBNAME);
   }
 
   public boolean getPushGatewayRandomJobNameSuffix() {
-    return getBoolean(HoodieMetricsPrometheusConfig.PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX);
+    return getBoolean(HoodieMetricsPrometheusConfig.PUSHGATEWAY_RANDOM_JOBNAME_SUFFIX);
   }
 
   /**
@@ -1239,47 +1586,47 @@ public class HoodieWriteConfig extends HoodieConfig {
    * Commit call back configs.
    */
   public boolean writeCommitCallbackOn() {
-    return getBoolean(HoodieWriteCommitCallbackConfig.CALLBACK_ON);
+    return getBoolean(HoodieWriteCommitCallbackConfig.TURN_CALLBACK_ON);
   }
 
   public String getCallbackClass() {
-    return getString(HoodieWriteCommitCallbackConfig.CALLBACK_CLASS);
+    return getString(HoodieWriteCommitCallbackConfig.CALLBACK_CLASS_NAME);
   }
 
   public String getBootstrapSourceBasePath() {
-    return getString(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH);
+    return getString(HoodieBootstrapConfig.BASE_PATH);
   }
 
   public String getBootstrapModeSelectorClass() {
-    return getString(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR);
+    return getString(HoodieBootstrapConfig.MODE_SELECTOR_CLASS_NAME);
   }
 
   public String getFullBootstrapInputProvider() {
-    return getString(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER);
+    return getString(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER_CLASS_NAME);
   }
 
   public String getBootstrapKeyGeneratorClass() {
-    return getString(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS);
+    return getString(HoodieBootstrapConfig.KEYGEN_CLASS_NAME);
   }
 
   public String getBootstrapKeyGeneratorType() {
-    return getString(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_TYPE);
+    return getString(HoodieBootstrapConfig.KEYGEN_TYPE);
   }
 
   public String getBootstrapModeSelectorRegex() {
-    return getString(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR_REGEX);
+    return getString(HoodieBootstrapConfig.PARTITION_SELECTOR_REGEX_PATTERN);
   }
 
   public BootstrapMode getBootstrapModeForRegexMatch() {
-    return BootstrapMode.valueOf(getString(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR_REGEX_MODE));
+    return BootstrapMode.valueOf(getString(HoodieBootstrapConfig.PARTITION_SELECTOR_REGEX_MODE));
   }
 
   public String getBootstrapPartitionPathTranslatorClass() {
-    return getString(HoodieBootstrapConfig.BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS);
+    return getString(HoodieBootstrapConfig.PARTITION_PATH_TRANSLATOR_CLASS_NAME);
   }
 
   public int getBootstrapParallelism() {
-    return getInt(HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM);
+    return getInt(HoodieBootstrapConfig.PARALLELISM_VALUE);
   }
 
   public Long getMaxMemoryPerPartitionMerge() {
@@ -1306,27 +1653,27 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public int getMetadataInsertParallelism() {
-    return getInt(HoodieMetadataConfig.METADATA_INSERT_PARALLELISM_PROP);
+    return getInt(HoodieMetadataConfig.INSERT_PARALLELISM_VALUE);
   }
 
   public int getMetadataCompactDeltaCommitMax() {
-    return getInt(HoodieMetadataConfig.METADATA_COMPACT_NUM_DELTA_COMMITS_PROP);
+    return getInt(HoodieMetadataConfig.COMPACT_NUM_DELTA_COMMITS);
   }
 
   public boolean isMetadataAsyncClean() {
-    return getBoolean(HoodieMetadataConfig.METADATA_ASYNC_CLEAN_PROP);
+    return getBoolean(HoodieMetadataConfig.ASYNC_CLEAN_ENABLE);
   }
 
   public int getMetadataMaxCommitsToKeep() {
-    return getInt(HoodieMetadataConfig.MAX_COMMITS_TO_KEEP_PROP);
+    return getInt(HoodieMetadataConfig.MAX_COMMITS_TO_KEEP);
   }
 
   public int getMetadataMinCommitsToKeep() {
-    return getInt(HoodieMetadataConfig.MIN_COMMITS_TO_KEEP_PROP);
+    return getInt(HoodieMetadataConfig.MIN_COMMITS_TO_KEEP);
   }
 
   public int getMetadataCleanerCommitsRetained() {
-    return getInt(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED_PROP);
+    return getInt(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED);
   }
 
   /**
@@ -1335,7 +1682,7 @@ public class HoodieWriteConfig extends HoodieConfig {
    */
 
   public String getLockProviderClass() {
-    return getString(HoodieLockConfig.LOCK_PROVIDER_CLASS);
+    return getString(HoodieLockConfig.LOCK_PROVIDER_CLASS_NAME);
   }
 
   public String getLockHiveDatabaseName() {
@@ -1347,7 +1694,7 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public ConflictResolutionStrategy getWriteConflictResolutionStrategy() {
-    return ReflectionUtils.loadClass(getString(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS));
+    return ReflectionUtils.loadClass(getString(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_NAME));
   }
 
   public Long getLockAcquireWaitTimeoutInMs() {
@@ -1355,7 +1702,7 @@ public class HoodieWriteConfig extends HoodieConfig {
   }
 
   public WriteConcurrencyMode getWriteConcurrencyMode() {
-    return WriteConcurrencyMode.fromValue(getString(WRITE_CONCURRENCY_MOD));
+    return WriteConcurrencyMode.fromValue(getString(WRITE_CONCURRENCY_MODE));
   }
 
   public Boolean inlineTableServices() {
@@ -1365,21 +1712,21 @@ public class HoodieWriteConfig extends HoodieConfig {
   public String getWriteMetaKeyPrefixes() {
     return getString(WRITE_META_KEY_PREFIXES);
   }
-  
+
   public String getPreCommitValidators() {
-    return getString(HoodiePreCommitValidatorConfig.PRE_COMMIT_VALIDATORS);
+    return getString(HoodiePreCommitValidatorConfig.VALIDATOR_CLASS_NAMES);
   }
 
   public String getPreCommitValidatorEqualitySqlQueries() {
-    return getString(HoodiePreCommitValidatorConfig.PRE_COMMIT_VALIDATORS_EQUALITY_SQL_QUERIES);
+    return getString(HoodiePreCommitValidatorConfig.EQUALITY_SQL_QUERIES);
   }
 
   public String getPreCommitValidatorSingleResultSqlQueries() {
-    return getString(HoodiePreCommitValidatorConfig.PRE_COMMIT_VALIDATORS_SINGLE_VALUE_SQL_QUERIES);
+    return getString(HoodiePreCommitValidatorConfig.SINGLE_VALUE_SQL_QUERIES);
   }
 
   public String getPreCommitValidatorInequalitySqlQueries() {
-    return getString(HoodiePreCommitValidatorConfig.PRE_COMMIT_VALIDATORS_INEQUALITY_SQL_QUERIES);
+    return getString(HoodiePreCommitValidatorConfig.INEQUALITY_SQL_QUERIES);
   }
 
   public boolean allowEmptyCommit() {
@@ -1441,17 +1788,17 @@ public class HoodieWriteConfig extends HoodieConfig {
     }
 
     public Builder withSchema(String schemaStr) {
-      writeConfig.setValue(AVRO_SCHEMA, schemaStr);
+      writeConfig.setValue(AVRO_SCHEMA_STRING, schemaStr);
       return this;
     }
 
     public Builder withAvroSchemaValidate(boolean enable) {
-      writeConfig.setValue(AVRO_SCHEMA_VALIDATE, String.valueOf(enable));
+      writeConfig.setValue(AVRO_SCHEMA_VALIDATE_ENABLE, String.valueOf(enable));
       return this;
     }
 
     public Builder forTable(String tableName) {
-      writeConfig.setValue(TABLE_NAME, tableName);
+      writeConfig.setValue(TBL_NAME, tableName);
       return this;
     }
 
@@ -1461,53 +1808,53 @@ public class HoodieWriteConfig extends HoodieConfig {
     }
 
     public Builder withWritePayLoad(String payload) {
-      writeConfig.setValue(WRITE_PAYLOAD_CLASS, payload);
+      writeConfig.setValue(WRITE_PAYLOAD_CLASS_NAME, payload);
       return this;
     }
 
     public Builder withKeyGenerator(String keyGeneratorClass) {
-      writeConfig.setValue(KEYGENERATOR_CLASS, keyGeneratorClass);
+      writeConfig.setValue(KEYGENERATOR_CLASS_NAME, keyGeneratorClass);
       return this;
     }
 
     public Builder withTimelineLayoutVersion(int version) {
-      writeConfig.setValue(TIMELINE_LAYOUT_VERSION, String.valueOf(version));
+      writeConfig.setValue(TIMELINE_LAYOUT_VERSION_NUM, String.valueOf(version));
       return this;
     }
 
     public Builder withBulkInsertParallelism(int bulkInsertParallelism) {
-      writeConfig.setValue(BULKINSERT_PARALLELISM, String.valueOf(bulkInsertParallelism));
+      writeConfig.setValue(BULKINSERT_PARALLELISM_VALUE, String.valueOf(bulkInsertParallelism));
       return this;
     }
 
     public Builder withUserDefinedBulkInsertPartitionerClass(String className) {
-      writeConfig.setValue(BULKINSERT_USER_DEFINED_PARTITIONER_CLASS, className);
+      writeConfig.setValue(BULKINSERT_USER_DEFINED_PARTITIONER_CLASS_NAME, className);
       return this;
     }
 
     public Builder withDeleteParallelism(int parallelism) {
-      writeConfig.setValue(DELETE_PARALLELISM, String.valueOf(parallelism));
+      writeConfig.setValue(DELETE_PARALLELISM_VALUE, String.valueOf(parallelism));
       return this;
     }
 
     public Builder withParallelism(int insertShuffleParallelism, int upsertShuffleParallelism) {
-      writeConfig.setValue(INSERT_PARALLELISM, String.valueOf(insertShuffleParallelism));
-      writeConfig.setValue(UPSERT_PARALLELISM, String.valueOf(upsertShuffleParallelism));
+      writeConfig.setValue(INSERT_PARALLELISM_VALUE, String.valueOf(insertShuffleParallelism));
+      writeConfig.setValue(UPSERT_PARALLELISM_VALUE, String.valueOf(upsertShuffleParallelism));
       return this;
     }
 
     public Builder withRollbackParallelism(int rollbackParallelism) {
-      writeConfig.setValue(ROLLBACK_PARALLELISM, String.valueOf(rollbackParallelism));
+      writeConfig.setValue(ROLLBACK_PARALLELISM_VALUE, String.valueOf(rollbackParallelism));
       return this;
     }
 
     public Builder withRollbackUsingMarkers(boolean rollbackUsingMarkers) {
-      writeConfig.setValue(ROLLBACK_USING_MARKERS, String.valueOf(rollbackUsingMarkers));
+      writeConfig.setValue(ROLLBACK_USING_MARKERS_ENABLE, String.valueOf(rollbackUsingMarkers));
       return this;
     }
 
     public Builder withWriteBufferLimitBytes(int writeBufferLimit) {
-      writeConfig.setValue(WRITE_BUFFER_LIMIT_BYTES, String.valueOf(writeBufferLimit));
+      writeConfig.setValue(WRITE_BUFFER_LIMIT_BYTES_VALUE, String.valueOf(writeBufferLimit));
       return this;
     }
 
@@ -1523,7 +1870,7 @@ public class HoodieWriteConfig extends HoodieConfig {
     }
 
     public Builder withWriteStatusStorageLevel(String level) {
-      writeConfig.setValue(WRITE_STATUS_STORAGE_LEVEL, level);
+      writeConfig.setValue(WRITE_STATUS_STORAGE_LEVEL_VALUE, level);
       return this;
     }
 
@@ -1594,12 +1941,12 @@ public class HoodieWriteConfig extends HoodieConfig {
     }
 
     public Builder withAutoCommit(boolean autoCommit) {
-      writeConfig.setValue(HOODIE_AUTO_COMMIT, String.valueOf(autoCommit));
+      writeConfig.setValue(AUTO_COMMIT_ENABLE, String.valueOf(autoCommit));
       return this;
     }
 
     public Builder withWriteStatusClass(Class<? extends WriteStatus> writeStatusClass) {
-      writeConfig.setValue(HOODIE_WRITE_STATUS_CLASS, writeStatusClass.getName());
+      writeConfig.setValue(WRITE_STATUS_CLASS_NAME, writeStatusClass.getName());
       return this;
     }
 
@@ -1622,7 +1969,7 @@ public class HoodieWriteConfig extends HoodieConfig {
     }
 
     public Builder withFinalizeWriteParallelism(int parallelism) {
-      writeConfig.setValue(FINALIZE_WRITE_PARALLELISM, String.valueOf(parallelism));
+      writeConfig.setValue(FINALIZE_WRITE_PARALLELISM_VALUE, String.valueOf(parallelism));
       return this;
     }
 
@@ -1642,12 +1989,12 @@ public class HoodieWriteConfig extends HoodieConfig {
     }
 
     public Builder withMarkersDeleteParallelism(int parallelism) {
-      writeConfig.setValue(MARKERS_DELETE_PARALLELISM, String.valueOf(parallelism));
+      writeConfig.setValue(MARKERS_DELETE_PARALLELISM_VALUE, String.valueOf(parallelism));
       return this;
     }
 
     public Builder withEmbeddedTimelineServerEnabled(boolean enabled) {
-      writeConfig.setValue(EMBEDDED_TIMELINE_SERVER_ENABLED, String.valueOf(enabled));
+      writeConfig.setValue(EMBEDDED_TIMELINE_SERVER_ENABLE, String.valueOf(enabled));
       return this;
     }
 
@@ -1657,32 +2004,32 @@ public class HoodieWriteConfig extends HoodieConfig {
     }
 
     public Builder withEmbeddedTimelineServerPort(int port) {
-      writeConfig.setValue(EMBEDDED_TIMELINE_SERVER_PORT, String.valueOf(port));
+      writeConfig.setValue(EMBEDDED_TIMELINE_SERVER_PORT_NUM, String.valueOf(port));
       return this;
     }
 
     public Builder withBulkInsertSortMode(String mode) {
-      writeConfig.setValue(BULKINSERT_SORT_MODE, mode);
+      writeConfig.setValue(BULK_INSERT_SORT_MODE, mode);
       return this;
     }
 
     public Builder withAllowMultiWriteOnSameInstant(boolean allow) {
-      writeConfig.setValue(ALLOW_MULTI_WRITE_ON_SAME_INSTANT, String.valueOf(allow));
+      writeConfig.setValue(ALLOW_MULTI_WRITE_ON_SAME_INSTANT_ENABLE, String.valueOf(allow));
       return this;
     }
 
     public Builder withExternalSchemaTrasformation(boolean enabled) {
-      writeConfig.setValue(EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION, String.valueOf(enabled));
+      writeConfig.setValue(AVRO_EXTERNAL_SCHEMA_TRANSFORMATION_ENABLE, String.valueOf(enabled));
       return this;
     }
 
     public Builder withMergeDataValidationCheckEnabled(boolean enabled) {
-      writeConfig.setValue(MERGE_DATA_VALIDATION_CHECK_ENABLED, String.valueOf(enabled));
+      writeConfig.setValue(MERGE_DATA_VALIDATION_CHECK_ENABLE, String.valueOf(enabled));
       return this;
     }
 
     public Builder withMergeAllowDuplicateOnInserts(boolean routeInsertsToNewFiles) {
-      writeConfig.setValue(MERGE_ALLOW_DUPLICATE_ON_INSERTS, String.valueOf(routeInsertsToNewFiles));
+      writeConfig.setValue(MERGE_ALLOW_DUPLICATE_ON_INSERTS_ENABLE, String.valueOf(routeInsertsToNewFiles));
       return this;
     }
 
@@ -1697,7 +2044,7 @@ public class HoodieWriteConfig extends HoodieConfig {
     }
 
     public Builder withWriteConcurrencyMode(WriteConcurrencyMode concurrencyMode) {
-      writeConfig.setValue(WRITE_CONCURRENCY_MOD, concurrencyMode.value());
+      writeConfig.setValue(WRITE_CONCURRENCY_MODE, concurrencyMode.value());
       return this;
     }
 
@@ -1707,7 +2054,7 @@ public class HoodieWriteConfig extends HoodieConfig {
     }
 
     public Builder withPopulateMetaFields(boolean populateMetaFields) {
-      writeConfig.setValue(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS, Boolean.toString(populateMetaFields));
+      writeConfig.setValue(HoodieTableConfig.POPULATE_META_FIELDS, Boolean.toString(populateMetaFields));
       return this;
     }
 
@@ -1754,15 +2101,15 @@ public class HoodieWriteConfig extends HoodieConfig {
           HoodieLockConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
       writeConfig.setDefaultOnCondition(!isPreCommitValidationConfigSet,
           HoodiePreCommitValidatorConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
-      writeConfig.setDefaultValue(TIMELINE_LAYOUT_VERSION, String.valueOf(TimelineLayoutVersion.CURR_VERSION));
+      writeConfig.setDefaultValue(TIMELINE_LAYOUT_VERSION_NUM, String.valueOf(TimelineLayoutVersion.CURR_VERSION));
     }
 
     private void validate() {
-      String layoutVersion = writeConfig.getString(TIMELINE_LAYOUT_VERSION);
+      String layoutVersion = writeConfig.getString(TIMELINE_LAYOUT_VERSION_NUM);
       // Ensure Layout Version is good
       new TimelineLayoutVersion(Integer.parseInt(layoutVersion));
       Objects.requireNonNull(writeConfig.getString(BASE_PATH));
-      if (writeConfig.getString(WRITE_CONCURRENCY_MOD)
+      if (writeConfig.getString(WRITE_CONCURRENCY_MODE)
           .equalsIgnoreCase(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.name())) {
         ValidationUtils.checkArgument(writeConfig.getString(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY)
             != HoodieFailedWritesCleaningPolicy.EAGER.name(), "To enable optimistic concurrency control, set hoodie.cleaner.policy.failed.writes=LAZY");
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java
index 0046c18..4923d98 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/KeyGenUtils.java
@@ -186,7 +186,7 @@ public class KeyGenUtils {
    */
   public static KeyGenerator createKeyGeneratorByClassName(TypedProperties props) throws IOException {
     KeyGenerator keyGenerator = null;
-    String keyGeneratorClass = props.getString(HoodieWriteConfig.KEYGENERATOR_CLASS.key(), null);
+    String keyGeneratorClass = props.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), null);
     if (!StringUtils.isNullOrEmpty(keyGeneratorClass)) {
       try {
         keyGenerator = (KeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, props);
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/factory/HoodieAvroKeyGeneratorFactory.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/factory/HoodieAvroKeyGeneratorFactory.java
index 6010ab9..0e17aff 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/factory/HoodieAvroKeyGeneratorFactory.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/factory/HoodieAvroKeyGeneratorFactory.java
@@ -41,7 +41,7 @@ import java.util.Objects;
 /**
  * Factory help to create {@link org.apache.hudi.keygen.KeyGenerator}.
  * <p>
- * This factory will try {@link HoodieWriteConfig#KEYGENERATOR_CLASS} firstly, this ensures the class prop
+ * This factory will try {@link HoodieWriteConfig#KEYGENERATOR_CLASS_NAME} firstly, this ensures the class prop
  * will not be overwritten by {@link KeyGeneratorType}
  */
 public class HoodieAvroKeyGeneratorFactory {
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
index 65c51f0..ab5fa99 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java
@@ -67,7 +67,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
 
-import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP;
+import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER;
 import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX;
 import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP;
 
@@ -299,9 +299,9 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
     LOG.info("Creating a new metadata table in " + metadataWriteConfig.getBasePath() + " at instant " + createInstantTime);
 
     HoodieTableMetaClient.withPropertyBuilder()
-      .setTableType(HoodieTableType.MERGE_ON_READ)
-      .setTableName(tableName)
-      .setArchiveLogFolder(HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue())
+        .setTableType(HoodieTableType.MERGE_ON_READ)
+        .setTableName(tableName)
+        .setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue())
       .setPayloadClassName(HoodieMetadataPayload.class.getName())
       .setBaseFileFormat(HoodieFileFormat.HFILE.toString())
       .initTable(hadoopConf.get(), metadataWriteConfig.getBasePath());
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
index 87dbd06..ad40c8e 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java
@@ -18,10 +18,6 @@
 
 package org.apache.hudi.table;
 
-import org.apache.avro.Schema;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hudi.avro.HoodieAvroUtils;
 import org.apache.hudi.avro.model.HoodieCleanMetadata;
 import org.apache.hudi.avro.model.HoodieCleanerPlan;
@@ -71,6 +67,10 @@ import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
 import org.apache.hudi.table.marker.WriteMarkers;
 import org.apache.hudi.table.marker.WriteMarkersFactory;
 
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
@@ -114,7 +114,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
     HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().fromProperties(config.getMetadataConfig().getProps())
         .build();
     this.metadata = HoodieTableMetadata.create(context, metadataConfig, config.getBasePath(),
-        FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue());
+        FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue());
 
     this.viewManager = FileSystemViewManager.createViewManager(context, config.getMetadataConfig(), config.getViewStorageConfig(), config.getCommonConfig(), () -> metadata);
     this.metaClient = metaClient;
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
index c63a683..317512f 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java
@@ -18,9 +18,6 @@
 
 package org.apache.hudi.table;
 
-import org.apache.avro.Schema;
-import org.apache.avro.generic.IndexedRecord;
-import org.apache.hadoop.fs.Path;
 import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
 import org.apache.hudi.client.utils.MetadataConversionUtils;
 import org.apache.hudi.common.engine.HoodieEngineContext;
@@ -49,6 +46,9 @@ import org.apache.hudi.metadata.HoodieTableMetadata;
 import org.apache.hudi.table.marker.WriteMarkers;
 import org.apache.hudi.table.marker.WriteMarkersFactory;
 
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
@@ -204,7 +204,7 @@ public class HoodieTimelineArchiveLog<T extends HoodieAvroPayload, I, K, O> {
     // instant on the metadata table. This is required for metadata table sync.
     if (config.isMetadataTableEnabled()) {
       try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(table.getContext(), config.getMetadataConfig(),
-          config.getBasePath(), FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue())) {
+          config.getBasePath(), FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue())) {
         Option<String> lastSyncedInstantTime = tableMetadata.getUpdateTime();
 
         if (lastSyncedInstantTime.isPresent()) {
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/BaseOneToTwoUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/BaseOneToTwoUpgradeHandler.java
index a832615..6a43f61 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/BaseOneToTwoUpgradeHandler.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/BaseOneToTwoUpgradeHandler.java
@@ -32,9 +32,9 @@ public abstract class BaseOneToTwoUpgradeHandler implements UpgradeHandler {
   @Override
   public Map<ConfigProperty, String> upgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) {
     Map<ConfigProperty, String> tablePropsToAdd = new HashMap<>();
-    tablePropsToAdd.put(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP, getPartitionColumns(config));
-    tablePropsToAdd.put(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD.key()));
-    tablePropsToAdd.put(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP, config.getString(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP));
+    tablePropsToAdd.put(HoodieTableConfig.PARTITION_FIELDS, getPartitionColumns(config));
+    tablePropsToAdd.put(HoodieTableConfig.RECORDKEY_FIELDS, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD.key()));
+    tablePropsToAdd.put(HoodieTableConfig.BASE_FILE_FORMAT, config.getString(HoodieTableConfig.BASE_FILE_FORMAT));
     return tablePropsToAdd;
   }
 
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 9fbe6d9..68143a2 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
@@ -69,7 +69,7 @@ public class TestHoodieOrcReaderWriter {
     Configuration conf = new Configuration();
     int orcStripSize = Integer.parseInt(HoodieStorageConfig.ORC_STRIPE_SIZE.defaultValue());
     int orcBlockSize = Integer.parseInt(HoodieStorageConfig.ORC_BLOCK_SIZE.defaultValue());
-    int maxFileSize = Integer.parseInt(HoodieStorageConfig.ORC_FILE_MAX_BYTES.defaultValue());
+    int maxFileSize = Integer.parseInt(HoodieStorageConfig.ORC_FILE_MAX_SIZE.defaultValue());
     HoodieOrcConfig config = new HoodieOrcConfig(conf, CompressionKind.ZLIB, orcStripSize, orcBlockSize, maxFileSize, filter);
     TaskContextSupplier mockTaskContextSupplier = Mockito.mock(TaskContextSupplier.class);
     String instantTime = "000";
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/factory/TestHoodieAvroKeyGeneratorFactory.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/factory/TestHoodieAvroKeyGeneratorFactory.java
index 11771c3..c17bc1b 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/factory/TestHoodieAvroKeyGeneratorFactory.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/factory/TestHoodieAvroKeyGeneratorFactory.java
@@ -44,7 +44,7 @@ public class TestHoodieAvroKeyGeneratorFactory {
 
     // set KeyGenerator class only
     props = getCommonProps();
-    props.put(HoodieWriteConfig.KEYGENERATOR_CLASS.key(), SimpleAvroKeyGenerator.class.getName());
+    props.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), SimpleAvroKeyGenerator.class.getName());
     KeyGenerator keyGenerator2 = HoodieAvroKeyGeneratorFactory.createKeyGenerator(props);
     Assertions.assertEquals(SimpleAvroKeyGenerator.class.getName(), keyGenerator2.getClass().getName());
 
@@ -56,7 +56,7 @@ public class TestHoodieAvroKeyGeneratorFactory {
 
     // set wrong class name
     final TypedProperties props2 = getCommonProps();
-    props2.put(HoodieWriteConfig.KEYGENERATOR_CLASS.key(), TestHoodieAvroKeyGeneratorFactory.class.getName());
+    props2.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), TestHoodieAvroKeyGeneratorFactory.class.getName());
     assertThrows(IOException.class, () -> HoodieAvroKeyGeneratorFactory.createKeyGenerator(props2));
 
     // set wrong keyGenerator type
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestHoodieMetricsDatadogConfig.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestHoodieMetricsDatadogConfig.java
index f4ecf42..016e070 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestHoodieMetricsDatadogConfig.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/metrics/datadog/TestHoodieMetricsDatadogConfig.java
@@ -33,7 +33,7 @@ public class TestHoodieMetricsDatadogConfig {
   @Test
   public void getDatadogMetricTagsShouldReturnEmptyListWhenNotSet() {
     HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").build();
-    writeConfig.getProps().remove(HoodieMetricsDatadogConfig.DATADOG_METRIC_TAGS.key());
+    writeConfig.getProps().remove(HoodieMetricsDatadogConfig.METRIC_TAG_VALUES.key());
     assertIterableEquals(Collections.emptyList(), writeConfig.getDatadogMetricTags());
   }
 }
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 353690d..0c4c771 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
@@ -92,12 +92,12 @@ public class HoodieWriteableTestTable extends HoodieTestTable {
     FileCreateUtils.createPartitionMetaFile(basePath, partition);
     String fileName = baseFileName(currentInstantTime, fileId);
 
-    if (HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().equals(HoodieFileFormat.PARQUET)) {
+    if (HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().equals(HoodieFileFormat.PARQUET)) {
       HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
           new AvroSchemaConverter().convert(schema), schema, Option.of(filter));
       HoodieAvroParquetConfig config = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP,
           ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024,
-          new Configuration(), Double.parseDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO.defaultValue()));
+          new Configuration(), Double.parseDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION.defaultValue()));
       try (HoodieParquetWriter writer = new HoodieParquetWriter(
           currentInstantTime,
           new Path(Paths.get(basePath, partition, fileName).toString()),
@@ -115,11 +115,11 @@ public class HoodieWriteableTestTable extends HoodieTestTable {
           }
         }
       }
-    } else if (HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().equals(HoodieFileFormat.ORC)) {
+    } else if (HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().equals(HoodieFileFormat.ORC)) {
       Configuration conf = new Configuration();
       int orcStripSize = Integer.parseInt(HoodieStorageConfig.ORC_STRIPE_SIZE.defaultValue());
       int orcBlockSize = Integer.parseInt(HoodieStorageConfig.ORC_BLOCK_SIZE.defaultValue());
-      int maxFileSize = Integer.parseInt(HoodieStorageConfig.ORC_FILE_MAX_BYTES.defaultValue());
+      int maxFileSize = Integer.parseInt(HoodieStorageConfig.ORC_FILE_MAX_SIZE.defaultValue());
       HoodieOrcConfig config = new HoodieOrcConfig(conf, CompressionKind.ZLIB, orcStripSize, orcBlockSize, maxFileSize, filter);
       try (HoodieOrcWriter writer = new HoodieOrcWriter(
           currentInstantTime,
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSizeBasedClusteringPlanStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSizeBasedClusteringPlanStrategy.java
index aa68f30..d58619b 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSizeBasedClusteringPlanStrategy.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkSizeBasedClusteringPlanStrategy.java
@@ -32,6 +32,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
 import org.apache.hudi.table.HoodieSparkMergeOnReadTable;
 import org.apache.hudi.table.action.cluster.strategy.PartitionAwareClusteringPlanStrategy;
+
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.spark.api.java.JavaRDD;
@@ -42,7 +43,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.stream.Stream;
 
-import static org.apache.hudi.config.HoodieClusteringConfig.CLUSTERING_SORT_COLUMNS_PROPERTY;
+import static org.apache.hudi.config.HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS;
 
 /**
  * Clustering Strategy based on following.
@@ -102,7 +103,7 @@ public class SparkSizeBasedClusteringPlanStrategy<T extends HoodieRecordPayload<
   protected Map<String, String> getStrategyParams() {
     Map<String, String> params = new HashMap<>();
     if (!StringUtils.isNullOrEmpty(getWriteConfig().getClusteringSortColumns())) {
-      params.put(CLUSTERING_SORT_COLUMNS_PROPERTY.key(), getWriteConfig().getClusteringSortColumns());
+      params.put(PLAN_STRATEGY_SORT_COLUMNS.key(), getWriteConfig().getClusteringSortColumns());
     }
     return params;
   }
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 4dd5cd0..1df6eee 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
@@ -18,7 +18,6 @@
 
 package org.apache.hudi.client.clustering.run.strategy;
 
-import org.apache.avro.Schema;
 import org.apache.hudi.avro.HoodieAvroUtils;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.common.engine.HoodieEngineContext;
@@ -32,6 +31,8 @@ import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner;
 import org.apache.hudi.table.BulkInsertPartitioner;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.table.action.commit.SparkBulkInsertHelper;
+
+import org.apache.avro.Schema;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.spark.api.java.JavaRDD;
@@ -40,7 +41,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
-import static org.apache.hudi.config.HoodieClusteringConfig.CLUSTERING_SORT_COLUMNS_PROPERTY;
+import static org.apache.hudi.config.HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS;
 
 /**
  * Clustering Strategy based on following.
@@ -63,11 +64,11 @@ public class SparkSortAndSizeExecutionStrategy<T extends HoodieRecordPayload<T>>
                                                 final List<HoodieFileGroupId> fileGroupIdList, final boolean preserveHoodieMetadata) {
     LOG.info("Starting clustering for a group, parallelism:" + numOutputGroups + " commit:" + instantTime);
     Properties props = getWriteConfig().getProps();
-    props.put(HoodieWriteConfig.BULKINSERT_PARALLELISM.key(), String.valueOf(numOutputGroups));
+    props.put(HoodieWriteConfig.BULKINSERT_PARALLELISM_VALUE.key(), String.valueOf(numOutputGroups));
     // We are calling another action executor - disable auto commit. Strategy is only expected to write data in new files.
-    props.put(HoodieWriteConfig.HOODIE_AUTO_COMMIT.key(), Boolean.FALSE.toString());
-    props.put(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES.key(), String.valueOf(getWriteConfig().getClusteringTargetFileMaxBytes()));
-    HoodieWriteConfig newConfig =  HoodieWriteConfig.newBuilder().withProps(props).build();
+    props.put(HoodieWriteConfig.AUTO_COMMIT_ENABLE.key(), Boolean.FALSE.toString());
+    props.put(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.key(), String.valueOf(getWriteConfig().getClusteringTargetFileMaxBytes()));
+    HoodieWriteConfig newConfig = HoodieWriteConfig.newBuilder().withProps(props).build();
     return (JavaRDD<WriteStatus>) SparkBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig,
         false, getPartitioner(strategyParams, schema), true, numOutputGroups, preserveHoodieMetadata);
   }
@@ -76,8 +77,8 @@ public class SparkSortAndSizeExecutionStrategy<T extends HoodieRecordPayload<T>>
    * Create BulkInsertPartitioner based on strategy params.
    */
   protected Option<BulkInsertPartitioner<T>> getPartitioner(Map<String, String> strategyParams, Schema schema) {
-    if (strategyParams.containsKey(CLUSTERING_SORT_COLUMNS_PROPERTY.key())) {
-      return Option.of(new RDDCustomColumnsSortPartitioner(strategyParams.get(CLUSTERING_SORT_COLUMNS_PROPERTY.key()).split(","),
+    if (strategyParams.containsKey(PLAN_STRATEGY_SORT_COLUMNS.key())) {
+      return Option.of(new RDDCustomColumnsSortPartitioner(strategyParams.get(PLAN_STRATEGY_SORT_COLUMNS.key()).split(","),
           HoodieAvroUtils.addMetadataFields(schema)));
     } else {
       return Option.empty();
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMemoryUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMemoryUtils.java
index 9ca5a0c..58d822a 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMemoryUtils.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMemoryUtils.java
@@ -25,21 +25,21 @@ import org.apache.spark.storage.StorageLevel;
 
 import java.util.Properties;
 
-import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL;
+import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL_VALUE;
 
 /**
  * Spark config utils.
  */
 public class SparkMemoryUtils {
   public static StorageLevel getWriteStatusStorageLevel(Properties properties) {
-    return StorageLevel.fromString(new HoodieConfig(properties).getString(WRITE_STATUS_STORAGE_LEVEL));
+    return StorageLevel.fromString(new HoodieConfig(properties).getString(WRITE_STATUS_STORAGE_LEVEL_VALUE));
   }
 
   public static StorageLevel getBloomIndexInputStorageLevel(Properties properties) {
-    return StorageLevel.fromString(new HoodieConfig(properties).getString(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL));
+    return StorageLevel.fromString(new HoodieConfig(properties).getString(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL_VALUE));
   }
 
   public static StorageLevel getSimpleIndexInputStorageLevel(Properties properties) {
-    return StorageLevel.fromString(new HoodieConfig(properties).getString(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL));
+    return StorageLevel.fromString(new HoodieConfig(properties).getString(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL_VALUE));
   }
 }
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java
index 9432a3c..b27f84e 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java
@@ -25,6 +25,7 @@ import org.apache.hudi.config.HoodiePreCommitValidatorConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieValidationException;
 import org.apache.hudi.table.HoodieSparkTable;
+
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.spark.api.java.JavaRDD;
@@ -49,7 +50,7 @@ public class SqlQueryEqualityPreCommitValidator<T extends HoodieRecordPayload, I
 
   @Override
   protected String getQueryConfigName() {
-    return HoodiePreCommitValidatorConfig.PRE_COMMIT_VALIDATORS_EQUALITY_SQL_QUERIES.key();
+    return HoodiePreCommitValidatorConfig.EQUALITY_SQL_QUERIES.key();
   }
 
   @Override
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java
index 0ff2757..454638c 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java
@@ -25,6 +25,7 @@ import org.apache.hudi.config.HoodiePreCommitValidatorConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieValidationException;
 import org.apache.hudi.table.HoodieSparkTable;
+
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.spark.api.java.JavaRDD;
@@ -48,7 +49,7 @@ public class SqlQueryInequalityPreCommitValidator<T extends HoodieRecordPayload,
 
   @Override
   protected String getQueryConfigName() {
-    return HoodiePreCommitValidatorConfig.PRE_COMMIT_VALIDATORS_INEQUALITY_SQL_QUERIES.key();
+    return HoodiePreCommitValidatorConfig.INEQUALITY_SQL_QUERIES.key();
   }
 
   @Override
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java
index c41c3ac..631f0e6 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java
@@ -25,6 +25,7 @@ import org.apache.hudi.config.HoodiePreCommitValidatorConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.exception.HoodieValidationException;
 import org.apache.hudi.table.HoodieSparkTable;
+
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.spark.api.java.JavaRDD;
@@ -48,7 +49,7 @@ public class SqlQuerySingleResultPreCommitValidator<T extends HoodieRecordPayloa
 
   @Override
   protected String getQueryConfigName() {
-    return HoodiePreCommitValidatorConfig.PRE_COMMIT_VALIDATORS_SINGLE_VALUE_SQL_QUERIES.key();
+    return HoodiePreCommitValidatorConfig.SINGLE_VALUE_SQL_QUERIES.key();
   }
 
   @Override
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java
index b3b9eb4..866d0d3 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java
@@ -18,8 +18,6 @@
 
 package org.apache.hudi.index.hbase;
 
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.client.common.HoodieSparkEngineContext;
 import org.apache.hudi.client.utils.SparkMemoryUtils;
@@ -56,6 +54,8 @@ import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.RegionLocator;
 import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
@@ -105,7 +105,7 @@ public class SparkHoodieHBaseIndex<T extends HoodieRecordPayload> extends SparkH
 
   /**
    * multiPutBatchSize will be computed and re-set in updateLocation if
-   * {@link HoodieHBaseIndexConfig#HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE} is set to true.
+   * {@link HoodieHBaseIndexConfig#PUT_BATCH_SIZE_AUTO_COMPUTE} is set to true.
    */
   private Integer multiPutBatchSize;
   private Integer numRegionServersForTable;
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java
index 0fd6011..d4e99f7 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/factory/HoodieSparkKeyGeneratorFactory.java
@@ -43,7 +43,7 @@ import java.util.Objects;
 /**
  * Factory help to create {@link org.apache.hudi.keygen.KeyGenerator}.
  * <p>
- * This factory will try {@link HoodieWriteConfig#KEYGENERATOR_CLASS} firstly, this ensures the class prop
+ * This factory will try {@link HoodieWriteConfig#KEYGENERATOR_CLASS_NAME} firstly, this ensures the class prop
  * will not be overwritten by {@link KeyGeneratorType}
  */
 public class HoodieSparkKeyGeneratorFactory {
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 e78c2c8..bec7ee4 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
@@ -18,24 +18,6 @@
 
 package org.apache.hudi.client.functional;
 
-import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertThrows;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.stream.Collectors;
-
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
 import org.apache.hudi.client.HoodieWriteResult;
 import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
@@ -81,6 +63,9 @@ import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
 import org.apache.hudi.table.HoodieSparkTable;
 import org.apache.hudi.table.HoodieTable;
 import org.apache.hudi.testutils.HoodieClientTestHarness;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.spark.api.java.JavaRDD;
@@ -93,6 +78,22 @@ import org.junit.jupiter.api.io.TempDir;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.EnumSource;
 
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 @Tag("functional")
 public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
 
@@ -800,8 +801,8 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
       assertEquals(writer.getMetadataReader().getUpdateTime().get(), beforeInflightActionTimestamp);
 
       // Reader should sync to all the completed instants
-      HoodieTableMetadata metadata  = HoodieTableMetadata.create(context, client.getConfig().getMetadataConfig(),
-          client.getConfig().getBasePath(), FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue());
+      HoodieTableMetadata metadata = HoodieTableMetadata.create(context, client.getConfig().getMetadataConfig(),
+          client.getConfig().getBasePath(), FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue());
       assertEquals(((HoodieBackedTableMetadata)metadata).getReaderTime().get(), newCommitTime);
 
       // Remove the inflight instance holding back table sync
@@ -813,8 +814,8 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
       assertEquals(writer.getMetadataReader().getUpdateTime().get(), newCommitTime);
 
       // Reader should sync to all the completed instants
-      metadata  = HoodieTableMetadata.create(context, client.getConfig().getMetadataConfig(),
-          client.getConfig().getBasePath(), FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue());
+      metadata = HoodieTableMetadata.create(context, client.getConfig().getMetadataConfig(),
+          client.getConfig().getBasePath(), FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue());
       assertEquals(writer.getMetadataReader().getUpdateTime().get(), newCommitTime);
     }
 
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 45dedd6..de4f421 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
@@ -18,21 +18,18 @@
 
 package org.apache.hudi.client.functional;
 
-import org.apache.avro.generic.GenericRecord;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.Path;
+import org.apache.hudi.avro.model.HoodieCleanMetadata;
 import org.apache.hudi.avro.model.HoodieClusteringPlan;
 import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata;
-import org.apache.hudi.avro.model.HoodieCleanMetadata;
 import org.apache.hudi.client.AbstractHoodieWriteClient;
 import org.apache.hudi.client.HoodieWriteResult;
 import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.SparkTaskContextSupplier;
 import org.apache.hudi.client.WriteStatus;
-import org.apache.hudi.common.config.TypedProperties;
 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.TypedProperties;
 import org.apache.hudi.common.engine.HoodieEngineContext;
 import org.apache.hudi.common.fs.ConsistencyGuardConfig;
 import org.apache.hudi.common.fs.FSUtils;
@@ -58,9 +55,9 @@ import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
 import org.apache.hudi.common.testutils.HoodieTestTable;
 import org.apache.hudi.common.testutils.HoodieTestUtils;
 import org.apache.hudi.common.testutils.RawTripTestPayload;
+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.BaseFileUtils;
 import org.apache.hudi.common.util.FileIOUtils;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.StringUtils;
@@ -93,6 +90,10 @@ import org.apache.hudi.table.marker.WriteMarkersFactory;
 import org.apache.hudi.testutils.HoodieClientTestBase;
 import org.apache.hudi.testutils.HoodieClientTestUtils;
 import org.apache.hudi.testutils.HoodieSparkWriteableTestTable;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.spark.api.java.JavaRDD;
@@ -136,7 +137,7 @@ import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAM
 import static org.apache.hudi.common.testutils.Transformations.randomSelectAsHoodieKeys;
 import static org.apache.hudi.common.testutils.Transformations.recordsToRecordKeySet;
 import static org.apache.hudi.config.HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE;
-import static org.apache.hudi.config.HoodieClusteringConfig.CLUSTERING_EXECUTION_STRATEGY_CLASS;
+import static org.apache.hudi.config.HoodieClusteringConfig.EXECUTION_STRATEGY_CLASS_NAME;
 import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -2353,7 +2354,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
 
   protected HoodieInstant createRequestedReplaceInstant(HoodieTableMetaClient metaClient, String clusterTime, List<FileSlice>[] fileSlices) throws IOException {
     HoodieClusteringPlan clusteringPlan =
-        ClusteringUtils.createClusteringPlan(CLUSTERING_EXECUTION_STRATEGY_CLASS.defaultValue(), STRATEGY_PARAMS, fileSlices, Collections.emptyMap());
+        ClusteringUtils.createClusteringPlan(EXECUTION_STRATEGY_CLASS_NAME.defaultValue(), STRATEGY_PARAMS, fileSlices, Collections.emptyMap());
 
     HoodieInstant clusteringInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, clusterTime);
     HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder()
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 9f52a01..be80e9d 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
@@ -46,7 +46,7 @@ public class TestHBaseQPSResourceAllocator {
 
   @Test
   public void testsExplicitDefaultQPSResourceAllocator() {
-    HoodieWriteConfig config = getConfig(Option.of(HoodieHBaseIndexConfig.HBASE_INDEX_QPS_ALLOCATOR_CLASS.defaultValue()));
+    HoodieWriteConfig config = getConfig(Option.of(HoodieHBaseIndexConfig.QPS_ALLOCATOR_CLASS_NAME.defaultValue()));
     SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
     HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config);
     assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(),
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java
index d479923..9dbb3a72 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestCustomKeyGenerator.java
@@ -53,7 +53,7 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
       properties.put(KeyGeneratorOptions.RECORDKEY_FIELD.key(), "_row_key");
     }
     if (useKeyGeneratorClassName) {
-      properties.put(HoodieWriteConfig.KEYGENERATOR_CLASS.key(), CustomKeyGenerator.class.getName());
+      properties.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), CustomKeyGenerator.class.getName());
     } else {
       properties.put(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), KeyGeneratorType.CUSTOM.name());
     }
@@ -96,7 +96,7 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
     TypedProperties properties = new TypedProperties();
     properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD.key(), "timestamp:simple");
     if (useKeyGeneratorClassName) {
-      properties.put(HoodieWriteConfig.KEYGENERATOR_CLASS.key(), CustomKeyGenerator.class.getName());
+      properties.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), CustomKeyGenerator.class.getName());
     } else {
       properties.put(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), KeyGeneratorType.CUSTOM.name());
     }
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestHoodieSparkKeyGeneratorFactory.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestHoodieSparkKeyGeneratorFactory.java
index 8aaba76..682329a 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestHoodieSparkKeyGeneratorFactory.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestHoodieSparkKeyGeneratorFactory.java
@@ -51,7 +51,7 @@ public class TestHoodieSparkKeyGeneratorFactory {
 
     // set KeyGenerator class only
     props = getCommonProps();
-    props.put(HoodieWriteConfig.KEYGENERATOR_CLASS.key(), SimpleKeyGenerator.class.getName());
+    props.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), SimpleKeyGenerator.class.getName());
     KeyGenerator keyGenerator2 = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
     Assertions.assertEquals(SimpleKeyGenerator.class.getName(), keyGenerator2.getClass().getName());
 
@@ -63,7 +63,7 @@ public class TestHoodieSparkKeyGeneratorFactory {
 
     // set wrong class name
     final TypedProperties props2 = getCommonProps();
-    props2.put(HoodieWriteConfig.KEYGENERATOR_CLASS.key(), TestHoodieSparkKeyGeneratorFactory.class.getName());
+    props2.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), TestHoodieSparkKeyGeneratorFactory.class.getName());
     assertThrows(IOException.class, () -> HoodieSparkKeyGeneratorFactory.createKeyGenerator(props2));
 
     // set wrong keyGenerator type
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestConsistencyGuard.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestConsistencyGuard.java
index 32e0b43..afbe949 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestConsistencyGuard.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestConsistencyGuard.java
@@ -45,7 +45,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
  */
 public class TestConsistencyGuard extends HoodieClientTestHarness {
 
-  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension();
+  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension();
 
   // multiple parameters, uses Collection<Object[]>
   public static List<Arguments> consistencyGuardType() {
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
index 5bc293a..0239138 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java
@@ -128,7 +128,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
     dfs.mkdirs(new Path(basePath));
 
     Properties properties = populateMetaFields ? new Properties() : getPropertiesForKeyGen();
-    properties.setProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.key(), baseFileFormat.toString());
+    properties.setProperty(HoodieTableConfig.BASE_FILE_FORMAT.key(), baseFileFormat.toString());
 
     metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ, properties);
     initTestDataGenerator();
@@ -140,7 +140,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
 
   @BeforeEach
   public void init() throws IOException {
-    init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), true);
+    init(HoodieTableConfig.BASE_FILE_FORMAT.defaultValue(), true);
   }
 
   @AfterEach
@@ -165,7 +165,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
   @MethodSource("populateMetaFieldsParams")
   public void testSimpleInsertAndUpdate(boolean populateMetaFields) throws Exception {
     clean();
-    init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
+    init(HoodieTableConfig.BASE_FILE_FORMAT.defaultValue(), populateMetaFields);
 
     HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true);
     addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
@@ -266,7 +266,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
   @MethodSource("populateMetaFieldsAndPreserveMetadataParams")
   public void testSimpleClusteringNoUpdates(boolean populateMetaFields, boolean preserveCommitMetadata) throws Exception {
     clean();
-    init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
+    init(HoodieTableConfig.BASE_FILE_FORMAT.defaultValue(), populateMetaFields);
     testClustering(false, populateMetaFields, preserveCommitMetadata);
   }
 
@@ -274,7 +274,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
   @MethodSource("populateMetaFieldsAndPreserveMetadataParams")
   public void testSimpleClusteringWithUpdates(boolean populateMetaFields, boolean preserveCommitMetadata) throws Exception {
     clean();
-    init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
+    init(HoodieTableConfig.BASE_FILE_FORMAT.defaultValue(), populateMetaFields);
     testClustering(true, populateMetaFields, preserveCommitMetadata);
   }
 
@@ -478,7 +478,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
   @MethodSource("populateMetaFieldsParams")
   public void testSimpleInsertUpdateAndDelete(boolean populateMetaFields) throws Exception {
     clean();
-    init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
+    init(HoodieTableConfig.BASE_FILE_FORMAT.defaultValue(), populateMetaFields);
     HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true);
     addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
     HoodieWriteConfig cfg = cfgBuilder.build();
@@ -777,7 +777,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
   @MethodSource("populateMetaFieldsParams")
   public void testRollbackWithDeltaAndCompactionCommitUsingFileList(boolean populateMetaFields) throws Exception {
     clean();
-    init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
+    init(HoodieTableConfig.BASE_FILE_FORMAT.defaultValue(), populateMetaFields);
     testRollbackWithDeltaAndCompactionCommit(false, populateMetaFields);
   }
 
@@ -785,7 +785,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
   @MethodSource("populateMetaFieldsParams")
   public void testRollbackWithDeltaAndCompactionCommitUsingMarkers(boolean populateMetaFields) throws Exception {
     clean();
-    init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
+    init(HoodieTableConfig.BASE_FILE_FORMAT.defaultValue(), populateMetaFields);
     testRollbackWithDeltaAndCompactionCommit(true, populateMetaFields);
   }
 
@@ -793,7 +793,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
   @MethodSource("populateMetaFieldsParams")
   public void testMultiRollbackWithDeltaAndCompactionCommit(boolean populateMetaFields) throws Exception {
     clean();
-    init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
+    init(HoodieTableConfig.BASE_FILE_FORMAT.defaultValue(), populateMetaFields);
     HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(false);
     addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
     HoodieWriteConfig cfg = cfgBuilder.build();
@@ -966,7 +966,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
   @MethodSource("populateMetaFieldsParams")
   public void testUpsertPartitioner(boolean populateMetaFields) throws Exception {
     clean();
-    init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(), populateMetaFields);
+    init(HoodieTableConfig.BASE_FILE_FORMAT.defaultValue(), populateMetaFields);
     HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true);
     addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
     HoodieWriteConfig cfg = cfgBuilder.build();
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java
index fa5155f..823eac4 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/strategy/TestHoodieCompactionStrategy.java
@@ -28,6 +28,7 @@ import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.config.HoodieCompactionConfig;
 import org.apache.hudi.config.HoodieWriteConfig;
+
 import org.junit.jupiter.api.Test;
 
 import java.text.SimpleDateFormat;
@@ -275,7 +276,7 @@ public class TestHoodieCompactionStrategy {
     private final long size;
 
     public TestHoodieBaseFile(long size) {
-      super("/tmp/XYXYXYXYXYYX_11_20180918020003" + HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension());
+      super("/tmp/XYXYXYXYXYYX_11_20180918020003" + HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension());
       this.size = size;
     }
 
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestRollbackUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestRollbackUtils.java
index 62623b1..f03d9f3 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestRollbackUtils.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestRollbackUtils.java
@@ -18,15 +18,16 @@
 
 package org.apache.hudi.table.action.rollback;
 
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hudi.common.HoodieRollbackStat;
 import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.table.log.block.HoodieLogBlock;
 import org.apache.hudi.common.table.timeline.HoodieInstant;
 import org.apache.hudi.common.table.timeline.HoodieTimeline;
 import org.apache.hudi.common.util.CollectionUtils;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.junit.jupiter.api.Test;
 
 import java.util.HashMap;
@@ -34,11 +35,11 @@ import java.util.Map;
 import java.util.stream.Collectors;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertIterableEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
 
 public class TestRollbackUtils {
-  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension();
+  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension();
 
   private FileStatus generateFileStatus(String filePath) {
     Path dataFile1Path = new Path(filePath);
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java
index 68876d7..123a33c 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java
@@ -75,8 +75,8 @@ import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
-import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP;
-import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_TABLE_TYPE_PROP;
+import static org.apache.hudi.common.table.HoodieTableConfig.BASE_FILE_FORMAT;
+import static org.apache.hudi.common.table.HoodieTableConfig.TYPE;
 import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
 import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH;
 import static org.apache.hudi.common.util.MarkerUtils.MARKERS_FILENAME_PREFIX;
@@ -145,7 +145,7 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
     // init config, table and client.
     Map<String, String> params = new HashMap<>();
     if (tableType == HoodieTableType.MERGE_ON_READ) {
-      params.put(HOODIE_TABLE_TYPE_PROP.key(), HoodieTableType.MERGE_ON_READ.name());
+      params.put(TYPE.key(), HoodieTableType.MERGE_ON_READ.name());
       metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ);
     }
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(false).withProps(params).build();
@@ -206,7 +206,7 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
     Map<String, String> params = new HashMap<>();
     addNewTableParamsToProps(params);
     if (tableType == HoodieTableType.MERGE_ON_READ) {
-      params.put(HOODIE_TABLE_TYPE_PROP.key(), HoodieTableType.MERGE_ON_READ.name());
+      params.put(TYPE.key(), HoodieTableType.MERGE_ON_READ.name());
       metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ);
     }
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(false).withProps(params).build();
@@ -233,8 +233,8 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
   private void addNewTableParamsToProps(Map<String, String> params) {
     params.put(KeyGeneratorOptions.RECORDKEY_FIELD.key(), "uuid");
     params.put(KeyGeneratorOptions.PARTITIONPATH_FIELD.key(), "partition_path");
-    params.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP.key(), metaClient.getTableConfig().getTableName());
-    params.put(HOODIE_BASE_FILE_FORMAT_PROP.key(), HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().name());
+    params.put(HoodieTableConfig.NAME.key(), metaClient.getTableConfig().getTableName());
+    params.put(BASE_FILE_FORMAT.key(), BASE_FILE_FORMAT.defaultValue().name());
   }
 
   private void doInsert(SparkRDDWriteClient client) {
@@ -248,11 +248,11 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
 
   private void downgradeTableConfigsFromTwoToOne(HoodieWriteConfig cfg) throws IOException {
     Properties properties = new Properties(cfg.getProps());
-    properties.remove(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS.key());
-    properties.remove(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP.key());
-    properties.remove(HoodieTableConfig.HOODIE_TABLE_NAME_PROP.key());
-    properties.remove(HOODIE_BASE_FILE_FORMAT_PROP.key());
-    properties.setProperty(HoodieTableConfig.HOODIE_TABLE_VERSION_PROP.key(), "1");
+    properties.remove(HoodieTableConfig.RECORDKEY_FIELDS.key());
+    properties.remove(HoodieTableConfig.PARTITION_FIELDS.key());
+    properties.remove(HoodieTableConfig.NAME.key());
+    properties.remove(BASE_FILE_FORMAT.key());
+    properties.setProperty(HoodieTableConfig.VERSION.key(), "1");
 
     metaClient = HoodieTestUtils.init(hadoopConf, basePath, getTableType(), properties);
     // set hoodie.table.version to 1 in hoodie.properties file
@@ -265,7 +265,7 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
     assertEquals(tableConfig.getPartitionFieldProp(), originalProps.getProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD.key()));
     assertEquals(tableConfig.getRecordKeyFieldProp(), originalProps.getProperty(KeyGeneratorOptions.RECORDKEY_FIELD.key()));
     assertEquals(tableConfig.getTableName(), cfg.getTableName());
-    assertEquals(tableConfig.getBaseFileFormat().name(), originalProps.getProperty(HOODIE_BASE_FILE_FORMAT_PROP.key()));
+    assertEquals(tableConfig.getBaseFileFormat().name(), originalProps.getProperty(BASE_FILE_FORMAT.key()));
   }
 
   @ParameterizedTest(name = TEST_NAME_WITH_DOWNGRADE_PARAMS)
@@ -278,7 +278,7 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
       addNewTableParamsToProps(params);
     }
     if (tableType == HoodieTableType.MERGE_ON_READ) {
-      params.put(HOODIE_TABLE_TYPE_PROP.key(), HoodieTableType.MERGE_ON_READ.name());
+      params.put(TYPE.key(), HoodieTableType.MERGE_ON_READ.name());
       metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ);
     }
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(true)
@@ -288,10 +288,10 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
     if (fromVersion == HoodieTableVersion.TWO) {
       // set table configs
       HoodieTableConfig tableConfig = metaClient.getTableConfig();
-      tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_NAME_PROP, cfg.getTableName());
-      tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP, cfg.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD.key()));
-      tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS, cfg.getString(KeyGeneratorOptions.RECORDKEY_FIELD.key()));
-      tableConfig.setValue(HOODIE_BASE_FILE_FORMAT_PROP, cfg.getString(HOODIE_BASE_FILE_FORMAT_PROP));
+      tableConfig.setValue(HoodieTableConfig.NAME, cfg.getTableName());
+      tableConfig.setValue(HoodieTableConfig.PARTITION_FIELDS, cfg.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD.key()));
+      tableConfig.setValue(HoodieTableConfig.RECORDKEY_FIELDS, cfg.getString(KeyGeneratorOptions.RECORDKEY_FIELD.key()));
+      tableConfig.setValue(BASE_FILE_FORMAT, cfg.getString(BASE_FILE_FORMAT));
     }
 
     // prepare data. Make 2 commits, in which 2nd is not committed.
@@ -433,7 +433,7 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
   private List<HoodieRecord> triggerCommit(String newCommitTime, HoodieTableType tableType, boolean enableMarkedBasedRollback) {
     Map<String, String> params = new HashMap<>();
     if (tableType == HoodieTableType.MERGE_ON_READ) {
-      params.put(HOODIE_TABLE_TYPE_PROP.key(), HoodieTableType.MERGE_ON_READ.name());
+      params.put(TYPE.key(), HoodieTableType.MERGE_ON_READ.name());
     }
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(enableMarkedBasedRollback).withProps(params).build();
     SparkRDDWriteClient client = getHoodieWriteClient(cfg);
@@ -571,6 +571,6 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
     HoodieConfig hoodieConfig = HoodieConfig.create(fsDataInputStream);
     fsDataInputStream.close();
     assertEquals(Integer.toString(expectedVersion.versionCode()), hoodieConfig
-        .getString(HoodieTableConfig.HOODIE_TABLE_VERSION_PROP));
+        .getString(HoodieTableConfig.VERSION));
   }
 }
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java
index 233b679..a6ac67b 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestHarness.java
@@ -17,13 +17,6 @@
 
 package org.apache.hudi.testutils;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocalFileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hudi.client.HoodieReadClient;
 import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.SparkTaskContextSupplier;
@@ -46,6 +39,14 @@ import org.apache.hudi.config.HoodieWriteConfig;
 import org.apache.hudi.index.HoodieIndex;
 import org.apache.hudi.keygen.SimpleKeyGenerator;
 import org.apache.hudi.table.WorkloadStat;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 import org.apache.spark.api.java.JavaRDD;
@@ -54,7 +55,6 @@ import org.apache.spark.sql.SQLContext;
 import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.TestInfo;
-import scala.Tuple2;
 
 import java.io.IOException;
 import java.io.Serializable;
@@ -64,13 +64,15 @@ import java.util.Properties;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
+import scala.Tuple2;
+
 /**
  * The test harness for resource initialization and cleanup.
  */
 public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness implements Serializable {
 
   private static final Logger LOG = LogManager.getLogger(HoodieClientTestHarness.class);
-  
+
   private String testMethodName;
   protected transient JavaSparkContext jsc = null;
   protected transient HoodieSparkEngineContext context = null;
@@ -232,12 +234,12 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
 
   protected Properties getPropertiesForKeyGen() {
     Properties properties = new Properties();
-    properties.put(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), "false");
-    properties.put("hoodie.datasource.write.recordkey.field","_row_key");
-    properties.put("hoodie.datasource.write.partitionpath.field","partition_path");
-    properties.put(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS.key(), "_row_key");
-    properties.put(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP.key(), "partition_path");
-    properties.put(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS.key(), SimpleKeyGenerator.class.getName());
+    properties.put(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false");
+    properties.put("hoodie.datasource.write.recordkey.field", "_row_key");
+    properties.put("hoodie.datasource.write.partitionpath.field", "partition_path");
+    properties.put(HoodieTableConfig.RECORDKEY_FIELDS.key(), "_row_key");
+    properties.put(HoodieTableConfig.PARTITION_FIELDS.key(), "partition_path");
+    properties.put(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key(), SimpleKeyGenerator.class.getName());
     return properties;
   }
 
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java
index 15fb50c..59172c5 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestUtils.java
@@ -121,9 +121,9 @@ public class HoodieClientTestUtils {
           getLatestFileIDsToFullPath(basePath, commitTimeline, Arrays.asList(commitInstant));
       LOG.info("Path :" + paths.values());
       Dataset<Row> unFilteredRows = null;
-      if (HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().equals(HoodieFileFormat.PARQUET)) {
+      if (HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().equals(HoodieFileFormat.PARQUET)) {
         unFilteredRows = sqlContext.read().parquet(paths.values().toArray(new String[paths.size()]));
-      } else if (HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().equals(HoodieFileFormat.ORC)) {
+      } else if (HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().equals(HoodieFileFormat.ORC)) {
         unFilteredRows = sqlContext.read().orc(paths.values().toArray(new String[paths.size()]));
       }
       if (unFilteredRows != null) {
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java
index 842d8fb..f55f46b 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java
@@ -19,6 +19,7 @@
 package org.apache.hudi.common.config;
 
 import javax.annotation.concurrent.Immutable;
+
 import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
@@ -38,21 +39,21 @@ public final class HoodieMetadataConfig extends HoodieConfig {
   public static final String METADATA_PREFIX = "hoodie.metadata";
 
   // Enable the internal Metadata Table which saves file listings
-  public static final ConfigProperty<Boolean> METADATA_ENABLE_PROP = ConfigProperty
+  public static final ConfigProperty<Boolean> ENABLE = ConfigProperty
       .key(METADATA_PREFIX + ".enable")
       .defaultValue(false)
       .sinceVersion("0.7.0")
       .withDocumentation("Enable the internal metadata table which serves table metadata like level file listings");
 
   // Enable syncing the Metadata Table
-  public static final ConfigProperty<Boolean> METADATA_SYNC_ENABLE_PROP = ConfigProperty
+  public static final ConfigProperty<Boolean> SYNC_ENABLE = ConfigProperty
       .key(METADATA_PREFIX + ".sync.enable")
       .defaultValue(true)
       .sinceVersion("0.9.0")
       .withDocumentation("Enable syncing of metadata table from actions on the dataset");
 
   // Validate contents of Metadata Table on each access against the actual filesystem
-  public static final ConfigProperty<Boolean> METADATA_VALIDATE_PROP = ConfigProperty
+  public static final ConfigProperty<Boolean> VALIDATE_ENABLE = ConfigProperty
       .key(METADATA_PREFIX + ".validate")
       .defaultValue(false)
       .sinceVersion("0.7.0")
@@ -61,68 +62,68 @@ public final class HoodieMetadataConfig extends HoodieConfig {
   public static final boolean DEFAULT_METADATA_ENABLE_FOR_READERS = false;
 
   // Enable metrics for internal Metadata Table
-  public static final ConfigProperty<Boolean> METADATA_METRICS_ENABLE_PROP = ConfigProperty
+  public static final ConfigProperty<Boolean> METRICS_ENABLE = ConfigProperty
       .key(METADATA_PREFIX + ".metrics.enable")
       .defaultValue(false)
       .sinceVersion("0.7.0")
       .withDocumentation("Enable publishing of metrics around metadata table.");
 
   // Parallelism for inserts
-  public static final ConfigProperty<Integer> METADATA_INSERT_PARALLELISM_PROP = ConfigProperty
+  public static final ConfigProperty<Integer> INSERT_PARALLELISM_VALUE = ConfigProperty
       .key(METADATA_PREFIX + ".insert.parallelism")
       .defaultValue(1)
       .sinceVersion("0.7.0")
       .withDocumentation("Parallelism to use when inserting to the metadata table");
 
   // Async clean
-  public static final ConfigProperty<Boolean> METADATA_ASYNC_CLEAN_PROP = ConfigProperty
+  public static final ConfigProperty<Boolean> ASYNC_CLEAN_ENABLE = ConfigProperty
       .key(METADATA_PREFIX + ".clean.async")
       .defaultValue(false)
       .sinceVersion("0.7.0")
       .withDocumentation("Enable asynchronous cleaning for metadata table");
 
   // Maximum delta commits before compaction occurs
-  public static final ConfigProperty<Integer> METADATA_COMPACT_NUM_DELTA_COMMITS_PROP = ConfigProperty
+  public static final ConfigProperty<Integer> COMPACT_NUM_DELTA_COMMITS = ConfigProperty
       .key(METADATA_PREFIX + ".compact.max.delta.commits")
       .defaultValue(24)
       .sinceVersion("0.7.0")
       .withDocumentation("Controls how often the metadata table is compacted.");
 
   // Archival settings
-  public static final ConfigProperty<Integer> MIN_COMMITS_TO_KEEP_PROP = ConfigProperty
+  public static final ConfigProperty<Integer> MIN_COMMITS_TO_KEEP = ConfigProperty
       .key(METADATA_PREFIX + ".keep.min.commits")
       .defaultValue(20)
       .sinceVersion("0.7.0")
       .withDocumentation("Controls the archival of the metadata table’s timeline.");
 
-  public static final ConfigProperty<Integer> MAX_COMMITS_TO_KEEP_PROP = ConfigProperty
+  public static final ConfigProperty<Integer> MAX_COMMITS_TO_KEEP = ConfigProperty
       .key(METADATA_PREFIX + ".keep.max.commits")
       .defaultValue(30)
       .sinceVersion("0.7.0")
       .withDocumentation("Controls the archival of the metadata table’s timeline.");
 
   // Cleaner commits retained
-  public static final ConfigProperty<Integer> CLEANER_COMMITS_RETAINED_PROP = ConfigProperty
+  public static final ConfigProperty<Integer> CLEANER_COMMITS_RETAINED = ConfigProperty
       .key(METADATA_PREFIX + ".cleaner.commits.retained")
       .defaultValue(3)
       .sinceVersion("0.7.0")
       .withDocumentation("Controls retention/history for metadata table.");
 
   // Regex to filter out matching directories during bootstrap
-  public static final ConfigProperty<String> DIRECTORY_FILTER_REGEX = ConfigProperty
+  public static final ConfigProperty<String> DIR_FILTER_REGEX = ConfigProperty
       .key(METADATA_PREFIX + ".dir.filter.regex")
       .defaultValue("")
       .sinceVersion("0.7.0")
       .withDocumentation("Directories matching this regex, will be filtered out when initializing metadata table from lake storage for the first time.");
 
-  public static final ConfigProperty<String> HOODIE_ASSUME_DATE_PARTITIONING_PROP = ConfigProperty
+  public static final ConfigProperty<String> ASSUME_DATE_PARTITIONING = ConfigProperty
       .key("hoodie.assume.date.partitioning")
       .defaultValue("false")
       .sinceVersion("0.3.0")
       .withDocumentation("Should HoodieWriteClient assume the data is partitioned by dates, i.e three levels from base path. "
           + "This is a stop-gap to support tables created by versions < 0.3.1. Will be removed eventually");
 
-  public static final ConfigProperty<Integer> FILE_LISTING_PARALLELISM_PROP = ConfigProperty
+  public static final ConfigProperty<Integer> FILE_LISTING_PARALLELISM_VALUE = ConfigProperty
       .key("hoodie.file.listing.parallelism")
       .defaultValue(1500)
       .sinceVersion("0.7.0")
@@ -137,31 +138,31 @@ public final class HoodieMetadataConfig extends HoodieConfig {
   }
 
   public int getFileListingParallelism() {
-    return Math.max(getInt(HoodieMetadataConfig.FILE_LISTING_PARALLELISM_PROP), 1);
+    return Math.max(getInt(HoodieMetadataConfig.FILE_LISTING_PARALLELISM_VALUE), 1);
   }
 
   public Boolean shouldAssumeDatePartitioning() {
-    return getBoolean(HoodieMetadataConfig.HOODIE_ASSUME_DATE_PARTITIONING_PROP);
+    return getBoolean(HoodieMetadataConfig.ASSUME_DATE_PARTITIONING);
   }
 
   public boolean enabled() {
-    return getBoolean(METADATA_ENABLE_PROP);
+    return getBoolean(ENABLE);
   }
 
   public boolean enableSync() {
-    return enabled() && getBoolean(HoodieMetadataConfig.METADATA_SYNC_ENABLE_PROP);
+    return enabled() && getBoolean(HoodieMetadataConfig.SYNC_ENABLE);
   }
 
   public boolean validateFileListingMetadata() {
-    return getBoolean(METADATA_VALIDATE_PROP);
+    return getBoolean(VALIDATE_ENABLE);
   }
 
   public boolean enableMetrics() {
-    return getBoolean(METADATA_METRICS_ENABLE_PROP);
+    return getBoolean(METRICS_ENABLE);
   }
 
   public String getDirectoryFilterRegex() {
-    return getString(DIRECTORY_FILTER_REGEX);
+    return getString(DIR_FILTER_REGEX);
   }
 
   public static class Builder {
@@ -181,63 +182,63 @@ public final class HoodieMetadataConfig extends HoodieConfig {
     }
 
     public Builder enable(boolean enable) {
-      metadataConfig.setValue(METADATA_ENABLE_PROP, String.valueOf(enable));
+      metadataConfig.setValue(ENABLE, String.valueOf(enable));
       return this;
     }
 
     public Builder enableSync(boolean enable) {
-      metadataConfig.setValue(METADATA_SYNC_ENABLE_PROP, String.valueOf(enable));
+      metadataConfig.setValue(SYNC_ENABLE, String.valueOf(enable));
       return this;
     }
 
     public Builder enableMetrics(boolean enableMetrics) {
-      metadataConfig.setValue(METADATA_METRICS_ENABLE_PROP, String.valueOf(enableMetrics));
+      metadataConfig.setValue(METRICS_ENABLE, String.valueOf(enableMetrics));
       return this;
     }
 
     public Builder validate(boolean validate) {
-      metadataConfig.setValue(METADATA_VALIDATE_PROP, String.valueOf(validate));
+      metadataConfig.setValue(VALIDATE_ENABLE, String.valueOf(validate));
       return this;
     }
 
     public Builder withInsertParallelism(int parallelism) {
-      metadataConfig.setValue(METADATA_INSERT_PARALLELISM_PROP, String.valueOf(parallelism));
+      metadataConfig.setValue(INSERT_PARALLELISM_VALUE, String.valueOf(parallelism));
       return this;
     }
 
     public Builder withAsyncClean(boolean asyncClean) {
-      metadataConfig.setValue(METADATA_ASYNC_CLEAN_PROP, String.valueOf(asyncClean));
+      metadataConfig.setValue(ASYNC_CLEAN_ENABLE, String.valueOf(asyncClean));
       return this;
     }
 
     public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) {
-      metadataConfig.setValue(METADATA_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(maxNumDeltaCommitsBeforeCompaction));
+      metadataConfig.setValue(COMPACT_NUM_DELTA_COMMITS, String.valueOf(maxNumDeltaCommitsBeforeCompaction));
       return this;
     }
 
     public Builder archiveCommitsWith(int minToKeep, int maxToKeep) {
-      metadataConfig.setValue(MIN_COMMITS_TO_KEEP_PROP, String.valueOf(minToKeep));
-      metadataConfig.setValue(MAX_COMMITS_TO_KEEP_PROP, String.valueOf(maxToKeep));
+      metadataConfig.setValue(MIN_COMMITS_TO_KEEP, String.valueOf(minToKeep));
+      metadataConfig.setValue(MAX_COMMITS_TO_KEEP, String.valueOf(maxToKeep));
       return this;
     }
 
     public Builder retainCommits(int commitsRetained) {
-      metadataConfig.setValue(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
+      metadataConfig.setValue(CLEANER_COMMITS_RETAINED, String.valueOf(commitsRetained));
       return this;
     }
 
     public Builder withFileListingParallelism(int parallelism) {
-      metadataConfig.setValue(FILE_LISTING_PARALLELISM_PROP, String.valueOf(parallelism));
+      metadataConfig.setValue(FILE_LISTING_PARALLELISM_VALUE, String.valueOf(parallelism));
       return this;
     }
 
     public Builder withAssumeDatePartitioning(boolean assumeDatePartitioning) {
-      metadataConfig.setValue(HOODIE_ASSUME_DATE_PARTITIONING_PROP, String.valueOf(assumeDatePartitioning));
+      metadataConfig.setValue(ASSUME_DATE_PARTITIONING, String.valueOf(assumeDatePartitioning));
       return this;
     }
 
     public Builder withDirectoryFilterRegex(String regex) {
-      metadataConfig.setValue(DIRECTORY_FILTER_REGEX, regex);
+      metadataConfig.setValue(DIR_FILTER_REGEX, regex);
       return this;
     }
 
@@ -246,4 +247,140 @@ public final class HoodieMetadataConfig extends HoodieConfig {
       return metadataConfig;
     }
   }
+
+  /**
+   * @deprecated Use {@link #ENABLE} and its methods.
+   */
+  @Deprecated
+  public static final String METADATA_ENABLE_PROP = ENABLE.key();
+  /**
+   * @deprecated Use {@link #ENABLE} and its methods.
+   */
+  @Deprecated
+  public static final boolean DEFAULT_METADATA_ENABLE = ENABLE.defaultValue();
+  /**
+   * @deprecated Use {@link #VALIDATE_ENABLE} and its methods.
+   */
+  @Deprecated
+  public static final String METADATA_VALIDATE_PROP = VALIDATE_ENABLE.key();
+  /**
+   * @deprecated Use {@link #VALIDATE_ENABLE} and its methods.
+   */
+  @Deprecated
+  public static final boolean DEFAULT_METADATA_VALIDATE = VALIDATE_ENABLE.defaultValue();
+
+  /**
+   * @deprecated Use {@link #METRICS_ENABLE} and its methods.
+   */
+  @Deprecated
+  public static final String METADATA_METRICS_ENABLE_PROP = METRICS_ENABLE.key();
+  /**
+   * @deprecated Use {@link #METRICS_ENABLE} and its methods.
+   */
+  @Deprecated
+  public static final boolean DEFAULT_METADATA_METRICS_ENABLE = METRICS_ENABLE.defaultValue();
+
+  /**
+   * @deprecated Use {@link #INSERT_PARALLELISM_VALUE} and its methods.
+   */
+  @Deprecated
+  public static final String METADATA_INSERT_PARALLELISM_PROP = INSERT_PARALLELISM_VALUE.key();
+  /**
+   * @deprecated Use {@link #INSERT_PARALLELISM_VALUE} and its methods.
+   */
+  @Deprecated
+  public static final int DEFAULT_METADATA_INSERT_PARALLELISM = INSERT_PARALLELISM_VALUE.defaultValue();
+
+  /**
+   * @deprecated Use {@link #ASYNC_CLEAN_ENABLE} and its methods.
+   */
+  @Deprecated
+  public static final String METADATA_ASYNC_CLEAN_PROP = ASYNC_CLEAN_ENABLE.key();
+  /**
+   * @deprecated Use {@link #ASYNC_CLEAN_ENABLE} and its methods.
+   */
+  @Deprecated
+  public static final boolean DEFAULT_METADATA_ASYNC_CLEAN = ASYNC_CLEAN_ENABLE.defaultValue();
+
+  /**
+   * @deprecated Use {@link #COMPACT_NUM_DELTA_COMMITS} and its methods.
+   */
+  @Deprecated
+  public static final String METADATA_COMPACT_NUM_DELTA_COMMITS_PROP = COMPACT_NUM_DELTA_COMMITS.key();
+  /**
+   * @deprecated Use {@link #COMPACT_NUM_DELTA_COMMITS} and its methods.
+   */
+  @Deprecated
+  public static final int DEFAULT_METADATA_COMPACT_NUM_DELTA_COMMITS = COMPACT_NUM_DELTA_COMMITS.defaultValue();
+
+  /**
+   * @deprecated Use {@link #MIN_COMMITS_TO_KEEP} and its methods.
+   */
+  @Deprecated
+  public static final String MIN_COMMITS_TO_KEEP_PROP = MIN_COMMITS_TO_KEEP.key();
+  /**
+   * @deprecated Use {@link #MIN_COMMITS_TO_KEEP} and its methods.
+   */
+  @Deprecated
+  public static final int DEFAULT_MIN_COMMITS_TO_KEEP = MIN_COMMITS_TO_KEEP.defaultValue();
+  /**
+   * @deprecated Use {@link #MAX_COMMITS_TO_KEEP} and its methods.
+   */
+  @Deprecated
+  public static final String MAX_COMMITS_TO_KEEP_PROP = MAX_COMMITS_TO_KEEP.key();
+  /**
+   * @deprecated Use {@link #MAX_COMMITS_TO_KEEP} and its methods.
+   */
+  @Deprecated
+  public static final int DEFAULT_MAX_COMMITS_TO_KEEP = MAX_COMMITS_TO_KEEP.defaultValue();
+  /**
+   * @deprecated Use {@link #CLEANER_COMMITS_RETAINED} and its methods.
+   */
+  @Deprecated
+  public static final String CLEANER_COMMITS_RETAINED_PROP = CLEANER_COMMITS_RETAINED.key();
+  /**
+   * @deprecated Use {@link #CLEANER_COMMITS_RETAINED} and its methods.
+   */
+  @Deprecated
+  public static final int DEFAULT_CLEANER_COMMITS_RETAINED = CLEANER_COMMITS_RETAINED.defaultValue();
+  /**
+   * @deprecated No longer takes any effect.
+   */
+  @Deprecated
+  public static final String ENABLE_FALLBACK_PROP = METADATA_PREFIX + ".fallback.enable";
+  /**
+   * @deprecated No longer takes any effect.
+   */
+  @Deprecated
+  public static final String DEFAULT_ENABLE_FALLBACK = "true";
+  /**
+   * @deprecated Use {@link #DIR_FILTER_REGEX} and its methods.
+   */
+  @Deprecated
+  public static final String DIRECTORY_FILTER_REGEX = DIR_FILTER_REGEX.key();
+  /**
+   * @deprecated Use {@link #DIR_FILTER_REGEX} and its methods.
+   */
+  @Deprecated
+  public static final String DEFAULT_DIRECTORY_FILTER_REGEX = DIR_FILTER_REGEX.defaultValue();
+  /**
+   * @deprecated Use {@link #ASSUME_DATE_PARTITIONING} and its methods.
+   */
+  @Deprecated
+  public static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = ASSUME_DATE_PARTITIONING.key();
+  /**
+   * @deprecated Use {@link #ASSUME_DATE_PARTITIONING} and its methods.
+   */
+  @Deprecated
+  public static final String DEFAULT_ASSUME_DATE_PARTITIONING = ASSUME_DATE_PARTITIONING.defaultValue();
+  /**
+   * @deprecated Use {@link #FILE_LISTING_PARALLELISM_VALUE} and its methods.
+   */
+  @Deprecated
+  public static final String FILE_LISTING_PARALLELISM_PROP = FILE_LISTING_PARALLELISM_VALUE.key();
+  /**
+   * @deprecated Use {@link #FILE_LISTING_PARALLELISM_VALUE} and its methods.
+   */
+  @Deprecated
+  public static final int DEFAULT_FILE_LISTING_PARALLELISM = FILE_LISTING_PARALLELISM_VALUE.defaultValue();
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/LockConfiguration.java b/hudi-common/src/main/java/org/apache/hudi/common/config/LockConfiguration.java
index 8f16149..41d8cd7 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/config/LockConfiguration.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/config/LockConfiguration.java
@@ -74,6 +74,60 @@ public class LockConfiguration implements Serializable {
 
   public static final String ZK_LOCK_KEY_PROP_KEY = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "lock_key";
 
+  /** @deprecated Use {@link #LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY} */
+  @Deprecated
+  public static final String LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP = LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
+  /** @deprecated Use {@link #LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP_KEY} */
+  @Deprecated
+  public static final String LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP = LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP_KEY;
+  @Deprecated
+  public static final String DEFAULT_LOCK_ACQUIRE_MAX_RETRY_WAIT_TIME_IN_MILLIS = String.valueOf(5000L);
+  /** @deprecated Use {@link #LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY} */
+  @Deprecated
+  public static final String LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP = LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
+  @Deprecated
+  public static final String DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS = String.valueOf(10000L);
+  /** @deprecated Use {@link #LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY} */
+  @Deprecated
+  public static final String LOCK_ACQUIRE_NUM_RETRIES_PROP = LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY;
+  /** @deprecated Use {@link #LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY} */
+  @Deprecated
+  public static final String LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP = LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY;
+  @Deprecated
+  public static final String DEFAULT_LOCK_ACQUIRE_CLIENT_NUM_RETRIES = String.valueOf(0);
+  /** @deprecated Use {@link #LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY} */
+  @Deprecated
+  public static final String LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP = LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY;
+  @Deprecated
+  public static final int DEFAULT_ACQUIRE_LOCK_WAIT_TIMEOUT_MS = 60 * 1000;
+  /** @deprecated Use {@link #HIVE_DATABASE_NAME_PROP_KEY} */
+  @Deprecated
+  public static final String HIVE_DATABASE_NAME_PROP = HIVE_DATABASE_NAME_PROP_KEY;
+  /** @deprecated Use {@link #HIVE_TABLE_NAME_PROP_KEY} */
+  @Deprecated
+  public static final String HIVE_TABLE_NAME_PROP = HIVE_TABLE_NAME_PROP_KEY;
+  /** @deprecated Use {@link #HIVE_METASTORE_URI_PROP_KEY} */
+  @Deprecated
+  public static final String HIVE_METASTORE_URI_PROP = HIVE_METASTORE_URI_PROP_KEY;
+  /** @deprecated Use {@link #ZK_BASE_PATH_PROP_KEY} */
+  @Deprecated
+  public static final String ZK_BASE_PATH_PROP = ZK_BASE_PATH_PROP_KEY;
+  /** @deprecated Use {@link #ZK_SESSION_TIMEOUT_MS_PROP_KEY} */
+  @Deprecated
+  public static final String ZK_SESSION_TIMEOUT_MS_PROP = ZK_SESSION_TIMEOUT_MS_PROP_KEY;
+  /** @deprecated Use {@link #ZK_CONNECTION_TIMEOUT_MS_PROP_KEY} */
+  @Deprecated
+  public static final String ZK_CONNECTION_TIMEOUT_MS_PROP = ZK_CONNECTION_TIMEOUT_MS_PROP_KEY;
+  /** @deprecated Use {@link #ZK_CONNECT_URL_PROP_KEY} */
+  @Deprecated
+  public static final String ZK_CONNECT_URL_PROP = ZK_CONNECT_URL_PROP_KEY;
+  /** @deprecated Use {@link #ZK_PORT_PROP_KEY} */
+  @Deprecated
+  public static final String ZK_PORT_PROP = ZK_PORT_PROP_KEY;
+  /** @deprecated Use {@link #ZK_LOCK_KEY_PROP_KEY} */
+  @Deprecated
+  public static final String ZK_LOCK_KEY_PROP = ZK_LOCK_KEY_PROP_KEY;
+
   private final TypedProperties props;
 
   public LockConfiguration(Properties props) {
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/ConsistencyGuardConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/ConsistencyGuardConfig.java
index 74830a4..323e41f 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/fs/ConsistencyGuardConfig.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/ConsistencyGuardConfig.java
@@ -37,7 +37,7 @@ import java.util.Properties;
         + "(Tip: S3 is NOT eventually consistent anymore!)")
 public class ConsistencyGuardConfig extends HoodieConfig {
 
-  public static final ConfigProperty<String> CONSISTENCY_CHECK_ENABLED_PROP = ConfigProperty
+  public static final ConfigProperty<String> ENABLE = ConfigProperty
       .key("hoodie.consistency.check.enabled")
       .defaultValue("false")
       .sinceVersion("0.5.0")
@@ -45,14 +45,14 @@ public class ConsistencyGuardConfig extends HoodieConfig {
       .withDocumentation("Enabled to handle S3 eventual consistency issue. This property is no longer required "
           + "since S3 is now strongly consistent. Will be removed in the future releases.");
 
-  public static final ConfigProperty<Long> INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP = ConfigProperty
+  public static final ConfigProperty<Long> INITIAL_CHECK_INTERVAL_MS = ConfigProperty
       .key("hoodie.consistency.check.initial_interval_ms")
       .defaultValue(400L)
       .sinceVersion("0.5.0")
       .deprecatedAfter("0.7.0")
       .withDocumentation("Amount of time (in ms) to wait, before checking for consistency after an operation on storage.");
 
-  public static final ConfigProperty<Long> MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = ConfigProperty
+  public static final ConfigProperty<Long> MAX_CHECK_INTERVAL_MS = ConfigProperty
       .key("hoodie.consistency.check.max_interval_ms")
       .defaultValue(20000L)
       .sinceVersion("0.5.0")
@@ -60,7 +60,7 @@ public class ConsistencyGuardConfig extends HoodieConfig {
       .withDocumentation("Maximum amount of time (in ms), to wait for consistency checking.");
 
   // maximum number of checks, for consistency of written data. Will wait upto 140 Secs
-  public static final ConfigProperty<Integer> MAX_CONSISTENCY_CHECKS_PROP = ConfigProperty
+  public static final ConfigProperty<Integer> MAX_CHECKS = ConfigProperty
       .key("hoodie.consistency.check.max_checks")
       .defaultValue(6)
       .sinceVersion("0.5.0")
@@ -68,14 +68,14 @@ public class ConsistencyGuardConfig extends HoodieConfig {
       .withDocumentation("Maximum number of consistency checks to perform, with exponential backoff.");
 
   // sleep time for OptimisticConsistencyGuard
-  public static final ConfigProperty<Long> OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP = ConfigProperty
+  public static final ConfigProperty<Long> OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS = ConfigProperty
       .key("hoodie.optimistic.consistency.guard.sleep_time_ms")
       .defaultValue(500L)
       .sinceVersion("0.6.0")
       .withDocumentation("Amount of time (in ms), to wait after which we assume storage is consistent.");
 
   // config to enable OptimisticConsistencyGuard in finalizeWrite instead of FailSafeConsistencyGuard
-  public static final ConfigProperty<Boolean> ENABLE_OPTIMISTIC_CONSISTENCY_GUARD_PROP = ConfigProperty
+  public static final ConfigProperty<Boolean> OPTIMISTIC_CONSISTENCY_GUARD_ENABLE = ConfigProperty
       .key("_hoodie.optimistic.consistency.guard.enable")
       .defaultValue(false)
       .sinceVersion("0.6.0")
@@ -90,27 +90,27 @@ public class ConsistencyGuardConfig extends HoodieConfig {
   }
 
   public boolean isConsistencyCheckEnabled() {
-    return getBoolean(CONSISTENCY_CHECK_ENABLED_PROP);
+    return getBoolean(ENABLE);
   }
 
   public int getMaxConsistencyChecks() {
-    return getInt(MAX_CONSISTENCY_CHECKS_PROP);
+    return getInt(MAX_CHECKS);
   }
 
   public int getInitialConsistencyCheckIntervalMs() {
-    return getInt(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP);
+    return getInt(INITIAL_CHECK_INTERVAL_MS);
   }
 
   public int getMaxConsistencyCheckIntervalMs() {
-    return getInt(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP);
+    return getInt(MAX_CHECK_INTERVAL_MS);
   }
 
   public long getOptimisticConsistencyGuardSleepTimeMs() {
-    return getLong(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP);
+    return getLong(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS);
   }
 
   public boolean shouldEnableOptimisticConsistencyGuard() {
-    return getBoolean(ENABLE_OPTIMISTIC_CONSISTENCY_GUARD_PROP);
+    return getBoolean(OPTIMISTIC_CONSISTENCY_GUARD_ENABLE);
   }
 
   /**
@@ -133,32 +133,32 @@ public class ConsistencyGuardConfig extends HoodieConfig {
     }
 
     public Builder withConsistencyCheckEnabled(boolean enabled) {
-      consistencyGuardConfig.setValue(CONSISTENCY_CHECK_ENABLED_PROP, String.valueOf(enabled));
+      consistencyGuardConfig.setValue(ENABLE, String.valueOf(enabled));
       return this;
     }
 
     public Builder withInitialConsistencyCheckIntervalMs(int initialIntevalMs) {
-      consistencyGuardConfig.setValue(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(initialIntevalMs));
+      consistencyGuardConfig.setValue(INITIAL_CHECK_INTERVAL_MS, String.valueOf(initialIntevalMs));
       return this;
     }
 
     public Builder withMaxConsistencyCheckIntervalMs(int maxIntervalMs) {
-      consistencyGuardConfig.setValue(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(maxIntervalMs));
+      consistencyGuardConfig.setValue(MAX_CHECK_INTERVAL_MS, String.valueOf(maxIntervalMs));
       return this;
     }
 
     public Builder withMaxConsistencyChecks(int maxConsistencyChecks) {
-      consistencyGuardConfig.setValue(MAX_CONSISTENCY_CHECKS_PROP, String.valueOf(maxConsistencyChecks));
+      consistencyGuardConfig.setValue(MAX_CHECKS, String.valueOf(maxConsistencyChecks));
       return this;
     }
 
     public Builder withOptimisticConsistencyGuardSleepTimeMs(long sleepTimeMs) {
-      consistencyGuardConfig.setValue(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP, String.valueOf(sleepTimeMs));
+      consistencyGuardConfig.setValue(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS, String.valueOf(sleepTimeMs));
       return this;
     }
 
     public Builder withEnableOptimisticConsistencyGuard(boolean enableOptimisticConsistencyGuard) {
-      consistencyGuardConfig.setValue(ENABLE_OPTIMISTIC_CONSISTENCY_GUARD_PROP, String.valueOf(enableOptimisticConsistencyGuard));
+      consistencyGuardConfig.setValue(OPTIMISTIC_CONSISTENCY_GUARD_ENABLE, String.valueOf(enableOptimisticConsistencyGuard));
       return this;
     }
 
@@ -167,4 +167,65 @@ public class ConsistencyGuardConfig extends HoodieConfig {
       return consistencyGuardConfig;
     }
   }
+
+  /**
+   * @deprecated use {@link #ENABLE} and its methods.
+   */
+  @Deprecated
+  private static final String CONSISTENCY_CHECK_ENABLED_PROP = ENABLE.key();
+  /**
+   * @deprecated use {@link #ENABLE} and its methods.
+   */
+  @Deprecated
+  private static final String DEFAULT_CONSISTENCY_CHECK_ENABLED = ENABLE.defaultValue();
+  /**
+   * @deprecated use {@link #INITIAL_CHECK_INTERVAL_MS} and its methods.
+   */
+  @Deprecated
+  private static final String INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP = INITIAL_CHECK_INTERVAL_MS.key();
+  /**
+   * @deprecated use {@link #INITIAL_CHECK_INTERVAL_MS} and its methods.
+   */
+  @Deprecated
+  private static long DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = INITIAL_CHECK_INTERVAL_MS.defaultValue();
+  /**
+   * @deprecated use {@link #MAX_CHECK_INTERVAL_MS} and its methods.
+   */
+  @Deprecated
+  private static final String MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = MAX_CHECK_INTERVAL_MS.key();
+  /**
+   * @deprecated use {@link #MAX_CHECK_INTERVAL_MS} and its methods.
+   */
+  @Deprecated
+  private static long DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS = MAX_CHECK_INTERVAL_MS.defaultValue();
+  /**
+   * @deprecated use {@link #MAX_CHECKS} and its methods.
+   */
+  @Deprecated
+  private static final String MAX_CONSISTENCY_CHECKS_PROP = MAX_CHECKS.key();
+  /**
+   * @deprecated use {@link #MAX_CHECKS} and its methods.
+   */
+  @Deprecated
+  private static int DEFAULT_MAX_CONSISTENCY_CHECKS = MAX_CHECKS.defaultValue();
+  /**
+   * @deprecated use {@link #OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS} and its methods.
+   */
+  @Deprecated
+  private static final String OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP = OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS.key();
+  /**
+   * @deprecated use {@link #OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS} and its methods.
+   */
+  @Deprecated
+  private static long DEFAULT_OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP = OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS.defaultValue();
+  /**
+   * @deprecated use {@link #OPTIMISTIC_CONSISTENCY_GUARD_ENABLE} and its methods.
+   */
+  @Deprecated
+  private static final String ENABLE_OPTIMISTIC_CONSISTENCY_GUARD = OPTIMISTIC_CONSISTENCY_GUARD_ENABLE.key();
+  /**
+   * @deprecated use {@link #OPTIMISTIC_CONSISTENCY_GUARD_ENABLE} and its methods.
+   */
+  @Deprecated
+  private static boolean DEFAULT_ENABLE_OPTIMISTIC_CONSISTENCY_GUARD = OPTIMISTIC_CONSISTENCY_GUARD_ENABLE.defaultValue();
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java
index 8088396..7e39053 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java
@@ -18,14 +18,6 @@
 
 package org.apache.hudi.common.fs;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.LocatedFileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.PathFilter;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hudi.common.config.HoodieMetadataConfig;
 import org.apache.hudi.common.config.SerializableConfiguration;
 import org.apache.hudi.common.engine.HoodieEngineContext;
@@ -42,6 +34,15 @@ import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.exception.InvalidHoodiePathException;
 import org.apache.hudi.metadata.HoodieTableMetadata;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
@@ -141,7 +142,7 @@ public class FSUtils {
   // TODO: this should be removed
   public static String makeDataFileName(String instantTime, String writeToken, String fileId) {
     return String.format("%s_%s_%s%s", fileId, writeToken, instantTime,
-        HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension());
+        HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension());
   }
 
   public static String makeDataFileName(String instantTime, String writeToken, String fileId, String fileExtension) {
@@ -153,7 +154,7 @@ public class FSUtils {
   }
 
   public static String maskWithoutFileId(String instantTime, int taskPartitionId) {
-    return String.format("*_%s_%s%s", taskPartitionId, instantTime, HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP
+    return String.format("*_%s_%s%s", taskPartitionId, instantTime, HoodieTableConfig.BASE_FILE_FORMAT
         .defaultValue().getFileExtension());
   }
 
@@ -271,7 +272,7 @@ public class FSUtils {
         .withAssumeDatePartitioning(assumeDatePartitioning)
         .build();
     try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr,
-        FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue())) {
+        FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue())) {
       return tableMetadata.getAllPartitionPaths();
     } catch (Exception e) {
       throw new HoodieException("Error fetching partition paths from metadata table", e);
@@ -281,7 +282,7 @@ public class FSUtils {
   public static List<String> getAllPartitionPaths(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
                                                   String basePathStr) {
     try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr,
-        FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue())) {
+        FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue())) {
       return tableMetadata.getAllPartitionPaths();
     } catch (Exception e) {
       throw new HoodieException("Error fetching partition paths from metadata table", e);
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePayloadProps.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePayloadProps.java
index 24bf3d7..b04b8df 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePayloadProps.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePayloadProps.java
@@ -48,4 +48,15 @@ public class HoodiePayloadProps {
    * to tell if the current record is a update record or insert record for mor table.
    */
   public static final String PAYLOAD_IS_UPDATE_RECORD_FOR_MOR = "hoodie.is.update.record.for.mor";
+
+  /** @deprecated Use {@link #PAYLOAD_ORDERING_FIELD_PROP_KEY} */
+  @Deprecated
+  public static final String PAYLOAD_ORDERING_FIELD_PROP = PAYLOAD_ORDERING_FIELD_PROP_KEY;
+  @Deprecated
+  public static String DEFAULT_PAYLOAD_ORDERING_FIELD_VAL = "ts";
+  /** @deprecated Use {@link #PAYLOAD_EVENT_TIME_FIELD_PROP_KEY} */
+  @Deprecated
+  public static final String PAYLOAD_EVENT_TIME_FIELD_PROP = PAYLOAD_EVENT_TIME_FIELD_PROP_KEY;
+  @Deprecated
+  public static String DEFAULT_PAYLOAD_EVENT_TIME_FIELD_VAL = "ts";
 }
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 853fffd..6d3a7d0 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
@@ -69,95 +69,95 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
 
   public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
 
-  public static final ConfigProperty<String> HOODIE_TABLE_NAME_PROP = ConfigProperty
+  public static final ConfigProperty<String> NAME = ConfigProperty
       .key("hoodie.table.name")
       .noDefaultValue()
       .withDocumentation("Table name that will be used for registering with Hive. Needs to be same across runs.");
 
-  public static final ConfigProperty<HoodieTableType> HOODIE_TABLE_TYPE_PROP = ConfigProperty
+  public static final ConfigProperty<HoodieTableType> TYPE = ConfigProperty
       .key("hoodie.table.type")
       .defaultValue(HoodieTableType.COPY_ON_WRITE)
       .withDocumentation("The table type for the underlying data, for this write. This can’t change between writes.");
 
-  public static final ConfigProperty<HoodieTableVersion> HOODIE_TABLE_VERSION_PROP = ConfigProperty
+  public static final ConfigProperty<HoodieTableVersion> VERSION = ConfigProperty
       .key("hoodie.table.version")
       .defaultValue(HoodieTableVersion.ZERO)
       .withDocumentation("Version of table, used for running upgrade/downgrade steps between releases with potentially "
           + "breaking/backwards compatible changes.");
 
-  public static final ConfigProperty<String> HOODIE_TABLE_PRECOMBINE_FIELD_PROP = ConfigProperty
+  public static final ConfigProperty<String> PRECOMBINE_FIELD = ConfigProperty
       .key("hoodie.table.precombine.field")
       .noDefaultValue()
       .withDocumentation("Field used in preCombining before actual write. By default, when two records have the same key value, "
           + "the largest value for the precombine field determined by Object.compareTo(..), is picked.");
 
-  public static final ConfigProperty<String> HOODIE_TABLE_PARTITION_FIELDS_PROP = ConfigProperty
+  public static final ConfigProperty<String> PARTITION_FIELDS = ConfigProperty
       .key("hoodie.table.partition.fields")
       .noDefaultValue()
       .withDocumentation("Fields used to partition the table. Concatenated values of these fields are used as "
           + "the partition path, by invoking toString()");
 
-  public static final ConfigProperty<String> HOODIE_TABLE_RECORDKEY_FIELDS = ConfigProperty
+  public static final ConfigProperty<String> RECORDKEY_FIELDS = ConfigProperty
       .key("hoodie.table.recordkey.fields")
       .noDefaultValue()
       .withDocumentation("Columns used to uniquely identify the table. Concatenated values of these fields are used as "
           + " the record key component of HoodieKey.");
 
-  public static final ConfigProperty<String> HOODIE_TABLE_CREATE_SCHEMA = ConfigProperty
+  public static final ConfigProperty<String> CREATE_SCHEMA = ConfigProperty
       .key("hoodie.table.create.schema")
       .noDefaultValue()
       .withDocumentation("Schema used when creating the table, for the first time.");
 
-  public static final ConfigProperty<HoodieFileFormat> HOODIE_BASE_FILE_FORMAT_PROP = ConfigProperty
+  public static final ConfigProperty<HoodieFileFormat> BASE_FILE_FORMAT = ConfigProperty
       .key("hoodie.table.base.file.format")
       .defaultValue(HoodieFileFormat.PARQUET)
       .withAlternatives("hoodie.table.ro.file.format")
       .withDocumentation("Base file format to store all the base file data.");
 
-  public static final ConfigProperty<HoodieFileFormat> HOODIE_LOG_FILE_FORMAT_PROP = ConfigProperty
+  public static final ConfigProperty<HoodieFileFormat> LOG_FILE_FORMAT = ConfigProperty
       .key("hoodie.table.log.file.format")
       .defaultValue(HoodieFileFormat.HOODIE_LOG)
       .withAlternatives("hoodie.table.rt.file.format")
       .withDocumentation("Log format used for the delta logs.");
 
-  public static final ConfigProperty<String> HOODIE_TIMELINE_LAYOUT_VERSION_PROP = ConfigProperty
+  public static final ConfigProperty<String> TIMELINE_LAYOUT_VERSION = ConfigProperty
       .key("hoodie.timeline.layout.version")
       .noDefaultValue()
       .withDocumentation("Version of timeline used, by the table.");
 
-  public static final ConfigProperty<String> HOODIE_PAYLOAD_CLASS_PROP = ConfigProperty
+  public static final ConfigProperty<String> PAYLOAD_CLASS_NAME = ConfigProperty
       .key("hoodie.compaction.payload.class")
       .defaultValue(OverwriteWithLatestAvroPayload.class.getName())
       .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> HOODIE_ARCHIVELOG_FOLDER_PROP = ConfigProperty
+  public static final ConfigProperty<String> ARCHIVELOG_FOLDER = ConfigProperty
       .key("hoodie.archivelog.folder")
       .defaultValue("archived")
       .withDocumentation("path under the meta folder, to store archived timeline instants at.");
 
-  public static final ConfigProperty<String> HOODIE_BOOTSTRAP_INDEX_ENABLE_PROP = ConfigProperty
+  public static final ConfigProperty<String> BOOTSTRAP_INDEX_ENABLE = ConfigProperty
       .key("hoodie.bootstrap.index.enable")
       .noDefaultValue()
       .withDocumentation("Whether or not, this is a bootstrapped table, with bootstrap base data and an mapping index defined.");
 
-  public static final ConfigProperty<String> HOODIE_BOOTSTRAP_INDEX_CLASS_PROP = ConfigProperty
+  public static final ConfigProperty<String> BOOTSTRAP_INDEX_CLASS_NAME = ConfigProperty
       .key("hoodie.bootstrap.index.class")
       .defaultValue(HFileBootstrapIndex.class.getName())
       .withDocumentation("Implementation to use, for mapping base files to bootstrap base file, that contain actual data.");
 
-  public static final ConfigProperty<String> HOODIE_BOOTSTRAP_BASE_PATH_PROP = ConfigProperty
+  public static final ConfigProperty<String> BOOTSTRAP_BASE_PATH = ConfigProperty
       .key("hoodie.bootstrap.base.path")
       .noDefaultValue()
       .withDocumentation("Base path of the dataset that needs to be bootstrapped as a Hudi table");
 
-  public static final ConfigProperty<String> HOODIE_POPULATE_META_FIELDS = ConfigProperty
+  public static final ConfigProperty<String> POPULATE_META_FIELDS = ConfigProperty
       .key("hoodie.populate.meta.fields")
       .defaultValue("true")
       .withDocumentation("When enabled, populates all meta fields. When disabled, no meta fields are populated "
           + "and incremental queries will not be functional. This is only meant to be used for append only/immutable data for batch processing");
 
-  public static final ConfigProperty<String> HOODIE_TABLE_KEY_GENERATOR_CLASS = ConfigProperty
+  public static final ConfigProperty<String> KEY_GENERATOR_CLASS_NAME = ConfigProperty
       .key("hoodie.table.keygenerator.class")
       .noDefaultValue()
       .withDocumentation("Key Generator class property for the hoodie table");
@@ -172,9 +172,9 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
       try (FSDataInputStream inputStream = fs.open(propertyPath)) {
         props.load(inputStream);
       }
-      if (contains(HOODIE_PAYLOAD_CLASS_PROP) && payloadClassName != null
-          && !getString(HOODIE_PAYLOAD_CLASS_PROP).equals(payloadClassName)) {
-        setValue(HOODIE_PAYLOAD_CLASS_PROP, payloadClassName);
+      if (contains(PAYLOAD_CLASS_NAME) && payloadClassName != null
+          && !getString(PAYLOAD_CLASS_NAME).equals(payloadClassName)) {
+        setValue(PAYLOAD_CLASS_NAME, payloadClassName);
         try (FSDataOutputStream outputStream = fs.create(propertyPath)) {
           props.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
         }
@@ -182,7 +182,7 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
     } catch (IOException e) {
       throw new HoodieIOException("Could not load Hoodie properties from " + propertyPath, e);
     }
-    ValidationUtils.checkArgument(contains(HOODIE_TABLE_TYPE_PROP) && contains(HOODIE_TABLE_NAME_PROP),
+    ValidationUtils.checkArgument(contains(TYPE) && contains(NAME),
         "hoodie.properties file seems invalid. Please check for left over `.updated` files if any, manually copy it to hoodie.properties and retry");
   }
 
@@ -205,21 +205,21 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
     HoodieConfig hoodieConfig = new HoodieConfig(properties);
     Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
     try (FSDataOutputStream outputStream = fs.create(propertyPath)) {
-      if (!hoodieConfig.contains(HOODIE_TABLE_NAME_PROP)) {
-        throw new IllegalArgumentException(HOODIE_TABLE_NAME_PROP.key() + " property needs to be specified");
+      if (!hoodieConfig.contains(NAME)) {
+        throw new IllegalArgumentException(NAME.key() + " property needs to be specified");
       }
-      hoodieConfig.setDefaultValue(HOODIE_TABLE_TYPE_PROP);
-      if (hoodieConfig.getString(HOODIE_TABLE_TYPE_PROP).equals(HoodieTableType.MERGE_ON_READ.name())) {
-        hoodieConfig.setDefaultValue(HOODIE_PAYLOAD_CLASS_PROP);
+      hoodieConfig.setDefaultValue(TYPE);
+      if (hoodieConfig.getString(TYPE).equals(HoodieTableType.MERGE_ON_READ.name())) {
+        hoodieConfig.setDefaultValue(PAYLOAD_CLASS_NAME);
       }
-      hoodieConfig.setDefaultValue(HOODIE_ARCHIVELOG_FOLDER_PROP);
-      if (!hoodieConfig.contains(HOODIE_TIMELINE_LAYOUT_VERSION_PROP)) {
+      hoodieConfig.setDefaultValue(ARCHIVELOG_FOLDER);
+      if (!hoodieConfig.contains(TIMELINE_LAYOUT_VERSION)) {
         // Use latest Version as default unless forced by client
-        hoodieConfig.setValue(HOODIE_TIMELINE_LAYOUT_VERSION_PROP, TimelineLayoutVersion.CURR_VERSION.toString());
+        hoodieConfig.setValue(TIMELINE_LAYOUT_VERSION, TimelineLayoutVersion.CURR_VERSION.toString());
       }
-      if (hoodieConfig.contains(HOODIE_BOOTSTRAP_BASE_PATH_PROP)) {
+      if (hoodieConfig.contains(BOOTSTRAP_BASE_PATH)) {
         // Use the default bootstrap index class.
-        hoodieConfig.setDefaultValue(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP, getDefaultBootstrapIndexClass(properties));
+        hoodieConfig.setDefaultValue(BOOTSTRAP_INDEX_CLASS_NAME, getDefaultBootstrapIndexClass(properties));
       }
       hoodieConfig.getProps().store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
     }
@@ -229,12 +229,12 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
    * Read the table type from the table properties and if not found, return the default.
    */
   public HoodieTableType getTableType() {
-    return HoodieTableType.valueOf(getStringOrDefault(HOODIE_TABLE_TYPE_PROP));
+    return HoodieTableType.valueOf(getStringOrDefault(TYPE));
   }
 
   public Option<TimelineLayoutVersion> getTimelineLayoutVersion() {
-    return contains(HOODIE_TIMELINE_LAYOUT_VERSION_PROP)
-        ? Option.of(new TimelineLayoutVersion(getInt(HOODIE_TIMELINE_LAYOUT_VERSION_PROP)))
+    return contains(TIMELINE_LAYOUT_VERSION)
+        ? Option.of(new TimelineLayoutVersion(getInt(TIMELINE_LAYOUT_VERSION)))
         : Option.empty();
   }
 
@@ -242,13 +242,13 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
    * @return the hoodie.table.version from hoodie.properties file.
    */
   public HoodieTableVersion getTableVersion() {
-    return contains(HOODIE_TABLE_VERSION_PROP)
-        ? HoodieTableVersion.versionFromCode(getInt(HOODIE_TABLE_VERSION_PROP))
-        : HOODIE_TABLE_VERSION_PROP.defaultValue();
+    return contains(VERSION)
+        ? HoodieTableVersion.versionFromCode(getInt(VERSION))
+        : VERSION.defaultValue();
   }
 
   public void setTableVersion(HoodieTableVersion tableVersion) {
-    setValue(HOODIE_TABLE_VERSION_PROP, Integer.toString(tableVersion.versionCode()));
+    setValue(VERSION, Integer.toString(tableVersion.versionCode()));
   }
 
   /**
@@ -257,26 +257,26 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
   public String getPayloadClass() {
     // There could be tables written with payload class from com.uber.hoodie. Need to transparently
     // change to org.apache.hudi
-    return getStringOrDefault(HOODIE_PAYLOAD_CLASS_PROP).replace("com.uber.hoodie",
+    return getStringOrDefault(PAYLOAD_CLASS_NAME).replace("com.uber.hoodie",
         "org.apache.hudi");
   }
 
   public String getPreCombineField() {
-    return getString(HOODIE_TABLE_PRECOMBINE_FIELD_PROP);
+    return getString(PRECOMBINE_FIELD);
   }
 
   public Option<String[]> getRecordKeyFields() {
-    if (contains(HOODIE_TABLE_RECORDKEY_FIELDS)) {
-      return Option.of(Arrays.stream(getString(HOODIE_TABLE_RECORDKEY_FIELDS).split(","))
-              .filter(p -> p.length() > 0).collect(Collectors.toList()).toArray(new String[]{}));
+    if (contains(RECORDKEY_FIELDS)) {
+      return Option.of(Arrays.stream(getString(RECORDKEY_FIELDS).split(","))
+          .filter(p -> p.length() > 0).collect(Collectors.toList()).toArray(new String[] {}));
     }
     return Option.empty();
   }
 
   public Option<String[]> getPartitionFields() {
-    if (contains(HOODIE_TABLE_PARTITION_FIELDS_PROP)) {
-      return Option.of(Arrays.stream(getString(HOODIE_TABLE_PARTITION_FIELDS_PROP).split(","))
-        .filter(p -> p.length() > 0).collect(Collectors.toList()).toArray(new String[]{}));
+    if (contains(PARTITION_FIELDS)) {
+      return Option.of(Arrays.stream(getString(PARTITION_FIELDS).split(","))
+          .filter(p -> p.length() > 0).collect(Collectors.toList()).toArray(new String[] {}));
     }
     return Option.empty();
   }
@@ -285,7 +285,7 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
    * @returns the partition field prop.
    */
   public String getPartitionFieldProp() {
-    return getString(HOODIE_TABLE_PARTITION_FIELDS_PROP);
+    return getString(PARTITION_FIELDS);
   }
 
   /**
@@ -294,24 +294,24 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
   public String getBootstrapIndexClass() {
     // There could be tables written with payload class from com.uber.hoodie. Need to transparently
     // change to org.apache.hudi
-    return getStringOrDefault(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP, getDefaultBootstrapIndexClass(props));
+    return getStringOrDefault(BOOTSTRAP_INDEX_CLASS_NAME, getDefaultBootstrapIndexClass(props));
   }
 
   public static String getDefaultBootstrapIndexClass(Properties props) {
-    String defaultClass = HOODIE_BOOTSTRAP_INDEX_CLASS_PROP.defaultValue();
-    if ("false".equalsIgnoreCase(props.getProperty(HOODIE_BOOTSTRAP_INDEX_ENABLE_PROP.key()))) {
+    String defaultClass = BOOTSTRAP_INDEX_CLASS_NAME.defaultValue();
+    if ("false".equalsIgnoreCase(props.getProperty(BOOTSTRAP_INDEX_ENABLE.key()))) {
       defaultClass = NO_OP_BOOTSTRAP_INDEX_CLASS;
     }
     return defaultClass;
   }
 
   public Option<String> getBootstrapBasePath() {
-    return Option.ofNullable(getString(HOODIE_BOOTSTRAP_BASE_PATH_PROP));
+    return Option.ofNullable(getString(BOOTSTRAP_BASE_PATH));
   }
 
   public Option<Schema> getTableCreateSchema() {
-    if (contains(HOODIE_TABLE_CREATE_SCHEMA)) {
-      return Option.of(new Schema.Parser().parse(getString(HOODIE_TABLE_CREATE_SCHEMA)));
+    if (contains(CREATE_SCHEMA)) {
+      return Option.of(new Schema.Parser().parse(getString(CREATE_SCHEMA)));
     } else {
       return Option.empty();
     }
@@ -321,7 +321,7 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
    * Read the table name.
    */
   public String getTableName() {
-    return getString(HOODIE_TABLE_NAME_PROP);
+    return getString(NAME);
   }
 
   /**
@@ -330,7 +330,7 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
    * @return HoodieFileFormat for the base file Storage format
    */
   public HoodieFileFormat getBaseFileFormat() {
-    return HoodieFileFormat.valueOf(getStringOrDefault(HOODIE_BASE_FILE_FORMAT_PROP));
+    return HoodieFileFormat.valueOf(getStringOrDefault(BASE_FILE_FORMAT));
   }
 
   /**
@@ -339,32 +339,133 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
    * @return HoodieFileFormat for the log Storage format
    */
   public HoodieFileFormat getLogFileFormat() {
-    return HoodieFileFormat.valueOf(getStringOrDefault(HOODIE_LOG_FILE_FORMAT_PROP));
+    return HoodieFileFormat.valueOf(getStringOrDefault(LOG_FILE_FORMAT));
   }
 
   /**
    * Get the relative path of archive log folder under metafolder, for this table.
    */
   public String getArchivelogFolder() {
-    return getStringOrDefault(HOODIE_ARCHIVELOG_FOLDER_PROP);
+    return getStringOrDefault(ARCHIVELOG_FOLDER);
   }
 
   /**
    * @returns true is meta fields need to be populated. else returns false.
    */
   public boolean populateMetaFields() {
-    return Boolean.parseBoolean(getStringOrDefault(HOODIE_POPULATE_META_FIELDS));
+    return Boolean.parseBoolean(getStringOrDefault(POPULATE_META_FIELDS));
   }
 
   /**
    * @returns the record key field prop.
    */
   public String getRecordKeyFieldProp() {
-    return getString(HOODIE_TABLE_RECORDKEY_FIELDS);
+    return getString(RECORDKEY_FIELDS);
   }
 
   public Map<String, String> propsMap() {
     return props.entrySet().stream()
         .collect(Collectors.toMap(e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getValue())));
   }
+
+  /**
+   * @deprecated Use {@link #BASE_FILE_FORMAT} and its methods.
+   */
+  @Deprecated
+  public static final String HOODIE_RO_FILE_FORMAT_PROP_NAME = "hoodie.table.ro.file.format";
+  /**
+   * @deprecated Use {@link #LOG_FILE_FORMAT} and its methods.
+   */
+  @Deprecated
+  public static final String HOODIE_RT_FILE_FORMAT_PROP_NAME = "hoodie.table.rt.file.format";
+  /**
+   * @deprecated Use {@link #NAME} and its methods.
+   */
+  @Deprecated
+  public static final String HOODIE_TABLE_NAME_PROP_NAME = NAME.key();
+  /**
+   * @deprecated Use {@link #TYPE} and its methods.
+   */
+  @Deprecated
+  public static final String HOODIE_TABLE_TYPE_PROP_NAME = TYPE.key();
+  /**
+   * @deprecated Use {@link #VERSION} and its methods.
+   */
+  @Deprecated
+  public static final String HOODIE_TABLE_VERSION_PROP_NAME = VERSION.key();
+  /**
+   * @deprecated Use {@link #PRECOMBINE_FIELD} and its methods.
+   */
+  @Deprecated
+  public static final String HOODIE_TABLE_PRECOMBINE_FIELD = PRECOMBINE_FIELD.key();
+  /**
+   * @deprecated Use {@link #BASE_FILE_FORMAT} and its methods.
+   */
+  @Deprecated
+  public static final String HOODIE_BASE_FILE_FORMAT_PROP_NAME = BASE_FILE_FORMAT.key();
+  /**
+   * @deprecated Use {@link #LOG_FILE_FORMAT} and its methods.
+   */
+  @Deprecated
+  public static final String HOODIE_LOG_FILE_FORMAT_PROP_NAME = LOG_FILE_FORMAT.key();
+  /**
+   * @deprecated Use {@link #TIMELINE_LAYOUT_VERSION} and its methods.
+   */
+  @Deprecated
+  public static final String HOODIE_TIMELINE_LAYOUT_VERSION = TIMELINE_LAYOUT_VERSION.key();
+  /**
+   * @deprecated Use {@link #PAYLOAD_CLASS_NAME} and its methods.
+   */
+  @Deprecated
+  public static final String HOODIE_PAYLOAD_CLASS_PROP_NAME = PAYLOAD_CLASS_NAME.key();
+  /**
+   * @deprecated Use {@link #ARCHIVELOG_FOLDER} and its methods.
+   */
+  @Deprecated
+  public static final String HOODIE_ARCHIVELOG_FOLDER_PROP_NAME = ARCHIVELOG_FOLDER.key();
+  /**
+   * @deprecated Use {@link #BOOTSTRAP_INDEX_CLASS_NAME} and its methods.
+   */
+  @Deprecated
+  public static final String HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME = BOOTSTRAP_INDEX_CLASS_NAME.key();
+  /**
+   * @deprecated Use {@link #BOOTSTRAP_BASE_PATH} and its methods.
+   */
+  @Deprecated
+  public static final String HOODIE_BOOTSTRAP_BASE_PATH = BOOTSTRAP_BASE_PATH.key();
+  /**
+   * @deprecated Use {@link #TYPE} and its methods.
+   */
+  @Deprecated
+  public static final HoodieTableType DEFAULT_TABLE_TYPE = TYPE.defaultValue();
+  /**
+   * @deprecated Use {@link #VERSION} and its methods.
+   */
+  @Deprecated
+  public static final HoodieTableVersion DEFAULT_TABLE_VERSION = VERSION.defaultValue();
+  /**
+   * @deprecated Use {@link #BASE_FILE_FORMAT} and its methods.
+   */
+  @Deprecated
+  public static final HoodieFileFormat DEFAULT_BASE_FILE_FORMAT = BASE_FILE_FORMAT.defaultValue();
+  /**
+   * @deprecated Use {@link #LOG_FILE_FORMAT} and its methods.
+   */
+  @Deprecated
+  public static final HoodieFileFormat DEFAULT_LOG_FILE_FORMAT = LOG_FILE_FORMAT.defaultValue();
+  /**
+   * @deprecated Use {@link #PAYLOAD_CLASS_NAME} and its methods.
+   */
+  @Deprecated
+  public static final String DEFAULT_PAYLOAD_CLASS = PAYLOAD_CLASS_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #BOOTSTRAP_INDEX_CLASS_NAME} and its methods.
+   */
+  @Deprecated
+  public static final String DEFAULT_BOOTSTRAP_INDEX_CLASS = BOOTSTRAP_INDEX_CLASS_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #ARCHIVELOG_FOLDER} and its methods.
+   */
+  @Deprecated
+  public static final String DEFAULT_ARCHIVELOG_FOLDER = ARCHIVELOG_FOLDER.defaultValue();
 }
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 6899b93..e4494bd 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
@@ -227,7 +227,7 @@ public class HoodieTableMetaClient implements Serializable {
    */
   public String getArchivePath() {
     String archiveFolder = tableConfig.getArchivelogFolder();
-    if (archiveFolder.equals(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue())) {
+    if (archiveFolder.equals(HoodieTableConfig.ARCHIVELOG_FOLDER.defaultValue())) {
       return getMetaPath();
     } else {
       return getMetaPath() + "/" + archiveFolder;
@@ -321,16 +321,16 @@ public class HoodieTableMetaClient implements Serializable {
   public void validateTableProperties(Properties properties, WriteOperationType operationType) {
     // once meta fields are disabled, it cant be re-enabled for a given table.
     if (!getTableConfig().populateMetaFields()
-        && Boolean.parseBoolean((String) properties.getOrDefault(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()))) {
-      throw new HoodieException(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key() + " already disabled for the table. Can't be re-enabled back");
+        && Boolean.parseBoolean((String) properties.getOrDefault(HoodieTableConfig.POPULATE_META_FIELDS.key(), HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()))) {
+      throw new HoodieException(HoodieTableConfig.POPULATE_META_FIELDS.key() + " already disabled for the table. Can't be re-enabled back");
     }
 
     // meta fields can be disabled only with SimpleKeyGenerator
     if (!getTableConfig().populateMetaFields()
-        && !properties.getProperty(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS.key(), "org.apache.hudi.keygen.SimpleKeyGenerator")
+        && !properties.getProperty(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key(), "org.apache.hudi.keygen.SimpleKeyGenerator")
         .equals("org.apache.hudi.keygen.SimpleKeyGenerator")) {
       throw new HoodieException("Only simple key generator is supported when meta fields are disabled. KeyGenerator used : "
-          + properties.getProperty(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS.key()));
+          + properties.getProperty(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key()));
     }
   }
 
@@ -353,7 +353,7 @@ public class HoodieTableMetaClient implements Serializable {
     }
 
     // if anything other than default archive log folder is specified, create that too
-    String archiveLogPropVal = new HoodieConfig(props).getStringOrDefault(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP);
+    String archiveLogPropVal = new HoodieConfig(props).getStringOrDefault(HoodieTableConfig.ARCHIVELOG_FOLDER);
     if (!StringUtils.isNullOrEmpty(archiveLogPropVal)) {
       Path archiveLogDir = new Path(metaPathDir, archiveLogPropVal);
       if (!fs.exists(archiveLogDir)) {
@@ -725,52 +725,52 @@ public class HoodieTableMetaClient implements Serializable {
 
     public PropertyBuilder fromProperties(Properties properties) {
       HoodieConfig hoodieConfig = new HoodieConfig(properties);
-      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_NAME_PROP)) {
-        setTableName(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_NAME_PROP));
+      if (hoodieConfig.contains(HoodieTableConfig.NAME)) {
+        setTableName(hoodieConfig.getString(HoodieTableConfig.NAME));
       }
-      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP)) {
-        setTableType(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP));
+      if (hoodieConfig.contains(HoodieTableConfig.TYPE)) {
+        setTableType(hoodieConfig.getString(HoodieTableConfig.TYPE));
       }
-      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP)) {
+      if (hoodieConfig.contains(HoodieTableConfig.ARCHIVELOG_FOLDER)) {
         setArchiveLogFolder(
-            hoodieConfig.getString(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP));
+            hoodieConfig.getString(HoodieTableConfig.ARCHIVELOG_FOLDER));
       }
-      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP)) {
+      if (hoodieConfig.contains(HoodieTableConfig.PAYLOAD_CLASS_NAME)) {
         setPayloadClassName(
-            hoodieConfig.getString(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP));
+            hoodieConfig.getString(HoodieTableConfig.PAYLOAD_CLASS_NAME));
       }
-      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION_PROP)) {
-        setTimelineLayoutVersion(hoodieConfig.getInt(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION_PROP));
+      if (hoodieConfig.contains(HoodieTableConfig.TIMELINE_LAYOUT_VERSION)) {
+        setTimelineLayoutVersion(hoodieConfig.getInt(HoodieTableConfig.TIMELINE_LAYOUT_VERSION));
       }
-      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP)) {
+      if (hoodieConfig.contains(HoodieTableConfig.BASE_FILE_FORMAT)) {
         setBaseFileFormat(
-            hoodieConfig.getString(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP));
+            hoodieConfig.getString(HoodieTableConfig.BASE_FILE_FORMAT));
       }
-      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP)) {
+      if (hoodieConfig.contains(HoodieTableConfig.BOOTSTRAP_INDEX_CLASS_NAME)) {
         setBootstrapIndexClass(
-            hoodieConfig.getString(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP));
+            hoodieConfig.getString(HoodieTableConfig.BOOTSTRAP_INDEX_CLASS_NAME));
       }
-      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP)) {
-        setBootstrapBasePath(hoodieConfig.getString(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP));
+      if (hoodieConfig.contains(HoodieTableConfig.BOOTSTRAP_BASE_PATH)) {
+        setBootstrapBasePath(hoodieConfig.getString(HoodieTableConfig.BOOTSTRAP_BASE_PATH));
       }
-      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP)) {
-        setPreCombineField(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP));
+      if (hoodieConfig.contains(HoodieTableConfig.PRECOMBINE_FIELD)) {
+        setPreCombineField(hoodieConfig.getString(HoodieTableConfig.PRECOMBINE_FIELD));
       }
-      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP)) {
+      if (hoodieConfig.contains(HoodieTableConfig.PARTITION_FIELDS)) {
         setPartitionFields(
-            hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP));
+            hoodieConfig.getString(HoodieTableConfig.PARTITION_FIELDS));
       }
-      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS)) {
-        setRecordKeyFields(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS));
+      if (hoodieConfig.contains(HoodieTableConfig.RECORDKEY_FIELDS)) {
+        setRecordKeyFields(hoodieConfig.getString(HoodieTableConfig.RECORDKEY_FIELDS));
       }
-      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA)) {
-        setTableCreateSchema(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA));
+      if (hoodieConfig.contains(HoodieTableConfig.CREATE_SCHEMA)) {
+        setTableCreateSchema(hoodieConfig.getString(HoodieTableConfig.CREATE_SCHEMA));
       }
-      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS)) {
-        setPopulateMetaFields(hoodieConfig.getBoolean(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS));
+      if (hoodieConfig.contains(HoodieTableConfig.POPULATE_META_FIELDS)) {
+        setPopulateMetaFields(hoodieConfig.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS));
       }
-      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS)) {
-        setKeyGeneratorClassProp(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS));
+      if (hoodieConfig.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)) {
+        setKeyGeneratorClassProp(hoodieConfig.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME));
       }
       return this;
     }
@@ -780,56 +780,56 @@ public class HoodieTableMetaClient implements Serializable {
       ValidationUtils.checkArgument(tableName != null, "tableName is null");
 
       HoodieTableConfig tableConfig = new HoodieTableConfig();
-      tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_NAME_PROP, tableName);
-      tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP, tableType.name());
-      tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_VERSION_PROP,
+      tableConfig.setValue(HoodieTableConfig.NAME, tableName);
+      tableConfig.setValue(HoodieTableConfig.TYPE, tableType.name());
+      tableConfig.setValue(HoodieTableConfig.VERSION,
           String.valueOf(HoodieTableVersion.current().versionCode()));
       if (tableType == HoodieTableType.MERGE_ON_READ && payloadClassName != null) {
-        tableConfig.setValue(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP, payloadClassName);
+        tableConfig.setValue(HoodieTableConfig.PAYLOAD_CLASS_NAME, payloadClassName);
       }
 
       if (null != tableCreateSchema) {
-        tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA, tableCreateSchema);
+        tableConfig.setValue(HoodieTableConfig.CREATE_SCHEMA, tableCreateSchema);
       }
 
       if (!StringUtils.isNullOrEmpty(archiveLogFolder)) {
-        tableConfig.setValue(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP, archiveLogFolder);
+        tableConfig.setValue(HoodieTableConfig.ARCHIVELOG_FOLDER, archiveLogFolder);
       } else {
-        tableConfig.setDefaultValue(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP);
+        tableConfig.setDefaultValue(HoodieTableConfig.ARCHIVELOG_FOLDER);
       }
 
       if (null != timelineLayoutVersion) {
-        tableConfig.setValue(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION_PROP,
+        tableConfig.setValue(HoodieTableConfig.TIMELINE_LAYOUT_VERSION,
             String.valueOf(timelineLayoutVersion));
       }
 
       if (null != baseFileFormat) {
-        tableConfig.setValue(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP, baseFileFormat.toUpperCase());
+        tableConfig.setValue(HoodieTableConfig.BASE_FILE_FORMAT, baseFileFormat.toUpperCase());
       }
 
       if (null != bootstrapIndexClass) {
-        tableConfig.setValue(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP, bootstrapIndexClass);
+        tableConfig.setValue(HoodieTableConfig.BOOTSTRAP_INDEX_CLASS_NAME, bootstrapIndexClass);
       }
 
       if (null != bootstrapBasePath) {
-        tableConfig.setValue(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP, bootstrapBasePath);
+        tableConfig.setValue(HoodieTableConfig.BOOTSTRAP_BASE_PATH, bootstrapBasePath);
       }
 
       if (null != preCombineField) {
-        tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP, preCombineField);
+        tableConfig.setValue(HoodieTableConfig.PRECOMBINE_FIELD, preCombineField);
       }
 
       if (null != partitionFields) {
-        tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP, partitionFields);
+        tableConfig.setValue(HoodieTableConfig.PARTITION_FIELDS, partitionFields);
       }
       if (null != recordKeyFields) {
-        tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS, recordKeyFields);
+        tableConfig.setValue(HoodieTableConfig.RECORDKEY_FIELDS, recordKeyFields);
       }
       if (null != populateMetaFields) {
-        tableConfig.setValue(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS, Boolean.toString(populateMetaFields));
+        tableConfig.setValue(HoodieTableConfig.POPULATE_META_FIELDS, Boolean.toString(populateMetaFields));
       }
       if (null != keyGeneratorClassProp) {
-        tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS, keyGeneratorClassProp);
+        tableConfig.setValue(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME, keyGeneratorClassProp);
       }
       return tableConfig.getProps();
     }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java
index 603f6bc..b77b0d3 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java
@@ -40,7 +40,7 @@ import java.util.stream.Collectors;
 public class FileSystemViewStorageConfig extends HoodieConfig {
 
   // Property Names
-  public static final ConfigProperty<FileSystemViewStorageType> FILESYSTEM_VIEW_STORAGE_TYPE = ConfigProperty
+  public static final ConfigProperty<FileSystemViewStorageType> VIEW_TYPE = ConfigProperty
       .key("hoodie.filesystem.view.type")
       .defaultValue(FileSystemViewStorageType.MEMORY)
       .withDocumentation("File system view provides APIs for viewing the files on the underlying lake storage, "
@@ -48,69 +48,69 @@ public class FileSystemViewStorageConfig extends HoodieConfig {
           + Arrays.stream(FileSystemViewStorageType.values()).map(Enum::name).collect(Collectors.joining(","))
           + " which provide different trade offs for memory usage and API request performance.");
 
-  public static final ConfigProperty<String> FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = ConfigProperty
+  public static final ConfigProperty<String> INCREMENTAL_TIMELINE_SYNC_ENABLE = ConfigProperty
       .key("hoodie.filesystem.view.incr.timeline.sync.enable")
       .defaultValue("false")
       .withDocumentation("Controls whether or not, the file system view is incrementally updated as "
           + "new actions are performed on the timeline.");
 
-  public static final ConfigProperty<FileSystemViewStorageType> FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE = ConfigProperty
+  public static final ConfigProperty<FileSystemViewStorageType> SECONDARY_VIEW_TYPE = ConfigProperty
       .key("hoodie.filesystem.view.secondary.type")
       .defaultValue(FileSystemViewStorageType.MEMORY)
       .withDocumentation("Specifies the secondary form of storage for file system view, if the primary (e.g timeline server) "
           + " is unavailable.");
 
-  public static final ConfigProperty<String> FILESYSTEM_VIEW_REMOTE_HOST = ConfigProperty
+  public static final ConfigProperty<String> REMOTE_HOST_NAME = ConfigProperty
       .key("hoodie.filesystem.view.remote.host")
       .defaultValue("localhost")
       .withDocumentation("We expect this to be rarely hand configured.");
 
-  public static final ConfigProperty<Integer> FILESYSTEM_VIEW_REMOTE_PORT = ConfigProperty
+  public static final ConfigProperty<Integer> REMOTE_PORT_NUM = ConfigProperty
       .key("hoodie.filesystem.view.remote.port")
       .defaultValue(26754)
       .withDocumentation("Port to serve file system view queries, when remote. We expect this to be rarely hand configured.");
 
-  public static final ConfigProperty<String> FILESYSTEM_VIEW_SPILLABLE_DIR = ConfigProperty
+  public static final ConfigProperty<String> SPILLABLE_DIR = ConfigProperty
       .key("hoodie.filesystem.view.spillable.dir")
       .defaultValue("/tmp/")
       .withDocumentation("Path on local storage to use, when file system view is held in a spillable map.");
 
-  public static final ConfigProperty<Long> FILESYSTEM_VIEW_SPILLABLE_MEM = ConfigProperty
+  public static final ConfigProperty<Long> SPILLABLE_MEMORY = ConfigProperty
       .key("hoodie.filesystem.view.spillable.mem")
       .defaultValue(100 * 1024 * 1024L) // 100 MB
       .withDocumentation("Amount of memory to be used for holding file system view, before spilling to disk.");
 
-  public static final ConfigProperty<Double> FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION = ConfigProperty
+  public static final ConfigProperty<Double> SPILLABLE_COMPACTION_MEM_FRACTION = ConfigProperty
       .key("hoodie.filesystem.view.spillable.compaction.mem.fraction")
       .defaultValue(0.8)
       .withDocumentation("Fraction of the file system view memory, to be used for holding compaction related metadata.");
 
-  public static final ConfigProperty<Double> FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION = ConfigProperty
+  public static final ConfigProperty<Double> BOOTSTRAP_BASE_FILE_MEM_FRACTION = ConfigProperty
       .key("hoodie.filesystem.view.spillable.bootstrap.base.file.mem.fraction")
       .defaultValue(0.05)
       .withDocumentation("Fraction of the file system view memory, to be used for holding mapping to bootstrap base files.");
 
-  public static final ConfigProperty<Double> FILESYSTEM_VIEW_REPLACED_MEM_FRACTION = ConfigProperty
+  public static final ConfigProperty<Double> SPILLABLE_REPLACED_MEM_FRACTION = ConfigProperty
       .key("hoodie.filesystem.view.spillable.replaced.mem.fraction")
       .defaultValue(0.01)
       .withDocumentation("Fraction of the file system view memory, to be used for holding replace commit related metadata.");
 
-  public static final ConfigProperty<Double> FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION = ConfigProperty
+  public static final ConfigProperty<Double> SPILLABLE_CLUSTERING_MEM_FRACTION = ConfigProperty
       .key("hoodie.filesystem.view.spillable.clustering.mem.fraction")
       .defaultValue(0.01)
       .withDocumentation("Fraction of the file system view memory, to be used for holding clustering related metadata.");
 
-  public static final ConfigProperty<String> ROCKSDB_BASE_PATH_PROP = ConfigProperty
+  public static final ConfigProperty<String> ROCKSDB_BASE_PATH = ConfigProperty
       .key("hoodie.filesystem.view.rocksdb.base.path")
       .defaultValue("/tmp/hoodie_timeline_rocksdb")
       .withDocumentation("Path on local storage to use, when storing file system view in embedded kv store/rocksdb.");
 
-  public static final ConfigProperty<Integer> FILESYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = ConfigProperty
+  public static final ConfigProperty<Integer> REMOTE_TIMEOUT_SECS = ConfigProperty
       .key("hoodie.filesystem.view.remote.timeout.secs")
       .defaultValue(5 * 60) // 5 min
       .withDocumentation("Timeout in seconds, to wait for API requests against a remote file system view. e.g timeline server.");
 
-  public static final ConfigProperty<String> REMOTE_BACKUP_VIEW_HANDLER_ENABLE = ConfigProperty
+  public static final ConfigProperty<String> REMOTE_BACKUP_VIEW_ENABLE = ConfigProperty
       .key("hoodie.filesystem.remote.backup.view.enable")
       .defaultValue("true") // Need to be disabled only for tests.
       .withDocumentation("Config to control whether backup needs to be configured if clients were not able to reach"
@@ -125,70 +125,70 @@ public class FileSystemViewStorageConfig extends HoodieConfig {
   }
 
   public FileSystemViewStorageType getStorageType() {
-    return FileSystemViewStorageType.valueOf(getString(FILESYSTEM_VIEW_STORAGE_TYPE));
+    return FileSystemViewStorageType.valueOf(getString(VIEW_TYPE));
   }
 
   public boolean isIncrementalTimelineSyncEnabled() {
-    return getBoolean(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE);
+    return getBoolean(INCREMENTAL_TIMELINE_SYNC_ENABLE);
   }
 
   public String getRemoteViewServerHost() {
-    return getString(FILESYSTEM_VIEW_REMOTE_HOST);
+    return getString(REMOTE_HOST_NAME);
   }
 
   public Integer getRemoteViewServerPort() {
-    return getInt(FILESYSTEM_VIEW_REMOTE_PORT);
+    return getInt(REMOTE_PORT_NUM);
   }
 
   public Integer getRemoteTimelineClientTimeoutSecs() {
-    return getInt(FILESYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS);
+    return getInt(REMOTE_TIMEOUT_SECS);
   }
 
   public long getMaxMemoryForFileGroupMap() {
-    long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
+    long totalMemory = getLong(SPILLABLE_MEMORY);
     return totalMemory - getMaxMemoryForPendingCompaction() - getMaxMemoryForBootstrapBaseFile();
   }
 
   public long getMaxMemoryForPendingCompaction() {
-    long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
-    return new Double(totalMemory * getDouble(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION))
+    long totalMemory = getLong(SPILLABLE_MEMORY);
+    return new Double(totalMemory * getDouble(SPILLABLE_COMPACTION_MEM_FRACTION))
         .longValue();
   }
 
   public long getMaxMemoryForBootstrapBaseFile() {
-    long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
+    long totalMemory = getLong(SPILLABLE_MEMORY);
     long reservedForExternalDataFile =
-        new Double(totalMemory * getDouble(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION))
+        new Double(totalMemory * getDouble(BOOTSTRAP_BASE_FILE_MEM_FRACTION))
             .longValue();
     return reservedForExternalDataFile;
   }
 
   public long getMaxMemoryForReplacedFileGroups() {
-    long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
-    return new Double(totalMemory * getDouble(FILESYSTEM_VIEW_REPLACED_MEM_FRACTION))
+    long totalMemory = getLong(SPILLABLE_MEMORY);
+    return new Double(totalMemory * getDouble(SPILLABLE_REPLACED_MEM_FRACTION))
         .longValue();
   }
 
   public long getMaxMemoryForPendingClusteringFileGroups() {
-    long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
-    return new Double(totalMemory * getDouble(FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION))
+    long totalMemory = getLong(SPILLABLE_MEMORY);
+    return new Double(totalMemory * getDouble(SPILLABLE_CLUSTERING_MEM_FRACTION))
         .longValue();
   }
 
   public String getSpillableDir() {
-    return getString(FILESYSTEM_VIEW_SPILLABLE_DIR);
+    return getString(SPILLABLE_DIR);
   }
 
   public FileSystemViewStorageType getSecondaryStorageType() {
-    return FileSystemViewStorageType.valueOf(getString(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
+    return FileSystemViewStorageType.valueOf(getString(SECONDARY_VIEW_TYPE));
   }
 
   public boolean shouldEnableBackupForRemoteFileSystemView() {
-    return getBoolean(REMOTE_BACKUP_VIEW_HANDLER_ENABLE);
+    return getBoolean(REMOTE_BACKUP_VIEW_ENABLE);
   }
 
   public String getRocksdbBasePath() {
-    return getString(ROCKSDB_BASE_PATH_PROP);
+    return getString(ROCKSDB_BASE_PATH);
   }
 
   /**
@@ -211,73 +211,203 @@ public class FileSystemViewStorageConfig extends HoodieConfig {
     }
 
     public Builder withStorageType(FileSystemViewStorageType storageType) {
-      fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_STORAGE_TYPE, storageType.name());
+      fileSystemViewStorageConfig.setValue(VIEW_TYPE, storageType.name());
       return this;
     }
 
     public Builder withSecondaryStorageType(FileSystemViewStorageType storageType) {
-      fileSystemViewStorageConfig.setValue(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE, storageType.name());
+      fileSystemViewStorageConfig.setValue(SECONDARY_VIEW_TYPE, storageType.name());
       return this;
     }
 
     public Builder withIncrementalTimelineSync(boolean enableIncrTimelineSync) {
-      fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE, Boolean.toString(enableIncrTimelineSync));
+      fileSystemViewStorageConfig.setValue(INCREMENTAL_TIMELINE_SYNC_ENABLE, Boolean.toString(enableIncrTimelineSync));
       return this;
     }
 
     public Builder withRemoteServerHost(String remoteServerHost) {
-      fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_REMOTE_HOST, remoteServerHost);
+      fileSystemViewStorageConfig.setValue(REMOTE_HOST_NAME, remoteServerHost);
       return this;
     }
 
     public Builder withRemoteServerPort(Integer remoteServerPort) {
-      fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_REMOTE_PORT, remoteServerPort.toString());
+      fileSystemViewStorageConfig.setValue(REMOTE_PORT_NUM, remoteServerPort.toString());
       return this;
     }
 
     public Builder withMaxMemoryForView(Long maxMemoryForView) {
-      fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_SPILLABLE_MEM, maxMemoryForView.toString());
+      fileSystemViewStorageConfig.setValue(SPILLABLE_MEMORY, maxMemoryForView.toString());
       return this;
     }
 
     public Builder withRemoteTimelineClientTimeoutSecs(Long timelineClientTimeoutSecs) {
-      fileSystemViewStorageConfig.setValue(FILESYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS, timelineClientTimeoutSecs.toString());
+      fileSystemViewStorageConfig.setValue(REMOTE_TIMEOUT_SECS, timelineClientTimeoutSecs.toString());
       return this;
     }
 
     public Builder withMemFractionForPendingCompaction(Double memFractionForPendingCompaction) {
-      fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, memFractionForPendingCompaction.toString());
+      fileSystemViewStorageConfig.setValue(SPILLABLE_COMPACTION_MEM_FRACTION, memFractionForPendingCompaction.toString());
       return this;
     }
 
     public Builder withMemFractionForExternalDataFile(Double memFractionForExternalDataFile) {
-      fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION, memFractionForExternalDataFile.toString());
+      fileSystemViewStorageConfig.setValue(BOOTSTRAP_BASE_FILE_MEM_FRACTION, memFractionForExternalDataFile.toString());
       return this;
     }
 
     public Builder withBaseStoreDir(String baseStorePath) {
-      fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_SPILLABLE_DIR, baseStorePath);
+      fileSystemViewStorageConfig.setValue(SPILLABLE_DIR, baseStorePath);
       return this;
     }
 
     public Builder withRocksDBPath(String basePath) {
-      fileSystemViewStorageConfig.setValue(ROCKSDB_BASE_PATH_PROP, basePath);
+      fileSystemViewStorageConfig.setValue(ROCKSDB_BASE_PATH, basePath);
       return this;
     }
 
     public Builder withEnableBackupForRemoteFileSystemView(boolean enable) {
-      fileSystemViewStorageConfig.setValue(REMOTE_BACKUP_VIEW_HANDLER_ENABLE, Boolean.toString(enable));
+      fileSystemViewStorageConfig.setValue(REMOTE_BACKUP_VIEW_ENABLE, Boolean.toString(enable));
       return this;
     }
 
     public FileSystemViewStorageConfig build() {
       fileSystemViewStorageConfig.setDefaults(FileSystemViewStorageConfig.class.getName());
       // Validations
-      FileSystemViewStorageType.valueOf(fileSystemViewStorageConfig.getString(FILESYSTEM_VIEW_STORAGE_TYPE));
-      FileSystemViewStorageType.valueOf(fileSystemViewStorageConfig.getString(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
-      ValidationUtils.checkArgument(fileSystemViewStorageConfig.getInt(FILESYSTEM_VIEW_REMOTE_PORT) > 0);
+      FileSystemViewStorageType.valueOf(fileSystemViewStorageConfig.getString(VIEW_TYPE));
+      FileSystemViewStorageType.valueOf(fileSystemViewStorageConfig.getString(SECONDARY_VIEW_TYPE));
+      ValidationUtils.checkArgument(fileSystemViewStorageConfig.getInt(REMOTE_PORT_NUM) > 0);
       return fileSystemViewStorageConfig;
     }
   }
 
+  /**
+   * @deprecated Use {@link #VIEW_TYPE} and its methods.
+   */
+  @Deprecated
+  public static final String FILESYSTEM_VIEW_STORAGE_TYPE = VIEW_TYPE.key();
+  /**
+   * @deprecated Use {@link #VIEW_TYPE} and its methods.
+   */
+  @Deprecated
+  public static final FileSystemViewStorageType DEFAULT_VIEW_STORAGE_TYPE = VIEW_TYPE.defaultValue();
+  /**
+   * @deprecated Use {@link #INCREMENTAL_TIMELINE_SYNC_ENABLE} and its methods.
+   */
+  @Deprecated
+  public static final String FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = INCREMENTAL_TIMELINE_SYNC_ENABLE.key();
+  /**
+   * @deprecated Use {@link #INCREMENTAL_TIMELINE_SYNC_ENABLE} and its methods.
+   */
+  @Deprecated
+  public static final String DEFAULT_FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = INCREMENTAL_TIMELINE_SYNC_ENABLE.defaultValue();
+  /**
+   * @deprecated Use {@link #SECONDARY_VIEW_TYPE} and its methods.
+   */
+  @Deprecated
+  public static final String FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE = SECONDARY_VIEW_TYPE.key();
+  /**
+   * @deprecated Use {@link #SECONDARY_VIEW_TYPE} and its methods.
+   */
+  @Deprecated
+  public static final FileSystemViewStorageType DEFAULT_SECONDARY_VIEW_STORAGE_TYPE = SECONDARY_VIEW_TYPE.defaultValue();
+  /**
+   * @deprecated Use {@link #REMOTE_HOST_NAME} and its methods.
+   */
+  @Deprecated
+  public static final String FILESYSTEM_VIEW_REMOTE_HOST = REMOTE_HOST_NAME.key();
+  /**
+   * @deprecated Use {@link #REMOTE_HOST_NAME} and its methods.
+   */
+  @Deprecated
+  public static final String DEFUALT_REMOTE_VIEW_SERVER_HOST = REMOTE_HOST_NAME.defaultValue();
+  /**
+   * @deprecated Use {@link #REMOTE_PORT_NUM} and its methods.
+   */
+  @Deprecated
+  public static final String FILESYSTEM_VIEW_REMOTE_PORT = REMOTE_PORT_NUM.key();
+  /**
+   * @deprecated Use {@link #REMOTE_PORT_NUM} and its methods.
+   */
+  @Deprecated
+  public static final Integer DEFAULT_REMOTE_VIEW_SERVER_PORT = REMOTE_PORT_NUM.defaultValue();
+  /**
+   * @deprecated Use {@link #SPILLABLE_DIR} and its methods.
+   */
+  @Deprecated
+  public static final String FILESYSTEM_VIEW_SPILLABLE_DIR = SPILLABLE_DIR.key();
+  /**
+   * @deprecated Use {@link #SPILLABLE_DIR} and its methods.
+   */
+  @Deprecated
+  public static final String DEFAULT_VIEW_SPILLABLE_DIR = SPILLABLE_DIR.defaultValue();
+  /**
+   * @deprecated Use {@link #SPILLABLE_MEMORY} and its methods.
+   */
+  @Deprecated
+  public static final String FILESYSTEM_VIEW_SPILLABLE_MEM = SPILLABLE_MEMORY.key();
+  /**
+   * @deprecated Use {@link #SPILLABLE_MEMORY} and its methods.
+   */
+  @Deprecated
+  private static final Long DEFAULT_MAX_MEMORY_FOR_VIEW = SPILLABLE_MEMORY.defaultValue();
+  /**
+   * @deprecated Use {@link #SPILLABLE_COMPACTION_MEM_FRACTION} and its methods.
+   */
+  @Deprecated
+  public static final String FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION = SPILLABLE_COMPACTION_MEM_FRACTION.key();
+  /**
+   * @deprecated Use {@link #SPILLABLE_COMPACTION_MEM_FRACTION} and its methods.
+   */
+  @Deprecated
+  private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION = SPILLABLE_COMPACTION_MEM_FRACTION.defaultValue();
+  /**
+   * @deprecated Use {@link #BOOTSTRAP_BASE_FILE_MEM_FRACTION} and its methods.
+   */
+  @Deprecated
+  public static final String FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION = BOOTSTRAP_BASE_FILE_MEM_FRACTION.key();
+  /**
+   * @deprecated Use {@link #SPILLABLE_REPLACED_MEM_FRACTION} and its methods.
+   */
+  @Deprecated
+  public static final String FILESYSTEM_VIEW_REPLACED_MEM_FRACTION = SPILLABLE_REPLACED_MEM_FRACTION.key();
+  /**
+   * @deprecated Use {@link #SPILLABLE_REPLACED_MEM_FRACTION} and its methods.
+   */
+  @Deprecated
+  private static final Double DEFAULT_MEM_FRACTION_FOR_REPLACED_FILEGROUPS = SPILLABLE_REPLACED_MEM_FRACTION.defaultValue();
+  /**
+   * @deprecated Use {@link #SPILLABLE_CLUSTERING_MEM_FRACTION} and its methods.
+   */
+  @Deprecated
+  public static final String FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION = SPILLABLE_CLUSTERING_MEM_FRACTION.key();
+  /**
+   * @deprecated Use {@link #SPILLABLE_CLUSTERING_MEM_FRACTION} and its methods.
+   */
+  @Deprecated
+  private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_CLUSTERING_FILEGROUPS = SPILLABLE_CLUSTERING_MEM_FRACTION.defaultValue();
+  /**
+   * @deprecated Use {@link #ROCKSDB_BASE_PATH} and its methods.
+   */
+  @Deprecated
+  private static final String ROCKSDB_BASE_PATH_PROP = ROCKSDB_BASE_PATH.key();
+  /**
+   * @deprecated Use {@link #ROCKSDB_BASE_PATH} and its methods.
+   */
+  @Deprecated
+  public static final String DEFAULT_ROCKSDB_BASE_PATH = ROCKSDB_BASE_PATH.defaultValue();
+  /**
+   * @deprecated Use {@link #REMOTE_TIMEOUT_SECS} and its methods.
+   */
+  @Deprecated
+  public static final String FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = REMOTE_TIMEOUT_SECS.key();
+  /**
+   * @deprecated Use {@link #REMOTE_TIMEOUT_SECS} and its methods.
+   */
+  @Deprecated
+  public static final Integer DEFAULT_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = REMOTE_TIMEOUT_SECS.defaultValue();
+  /**
+   * @deprecated Use {@link #BOOTSTRAP_BASE_FILE_MEM_FRACTION} and its methods.
+   */
+  @Deprecated
+  private static final Double DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE = BOOTSTRAP_BASE_FILE_MEM_FRACTION.defaultValue();
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java b/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java
index d781b37..ab51b8e 100644
--- a/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java
+++ b/hudi-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java
@@ -55,5 +55,36 @@ public class KeyGeneratorOptions extends HoodieConfig {
       .defaultValue("partitionpath")
       .withDocumentation("Partition path field. Value to be used at the partitionPath component of HoodieKey. "
           + "Actual value ontained by invoking .toString()");
+
+  /**
+   * @deprecated Use {@link #URL_ENCODE_PARTITIONING} and its methods.
+   */
+  @Deprecated
+  public static final String URL_ENCODE_PARTITIONING_OPT_KEY = URL_ENCODE_PARTITIONING.key();
+  /**
+   * @deprecated Use {@link #URL_ENCODE_PARTITIONING} and its methods.
+   */
+  @Deprecated
+  public static final String DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL = URL_ENCODE_PARTITIONING.defaultValue();
+  /**
+   * @deprecated Use {@link #HIVE_STYLE_PARTITIONING} and its methods.
+   */
+  @Deprecated
+  public static final String HIVE_STYLE_PARTITIONING_OPT_KEY = HIVE_STYLE_PARTITIONING.key();
+  /**
+   * @deprecated Use {@link #HIVE_STYLE_PARTITIONING} and its methods.
+   */
+  @Deprecated
+  public static final String DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL = HIVE_STYLE_PARTITIONING.defaultValue();
+  /**
+   * @deprecated Use {@link #RECORDKEY_FIELD} and its methods.
+   */
+  @Deprecated
+  public static final String RECORDKEY_FIELD_OPT_KEY = RECORDKEY_FIELD.key();
+  /**
+   * @deprecated Use {@link #PARTITIONPATH_FIELD} and its methods.
+   */
+  @Deprecated
+  public static final String PARTITIONPATH_FIELD_OPT_KEY = PARTITIONPATH_FIELD.key();
 }
 
diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java
index 999ea8e..453ec8f 100644
--- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java
+++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java
@@ -51,7 +51,7 @@ public class HoodieMetadataFileSystemView extends HoodieTableFileSystemView {
                                       HoodieMetadataConfig metadataConfig) {
     super(metaClient, visibleActiveTimeline);
     this.tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, metaClient.getBasePath(),
-        FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue());
+        FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue());
   }
 
   /**
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/bootstrap/TestBootstrapIndex.java b/hudi-common/src/test/java/org/apache/hudi/common/bootstrap/TestBootstrapIndex.java
index 391435e..aba1122 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/bootstrap/TestBootstrapIndex.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/bootstrap/TestBootstrapIndex.java
@@ -34,6 +34,8 @@ import org.apache.hudi.common.util.collection.Pair;
 
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -52,9 +54,6 @@ import java.util.concurrent.Future;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertNotNull;
 import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -92,7 +91,7 @@ public class TestBootstrapIndex extends HoodieCommonTestHarness {
   @Test
   public void testNoOpBootstrapIndex() throws IOException {
     Properties props = metaClient.getTableConfig().getProps();
-    props.put(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_ENABLE_PROP.key(), "false");
+    props.put(HoodieTableConfig.BOOTSTRAP_INDEX_ENABLE.key(), "false");
     Properties properties = new Properties();
     properties.putAll(props);
     HoodieTableConfig.createHoodieProperties(metaClient.getFs(), new Path(metaClient.getMetaPath()), properties);
@@ -185,7 +184,7 @@ public class TestBootstrapIndex extends HoodieCommonTestHarness {
     return Arrays.stream(partitions).map(partition -> {
       return Pair.of(partition, IntStream.range(0, numEntriesPerPartition).mapToObj(idx -> {
         String hudiFileId = UUID.randomUUID().toString();
-        String sourceFileName = idx + HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension();
+        String sourceFileName = idx + HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension();
         HoodieFileStatus sourceFileStatus = HoodieFileStatus.newBuilder()
             .setPath(HoodiePath.newBuilder().setUri(sourceBasePath + "/" + partition + "/" + sourceFileName).build())
             .setLength(256 * 1024 * 1024L)
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java
index 5a73667..abfeee9 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java
@@ -18,14 +18,15 @@
 
 package org.apache.hudi.common.fs;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
 import org.apache.hudi.common.model.HoodieLogFile;
 import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
 import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
 import org.apache.hudi.common.testutils.HoodieTestUtils;
 import org.apache.hudi.exception.HoodieException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
 import org.junit.Rule;
 import org.junit.contrib.java.lang.system.EnvironmentVariables;
 import org.junit.jupiter.api.BeforeEach;
@@ -57,7 +58,7 @@ public class TestFSUtils extends HoodieCommonTestHarness {
   private final long minCleanToKeep = 10;
 
   private static String TEST_WRITE_TOKEN = "1-0-1";
-  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension();
+  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension();
 
   @Rule
   public final EnvironmentVariables environmentVariables = new EnvironmentVariables();
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieDeltaWriteStat.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieDeltaWriteStat.java
index 8393de5..b774e06 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieDeltaWriteStat.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieDeltaWriteStat.java
@@ -19,6 +19,7 @@
 package org.apache.hudi.common.model;
 
 import org.apache.hudi.common.table.HoodieTableConfig;
+
 import org.junit.jupiter.api.Test;
 
 import java.util.ArrayList;
@@ -34,7 +35,7 @@ public class TestHoodieDeltaWriteStat {
   @Test
   public void testBaseFileAndLogFiles() {
     HoodieDeltaWriteStat writeStat = new HoodieDeltaWriteStat();
-    String baseFile = "file1" + HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension();
+    String baseFile = "file1" + HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension();
     String logFile1 = ".log1.log";
     String logFile2 = ".log2.log";
 
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestPriorityBasedFileSystemView.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestPriorityBasedFileSystemView.java
index 5e73d18..e0f1f42 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestPriorityBasedFileSystemView.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestPriorityBasedFileSystemView.java
@@ -68,7 +68,7 @@ public class TestPriorityBasedFileSystemView {
     fsView = new PriorityBasedFileSystemView(primary, secondary);
     testBaseFileStream = Stream.of(new HoodieBaseFile("test"));
     testFileSliceStream = Stream.of(new FileSlice("2020-01-01", "20:20",
-        "file0001" + HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension()));
+        "file0001" + HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension()));
   }
 
   private void resetMocks() {
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java
index 960ed28..bb6c0b4 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java
@@ -51,16 +51,16 @@ import java.time.Instant;
 import java.util.HashMap;
 import java.util.Map;
 
-import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCompactionPlan;
 import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCleanMetadata;
 import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCleanerPlan;
+import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCompactionPlan;
 import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeRequestedReplaceMetadata;
 import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeRollbackMetadata;
 
 public class FileCreateUtils {
 
   private static final String WRITE_TOKEN = "1-0-1";
-  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension();
+  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension();
 
   public static String baseFileName(String instantTime, String fileId) {
     return baseFileName(instantTime, fileId, BASE_FILE_EXTENSION);
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java
index 046fc9e..e6c488e 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java
@@ -397,7 +397,7 @@ public class HoodieTestTable {
   }
 
   public FileStatus[] listAllBaseFiles() throws IOException {
-    return listAllBaseFiles(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension());
+    return listAllBaseFiles(HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension());
   }
 
   public FileStatus[] listAllBaseFiles(String fileExtension) throws IOException {
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java
index cf83eac..bc1c18a 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java
@@ -65,7 +65,7 @@ public class HoodieTestUtils {
 
   public static HoodieTableMetaClient init(String basePath, HoodieTableType tableType, String bootstrapBasePath) throws IOException {
     Properties props = new Properties();
-    props.setProperty(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP.key(), bootstrapBasePath);
+    props.setProperty(HoodieTableConfig.BOOTSTRAP_BASE_PATH.key(), bootstrapBasePath);
     return init(getDefaultHadoopConf(), basePath, tableType, props);
   }
 
@@ -86,7 +86,7 @@ public class HoodieTestUtils {
                                            String tableName)
       throws IOException {
     Properties properties = new Properties();
-    properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP.key(), tableName);
+    properties.setProperty(HoodieTableConfig.NAME.key(), tableName);
     return init(hadoopConf, basePath, tableType, properties);
   }
 
@@ -94,7 +94,7 @@ public class HoodieTestUtils {
                                            HoodieFileFormat baseFileFormat)
       throws IOException {
     Properties properties = new Properties();
-    properties.setProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.key(), baseFileFormat.toString());
+    properties.setProperty(HoodieTableConfig.BASE_FILE_FORMAT.key(), baseFileFormat.toString());
     return init(hadoopConf, basePath, tableType, properties);
   }
 
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestTablePathUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestTablePathUtils.java
index 49659d1..ab573f0 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestTablePathUtils.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestTablePathUtils.java
@@ -17,12 +17,13 @@
 
 package org.apache.hudi.common.util;
 
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hudi.common.model.HoodiePartitionMetadata;
 import org.apache.hudi.common.table.HoodieTableConfig;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.io.TempDir;
@@ -37,7 +38,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public final class TestTablePathUtils {
-  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension();
+  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension();
 
   @TempDir
   static File tempDir;
diff --git a/hudi-examples/src/main/scala/org/apache/hudi/examples/spark/HoodieDataSourceExample.scala b/hudi-examples/src/main/scala/org/apache/hudi/examples/spark/HoodieDataSourceExample.scala
index ada5aea..cb221fc 100644
--- a/hudi-examples/src/main/scala/org/apache/hudi/examples/spark/HoodieDataSourceExample.scala
+++ b/hudi-examples/src/main/scala/org/apache/hudi/examples/spark/HoodieDataSourceExample.scala
@@ -18,11 +18,11 @@
 
 package org.apache.hudi.examples.spark
 
-import org.apache.hudi.DataSourceReadOptions.{BEGIN_INSTANTTIME, END_INSTANTTIME, QUERY_TYPE_INCREMENTAL_OPT_VAL, QUERY_TYPE}
+import org.apache.hudi.DataSourceReadOptions.{BEGIN_INSTANTTIME, END_INSTANTTIME, QUERY_TYPE, QUERY_TYPE_INCREMENTAL_OPT_VAL}
 import org.apache.hudi.DataSourceWriteOptions.{PARTITIONPATH_FIELD, PRECOMBINE_FIELD, RECORDKEY_FIELD, PARTITIONS_TO_DELETE, OPERATION, DELETE_PARTITION_OPERATION_OPT_VAL, DELETE_OPERATION_OPT_VAL}
 import org.apache.hudi.QuickstartUtils.getQuickstartWriteConfigs
 import org.apache.hudi.common.model.HoodieAvroPayload
-import org.apache.hudi.config.HoodieWriteConfig.TABLE_NAME
+import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
 import org.apache.hudi.examples.common.{HoodieExampleDataGenerator, HoodieExampleSparkUtils}
 import org.apache.spark.sql.SaveMode.{Append, Overwrite}
 import org.apache.spark.sql.SparkSession
@@ -76,12 +76,12 @@ object HoodieDataSourceExample {
     val inserts = dataGen.convertToStringList(dataGen.generateInserts(commitTime, 20))
     val df = spark.read.json(spark.sparkContext.parallelize(inserts, 1))
     df.write.format("org.apache.hudi").
-        options(getQuickstartWriteConfigs).
-        option(PRECOMBINE_FIELD.key, "ts").
-        option(RECORDKEY_FIELD.key, "uuid").
-        option(PARTITIONPATH_FIELD.key, "partitionpath").
-        option(TABLE_NAME.key, tableName).
-        mode(Overwrite).
+      options(getQuickstartWriteConfigs).
+      option(PRECOMBINE_FIELD.key, "ts").
+      option(RECORDKEY_FIELD.key, "uuid").
+      option(PARTITIONPATH_FIELD.key, "partitionpath").
+      option(TBL_NAME.key, tableName).
+      mode(Overwrite).
         save(tablePath)
   }
 
@@ -121,12 +121,12 @@ object HoodieDataSourceExample {
     val updates = dataGen.convertToStringList(dataGen.generateUpdates(commitTime, 10))
     val df = spark.read.json(spark.sparkContext.parallelize(updates, 1))
     df.write.format("org.apache.hudi").
-        options(getQuickstartWriteConfigs).
-        option(PRECOMBINE_FIELD.key, "ts").
-        option(RECORDKEY_FIELD.key, "uuid").
-        option(PARTITIONPATH_FIELD.key, "partitionpath").
-        option(TABLE_NAME.key, tableName).
-        mode(Append).
+      options(getQuickstartWriteConfigs).
+      option(PRECOMBINE_FIELD.key, "ts").
+      option(RECORDKEY_FIELD.key, "uuid").
+      option(PARTITIONPATH_FIELD.key, "partitionpath").
+      option(TBL_NAME.key, tableName).
+      mode(Append).
         save(tablePath)
   }
 
@@ -144,7 +144,7 @@ object HoodieDataSourceExample {
       option(PRECOMBINE_FIELD.key, "ts").
       option(RECORDKEY_FIELD.key, "uuid").
       option(PARTITIONPATH_FIELD.key, "partitionpath").
-      option(TABLE_NAME.key, tableName).
+      option(TBL_NAME.key, tableName).
       option(OPERATION.key, DELETE_OPERATION_OPT_VAL).
       mode(Append).
       save(tablePath)
@@ -160,7 +160,7 @@ object HoodieDataSourceExample {
       option(PRECOMBINE_FIELD.key, "ts").
       option(RECORDKEY_FIELD.key, "uuid").
       option(PARTITIONPATH_FIELD.key, "partitionpath").
-      option(TABLE_NAME.key, tableName).
+      option(TBL_NAME.key, tableName).
       option(OPERATION.key, DELETE_PARTITION_OPERATION_OPT_VAL).
       option(PARTITIONS_TO_DELETE.key(), HoodieExampleDataGenerator.DEFAULT_PARTITION_PATHS.mkString(",")).
       mode(Append).
diff --git a/hudi-examples/src/main/scala/org/apache/hudi/examples/spark/HoodieMorCompactionJob.scala b/hudi-examples/src/main/scala/org/apache/hudi/examples/spark/HoodieMorCompactionJob.scala
index ace0867..2d96f9c 100644
--- a/hudi-examples/src/main/scala/org/apache/hudi/examples/spark/HoodieMorCompactionJob.scala
+++ b/hudi-examples/src/main/scala/org/apache/hudi/examples/spark/HoodieMorCompactionJob.scala
@@ -25,7 +25,7 @@ import org.apache.hudi.client.SparkRDDWriteClient
 import org.apache.hudi.client.common.HoodieSparkEngineContext
 import org.apache.hudi.common.model.{HoodieAvroPayload, HoodieRecordPayload, HoodieTableType}
 import org.apache.hudi.common.util.Option
-import org.apache.hudi.config.HoodieWriteConfig.TABLE_NAME
+import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
 import org.apache.hudi.config.{HoodieCompactionConfig, HoodieWriteConfig}
 import org.apache.hudi.examples.common.{HoodieExampleDataGenerator, HoodieExampleSparkUtils}
 import org.apache.spark.sql.SaveMode.{Append, Overwrite}
@@ -89,7 +89,7 @@ object HoodieMorCompactionJob {
       option(PRECOMBINE_FIELD.key, "ts").
       option(RECORDKEY_FIELD.key, "uuid").
       option(PARTITIONPATH_FIELD.key, "partitionpath").
-      option(TABLE_NAME.key, tableName).
+      option(TBL_NAME.key, tableName).
       option(TABLE_TYPE.key, tableType).
       mode(Overwrite).
       save(tablePath)
@@ -105,7 +105,7 @@ object HoodieMorCompactionJob {
       option(PRECOMBINE_FIELD.key, "ts").
       option(RECORDKEY_FIELD.key, "uuid").
       option(PARTITIONPATH_FIELD.key, "partitionpath").
-      option(TABLE_NAME.key, tableName).
+      option(TBL_NAME.key, tableName).
       option(TABLE_TYPE.key, tableType).
       mode(Append).
       save(tablePath)
diff --git a/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java b/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
index 5a2751a..72e8fcc 100644
--- a/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
+++ b/hudi-flink/src/main/java/org/apache/hudi/configuration/FlinkOptions.java
@@ -207,7 +207,7 @@ public class FlinkOptions extends HoodieConfig {
   //  Write Options
   // ------------------------------------------------------------------------
   public static final ConfigOption<String> TABLE_NAME = ConfigOptions
-      .key(HoodieWriteConfig.TABLE_NAME.key())
+      .key(HoodieWriteConfig.TBL_NAME.key())
       .stringType()
       .noDefaultValue()
       .withDescription("Table name to register to Hive metastore");
@@ -240,7 +240,7 @@ public class FlinkOptions extends HoodieConfig {
           + "key value, we will pick the one with the largest value for the precombine field,\n"
           + "determined by Object.compareTo(..)");
 
-  public static final ConfigOption<String> PAYLOAD_CLASS = ConfigOptions
+  public static final ConfigOption<String> PAYLOAD_CLASS_NAME = ConfigOptions
       .key("write.payload.class")
       .stringType()
       .defaultValue(OverwriteWithLatestAvroPayload.class.getName())
@@ -308,8 +308,8 @@ public class FlinkOptions extends HoodieConfig {
           + "If set true, the names of partition folders follow <partition_column_name>=<partition_value> format.\n"
           + "By default false (the names of partition folders are only partition values)");
 
-  public static final ConfigOption<String> KEYGEN_CLASS = ConfigOptions
-      .key(HoodieWriteConfig.KEYGENERATOR_CLASS.key())
+  public static final ConfigOption<String> KEYGEN_CLASS_NAME = ConfigOptions
+      .key(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key())
       .stringType()
       .defaultValue("")
       .withDescription("Key generator class, that implements will extract the key out of incoming record");
@@ -549,7 +549,7 @@ public class FlinkOptions extends HoodieConfig {
       .defaultValue("")
       .withDescription("Partition fields for hive sync, default ''");
 
-  public static final ConfigOption<String> HIVE_SYNC_PARTITION_EXTRACTOR_CLASS = ConfigOptions
+  public static final ConfigOption<String> HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME = ConfigOptions
       .key("hive_sync.partition_extractor_class")
       .stringType()
       .defaultValue(SlashEncodedDayPartitionValueExtractor.class.getCanonicalName())
diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java
index 3ccfb26..4d953c2 100644
--- a/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java
+++ b/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfile.java
@@ -81,7 +81,7 @@ public class WriteProfile {
 
   /**
    * Total records to write for each bucket based on
-   * the config option {@link org.apache.hudi.config.HoodieStorageConfig#PARQUET_FILE_MAX_BYTES}.
+   * the config option {@link org.apache.hudi.config.HoodieStorageConfig#PARQUET_MAX_FILE_SIZE}.
    */
   private long recordsPerBucket;
 
diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java
index 192feb5..d374882 100644
--- a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java
+++ b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java
@@ -76,7 +76,7 @@ public class HiveSyncContext {
     hiveSyncConfig.hivePass = conf.getString(FlinkOptions.HIVE_SYNC_PASSWORD);
     hiveSyncConfig.jdbcUrl = conf.getString(FlinkOptions.HIVE_SYNC_JDBC_URL);
     hiveSyncConfig.partitionFields = Arrays.asList(FilePathUtils.extractPartitionKeys(conf));
-    hiveSyncConfig.partitionValueExtractorClass = conf.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS);
+    hiveSyncConfig.partitionValueExtractorClass = conf.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME);
     hiveSyncConfig.useJdbc = conf.getBoolean(FlinkOptions.HIVE_SYNC_USE_JDBC);
     hiveSyncConfig.useFileListingFromMetadata = conf.getBoolean(FlinkOptions.METADATA_ENABLED);
     hiveSyncConfig.verifyMetadataFileListing = false;
diff --git a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/PayloadCreation.java b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/PayloadCreation.java
index 20070b3..d10447f 100644
--- a/hudi-flink/src/main/java/org/apache/hudi/sink/utils/PayloadCreation.java
+++ b/hudi-flink/src/main/java/org/apache/hudi/sink/utils/PayloadCreation.java
@@ -66,7 +66,7 @@ public class PayloadCreation implements Serializable {
     } else {
       argTypes = new Class<?>[] {Option.class};
     }
-    final String clazz = conf.getString(FlinkOptions.PAYLOAD_CLASS);
+    final String clazz = conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME);
     constructor = ReflectionUtils.getClass(clazz).getConstructor(argTypes);
     return new PayloadCreation(shouldCombine, constructor, preCombineField);
   }
diff --git a/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java b/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java
index a4591c8..75272ab 100644
--- a/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java
+++ b/hudi-flink/src/main/java/org/apache/hudi/streamer/FlinkStreamerConfig.java
@@ -311,7 +311,7 @@ public class FlinkStreamerConfig extends Configuration {
     conf.setBoolean(FlinkOptions.INSERT_DEDUP, config.insertDedup);
     conf.setString(FlinkOptions.OPERATION, config.operation.value());
     conf.setString(FlinkOptions.PRECOMBINE_FIELD, config.sourceOrderingField);
-    conf.setString(FlinkOptions.PAYLOAD_CLASS, config.payloadClassName);
+    conf.setString(FlinkOptions.PAYLOAD_CLASS_NAME, config.payloadClassName);
     conf.setBoolean(FlinkOptions.INSERT_DROP_DUPS, config.filterDupes);
     conf.setInteger(FlinkOptions.RETRY_TIMES, Integer.parseInt(config.instantRetryTimes));
     conf.setLong(FlinkOptions.RETRY_INTERVAL_MS, Long.parseLong(config.instantRetryInterval));
@@ -319,7 +319,7 @@ public class FlinkStreamerConfig extends Configuration {
     conf.setString(FlinkOptions.RECORD_KEY_FIELD, config.recordKeyField);
     conf.setString(FlinkOptions.PARTITION_PATH_FIELD, config.partitionPathField);
     if (!StringUtils.isNullOrEmpty(config.keygenClass)) {
-      conf.setString(FlinkOptions.KEYGEN_CLASS, config.keygenClass);
+      conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, config.keygenClass);
     } else {
       conf.setString(FlinkOptions.KEYGEN_TYPE, config.keygenType);
     }
@@ -362,7 +362,7 @@ public class FlinkStreamerConfig extends Configuration {
     conf.setString(FlinkOptions.HIVE_SYNC_JDBC_URL, config.hiveSyncJdbcUrl);
     conf.setString(FlinkOptions.HIVE_SYNC_METASTORE_URIS, config.hiveSyncMetastoreUri);
     conf.setString(FlinkOptions.HIVE_SYNC_PARTITION_FIELDS, config.hiveSyncPartitionFields);
-    conf.setString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS, config.hiveSyncPartitionExtractorClass);
+    conf.setString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME, config.hiveSyncPartitionExtractorClass);
     conf.setBoolean(FlinkOptions.HIVE_SYNC_ASSUME_DATE_PARTITION, config.hiveSyncAssumeDatePartition);
     conf.setBoolean(FlinkOptions.HIVE_SYNC_USE_JDBC, config.hiveSyncUseJdbc);
     conf.setBoolean(FlinkOptions.HIVE_SYNC_AUTO_CREATE_DB, config.hiveSyncAutoCreateDb);
diff --git a/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java b/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java
index c753dde..89f7ed7 100644
--- a/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java
+++ b/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableFactory.java
@@ -174,17 +174,17 @@ public class HoodieTableFactory implements DynamicTableSourceFactory, DynamicTab
     // tweak the key gen class if possible
     final String[] partitions = conf.getString(FlinkOptions.PARTITION_PATH_FIELD).split(",");
     if (partitions.length == 1 && partitions[0].equals("")) {
-      conf.setString(FlinkOptions.KEYGEN_CLASS, NonpartitionedAvroKeyGenerator.class.getName());
+      conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, NonpartitionedAvroKeyGenerator.class.getName());
       LOG.info("Table option [{}] is reset to {} because this is a non-partitioned table",
-          FlinkOptions.KEYGEN_CLASS.key(), NonpartitionedAvroKeyGenerator.class.getName());
+          FlinkOptions.KEYGEN_CLASS_NAME.key(), NonpartitionedAvroKeyGenerator.class.getName());
       return;
     }
     final String[] pks = conf.getString(FlinkOptions.RECORD_KEY_FIELD).split(",");
     boolean complexHoodieKey = pks.length > 1 || partitions.length > 1;
-    if (complexHoodieKey && FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.KEYGEN_CLASS)) {
-      conf.setString(FlinkOptions.KEYGEN_CLASS, ComplexAvroKeyGenerator.class.getName());
+    if (complexHoodieKey && FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.KEYGEN_CLASS_NAME)) {
+      conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, ComplexAvroKeyGenerator.class.getName());
       LOG.info("Table option [{}] is reset to {} because record key or partition path has two or more fields",
-          FlinkOptions.KEYGEN_CLASS.key(), ComplexAvroKeyGenerator.class.getName());
+          FlinkOptions.KEYGEN_CLASS_NAME.key(), ComplexAvroKeyGenerator.class.getName());
     }
   }
 
@@ -220,8 +220,8 @@ public class HoodieTableFactory implements DynamicTableSourceFactory, DynamicTab
    * */
   private static void setupHiveOptions(Configuration conf) {
     if (!conf.getBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING)
-        && FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS)) {
-      conf.setString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS, MultiPartKeysValueExtractor.class.getName());
+        && FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME)) {
+      conf.setString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME, MultiPartKeysValueExtractor.class.getName());
     }
   }
 
diff --git a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
index f89d089..6dc6add 100644
--- a/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
+++ b/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
@@ -76,7 +76,7 @@ import java.util.Properties;
 import static org.apache.hudi.common.model.HoodieFileFormat.HOODIE_LOG;
 import static org.apache.hudi.common.model.HoodieFileFormat.ORC;
 import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
-import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP;
+import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER;
 
 /**
  * Utilities for Flink stream read and write.
@@ -157,7 +157,7 @@ public class StreamerUtil {
             .withMergeAllowDuplicateOnInserts(allowDuplicateInserts(conf))
             .withCompactionConfig(
                 HoodieCompactionConfig.newBuilder()
-                    .withPayloadClass(conf.getString(FlinkOptions.PAYLOAD_CLASS))
+                    .withPayloadClass(conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME))
                     .withTargetIOPerCompactionInMB(conf.getLong(FlinkOptions.COMPACTION_TARGET_IO))
                     .withInlineCompactionTriggerStrategy(
                         CompactionTriggerStrategy.valueOf(conf.getString(FlinkOptions.COMPACTION_TRIGGER_STRATEGY).toUpperCase(Locale.ROOT)))
@@ -236,8 +236,8 @@ public class StreamerUtil {
           .setTableType(conf.getString(FlinkOptions.TABLE_TYPE))
           .setTableName(conf.getString(FlinkOptions.TABLE_NAME))
           .setRecordKeyFields(conf.getString(FlinkOptions.RECORD_KEY_FIELD, null))
-          .setPayloadClassName(conf.getString(FlinkOptions.PAYLOAD_CLASS))
-          .setArchiveLogFolder(HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue())
+          .setPayloadClassName(conf.getString(FlinkOptions.PAYLOAD_CLASS_NAME))
+          .setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue())
           .setPartitionFields(conf.getString(FlinkOptions.PARTITION_PATH_FIELD, null))
           .setPreCombineField(conf.getString(FlinkOptions.PRECOMBINE_FIELD))
           .setTimelineLayoutVersion(1)
diff --git a/hudi-flink/src/test/java/org/apache/hudi/sink/partitioner/TestBucketAssigner.java b/hudi-flink/src/test/java/org/apache/hudi/sink/partitioner/TestBucketAssigner.java
index bea8600..d10421d 100644
--- a/hudi-flink/src/test/java/org/apache/hudi/sink/partitioner/TestBucketAssigner.java
+++ b/hudi-flink/src/test/java/org/apache/hudi/sink/partitioner/TestBucketAssigner.java
@@ -152,7 +152,7 @@ public class TestBucketAssigner {
 
   @Test
   public void testInsertOverBucketAssigned() {
-    conf.setInteger(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE.key(), 2);
+    conf.setInteger(HoodieCompactionConfig.COPY_ON_WRITE_INSERT_SPLIT_SIZE.key(), 2);
     writeConfig = StreamerUtil.getHoodieClientConfig(conf);
 
     MockBucketAssigner mockBucketAssigner = new MockBucketAssigner(context, writeConfig);
diff --git a/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java b/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java
index 0439c4d..6e4b215 100644
--- a/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java
+++ b/hudi-flink/src/test/java/org/apache/hudi/table/TestHoodieTableFactory.java
@@ -137,7 +137,7 @@ public class TestHoodieTableFactory {
   @Test
   void testSetupHoodieKeyOptionsForSource() {
     this.conf.setString(FlinkOptions.RECORD_KEY_FIELD, "dummyField");
-    this.conf.setString(FlinkOptions.KEYGEN_CLASS, "dummyKeyGenClass");
+    this.conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, "dummyKeyGenClass");
     // definition with simple primary key and partition path
     ResolvedSchema schema1 = SchemaBuilder.instance()
         .field("f0", DataTypes.INT().notNull())
@@ -150,10 +150,10 @@ public class TestHoodieTableFactory {
     final HoodieTableSource tableSource1 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext1);
     final Configuration conf1 = tableSource1.getConf();
     assertThat(conf1.get(FlinkOptions.RECORD_KEY_FIELD), is("f0"));
-    assertThat(conf1.get(FlinkOptions.KEYGEN_CLASS), is("dummyKeyGenClass"));
+    assertThat(conf1.get(FlinkOptions.KEYGEN_CLASS_NAME), is("dummyKeyGenClass"));
 
     // definition with complex primary keys and partition paths
-    this.conf.setString(FlinkOptions.KEYGEN_CLASS, FlinkOptions.KEYGEN_CLASS.defaultValue());
+    this.conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, FlinkOptions.KEYGEN_CLASS_NAME.defaultValue());
     ResolvedSchema schema2 = SchemaBuilder.instance()
         .field("f0", DataTypes.INT().notNull())
         .field("f1", DataTypes.VARCHAR(20).notNull())
@@ -165,15 +165,15 @@ public class TestHoodieTableFactory {
     final HoodieTableSource tableSource2 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext2);
     final Configuration conf2 = tableSource2.getConf();
     assertThat(conf2.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1"));
-    assertThat(conf2.get(FlinkOptions.KEYGEN_CLASS), is(ComplexAvroKeyGenerator.class.getName()));
+    assertThat(conf2.get(FlinkOptions.KEYGEN_CLASS_NAME), is(ComplexAvroKeyGenerator.class.getName()));
 
     // definition with complex primary keys and empty partition paths
-    this.conf.setString(FlinkOptions.KEYGEN_CLASS, FlinkOptions.KEYGEN_CLASS.defaultValue());
+    this.conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, FlinkOptions.KEYGEN_CLASS_NAME.defaultValue());
     final MockContext sourceContext3 = MockContext.getInstance(this.conf, schema2, "");
     final HoodieTableSource tableSource3 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext3);
     final Configuration conf3 = tableSource3.getConf();
     assertThat(conf3.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1"));
-    assertThat(conf3.get(FlinkOptions.KEYGEN_CLASS), is(NonpartitionedAvroKeyGenerator.class.getName()));
+    assertThat(conf3.get(FlinkOptions.KEYGEN_CLASS_NAME), is(NonpartitionedAvroKeyGenerator.class.getName()));
   }
 
   @Test
@@ -190,7 +190,7 @@ public class TestHoodieTableFactory {
     final MockContext sourceContext1 = MockContext.getInstance(this.conf, schema1, "f2");
     final HoodieTableSource tableSource1 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext1);
     final Configuration conf1 = tableSource1.getConf();
-    assertThat(conf1.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS), is(MultiPartKeysValueExtractor.class.getName()));
+    assertThat(conf1.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME), is(MultiPartKeysValueExtractor.class.getName()));
 
     // set up hive style partitioning is true.
     this.conf.setBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING, true);
@@ -198,7 +198,7 @@ public class TestHoodieTableFactory {
     final MockContext sourceContext2 = MockContext.getInstance(this.conf, schema1, "f2");
     final HoodieTableSource tableSource2 = (HoodieTableSource) new HoodieTableFactory().createDynamicTableSource(sourceContext2);
     final Configuration conf2 = tableSource2.getConf();
-    assertThat(conf2.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS), is(SlashEncodedDayPartitionValueExtractor.class.getName()));
+    assertThat(conf2.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME), is(SlashEncodedDayPartitionValueExtractor.class.getName()));
   }
 
   @Test
@@ -249,7 +249,7 @@ public class TestHoodieTableFactory {
   @Test
   void testSetupHoodieKeyOptionsForSink() {
     this.conf.setString(FlinkOptions.RECORD_KEY_FIELD, "dummyField");
-    this.conf.setString(FlinkOptions.KEYGEN_CLASS, "dummyKeyGenClass");
+    this.conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, "dummyKeyGenClass");
     // definition with simple primary key and partition path
     ResolvedSchema schema1 = SchemaBuilder.instance()
         .field("f0", DataTypes.INT().notNull())
@@ -262,10 +262,10 @@ public class TestHoodieTableFactory {
     final HoodieTableSink tableSink1 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext1);
     final Configuration conf1 = tableSink1.getConf();
     assertThat(conf1.get(FlinkOptions.RECORD_KEY_FIELD), is("f0"));
-    assertThat(conf1.get(FlinkOptions.KEYGEN_CLASS), is("dummyKeyGenClass"));
+    assertThat(conf1.get(FlinkOptions.KEYGEN_CLASS_NAME), is("dummyKeyGenClass"));
 
     // definition with complex primary keys and partition paths
-    this.conf.setString(FlinkOptions.KEYGEN_CLASS, FlinkOptions.KEYGEN_CLASS.defaultValue());
+    this.conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, FlinkOptions.KEYGEN_CLASS_NAME.defaultValue());
     ResolvedSchema schema2 = SchemaBuilder.instance()
         .field("f0", DataTypes.INT().notNull())
         .field("f1", DataTypes.VARCHAR(20).notNull())
@@ -277,15 +277,15 @@ public class TestHoodieTableFactory {
     final HoodieTableSink tableSink2 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext2);
     final Configuration conf2 = tableSink2.getConf();
     assertThat(conf2.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1"));
-    assertThat(conf2.get(FlinkOptions.KEYGEN_CLASS), is(ComplexAvroKeyGenerator.class.getName()));
+    assertThat(conf2.get(FlinkOptions.KEYGEN_CLASS_NAME), is(ComplexAvroKeyGenerator.class.getName()));
 
     // definition with complex primary keys and empty partition paths
-    this.conf.setString(FlinkOptions.KEYGEN_CLASS, FlinkOptions.KEYGEN_CLASS.defaultValue());
+    this.conf.setString(FlinkOptions.KEYGEN_CLASS_NAME, FlinkOptions.KEYGEN_CLASS_NAME.defaultValue());
     final MockContext sinkContext3 = MockContext.getInstance(this.conf, schema2, "");
     final HoodieTableSink tableSink3 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext3);
     final Configuration conf3 = tableSink3.getConf();
     assertThat(conf3.get(FlinkOptions.RECORD_KEY_FIELD), is("f0,f1"));
-    assertThat(conf3.get(FlinkOptions.KEYGEN_CLASS), is(NonpartitionedAvroKeyGenerator.class.getName()));
+    assertThat(conf3.get(FlinkOptions.KEYGEN_CLASS_NAME), is(NonpartitionedAvroKeyGenerator.class.getName()));
   }
 
   @Test
@@ -302,7 +302,7 @@ public class TestHoodieTableFactory {
     final MockContext sinkContext1 = MockContext.getInstance(this.conf, schema1, "f2");
     final HoodieTableSink tableSink1 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext1);
     final Configuration conf1 = tableSink1.getConf();
-    assertThat(conf1.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS), is(MultiPartKeysValueExtractor.class.getName()));
+    assertThat(conf1.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME), is(MultiPartKeysValueExtractor.class.getName()));
 
     // set up hive style partitioning is true.
     this.conf.setBoolean(FlinkOptions.HIVE_STYLE_PARTITIONING, true);
@@ -310,7 +310,7 @@ public class TestHoodieTableFactory {
     final MockContext sinkContext2 = MockContext.getInstance(this.conf, schema1, "f2");
     final HoodieTableSink tableSink2 = (HoodieTableSink) new HoodieTableFactory().createDynamicTableSink(sinkContext2);
     final Configuration conf2 = tableSink2.getConf();
-    assertThat(conf2.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS), is(SlashEncodedDayPartitionValueExtractor.class.getName()));
+    assertThat(conf2.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME), is(SlashEncodedDayPartitionValueExtractor.class.getName()));
   }
 
   @Test
diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java
index f4bddbd..70c4301 100644
--- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java
+++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/utils/HoodieInputFormatUtils.java
@@ -18,9 +18,6 @@
 
 package org.apache.hudi.hadoop.utils;
 
-import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
-import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat;
-import org.apache.hadoop.hive.ql.io.orc.OrcSerde;
 import org.apache.hudi.common.config.HoodieMetadataConfig;
 import org.apache.hudi.common.engine.HoodieLocalEngineContext;
 import org.apache.hudi.common.fs.FSUtils;
@@ -51,6 +48,9 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
+import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat;
+import org.apache.hadoop.hive.ql.io.orc.OrcSerde;
 import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat;
 import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
 import org.apache.hadoop.mapred.FileInputFormat;
@@ -71,8 +71,8 @@ import java.util.function.Function;
 import java.util.stream.Collectors;
 
 import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
-import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP;
-import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP;
+import static org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE;
+import static org.apache.hudi.common.config.HoodieMetadataConfig.VALIDATE_ENABLE;
 
 public class HoodieInputFormatUtils {
 
@@ -418,8 +418,8 @@ public class HoodieInputFormatUtils {
 
   public static HoodieMetadataConfig buildMetadataConfig(Configuration conf) {
     return HoodieMetadataConfig.newBuilder()
-        .enable(conf.getBoolean(METADATA_ENABLE_PROP.key(), DEFAULT_METADATA_ENABLE_FOR_READERS))
-        .validate(conf.getBoolean(METADATA_VALIDATE_PROP.key(), METADATA_VALIDATE_PROP.defaultValue()))
+        .enable(conf.getBoolean(ENABLE.key(), DEFAULT_METADATA_ENABLE_FOR_READERS))
+        .validate(conf.getBoolean(VALIDATE_ENABLE.key(), VALIDATE_ENABLE.defaultValue()))
         .build();
   }
 
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 ee6502c..3966aa3 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
@@ -28,9 +28,9 @@ import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapred.JobConf;
 import org.junit.jupiter.api.AfterAll;
 import org.junit.jupiter.api.BeforeAll;
 import org.junit.jupiter.api.Test;
@@ -138,9 +138,9 @@ public class TestInputPathHandler {
   static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath,
                                              String tableName, HoodieTableType tableType) throws IOException {
     Properties properties = new Properties();
-    properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP.key(), tableName);
-    properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP.key(), tableType.name());
-    properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP.key(), HoodieAvroPayload.class.getName());
+    properties.setProperty(HoodieTableConfig.NAME.key(), tableName);
+    properties.setProperty(HoodieTableConfig.TYPE.key(), tableType.name());
+    properties.setProperty(HoodieTableConfig.PAYLOAD_CLASS_NAME.key(), HoodieAvroPayload.class.getName());
     return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, basePath, properties);
   }
 
diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java
index 39112c1..a647da9 100644
--- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java
+++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java
@@ -31,12 +31,12 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
 import org.apache.hudi.common.testutils.FileCreateUtils;
 import org.apache.hudi.common.testutils.HoodieTestUtils;
 import org.apache.hudi.common.testutils.SchemaTestUtil;
-import org.apache.hudi.common.util.collection.ExternalSpillableMap;
 import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.ExternalSpillableMap;
 import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.exception.HoodieException;
-import org.apache.hudi.hadoop.testutils.InputFormatTestUtil;
 import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
+import org.apache.hudi.hadoop.testutils.InputFormatTestUtil;
 
 import org.apache.avro.Schema;
 import org.apache.avro.Schema.Field;
@@ -124,11 +124,11 @@ public class TestHoodieRealtimeRecordReader {
 
   protected Properties getPropertiesForKeyGen() {
     Properties properties = new Properties();
-    properties.put(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), "false");
-    properties.put("hoodie.datasource.write.recordkey.field","_row_key");
-    properties.put("hoodie.datasource.write.partitionpath.field","partition_path");
-    properties.put(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS.key(), "_row_key");
-    properties.put(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP.key(), "partition_path");
+    properties.put(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false");
+    properties.put("hoodie.datasource.write.recordkey.field", "_row_key");
+    properties.put("hoodie.datasource.write.partitionpath.field", "partition_path");
+    properties.put(HoodieTableConfig.RECORDKEY_FIELDS.key(), "_row_key");
+    properties.put(HoodieTableConfig.PARTITION_FIELDS.key(), "partition_path");
     return properties;
   }
 
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java
index 61e310a..0bcbaf8 100644
--- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java
@@ -61,7 +61,7 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 
-import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP;
+import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER;
 
 /**
  * This is the entry point for running a Hudi Test Suite. Although this class has similarities with {@link HoodieDeltaStreamer} this class does not extend it since do not want to create a dependency
@@ -111,7 +111,7 @@ public class HoodieTestSuiteJob {
     metaClient = HoodieTableMetaClient.withPropertyBuilder()
         .setTableType(cfg.tableType)
         .setTableName(cfg.targetTableName)
-        .setArchiveLogFolder(HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue())
+        .setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue())
         .initTable(jsc.hadoopConfiguration(), cfg.targetBasePath);
 
     if (cfg.cleanInput) {
diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java
index 603fda4..1fd3d2f 100644
--- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java
+++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java
@@ -34,13 +34,13 @@ import org.apache.hudi.common.table.view.TableFileSystemView;
 import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.hudi.config.HoodieMemoryConfig;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 
 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.io.storage.HoodieFileReader;
-import org.apache.hudi.io.storage.HoodieFileReaderFactory;
 import org.apache.spark.api.java.JavaPairRDD;
 import org.apache.spark.api.java.JavaRDD;
 import org.apache.spark.api.java.JavaSparkContext;
@@ -88,7 +88,7 @@ public class DFSHoodieDatasetInputReader extends DFSDeltaInputReader {
     // calls in metrics as they are not part of normal HUDI operation.
     HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
     List<String> partitionPaths = FSUtils.getAllPartitionPaths(engineContext, metaClient.getBasePath(),
-        HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false);
+        HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false);
     // Sort partition so we can pick last N partitions by default
     Collections.sort(partitionPaths);
     if (!partitionPaths.isEmpty()) {
diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkBulkInsertNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkBulkInsertNode.scala
index d55bb96..4d17570 100644
--- a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkBulkInsertNode.scala
+++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkBulkInsertNode.scala
@@ -59,7 +59,7 @@ class SparkBulkInsertNode(config1: Config) extends DagNode[RDD[WriteStatus]] {
       .option(DataSourceWriteOptions.ENABLE_ROW_WRITER.key(), String.valueOf(config.enableRowWriting()))
       .option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX.key(), "deltastreamer.checkpoint.key")
       .option("deltastreamer.checkpoint.key", context.getWriterContext.getHoodieTestSuiteWriter.getLastCheckpoint.orElse(""))
-      .option(HoodieWriteConfig.TABLE_NAME.key(), context.getHoodieTestSuiteWriter.getCfg.targetTableName)
+      .option(HoodieWriteConfig.TBL_NAME.key(), context.getHoodieTestSuiteWriter.getCfg.targetTableName)
       .mode(saveMode)
       .save(context.getHoodieTestSuiteWriter.getWriteConfig.getBasePath)
   }
diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala
index e9720fd..f962e41 100644
--- a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala
+++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkInsertNode.scala
@@ -58,7 +58,7 @@ class SparkInsertNode(config1: Config) extends DagNode[RDD[WriteStatus]] {
       .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
       .option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX.key, "deltastreamer.checkpoint.key")
       .option("deltastreamer.checkpoint.key", context.getWriterContext.getHoodieTestSuiteWriter.getLastCheckpoint.orElse(""))
-      .option(HoodieWriteConfig.TABLE_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
+      .option(HoodieWriteConfig.TBL_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
       .mode(SaveMode.Overwrite)
       .save(context.getHoodieTestSuiteWriter.getWriteConfig.getBasePath)
   }
diff --git a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala
index 3666b38..6486fed 100644
--- a/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala
+++ b/hudi-integ-test/src/main/scala/org/apache/hudi/integ/testsuite/dag/nodes/SparkUpsertNode.scala
@@ -58,7 +58,7 @@ class SparkUpsertNode(config1: Config) extends DagNode[RDD[WriteStatus]] {
       .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
       .option(DataSourceWriteOptions.COMMIT_METADATA_KEYPREFIX.key, "deltastreamer.checkpoint.key")
       .option("deltastreamer.checkpoint.key", context.getWriterContext.getHoodieTestSuiteWriter.getLastCheckpoint.orElse(""))
-      .option(HoodieWriteConfig.TABLE_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
+      .option(HoodieWriteConfig.TBL_NAME.key, context.getHoodieTestSuiteWriter.getCfg.targetTableName)
       .mode(SaveMode.Append)
       .save(context.getHoodieTestSuiteWriter.getWriteConfig.getBasePath)
   }
diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java
index 56aa7af..be6e552 100644
--- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java
+++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java
@@ -169,7 +169,7 @@ public class TestHoodieTestSuiteJob extends UtilitiesTestBase {
     props.setProperty(DataSourceWriteOptions.HIVE_DATABASE().key(), "testdb1");
     props.setProperty(DataSourceWriteOptions.HIVE_TABLE().key(), "table1");
     props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "datestr");
-    props.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(), TimestampBasedKeyGenerator.class.getName());
+    props.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), TimestampBasedKeyGenerator.class.getName());
 
     props.setProperty("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider");
     props.setProperty("hoodie.write.lock.hivemetastore.database", "testdb1");
diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java
index 07ba8a9..7e043eb 100644
--- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java
@@ -185,7 +185,7 @@ public class DataSourceUtils {
     return builder.forTable(tblName)
         .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build())
         .withCompactionConfig(HoodieCompactionConfig.newBuilder()
-            .withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS().key()))
+            .withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_NAME().key()))
             .withInlineCompaction(inlineCompact).build())
         .withClusteringConfig(HoodieClusteringConfig.newBuilder()
             .withInlineClustering(inlineClusteringEnabled)
@@ -300,10 +300,10 @@ public class DataSourceUtils {
         DataSourceWriteOptions.HIVE_AUTO_CREATE_DATABASE().defaultValue()));
     hiveSyncConfig.ignoreExceptions = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_IGNORE_EXCEPTIONS().key(),
         DataSourceWriteOptions.HIVE_IGNORE_EXCEPTIONS().defaultValue()));
-    hiveSyncConfig.skipROSuffix = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX().key(),
-        DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX().defaultValue()));
-    hiveSyncConfig.supportTimestamp = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP().key(),
-        DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP().defaultValue()));
+    hiveSyncConfig.skipROSuffix = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE().key(),
+        DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE().defaultValue()));
+    hiveSyncConfig.supportTimestamp = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP_TYPE().key(),
+        DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP_TYPE().defaultValue()));
     return hiveSyncConfig;
   }
 }
diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java
index 0a845f7..823de99 100644
--- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/BulkInsertDataInternalWriterHelper.java
@@ -106,7 +106,7 @@ public class BulkInsertDataInternalWriterHelper {
   private Option<BuiltinKeyGenerator> getKeyGenerator(Properties properties) {
     TypedProperties typedProperties = new TypedProperties();
     typedProperties.putAll(properties);
-    if (properties.get(DataSourceWriteOptions.KEYGENERATOR_CLASS().key()).equals(NonpartitionedKeyGenerator.class.getName())) {
+    if (properties.get(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key()).equals(NonpartitionedKeyGenerator.class.getName())) {
       return Option.empty(); // Do not instantiate NonPartitionKeyGen
     } else {
       try {
diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala
index 5044ab6..9881e43 100644
--- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala
+++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala
@@ -204,7 +204,7 @@ object DataSourceWriteOptions {
       val partitionColumns = optParams.get(SparkDataSourceUtils.PARTITIONING_COLUMNS_KEY)
         .map(SparkDataSourceUtils.decodePartitioningColumns)
         .getOrElse(Nil)
-      val keyGeneratorClass = optParams.getOrElse(DataSourceWriteOptions.KEYGENERATOR_CLASS.key(),
+      val keyGeneratorClass = optParams.getOrElse(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key(),
         DataSourceWriteOptions.DEFAULT_KEYGENERATOR_CLASS_OPT_VAL)
 
       val partitionPathField =
@@ -244,7 +244,7 @@ object DataSourceWriteOptions {
    * 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
    */
-  val PAYLOAD_CLASS = HoodieWriteConfig.WRITE_PAYLOAD_CLASS
+  val PAYLOAD_CLASS_NAME = HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME
 
   /**
    * Record key field. Value to be used as the `recordKey` component of `HoodieKey`. Actual value
@@ -267,7 +267,7 @@ object DataSourceWriteOptions {
    */
   val HIVE_STYLE_PARTITIONING = KeyGeneratorOptions.HIVE_STYLE_PARTITIONING
 
-  val KEYGENERATOR_CLASS = ConfigProperty.key("hoodie.datasource.write.keygenerator.class")
+  val KEYGENERATOR_CLASS_NAME = ConfigProperty.key("hoodie.datasource.write.keygenerator.class")
     .defaultValue(classOf[SimpleKeyGenerator].getName)
     .withDocumentation("Key generator class, that implements `org.apache.hudi.keygen.KeyGenerator`")
 
@@ -328,7 +328,7 @@ object DataSourceWriteOptions {
     .withDocumentation("Config to indicate whether to ignore any non exception error (e.g. writestatus error)"
       + " within a streaming microbatch")
 
-  val META_SYNC_CLIENT_TOOL_CLASS: ConfigProperty[String] = ConfigProperty
+  val META_SYNC_CLIENT_TOOL_CLASS_NAME: ConfigProperty[String] = ConfigProperty
     .key("hoodie.meta.sync.client.tool.class")
     .defaultValue(classOf[HiveSyncTool].getName)
     .withDocumentation("Sync tool class name used to sync to metastore. Defaults to Hive.")
@@ -403,7 +403,7 @@ object DataSourceWriteOptions {
     .defaultValue("false")
     .withDocumentation("")
 
-  /* @deprecated We should use {@link HIVE_SYNC_MODE} instead of this config from 0.9.0 */
+  /** @deprecated Use {@link HIVE_SYNC_MODE} instead of this config from 0.9.0 */
   @Deprecated
   val HIVE_USE_JDBC: ConfigProperty[String] = ConfigProperty
     .key("hoodie.datasource.hive_sync.use_jdbc")
@@ -421,12 +421,12 @@ object DataSourceWriteOptions {
     .defaultValue("false")
     .withDocumentation("")
 
-  val HIVE_SKIP_RO_SUFFIX: ConfigProperty[String] = ConfigProperty
+  val HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE: ConfigProperty[String] = ConfigProperty
     .key("hoodie.datasource.hive_sync.skip_ro_suffix")
     .defaultValue("false")
     .withDocumentation("Skip the _ro suffix for Read optimized table, when registering")
 
-  val HIVE_SUPPORT_TIMESTAMP: ConfigProperty[String] = ConfigProperty
+  val HIVE_SUPPORT_TIMESTAMP_TYPE: ConfigProperty[String] = ConfigProperty
     .key("hoodie.datasource.hive_sync.support_timestamp")
     .defaultValue("false")
     .withDocumentation("‘INT64’ with original type TIMESTAMP_MICROS is converted to hive ‘timestamp’ type. " +
@@ -514,9 +514,12 @@ object DataSourceWriteOptions {
   /** @deprecated Use {@link STREAMING_IGNORE_FAILED_BATCH} and its methods instead */
   @Deprecated
   val DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL = STREAMING_IGNORE_FAILED_BATCH.defaultValue()
-  /** @deprecated Use {@link META_SYNC_CLIENT_TOOL_CLASS} and its methods instead */
+  /** @deprecated Use {@link META_SYNC_CLIENT_TOOL_CLASS_NAME} and its methods instead */
   @Deprecated
-  val DEFAULT_META_SYNC_CLIENT_TOOL_CLASS = META_SYNC_CLIENT_TOOL_CLASS.defaultValue()
+  val META_SYNC_CLIENT_TOOL_CLASS = META_SYNC_CLIENT_TOOL_CLASS_NAME.key()
+  /** @deprecated Use {@link META_SYNC_CLIENT_TOOL_CLASS_NAME} and its methods instead */
+  @Deprecated
+  val DEFAULT_META_SYNC_CLIENT_TOOL_CLASS = META_SYNC_CLIENT_TOOL_CLASS_NAME.defaultValue()
   /** @deprecated Use {@link HIVE_SYNC_ENABLED} and its methods instead */
   @Deprecated
   val HIVE_SYNC_ENABLED_OPT_KEY = HIVE_SYNC_ENABLED.key()
@@ -550,10 +553,10 @@ object DataSourceWriteOptions {
 
   /** @deprecated Use {@link KEYGENERATOR_CLASS} and its methods instead */
   @Deprecated
-  val DEFAULT_KEYGENERATOR_CLASS_OPT_VAL = KEYGENERATOR_CLASS.defaultValue()
+  val DEFAULT_KEYGENERATOR_CLASS_OPT_VAL = KEYGENERATOR_CLASS_NAME.defaultValue()
   /** @deprecated Use {@link KEYGENERATOR_CLASS} and its methods instead */
   @Deprecated
-  val KEYGENERATOR_CLASS_OPT_KEY = HoodieWriteConfig.KEYGENERATOR_CLASS.key()
+  val KEYGENERATOR_CLASS_OPT_KEY = HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key()
   /** @deprecated Use {@link ENABLE_ROW_WRITER} and its methods instead */
   @Deprecated
   val ENABLE_ROW_WRITER_OPT_KEY = ENABLE_ROW_WRITER.key()
@@ -622,12 +625,12 @@ object DataSourceWriteOptions {
   @Deprecated
   val DEFAULT_PRECOMBINE_FIELD_OPT_VAL = PRECOMBINE_FIELD.defaultValue()
 
-  /** @deprecated Use {@link HoodieWriteConfig.WRITE_PAYLOAD_CLASS} and its methods instead */
+  /** @deprecated Use {@link HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME} and its methods instead */
   @Deprecated
-  val PAYLOAD_CLASS_OPT_KEY = HoodieWriteConfig.WRITE_PAYLOAD_CLASS.key()
-  /** @deprecated Use {@link HoodieWriteConfig.WRITE_PAYLOAD_CLASS} and its methods instead */
+  val PAYLOAD_CLASS_OPT_KEY = HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME.key()
+  /** @deprecated Use {@link HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME} and its methods instead */
   @Deprecated
-  val DEFAULT_PAYLOAD_OPT_VAL = PAYLOAD_CLASS.defaultValue()
+  val DEFAULT_PAYLOAD_OPT_VAL = PAYLOAD_CLASS_NAME.defaultValue()
 
   /** @deprecated Use {@link TABLE_TYPE} and its methods instead */
   @Deprecated
@@ -638,7 +641,7 @@ object DataSourceWriteOptions {
 
   /** @deprecated Use {@link TABLE_TYPE} and its methods instead */
   @Deprecated
-  val STORAGE_TYPE_OPT = "hoodie.datasource.write.storage.type"
+  val STORAGE_TYPE_OPT_KEY = "hoodie.datasource.write.storage.type"
   @Deprecated
   val COW_STORAGE_TYPE_OPT_VAL = HoodieTableType.COPY_ON_WRITE.name
   @Deprecated
@@ -698,12 +701,18 @@ object DataSourceWriteOptions {
   /** @deprecated Use {@link HIVE_IGNORE_EXCEPTIONS} and its methods instead */
   @Deprecated
   val DEFAULT_HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HIVE_IGNORE_EXCEPTIONS.defaultValue()
-  /** @deprecated Use {@link HIVE_SKIP_RO_SUFFIX} and its methods instead */
+  /** @deprecated Use {@link HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE} and its methods instead */
+  @Deprecated
+  val HIVE_SKIP_RO_SUFFIX = HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key()
+  /** @deprecated Use {@link HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE} and its methods instead */
   @Deprecated
-  val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = HIVE_SKIP_RO_SUFFIX.defaultValue()
-  /** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP} and its methods instead */
+  val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue()
+  /** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP_TYPE} and its methods instead */
   @Deprecated
-  val DEFAULT_HIVE_SUPPORT_TIMESTAMP = HIVE_SUPPORT_TIMESTAMP.defaultValue()
+  val HIVE_SUPPORT_TIMESTAMP = HIVE_SUPPORT_TIMESTAMP_TYPE.key()
+  /** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP_TYPE} and its methods instead */
+  @Deprecated
+  val DEFAULT_HIVE_SUPPORT_TIMESTAMP = HIVE_SUPPORT_TIMESTAMP_TYPE.defaultValue()
   /** @deprecated Use {@link ASYNC_COMPACT_ENABLE} and its methods instead */
   @Deprecated
   val ASYNC_COMPACT_ENABLE_OPT_KEY = ASYNC_COMPACT_ENABLE.key()
@@ -713,6 +722,8 @@ object DataSourceWriteOptions {
   /** @deprecated Use {@link KAFKA_AVRO_VALUE_DESERIALIZER_CLASS} and its methods instead */
   @Deprecated
   val KAFKA_AVRO_VALUE_DESERIALIZER = KAFKA_AVRO_VALUE_DESERIALIZER_CLASS.key()
+  @Deprecated
+  val SCHEMA_PROVIDER_CLASS_PROP = "hoodie.deltastreamer.schemaprovider.class"
 }
 
 object DataSourceOptionsHelper {
@@ -723,13 +734,13 @@ object DataSourceOptionsHelper {
   val allConfigsWithAlternatives = List(
     DataSourceReadOptions.QUERY_TYPE,
     DataSourceWriteOptions.TABLE_TYPE,
-    HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP,
-    HoodieTableConfig.HOODIE_LOG_FILE_FORMAT_PROP
+    HoodieTableConfig.BASE_FILE_FORMAT,
+    HoodieTableConfig.LOG_FILE_FORMAT
   )
 
   // put all the deprecated configs here
   val allDeprecatedConfigs: Set[String] = Set(
-    ConsistencyGuardConfig.CONSISTENCY_CHECK_ENABLED_PROP.key
+    ConsistencyGuardConfig.ENABLE.key
   )
 
   // maps the deprecated config name to its latest name
diff --git a/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java b/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java
index 690f613..b19b098 100644
--- a/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java
+++ b/hudi-spark-datasource/hudi-spark-common/src/test/java/org/apache/hudi/internal/HoodieBulkInsertInternalWriterTestBase.java
@@ -72,10 +72,10 @@ public class HoodieBulkInsertInternalWriterTestBase extends HoodieClientTestHarn
   protected HoodieWriteConfig getWriteConfig(boolean populateMetaFields) {
     Properties properties = new Properties();
     if (!populateMetaFields) {
-      properties.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(), SimpleKeyGenerator.class.getName());
+      properties.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), SimpleKeyGenerator.class.getName());
       properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD().key(), SparkDatasetTestUtils.RECORD_KEY_FIELD_NAME);
       properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), SparkDatasetTestUtils.PARTITION_PATH_FIELD_NAME);
-      properties.setProperty(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), "false");
+      properties.setProperty(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false");
     }
     return getConfigBuilder(basePath).withProperties(properties).build();
   }
diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java
index 47e97aa..c96d216 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java
+++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/HoodieDatasetBulkInsertHelper.java
@@ -77,7 +77,7 @@ public class HoodieDatasetBulkInsertHelper {
 
     TypedProperties properties = new TypedProperties();
     properties.putAll(config.getProps());
-    String keyGeneratorClass = properties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS().key());
+    String keyGeneratorClass = properties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key());
     BuiltinKeyGenerator keyGenerator = (BuiltinKeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, properties);
     StructType structTypeForUDF = rows.schema();
 
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala
index f94d228..b87be99 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieFileIndex.scala
@@ -17,8 +17,6 @@
 
 package org.apache.hudi
 
-import java.util.Properties
-import scala.collection.JavaConverters._
 import org.apache.hadoop.fs.{FileStatus, Path}
 import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_SNAPSHOT_OPT_VAL}
 import org.apache.hudi.client.common.HoodieSparkEngineContext
@@ -26,21 +24,23 @@ import org.apache.hudi.common.config.HoodieMetadataConfig
 import org.apache.hudi.common.fs.FSUtils
 import org.apache.hudi.common.model.FileSlice
 import org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ
-import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
 import org.apache.hudi.common.table.view.{FileSystemViewStorageConfig, HoodieTableFileSystemView}
+import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
 import org.apache.spark.api.java.JavaSparkContext
 import org.apache.spark.internal.Logging
-import org.apache.spark.sql.catalyst.{InternalRow, expressions}
 import org.apache.spark.sql.SparkSession
 import org.apache.spark.sql.avro.SchemaConverters
 import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BoundReference, Expression, InterpretedPredicate}
 import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils}
+import org.apache.spark.sql.catalyst.{InternalRow, expressions}
 import org.apache.spark.sql.execution.datasources.{FileIndex, FileStatusCache, NoopCache, PartitionDirectory}
 import org.apache.spark.sql.hudi.HoodieSqlUtils
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.unsafe.types.UTF8String
 
+import java.util.Properties
+import scala.collection.JavaConverters._
 import scala.collection.mutable
 
 /**
@@ -122,12 +122,12 @@ case class HoodieFileIndex(
 
     // To support metadata listing via Spark SQL we allow users to pass the config via SQL Conf in spark session. Users
     // would be able to run SET hoodie.metadata.enable=true in the spark sql session to enable metadata listing.
-    properties.put(HoodieMetadataConfig.METADATA_ENABLE_PROP,
-      sqlConf.getConfString(HoodieMetadataConfig.METADATA_ENABLE_PROP.key(),
+    properties.put(HoodieMetadataConfig.ENABLE,
+      sqlConf.getConfString(HoodieMetadataConfig.ENABLE.key(),
         HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS.toString))
-    properties.put(HoodieMetadataConfig.METADATA_VALIDATE_PROP,
-      sqlConf.getConfString(HoodieMetadataConfig.METADATA_VALIDATE_PROP.key(),
-        HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue().toString))
+    properties.put(HoodieMetadataConfig.VALIDATE_ENABLE,
+      sqlConf.getConfString(HoodieMetadataConfig.VALIDATE_ENABLE.key(),
+        HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue().toString))
     properties.putAll(options.asJava)
     properties
   }
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
index 87e7b22..83b37ac 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
@@ -18,9 +18,6 @@
 package org.apache.hudi
 
 
-import java.util
-import java.util.Properties
-
 import org.apache.avro.Schema
 import org.apache.avro.generic.GenericRecord
 import org.apache.hadoop.conf.Configuration
@@ -35,7 +32,7 @@ import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, Write
 import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
 import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
 import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils}
-import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH, BOOTSTRAP_INDEX_CLASS}
+import org.apache.hudi.config.HoodieBootstrapConfig.{BASE_PATH, INDEX_CLASS_NAME}
 import org.apache.hudi.config.{HoodieInternalConfig, HoodieWriteConfig}
 import org.apache.hudi.exception.HoodieException
 import org.apache.hudi.execution.bulkinsert.{BulkInsertInternalPartitionerWithRowsFactory, NonSortPartitionerWithRows}
@@ -53,6 +50,8 @@ import org.apache.spark.sql.types.StructType
 import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext, SaveMode, SparkSession}
 import org.apache.spark.{SPARK_VERSION, SparkContext}
 
+import java.util
+import java.util.Properties
 import scala.collection.JavaConversions._
 import scala.collection.mutable.ListBuffer
 
@@ -78,7 +77,7 @@ object HoodieSparkSqlWriter {
     val sparkContext = sqlContext.sparkContext
     val path = parameters.get("path")
     val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters)
-    val tblNameOp = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TABLE_NAME, s"'${HoodieWriteConfig.TABLE_NAME.key}' must be set.")
+    val tblNameOp = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TBL_NAME, s"'${HoodieWriteConfig.TBL_NAME.key}' must be set.")
     asyncCompactionTriggerFnDefined = asyncCompactionTriggerFn.isDefined
     asyncClusteringTriggerFnDefined = asyncClusteringTriggerFn.isDefined
     if (path.isEmpty) {
@@ -121,10 +120,10 @@ object HoodieSparkSqlWriter {
       val partitionColumns = HoodieSparkUtils.getPartitionColumns(keyGenerator, toProperties(parameters))
       // Create the table if not present
       if (!tableExists) {
-        val baseFileFormat = hoodieConfig.getStringOrDefault(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP)
-        val archiveLogFolder = hoodieConfig.getStringOrDefault(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP)
+        val baseFileFormat = hoodieConfig.getStringOrDefault(HoodieTableConfig.BASE_FILE_FORMAT)
+        val archiveLogFolder = hoodieConfig.getStringOrDefault(HoodieTableConfig.ARCHIVELOG_FOLDER)
         val recordKeyFields = hoodieConfig.getString(DataSourceWriteOptions.RECORDKEY_FIELD)
-        val populateMetaFields = parameters.getOrElse(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()).toBoolean
+        val populateMetaFields = parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(), HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()).toBoolean
 
         val tableMetaClient = HoodieTableMetaClient.withPropertyBuilder()
           .setTableType(tableType)
@@ -132,12 +131,12 @@ object HoodieSparkSqlWriter {
           .setRecordKeyFields(recordKeyFields)
           .setBaseFileFormat(baseFileFormat)
           .setArchiveLogFolder(archiveLogFolder)
-          .setPayloadClassName(hoodieConfig.getString(PAYLOAD_CLASS))
+          .setPayloadClassName(hoodieConfig.getString(PAYLOAD_CLASS_NAME))
           .setPreCombineField(hoodieConfig.getStringOrDefault(PRECOMBINE_FIELD, null))
           .setPartitionFields(partitionColumns)
           .setPopulateMetaFields(populateMetaFields)
           .setRecordKeyFields(hoodieConfig.getString(RECORDKEY_FIELD))
-          .setKeyGeneratorClassProp(hoodieConfig.getString(KEYGENERATOR_CLASS))
+          .setKeyGeneratorClassProp(hoodieConfig.getString(KEYGENERATOR_CLASS_NAME))
           .initTable(sparkContext.hadoopConfiguration, path.get)
         tableConfig = tableMetaClient.getTableConfig
       }
@@ -169,8 +168,8 @@ object HoodieSparkSqlWriter {
 
             // Create a HoodieWriteClient & issue the delete.
             val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
-            null, path.get, tblName,
-            mapAsJavaMap(parameters - HoodieWriteConfig.HOODIE_AUTO_COMMIT.key)))
+              null, path.get, tblName,
+              mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)))
             .asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
 
             if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
@@ -201,7 +200,7 @@ object HoodieSparkSqlWriter {
             // Create a HoodieWriteClient & issue the delete.
             val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
               null, path.get, tblName,
-              mapAsJavaMap(parameters - HoodieWriteConfig.HOODIE_AUTO_COMMIT.key)))
+              mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)))
               .asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
             // Issue delete partitions
             client.startCommitWithTime(instantTime, commitActionType)
@@ -235,9 +234,9 @@ object HoodieSparkSqlWriter {
                   .asInstanceOf[Comparable[_]]
                 DataSourceUtils.createHoodieRecord(processedRecord,
                   orderingVal, keyGenerator.getKey(gr),
-                  hoodieConfig.getString(PAYLOAD_CLASS))
+                  hoodieConfig.getString(PAYLOAD_CLASS_NAME))
               } else {
-                DataSourceUtils.createHoodieRecord(processedRecord, keyGenerator.getKey(gr), hoodieConfig.getString(PAYLOAD_CLASS))
+                DataSourceUtils.createHoodieRecord(processedRecord, keyGenerator.getKey(gr), hoodieConfig.getString(PAYLOAD_CLASS_NAME))
               }
               hoodieRecord
             }).toJavaRDD()
@@ -245,7 +244,7 @@ object HoodieSparkSqlWriter {
             val writeSchema = if (dropPartitionColumns) generateSchemaWithoutPartitionColumns(partitionColumns, schema) else schema
             // Create a HoodieWriteClient & issue the write.
             val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, writeSchema.toString, path.get,
-              tblName, mapAsJavaMap(parameters - HoodieWriteConfig.HOODIE_AUTO_COMMIT.key)
+              tblName, mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)
             )).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
 
             if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
@@ -334,12 +333,12 @@ object HoodieSparkSqlWriter {
     val sparkContext = sqlContext.sparkContext
     val path = parameters.getOrElse("path", throw new HoodieException("'path' must be set."))
     val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters)
-    val tableName = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TABLE_NAME, s"'${HoodieWriteConfig.TABLE_NAME.key}' must be set.")
+    val tableName = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TBL_NAME, s"'${HoodieWriteConfig.TBL_NAME.key}' must be set.")
     val tableType = hoodieConfig.getStringOrDefault(TABLE_TYPE)
-    val bootstrapBasePath = hoodieConfig.getStringOrThrow(BOOTSTRAP_BASE_PATH,
-      s"'${BOOTSTRAP_BASE_PATH.key}' is required for '${BOOTSTRAP_OPERATION_OPT_VAL}'" +
+    val bootstrapBasePath = hoodieConfig.getStringOrThrow(BASE_PATH,
+      s"'${BASE_PATH.key}' is required for '${BOOTSTRAP_OPERATION_OPT_VAL}'" +
         " operation'")
-    val bootstrapIndexClass = hoodieConfig.getStringOrDefault(BOOTSTRAP_INDEX_CLASS)
+    val bootstrapIndexClass = hoodieConfig.getStringOrDefault(INDEX_CLASS_NAME)
 
     var schema: String = null
     if (df.schema.nonEmpty) {
@@ -363,18 +362,18 @@ object HoodieSparkSqlWriter {
     }
 
     if (!tableExists) {
-      val archiveLogFolder = hoodieConfig.getStringOrDefault(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP)
+      val archiveLogFolder = hoodieConfig.getStringOrDefault(HoodieTableConfig.ARCHIVELOG_FOLDER)
       val partitionColumns = HoodieWriterUtils.getPartitionColumns(parameters)
       val recordKeyFields = hoodieConfig.getString(DataSourceWriteOptions.RECORDKEY_FIELD)
-      val keyGenProp = hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS)
-      val populateMetaFields = parameters.getOrElse(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()).toBoolean
+      val keyGenProp = hoodieConfig.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)
+      val populateMetaFields = parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(), HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()).toBoolean
 
       HoodieTableMetaClient.withPropertyBuilder()
         .setTableType(HoodieTableType.valueOf(tableType))
         .setTableName(tableName)
         .setRecordKeyFields(recordKeyFields)
         .setArchiveLogFolder(archiveLogFolder)
-        .setPayloadClassName(hoodieConfig.getStringOrDefault(PAYLOAD_CLASS))
+        .setPayloadClassName(hoodieConfig.getStringOrDefault(PAYLOAD_CLASS_NAME))
         .setPreCombineField(hoodieConfig.getStringOrDefault(PRECOMBINE_FIELD, null))
         .setBootstrapIndexClass(bootstrapIndexClass)
         .setBootstrapBasePath(bootstrapBasePath)
@@ -405,8 +404,8 @@ object HoodieSparkSqlWriter {
                       instantTime: String,
                       partitionColumns: String): (Boolean, common.util.Option[String]) = {
     val sparkContext = sqlContext.sparkContext
-    val populateMetaFields = parameters.getOrElse(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(),
-      HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()).toBoolean
+    val populateMetaFields = parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(),
+      HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()).toBoolean
     val dropPartitionColumns =
       parameters.getOrElse(DataSourceWriteOptions.DROP_PARTITION_COLUMNS.key(), DataSourceWriteOptions.DROP_PARTITION_COLUMNS.defaultValue()).toBoolean
     // register classes & schemas
@@ -423,7 +422,7 @@ object HoodieSparkSqlWriter {
     if (parameters(INSERT_DROP_DUPS.key).toBoolean) {
       throw new HoodieException("Dropping duplicates with bulk_insert in row writer path is not supported yet")
     }
-    val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA.key, schema.toString)
+    val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA_STRING.key, schema.toString)
     val writeConfig = DataSourceUtils.createHoodieConfig(schema.toString, path.get, tblName, mapAsJavaMap(params))
     val bulkInsertPartitionerRows : BulkInsertPartitioner[Dataset[Row]] = if (populateMetaFields) {
       val userDefinedBulkInsertPartitionerOpt = DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(writeConfig)
@@ -533,21 +532,21 @@ object HoodieSparkSqlWriter {
     hiveSyncConfig.hiveUser = hoodieConfig.getString(HIVE_USER)
     hiveSyncConfig.hivePass = hoodieConfig.getString(HIVE_PASS)
     hiveSyncConfig.jdbcUrl = hoodieConfig.getString(HIVE_URL)
-    hiveSyncConfig.skipROSuffix = hoodieConfig.getStringOrDefault(HIVE_SKIP_RO_SUFFIX,
-      DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX.defaultValue).toBoolean
+    hiveSyncConfig.skipROSuffix = hoodieConfig.getStringOrDefault(HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE,
+      DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue).toBoolean
     hiveSyncConfig.partitionFields =
       ListBuffer(hoodieConfig.getString(HIVE_PARTITION_FIELDS).split(",").map(_.trim).filter(!_.isEmpty).toList: _*)
     hiveSyncConfig.partitionValueExtractorClass = hoodieConfig.getString(HIVE_PARTITION_EXTRACTOR_CLASS)
     hiveSyncConfig.useJdbc = hoodieConfig.getBoolean(HIVE_USE_JDBC)
-    hiveSyncConfig.useFileListingFromMetadata = hoodieConfig.getBoolean(HoodieMetadataConfig.METADATA_ENABLE_PROP)
-    hiveSyncConfig.verifyMetadataFileListing = hoodieConfig.getBoolean(HoodieMetadataConfig.METADATA_VALIDATE_PROP)
+    hiveSyncConfig.useFileListingFromMetadata = hoodieConfig.getBoolean(HoodieMetadataConfig.ENABLE)
+    hiveSyncConfig.verifyMetadataFileListing = hoodieConfig.getBoolean(HoodieMetadataConfig.VALIDATE_ENABLE)
     hiveSyncConfig.ignoreExceptions = hoodieConfig.getStringOrDefault(HIVE_IGNORE_EXCEPTIONS).toBoolean
-    hiveSyncConfig.supportTimestamp = hoodieConfig.getStringOrDefault(HIVE_SUPPORT_TIMESTAMP).toBoolean
+    hiveSyncConfig.supportTimestamp = hoodieConfig.getStringOrDefault(HIVE_SUPPORT_TIMESTAMP_TYPE).toBoolean
     hiveSyncConfig.autoCreateDatabase = hoodieConfig.getStringOrDefault(HIVE_AUTO_CREATE_DATABASE).toBoolean
     hiveSyncConfig.decodePartition = hoodieConfig.getStringOrDefault(URL_ENCODE_PARTITIONING).toBoolean
     hiveSyncConfig.batchSyncNum = hoodieConfig.getStringOrDefault(HIVE_BATCH_SYNC_PARTITION_NUM).toInt
 
-    hiveSyncConfig.syncAsSparkDataSourceTable =  hoodieConfig.getStringOrDefault(HIVE_SYNC_AS_DATA_SOURCE_TABLE).toBoolean
+    hiveSyncConfig.syncAsSparkDataSourceTable = hoodieConfig.getStringOrDefault(HIVE_SYNC_AS_DATA_SOURCE_TABLE).toBoolean
     hiveSyncConfig.sparkSchemaLengthThreshold = sqlConf.getConf(StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD)
     hiveSyncConfig.createManagedTable = hoodieConfig.getBoolean(HIVE_CREATE_MANAGED_TABLE)
     hiveSyncConfig.syncMode = hoodieConfig.getString(HIVE_SYNC_MODE)
@@ -561,7 +560,7 @@ object HoodieSparkSqlWriter {
     val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HIVE_SYNC_ENABLED).toBoolean
     var metaSyncEnabled = hoodieConfig.getStringOrDefault(META_SYNC_ENABLED).toBoolean
     var syncClientToolClassSet = scala.collection.mutable.Set[String]()
-    hoodieConfig.getString(META_SYNC_CLIENT_TOOL_CLASS).split(",").foreach(syncClass =>  syncClientToolClassSet += syncClass)
+    hoodieConfig.getString(META_SYNC_CLIENT_TOOL_CLASS_NAME).split(",").foreach(syncClass =>  syncClientToolClassSet += syncClass)
 
     // for backward compatibility
     if (hiveSyncEnabled) {
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala
index da06cbc..51b632b 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala
@@ -17,13 +17,12 @@
 
 package org.apache.hudi
 
-import java.util.Properties
-
 import org.apache.hudi.DataSourceWriteOptions._
-import org.apache.hudi.common.config.HoodieMetadataConfig.{METADATA_ENABLE_PROP, METADATA_VALIDATE_PROP}
+import org.apache.hudi.common.config.HoodieMetadataConfig.{ENABLE, VALIDATE_ENABLE}
 import org.apache.hudi.common.config.{HoodieConfig, TypedProperties}
 import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
 
+import java.util.Properties
 import scala.collection.JavaConversions.mapAsJavaMap
 import scala.collection.JavaConverters.{mapAsScalaMapConverter, _}
 
@@ -46,18 +45,18 @@ object HoodieWriterUtils {
     Map(OPERATION.key -> OPERATION.defaultValue,
       TABLE_TYPE.key -> TABLE_TYPE.defaultValue,
       PRECOMBINE_FIELD.key -> PRECOMBINE_FIELD.defaultValue,
-      PAYLOAD_CLASS.key -> PAYLOAD_CLASS.defaultValue,
+      PAYLOAD_CLASS_NAME.key -> PAYLOAD_CLASS_NAME.defaultValue,
       RECORDKEY_FIELD.key -> RECORDKEY_FIELD.defaultValue,
       PARTITIONPATH_FIELD.key -> PARTITIONPATH_FIELD.defaultValue,
-      KEYGENERATOR_CLASS.key -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL,
-      METADATA_ENABLE_PROP.key -> METADATA_ENABLE_PROP.defaultValue.toString,
-      METADATA_VALIDATE_PROP.key -> METADATA_VALIDATE_PROP.defaultValue.toString,
+      KEYGENERATOR_CLASS_NAME.key -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL,
+      ENABLE.key -> ENABLE.defaultValue.toString,
+      VALIDATE_ENABLE.key -> VALIDATE_ENABLE.defaultValue.toString,
       COMMIT_METADATA_KEYPREFIX.key -> COMMIT_METADATA_KEYPREFIX.defaultValue,
       INSERT_DROP_DUPS.key -> INSERT_DROP_DUPS.defaultValue,
       STREAMING_RETRY_CNT.key -> STREAMING_RETRY_CNT.defaultValue,
       STREAMING_RETRY_INTERVAL_MS.key -> STREAMING_RETRY_INTERVAL_MS.defaultValue,
       STREAMING_IGNORE_FAILED_BATCH.key -> STREAMING_IGNORE_FAILED_BATCH.defaultValue,
-      META_SYNC_CLIENT_TOOL_CLASS.key -> META_SYNC_CLIENT_TOOL_CLASS.defaultValue,
+      META_SYNC_CLIENT_TOOL_CLASS_NAME.key -> META_SYNC_CLIENT_TOOL_CLASS_NAME.defaultValue,
       HIVE_SYNC_ENABLED.key -> HIVE_SYNC_ENABLED.defaultValue,
       META_SYNC_ENABLED.key -> META_SYNC_ENABLED.defaultValue,
       HIVE_DATABASE.key -> HIVE_DATABASE.defaultValue,
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala
index 5120fe7..25d3026 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/HoodieOptionConfig.scala
@@ -42,26 +42,26 @@ object HoodieOptionConfig {
   val SQL_KEY_TABLE_PRIMARY_KEY: HoodieOption[String] = buildConf()
     .withSqlKey("primaryKey")
     .withHoodieKey(DataSourceWriteOptions.RECORDKEY_FIELD.key)
-    .withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS.key)
+    .withTableConfigKey(HoodieTableConfig.RECORDKEY_FIELDS.key)
     .build()
 
   val SQL_KEY_TABLE_TYPE: HoodieOption[String] = buildConf()
     .withSqlKey("type")
     .withHoodieKey(DataSourceWriteOptions.TABLE_TYPE.key)
-    .withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP.key)
+    .withTableConfigKey(HoodieTableConfig.TYPE.key)
     .defaultValue(SQL_VALUE_TABLE_TYPE_COW)
     .build()
 
   val SQL_KEY_PRECOMBINE_FIELD: HoodieOption[String] = buildConf()
     .withSqlKey("preCombineField")
     .withHoodieKey(DataSourceWriteOptions.PRECOMBINE_FIELD.key)
-    .withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP.key)
+    .withTableConfigKey(HoodieTableConfig.PRECOMBINE_FIELD.key)
     .build()
 
   val SQL_PAYLOAD_CLASS: HoodieOption[String] = buildConf()
     .withSqlKey("payloadClass")
-    .withHoodieKey(DataSourceWriteOptions.PAYLOAD_CLASS.key)
-    .withTableConfigKey(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP.key)
+    .withHoodieKey(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.key)
+    .withTableConfigKey(HoodieTableConfig.PAYLOAD_CLASS_NAME.key)
     .defaultValue(classOf[DefaultHoodieRecordPayload].getName)
     .build()
 
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
index 12a67bb..00e8afb 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableCommand.scala
@@ -17,22 +17,19 @@
 
 package org.apache.spark.sql.hudi.command
 
-import scala.collection.JavaConverters._
-import java.util.{Locale, Properties}
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 import org.apache.hudi.client.common.HoodieSparkEngineContext
 import org.apache.hudi.common.config.HoodieMetadataConfig
 import org.apache.hudi.common.fs.FSUtils
-import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
 import org.apache.hudi.common.model.HoodieFileFormat
 import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
 import org.apache.hudi.common.util.ValidationUtils
 import org.apache.hudi.hadoop.HoodieParquetInputFormat
 import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat
 import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils
+import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
 import org.apache.spark.api.java.JavaSparkContext
-import org.apache.spark.{SPARK_VERSION, SparkConf}
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.avro.SchemaConverters
 import org.apache.spark.sql.catalyst.TableIdentifier
@@ -41,13 +38,16 @@ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat,
 import org.apache.spark.sql.execution.command.RunnableCommand
 import org.apache.spark.sql.hive.HiveClientUtils
 import org.apache.spark.sql.hive.HiveExternalCatalog._
-import org.apache.spark.sql.hudi.HoodieSqlUtils._
 import org.apache.spark.sql.hudi.HoodieOptionConfig
+import org.apache.spark.sql.hudi.HoodieSqlUtils._
 import org.apache.spark.sql.hudi.command.CreateHoodieTableCommand.{initTableIfNeed, isEmptyPath}
 import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD
 import org.apache.spark.sql.types.StructType
 import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
+import org.apache.spark.{SPARK_VERSION, SparkConf}
 
+import java.util.{Locale, Properties}
+import scala.collection.JavaConverters._
 import scala.collection.mutable
 
 /**
@@ -107,27 +107,27 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean
      upgrateConfig = if (isNotHiveStyledPartitionTable(allPartitionPaths, table)) {
         upgrateConfig + (DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key -> "false")
      } else {
-        upgrateConfig
-     }
-     upgrateConfig = if (isUrlEncodeDisable(allPartitionPaths, table)) {
-       upgrateConfig + (DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key -> "false")
-     } else {
        upgrateConfig
      }
+      upgrateConfig = if (isUrlEncodeDisable(allPartitionPaths, table)) {
+        upgrateConfig + (DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key -> "false")
+      } else {
+        upgrateConfig
+      }
 
       // Use the origin keygen to generate record key to keep the rowkey consistent with the old table for spark sql.
       // See SqlKeyGenerator#getRecordKey for detail.
-     upgrateConfig = if (originTableConfig.contains(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS.key)) {
-        upgrateConfig + (SqlKeyGenerator.ORIGIN_KEYGEN_CLASS -> originTableConfig(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS.key))
-     } else {
+      upgrateConfig = if (originTableConfig.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) {
+        upgrateConfig + (SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> originTableConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key))
+      } else {
         upgrateConfig
-     }
-     val options = originTableConfig ++ upgrateConfig ++ table.storage.properties
+      }
+      val options = originTableConfig ++ upgrateConfig ++ table.storage.properties
 
-     val userSpecifiedSchema = table.schema
-     if (userSpecifiedSchema.isEmpty && tableSchema.isDefined) {
+      val userSpecifiedSchema = table.schema
+      if (userSpecifiedSchema.isEmpty && tableSchema.isDefined) {
         (addMetaFields(tableSchema.get), options)
-     }  else if (userSpecifiedSchema.nonEmpty) {
+      } else if (userSpecifiedSchema.nonEmpty) {
         (addMetaFields(userSpecifiedSchema), options)
     } else {
         throw new IllegalArgumentException(s"Missing schema for Create Table: $tableName")
@@ -392,19 +392,19 @@ object CreateHoodieTableCommand extends Logging {
     val tableName = table.identifier.table
     logInfo(s"Init hoodie.properties for $tableName")
     val tableOptions = HoodieOptionConfig.mappingSqlOptionToTableConfig(table.storage.properties)
-    checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP.key)
-    checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP.key)
-    checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS.key)
+    checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PRECOMBINE_FIELD.key)
+    checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PARTITION_FIELDS.key)
+    checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.RECORDKEY_FIELDS.key)
     // Save all the table config to the hoodie.properties.
     val parameters = originTableConfig ++ tableOptions
     val properties = new Properties()
-      properties.putAll(parameters.asJava)
-      HoodieTableMetaClient.withPropertyBuilder()
-        .fromProperties(properties)
-        .setTableName(tableName)
-        .setTableCreateSchema(SchemaConverters.toAvroType(table.schema).toString())
-        .setPartitionFields(table.partitionColumnNames.mkString(","))
-        .initTable(conf, location)
+    properties.putAll(parameters.asJava)
+    HoodieTableMetaClient.withPropertyBuilder()
+      .fromProperties(properties)
+      .setTableName(tableName)
+      .setTableCreateSchema(SchemaConverters.toAvroType(table.schema).toString())
+      .setPartitionFields(table.partitionColumnNames.mkString(","))
+      .initTable(conf, location)
   }
 
   def checkTableConfigEqual(originTableConfig: Map[String, String],
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala
index cca9e15..4d6d0a2 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala
@@ -17,14 +17,13 @@
 
 package org.apache.spark.sql.hudi.command
 
-import org.apache.hudi.DataSourceWriteOptions.OPERATION
-import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
-import org.apache.hudi.DataSourceWriteOptions._
+import org.apache.hudi.DataSourceWriteOptions.{OPERATION, _}
 import org.apache.hudi.config.HoodieWriteConfig
-import org.apache.hudi.config.HoodieWriteConfig.TABLE_NAME
+import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
 import org.apache.hudi.hive.ddl.HiveSyncMode
+import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
 import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.logical.DeleteFromTable
 import org.apache.spark.sql.execution.command.RunnableCommand
 import org.apache.spark.sql.hudi.HoodieOptionConfig
 import org.apache.spark.sql.hudi.HoodieSqlUtils._
@@ -67,14 +66,14 @@ case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends Runnab
     withSparkConf(sparkSession, targetTable.storage.properties) {
       Map(
         "path" -> path,
-        KEYGENERATOR_CLASS.key -> classOf[SqlKeyGenerator].getCanonicalName,
-        TABLE_NAME.key -> tableId.table,
+        KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName,
+        TBL_NAME.key -> tableId.table,
         OPERATION.key -> DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL,
         PARTITIONPATH_FIELD.key -> targetTable.partitionColumnNames.mkString(","),
         HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
-        HIVE_SUPPORT_TIMESTAMP.key -> "true",
+        HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
         HIVE_STYLE_PARTITIONING.key -> "true",
-        HoodieWriteConfig.DELETE_PARALLELISM.key -> "200",
+        HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200",
         SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL
       )
     }
diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala
index ec3e411..8ced2fe 100644
--- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala
+++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala
@@ -17,15 +17,14 @@
 
 package org.apache.spark.sql.hudi.command
 
-import java.util.Properties
 import org.apache.avro.Schema
 import org.apache.avro.generic.{GenericRecord, IndexedRecord}
+import org.apache.hudi.DataSourceWriteOptions._
 import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecord}
 import org.apache.hudi.common.util.{Option => HOption}
-import org.apache.hudi.exception.HoodieDuplicateKeyException
-import org.apache.hudi.DataSourceWriteOptions._
 import org.apache.hudi.config.HoodieWriteConfig
-import org.apache.hudi.config.HoodieWriteConfig.TABLE_NAME
+import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
+import org.apache.hudi.exception.HoodieDuplicateKeyException
 import org.apache.hudi.hive.MultiPartKeysValueExtractor
 import org.apache.hudi.hive.ddl.HiveSyncMode
 import org.apache.hudi.sql.InsertMode
@@ -33,22 +32,24 @@ import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkSqlWriter, HoodieWrit
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.catalyst.catalog.CatalogTable
 import org.apache.spark.sql.catalyst.expressions.{Alias, Literal}
-import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession}
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
 import org.apache.spark.sql.execution.command.RunnableCommand
 import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlUtils}
 import org.apache.spark.sql.hudi.HoodieSqlUtils._
+import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlUtils}
 import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession}
+
+import java.util.Properties
 
 /**
  * Command for insert into hoodie table.
  */
 case class InsertIntoHoodieTableCommand(
-    logicalRelation: LogicalRelation,
-    query: LogicalPlan,
-    partition: Map[String, Option[String]],
-    overwrite: Boolean)
+                                         logicalRelation: LogicalRelation,
+                                         query: LogicalPlan,
+                                         partition: Map[String, Option[String]],
+                                         overwrite: Boolean)
   extends RunnableCommand {
 
   override def run(sparkSession: SparkSession): Seq[Row] = {
@@ -262,13 +263,13 @@ object InsertIntoHoodieTableCommand extends Logging {
       Map(
         "path" -> path,
         TABLE_TYPE.key -> tableType,
-        TABLE_NAME.key -> table.identifier.table,
+        TBL_NAME.key -> table.identifier.table,
         PRECOMBINE_FIELD.key -> tableSchema.fields.last.name,
         OPERATION.key -> operation,
-        KEYGENERATOR_CLASS.key -> keyGenClass,
+        KEYGENERATOR_CLASS_NAME.key -> keyGenClass,
         RECORDKEY_FIELD.key -> primaryColumns.mkString(","),
         PARTITIONPATH_FIELD.key -> partitionFields,
-        PAYLOAD_CLASS.key -> payloadClassName,
+        PAYLOAD_CLASS_NAME.key -> payloadClassName,
         ENABLE_ROW_WRITER.key -> enableBulkInsert.toString,
         HoodieWriteConfig.COMBINE_BEFORE_INSERT.key -> isPrimaryKeyTable.toString,
         META_SYNC_ENABLED.key -> enableHive.toString,
@@ -276,13 +277,13 @@ object InsertIntoHoodieTableCommand extends Logging {
         HIVE_USE_JDBC.key -> "false",
         HIVE_DATABASE.key -> table.identifier.database.getOrElse("default"),
         HIVE_TABLE.key -> table.identifier.table,
-        HIVE_SUPPORT_TIMESTAMP.key -> "true",
+        HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
         HIVE_STYLE_PARTITIONING.key -> "true",
         HIVE_PARTITION_FIELDS.key -> partitionFields,
         HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
         URL_ENCODE_PARTITIONING.key -> "true",
-        HoodieWriteConfig.INSERT_PARALLELISM.key -> "200",
-        HoodieWriteConfig.UPSERT_PARALLELISM.key -> "200",
+        HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "200",
+        HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200",
         SqlKeyGenerator.PARTITION_SCHEMA -> table.partitionSchema.toDDL
... 1885 lines suppressed ...