You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by vi...@apache.org on 2021/06/30 21:26:41 UTC

[hudi] branch master updated: [HUDI-89] Add configOption & refactor all configs based on that (#2833)

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

vinoth 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 d412fb2  [HUDI-89] Add configOption & refactor all configs based on that (#2833)
d412fb2 is described below

commit d412fb2fe642417460532044cac162bb68f4bec4
Author: wenningd <we...@gmail.com>
AuthorDate: Wed Jun 30 14:26:30 2021 -0700

    [HUDI-89] Add configOption & refactor all configs based on that (#2833)
    
    Co-authored-by: Wenning Ding <we...@amazon.com>
---
 docker/demo/sparksql-incremental.commands          |  68 +-
 .../hudi/cli/commands/HoodieLogFileCommand.java    |   8 +-
 .../apache/hudi/cli/commands/RepairsCommand.java   |   2 +-
 .../org/apache/hudi/cli/commands/SparkMain.java    |  14 +-
 .../org/apache/hudi/cli/commands/TableCommand.java |   2 +-
 .../scala/org/apache/hudi/cli/SparkHelpers.scala   |   6 +-
 .../cli/commands/TestHoodieLogFileCommand.java     |   8 +-
 .../hudi/cli/commands/TestRepairsCommand.java      |   6 +-
 .../cli/commands/TestUpgradeDowngradeCommand.java  |  10 +-
 .../http/HoodieWriteCommitHttpCallbackClient.java  |  11 +-
 .../callback/util/HoodieCommitCallbackFactory.java |   2 +-
 .../hudi/client/AbstractHoodieWriteClient.java     |   8 +-
 .../hudi/client/transaction/lock/LockManager.java  |  10 +-
 .../lock/ZookeeperBasedLockProvider.java           |  44 +-
 .../apache/hudi/config/HoodieBootstrapConfig.java  | 135 +--
 .../apache/hudi/config/HoodieClusteringConfig.java | 193 ++--
 .../apache/hudi/config/HoodieCompactionConfig.java | 392 +++++----
 .../apache/hudi/config/HoodieHBaseIndexConfig.java | 300 ++++---
 .../org/apache/hudi/config/HoodieIndexConfig.java  | 307 ++++---
 .../org/apache/hudi/config/HoodieLockConfig.java   | 229 +++--
 .../org/apache/hudi/config/HoodieMemoryConfig.java | 105 ++-
 .../apache/hudi/config/HoodieMetricsConfig.java    | 135 +--
 .../hudi/config/HoodieMetricsDatadogConfig.java    | 115 ++-
 .../hudi/config/HoodieMetricsPrometheusConfig.java |  97 +-
 .../apache/hudi/config/HoodiePayloadConfig.java    |  43 +-
 .../apache/hudi/config/HoodieStorageConfig.java    | 202 +++--
 .../config/HoodieWriteCommitCallbackConfig.java    |  75 +-
 .../org/apache/hudi/config/HoodieWriteConfig.java  | 976 +++++++++++----------
 .../java/org/apache/hudi/io/HoodieMergeHandle.java |   2 +-
 .../src/main/java/org/apache/hudi/io/IOUtils.java  |  21 +-
 .../org/apache/hudi/keygen/BaseKeyGenerator.java   |   8 +-
 .../hudi/keygen/ComplexAvroKeyGenerator.java       |   4 +-
 .../apache/hudi/keygen/CustomAvroKeyGenerator.java |   4 +-
 .../hudi/keygen/GlobalAvroDeleteKeyGenerator.java  |   2 +-
 .../java/org/apache/hudi/keygen/KeyGenUtils.java   |   2 +-
 .../keygen/NonpartitionedAvroKeyGenerator.java     |   2 +-
 .../apache/hudi/keygen/SimpleAvroKeyGenerator.java |   4 +-
 .../keygen/TimestampBasedAvroKeyGenerator.java     |   8 +-
 .../hudi/keygen/constant/KeyGeneratorOptions.java  |  43 +-
 .../factory/HoodieAvroKeyGeneratorFactory.java     |   2 +-
 .../metadata/HoodieBackedTableMetadataWriter.java  |   4 +-
 .../java/org/apache/hudi/table/HoodieTable.java    |   2 +-
 .../hudi/table/HoodieTimelineArchiveLog.java       |   2 +-
 .../table/upgrade/AbstractUpgradeDowngrade.java    |   2 +-
 .../FileSystemBasedLockProviderTestClass.java      |  12 +-
 .../TestZookeeperBasedLockProvider.java            |  38 +-
 .../apache/hudi/config/TestHoodieWriteConfig.java  |   6 +-
 .../hudi/io/storage/TestHoodieOrcReaderWriter.java |   6 +-
 ...estCreateAvroKeyGeneratorByTypeWithFactory.java |   8 +-
 .../factory/TestHoodieAvroKeyGeneratorFactory.java |  16 +-
 .../datadog/TestHoodieMetricsDatadogConfig.java    |   2 +-
 .../hudi/testutils/HoodieWriteableTestTable.java   |   2 +-
 .../HoodieFlinkMergeOnReadTableCompactor.java      |   2 +-
 .../SparkRecentDaysClusteringPlanStrategy.java     |   2 +-
 .../SparkSortAndSizeExecutionStrategy.java         |  10 +-
 .../apache/hudi/client/utils/SparkMemoryUtils.java |   7 +-
 .../apache/hudi/keygen/ComplexKeyGenerator.java    |   4 +-
 .../org/apache/hudi/keygen/CustomKeyGenerator.java |   4 +-
 .../hudi/keygen/GlobalDeleteKeyGenerator.java      |   2 +-
 .../hudi/keygen/NonpartitionedKeyGenerator.java    |   2 +-
 .../org/apache/hudi/keygen/SimpleKeyGenerator.java |   4 +-
 .../hudi/keygen/TimestampBasedKeyGenerator.java    |   4 +-
 .../factory/HoodieSparkKeyGeneratorFactory.java    |   2 +-
 ...SparkExecuteClusteringCommitActionExecutor.java |   2 +-
 .../HoodieSparkMergeOnReadTableCompactor.java      |   2 +-
 .../hudi/client/TestHoodieClientMultiWriter.java   |   8 +-
 .../TestHoodieClientOnCopyOnWriteStorage.java      |   8 +-
 .../apache/hudi/index/hbase/TestHBaseIndex.java    |   6 +-
 .../index/hbase/TestHBaseQPSResourceAllocator.java |   2 +-
 .../java/org/apache/hudi/io/TestSparkIOUtils.java  |  14 +-
 .../hudi/keygen/TestComplexKeyGenerator.java       |  26 +-
 .../apache/hudi/keygen/TestCustomKeyGenerator.java |  30 +-
 .../hudi/keygen/TestGlobalDeleteKeyGenerator.java  |  12 +-
 .../keygen/TestNonpartitionedKeyGenerator.java     |  22 +-
 .../apache/hudi/keygen/TestSimpleKeyGenerator.java |  22 +-
 .../keygen/TestTimestampBasedKeyGenerator.java     |   6 +-
 .../TestCreateKeyGeneratorByTypeWithFactory.java   |   8 +-
 .../TestHoodieSparkKeyGeneratorFactory.java        |  16 +-
 .../hudi/metadata/TestHoodieBackedMetadata.java    |   4 +-
 .../apache/hudi/table/TestConsistencyGuard.java    |   2 +-
 .../hudi/table/TestHoodieMergeOnReadTable.java     |   2 +-
 .../strategy/TestHoodieCompactionStrategy.java     |   2 +-
 .../table/action/rollback/TestRollbackUtils.java   |   2 +-
 .../hudi/table/upgrade/TestUpgradeDowngrade.java   |  18 +-
 .../apache/hudi/common/config/ConfigProperty.java  | 154 ++++
 .../hudi/common/config/DefaultHoodieConfig.java    |  51 --
 .../apache/hudi/common/config/HoodieConfig.java    | 177 ++++
 .../hudi/common/config/HoodieMetadataConfig.java   | 169 ++--
 .../hudi/common/config/LockConfiguration.java      |  55 +-
 .../hudi/common/fs/ConsistencyGuardConfig.java     | 104 ++-
 .../java/org/apache/hudi/common/fs/FSUtils.java    |  11 +-
 .../common/model/DefaultHoodieRecordPayload.java   |   9 +-
 .../hudi/common/model/HoodiePayloadProps.java      |   6 +-
 .../hudi/common/table/HoodieTableConfig.java       | 228 ++---
 .../hudi/common/table/HoodieTableMetaClient.java   |  90 +-
 .../table/view/FileSystemViewStorageConfig.java    | 232 +++--
 .../metadata/HoodieMetadataFileSystemView.java     |   2 +-
 .../hudi/common/bootstrap/TestBootstrapIndex.java  |  10 +-
 .../hudi/common/config/TestConfigProperty.java     | 101 +++
 .../org/apache/hudi/common/fs/TestFSUtils.java     |   2 +-
 .../model/TestDefaultHoodieRecordPayload.java      |   4 +-
 .../common/model/TestHoodieDeltaWriteStat.java     |   2 +-
 .../view/TestPriorityBasedFileSystemView.java      |   2 +-
 .../hudi/common/testutils/FileCreateUtils.java     |   2 +-
 .../hudi/common/testutils/HoodieTestTable.java     |   2 +-
 .../hudi/common/testutils/HoodieTestUtils.java     |   6 +-
 .../hudi/common/util/TestTablePathUtils.java       |   2 +-
 .../examples/spark/HoodieDataSourceExample.scala   |  26 +-
 .../apache/hudi/configuration/FlinkOptions.java    |  10 +-
 .../java/org/apache/hudi/util/StreamerUtil.java    |   4 +-
 .../hudi/hadoop/utils/HoodieInputFormatUtils.java  |   5 +-
 .../apache/hudi/hadoop/TestInputPathHandler.java   |   6 +-
 .../hudi/integ/testsuite/HoodieTestSuiteJob.java   |   4 +-
 .../testsuite/dag/nodes/ValidateDatasetNode.java   |   8 +-
 .../reader/DFSHoodieDatasetInputReader.java        |   6 +-
 .../testsuite/dag/nodes/SparkInsertNode.scala      |  10 +-
 .../testsuite/dag/nodes/SparkUpsertNode.scala      |  10 +-
 .../testsuite/job/TestHoodieTestSuiteJob.java      |  10 +-
 .../main/java/org/apache/hudi/DataSourceUtils.java |  52 +-
 .../scala/org/apache/hudi/DataSourceOptions.scala  | 508 +++++++----
 .../apache/hudi/HoodieDatasetBulkInsertHelper.java |   2 +-
 .../main/scala/org/apache/hudi/DefaultSource.scala |  14 +-
 .../org/apache/hudi/HoodieSparkSqlWriter.scala     | 181 ++--
 .../org/apache/hudi/HoodieStreamingSink.scala      |  10 +-
 .../scala/org/apache/hudi/HoodieWriterUtils.scala  |  72 +-
 .../org/apache/hudi/IncrementalRelation.scala      |  24 +-
 .../hudi/MergeOnReadIncrementalRelation.scala      |  20 +-
 .../apache/hudi/MergeOnReadSnapshotRelation.scala  |   6 +-
 .../apache/spark/sql/hudi/HoodieOptionConfig.scala |  24 +-
 .../command/AlterHoodieTableRenameCommand.scala    |   2 +-
 .../hudi/command/DeleteHoodieTableCommand.scala    |  14 +-
 .../command/InsertIntoHoodieTableCommand.scala     |  44 +-
 .../hudi/command/MergeIntoHoodieTableCommand.scala |  46 +-
 .../hudi/command/TruncateHoodieTableCommand.scala  |   2 +-
 .../hudi/command/UpdateHoodieTableCommand.scala    |  32 +-
 .../hudi/command/payload/ExpressionPayload.scala   |   8 +-
 .../sql/hudi/streaming/HoodieStreamSource.scala    |   4 +-
 .../hudi-spark/src/test/java/HoodieJavaApp.java    |  76 +-
 .../src/test/java/HoodieJavaGenerateApp.java       |  36 +-
 .../src/test/java/HoodieJavaStreamingApp.java      |  38 +-
 .../hudi/TestHoodieDatasetBulkInsertHelper.java    |  12 +-
 .../java/org/apache/hudi/client/TestBootstrap.java |  16 +-
 .../org/apache/hudi/TestDataSourceDefaults.scala   |  62 +-
 .../org/apache/hudi/TestHoodieFileIndex.scala      |  52 +-
 .../functional/HoodieSparkSqlWriterSuite.scala     | 148 ++--
 .../apache/hudi/functional/TestCOWDataSource.scala |  94 +-
 .../functional/TestDataSourceForBootstrap.scala    | 126 +--
 .../apache/hudi/functional/TestMORDataSource.scala | 160 ++--
 .../hudi/functional/TestStreamingSource.scala      |  16 +-
 .../hudi/functional/TestStructuredStreaming.scala  |  24 +-
 .../apache/spark/sql/hudi/TestMergeIntoTable.scala |  10 +-
 .../org/apache/hudi/internal/DefaultSource.java    |   4 +-
 .../apache/hudi/spark3/internal/DefaultSource.java |   6 +-
 .../java/org/apache/hudi/dla/DLASyncConfig.java    |   2 +-
 .../hudi/hive/HiveMetastoreBasedLockProvider.java  |  38 +-
 .../java/org/apache/hudi/hive/HiveSyncConfig.java  |   2 +-
 .../TestHiveMetastoreBasedLockProvider.java        |  50 +-
 .../hudi/timeline/service/TimelineService.java     |   4 +-
 .../hudi/utilities/HoodieSnapshotCopier.java       |   2 +-
 .../kafka/HoodieWriteCommitKafkaCallback.java      |  38 +-
 .../HoodieWriteCommitKafkaCallbackConfig.java      |  50 +-
 .../utilities/deltastreamer/BootstrapExecutor.java |  15 +-
 .../hudi/utilities/deltastreamer/DeltaSync.java    |  16 +-
 .../HoodieMultiTableDeltaStreamer.java             |   2 +-
 .../deser/KafkaAvroSchemaDeserializer.java         |  10 +-
 .../hudi/utilities/perf/TimelineServerPerf.java    |   6 +-
 .../hudi/utilities/sources/AvroKafkaSource.java    |   4 +-
 .../hudi/utilities/sources/HoodieIncrSource.java   |   6 +-
 .../TestKafkaConnectHdfsProvider.java              |   2 +-
 .../deser/TestKafkaAvroSchemaDeserializer.java     |   5 +-
 .../functional/TestHoodieDeltaStreamer.java        |  51 +-
 .../TestHoodieMultiTableDeltaStreamer.java         |   8 +-
 .../functional/TestHoodieSnapshotCopier.java       |   4 +-
 173 files changed, 4355 insertions(+), 3387 deletions(-)

diff --git a/docker/demo/sparksql-incremental.commands b/docker/demo/sparksql-incremental.commands
index 6409cc7..e5120b2 100644
--- a/docker/demo/sparksql-incremental.commands
+++ b/docker/demo/sparksql-incremental.commands
@@ -27,8 +27,8 @@ import org.apache.hadoop.fs.FileSystem;
 val fs = FileSystem.get(spark.sparkContext.hadoopConfiguration)
 val beginInstantTime = HoodieDataSourceHelpers.listCommitsSince(fs, "/user/hive/warehouse/stock_ticks_cow", "00000").get(0)
 val hoodieIncQueryDF =  spark.read.format("org.apache.hudi").
-                      option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL).
-                      option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, beginInstantTime).
+                      option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL).
+                      option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key(), beginInstantTime).
                       load("/user/hive/warehouse/stock_ticks_cow");
 hoodieIncQueryDF.registerTempTable("stock_ticks_cow_incr")
 spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close  from stock_ticks_cow_incr where  symbol = 'GOOG'").show(100, false);
@@ -37,21 +37,21 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl
     write.format("org.apache.hudi").
     option("hoodie.insert.shuffle.parallelism", "2").
     option("hoodie.upsert.shuffle.parallelism","2").
-    option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL).
-    option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL).
-    option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "key").
-    option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr").
-    option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "ts").
-    option(HoodieWriteConfig.TABLE_NAME, "stock_ticks_derived_mor").
-    option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY, "stock_ticks_derived_mor").
-    option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY, "default").
-    option(DataSourceWriteOptions.HIVE_URL_OPT_KEY, "jdbc:hive2://hiveserver:10000").
-    option(DataSourceWriteOptions.HIVE_USER_OPT_KEY, "hive").
-    option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY, "hive").
-    option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY, "true").
-    option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY, "datestr").
-    option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY, classOf[MultiPartKeysValueExtractor].getCanonicalName).
-    option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, "true").
+    option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key(), DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL).
+    option(DataSourceWriteOptions.OPERATION_OPT_KEY.key(), DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL).
+    option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key(), "key").
+    option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "datestr").
+    option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY.key(), "ts").
+    option(HoodieWriteConfig.TABLE_NAME.key(), "stock_ticks_derived_mor").
+    option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY.key(), "stock_ticks_derived_mor").
+    option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY.key(), "default").
+    option(DataSourceWriteOptions.HIVE_URL_OPT_KEY.key(), "jdbc:hive2://hiveserver:10000").
+    option(DataSourceWriteOptions.HIVE_USER_OPT_KEY.key(), "hive").
+    option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY.key(), "hive").
+    option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY.key(), "true").
+    option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY.key(), "datestr").
+    option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY.key(), classOf[MultiPartKeysValueExtractor].getCanonicalName).
+    option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key(), "true").
     mode(SaveMode.Overwrite).
     save("/user/hive/warehouse/stock_ticks_derived_mor");
 
@@ -59,8 +59,8 @@ spark.sql("select count(*) from stock_ticks_derived_mor_ro").show(20, false)
 spark.sql("select count(*) from stock_ticks_derived_mor_rt").show(20, false)
 
 val hoodieIncQueryBsDF =  spark.read.format("org.apache.hudi").
-                      option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL).
-                      option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "00000000000001").
+                      option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL).
+                      option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key(), "00000000000001").
                       load("/user/hive/warehouse/stock_ticks_cow_bs");
 hoodieIncQueryBsDF.registerTempTable("stock_ticks_cow_bs_incr")
 spark.sql("select `_hoodie_commit_time`, symbol, ts, volume, open, close  from stock_ticks_cow_bs_incr where  symbol = 'GOOG'").show(100, false);
@@ -69,21 +69,21 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl
     write.format("org.apache.hudi").
     option("hoodie.insert.shuffle.parallelism", "2").
     option("hoodie.upsert.shuffle.parallelism","2").
-    option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL).
-    option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL).
-    option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "key").
-    option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr").
-    option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, "ts").
-    option(HoodieWriteConfig.TABLE_NAME, "stock_ticks_derived_mor_bs").
-    option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY, "stock_ticks_derived_mor_bs").
-    option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY, "default").
-    option(DataSourceWriteOptions.HIVE_URL_OPT_KEY, "jdbc:hive2://hiveserver:10000").
-    option(DataSourceWriteOptions.HIVE_USER_OPT_KEY, "hive").
-    option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY, "hive").
-    option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY, "true").
-    option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY, "datestr").
-    option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY, classOf[MultiPartKeysValueExtractor].getCanonicalName).
-    option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, "true").
+    option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key(), DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL).
+    option(DataSourceWriteOptions.OPERATION_OPT_KEY.key(), DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL).
+    option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key(), "key").
+    option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "datestr").
+    option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY.key(), "ts").
+    option(HoodieWriteConfig.TABLE_NAME.key(), "stock_ticks_derived_mor_bs").
+    option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY.key(), "stock_ticks_derived_mor_bs").
+    option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY.key(), "default").
+    option(DataSourceWriteOptions.HIVE_URL_OPT_KEY.key(), "jdbc:hive2://hiveserver:10000").
+    option(DataSourceWriteOptions.HIVE_USER_OPT_KEY.key(), "hive").
+    option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY.key(), "hive").
+    option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY.key(), "true").
+    option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY.key(), "datestr").
+    option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY.key(), classOf[MultiPartKeysValueExtractor].getCanonicalName).
+    option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key(), "true").
     mode(SaveMode.Overwrite).
     save("/user/hive/warehouse/stock_ticks_derived_mor_bs");
 
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 e53dd38..13963c5 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
@@ -203,14 +203,14 @@ public class HoodieLogFileCommand implements CommandMarker {
                       .getCommitTimeline().lastInstant().get().getTimestamp())
               .withReadBlocksLazily(
                   Boolean.parseBoolean(
-                      HoodieCompactionConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED))
+                      HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP.defaultValue()))
               .withReverseReader(
                   Boolean.parseBoolean(
-                      HoodieCompactionConfig.DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED))
-              .withBufferSize(HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)
+                      HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP.defaultValue()))
+              .withBufferSize(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP.defaultValue())
               .withMaxMemorySizeInBytes(
                   HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES)
-              .withSpillableMapBasePath(HoodieMemoryConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)
+              .withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH_PROP.defaultValue())
               .build();
       for (HoodieRecord<? extends HoodieRecordPayload> hoodieRecord : scanner) {
         Option<IndexedRecord> record = hoodieRecord.getData().getInsertValue(readerSchema);
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java
index 40dddfc..433e9df 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RepairsCommand.java
@@ -153,7 +153,7 @@ public class RepairsCommand implements CommandMarker {
     HoodieTableMetaClient client = HoodieCLI.getTableMetaClient();
     Properties newProps = new Properties();
     newProps.load(new FileInputStream(new File(overwriteFilePath)));
-    Map<String, String> oldProps = client.getTableConfig().getProps();
+    Map<String, String> oldProps = client.getTableConfig().propsMap();
     Path metaPathDir = new Path(client.getBasePath(), METAFOLDER_NAME);
     HoodieTableConfig.createHoodieProperties(client.getFs(), metaPathDir, newProps);
 
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 66cc5f3..13fa91f 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
@@ -359,13 +359,13 @@ 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_PROP, sourcePath);
-    properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, keyGeneratorClass);
-    properties.setProperty(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER, fullBootstrapInputProvider);
-    properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM, parallelism);
-    properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR, selectorClass);
-    properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), recordKeyCols);
-    properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), partitionFields);
+    properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP.key(), sourcePath);
+    properties.setProperty(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS.key(), keyGeneratorClass);
+    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(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), recordKeyCols);
+    properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), partitionFields);
 
     HoodieDeltaStreamer.Config cfg = new HoodieDeltaStreamer.Config();
     cfg.targetTableName = tableName;
diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java
index d1fd694..03c7637 100644
--- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java
+++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java
@@ -136,7 +136,7 @@ public class TableCommand implements CommandMarker {
     rows.add(new Comparable[] {"basePath", client.getBasePath()});
     rows.add(new Comparable[] {"metaPath", client.getMetaPath()});
     rows.add(new Comparable[] {"fileSystem", client.getFs().getScheme()});
-    client.getTableConfig().getProps().entrySet().forEach(e -> {
+    client.getTableConfig().propsMap().entrySet().forEach(e -> {
       rows.add(new Comparable[] {e.getKey(), e.getValue()});
     });
     return HoodiePrintHelper.print(header, new HashMap<>(), "", false, -1, false, rows);
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 8aa9709..9212552 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
@@ -43,10 +43,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.DEFAULT_BLOOM_FILTER_NUM_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_FILTER_FPP.toDouble,
-      HoodieIndexConfig.DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_INDEX_FILTER_TYPE);
+    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 writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(fs.getConf).convert(schema), schema, filter)
-    val parquetConfig: HoodieAvroParquetConfig = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.DEFAULT_PARQUET_BLOCK_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_PAGE_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_FILE_MAX_BYTES.toInt, fs.getConf, HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO.toDouble)
+    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)
 
     // 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 78460b1..3160912 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
@@ -207,12 +207,12 @@ public class TestHoodieLogFileCommand extends AbstractShellIntegrationTest {
             HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES)
         .withReadBlocksLazily(
             Boolean.parseBoolean(
-                HoodieCompactionConfig.DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED))
+                HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP.defaultValue()))
         .withReverseReader(
             Boolean.parseBoolean(
-                HoodieCompactionConfig.DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED))
-        .withBufferSize(HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE)
-        .withSpillableMapBasePath(HoodieMemoryConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH)
+                HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP.defaultValue()))
+        .withBufferSize(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP.defaultValue())
+        .withSpillableMapBasePath(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH_PROP.defaultValue())
         .build();
 
     Iterator<HoodieRecord<? extends HoodieRecordPayload>> records = scanner.iterator();
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 ca8cb19..f500f30 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.DEFAULT_ARCHIVELOG_FOLDER, TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload");
+        HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue(), TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload");
   }
 
   /**
@@ -156,10 +156,10 @@ public class TestRepairsCommand extends AbstractShellIntegrationTest {
     CommandResult cr = getShell().executeCommand("repair overwrite-hoodie-props --new-props-file " + newProps.getPath());
     assertTrue(cr.isSuccess());
 
-    Map<String, String> oldProps = HoodieCLI.getTableMetaClient().getTableConfig().getProps();
+    Map<String, String> oldProps = HoodieCLI.getTableMetaClient().getTableConfig().propsMap();
 
     // after overwrite, the stored value in .hoodie is equals to which read from properties.
-    Map<String, String> result = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient()).getTableConfig().getProps();
+    Map<String, String> result = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient()).getTableConfig().propsMap();
     Properties expectProps = new Properties();
     expectProps.load(new FileInputStream(new File(newProps.getPath())));
 
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 5dd556e..5fb9191 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
@@ -20,6 +20,7 @@ package org.apache.hudi.cli.commands;
 
 import org.apache.hudi.cli.HoodieCLI;
 import org.apache.hudi.cli.testutils.AbstractShellIntegrationTest;
+import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.model.IOType;
 import org.apache.hudi.common.table.HoodieTableConfig;
@@ -37,7 +38,6 @@ import org.junit.jupiter.api.Test;
 
 import java.io.IOException;
 import java.nio.file.Paths;
-import java.util.Properties;
 
 import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
 import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS;
@@ -81,7 +81,7 @@ public class TestUpgradeDowngradeCommand extends AbstractShellIntegrationTest {
     // update hoodie.table.version to 1
     metaClient.getTableConfig().setTableVersion(HoodieTableVersion.ONE);
     try (FSDataOutputStream os = metaClient.getFs().create(new Path(metaClient.getMetaPath() + "/" + HoodieTableConfig.HOODIE_PROPERTIES_FILE), true)) {
-      metaClient.getTableConfig().getProperties().store(os, "");
+      metaClient.getTableConfig().getProps().store(os, "");
     }
     metaClient = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient());
 
@@ -109,9 +109,9 @@ public class TestUpgradeDowngradeCommand extends AbstractShellIntegrationTest {
     Path propertyFile = new Path(metaClient.getMetaPath() + "/" + HoodieTableConfig.HOODIE_PROPERTIES_FILE);
     // Load the properties and verify
     FSDataInputStream fsDataInputStream = metaClient.getFs().open(propertyFile);
-    Properties prop = new Properties();
-    prop.load(fsDataInputStream);
+    HoodieConfig hoodieConfig = HoodieConfig.create(fsDataInputStream);
     fsDataInputStream.close();
-    assertEquals(Integer.toString(HoodieTableVersion.ZERO.versionCode()), prop.getProperty(HoodieTableConfig.HOODIE_TABLE_VERSION_PROP_NAME));
+    assertEquals(Integer.toString(HoodieTableVersion.ZERO.versionCode()), hoodieConfig
+        .getString(HoodieTableConfig.HOODIE_TABLE_VERSION_PROP));
   }
 }
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 6c41e2f..e22d7dc 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
@@ -33,7 +33,6 @@ import org.apache.log4j.Logger;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.Properties;
 
 /**
  * Write commit callback http client.
@@ -47,10 +46,10 @@ public class HoodieWriteCommitHttpCallbackClient implements Closeable {
   private final String apiKey;
   private final String url;
   private final CloseableHttpClient client;
-  private Properties props;
+  private HoodieWriteConfig writeConfig;
 
   public HoodieWriteCommitHttpCallbackClient(HoodieWriteConfig config) {
-    this.props = config.getProps();
+    this.writeConfig = config;
     this.apiKey = getApiKey();
     this.url = getUrl();
     this.client = getClient();
@@ -80,11 +79,11 @@ public class HoodieWriteCommitHttpCallbackClient implements Closeable {
   }
 
   private String getApiKey() {
-    return props.getProperty(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_API_KEY);
+    return writeConfig.getString(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_API_KEY);
   }
 
   private String getUrl() {
-    return props.getProperty(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_URL_PROP);
+    return writeConfig.getString(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_URL_PROP);
   }
 
   private CloseableHttpClient getClient() {
@@ -98,7 +97,7 @@ public class HoodieWriteCommitHttpCallbackClient implements Closeable {
   }
 
   private Integer getHttpTimeoutSeconds() {
-    return Integer.parseInt(props.getProperty(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_TIMEOUT_SECONDS));
+    return writeConfig.getInt(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_TIMEOUT_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 74eb8b6..00610de 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_PROP));
+          + "empty", HoodieWriteCommitCallbackConfig.CALLBACK_CLASS_PROP.key()));
     }
   }
 
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
index b0b8d55..f1eb6f1 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java
@@ -441,19 +441,19 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
       // Do an inline compaction if enabled
       if (config.inlineCompactionEnabled()) {
         runAnyPendingCompactions(table);
-        metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true");
+        metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP.key(), "true");
         inlineCompact(extraMetadata);
       } else {
-        metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false");
+        metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP.key(), "false");
       }
 
       // Do an inline clustering if enabled
       if (config.inlineClusteringEnabled()) {
         runAnyPendingClustering(table);
-        metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "true");
+        metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP.key(), "true");
         inlineCluster(extraMetadata);
       } else {
-        metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "false");
+        metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP.key(), "false");
       }
     }
   }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java
index dab4c48..33519b9 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/LockManager.java
@@ -18,9 +18,6 @@
 
 package org.apache.hudi.client.transaction.lock;
 
-import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP;
-
 import java.io.Serializable;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
@@ -36,6 +33,9 @@ import org.apache.hudi.exception.HoodieLockException;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
 
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
+
 /**
  * This class wraps implementations of {@link LockProvider} and provides an easy way to manage the lifecycle of a lock.
  */
@@ -61,8 +61,8 @@ public class LockManager implements Serializable, AutoCloseable {
       LockProvider lockProvider = getLockProvider();
       int retryCount = 0;
       boolean acquired = false;
-      int retries = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP);
-      long waitTimeInMs = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP);
+      int retries = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY);
+      long waitTimeInMs = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY);
       while (retryCount <= retries) {
         try {
           acquired = lockProvider.tryLock(writeConfig.getLockAcquireWaitTimeoutInMs(), TimeUnit.MILLISECONDS);
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java
index 8a80685..8a1c7c0 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java
@@ -38,14 +38,14 @@ import java.util.concurrent.TimeUnit;
 
 import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_CONNECTION_TIMEOUT_MS;
 import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_SESSION_TIMEOUT_MS;
-import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP;
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP_KEY;
 
 /**
  * A zookeeper based lock. This {@link LockProvider} implementation allows to lock table operations
@@ -64,11 +64,11 @@ public class ZookeeperBasedLockProvider implements LockProvider<InterProcessMute
     checkRequiredProps(lockConfiguration);
     this.lockConfiguration = lockConfiguration;
     this.curatorFrameworkClient = CuratorFrameworkFactory.builder()
-        .connectString(lockConfiguration.getConfig().getString(ZK_CONNECT_URL_PROP))
-        .retryPolicy(new BoundedExponentialBackoffRetry(lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP),
-            lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP), lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP)))
-        .sessionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_SESSION_TIMEOUT_MS_PROP, DEFAULT_ZK_SESSION_TIMEOUT_MS))
-        .connectionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_CONNECTION_TIMEOUT_MS_PROP, DEFAULT_ZK_CONNECTION_TIMEOUT_MS))
+        .connectString(lockConfiguration.getConfig().getString(ZK_CONNECT_URL_PROP_KEY))
+        .retryPolicy(new BoundedExponentialBackoffRetry(lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY),
+            lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP_KEY), lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY)))
+        .sessionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_SESSION_TIMEOUT_MS_PROP_KEY, DEFAULT_ZK_SESSION_TIMEOUT_MS))
+        .connectionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_CONNECTION_TIMEOUT_MS_PROP_KEY, DEFAULT_ZK_CONNECTION_TIMEOUT_MS))
         .build();
     this.curatorFrameworkClient.start();
   }
@@ -136,8 +136,8 @@ public class ZookeeperBasedLockProvider implements LockProvider<InterProcessMute
   private void acquireLock(long time, TimeUnit unit) throws Exception {
     ValidationUtils.checkArgument(this.lock == null, generateLogStatement(LockState.ALREADY_ACQUIRED, generateLogSuffixString()));
     InterProcessMutex newLock = new InterProcessMutex(
-        this.curatorFrameworkClient, lockConfiguration.getConfig().getString(ZK_BASE_PATH_PROP) + "/"
-        + this.lockConfiguration.getConfig().getString(ZK_LOCK_KEY_PROP));
+        this.curatorFrameworkClient, lockConfiguration.getConfig().getString(ZK_BASE_PATH_PROP_KEY) + "/"
+        + this.lockConfiguration.getConfig().getString(ZK_LOCK_KEY_PROP_KEY));
     boolean acquired = newLock.acquire(time, unit);
     if (!acquired) {
       throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_ACQUIRE, generateLogSuffixString()));
@@ -150,16 +150,16 @@ public class ZookeeperBasedLockProvider implements LockProvider<InterProcessMute
   }
 
   private void checkRequiredProps(final LockConfiguration config) {
-    ValidationUtils.checkArgument(config.getConfig().getString(ZK_CONNECT_URL_PROP) != null);
-    ValidationUtils.checkArgument(config.getConfig().getString(ZK_BASE_PATH_PROP) != null);
-    ValidationUtils.checkArgument(config.getConfig().getString(ZK_SESSION_TIMEOUT_MS_PROP) != null);
-    ValidationUtils.checkArgument(config.getConfig().getString(ZK_CONNECTION_TIMEOUT_MS_PROP) != null);
-    ValidationUtils.checkArgument(config.getConfig().getString(ZK_LOCK_KEY_PROP) != null);
+    ValidationUtils.checkArgument(config.getConfig().getString(ZK_CONNECT_URL_PROP_KEY) != null);
+    ValidationUtils.checkArgument(config.getConfig().getString(ZK_BASE_PATH_PROP_KEY) != null);
+    ValidationUtils.checkArgument(config.getConfig().getString(ZK_SESSION_TIMEOUT_MS_PROP_KEY) != null);
+    ValidationUtils.checkArgument(config.getConfig().getString(ZK_CONNECTION_TIMEOUT_MS_PROP_KEY) != null);
+    ValidationUtils.checkArgument(config.getConfig().getString(ZK_LOCK_KEY_PROP_KEY) != null);
   }
 
   private String generateLogSuffixString() {
-    String zkBasePath = this.lockConfiguration.getConfig().getString(ZK_BASE_PATH_PROP);
-    String lockKey = this.lockConfiguration.getConfig().getString(ZK_LOCK_KEY_PROP);
+    String zkBasePath = this.lockConfiguration.getConfig().getString(ZK_BASE_PATH_PROP_KEY);
+    String lockKey = this.lockConfiguration.getConfig().getString(ZK_LOCK_KEY_PROP_KEY);
     return StringUtils.join("ZkBasePath = ", zkBasePath, ", lock key = ", lockKey);
   }
 
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 fb55442..777715f 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
@@ -22,7 +22,8 @@ import org.apache.hudi.client.bootstrap.BootstrapMode;
 import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector;
 import org.apache.hudi.client.bootstrap.translator.IdentityBootstrapPartitionPathTranslator;
 import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
-import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.common.table.HoodieTableConfig;
 
 import java.io.File;
@@ -33,33 +34,66 @@ import java.util.Properties;
 /**
  * Bootstrap specific configs.
  */
-public class HoodieBootstrapConfig extends DefaultHoodieConfig {
-
-  public static final String BOOTSTRAP_BASE_PATH_PROP = "hoodie.bootstrap.base.path";
-  public static final String BOOTSTRAP_MODE_SELECTOR = "hoodie.bootstrap.mode.selector";
-  public static final String FULL_BOOTSTRAP_INPUT_PROVIDER = "hoodie.bootstrap.full.input.provider";
-  public static final String DEFAULT_FULL_BOOTSTRAP_INPUT_PROVIDER = "org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider";
-  public static final String BOOTSTRAP_KEYGEN_CLASS = "hoodie.bootstrap.keygen.class";
-  public static final String BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS =
-      "hoodie.bootstrap.partitionpath.translator.class";
-  public static final String DEFAULT_BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS =
-      IdentityBootstrapPartitionPathTranslator.class.getName();
-
-  public static final String BOOTSTRAP_PARALLELISM = "hoodie.bootstrap.parallelism";
-  public static final String DEFAULT_BOOTSTRAP_PARALLELISM = "1500";
-
-  // Used By BootstrapRegexModeSelector class. When a partition path matches the regex, the corresponding
-  // mode will be used. Otherwise, the alternative mode will be used.
-  public static final String BOOTSTRAP_MODE_SELECTOR_REGEX = "hoodie.bootstrap.mode.selector.regex";
-  public static final String BOOTSTRAP_MODE_SELECTOR_REGEX_MODE = "hoodie.bootstrap.mode.selector.regex.mode";
-  public static final String DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX = ".*";
-  public static final String DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX_MODE = BootstrapMode.METADATA_ONLY.name();
-
-  public static final String BOOTSTRAP_INDEX_CLASS_PROP = "hoodie.bootstrap.index.class";
-  public static final String DEFAULT_BOOTSTRAP_INDEX_CLASS = HFileBootstrapIndex.class.getName();
-
-  public HoodieBootstrapConfig(Properties props) {
-    super(props);
+public class HoodieBootstrapConfig extends HoodieConfig {
+
+  public static final ConfigProperty<String> BOOTSTRAP_BASE_PATH_PROP = 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");
+
+  public static final ConfigProperty<String> BOOTSTRAP_MODE_SELECTOR = 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
+      .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
+      .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_PARTITION_PATH_TRANSLATOR_CLASS = 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
+      .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
+      .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
+      .key("hoodie.bootstrap.mode.selector.regex.mode")
+      .defaultValue(BootstrapMode.METADATA_ONLY.name())
+      .sinceVersion("0.6.0")
+      .withDocumentation("Bootstrap mode to apply for partition paths, that match regex above. "
+          + "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_PROP = ConfigProperty
+      .key("hoodie.bootstrap.index.class")
+      .defaultValue(HFileBootstrapIndex.class.getName())
+      .sinceVersion("0.6.0")
+      .withDocumentation("");
+
+  private HoodieBootstrapConfig() {
+    super();
   }
 
   public static Builder newBuilder() {
@@ -68,78 +102,67 @@ public class HoodieBootstrapConfig extends DefaultHoodieConfig {
 
   public static class Builder {
 
-    private final Properties props = new Properties();
+    private final HoodieBootstrapConfig bootstrapConfig = new HoodieBootstrapConfig();
 
     public Builder fromFile(File propertiesFile) throws IOException {
       try (FileReader reader = new FileReader(propertiesFile)) {
-        this.props.load(reader);
+        this.bootstrapConfig.getProps().load(reader);
         return this;
       }
     }
 
     public Builder withBootstrapBasePath(String basePath) {
-      props.setProperty(BOOTSTRAP_BASE_PATH_PROP, basePath);
+      bootstrapConfig.setValue(BOOTSTRAP_BASE_PATH_PROP, basePath);
       return this;
     }
 
     public Builder withBootstrapModeSelector(String partitionSelectorClass) {
-      props.setProperty(BOOTSTRAP_MODE_SELECTOR, partitionSelectorClass);
+      bootstrapConfig.setValue(BOOTSTRAP_MODE_SELECTOR, partitionSelectorClass);
       return this;
     }
 
     public Builder withFullBootstrapInputProvider(String partitionSelectorClass) {
-      props.setProperty(FULL_BOOTSTRAP_INPUT_PROVIDER, partitionSelectorClass);
+      bootstrapConfig.setValue(FULL_BOOTSTRAP_INPUT_PROVIDER, partitionSelectorClass);
       return this;
     }
 
     public Builder withBootstrapKeyGenClass(String keyGenClass) {
-      props.setProperty(BOOTSTRAP_KEYGEN_CLASS, keyGenClass);
+      bootstrapConfig.setValue(BOOTSTRAP_KEYGEN_CLASS, keyGenClass);
       return this;
     }
 
     public Builder withBootstrapPartitionPathTranslatorClass(String partitionPathTranslatorClass) {
-      props.setProperty(BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS, partitionPathTranslatorClass);
+      bootstrapConfig
+          .setValue(BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS, partitionPathTranslatorClass);
       return this;
     }
 
     public Builder withBootstrapParallelism(int parallelism) {
-      props.setProperty(BOOTSTRAP_PARALLELISM, String.valueOf(parallelism));
+      bootstrapConfig.setValue(BOOTSTRAP_PARALLELISM, String.valueOf(parallelism));
       return this;
     }
 
     public Builder withBootstrapModeSelectorRegex(String regex) {
-      props.setProperty(BOOTSTRAP_MODE_SELECTOR_REGEX, regex);
+      bootstrapConfig.setValue(BOOTSTRAP_MODE_SELECTOR_REGEX, regex);
       return this;
     }
 
     public Builder withBootstrapModeForRegexMatch(BootstrapMode modeForRegexMatch) {
-      props.setProperty(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE, modeForRegexMatch.name());
+      bootstrapConfig.setValue(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE, modeForRegexMatch.name());
       return this;
     }
 
     public Builder fromProperties(Properties props) {
-      this.props.putAll(props);
+      this.bootstrapConfig.getProps().putAll(props);
       return this;
     }
 
     public HoodieBootstrapConfig build() {
-      HoodieBootstrapConfig config = new HoodieBootstrapConfig(props);
-      setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_PARALLELISM), BOOTSTRAP_PARALLELISM,
-          DEFAULT_BOOTSTRAP_PARALLELISM);
-      setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS),
-          BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS, DEFAULT_BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS);
-      setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_MODE_SELECTOR), BOOTSTRAP_MODE_SELECTOR,
-          MetadataOnlyBootstrapModeSelector.class.getCanonicalName());
-      setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_MODE_SELECTOR_REGEX), BOOTSTRAP_MODE_SELECTOR_REGEX,
-          DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX);
-      setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE),
-          BOOTSTRAP_MODE_SELECTOR_REGEX_MODE, DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX_MODE);
-      BootstrapMode.valueOf(props.getProperty(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE));
-      setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_INDEX_CLASS_PROP), BOOTSTRAP_INDEX_CLASS_PROP,
-          HoodieTableConfig.getDefaultBootstrapIndexClass(props));
-      setDefaultOnCondition(props, !props.containsKey(FULL_BOOTSTRAP_INPUT_PROVIDER), FULL_BOOTSTRAP_INPUT_PROVIDER,
-          DEFAULT_FULL_BOOTSTRAP_INPUT_PROVIDER);
-      return config;
+      // TODO: use infer function instead
+      bootstrapConfig.setDefaultValue(BOOTSTRAP_INDEX_CLASS_PROP, 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 91acd30..86d26d3 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
@@ -18,7 +18,8 @@
 
 package org.apache.hudi.config;
 
-import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 
 import java.io.File;
 import java.io.FileReader;
@@ -28,64 +29,87 @@ import java.util.Properties;
 /**
  * Clustering specific configs.
  */
-public class HoodieClusteringConfig extends DefaultHoodieConfig {
-
-  // Config to provide a strategy class to create ClusteringPlan. Class has to be subclass of ClusteringPlanStrategy
-  public static final String CLUSTERING_PLAN_STRATEGY_CLASS = "hoodie.clustering.plan.strategy.class";
-  public static final String DEFAULT_CLUSTERING_PLAN_STRATEGY_CLASS =
-      "org.apache.hudi.client.clustering.plan.strategy.SparkRecentDaysClusteringPlanStrategy";
-
-  // Config to provide a strategy class to execute a ClusteringPlan. Class has to be subclass of RunClusteringStrategy
-  public static final String CLUSTERING_EXECUTION_STRATEGY_CLASS = "hoodie.clustering.execution.strategy.class";
-  public static final String DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS =
-      "org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy";
-
-  // Turn on inline clustering - clustering will be run after write operation is complete.
-  public static final String INLINE_CLUSTERING_PROP = "hoodie.clustering.inline";
-  private static final String DEFAULT_INLINE_CLUSTERING = "false";
-
-  // Config to control frequency of clustering
-  public static final String INLINE_CLUSTERING_MAX_COMMIT_PROP = "hoodie.clustering.inline.max.commits";
-  private static final String DEFAULT_INLINE_CLUSTERING_NUM_COMMITS = "4";
-  
+public class HoodieClusteringConfig extends HoodieConfig {
+
+  public static final ConfigProperty<String> CLUSTERING_PLAN_STRATEGY_CLASS = 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 to create ClusteringPlan. Class has to be subclass of ClusteringPlanStrategy");
+
+  public static final ConfigProperty<String> CLUSTERING_EXECUTION_STRATEGY_CLASS = ConfigProperty
+      .key("hoodie.clustering.execution.strategy.class")
+      .defaultValue("org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy")
+      .sinceVersion("0.7.0")
+      .withDocumentation("Config to provide a strategy class to execute a ClusteringPlan. Class has to be subclass of RunClusteringStrategy");
+
+  public static final ConfigProperty<String> INLINE_CLUSTERING_PROP = ConfigProperty
+      .key("hoodie.clustering.inline")
+      .defaultValue("false")
+      .sinceVersion("0.7.0")
+      .withDocumentation("Turn on inline clustering - clustering will be run after write operation is complete");
+
+  public static final ConfigProperty<String> INLINE_CLUSTERING_MAX_COMMIT_PROP = ConfigProperty
+      .key("hoodie.clustering.inline.max.commits")
+      .defaultValue("4")
+      .sinceVersion("0.7.0")
+      .withDocumentation("Config to control frequency of clustering");
+
   // Any strategy specific params can be saved with this prefix
   public static final String CLUSTERING_STRATEGY_PARAM_PREFIX = "hoodie.clustering.plan.strategy.";
 
-  // Number of partitions to list to create ClusteringPlan.
-  public static final String CLUSTERING_TARGET_PARTITIONS = CLUSTERING_STRATEGY_PARAM_PREFIX + "daybased.lookback.partitions";
-  public static final String DEFAULT_CLUSTERING_TARGET_PARTITIONS = String.valueOf(2);
-
-  // Files smaller than the size specified here are candidates for clustering.
-  public static final String CLUSTERING_PLAN_SMALL_FILE_LIMIT = CLUSTERING_STRATEGY_PARAM_PREFIX + "small.file.limit";
-  public static final String DEFAULT_CLUSTERING_PLAN_SMALL_FILE_LIMIT = String.valueOf(600 * 1024 * 1024L); // 600MB
-
-  // Each clustering operation can create multiple groups. Total amount of data processed by clustering operation
-  // 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 String CLUSTERING_MAX_BYTES_PER_GROUP = CLUSTERING_STRATEGY_PARAM_PREFIX + "max.bytes.per.group";
-  public static final String DEFAULT_CLUSTERING_MAX_GROUP_SIZE = String.valueOf(2 * 1024 * 1024 * 1024L);
-
-  // Maximum number of groups to create as part of ClusteringPlan. Increasing groups will increase parallelism.
-  public static final String CLUSTERING_MAX_NUM_GROUPS = CLUSTERING_STRATEGY_PARAM_PREFIX + "max.num.groups";
-  public static final String DEFAULT_CLUSTERING_MAX_NUM_GROUPS = "30";
-
-  // Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups.
-  public static final String CLUSTERING_TARGET_FILE_MAX_BYTES = CLUSTERING_STRATEGY_PARAM_PREFIX + "target.file.max.bytes";
-  public static final String DEFAULT_CLUSTERING_TARGET_FILE_MAX_BYTES = String.valueOf(1 * 1024 * 1024 * 1024L); // 1GB
-  
-  // Constants related to clustering that may be used by more than 1 strategy.
-  public static final String CLUSTERING_SORT_COLUMNS_PROPERTY = HoodieClusteringConfig.CLUSTERING_STRATEGY_PARAM_PREFIX + "sort.columns";
-
-  // When file groups is in clustering, need to handle the update to these file groups. Default strategy just reject the update
-  public static final String CLUSTERING_UPDATES_STRATEGY_PROP = "hoodie.clustering.updates.strategy";
-  public static final String DEFAULT_CLUSTERING_UPDATES_STRATEGY = "org.apache.hudi.client.clustering.update.strategy.SparkRejectUpdateStrategy";
-
-  // Async clustering
-  public static final String ASYNC_CLUSTERING_ENABLE_OPT_KEY = "hoodie.clustering.async.enabled";
-  public static final String DEFAULT_ASYNC_CLUSTERING_ENABLE_OPT_VAL = "false";
-
-  public HoodieClusteringConfig(Properties props) {
-    super(props);
+  public static final ConfigProperty<String> CLUSTERING_TARGET_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_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
+      .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "max.bytes.per.group")
+      .defaultValue(String.valueOf(2 * 1024 * 1024 * 1024L))
+      .sinceVersion("0.7.0")
+      .withDocumentation("Each clustering operation can create multiple groups. Total amount of data processed by clustering operation"
+          + " 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
+      .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
+      .key(CLUSTERING_STRATEGY_PARAM_PREFIX + "target.file.max.bytes")
+      .defaultValue(String.valueOf(1 * 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
+      .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_PROP = ConfigProperty
+      .key("hoodie.clustering.updates.strategy")
+      .defaultValue("org.apache.hudi.client.clustering.update.strategy.SparkRejectUpdateStrategy")
+      .sinceVersion("0.7.0")
+      .withDocumentation("When file groups is in clustering, need to handle the update to these file groups. Default strategy just reject the update");
+
+  public static final ConfigProperty<String> ASYNC_CLUSTERING_ENABLE_OPT_KEY = ConfigProperty
+      .key("hoodie.clustering.async.enabled")
+      .defaultValue("false")
+      .sinceVersion("0.7.0")
+      .withDocumentation("Async clustering");
+
+  private HoodieClusteringConfig() {
+    super();
   }
 
   public static Builder newBuilder() {
@@ -94,106 +118,83 @@ public class HoodieClusteringConfig extends DefaultHoodieConfig {
 
   public static class Builder {
 
-    private final Properties props = new Properties();
+    private final HoodieClusteringConfig clusteringConfig = new HoodieClusteringConfig();
 
     public Builder fromFile(File propertiesFile) throws IOException {
       try (FileReader reader = new FileReader(propertiesFile)) {
-        this.props.load(reader);
+        this.clusteringConfig.getProps().load(reader);
         return this;
       }
     }
 
     public Builder withClusteringPlanStrategyClass(String clusteringStrategyClass) {
-      props.setProperty(CLUSTERING_PLAN_STRATEGY_CLASS, clusteringStrategyClass);
+      clusteringConfig.setValue(CLUSTERING_PLAN_STRATEGY_CLASS, clusteringStrategyClass);
       return this;
     }
 
     public Builder withClusteringExecutionStrategyClass(String runClusteringStrategyClass) {
-      props.setProperty(CLUSTERING_EXECUTION_STRATEGY_CLASS, runClusteringStrategyClass);
+      clusteringConfig.setValue(CLUSTERING_EXECUTION_STRATEGY_CLASS, runClusteringStrategyClass);
       return this;
     }
 
     public Builder withClusteringTargetPartitions(int clusteringTargetPartitions) {
-      props.setProperty(CLUSTERING_TARGET_PARTITIONS, String.valueOf(clusteringTargetPartitions));
+      clusteringConfig.setValue(CLUSTERING_TARGET_PARTITIONS, String.valueOf(clusteringTargetPartitions));
       return this;
     }
 
     public Builder withClusteringPlanSmallFileLimit(long clusteringSmallFileLimit) {
-      props.setProperty(CLUSTERING_PLAN_SMALL_FILE_LIMIT, String.valueOf(clusteringSmallFileLimit));
+      clusteringConfig.setValue(CLUSTERING_PLAN_SMALL_FILE_LIMIT, String.valueOf(clusteringSmallFileLimit));
       return this;
     }
     
     public Builder withClusteringSortColumns(String sortColumns) {
-      props.setProperty(CLUSTERING_SORT_COLUMNS_PROPERTY, sortColumns);
+      clusteringConfig.setValue(CLUSTERING_SORT_COLUMNS_PROPERTY, sortColumns);
       return this;
     }
 
     public Builder withClusteringMaxBytesInGroup(long clusteringMaxGroupSize) {
-      props.setProperty(CLUSTERING_MAX_BYTES_PER_GROUP, String.valueOf(clusteringMaxGroupSize));
+      clusteringConfig.setValue(CLUSTERING_MAX_BYTES_PER_GROUP, String.valueOf(clusteringMaxGroupSize));
       return this;
     }
 
     public Builder withClusteringMaxNumGroups(int maxNumGroups) {
-      props.setProperty(CLUSTERING_MAX_NUM_GROUPS, String.valueOf(maxNumGroups));
+      clusteringConfig.setValue(CLUSTERING_MAX_NUM_GROUPS, String.valueOf(maxNumGroups));
       return this;
     }
 
     public Builder withClusteringTargetFileMaxBytes(long targetFileSize) {
-      props.setProperty(CLUSTERING_TARGET_FILE_MAX_BYTES, String.valueOf(targetFileSize));
+      clusteringConfig.setValue(CLUSTERING_TARGET_FILE_MAX_BYTES, String.valueOf(targetFileSize));
       return this;
     }
 
     public Builder withInlineClustering(Boolean inlineClustering) {
-      props.setProperty(INLINE_CLUSTERING_PROP, String.valueOf(inlineClustering));
+      clusteringConfig.setValue(INLINE_CLUSTERING_PROP, String.valueOf(inlineClustering));
       return this;
     }
 
     public Builder withInlineClusteringNumCommits(int numCommits) {
-      props.setProperty(INLINE_CLUSTERING_MAX_COMMIT_PROP, String.valueOf(numCommits));
+      clusteringConfig.setValue(INLINE_CLUSTERING_MAX_COMMIT_PROP, String.valueOf(numCommits));
       return this;
     }
 
     public Builder fromProperties(Properties props) {
-      this.props.putAll(props);
+      this.clusteringConfig.getProps().putAll(props);
       return this;
     }
 
     public Builder withClusteringUpdatesStrategy(String updatesStrategyClass) {
-      props.setProperty(CLUSTERING_UPDATES_STRATEGY_PROP, updatesStrategyClass);
+      clusteringConfig.setValue(CLUSTERING_UPDATES_STRATEGY_PROP, updatesStrategyClass);
       return this;
     }
 
     public Builder withAsyncClustering(Boolean asyncClustering) {
-      props.setProperty(ASYNC_CLUSTERING_ENABLE_OPT_KEY, String.valueOf(asyncClustering));
+      clusteringConfig.setValue(ASYNC_CLUSTERING_ENABLE_OPT_KEY, String.valueOf(asyncClustering));
       return this;
     }
 
     public HoodieClusteringConfig build() {
-      HoodieClusteringConfig config = new HoodieClusteringConfig(props);
-
-      setDefaultOnCondition(props, !props.containsKey(CLUSTERING_PLAN_STRATEGY_CLASS),
-          CLUSTERING_PLAN_STRATEGY_CLASS, DEFAULT_CLUSTERING_PLAN_STRATEGY_CLASS);
-      setDefaultOnCondition(props, !props.containsKey(CLUSTERING_EXECUTION_STRATEGY_CLASS),
-          CLUSTERING_EXECUTION_STRATEGY_CLASS, DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS);
-      setDefaultOnCondition(props, !props.containsKey(CLUSTERING_MAX_BYTES_PER_GROUP), CLUSTERING_MAX_BYTES_PER_GROUP,
-          DEFAULT_CLUSTERING_MAX_GROUP_SIZE);
-      setDefaultOnCondition(props, !props.containsKey(CLUSTERING_MAX_NUM_GROUPS), CLUSTERING_MAX_NUM_GROUPS,
-          DEFAULT_CLUSTERING_MAX_NUM_GROUPS);
-      setDefaultOnCondition(props, !props.containsKey(CLUSTERING_TARGET_FILE_MAX_BYTES), CLUSTERING_TARGET_FILE_MAX_BYTES,
-          DEFAULT_CLUSTERING_TARGET_FILE_MAX_BYTES);
-      setDefaultOnCondition(props, !props.containsKey(INLINE_CLUSTERING_PROP), INLINE_CLUSTERING_PROP,
-          DEFAULT_INLINE_CLUSTERING);
-      setDefaultOnCondition(props, !props.containsKey(INLINE_CLUSTERING_MAX_COMMIT_PROP), INLINE_CLUSTERING_MAX_COMMIT_PROP,
-          DEFAULT_INLINE_CLUSTERING_NUM_COMMITS);
-      setDefaultOnCondition(props, !props.containsKey(CLUSTERING_TARGET_PARTITIONS), CLUSTERING_TARGET_PARTITIONS,
-          DEFAULT_CLUSTERING_TARGET_PARTITIONS);
-      setDefaultOnCondition(props, !props.containsKey(CLUSTERING_PLAN_SMALL_FILE_LIMIT), CLUSTERING_PLAN_SMALL_FILE_LIMIT,
-          DEFAULT_CLUSTERING_PLAN_SMALL_FILE_LIMIT);
-      setDefaultOnCondition(props, !props.containsKey(CLUSTERING_UPDATES_STRATEGY_PROP), CLUSTERING_UPDATES_STRATEGY_PROP, 
-          DEFAULT_CLUSTERING_UPDATES_STRATEGY);
-      setDefaultOnCondition(props, !props.containsKey(ASYNC_CLUSTERING_ENABLE_OPT_KEY), ASYNC_CLUSTERING_ENABLE_OPT_KEY,
-          DEFAULT_ASYNC_CLUSTERING_ENABLE_OPT_VAL);
-      return config;
+      clusteringConfig.setDefaults(HoodieClusteringConfig.class.getName());
+      return clusteringConfig;
     }
   }
 }
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 176b17c..aa9e75c 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
@@ -18,7 +18,8 @@
 
 package org.apache.hudi.config;
 
-import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.common.model.HoodieCleaningPolicy;
 import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
 import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
@@ -38,99 +39,176 @@ import java.util.Properties;
  * Compaction related config.
  */
 @Immutable
-public class HoodieCompactionConfig extends DefaultHoodieConfig {
-
-  public static final String CLEANER_POLICY_PROP = "hoodie.cleaner.policy";
-  public static final String AUTO_CLEAN_PROP = "hoodie.clean.automatic";
-  public static final String ASYNC_CLEAN_PROP = "hoodie.clean.async";
-  // Turn on inline compaction - after fw delta commits a inline compaction will be run
-  public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
-  // Run a compaction every N delta commits
-  public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max.delta.commits";
-  // Run a compaction when time elapsed > N seconds since last compaction
-  public static final String INLINE_COMPACT_TIME_DELTA_SECONDS_PROP = "hoodie.compact.inline.max.delta.seconds";
-  public static final String INLINE_COMPACT_TRIGGER_STRATEGY_PROP = "hoodie.compact.inline.trigger.strategy";
-  public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP = "hoodie.cleaner.fileversions.retained";
-  public static final String CLEANER_COMMITS_RETAINED_PROP = "hoodie.cleaner.commits.retained";
-  public static final String CLEANER_INCREMENTAL_MODE = "hoodie.cleaner.incremental.mode";
-  public static final String MAX_COMMITS_TO_KEEP_PROP = "hoodie.keep.max.commits";
-  public static final String MIN_COMMITS_TO_KEEP_PROP = "hoodie.keep.min.commits";
-  public static final String COMMITS_ARCHIVAL_BATCH_SIZE_PROP = "hoodie.commits.archival.batch";
-  // Set true to clean bootstrap source files when necessary
-  public static final String CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = "hoodie.cleaner.delete.bootstrap.base.file";
-  // Upsert uses this file size to compact new data onto existing files..
-  public static final String PARQUET_SMALL_FILE_LIMIT_BYTES = "hoodie.parquet.small.file.limit";
-  // By default, treat any file <= 100MB as a small file.
-  public static final String DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES = String.valueOf(104857600);
-  // Hudi will use the previous commit to calculate the estimated record size by totalBytesWritten/totalRecordsWritten.
-  // If the previous commit is too small to make an accurate estimation, Hudi will search commits in the reverse order,
-  // until find a commit has totalBytesWritten larger than (PARQUET_SMALL_FILE_LIMIT_BYTES * RECORD_SIZE_ESTIMATION_THRESHOLD)
-  public static final String RECORD_SIZE_ESTIMATION_THRESHOLD_PROP = "hoodie.record.size.estimation.threshold";
-  public static final String DEFAULT_RECORD_SIZE_ESTIMATION_THRESHOLD = "1.0";
+public class HoodieCompactionConfig extends HoodieConfig {
+
+  public static final ConfigProperty<String> CLEANER_POLICY_PROP = ConfigProperty
+      .key("hoodie.cleaner.policy")
+      .defaultValue(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name())
+      .withDocumentation("Cleaning policy to be used. Hudi will delete older versions of parquet files to re-claim space."
+          + " Any Query/Computation referring to this version of the file will fail. "
+          + "It is good to make sure that the data is retained for more than the maximum query execution time.");
+
+  public static final ConfigProperty<String> AUTO_CLEAN_PROP = ConfigProperty
+      .key("hoodie.clean.automatic")
+      .defaultValue("true")
+      .withDocumentation("Should cleanup if there is anything to cleanup immediately after the commit");
+
+  public static final ConfigProperty<String> ASYNC_CLEAN_PROP = ConfigProperty
+      .key("hoodie.clean.async")
+      .defaultValue("false")
+      .withDocumentation("Only applies when #withAutoClean is turned on. When turned on runs cleaner async with writing.");
+
+  public static final ConfigProperty<String> INLINE_COMPACT_PROP = ConfigProperty
+      .key("hoodie.compact.inline")
+      .defaultValue("false")
+      .withDocumentation("When set to true, compaction is triggered by the ingestion itself, "
+          + "right after a commit/deltacommit action as part of insert/upsert/bulk_insert");
+
+  public static final ConfigProperty<String> INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = ConfigProperty
+      .key("hoodie.compact.inline.max.delta.commits")
+      .defaultValue("5")
+      .withDocumentation("Number of max delta commits to keep before triggering an inline compaction");
+
+  public static final ConfigProperty<String> INLINE_COMPACT_TIME_DELTA_SECONDS_PROP = ConfigProperty
+      .key("hoodie.compact.inline.max.delta.seconds")
+      .defaultValue(String.valueOf(60 * 60))
+      .withDocumentation("Run a compaction when time elapsed > N seconds since last compaction");
+
+  public static final ConfigProperty<String> INLINE_COMPACT_TRIGGER_STRATEGY_PROP = ConfigProperty
+      .key("hoodie.compact.inline.trigger.strategy")
+      .defaultValue(CompactionTriggerStrategy.NUM_COMMITS.name())
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> CLEANER_FILE_VERSIONS_RETAINED_PROP = ConfigProperty
+      .key("hoodie.cleaner.fileversions.retained")
+      .defaultValue("3")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> CLEANER_COMMITS_RETAINED_PROP = ConfigProperty
+      .key("hoodie.cleaner.commits.retained")
+      .defaultValue("10")
+      .withDocumentation("Number of commits to retain. So data will be retained for num_of_commits * time_between_commits "
+          + "(scheduled). This also directly translates into how much you can incrementally pull on this table");
+
+  public static final ConfigProperty<String> CLEANER_INCREMENTAL_MODE = ConfigProperty
+      .key("hoodie.cleaner.incremental.mode")
+      .defaultValue("true")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> MAX_COMMITS_TO_KEEP_PROP = ConfigProperty
+      .key("hoodie.keep.max.commits")
+      .defaultValue("30")
+      .withDocumentation("Each commit is a small file in the .hoodie directory. Since DFS typically does not favor lots of "
+          + "small files, Hudi archives older commits into a sequential log. A commit is published atomically "
+          + "by a rename of the commit file.");
+
+  public static final ConfigProperty<String> MIN_COMMITS_TO_KEEP_PROP = ConfigProperty
+      .key("hoodie.keep.min.commits")
+      .defaultValue("20")
+      .withDocumentation("Each commit is a small file in the .hoodie directory. Since DFS typically does not favor lots of "
+          + "small files, Hudi archives older commits into a sequential log. A commit is published atomically "
+          + "by a rename of the commit file.");
+
+  public static final ConfigProperty<String> COMMITS_ARCHIVAL_BATCH_SIZE_PROP = ConfigProperty
+      .key("hoodie.commits.archival.batch")
+      .defaultValue(String.valueOf(10))
+      .withDocumentation("This controls the number of commit instants read in memory as a batch and archived together.");
+
+  public static final ConfigProperty<String> CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = ConfigProperty
+      .key("hoodie.cleaner.delete.bootstrap.base.file")
+      .defaultValue("false")
+      .withDocumentation("Set true to clean bootstrap source files when necessary");
+
+  public static final ConfigProperty<String> PARQUET_SMALL_FILE_LIMIT_BYTES = ConfigProperty
+      .key("hoodie.parquet.small.file.limit")
+      .defaultValue(String.valueOf(104857600))
+      .withDocumentation("Upsert uses this file size to compact new data onto existing files. "
+          + "By default, treat any file <= 100MB as a small file.");
+
+  public static final ConfigProperty<String> RECORD_SIZE_ESTIMATION_THRESHOLD_PROP = ConfigProperty
+      .key("hoodie.record.size.estimation.threshold")
+      .defaultValue("1.0")
+      .withDocumentation("Hudi will use the previous commit to calculate the estimated record size by totalBytesWritten/totalRecordsWritten. "
+          + "If the previous commit is too small to make an accurate estimation, Hudi will search commits in the reverse order, "
+          + "until find a commit has totalBytesWritten larger than (PARQUET_SMALL_FILE_LIMIT_BYTES * RECORD_SIZE_ESTIMATION_THRESHOLD)");
+
+  public static final ConfigProperty<String> CLEANER_PARALLELISM = ConfigProperty
+      .key("hoodie.cleaner.parallelism")
+      .defaultValue("200")
+      .withDocumentation("Increase this if cleaning becomes slow.");
+
+  // 500GB of target IO per compaction (both read and write
+  public static final ConfigProperty<String> TARGET_IO_PER_COMPACTION_IN_MB_PROP = ConfigProperty
+      .key("hoodie.compaction.target.io")
+      .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.");
+
+  public static final ConfigProperty<String> COMPACTION_STRATEGY_PROP = ConfigProperty
+      .key("hoodie.compaction.strategy")
+      .defaultValue(LogFileSizeBasedCompactionStrategy.class.getName())
+      .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");
+
+  public static final ConfigProperty<String> PAYLOAD_CLASS_PROP = 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.");
+
+  public static final ConfigProperty<String> COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = ConfigProperty
+      .key("hoodie.compaction.lazy.block.read")
+      .defaultValue("false")
+      .withDocumentation("When a CompactedLogScanner merges all log files, this config helps to choose whether the logblocks "
+          + "should be read lazily or not. Choose true to use I/O intensive lazy block reading (low memory usage) or false "
+          + "for Memory intensive immediate block read (high memory usage)");
+
+  public static final ConfigProperty<String> COMPACTION_REVERSE_LOG_READ_ENABLED_PROP = 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");
+
+  public static final ConfigProperty<String> FAILED_WRITES_CLEANER_POLICY_PROP = ConfigProperty
+      .key("hoodie.cleaner.policy.failed.writes")
+      .defaultValue(HoodieFailedWritesCleaningPolicy.EAGER.name())
+      .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)");
+
+  public static final ConfigProperty<String> TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP = 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.");
 
   /**
    * Configs related to specific table types.
    */
-  // Number of inserts, that will be put each partition/bucket for writing
-  public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = "hoodie.copyonwrite.insert.split.size";
-  // The rationale to pick the insert parallelism is the following. Writing out 100MB files,
-  // with atleast 1kb records, means 100K records per file. we just overprovision to 500K
-  public static final String DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = String.valueOf(500000);
-  // Config to control whether we control insert split sizes automatically based on average
-  // record sizes
-  public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = "hoodie.copyonwrite.insert.auto.split";
-  // its off by default
-  public static final String DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = String.valueOf(true);
-  // This value is used as a guesstimate for the record size, if we can't determine this from
-  // previous commits
-  public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = "hoodie.copyonwrite.record.size.estimate";
-  // Used to determine how much more can be packed into a small file, before it exceeds the size
-  // limit.
-  public static final String DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = String.valueOf(1024);
-  public static final String CLEANER_PARALLELISM = "hoodie.cleaner.parallelism";
-  public static final String DEFAULT_CLEANER_PARALLELISM = String.valueOf(200);
-  public static final String TARGET_IO_PER_COMPACTION_IN_MB_PROP = "hoodie.compaction.target.io";
-  // 500GB of target IO per compaction (both read and write)
-  public static final String DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB = String.valueOf(500 * 1024);
-  public static final String COMPACTION_STRATEGY_PROP = "hoodie.compaction.strategy";
-  // 200GB of target IO per compaction
-  public static final String DEFAULT_COMPACTION_STRATEGY = LogFileSizeBasedCompactionStrategy.class.getName();
-  // used to merge records written to log file
-  public static final String DEFAULT_PAYLOAD_CLASS = OverwriteWithLatestAvroPayload.class.getName();
-  public static final String PAYLOAD_CLASS_PROP = "hoodie.compaction.payload.class";
-
-  // used to choose a trade off between IO vs Memory when performing compaction process
-  // Depending on outputfile_size and memory provided, choose true to avoid OOM for large file
-  // size + small memory
-  public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = "hoodie.compaction.lazy.block.read";
-  public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "false";
-  // used to choose whether to enable reverse log reading (reverse log traversal)
-  public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP = "hoodie.compaction.reverse.log.read";
-  public static final String DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED = "false";
-  private static final String DEFAULT_CLEANER_POLICY = HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name();
-  public static final String FAILED_WRITES_CLEANER_POLICY_PROP = "hoodie.cleaner.policy.failed.writes";
-  private  static final String DEFAULT_FAILED_WRITES_CLEANER_POLICY =
-      HoodieFailedWritesCleaningPolicy.EAGER.name();
-  private static final String DEFAULT_AUTO_CLEAN = "true";
-  private static final String DEFAULT_ASYNC_CLEAN = "false";
-  private static final String DEFAULT_INLINE_COMPACT = "false";
-  private static final String DEFAULT_INCREMENTAL_CLEANER = "true";
-  private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "5";
-  private static final String DEFAULT_INLINE_COMPACT_TIME_DELTA_SECONDS = String.valueOf(60 * 60);
-  private static final String DEFAULT_INLINE_COMPACT_TRIGGER_STRATEGY = CompactionTriggerStrategy.NUM_COMMITS.name();
-  private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = "3";
-  private static final String DEFAULT_CLEANER_COMMITS_RETAINED = "10";
-  private static final String DEFAULT_MAX_COMMITS_TO_KEEP = "30";
-  private static final String DEFAULT_MIN_COMMITS_TO_KEEP = "20";
-  private static final String DEFAULT_COMMITS_ARCHIVAL_BATCH_SIZE = String.valueOf(10);
-  private static final String DEFAULT_CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = "false";
-  public static final String TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP =
-      "hoodie.compaction.daybased.target.partitions";
-  // 500GB of target IO per compaction (both read and write)
-  public static final String DEFAULT_TARGET_PARTITIONS_PER_DAYBASED_COMPACTION = String.valueOf(10);
-
-  private HoodieCompactionConfig(Properties props) {
-    super(props);
+  public static final ConfigProperty<String> COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = ConfigProperty
+      .key("hoodie.copyonwrite.insert.split.size")
+      .defaultValue(String.valueOf(500000))
+      .withDocumentation("Number of inserts, that will be put each partition/bucket for writing. "
+          + "The rationale to pick the insert parallelism is the following. Writing out 100MB files, "
+          + "with at least 1kb records, means 100K records per file. we just over provision to 500K.");
+
+  public static final ConfigProperty<String> COPY_ON_WRITE_TABLE_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.");
+
+  public static final ConfigProperty<String> COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = ConfigProperty
+      .key("hoodie.copyonwrite.record.size.estimate")
+      .defaultValue(String.valueOf(1024))
+      .withDocumentation("The average record size. If specified, hudi will use this and not compute dynamically "
+          + "based on the last 24 commit’s metadata. No value set as default. This is critical in computing "
+          + "the insert parallelism and bin-packing inserts into small files. See above.");
+
+  private HoodieCompactionConfig() {
+    super();
   }
 
   public static HoodieCompactionConfig.Builder newBuilder() {
@@ -139,226 +217,174 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
 
   public static class Builder {
 
-    private final Properties props = new Properties();
+    private final HoodieCompactionConfig compactionConfig = new HoodieCompactionConfig();
 
     public Builder fromFile(File propertiesFile) throws IOException {
       try (FileReader reader = new FileReader(propertiesFile)) {
-        this.props.load(reader);
+        this.compactionConfig.getProps().load(reader);
         return this;
       }
     }
 
     public Builder fromProperties(Properties props) {
-      this.props.putAll(props);
+      this.compactionConfig.getProps().putAll(props);
       return this;
     }
 
     public Builder withAutoClean(Boolean autoClean) {
-      props.setProperty(AUTO_CLEAN_PROP, String.valueOf(autoClean));
+      compactionConfig.setValue(AUTO_CLEAN_PROP, String.valueOf(autoClean));
       return this;
     }
 
     public Builder withAsyncClean(Boolean asyncClean) {
-      props.setProperty(ASYNC_CLEAN_PROP, String.valueOf(asyncClean));
+      compactionConfig.setValue(ASYNC_CLEAN_PROP, String.valueOf(asyncClean));
       return this;
     }
 
     public Builder withIncrementalCleaningMode(Boolean incrementalCleaningMode) {
-      props.setProperty(CLEANER_INCREMENTAL_MODE, String.valueOf(incrementalCleaningMode));
+      compactionConfig.setValue(CLEANER_INCREMENTAL_MODE, String.valueOf(incrementalCleaningMode));
       return this;
     }
 
     public Builder withInlineCompaction(Boolean inlineCompaction) {
-      props.setProperty(INLINE_COMPACT_PROP, String.valueOf(inlineCompaction));
+      compactionConfig.setValue(INLINE_COMPACT_PROP, String.valueOf(inlineCompaction));
       return this;
     }
 
     public Builder withInlineCompactionTriggerStrategy(CompactionTriggerStrategy compactionTriggerStrategy) {
-      props.setProperty(INLINE_COMPACT_TRIGGER_STRATEGY_PROP, compactionTriggerStrategy.name());
+      compactionConfig.setValue(INLINE_COMPACT_TRIGGER_STRATEGY_PROP, compactionTriggerStrategy.name());
       return this;
     }
 
     public Builder withCleanerPolicy(HoodieCleaningPolicy policy) {
-      props.setProperty(CLEANER_POLICY_PROP, policy.name());
+      compactionConfig.setValue(CLEANER_POLICY_PROP, policy.name());
       return this;
     }
 
     public Builder retainFileVersions(int fileVersionsRetained) {
-      props.setProperty(CLEANER_FILE_VERSIONS_RETAINED_PROP, String.valueOf(fileVersionsRetained));
+      compactionConfig.setValue(CLEANER_FILE_VERSIONS_RETAINED_PROP, String.valueOf(fileVersionsRetained));
       return this;
     }
 
     public Builder retainCommits(int commitsRetained) {
-      props.setProperty(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
+      compactionConfig.setValue(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
       return this;
     }
 
     public Builder archiveCommitsWith(int minToKeep, int maxToKeep) {
-      props.setProperty(MIN_COMMITS_TO_KEEP_PROP, String.valueOf(minToKeep));
-      props.setProperty(MAX_COMMITS_TO_KEEP_PROP, String.valueOf(maxToKeep));
+      compactionConfig.setValue(MIN_COMMITS_TO_KEEP_PROP, String.valueOf(minToKeep));
+      compactionConfig.setValue(MAX_COMMITS_TO_KEEP_PROP, String.valueOf(maxToKeep));
       return this;
     }
 
     public Builder compactionSmallFileSize(long smallFileLimitBytes) {
-      props.setProperty(PARQUET_SMALL_FILE_LIMIT_BYTES, String.valueOf(smallFileLimitBytes));
+      compactionConfig.setValue(PARQUET_SMALL_FILE_LIMIT_BYTES, String.valueOf(smallFileLimitBytes));
       return this;
     }
 
     public Builder compactionRecordSizeEstimateThreshold(double threshold) {
-      props.setProperty(RECORD_SIZE_ESTIMATION_THRESHOLD_PROP, String.valueOf(threshold));
+      compactionConfig.setValue(RECORD_SIZE_ESTIMATION_THRESHOLD_PROP, String.valueOf(threshold));
       return this;
     }
 
     public Builder insertSplitSize(int insertSplitSize) {
-      props.setProperty(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, String.valueOf(insertSplitSize));
+      compactionConfig.setValue(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, String.valueOf(insertSplitSize));
       return this;
     }
 
     public Builder autoTuneInsertSplits(boolean autoTuneInsertSplits) {
-      props.setProperty(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, String.valueOf(autoTuneInsertSplits));
+      compactionConfig.setValue(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, String.valueOf(autoTuneInsertSplits));
       return this;
     }
 
     public Builder approxRecordSize(int recordSizeEstimate) {
-      props.setProperty(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE, String.valueOf(recordSizeEstimate));
+      compactionConfig.setValue(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE, String.valueOf(recordSizeEstimate));
       return this;
     }
 
     public Builder withCleanerParallelism(int cleanerParallelism) {
-      props.setProperty(CLEANER_PARALLELISM, String.valueOf(cleanerParallelism));
+      compactionConfig.setValue(CLEANER_PARALLELISM, String.valueOf(cleanerParallelism));
       return this;
     }
 
     public Builder withCompactionStrategy(CompactionStrategy compactionStrategy) {
-      props.setProperty(COMPACTION_STRATEGY_PROP, compactionStrategy.getClass().getName());
+      compactionConfig.setValue(COMPACTION_STRATEGY_PROP, compactionStrategy.getClass().getName());
       return this;
     }
 
     public Builder withPayloadClass(String payloadClassName) {
-      props.setProperty(PAYLOAD_CLASS_PROP, payloadClassName);
+      compactionConfig.setValue(PAYLOAD_CLASS_PROP, payloadClassName);
       return this;
     }
 
     public Builder withTargetIOPerCompactionInMB(long targetIOPerCompactionInMB) {
-      props.setProperty(TARGET_IO_PER_COMPACTION_IN_MB_PROP, String.valueOf(targetIOPerCompactionInMB));
+      compactionConfig.setValue(TARGET_IO_PER_COMPACTION_IN_MB_PROP, String.valueOf(targetIOPerCompactionInMB));
       return this;
     }
 
     public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) {
-      props.setProperty(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(maxNumDeltaCommitsBeforeCompaction));
+      compactionConfig.setValue(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(maxNumDeltaCommitsBeforeCompaction));
       return this;
     }
 
     public Builder withMaxDeltaSecondsBeforeCompaction(int maxDeltaSecondsBeforeCompaction) {
-      props.setProperty(INLINE_COMPACT_TIME_DELTA_SECONDS_PROP, String.valueOf(maxDeltaSecondsBeforeCompaction));
+      compactionConfig.setValue(INLINE_COMPACT_TIME_DELTA_SECONDS_PROP, String.valueOf(maxDeltaSecondsBeforeCompaction));
       return this;
     }
 
     public Builder withCompactionLazyBlockReadEnabled(Boolean compactionLazyBlockReadEnabled) {
-      props.setProperty(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, String.valueOf(compactionLazyBlockReadEnabled));
+      compactionConfig.setValue(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, String.valueOf(compactionLazyBlockReadEnabled));
       return this;
     }
 
     public Builder withCompactionReverseLogReadEnabled(Boolean compactionReverseLogReadEnabled) {
-      props.setProperty(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP, String.valueOf(compactionReverseLogReadEnabled));
+      compactionConfig.setValue(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP, String.valueOf(compactionReverseLogReadEnabled));
       return this;
     }
 
     public Builder withTargetPartitionsPerDayBasedCompaction(int targetPartitionsPerCompaction) {
-      props.setProperty(TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP, String.valueOf(targetPartitionsPerCompaction));
+      compactionConfig.setValue(TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP, String.valueOf(targetPartitionsPerCompaction));
       return this;
     }
 
     public Builder withCommitsArchivalBatchSize(int batchSize) {
-      props.setProperty(COMMITS_ARCHIVAL_BATCH_SIZE_PROP, String.valueOf(batchSize));
+      compactionConfig.setValue(COMMITS_ARCHIVAL_BATCH_SIZE_PROP, String.valueOf(batchSize));
       return this;
     }
 
     public Builder withCleanBootstrapBaseFileEnabled(Boolean cleanBootstrapSourceFileEnabled) {
-      props.setProperty(CLEANER_BOOTSTRAP_BASE_FILE_ENABLED, String.valueOf(cleanBootstrapSourceFileEnabled));
+      compactionConfig.setValue(CLEANER_BOOTSTRAP_BASE_FILE_ENABLED, String.valueOf(cleanBootstrapSourceFileEnabled));
       return this;
     }
 
     public Builder withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy failedWritesPolicy) {
-      props.setProperty(FAILED_WRITES_CLEANER_POLICY_PROP, failedWritesPolicy.name());
+      compactionConfig.setValue(FAILED_WRITES_CLEANER_POLICY_PROP, failedWritesPolicy.name());
       return this;
     }
 
     public HoodieCompactionConfig build() {
-      HoodieCompactionConfig config = new HoodieCompactionConfig(props);
-      setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), AUTO_CLEAN_PROP, DEFAULT_AUTO_CLEAN);
-      setDefaultOnCondition(props, !props.containsKey(ASYNC_CLEAN_PROP), ASYNC_CLEAN_PROP,
-              DEFAULT_ASYNC_CLEAN);
-      setDefaultOnCondition(props, !props.containsKey(CLEANER_INCREMENTAL_MODE), CLEANER_INCREMENTAL_MODE,
-          DEFAULT_INCREMENTAL_CLEANER);
-      setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_PROP), INLINE_COMPACT_PROP,
-          DEFAULT_INLINE_COMPACT);
-      setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP),
-          INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS);
-      setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_TIME_DELTA_SECONDS_PROP),
-          INLINE_COMPACT_TIME_DELTA_SECONDS_PROP, DEFAULT_INLINE_COMPACT_TIME_DELTA_SECONDS);
-      setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_TRIGGER_STRATEGY_PROP),
-          INLINE_COMPACT_TRIGGER_STRATEGY_PROP, DEFAULT_INLINE_COMPACT_TRIGGER_STRATEGY);
-      setDefaultOnCondition(props, !props.containsKey(CLEANER_POLICY_PROP), CLEANER_POLICY_PROP,
-          DEFAULT_CLEANER_POLICY);
-      setDefaultOnCondition(props, !props.containsKey(CLEANER_FILE_VERSIONS_RETAINED_PROP),
-          CLEANER_FILE_VERSIONS_RETAINED_PROP, DEFAULT_CLEANER_FILE_VERSIONS_RETAINED);
-      setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP), CLEANER_COMMITS_RETAINED_PROP,
-          DEFAULT_CLEANER_COMMITS_RETAINED);
-      setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP_PROP), MAX_COMMITS_TO_KEEP_PROP,
-          DEFAULT_MAX_COMMITS_TO_KEEP);
-      setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP_PROP), MIN_COMMITS_TO_KEEP_PROP,
-          DEFAULT_MIN_COMMITS_TO_KEEP);
-      setDefaultOnCondition(props, !props.containsKey(PARQUET_SMALL_FILE_LIMIT_BYTES), PARQUET_SMALL_FILE_LIMIT_BYTES,
-          DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES);
-      setDefaultOnCondition(props, !props.containsKey(RECORD_SIZE_ESTIMATION_THRESHOLD_PROP), RECORD_SIZE_ESTIMATION_THRESHOLD_PROP,
-          DEFAULT_RECORD_SIZE_ESTIMATION_THRESHOLD);
-      setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE),
-          COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE);
-      setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS),
-          COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS);
-      setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE),
-          COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE, DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE);
-      setDefaultOnCondition(props, !props.containsKey(CLEANER_PARALLELISM), CLEANER_PARALLELISM,
-          DEFAULT_CLEANER_PARALLELISM);
-      setDefaultOnCondition(props, !props.containsKey(COMPACTION_STRATEGY_PROP), COMPACTION_STRATEGY_PROP,
-          DEFAULT_COMPACTION_STRATEGY);
-      setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS_PROP), PAYLOAD_CLASS_PROP, DEFAULT_PAYLOAD_CLASS);
-      setDefaultOnCondition(props, !props.containsKey(TARGET_IO_PER_COMPACTION_IN_MB_PROP),
-          TARGET_IO_PER_COMPACTION_IN_MB_PROP, DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB);
-      setDefaultOnCondition(props, !props.containsKey(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP),
-          COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED);
-      setDefaultOnCondition(props, !props.containsKey(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP),
-          COMPACTION_REVERSE_LOG_READ_ENABLED_PROP, DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED);
-      setDefaultOnCondition(props, !props.containsKey(TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP),
-          TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP, DEFAULT_TARGET_PARTITIONS_PER_DAYBASED_COMPACTION);
-      setDefaultOnCondition(props, !props.containsKey(COMMITS_ARCHIVAL_BATCH_SIZE_PROP),
-          COMMITS_ARCHIVAL_BATCH_SIZE_PROP, DEFAULT_COMMITS_ARCHIVAL_BATCH_SIZE);
-      setDefaultOnCondition(props, !props.containsKey(CLEANER_BOOTSTRAP_BASE_FILE_ENABLED),
-          CLEANER_BOOTSTRAP_BASE_FILE_ENABLED, DEFAULT_CLEANER_BOOTSTRAP_BASE_FILE_ENABLED);
-      setDefaultOnCondition(props, !props.containsKey(FAILED_WRITES_CLEANER_POLICY_PROP),
-          FAILED_WRITES_CLEANER_POLICY_PROP, DEFAULT_FAILED_WRITES_CLEANER_POLICY);
-
-      HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP));
+      compactionConfig.setDefaults(HoodieCompactionConfig.class.getName());
+      // validation
+      HoodieCleaningPolicy.valueOf(compactionConfig.getString(CLEANER_POLICY_PROP));
 
       // Ensure minInstantsToKeep > cleanerCommitsRetained, otherwise we will archive some
       // commit instant on timeline, that still has not been cleaned. Could miss some data via incr pull
-      int minInstantsToKeep = Integer.parseInt(props.getProperty(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP));
-      int maxInstantsToKeep = Integer.parseInt(props.getProperty(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP));
+      int minInstantsToKeep = Integer.parseInt(compactionConfig.getStringOrDefault(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP));
+      int maxInstantsToKeep = Integer.parseInt(compactionConfig.getStringOrDefault(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP));
       int cleanerCommitsRetained =
-          Integer.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
+          Integer.parseInt(compactionConfig.getStringOrDefault(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
       ValidationUtils.checkArgument(maxInstantsToKeep > minInstantsToKeep,
           String.format(
               "Increase %s=%d to be greater than %s=%d.",
-              HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP, maxInstantsToKeep,
-              HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP, minInstantsToKeep));
+              HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP.key(), maxInstantsToKeep,
+              HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP.key(), minInstantsToKeep));
       ValidationUtils.checkArgument(minInstantsToKeep > cleanerCommitsRetained,
           String.format(
               "Increase %s=%d to be greater than %s=%d. Otherwise, there is risk of incremental pull "
                   + "missing data from few instants.",
-              HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP, minInstantsToKeep,
-              HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP, cleanerCommitsRetained));
-      return config;
+              HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP.key(), minInstantsToKeep,
+              HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP.key(), cleanerCommitsRetained));
+      return compactionConfig;
     }
   }
 }
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 7a8d953..9a47ff8 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
@@ -18,7 +18,8 @@
 
 package org.apache.hudi.config;
 
-import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.index.hbase.DefaultHBaseQPSResourceAllocator;
 
 import java.io.File;
@@ -26,102 +27,127 @@ import java.io.FileReader;
 import java.io.IOException;
 import java.util.Properties;
 
-public class HoodieHBaseIndexConfig extends DefaultHoodieConfig {
-
-  public static final String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum";
-  public static final String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport";
-  public static final String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table";
-  public static final String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size";
-  public static final String HBASE_ZK_ZNODEPARENT = "hoodie.index.hbase.zknode.path";
-  /**
-   * Note that if HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP is set to true, this batch size will not be honored for HBase
-   * Puts.
-   */
-  public static final String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size";
-
-  /**
-   * Property to set which implementation of HBase QPS resource allocator to be used.
-   */
-  public static final String HBASE_INDEX_QPS_ALLOCATOR_CLASS = "hoodie.index.hbase.qps.allocator.class";
-  public static final String DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS = DefaultHBaseQPSResourceAllocator.class.getName();
-  /**
-   * Property to set to enable auto computation of put batch size.
-   */
-  public static final String HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP = "hoodie.index.hbase.put.batch.size.autocompute";
-  public static final String DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE = "false";
-  /**
-   * 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.
-   */
-  public static final String HBASE_QPS_FRACTION_PROP = "hoodie.index.hbase.qps.fraction";
-  /**
-   * Property to set maximum QPS allowed per Region Server. This should be same across various jobs. This is intended to
-   * limit the aggregate QPS generated across various jobs to an Hbase Region Server. It is recommended to set this
-   * value based on global indexing throughput needs and most importantly, how much the HBase installation in use is
-   * able to tolerate without Region Servers going down.
-   */
-  public static final String HBASE_MAX_QPS_PER_REGION_SERVER_PROP = "hoodie.index.hbase.max.qps.per.region.server";
-  /**
-   * Default batch size, used only for Get, but computed for Put.
-   */
-  public static final int DEFAULT_HBASE_BATCH_SIZE = 100;
-  /**
-   * A low default value.
-   */
-  public static final int DEFAULT_HBASE_MAX_QPS_PER_REGION_SERVER = 1000;
-  /**
-   * Default is 50%, which means a total of 2 jobs can run using HbaseIndex without overwhelming Region Servers.
-   */
-  public static final float DEFAULT_HBASE_QPS_FRACTION = 0.5f;
-
-  /**
-   * Property to decide if HBASE_QPS_FRACTION_PROP is dynamically calculated based on volume.
-   */
-  public static final String HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY = "hoodie.index.hbase.dynamic_qps";
-  public static final boolean DEFAULT_HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY = false;
-  /**
-   * Min and Max for HBASE_QPS_FRACTION_PROP to stabilize skewed volume workloads.
-   */
-  public static final String HBASE_MIN_QPS_FRACTION_PROP = "hoodie.index.hbase.min.qps.fraction";
-
-  public static final String HBASE_MAX_QPS_FRACTION_PROP = "hoodie.index.hbase.max.qps.fraction";
-
-  /**
-   * Hoodie index desired puts operation time in seconds.
-   */
-  public static final String HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS = "hoodie.index.hbase.desired_puts_time_in_secs";
-  public static final int DEFAULT_HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS = 600;
-  public static final String HBASE_SLEEP_MS_PUT_BATCH_PROP = "hoodie.index.hbase.sleep.ms.for.put.batch";
-  public static final String HBASE_SLEEP_MS_GET_BATCH_PROP = "hoodie.index.hbase.sleep.ms.for.get.batch";
-  public static final String HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS = "hoodie.index.hbase.zk.session_timeout_ms";
-  public static final int DEFAULT_ZK_SESSION_TIMEOUT_MS = 60 * 1000;
-  public static final String HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS =
-      "hoodie.index.hbase.zk.connection_timeout_ms";
-  public static final int DEFAULT_ZK_CONNECTION_TIMEOUT_MS = 15 * 1000;
-  public static final String HBASE_ZK_PATH_QPS_ROOT = "hoodie.index.hbase.zkpath.qps_root";
-  public static final String DEFAULT_HBASE_ZK_PATH_QPS_ROOT = "/QPS_ROOT";
-
-  /**
-   * Only applies if index type is Hbase.
-   * <p>
-   * When set to true, an update to a record with a different partition from its existing one
-   * will insert the record to the new partition and delete it from the old partition.
-   * <p>
-   * When set to false, a record will be updated to the old partition.
-   */
-  public static final String HBASE_INDEX_UPDATE_PARTITION_PATH = "hoodie.hbase.index.update.partition.path";
-  public static final Boolean DEFAULT_HBASE_INDEX_UPDATE_PARTITION_PATH = false;
-
-  /**
-   * 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.
-  */
-  public static final String HBASE_INDEX_ROLLBACK_SYNC = "hoodie.index.hbase.rollback.sync";
-  public static final Boolean DEFAULT_HBASE_INDEX_ROLLBACK_SYNC = false;
-
-  public HoodieHBaseIndexConfig(final Properties props) {
-    super(props);
+public class HoodieHBaseIndexConfig extends HoodieConfig {
+
+  public static final ConfigProperty<String> HBASE_ZKQUORUM_PROP = ConfigProperty
+      .key("hoodie.index.hbase.zkquorum")
+      .noDefaultValue()
+      .withDocumentation("Only applies if index type is HBASE. HBase ZK Quorum url to connect to");
+
+  public static final ConfigProperty<String> HBASE_ZKPORT_PROP = ConfigProperty
+      .key("hoodie.index.hbase.zkport")
+      .noDefaultValue()
+      .withDocumentation("Only applies if index type is HBASE. HBase ZK Quorum port to connect to");
+
+  public static final ConfigProperty<String> HBASE_TABLENAME_PROP = 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");
+
+  public static final ConfigProperty<Integer> HBASE_GET_BATCH_SIZE_PROP = ConfigProperty
+      .key("hoodie.index.hbase.get.batch.size")
+      .defaultValue(100)
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HBASE_ZK_ZNODEPARENT = 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_PROP = ConfigProperty
+      .key("hoodie.index.hbase.put.batch.size")
+      .defaultValue(100)
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HBASE_INDEX_QPS_ALLOCATOR_CLASS = 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");
+
+  public static final ConfigProperty<String> HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP = ConfigProperty
+      .key("hoodie.index.hbase.put.batch.size.autocompute")
+      .defaultValue("false")
+      .withDocumentation("Property to set to enable auto computation of put batch size");
+
+  public static final ConfigProperty<Float> HBASE_QPS_FRACTION_PROP = 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.");
+
+  public static final ConfigProperty<Integer> HBASE_MAX_QPS_PER_REGION_SERVER_PROP = 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.");
+
+  public static final ConfigProperty<Boolean> HOODIE_INDEX_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 volume");
+
+  public static final ConfigProperty<String> HBASE_MIN_QPS_FRACTION_PROP = ConfigProperty
+      .key("hoodie.index.hbase.min.qps.fraction")
+      .noDefaultValue()
+      .withDocumentation("Min for HBASE_QPS_FRACTION_PROP to stabilize skewed volume workloads");
+
+  public static final ConfigProperty<String> HBASE_MAX_QPS_FRACTION_PROP = ConfigProperty
+      .key("hoodie.index.hbase.max.qps.fraction")
+      .noDefaultValue()
+      .withDocumentation("Max for HBASE_QPS_FRACTION_PROP to stabilize skewed volume workloads");
+
+  public static final ConfigProperty<Integer> HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS = ConfigProperty
+      .key("hoodie.index.hbase.desired_puts_time_in_secs")
+      .defaultValue(600)
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HBASE_SLEEP_MS_PUT_BATCH_PROP = ConfigProperty
+      .key("hoodie.index.hbase.sleep.ms.for.put.batch")
+      .noDefaultValue()
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HBASE_SLEEP_MS_GET_BATCH_PROP = ConfigProperty
+      .key("hoodie.index.hbase.sleep.ms.for.get.batch")
+      .noDefaultValue()
+      .withDocumentation("");
+
+  public static final ConfigProperty<Integer> HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS = ConfigProperty
+      .key("hoodie.index.hbase.zk.session_timeout_ms")
+      .defaultValue(60 * 1000)
+      .withDocumentation("");
+
+  public static final ConfigProperty<Integer> HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS = ConfigProperty
+      .key("hoodie.index.hbase.zk.connection_timeout_ms")
+      .defaultValue(15 * 1000)
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HBASE_ZK_PATH_QPS_ROOT = ConfigProperty
+      .key("hoodie.index.hbase.zkpath.qps_root")
+      .defaultValue("/QPS_ROOT")
+      .withDocumentation("");
+
+  public static final ConfigProperty<Boolean> HBASE_INDEX_UPDATE_PARTITION_PATH = ConfigProperty
+      .key("hoodie.hbase.index.update.partition.path")
+      .defaultValue(false)
+      .withDocumentation("Only applies if index type is HBASE. "
+          + "When an already existing record is upserted to a new partition compared to whats in storage, "
+          + "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
+      .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");
+
+  private HoodieHBaseIndexConfig() {
+    super();
   }
 
   public static HoodieHBaseIndexConfig.Builder newBuilder() {
@@ -130,117 +156,117 @@ public class HoodieHBaseIndexConfig extends DefaultHoodieConfig {
 
   public static class Builder {
 
-    private final Properties props = new Properties();
+    private final HoodieHBaseIndexConfig hBaseIndexConfig = new HoodieHBaseIndexConfig();
 
     public HoodieHBaseIndexConfig.Builder fromFile(File propertiesFile) throws IOException {
       try (FileReader reader = new FileReader(propertiesFile)) {
-        this.props.load(reader);
+        this.hBaseIndexConfig.getProps().load(reader);
         return this;
       }
     }
 
     public HoodieHBaseIndexConfig.Builder fromProperties(Properties props) {
-      this.props.putAll(props);
+      this.hBaseIndexConfig.getProps().putAll(props);
       return this;
     }
 
     public HoodieHBaseIndexConfig.Builder hbaseZkQuorum(String zkString) {
-      props.setProperty(HBASE_ZKQUORUM_PROP, zkString);
+      hBaseIndexConfig.setValue(HBASE_ZKQUORUM_PROP, zkString);
       return this;
     }
 
     public HoodieHBaseIndexConfig.Builder hbaseZkPort(int port) {
-      props.setProperty(HBASE_ZKPORT_PROP, String.valueOf(port));
+      hBaseIndexConfig.setValue(HBASE_ZKPORT_PROP, String.valueOf(port));
       return this;
     }
 
     public HoodieHBaseIndexConfig.Builder hbaseTableName(String tableName) {
-      props.setProperty(HBASE_TABLENAME_PROP, tableName);
+      hBaseIndexConfig.setValue(HBASE_TABLENAME_PROP, tableName);
       return this;
     }
 
     public Builder hbaseZkZnodeQPSPath(String zkZnodeQPSPath) {
-      props.setProperty(HBASE_ZK_PATH_QPS_ROOT, zkZnodeQPSPath);
+      hBaseIndexConfig.setValue(HBASE_ZK_PATH_QPS_ROOT, zkZnodeQPSPath);
       return this;
     }
 
     public Builder hbaseIndexGetBatchSize(int getBatchSize) {
-      props.setProperty(HBASE_GET_BATCH_SIZE_PROP, String.valueOf(getBatchSize));
+      hBaseIndexConfig.setValue(HBASE_GET_BATCH_SIZE_PROP, String.valueOf(getBatchSize));
       return this;
     }
 
     public Builder hbaseIndexPutBatchSize(int putBatchSize) {
-      props.setProperty(HBASE_PUT_BATCH_SIZE_PROP, String.valueOf(putBatchSize));
+      hBaseIndexConfig.setValue(HBASE_PUT_BATCH_SIZE_PROP, String.valueOf(putBatchSize));
       return this;
     }
 
     public Builder hbaseIndexPutBatchSizeAutoCompute(boolean putBatchSizeAutoCompute) {
-      props.setProperty(HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP, String.valueOf(putBatchSizeAutoCompute));
+      hBaseIndexConfig.setValue(HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP, String.valueOf(putBatchSizeAutoCompute));
       return this;
     }
 
     public Builder hbaseIndexDesiredPutsTime(int desiredPutsTime) {
-      props.setProperty(HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS, String.valueOf(desiredPutsTime));
+      hBaseIndexConfig.setValue(HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS, String.valueOf(desiredPutsTime));
       return this;
     }
 
     public Builder hbaseIndexShouldComputeQPSDynamically(boolean shouldComputeQPsDynamically) {
-      props.setProperty(HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY, String.valueOf(shouldComputeQPsDynamically));
+      hBaseIndexConfig.setValue(HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY, String.valueOf(shouldComputeQPsDynamically));
       return this;
     }
 
     public Builder hbaseIndexQPSFraction(float qpsFraction) {
-      props.setProperty(HBASE_QPS_FRACTION_PROP, String.valueOf(qpsFraction));
+      hBaseIndexConfig.setValue(HBASE_QPS_FRACTION_PROP, String.valueOf(qpsFraction));
       return this;
     }
 
     public Builder hbaseIndexMinQPSFraction(float minQPSFraction) {
-      props.setProperty(HBASE_MIN_QPS_FRACTION_PROP, String.valueOf(minQPSFraction));
+      hBaseIndexConfig.setValue(HBASE_MIN_QPS_FRACTION_PROP, String.valueOf(minQPSFraction));
       return this;
     }
 
     public Builder hbaseIndexMaxQPSFraction(float maxQPSFraction) {
-      props.setProperty(HBASE_MAX_QPS_FRACTION_PROP, String.valueOf(maxQPSFraction));
+      hBaseIndexConfig.setValue(HBASE_MAX_QPS_FRACTION_PROP, String.valueOf(maxQPSFraction));
       return this;
     }
 
     public Builder hbaseIndexSleepMsBetweenPutBatch(int sleepMsBetweenPutBatch) {
-      props.setProperty(HBASE_SLEEP_MS_PUT_BATCH_PROP, String.valueOf(sleepMsBetweenPutBatch));
+      hBaseIndexConfig.setValue(HBASE_SLEEP_MS_PUT_BATCH_PROP, String.valueOf(sleepMsBetweenPutBatch));
       return this;
     }
 
     public Builder hbaseIndexSleepMsBetweenGetBatch(int sleepMsBetweenGetBatch) {
-      props.setProperty(HBASE_SLEEP_MS_GET_BATCH_PROP, String.valueOf(sleepMsBetweenGetBatch));
+      hBaseIndexConfig.setValue(HBASE_SLEEP_MS_GET_BATCH_PROP, String.valueOf(sleepMsBetweenGetBatch));
       return this;
     }
 
     public Builder hbaseIndexUpdatePartitionPath(boolean updatePartitionPath) {
-      props.setProperty(HBASE_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
+      hBaseIndexConfig.setValue(HBASE_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
       return this;
     }
 
     public Builder hbaseIndexRollbackSync(boolean rollbackSync) {
-      props.setProperty(HBASE_INDEX_ROLLBACK_SYNC, String.valueOf(rollbackSync));
+      hBaseIndexConfig.setValue(HBASE_INDEX_ROLLBACK_SYNC, String.valueOf(rollbackSync));
       return this;
     }
 
     public Builder withQPSResourceAllocatorType(String qpsResourceAllocatorClass) {
-      props.setProperty(HBASE_INDEX_QPS_ALLOCATOR_CLASS, qpsResourceAllocatorClass);
+      hBaseIndexConfig.setValue(HBASE_INDEX_QPS_ALLOCATOR_CLASS, qpsResourceAllocatorClass);
       return this;
     }
 
     public Builder hbaseIndexZkSessionTimeout(int zkSessionTimeout) {
-      props.setProperty(HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS, String.valueOf(zkSessionTimeout));
+      hBaseIndexConfig.setValue(HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS, String.valueOf(zkSessionTimeout));
       return this;
     }
 
     public Builder hbaseIndexZkConnectionTimeout(int zkConnectionTimeout) {
-      props.setProperty(HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS, String.valueOf(zkConnectionTimeout));
+      hBaseIndexConfig.setValue(HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS, String.valueOf(zkConnectionTimeout));
       return this;
     }
 
     public Builder hbaseZkZnodeParent(String zkZnodeParent) {
-      props.setProperty(HBASE_ZK_ZNODEPARENT, zkZnodeParent);
+      hBaseIndexConfig.setValue(HBASE_ZK_ZNODEPARENT, zkZnodeParent);
       return this;
     }
 
@@ -256,42 +282,14 @@ public class HoodieHBaseIndexConfig extends DefaultHoodieConfig {
      */
     public HoodieHBaseIndexConfig.Builder hbaseIndexMaxQPSPerRegionServer(int maxQPSPerRegionServer) {
       // This should be same across various jobs
-      props.setProperty(HoodieHBaseIndexConfig.HBASE_MAX_QPS_PER_REGION_SERVER_PROP,
+      hBaseIndexConfig.setValue(HoodieHBaseIndexConfig.HBASE_MAX_QPS_PER_REGION_SERVER_PROP,
           String.valueOf(maxQPSPerRegionServer));
       return this;
     }
 
     public HoodieHBaseIndexConfig build() {
-      HoodieHBaseIndexConfig config = new HoodieHBaseIndexConfig(props);
-      setDefaultOnCondition(props, !props.containsKey(HBASE_GET_BATCH_SIZE_PROP), HBASE_GET_BATCH_SIZE_PROP,
-          String.valueOf(DEFAULT_HBASE_BATCH_SIZE));
-      setDefaultOnCondition(props, !props.containsKey(HBASE_PUT_BATCH_SIZE_PROP), HBASE_PUT_BATCH_SIZE_PROP,
-          String.valueOf(DEFAULT_HBASE_BATCH_SIZE));
-      setDefaultOnCondition(props, !props.containsKey(HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP),
-          HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP, DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE);
-      setDefaultOnCondition(props, !props.containsKey(HBASE_QPS_FRACTION_PROP), HBASE_QPS_FRACTION_PROP,
-          String.valueOf(DEFAULT_HBASE_QPS_FRACTION));
-      setDefaultOnCondition(props, !props.containsKey(HBASE_MAX_QPS_PER_REGION_SERVER_PROP),
-          HBASE_MAX_QPS_PER_REGION_SERVER_PROP, String.valueOf(DEFAULT_HBASE_MAX_QPS_PER_REGION_SERVER));
-      setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY),
-          HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY, String.valueOf(DEFAULT_HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY));
-      setDefaultOnCondition(props, !props.containsKey(HBASE_INDEX_QPS_ALLOCATOR_CLASS), HBASE_INDEX_QPS_ALLOCATOR_CLASS,
-          String.valueOf(DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
-      setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS),
-          HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS, String.valueOf(DEFAULT_HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS));
-      setDefaultOnCondition(props, !props.containsKey(HBASE_ZK_PATH_QPS_ROOT), HBASE_ZK_PATH_QPS_ROOT,
-          DEFAULT_HBASE_ZK_PATH_QPS_ROOT);
-      setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS),
-          HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS, String.valueOf(DEFAULT_ZK_SESSION_TIMEOUT_MS));
-      setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS),
-          HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS, String.valueOf(DEFAULT_ZK_CONNECTION_TIMEOUT_MS));
-      setDefaultOnCondition(props, !props.containsKey(HBASE_INDEX_QPS_ALLOCATOR_CLASS), HBASE_INDEX_QPS_ALLOCATOR_CLASS,
-          String.valueOf(DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
-      setDefaultOnCondition(props, !props.containsKey(HBASE_INDEX_UPDATE_PARTITION_PATH), HBASE_INDEX_UPDATE_PARTITION_PATH,
-          String.valueOf(DEFAULT_HBASE_INDEX_UPDATE_PARTITION_PATH));
-      setDefaultOnCondition(props, !props.containsKey(HBASE_INDEX_ROLLBACK_SYNC), HBASE_INDEX_ROLLBACK_SYNC,
-          String.valueOf(DEFAULT_HBASE_INDEX_ROLLBACK_SYNC));
-      return config;
+      hBaseIndexConfig.setDefaults(HoodieHBaseIndexConfig.class.getName());
+      return hBaseIndexConfig;
     }
 
   }
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 41b0a10..ba9cb18 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
@@ -19,7 +19,8 @@
 package org.apache.hudi.config;
 
 import org.apache.hudi.common.bloom.BloomFilterTypeCode;
-import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.common.engine.EngineType;
 import org.apache.hudi.exception.HoodieNotSupportedException;
 import org.apache.hudi.index.HoodieIndex;
@@ -35,60 +36,137 @@ import java.util.Properties;
  * Indexing related config.
  */
 @Immutable
-public class HoodieIndexConfig extends DefaultHoodieConfig {
-
-  public static final String INDEX_TYPE_PROP = "hoodie.index.type";
-
-  public static final String INDEX_CLASS_PROP = "hoodie.index.class";
-  public static final String DEFAULT_INDEX_CLASS = "";
+public class HoodieIndexConfig extends HoodieConfig {
+
+  public static final ConfigProperty<String> INDEX_TYPE_PROP = ConfigProperty
+      .key("hoodie.index.type")
+      .noDefaultValue()
+      .withDocumentation("Type of index to use. Default is Bloom filter. "
+          + "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");
+
+  public static final ConfigProperty<String> INDEX_CLASS_PROP = 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");
 
   // ***** Bloom Index configs *****
-  public static final String BLOOM_FILTER_NUM_ENTRIES = "hoodie.index.bloom.num_entries";
-  public static final String DEFAULT_BLOOM_FILTER_NUM_ENTRIES = "60000";
-  public static final String BLOOM_FILTER_FPP = "hoodie.index.bloom.fpp";
-  public static final String DEFAULT_BLOOM_FILTER_FPP = "0.000000001";
-  public static final String BLOOM_INDEX_PARALLELISM_PROP = "hoodie.bloom.index.parallelism";
-  // Disable explicit bloom index parallelism setting by default - hoodie auto computes
-  public static final String DEFAULT_BLOOM_INDEX_PARALLELISM = "0";
-  public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP = "hoodie.bloom.index.prune.by.ranges";
-  public static final String DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES = "true";
-  public static final String BLOOM_INDEX_USE_CACHING_PROP = "hoodie.bloom.index.use.caching";
-  public static final String DEFAULT_BLOOM_INDEX_USE_CACHING = "true";
-  public static final String BLOOM_INDEX_TREE_BASED_FILTER_PROP = "hoodie.bloom.index.use.treebased.filter";
-  public static final String DEFAULT_BLOOM_INDEX_TREE_BASED_FILTER = "true";
+  public static final ConfigProperty<String> BLOOM_FILTER_NUM_ENTRIES = ConfigProperty
+      .key("hoodie.index.bloom.num_entries")
+      .defaultValue("60000")
+      .withDocumentation("Only applies if index type is BLOOM. "
+          + "This is the number of entries to be stored in the bloom filter. "
+          + "We assume the maxParquetFileSize is 128MB and averageRecordSize is 1024B and "
+          + "hence we approx a total of 130K records in a file. The default (60000) is roughly half of this approximation. "
+          + "HUDI-56 tracks computing this dynamically. Warning: Setting this very low, "
+          + "will generate a lot of false positives and index lookup will have to scan a lot more files "
+          + "than it has to and Setting this to a very high number will increase the size every data file linearly "
+          + "(roughly 4KB for every 50000 entries). "
+          + "This config is also used with DYNNAMIC bloom filter which determines the initial size for the bloom.");
+
+  public static final ConfigProperty<String> BLOOM_FILTER_FPP = ConfigProperty
+      .key("hoodie.index.bloom.fpp")
+      .defaultValue("0.000000001")
+      .withDocumentation("Only applies if index type is BLOOM. "
+          + "Error rate allowed given the number of entries. This is used to calculate how many bits should be "
+          + "assigned for the bloom filter and the number of hash functions. This is usually set very low (default: 0.000000001), "
+          + "we like to tradeoff disk space for lower false positives. "
+          + "If the number of entries added to bloom filter exceeds the congfigured value (hoodie.index.bloom.num_entries), "
+          + "then this fpp may not be honored.");
+
+  public static final ConfigProperty<String> BLOOM_INDEX_PARALLELISM_PROP = ConfigProperty
+      .key("hoodie.bloom.index.parallelism")
+      .defaultValue("0")
+      .withDocumentation("Only applies if index type is BLOOM. "
+          + "This is the amount of parallelism for index lookup, which involves a Spark Shuffle. "
+          + "By default, this is auto computed based on input workload characteristics. "
+          + "Disable explicit bloom index parallelism setting by default - hoodie auto computes");
+
+  public static final ConfigProperty<String> BLOOM_INDEX_PRUNE_BY_RANGES_PROP = ConfigProperty
+      .key("hoodie.bloom.index.prune.by.ranges")
+      .defaultValue("true")
+      .withDocumentation("Only applies if index type is BLOOM. "
+          + "When true, range information from files to leveraged speed up index lookups. Particularly helpful, "
+          + "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.");
+
+  public static final ConfigProperty<String> BLOOM_INDEX_USE_CACHING_PROP = ConfigProperty
+      .key("hoodie.bloom.index.use.caching")
+      .defaultValue("true")
+      .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");
+
+  public static final ConfigProperty<String> BLOOM_INDEX_TREE_BASED_FILTER_PROP = ConfigProperty
+      .key("hoodie.bloom.index.use.treebased.filter")
+      .defaultValue("true")
+      .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");
+
   // TODO: On by default. Once stable, we will remove the other mode.
-  public static final String BLOOM_INDEX_BUCKETIZED_CHECKING_PROP = "hoodie.bloom.index.bucketized.checking";
-  public static final String DEFAULT_BLOOM_INDEX_BUCKETIZED_CHECKING = "true";
-  public static final String BLOOM_INDEX_FILTER_TYPE = "hoodie.bloom.index.filter.type";
-  public static final String DEFAULT_BLOOM_INDEX_FILTER_TYPE = BloomFilterTypeCode.SIMPLE.name();
-  public static final String HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES = "hoodie.bloom.index.filter.dynamic.max.entries";
-  public static final String DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES = "100000";
-  public static final String SIMPLE_INDEX_USE_CACHING_PROP = "hoodie.simple.index.use.caching";
-  public static final String DEFAULT_SIMPLE_INDEX_USE_CACHING = "true";
-  public static final String SIMPLE_INDEX_PARALLELISM_PROP = "hoodie.simple.index.parallelism";
-  public static final String DEFAULT_SIMPLE_INDEX_PARALLELISM = "50";
-  public static final String GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP = "hoodie.global.simple.index.parallelism";
-  public static final String DEFAULT_GLOBAL_SIMPLE_INDEX_PARALLELISM = "100";
+  public static final ConfigProperty<String> BLOOM_INDEX_BUCKETIZED_CHECKING_PROP = ConfigProperty
+      .key("hoodie.bloom.index.bucketized.checking")
+      .defaultValue("true")
+      .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");
+
+  public static final ConfigProperty<String> BLOOM_INDEX_FILTER_TYPE = ConfigProperty
+      .key("hoodie.bloom.index.filter.type")
+      .defaultValue(BloomFilterTypeCode.SIMPLE.name())
+      .withDocumentation("Filter type used. Default is BloomFilterTypeCode.SIMPLE. "
+          + "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
+      .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. "
+          + "Only applies if filter type is BloomFilterTypeCode.DYNAMIC_V0.");
+
+  public static final ConfigProperty<String> SIMPLE_INDEX_USE_CACHING_PROP = ConfigProperty
+      .key("hoodie.simple.index.use.caching")
+      .defaultValue("true")
+      .withDocumentation("Only applies if index type is SIMPLE. "
+          + "When true, the input RDD will cached to speed up index lookup by reducing IO "
+          + "for computing parallelism or affected partitions");
+
+  public static final ConfigProperty<String> SIMPLE_INDEX_PARALLELISM_PROP = 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");
+
+  public static final ConfigProperty<String> GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP = 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");
 
   // 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.
-  public static final String BLOOM_INDEX_KEYS_PER_BUCKET_PROP = "hoodie.bloom.index.keys.per.bucket";
-  public static final String DEFAULT_BLOOM_INDEX_KEYS_PER_BUCKET = "10000000";
-
-  // ***** HBase Index Configs *****
-  public static final String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum";
-  public static final String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport";
-  public static final String HBASE_ZK_ZNODEPARENT = "hoodie.index.hbase.zknode.path";
-  public static final String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table";
-  public static final String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size";
-  public static final String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size";
-  public static final String DEFAULT_HBASE_BATCH_SIZE = "100";
-
-
-  public static final String BLOOM_INDEX_INPUT_STORAGE_LEVEL = "hoodie.bloom.index.input.storage.level";
-  public static final String DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
-  public static final String SIMPLE_INDEX_INPUT_STORAGE_LEVEL = "hoodie.simple.index.input.storage.level";
-  public static final String DEFAULT_SIMPLE_INDEX_INPUT_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
+  public static final ConfigProperty<String> BLOOM_INDEX_KEYS_PER_BUCKET_PROP = ConfigProperty
+      .key("hoodie.bloom.index.keys.per.bucket")
+      .defaultValue("10000000")
+      .withDocumentation("Only applies if bloomIndexBucketizedChecking is enabled and index type is bloom. "
+          + "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.");
+
+  public static final ConfigProperty<String> BLOOM_INDEX_INPUT_STORAGE_LEVEL = 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
+      .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. "
+          + "Refer to org.apache.spark.storage.StorageLevel for different values");
 
   /**
    * Only applies if index type is GLOBAL_BLOOM.
@@ -98,23 +176,31 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
    * <p>
    * When set to false, a record will be updated to the old partition.
    */
-  public static final String BLOOM_INDEX_UPDATE_PARTITION_PATH = "hoodie.bloom.index.update.partition.path";
-  public static final String DEFAULT_BLOOM_INDEX_UPDATE_PARTITION_PATH = "false";
-
-  public static final String SIMPLE_INDEX_UPDATE_PARTITION_PATH = "hoodie.simple.index.update.partition.path";
-  public static final String DEFAULT_SIMPLE_INDEX_UPDATE_PARTITION_PATH = "false";
+  public static final ConfigProperty<String> BLOOM_INDEX_UPDATE_PARTITION_PATH = ConfigProperty
+      .key("hoodie.bloom.index.update.partition.path")
+      .defaultValue("false")
+      .withDocumentation("Only applies if index type is GLOBAL_BLOOM. "
+          + "When set to true, an update including the partition path of a record that already exists will result in "
+          + "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
+      .key("hoodie.simple.index.update.partition.path")
+      .defaultValue("false")
+      .withDocumentation("");
 
   private EngineType engineType;
 
   /**
    * Use Spark engine by default.
    */
-  private HoodieIndexConfig(Properties props) {
-    this(EngineType.SPARK, props);
+
+  private HoodieIndexConfig() {
+    this(EngineType.SPARK);
   }
 
-  private HoodieIndexConfig(EngineType engineType, Properties props) {
-    super(props);
+  private HoodieIndexConfig(EngineType engineType) {
+    super();
     this.engineType = engineType;
   }
 
@@ -125,127 +211,107 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
   public static class Builder {
 
     private EngineType engineType = EngineType.SPARK;
-    private final Properties props = new Properties();
+    private final HoodieIndexConfig hoodieIndexConfig = new HoodieIndexConfig();
 
     public Builder fromFile(File propertiesFile) throws IOException {
       try (FileReader reader = new FileReader(propertiesFile)) {
-        this.props.load(reader);
+        this.hoodieIndexConfig.getProps().load(reader);
         return this;
       }
     }
 
     public Builder fromProperties(Properties props) {
-      this.props.putAll(props);
+      this.hoodieIndexConfig.getProps().putAll(props);
       return this;
     }
 
     public Builder withIndexType(HoodieIndex.IndexType indexType) {
-      props.setProperty(INDEX_TYPE_PROP, indexType.name());
+      hoodieIndexConfig.setValue(INDEX_TYPE_PROP, indexType.name());
       return this;
     }
 
     public Builder withIndexClass(String indexClass) {
-      props.setProperty(INDEX_CLASS_PROP, indexClass);
+      hoodieIndexConfig.setValue(INDEX_CLASS_PROP, indexClass);
       return this;
     }
 
     public Builder withHBaseIndexConfig(HoodieHBaseIndexConfig hBaseIndexConfig) {
-      props.putAll(hBaseIndexConfig.getProps());
+      hoodieIndexConfig.getProps().putAll(hBaseIndexConfig.getProps());
       return this;
     }
 
     public Builder bloomFilterNumEntries(int numEntries) {
-      props.setProperty(BLOOM_FILTER_NUM_ENTRIES, String.valueOf(numEntries));
+      hoodieIndexConfig.setValue(BLOOM_FILTER_NUM_ENTRIES, String.valueOf(numEntries));
       return this;
     }
 
     public Builder bloomFilterFPP(double fpp) {
-      props.setProperty(BLOOM_FILTER_FPP, String.valueOf(fpp));
-      return this;
-    }
-
-    public Builder hbaseZkQuorum(String zkString) {
-      props.setProperty(HBASE_ZKQUORUM_PROP, zkString);
-      return this;
-    }
-
-    public Builder hbaseZkPort(int port) {
-      props.setProperty(HBASE_ZKPORT_PROP, String.valueOf(port));
-      return this;
-    }
-
-    public Builder hbaseZkZnodeParent(String zkZnodeParent) {
-      props.setProperty(HBASE_ZK_ZNODEPARENT, zkZnodeParent);
-      return this;
-    }
-
-    public Builder hbaseTableName(String tableName) {
-      props.setProperty(HBASE_TABLENAME_PROP, tableName);
+      hoodieIndexConfig.setValue(BLOOM_FILTER_FPP, String.valueOf(fpp));
       return this;
     }
 
     public Builder bloomIndexParallelism(int parallelism) {
-      props.setProperty(BLOOM_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
+      hoodieIndexConfig.setValue(BLOOM_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
       return this;
     }
 
     public Builder bloomIndexPruneByRanges(boolean pruneRanges) {
-      props.setProperty(BLOOM_INDEX_PRUNE_BY_RANGES_PROP, String.valueOf(pruneRanges));
+      hoodieIndexConfig.setValue(BLOOM_INDEX_PRUNE_BY_RANGES_PROP, String.valueOf(pruneRanges));
       return this;
     }
 
     public Builder bloomIndexUseCaching(boolean useCaching) {
-      props.setProperty(BLOOM_INDEX_USE_CACHING_PROP, String.valueOf(useCaching));
+      hoodieIndexConfig.setValue(BLOOM_INDEX_USE_CACHING_PROP, String.valueOf(useCaching));
       return this;
     }
 
     public Builder bloomIndexTreebasedFilter(boolean useTreeFilter) {
-      props.setProperty(BLOOM_INDEX_TREE_BASED_FILTER_PROP, String.valueOf(useTreeFilter));
+      hoodieIndexConfig.setValue(BLOOM_INDEX_TREE_BASED_FILTER_PROP, String.valueOf(useTreeFilter));
       return this;
     }
 
     public Builder bloomIndexBucketizedChecking(boolean bucketizedChecking) {
-      props.setProperty(BLOOM_INDEX_BUCKETIZED_CHECKING_PROP, String.valueOf(bucketizedChecking));
+      hoodieIndexConfig.setValue(BLOOM_INDEX_BUCKETIZED_CHECKING_PROP, String.valueOf(bucketizedChecking));
       return this;
     }
 
     public Builder bloomIndexKeysPerBucket(int keysPerBucket) {
-      props.setProperty(BLOOM_INDEX_KEYS_PER_BUCKET_PROP, String.valueOf(keysPerBucket));
+      hoodieIndexConfig.setValue(BLOOM_INDEX_KEYS_PER_BUCKET_PROP, String.valueOf(keysPerBucket));
       return this;
     }
 
     public Builder withBloomIndexInputStorageLevel(String level) {
-      props.setProperty(BLOOM_INDEX_INPUT_STORAGE_LEVEL, level);
+      hoodieIndexConfig.setValue(BLOOM_INDEX_INPUT_STORAGE_LEVEL, level);
       return this;
     }
 
     public Builder withBloomIndexUpdatePartitionPath(boolean updatePartitionPath) {
-      props.setProperty(BLOOM_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
+      hoodieIndexConfig.setValue(BLOOM_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
       return this;
     }
 
     public Builder withSimpleIndexParallelism(int parallelism) {
-      props.setProperty(SIMPLE_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
+      hoodieIndexConfig.setValue(SIMPLE_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
       return this;
     }
 
     public Builder simpleIndexUseCaching(boolean useCaching) {
-      props.setProperty(SIMPLE_INDEX_USE_CACHING_PROP, String.valueOf(useCaching));
+      hoodieIndexConfig.setValue(SIMPLE_INDEX_USE_CACHING_PROP, String.valueOf(useCaching));
       return this;
     }
 
     public Builder withSimpleIndexInputStorageLevel(String level) {
-      props.setProperty(SIMPLE_INDEX_INPUT_STORAGE_LEVEL, level);
+      hoodieIndexConfig.setValue(SIMPLE_INDEX_INPUT_STORAGE_LEVEL, level);
       return this;
     }
 
     public Builder withGlobalSimpleIndexParallelism(int parallelism) {
-      props.setProperty(GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
+      hoodieIndexConfig.setValue(GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
       return this;
     }
 
     public Builder withGlobalSimpleIndexUpdatePartitionPath(boolean updatePartitionPath) {
-      props.setProperty(SIMPLE_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
+      hoodieIndexConfig.setValue(SIMPLE_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
       return this;
     }
 
@@ -255,45 +321,12 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
     }
 
     public HoodieIndexConfig build() {
-      HoodieIndexConfig config = new HoodieIndexConfig(engineType, props);
-      setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP), INDEX_TYPE_PROP, getDefaultIndexType(engineType));
-      setDefaultOnCondition(props, !props.containsKey(INDEX_CLASS_PROP), INDEX_CLASS_PROP, DEFAULT_INDEX_CLASS);
-      setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_NUM_ENTRIES), BLOOM_FILTER_NUM_ENTRIES,
-          DEFAULT_BLOOM_FILTER_NUM_ENTRIES);
-      setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_FPP), BLOOM_FILTER_FPP, DEFAULT_BLOOM_FILTER_FPP);
-      setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PARALLELISM_PROP), BLOOM_INDEX_PARALLELISM_PROP,
-          DEFAULT_BLOOM_INDEX_PARALLELISM);
-      setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PRUNE_BY_RANGES_PROP),
-          BLOOM_INDEX_PRUNE_BY_RANGES_PROP, DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES);
-      setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_USE_CACHING_PROP), BLOOM_INDEX_USE_CACHING_PROP,
-          DEFAULT_BLOOM_INDEX_USE_CACHING);
-      setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_INPUT_STORAGE_LEVEL), BLOOM_INDEX_INPUT_STORAGE_LEVEL,
-          DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL);
-      setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_UPDATE_PARTITION_PATH),
-          BLOOM_INDEX_UPDATE_PARTITION_PATH, DEFAULT_BLOOM_INDEX_UPDATE_PARTITION_PATH);
-      setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_TREE_BASED_FILTER_PROP),
-          BLOOM_INDEX_TREE_BASED_FILTER_PROP, DEFAULT_BLOOM_INDEX_TREE_BASED_FILTER);
-      setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_BUCKETIZED_CHECKING_PROP),
-          BLOOM_INDEX_BUCKETIZED_CHECKING_PROP, DEFAULT_BLOOM_INDEX_BUCKETIZED_CHECKING);
-      setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_KEYS_PER_BUCKET_PROP),
-          BLOOM_INDEX_KEYS_PER_BUCKET_PROP, DEFAULT_BLOOM_INDEX_KEYS_PER_BUCKET);
-      setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_FILTER_TYPE),
-          BLOOM_INDEX_FILTER_TYPE, DEFAULT_BLOOM_INDEX_FILTER_TYPE);
-      setDefaultOnCondition(props, !props.containsKey(HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES),
-          HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES, DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES);
-      setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_PARALLELISM_PROP), SIMPLE_INDEX_PARALLELISM_PROP,
-          DEFAULT_SIMPLE_INDEX_PARALLELISM);
-      setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_USE_CACHING_PROP), SIMPLE_INDEX_USE_CACHING_PROP,
-          DEFAULT_SIMPLE_INDEX_USE_CACHING);
-      setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_INPUT_STORAGE_LEVEL), SIMPLE_INDEX_INPUT_STORAGE_LEVEL,
-          DEFAULT_SIMPLE_INDEX_INPUT_STORAGE_LEVEL);
-      setDefaultOnCondition(props, !props.containsKey(GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP), GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP,
-          DEFAULT_GLOBAL_SIMPLE_INDEX_PARALLELISM);
-      setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_UPDATE_PARTITION_PATH),
-          SIMPLE_INDEX_UPDATE_PARTITION_PATH, DEFAULT_SIMPLE_INDEX_UPDATE_PARTITION_PATH);
+      hoodieIndexConfig.setDefaultValue(INDEX_TYPE_PROP, getDefaultIndexType(engineType));
+      hoodieIndexConfig.setDefaults(HoodieIndexConfig.class.getName());
+
       // Throws IllegalArgumentException if the value set is not a known Hoodie Index Type
-      HoodieIndex.IndexType.valueOf(props.getProperty(INDEX_TYPE_PROP));
-      return config;
+      HoodieIndex.IndexType.valueOf(hoodieIndexConfig.getString(INDEX_TYPE_PROP));
+      return hoodieIndexConfig;
     }
 
     private String getDefaultIndexType(EngineType engineType) {
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 7d01487..0609419 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
@@ -20,7 +20,8 @@ package org.apache.hudi.config;
 import org.apache.hudi.client.transaction.SimpleConcurrentFileWritesConflictResolutionStrategy;
 import org.apache.hudi.client.transaction.ConflictResolutionStrategy;
 import org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider;
-import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.common.lock.LockProvider;
 
 import java.io.File;
@@ -28,48 +29,152 @@ import java.io.FileReader;
 import java.io.IOException;
 import java.util.Properties;
 
-import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ACQUIRE_LOCK_WAIT_TIMEOUT_MS;
-import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_CLIENT_NUM_RETRIES;
-import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS;
-import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_MAX_RETRY_WAIT_TIME_IN_MILLIS;
 import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_NUM_RETRIES;
 import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS;
 import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_CONNECTION_TIMEOUT_MS;
 import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_SESSION_TIMEOUT_MS;
-import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.HIVE_METASTORE_URI_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP;
+import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.HIVE_METASTORE_URI_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY;
 import static org.apache.hudi.common.config.LockConfiguration.LOCK_PREFIX;
-import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.ZK_PORT_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP;
+import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.ZK_PORT_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP_KEY;
 
 
 /**
  * Hoodie Configs for Locks.
  */
-public class HoodieLockConfig extends DefaultHoodieConfig {
+public class HoodieLockConfig extends HoodieConfig {
+
+  public static final ConfigProperty<String> LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP = ConfigProperty
+      .key(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY)
+      .defaultValue(DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS)
+      .sinceVersion("0.8.0")
+      .withDocumentation("Parameter used in the exponential backoff retry policy. Stands for the Initial amount "
+          + "of time to wait between retries by lock provider client");
+
+  public static final ConfigProperty<String> LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP = ConfigProperty
+      .key(LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP_KEY)
+      .defaultValue(String.valueOf(5000L))
+      .sinceVersion("0.8.0")
+      .withDocumentation("Parameter used in the exponential backoff retry policy. Stands for the maximum amount "
+          + "of time to wait between retries by lock provider client");
+
+  public static final ConfigProperty<String> LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP = 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 from the hudi client");
+
+  public static final ConfigProperty<String> LOCK_ACQUIRE_NUM_RETRIES_PROP = 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 by lock provider client");
+
+  public static final ConfigProperty<String> LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP = 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 hudi client");
+
+  public static final ConfigProperty<Integer> LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP = ConfigProperty
+      .key(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY)
+      .defaultValue(60 * 1000)
+      .sinceVersion("0.8.0")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> FILESYSTEM_LOCK_PATH_PROP = ConfigProperty
+      .key(FILESYSTEM_LOCK_PATH_PROP_KEY)
+      .noDefaultValue()
+      .sinceVersion("0.8.0")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HIVE_DATABASE_NAME_PROP = ConfigProperty
+      .key(HIVE_DATABASE_NAME_PROP_KEY)
+      .noDefaultValue()
+      .sinceVersion("0.8.0")
+      .withDocumentation("The Hive database to acquire lock against");
+
+  public static final ConfigProperty<String> HIVE_TABLE_NAME_PROP = ConfigProperty
+      .key(HIVE_TABLE_NAME_PROP_KEY)
+      .noDefaultValue()
+      .sinceVersion("0.8.0")
+      .withDocumentation("The Hive table under the hive database to acquire lock against");
+
+  public static final ConfigProperty<String> HIVE_METASTORE_URI_PROP = ConfigProperty
+      .key(HIVE_METASTORE_URI_PROP_KEY)
+      .noDefaultValue()
+      .sinceVersion("0.8.0")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> ZK_BASE_PATH_PROP = ConfigProperty
+      .key(ZK_BASE_PATH_PROP_KEY)
+      .noDefaultValue()
+      .sinceVersion("0.8.0")
+      .withDocumentation("The base path on Zookeeper under which to create a ZNode to acquire the lock. "
+          + "This should be common for all jobs writing to the same table");
+
+  public static final ConfigProperty<Integer> ZK_SESSION_TIMEOUT_MS_PROP = ConfigProperty
+      .key(ZK_SESSION_TIMEOUT_MS_PROP_KEY)
+      .defaultValue(DEFAULT_ZK_SESSION_TIMEOUT_MS)
+      .sinceVersion("0.8.0")
+      .withDocumentation("How long to wait after losing a connection to ZooKeeper before the session is expired");
+
+  public static final ConfigProperty<Integer> ZK_CONNECTION_TIMEOUT_MS_PROP = ConfigProperty
+      .key(ZK_CONNECTION_TIMEOUT_MS_PROP_KEY)
+      .defaultValue(DEFAULT_ZK_CONNECTION_TIMEOUT_MS)
+      .sinceVersion("0.8.0")
+      .withDocumentation("How long to wait when connecting to ZooKeeper before considering the connection a failure");
+
+  public static final ConfigProperty<String> ZK_CONNECT_URL_PROP = ConfigProperty
+      .key(ZK_CONNECT_URL_PROP_KEY)
+      .noDefaultValue()
+      .sinceVersion("0.8.0")
+      .withDocumentation("Set the list of comma separated servers to connect to");
+
+  public static final ConfigProperty<String> ZK_PORT_PROP = ConfigProperty
+      .key(ZK_PORT_PROP_KEY)
+      .noDefaultValue()
+      .sinceVersion("0.8.0")
+      .withDocumentation("The connection port to be used for Zookeeper");
+
+  public static final ConfigProperty<String> ZK_LOCK_KEY_PROP = ConfigProperty
+      .key(ZK_LOCK_KEY_PROP_KEY)
+      .noDefaultValue()
+      .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");
 
   // Pluggable type of lock provider
-  public static final String LOCK_PROVIDER_CLASS_PROP = LOCK_PREFIX + "provider";
-  public static final String DEFAULT_LOCK_PROVIDER_CLASS = ZookeeperBasedLockProvider.class.getName();
-  // Pluggable strategies to use when resolving conflicts
-  public static final String WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP =
-      LOCK_PREFIX + "conflict.resolution.strategy";
-  public static final String DEFAULT_WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS =
-      SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName();
+  public static final ConfigProperty<String> LOCK_PROVIDER_CLASS_PROP = 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");
 
-  private HoodieLockConfig(Properties props) {
-    super(props);
+  // Pluggable strategies to use when resolving conflicts
+  public static final ConfigProperty<String> WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP = 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");
+
+  private HoodieLockConfig() {
+    super();
   }
 
   public static HoodieLockConfig.Builder newBuilder() {
@@ -78,128 +183,108 @@ public class HoodieLockConfig extends DefaultHoodieConfig {
 
   public static class Builder {
 
-    private final Properties props = new Properties();
+    private final HoodieLockConfig lockConfig = new HoodieLockConfig();
 
     public HoodieLockConfig.Builder fromFile(File propertiesFile) throws IOException {
       try (FileReader reader = new FileReader(propertiesFile)) {
-        this.props.load(reader);
+        this.lockConfig.getProps().load(reader);
         return this;
       }
     }
 
     public HoodieLockConfig.Builder fromProperties(Properties props) {
-      this.props.putAll(props);
+      this.lockConfig.getProps().putAll(props);
       return this;
     }
 
     public HoodieLockConfig.Builder withLockProvider(Class<? extends LockProvider> lockProvider) {
-      props.setProperty(LOCK_PROVIDER_CLASS_PROP, lockProvider.getName());
+      lockConfig.setValue(LOCK_PROVIDER_CLASS_PROP, lockProvider.getName());
       return this;
     }
 
     public HoodieLockConfig.Builder withHiveDatabaseName(String databaseName) {
-      props.setProperty(HIVE_DATABASE_NAME_PROP, databaseName);
+      lockConfig.setValue(HIVE_DATABASE_NAME_PROP, databaseName);
       return this;
     }
 
     public HoodieLockConfig.Builder withHiveTableName(String tableName) {
-      props.setProperty(HIVE_TABLE_NAME_PROP, tableName);
+      lockConfig.setValue(HIVE_TABLE_NAME_PROP, tableName);
       return this;
     }
 
     public HoodieLockConfig.Builder withHiveMetastoreURIs(String hiveMetastoreURIs) {
-      props.setProperty(HIVE_METASTORE_URI_PROP, hiveMetastoreURIs);
+      lockConfig.setValue(HIVE_METASTORE_URI_PROP, hiveMetastoreURIs);
       return this;
     }
 
     public HoodieLockConfig.Builder withZkQuorum(String zkQuorum) {
-      props.setProperty(ZK_CONNECT_URL_PROP, zkQuorum);
+      lockConfig.setValue(ZK_CONNECT_URL_PROP, zkQuorum);
       return this;
     }
 
     public HoodieLockConfig.Builder withZkBasePath(String zkBasePath) {
-      props.setProperty(ZK_BASE_PATH_PROP, zkBasePath);
+      lockConfig.setValue(ZK_BASE_PATH_PROP, zkBasePath);
       return this;
     }
 
     public HoodieLockConfig.Builder withZkPort(String zkPort) {
-      props.setProperty(ZK_PORT_PROP, zkPort);
+      lockConfig.setValue(ZK_PORT_PROP, zkPort);
       return this;
     }
 
     public HoodieLockConfig.Builder withZkLockKey(String zkLockKey) {
-      props.setProperty(ZK_LOCK_KEY_PROP, zkLockKey);
+      lockConfig.setValue(ZK_LOCK_KEY_PROP, zkLockKey);
       return this;
     }
 
     public HoodieLockConfig.Builder withZkConnectionTimeoutInMs(Long connectionTimeoutInMs) {
-      props.setProperty(ZK_CONNECTION_TIMEOUT_MS_PROP, String.valueOf(connectionTimeoutInMs));
+      lockConfig.setValue(ZK_CONNECTION_TIMEOUT_MS_PROP, String.valueOf(connectionTimeoutInMs));
       return this;
     }
 
     public HoodieLockConfig.Builder withZkSessionTimeoutInMs(Long sessionTimeoutInMs) {
-      props.setProperty(ZK_SESSION_TIMEOUT_MS_PROP, String.valueOf(sessionTimeoutInMs));
+      lockConfig.setValue(ZK_SESSION_TIMEOUT_MS_PROP, String.valueOf(sessionTimeoutInMs));
       return this;
     }
 
     public HoodieLockConfig.Builder withNumRetries(int numRetries) {
-      props.setProperty(LOCK_ACQUIRE_NUM_RETRIES_PROP, String.valueOf(numRetries));
+      lockConfig.setValue(LOCK_ACQUIRE_NUM_RETRIES_PROP, String.valueOf(numRetries));
       return this;
     }
 
     public HoodieLockConfig.Builder withRetryWaitTimeInMillis(Long retryWaitTimeInMillis) {
-      props.setProperty(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP, String.valueOf(retryWaitTimeInMillis));
+      lockConfig.setValue(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP, String.valueOf(retryWaitTimeInMillis));
       return this;
     }
 
     public HoodieLockConfig.Builder withRetryMaxWaitTimeInMillis(Long retryMaxWaitTimeInMillis) {
-      props.setProperty(LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP, String.valueOf(retryMaxWaitTimeInMillis));
+      lockConfig.setValue(LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP, String.valueOf(retryMaxWaitTimeInMillis));
       return this;
     }
 
     public HoodieLockConfig.Builder withClientNumRetries(int clientNumRetries) {
-      props.setProperty(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP, String.valueOf(clientNumRetries));
+      lockConfig.setValue(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP, String.valueOf(clientNumRetries));
       return this;
     }
 
     public HoodieLockConfig.Builder withClientRetryWaitTimeInMillis(Long clientRetryWaitTimeInMillis) {
-      props.setProperty(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP, String.valueOf(clientRetryWaitTimeInMillis));
+      lockConfig.setValue(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP, String.valueOf(clientRetryWaitTimeInMillis));
       return this;
     }
 
     public HoodieLockConfig.Builder withLockWaitTimeInMillis(Long waitTimeInMillis) {
-      props.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP, String.valueOf(waitTimeInMillis));
+      lockConfig.setValue(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP, String.valueOf(waitTimeInMillis));
       return this;
     }
 
     public HoodieLockConfig.Builder withConflictResolutionStrategy(ConflictResolutionStrategy conflictResolutionStrategy) {
-      props.setProperty(WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP, conflictResolutionStrategy.getClass().getName());
+      lockConfig.setValue(WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP, conflictResolutionStrategy.getClass().getName());
       return this;
     }
 
     public HoodieLockConfig build() {
-      HoodieLockConfig config = new HoodieLockConfig(props);
-      setDefaultOnCondition(props, !props.containsKey(LOCK_PROVIDER_CLASS_PROP),
-          LOCK_PROVIDER_CLASS_PROP, DEFAULT_LOCK_PROVIDER_CLASS);
-      setDefaultOnCondition(props, !props.containsKey(WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP),
-          WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP, DEFAULT_WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS);
-      setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_NUM_RETRIES_PROP),
-          LOCK_ACQUIRE_NUM_RETRIES_PROP, DEFAULT_LOCK_ACQUIRE_NUM_RETRIES);
-      setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP),
-          LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP, DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS);
-      setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP),
-          LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP, DEFAULT_LOCK_ACQUIRE_MAX_RETRY_WAIT_TIME_IN_MILLIS);
-      setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP),
-          LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP, DEFAULT_LOCK_ACQUIRE_CLIENT_NUM_RETRIES);
-      setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP),
-          LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP, DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS);
-      setDefaultOnCondition(props, !props.containsKey(ZK_CONNECTION_TIMEOUT_MS_PROP),
-          ZK_CONNECTION_TIMEOUT_MS_PROP, String.valueOf(DEFAULT_ZK_CONNECTION_TIMEOUT_MS));
-      setDefaultOnCondition(props, !props.containsKey(ZK_SESSION_TIMEOUT_MS_PROP),
-          ZK_SESSION_TIMEOUT_MS_PROP, String.valueOf(DEFAULT_ZK_SESSION_TIMEOUT_MS));
-      setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP),
-          LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP, String.valueOf(DEFAULT_ACQUIRE_LOCK_WAIT_TIMEOUT_MS));
-      return config;
+      lockConfig.setDefaults(HoodieLockConfig.class.getName());
+      return lockConfig;
     }
   }
 
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 687033f..fb4cb04 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
@@ -18,7 +18,8 @@
 
 package org.apache.hudi.config;
 
-import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 
 import javax.annotation.concurrent.Immutable;
 
@@ -31,39 +32,59 @@ import java.util.Properties;
  * Memory related config.
  */
 @Immutable
-public class HoodieMemoryConfig extends DefaultHoodieConfig {
+public class HoodieMemoryConfig extends HoodieConfig {
 
-  // This fraction is multiplied with the spark.memory.fraction to get a final fraction of heap space to use
-  // during merge. This makes it easier to scale this value as one increases the spark.executor.memory
-  public static final String MAX_MEMORY_FRACTION_FOR_MERGE_PROP = "hoodie.memory.merge.fraction";
   // Default max memory fraction during hash-merge, excess spills to disk
-  public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE = String.valueOf(0.6);
-  public static final String MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP = "hoodie.memory.compaction.fraction";
+  public static final ConfigProperty<String> MAX_MEMORY_FRACTION_FOR_MERGE_PROP = ConfigProperty
+      .key("hoodie.memory.merge.fraction")
+      .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");
+
   // Default max memory fraction during compaction, excess spills to disk
-  public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION = String.valueOf(0.6);
+  public static final ConfigProperty<String> MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP = ConfigProperty
+      .key("hoodie.memory.compaction.fraction")
+      .defaultValue(String.valueOf(0.6))
+      .withDocumentation("HoodieCompactedLogScanner reads logblocks, converts records to HoodieRecords and then "
+          + "merges these log blocks and records. At any point, the number of entries in a log block can be "
+          + "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");
+
   // 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;
   // Minimum memory size (100MB) for the spillable map.
   public static final long DEFAULT_MIN_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES = 100 * 1024 * 1024L;
-  // Property to set the max memory for merge
-  public static final String MAX_MEMORY_FOR_MERGE_PROP = "hoodie.memory.merge.max.size";
-  // Property to set the max memory for compaction
-  public static final String MAX_MEMORY_FOR_COMPACTION_PROP = "hoodie.memory.compaction.max.size";
-  // Property to set the max memory for dfs inputstream buffer size
-  public static final String MAX_DFS_STREAM_BUFFER_SIZE_PROP = "hoodie.memory.dfs.buffer.max.size";
-  public static final int DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE = 16 * 1024 * 1024; // 16MB
-  public static final String SPILLABLE_MAP_BASE_PATH_PROP = "hoodie.memory.spillable.map.path";
-  // Default file path prefix for spillable file
-  public static final String DEFAULT_SPILLABLE_MAP_BASE_PATH = "/tmp/";
-
-  // Property to control how what fraction of the failed record, exceptions we report back to driver.
-  public static final String WRITESTATUS_FAILURE_FRACTION_PROP = "hoodie.memory.writestatus.failure.fraction";
-  // Default is 10%. If set to 100%, with lot of failures, this can cause memory pressure, cause OOMs and
-  // mask actual data errors.
-  public static final double DEFAULT_WRITESTATUS_FAILURE_FRACTION = 0.1;
-
-  private HoodieMemoryConfig(Properties props) {
-    super(props);
+
+  public static final ConfigProperty<Long> MAX_MEMORY_FOR_MERGE_PROP = ConfigProperty
+      .key("hoodie.memory.merge.max.size")
+      .defaultValue(DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES)
+      .withDocumentation("Property to set the max memory for merge");
+
+  public static final ConfigProperty<String> MAX_MEMORY_FOR_COMPACTION_PROP = ConfigProperty
+      .key("hoodie.memory.compaction.max.size")
+      .noDefaultValue()
+      .withDocumentation("Property to set the max memory for compaction");
+
+  public static final ConfigProperty<Integer> MAX_DFS_STREAM_BUFFER_SIZE_PROP = ConfigProperty
+      .key("hoodie.memory.dfs.buffer.max.size")
+      .defaultValue(16 * 1024 * 1024)
+      .withDocumentation("Property to set the max memory for dfs inputstream buffer size");
+
+  public static final ConfigProperty<String> SPILLABLE_MAP_BASE_PATH_PROP = ConfigProperty
+      .key("hoodie.memory.spillable.map.path")
+      .defaultValue("/tmp/")
+      .withDocumentation("Default file path prefix for spillable file");
+
+  public static final ConfigProperty<Double> WRITESTATUS_FAILURE_FRACTION_PROP = ConfigProperty
+      .key("hoodie.memory.writestatus.failure.fraction")
+      .defaultValue(0.1)
+      .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.");
+
+  private HoodieMemoryConfig() {
+    super();
   }
 
   public static HoodieMemoryConfig.Builder newBuilder() {
@@ -72,57 +93,49 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig {
 
   public static class Builder {
 
-    private final Properties props = new Properties();
+    private final HoodieMemoryConfig memoryConfig = new HoodieMemoryConfig();
 
     public Builder fromFile(File propertiesFile) throws IOException {
       try (FileReader reader = new FileReader(propertiesFile)) {
-        this.props.load(reader);
+        this.memoryConfig.getProps().load(reader);
         return this;
       }
     }
 
     public Builder fromProperties(Properties props) {
-      this.props.putAll(props);
+      this.memoryConfig.getProps().putAll(props);
       return this;
     }
 
     public Builder withMaxMemoryFractionPerPartitionMerge(double maxMemoryFractionPerPartitionMerge) {
-      props.setProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, String.valueOf(maxMemoryFractionPerPartitionMerge));
+      memoryConfig.setValue(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, String.valueOf(maxMemoryFractionPerPartitionMerge));
       return this;
     }
 
     public Builder withMaxMemoryMaxSize(long mergeMaxSize, long compactionMaxSize) {
-      props.setProperty(MAX_MEMORY_FOR_MERGE_PROP, String.valueOf(mergeMaxSize));
-      props.setProperty(MAX_MEMORY_FOR_COMPACTION_PROP, String.valueOf(compactionMaxSize));
+      memoryConfig.setValue(MAX_MEMORY_FOR_MERGE_PROP, String.valueOf(mergeMaxSize));
+      memoryConfig.setValue(MAX_MEMORY_FOR_COMPACTION_PROP, String.valueOf(compactionMaxSize));
       return this;
     }
 
     public Builder withMaxMemoryFractionPerCompaction(double maxMemoryFractionPerCompaction) {
-      props.setProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, String.valueOf(maxMemoryFractionPerCompaction));
+      memoryConfig.setValue(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, String.valueOf(maxMemoryFractionPerCompaction));
       return this;
     }
 
     public Builder withMaxDFSStreamBufferSize(int maxStreamBufferSize) {
-      props.setProperty(MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(maxStreamBufferSize));
+      memoryConfig.setValue(MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(maxStreamBufferSize));
       return this;
     }
 
     public Builder withWriteStatusFailureFraction(double failureFraction) {
-      props.setProperty(WRITESTATUS_FAILURE_FRACTION_PROP, String.valueOf(failureFraction));
+      memoryConfig.setValue(WRITESTATUS_FAILURE_FRACTION_PROP, String.valueOf(failureFraction));
       return this;
     }
 
     public HoodieMemoryConfig build() {
-      HoodieMemoryConfig config = new HoodieMemoryConfig(props);
-      setDefaultOnCondition(props, !props.containsKey(MAX_DFS_STREAM_BUFFER_SIZE_PROP), MAX_DFS_STREAM_BUFFER_SIZE_PROP,
-          String.valueOf(DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE));
-      setDefaultOnCondition(props, !props.containsKey(SPILLABLE_MAP_BASE_PATH_PROP), SPILLABLE_MAP_BASE_PATH_PROP,
-          DEFAULT_SPILLABLE_MAP_BASE_PATH);
-      setDefaultOnCondition(props, !props.containsKey(MAX_MEMORY_FOR_MERGE_PROP), MAX_MEMORY_FOR_MERGE_PROP,
-          String.valueOf(DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES));
-      setDefaultOnCondition(props, !props.containsKey(WRITESTATUS_FAILURE_FRACTION_PROP),
-          WRITESTATUS_FAILURE_FRACTION_PROP, String.valueOf(DEFAULT_WRITESTATUS_FAILURE_FRACTION));
-      return config;
+      memoryConfig.setDefaults(HoodieMemoryConfig.class.getName());
+      return memoryConfig;
     }
   }
 }
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 b6cb6e5..599ae30 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
@@ -18,7 +18,8 @@
 
 package org.apache.hudi.config;
 
-import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.metrics.MetricsReporterType;
 
 import javax.annotation.concurrent.Immutable;
@@ -32,41 +33,74 @@ import java.util.Properties;
  * Fetch the configurations used by the Metrics system.
  */
 @Immutable
-public class HoodieMetricsConfig extends DefaultHoodieConfig {
+public class HoodieMetricsConfig extends HoodieConfig {
 
   public static final String METRIC_PREFIX = "hoodie.metrics";
-  public static final String METRICS_ON = METRIC_PREFIX + ".on";
-  public static final boolean DEFAULT_METRICS_ON = false;
-  public static final String METRICS_REPORTER_TYPE = METRIC_PREFIX + ".reporter.type";
-  public static final MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE = MetricsReporterType.GRAPHITE;
+
+  public static final ConfigProperty<Boolean> 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
+      .key(METRIC_PREFIX + ".reporter.type")
+      .defaultValue(MetricsReporterType.GRAPHITE)
+      .sinceVersion("0.5.0")
+      .withDocumentation("Type of metrics reporter.");
 
   // Graphite
   public static final String GRAPHITE_PREFIX = METRIC_PREFIX + ".graphite";
-  public static final String GRAPHITE_SERVER_HOST = GRAPHITE_PREFIX + ".host";
-  public static final String DEFAULT_GRAPHITE_SERVER_HOST = "localhost";
 
-  public static final String GRAPHITE_SERVER_PORT = GRAPHITE_PREFIX + ".port";
-  public static final int DEFAULT_GRAPHITE_SERVER_PORT = 4756;
+  public static final ConfigProperty<String> GRAPHITE_SERVER_HOST = 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
+      .key(GRAPHITE_PREFIX + ".port")
+      .defaultValue(4756)
+      .sinceVersion("0.5.0")
+      .withDocumentation("Graphite port to connect to");
 
   // Jmx
   public static final String JMX_PREFIX = METRIC_PREFIX + ".jmx";
-  public static final String JMX_HOST = JMX_PREFIX + ".host";
-  public static final String DEFAULT_JMX_HOST = "localhost";
 
-  public static final String JMX_PORT = JMX_PREFIX + ".port";
-  public static final int DEFAULT_JMX_PORT = 9889;
+  public static final ConfigProperty<String> JMX_HOST = 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
+      .key(JMX_PREFIX + ".port")
+      .defaultValue(9889)
+      .sinceVersion("0.5.1")
+      .withDocumentation("Jmx port to connect to");
 
-  public static final String GRAPHITE_METRIC_PREFIX = GRAPHITE_PREFIX + ".metric.prefix";
+  public static final ConfigProperty<String> GRAPHITE_METRIC_PREFIX = 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 String METRICS_REPORTER_CLASS = METRIC_PREFIX + ".reporter.class";
-  public static final String DEFAULT_METRICS_REPORTER_CLASS = "";
+  public static final ConfigProperty<String> METRICS_REPORTER_CLASS = ConfigProperty
+      .key(METRIC_PREFIX + ".reporter.class")
+      .defaultValue("")
+      .sinceVersion("0.6.0")
+      .withDocumentation("");
 
   // Enable metrics collection from executors
-  public static final String ENABLE_EXECUTOR_METRICS = METRIC_PREFIX + ".executor.enable";
-
-  private HoodieMetricsConfig(Properties props) {
-    super(props);
+  public static final ConfigProperty<String> ENABLE_EXECUTOR_METRICS = ConfigProperty
+      .key(METRIC_PREFIX + ".executor.enable")
+      .noDefaultValue()
+      .sinceVersion("0.7.0")
+      .withDocumentation("");
+
+  private HoodieMetricsConfig() {
+    super();
   }
 
   public static HoodieMetricsConfig.Builder newBuilder() {
@@ -75,89 +109,78 @@ public class HoodieMetricsConfig extends DefaultHoodieConfig {
 
   public static class Builder {
 
-    private final Properties props = new Properties();
+    private final HoodieMetricsConfig hoodieMetricsConfig = new HoodieMetricsConfig();
 
     public Builder fromFile(File propertiesFile) throws IOException {
       try (FileReader reader = new FileReader(propertiesFile)) {
-        this.props.load(reader);
+        this.hoodieMetricsConfig.getProps().load(reader);
         return this;
       }
     }
 
     public Builder fromProperties(Properties props) {
-      this.props.putAll(props);
+      this.hoodieMetricsConfig.getProps().putAll(props);
       return this;
     }
 
     public Builder on(boolean metricsOn) {
-      props.setProperty(METRICS_ON, String.valueOf(metricsOn));
+      hoodieMetricsConfig.setValue(METRICS_ON, String.valueOf(metricsOn));
       return this;
     }
 
     public Builder withReporterType(String reporterType) {
-      props.setProperty(METRICS_REPORTER_TYPE, reporterType);
+      hoodieMetricsConfig.setValue(METRICS_REPORTER_TYPE, reporterType);
       return this;
     }
 
     public Builder toGraphiteHost(String host) {
-      props.setProperty(GRAPHITE_SERVER_HOST, host);
+      hoodieMetricsConfig.setValue(GRAPHITE_SERVER_HOST, host);
       return this;
     }
 
     public Builder onGraphitePort(int port) {
-      props.setProperty(GRAPHITE_SERVER_PORT, String.valueOf(port));
+      hoodieMetricsConfig.setValue(GRAPHITE_SERVER_PORT, String.valueOf(port));
       return this;
     }
 
     public Builder toJmxHost(String host) {
-      props.setProperty(JMX_HOST, host);
+      hoodieMetricsConfig.setValue(JMX_HOST, host);
       return this;
     }
 
     public Builder onJmxPort(String port) {
-      props.setProperty(JMX_PORT, port);
+      hoodieMetricsConfig.setValue(JMX_PORT, port);
       return this;
     }
 
     public Builder usePrefix(String prefix) {
-      props.setProperty(GRAPHITE_METRIC_PREFIX, prefix);
+      hoodieMetricsConfig.setValue(GRAPHITE_METRIC_PREFIX, prefix);
       return this;
     }
 
     public Builder withReporterClass(String className) {
-      props.setProperty(METRICS_REPORTER_CLASS, className);
+      hoodieMetricsConfig.setValue(METRICS_REPORTER_CLASS, className);
       return this;
     }
 
     public Builder withExecutorMetrics(boolean enable) {
-      props.setProperty(ENABLE_EXECUTOR_METRICS, String.valueOf(enable));
+      hoodieMetricsConfig.setValue(ENABLE_EXECUTOR_METRICS, String.valueOf(enable));
       return this;
     }
 
     public HoodieMetricsConfig build() {
-      HoodieMetricsConfig config = new HoodieMetricsConfig(props);
-      setDefaultOnCondition(props, !props.containsKey(METRICS_ON), METRICS_ON, String.valueOf(DEFAULT_METRICS_ON));
-      setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_TYPE), METRICS_REPORTER_TYPE,
-          DEFAULT_METRICS_REPORTER_TYPE.name());
-      setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_HOST), GRAPHITE_SERVER_HOST,
-          DEFAULT_GRAPHITE_SERVER_HOST);
-      setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT), GRAPHITE_SERVER_PORT,
-          String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
-      setDefaultOnCondition(props, !props.containsKey(JMX_HOST), JMX_HOST,
-          DEFAULT_JMX_HOST);
-      setDefaultOnCondition(props, !props.containsKey(JMX_PORT), JMX_PORT,
-          String.valueOf(DEFAULT_JMX_PORT));
-      MetricsReporterType reporterType = MetricsReporterType.valueOf(props.getProperty(METRICS_REPORTER_TYPE));
-      setDefaultOnCondition(props, reporterType == MetricsReporterType.DATADOG,
-          HoodieMetricsDatadogConfig.newBuilder().fromProperties(props).build());
-      setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_CLASS),
-              METRICS_REPORTER_CLASS, DEFAULT_METRICS_REPORTER_CLASS);
-      setDefaultOnCondition(props, reporterType == MetricsReporterType.PROMETHEUS_PUSHGATEWAY,
-              HoodieMetricsPrometheusConfig.newBuilder().fromProperties(props).build());
-      setDefaultOnCondition(props, reporterType == MetricsReporterType.PROMETHEUS,
-              HoodieMetricsPrometheusConfig.newBuilder().fromProperties(props).build());
-
-      return config;
+
+      hoodieMetricsConfig.setDefaults(HoodieMetricsConfig.class.getName());
+
+      MetricsReporterType reporterType = MetricsReporterType.valueOf(hoodieMetricsConfig.getString(METRICS_REPORTER_TYPE));
+
+      hoodieMetricsConfig.setDefaultOnCondition(reporterType == MetricsReporterType.DATADOG,
+          HoodieMetricsDatadogConfig.newBuilder().fromProperties(hoodieMetricsConfig.getProps()).build());
+      hoodieMetricsConfig.setDefaultOnCondition(reporterType == MetricsReporterType.PROMETHEUS_PUSHGATEWAY,
+              HoodieMetricsPrometheusConfig.newBuilder().fromProperties(hoodieMetricsConfig.getProps()).build());
+      hoodieMetricsConfig.setDefaultOnCondition(reporterType == MetricsReporterType.PROMETHEUS,
+              HoodieMetricsPrometheusConfig.newBuilder().fromProperties(hoodieMetricsConfig.getProps()).build());
+      return hoodieMetricsConfig;
     }
   }
 
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 e6dcc28..2be028d 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
@@ -18,7 +18,8 @@
 
 package org.apache.hudi.config;
 
-import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 
 import javax.annotation.concurrent.Immutable;
 
@@ -32,24 +33,69 @@ import static org.apache.hudi.config.HoodieMetricsConfig.METRIC_PREFIX;
  * {@link org.apache.hudi.metrics.MetricsReporterType#DATADOG}
  */
 @Immutable
-public class HoodieMetricsDatadogConfig extends DefaultHoodieConfig {
+public class HoodieMetricsDatadogConfig extends HoodieConfig {
 
   public static final String DATADOG_PREFIX = METRIC_PREFIX + ".datadog";
-  public static final String DATADOG_REPORT_PERIOD_SECONDS = DATADOG_PREFIX + ".report.period.seconds";
-  public static final int DEFAULT_DATADOG_REPORT_PERIOD_SECONDS = 30;
-  public static final String DATADOG_API_SITE = DATADOG_PREFIX + ".api.site";
-  public static final String DATADOG_API_KEY = DATADOG_PREFIX + ".api.key";
-  public static final String DATADOG_API_KEY_SKIP_VALIDATION = DATADOG_PREFIX + ".api.key.skip.validation";
-  public static final boolean DEFAULT_DATADOG_API_KEY_SKIP_VALIDATION = false;
-  public static final String DATADOG_API_KEY_SUPPLIER = DATADOG_PREFIX + ".api.key.supplier";
-  public static final String DATADOG_API_TIMEOUT_SECONDS = DATADOG_PREFIX + ".api.timeout.seconds";
-  public static final int DEFAULT_DATADOG_API_TIMEOUT_SECONDS = 3;
-  public static final String DATADOG_METRIC_PREFIX = DATADOG_PREFIX + ".metric.prefix";
-  public static final String DATADOG_METRIC_HOST = DATADOG_PREFIX + ".metric.host";
-  public static final String DATADOG_METRIC_TAGS = DATADOG_PREFIX + ".metric.tags";
-
-  private HoodieMetricsDatadogConfig(Properties props) {
-    super(props);
+
+  public static final ConfigProperty<Integer> DATADOG_REPORT_PERIOD_SECONDS = ConfigProperty
+      .key(DATADOG_PREFIX + ".report.period.seconds")
+      .defaultValue(30)
+      .sinceVersion("0.6.0")
+      .withDocumentation("Datadog report period in seconds. Default to 30.");
+
+  public static final ConfigProperty<String> DATADOG_API_SITE = 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
+      .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
+      .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
+      .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
+      .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
+      .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
+      .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
+      .key(DATADOG_PREFIX + ".metric.tags")
+      .noDefaultValue()
+      .sinceVersion("0.6.0")
+      .withDocumentation("Datadog metric tags (comma-delimited) to be sent along with metrics data.");
+
+  private HoodieMetricsDatadogConfig() {
+    super();
   }
 
   public static HoodieMetricsDatadogConfig.Builder newBuilder() {
@@ -58,70 +104,61 @@ public class HoodieMetricsDatadogConfig extends DefaultHoodieConfig {
 
   public static class Builder {
 
-    private final Properties props = new Properties();
+    private final HoodieMetricsDatadogConfig metricsDatadogConfig = new HoodieMetricsDatadogConfig();
 
     public Builder fromProperties(Properties props) {
-      this.props.putAll(props);
+      this.metricsDatadogConfig.getProps().putAll(props);
       return this;
     }
 
     public Builder withDatadogReportPeriodSeconds(int period) {
-      props.setProperty(DATADOG_REPORT_PERIOD_SECONDS, String.valueOf(period));
+      metricsDatadogConfig.setValue(DATADOG_REPORT_PERIOD_SECONDS, String.valueOf(period));
       return this;
     }
 
     public Builder withDatadogApiSite(String apiSite) {
-      props.setProperty(DATADOG_API_SITE, apiSite);
+      metricsDatadogConfig.setValue(DATADOG_API_SITE, apiSite);
       return this;
     }
 
     public Builder withDatadogApiKey(String apiKey) {
-      props.setProperty(DATADOG_API_KEY, apiKey);
+      metricsDatadogConfig.setValue(DATADOG_API_KEY, apiKey);
       return this;
     }
 
     public Builder withDatadogApiKeySkipValidation(boolean skip) {
-      props.setProperty(DATADOG_API_KEY_SKIP_VALIDATION, String.valueOf(skip));
+      metricsDatadogConfig.setValue(DATADOG_API_KEY_SKIP_VALIDATION, String.valueOf(skip));
       return this;
     }
 
     public Builder withDatadogApiKeySupplier(String apiKeySupplier) {
-      props.setProperty(DATADOG_API_KEY_SUPPLIER, apiKeySupplier);
+      metricsDatadogConfig.setValue(DATADOG_API_KEY_SUPPLIER, apiKeySupplier);
       return this;
     }
 
     public Builder withDatadogApiTimeoutSeconds(int timeout) {
-      props.setProperty(DATADOG_API_TIMEOUT_SECONDS, String.valueOf(timeout));
+      metricsDatadogConfig.setValue(DATADOG_API_TIMEOUT_SECONDS, String.valueOf(timeout));
       return this;
     }
 
     public Builder withDatadogPrefix(String prefix) {
-      props.setProperty(DATADOG_METRIC_PREFIX, prefix);
+      metricsDatadogConfig.setValue(DATADOG_METRIC_PREFIX, prefix);
       return this;
     }
 
     public Builder withDatadogHost(String host) {
-      props.setProperty(DATADOG_METRIC_HOST, host);
+      metricsDatadogConfig.setValue(DATADOG_METRIC_HOST, host);
       return this;
     }
 
     public Builder withDatadogTags(String tags) {
-      props.setProperty(DATADOG_METRIC_TAGS, tags);
+      metricsDatadogConfig.setValue(DATADOG_METRIC_TAGS, tags);
       return this;
     }
 
     public HoodieMetricsDatadogConfig build() {
-      HoodieMetricsDatadogConfig config = new HoodieMetricsDatadogConfig(props);
-      setDefaultOnCondition(props, !props.containsKey(DATADOG_REPORT_PERIOD_SECONDS),
-          DATADOG_REPORT_PERIOD_SECONDS,
-          String.valueOf(DEFAULT_DATADOG_REPORT_PERIOD_SECONDS));
-      setDefaultOnCondition(props, !props.containsKey(DATADOG_API_KEY_SKIP_VALIDATION),
-          DATADOG_API_KEY_SKIP_VALIDATION,
-          String.valueOf(DEFAULT_DATADOG_API_KEY_SKIP_VALIDATION));
-      setDefaultOnCondition(props, !props.containsKey(DATADOG_API_TIMEOUT_SECONDS),
-          DATADOG_API_TIMEOUT_SECONDS,
-          String.valueOf(DEFAULT_DATADOG_API_TIMEOUT_SECONDS));
-      return config;
+      metricsDatadogConfig.setDefaults(HoodieMetricsDatadogConfig.class.getName());
+      return metricsDatadogConfig;
     }
   }
 }
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 3e2d50f..bc33b39 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
@@ -18,43 +18,65 @@
 
 package org.apache.hudi.config;
 
-import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 
 import java.util.Properties;
 
 import static org.apache.hudi.config.HoodieMetricsConfig.METRIC_PREFIX;
 
-public class HoodieMetricsPrometheusConfig extends DefaultHoodieConfig {
+public class HoodieMetricsPrometheusConfig extends HoodieConfig {
 
   // Prometheus PushGateWay
   public static final String PUSHGATEWAY_PREFIX = METRIC_PREFIX + ".pushgateway";
 
-  public static final String PUSHGATEWAY_HOST = PUSHGATEWAY_PREFIX + ".host";
-  public static final String DEFAULT_PUSHGATEWAY_HOST = "localhost";
-
-  public static final String PUSHGATEWAY_PORT = PUSHGATEWAY_PREFIX + ".port";
-  public static final int DEFAULT_PUSHGATEWAY_PORT = 9091;
-
-  public static final String PUSHGATEWAY_REPORT_PERIOD_SECONDS = PUSHGATEWAY_PREFIX + ".report.period.seconds";
-  public static final int DEFAULT_PUSHGATEWAY_REPORT_PERIOD_SECONDS = 30;
-
-  public static final String PUSHGATEWAY_DELETE_ON_SHUTDOWN = PUSHGATEWAY_PREFIX + ".delete.on.shutdown";
-  public static final boolean DEFAULT_PUSHGATEWAY_DELETE_ON_SHUTDOWN = true;
-
-  public static final String PUSHGATEWAY_JOB_NAME = PUSHGATEWAY_PREFIX + ".job.name";
-  public static final String DEFAULT_PUSHGATEWAY_JOB_NAME = "";
-
-  public static final String PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX = PUSHGATEWAY_PREFIX + ".random.job.name.suffix";
-  public static final boolean DEFAULT_PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX = true;
-
+  public static final ConfigProperty<String> PUSHGATEWAY_HOST = ConfigProperty
+      .key(PUSHGATEWAY_PREFIX + ".host")
+      .defaultValue("localhost")
+      .sinceVersion("0.6.0")
+      .withDocumentation("");
+
+  public static final ConfigProperty<Integer> PUSHGATEWAY_PORT = ConfigProperty
+      .key(PUSHGATEWAY_PREFIX + ".port")
+      .defaultValue(9091)
+      .sinceVersion("0.6.0")
+      .withDocumentation("");
+
+  public static final ConfigProperty<Integer> PUSHGATEWAY_REPORT_PERIOD_SECONDS = ConfigProperty
+      .key(PUSHGATEWAY_PREFIX + ".report.period.seconds")
+      .defaultValue(30)
+      .sinceVersion("0.6.0")
+      .withDocumentation("");
+
+  public static final ConfigProperty<Boolean> PUSHGATEWAY_DELETE_ON_SHUTDOWN = ConfigProperty
+      .key(PUSHGATEWAY_PREFIX + ".delete.on.shutdown")
+      .defaultValue(true)
+      .sinceVersion("0.6.0")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> PUSHGATEWAY_JOB_NAME = ConfigProperty
+      .key(PUSHGATEWAY_PREFIX + ".job.name")
+      .defaultValue("")
+      .sinceVersion("0.6.0")
+      .withDocumentation("");
+
+  public static final ConfigProperty<Boolean> PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX = ConfigProperty
+      .key(PUSHGATEWAY_PREFIX + ".random.job.name.suffix")
+      .defaultValue(true)
+      .sinceVersion("0.6.0")
+      .withDocumentation("");
 
   // Prometheus HttpServer
   public static final String PROMETHEUS_PREFIX = METRIC_PREFIX + ".prometheus";
-  public static final String PROMETHEUS_PORT = PROMETHEUS_PREFIX + ".port";
-  public static final int DEFAULT_PROMETHEUS_PORT = 9090;
 
-  public HoodieMetricsPrometheusConfig(Properties props) {
-    super(props);
+  public static final ConfigProperty<Integer> PROMETHEUS_PORT = ConfigProperty
+      .key(PROMETHEUS_PREFIX + ".port")
+      .defaultValue(9090)
+      .sinceVersion("0.6.0")
+      .withDocumentation("");
+
+  private HoodieMetricsPrometheusConfig() {
+    super();
   }
 
   public static HoodieMetricsPrometheusConfig.Builder newBuilder() {
@@ -68,35 +90,16 @@ public class HoodieMetricsPrometheusConfig extends DefaultHoodieConfig {
 
   public static class Builder {
 
-    private Properties props = new Properties();
+    private HoodieMetricsPrometheusConfig hoodieMetricsPrometheusConfig = new HoodieMetricsPrometheusConfig();
 
     public Builder fromProperties(Properties props) {
-      this.props.putAll(props);
+      this.hoodieMetricsPrometheusConfig.getProps().putAll(props);
       return this;
     }
 
     public HoodieMetricsPrometheusConfig build() {
-      HoodieMetricsPrometheusConfig config = new HoodieMetricsPrometheusConfig(props);
-      setDefaultOnCondition(props, !props.containsKey(PROMETHEUS_PORT), PROMETHEUS_PORT,
-              String.valueOf(DEFAULT_PROMETHEUS_PORT));
-      setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_HOST),
-              PUSHGATEWAY_HOST,
-              DEFAULT_PUSHGATEWAY_HOST);
-      setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_PORT),
-              PUSHGATEWAY_PORT,
-              String.valueOf(DEFAULT_PUSHGATEWAY_PORT));
-      setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_REPORT_PERIOD_SECONDS),
-              PUSHGATEWAY_REPORT_PERIOD_SECONDS,
-              String.valueOf(DEFAULT_PUSHGATEWAY_REPORT_PERIOD_SECONDS));
-      setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_DELETE_ON_SHUTDOWN),
-              PUSHGATEWAY_DELETE_ON_SHUTDOWN,
-              String.valueOf(DEFAULT_PUSHGATEWAY_DELETE_ON_SHUTDOWN));
-      setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_JOB_NAME),
-              PUSHGATEWAY_JOB_NAME, DEFAULT_PUSHGATEWAY_JOB_NAME);
-      setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX),
-              PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX,
-              String.valueOf(DEFAULT_PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX));
-      return config;
+      hoodieMetricsPrometheusConfig.setDefaults(HoodieMetricsPrometheusConfig.class.getName());
+      return hoodieMetricsPrometheusConfig;
     }
   }
 }
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 489d23c..086c338 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
@@ -18,25 +18,34 @@
 
 package org.apache.hudi.config;
 
-import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 
 import java.io.File;
 import java.io.FileReader;
 import java.io.IOException;
 import java.util.Properties;
 
-import static org.apache.hudi.common.model.HoodiePayloadProps.DEFAULT_PAYLOAD_EVENT_TIME_FIELD_VAL;
-import static org.apache.hudi.common.model.HoodiePayloadProps.DEFAULT_PAYLOAD_ORDERING_FIELD_VAL;
-import static org.apache.hudi.common.model.HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP;
-import static org.apache.hudi.common.model.HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP;
+import static org.apache.hudi.common.model.HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP_KEY;
+import static org.apache.hudi.common.model.HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY;
 
 /**
  * Hoodie payload related configs.
  */
-public class HoodiePayloadConfig extends DefaultHoodieConfig {
+public class HoodiePayloadConfig extends HoodieConfig {
 
-  public HoodiePayloadConfig(Properties props) {
-    super(props);
+  public static final ConfigProperty<String> PAYLOAD_ORDERING_FIELD_PROP = ConfigProperty
+      .key(PAYLOAD_ORDERING_FIELD_PROP_KEY)
+      .defaultValue("ts")
+      .withDocumentation("Property to hold the payload ordering field name");
+
+  public static final ConfigProperty<String> PAYLOAD_EVENT_TIME_FIELD_PROP = ConfigProperty
+      .key(PAYLOAD_EVENT_TIME_FIELD_PROP_KEY)
+      .defaultValue("ts")
+      .withDocumentation("Property for payload event time field");
+
+  private HoodiePayloadConfig() {
+    super();
   }
 
   public static HoodiePayloadConfig.Builder newBuilder() {
@@ -45,37 +54,33 @@ public class HoodiePayloadConfig extends DefaultHoodieConfig {
 
   public static class Builder {
 
-    private final Properties props = new Properties();
+    private final HoodiePayloadConfig payloadConfig = new HoodiePayloadConfig();
 
     public Builder fromFile(File propertiesFile) throws IOException {
       try (FileReader reader = new FileReader(propertiesFile)) {
-        this.props.load(reader);
+        this.payloadConfig.getProps().load(reader);
         return this;
       }
     }
 
     public Builder fromProperties(Properties props) {
-      this.props.putAll(props);
+      this.payloadConfig.getProps().putAll(props);
       return this;
     }
 
     public Builder withPayloadOrderingField(String payloadOrderingField) {
-      props.setProperty(PAYLOAD_ORDERING_FIELD_PROP, String.valueOf(payloadOrderingField));
+      payloadConfig.setValue(PAYLOAD_ORDERING_FIELD_PROP, String.valueOf(payloadOrderingField));
       return this;
     }
 
     public Builder withPayloadEventTimeField(String payloadEventTimeField) {
-      props.setProperty(PAYLOAD_EVENT_TIME_FIELD_PROP, String.valueOf(payloadEventTimeField));
+      payloadConfig.setValue(PAYLOAD_EVENT_TIME_FIELD_PROP, String.valueOf(payloadEventTimeField));
       return this;
     }
 
     public HoodiePayloadConfig build() {
-      HoodiePayloadConfig config = new HoodiePayloadConfig(props);
-      setDefaultOnCondition(props, !props.containsKey(PAYLOAD_ORDERING_FIELD_PROP), PAYLOAD_ORDERING_FIELD_PROP,
-          String.valueOf(DEFAULT_PAYLOAD_ORDERING_FIELD_VAL));
-      setDefaultOnCondition(props, !props.containsKey(PAYLOAD_EVENT_TIME_FIELD_PROP), PAYLOAD_EVENT_TIME_FIELD_PROP,
-              String.valueOf(DEFAULT_PAYLOAD_EVENT_TIME_FIELD_VAL));
-      return config;
+      payloadConfig.setDefaults(HoodiePayloadConfig.class.getName());
+      return payloadConfig;
     }
   }
 
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 3cd8817..9aff8c9 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
@@ -18,7 +18,8 @@
 
 package org.apache.hudi.config;
 
-import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 
 import javax.annotation.concurrent.Immutable;
 
@@ -31,51 +32,97 @@ import java.util.Properties;
  * Storage related config.
  */
 @Immutable
-public class HoodieStorageConfig extends DefaultHoodieConfig {
-
-  public static final String PARQUET_FILE_MAX_BYTES = "hoodie.parquet.max.file.size";
-  public static final String DEFAULT_PARQUET_FILE_MAX_BYTES = String.valueOf(120 * 1024 * 1024);
-  public static final String PARQUET_BLOCK_SIZE_BYTES = "hoodie.parquet.block.size";
-  public static final String DEFAULT_PARQUET_BLOCK_SIZE_BYTES = DEFAULT_PARQUET_FILE_MAX_BYTES;
-  public static final String PARQUET_PAGE_SIZE_BYTES = "hoodie.parquet.page.size";
-  public static final String DEFAULT_PARQUET_PAGE_SIZE_BYTES = String.valueOf(1 * 1024 * 1024);
-
-  public static final String HFILE_FILE_MAX_BYTES = "hoodie.hfile.max.file.size";
-  public static final String HFILE_BLOCK_SIZE_BYTES = "hoodie.hfile.block.size";
-  public static final String DEFAULT_HFILE_BLOCK_SIZE_BYTES = String.valueOf(1 * 1024 * 1024);
-  public static final String DEFAULT_HFILE_FILE_MAX_BYTES = String.valueOf(120 * 1024 * 1024);
-
-  public static final String ORC_FILE_MAX_BYTES = "hoodie.orc.max.file.size";
-  public static final String DEFAULT_ORC_FILE_MAX_BYTES = String.valueOf(120 * 1024 * 1024);
-  // size of the memory buffer in bytes for writing
-  public static final String ORC_STRIPE_SIZE = "hoodie.orc.stripe.size";
-  public static final String DEFAULT_ORC_STRIPE_SIZE = String.valueOf(64 * 1024 * 1024);
-  // file system block size
-  public static final String ORC_BLOCK_SIZE = "hoodie.orc.block.size";
-  public static final String DEFAULT_ORC_BLOCK_SIZE = DEFAULT_ORC_FILE_MAX_BYTES;
+public class HoodieStorageConfig extends HoodieConfig {
+
+  public static final ConfigProperty<String> PARQUET_FILE_MAX_BYTES = 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
+      .key("hoodie.parquet.block.size")
+      .defaultValue(String.valueOf(120 * 1024 * 1024))
+      .withDocumentation("Parquet RowGroup size. Its better this is same as the file size, so that a single column "
+          + "within a file is stored continuously on disk");
+
+  public static final ConfigProperty<String> PARQUET_PAGE_SIZE_BYTES = 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 seperately.");
+
+  public static final ConfigProperty<String> ORC_FILE_MAX_BYTES = ConfigProperty
+      .key("hoodie.orc.max.file.size")
+      .defaultValue(String.valueOf(120 * 1024 * 1024))
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> ORC_STRIPE_SIZE = ConfigProperty
+      .key("hoodie.orc.stripe.size")
+      .defaultValue(String.valueOf(64 * 1024 * 1024))
+      .withDocumentation("Size of the memory buffer in bytes for writing");
+
+  public static final ConfigProperty<String> ORC_BLOCK_SIZE = ConfigProperty
+      .key("hoodie.orc.block.size")
+      .defaultValue(ORC_FILE_MAX_BYTES.defaultValue())
+      .withDocumentation("File system block size");
+
+  public static final ConfigProperty<String> HFILE_FILE_MAX_BYTES = ConfigProperty
+      .key("hoodie.hfile.max.file.size")
+      .defaultValue(String.valueOf(120 * 1024 * 1024))
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HFILE_BLOCK_SIZE_BYTES = ConfigProperty
+      .key("hoodie.hfile.block.size")
+      .defaultValue(String.valueOf(1 * 1024 * 1024))
+      .withDocumentation("");
 
   // used to size log files
-  public static final String LOGFILE_SIZE_MAX_BYTES = "hoodie.logfile.max.size";
-  public static final String DEFAULT_LOGFILE_SIZE_MAX_BYTES = String.valueOf(1024 * 1024 * 1024); // 1 GB
+  public static final ConfigProperty<String> LOGFILE_SIZE_MAX_BYTES = 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 String LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = "hoodie.logfile.data.block.max.size";
-  public static final String DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = String.valueOf(256 * 1024 * 1024); // 256 MB
-  public static final String PARQUET_COMPRESSION_RATIO = "hoodie.parquet.compression.ratio";
-  // Default compression ratio for parquet
-  public static final String DEFAULT_STREAM_COMPRESSION_RATIO = String.valueOf(0.1);
-  public static final String PARQUET_COMPRESSION_CODEC = "hoodie.parquet.compression.codec";
-  public static final String HFILE_COMPRESSION_ALGORITHM = "hoodie.hfile.compression.algorithm";
-  public static final String ORC_COMPRESSION_CODEC = "hoodie.orc.compression.codec";
+  public static final ConfigProperty<String> LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = 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
+      .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 String DEFAULT_PARQUET_COMPRESSION_CODEC = "gzip";
-  public static final String DEFAULT_HFILE_COMPRESSION_ALGORITHM = "GZ";
-  public static final String DEFAULT_ORC_COMPRESSION_CODEC = "ZLIB";
-  public static final String LOGFILE_TO_PARQUET_COMPRESSION_RATIO = "hoodie.logfile.to.parquet.compression.ratio";
-  // Default compression ratio for log file to parquet, general 3x
-  public static final String DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO = String.valueOf(0.35);
+  public static final ConfigProperty<String> PARQUET_COMPRESSION_CODEC = ConfigProperty
+      .key("hoodie.parquet.compression.codec")
+      .defaultValue("gzip")
+      .withDocumentation("Compression Codec for parquet files");
 
-  private HoodieStorageConfig(Properties props) {
-    super(props);
+  public static final ConfigProperty<String> HFILE_COMPRESSION_ALGORITHM = ConfigProperty
+      .key("hoodie.hfile.compression.algorithm")
+      .defaultValue("GZ")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> ORC_COMPRESSION_CODEC = ConfigProperty
+      .key("hoodie.orc.compression.codec")
+      .defaultValue("ZLIB")
+      .withDocumentation("");
+
+  // Default compression ratio for log file to parquet, general 3x
+  public static final ConfigProperty<String> LOGFILE_TO_PARQUET_COMPRESSION_RATIO = 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.");
+
+  private HoodieStorageConfig() {
+    super();
   }
 
   public static HoodieStorageConfig.Builder newBuilder() {
@@ -84,131 +131,98 @@ public class HoodieStorageConfig extends DefaultHoodieConfig {
 
   public static class Builder {
 
-    private final Properties props = new Properties();
+    private final HoodieStorageConfig storageConfig = new HoodieStorageConfig();
 
     public Builder fromFile(File propertiesFile) throws IOException {
       try (FileReader reader = new FileReader(propertiesFile)) {
-        this.props.load(reader);
+        this.storageConfig.getProps().load(reader);
         return this;
       }
     }
 
     public Builder fromProperties(Properties props) {
-      this.props.putAll(props);
+      this.storageConfig.getProps().putAll(props);
       return this;
     }
 
     public Builder parquetMaxFileSize(long maxFileSize) {
-      props.setProperty(PARQUET_FILE_MAX_BYTES, String.valueOf(maxFileSize));
+      storageConfig.setValue(PARQUET_FILE_MAX_BYTES, String.valueOf(maxFileSize));
       return this;
     }
 
     public Builder parquetBlockSize(int blockSize) {
-      props.setProperty(PARQUET_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
+      storageConfig.setValue(PARQUET_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
       return this;
     }
 
     public Builder parquetPageSize(int pageSize) {
-      props.setProperty(PARQUET_PAGE_SIZE_BYTES, String.valueOf(pageSize));
+      storageConfig.setValue(PARQUET_PAGE_SIZE_BYTES, String.valueOf(pageSize));
       return this;
     }
 
     public Builder hfileMaxFileSize(long maxFileSize) {
-      props.setProperty(HFILE_FILE_MAX_BYTES, String.valueOf(maxFileSize));
+      storageConfig.setValue(HFILE_FILE_MAX_BYTES, String.valueOf(maxFileSize));
       return this;
     }
 
     public Builder hfileBlockSize(int blockSize) {
-      props.setProperty(HFILE_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
+      storageConfig.setValue(HFILE_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
       return this;
     }
 
     public Builder logFileDataBlockMaxSize(int dataBlockSize) {
-      props.setProperty(LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES, String.valueOf(dataBlockSize));
+      storageConfig.setValue(LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES, String.valueOf(dataBlockSize));
       return this;
     }
 
     public Builder logFileMaxSize(int logFileSize) {
-      props.setProperty(LOGFILE_SIZE_MAX_BYTES, String.valueOf(logFileSize));
+      storageConfig.setValue(LOGFILE_SIZE_MAX_BYTES, String.valueOf(logFileSize));
       return this;
     }
 
     public Builder parquetCompressionRatio(double parquetCompressionRatio) {
-      props.setProperty(PARQUET_COMPRESSION_RATIO, String.valueOf(parquetCompressionRatio));
+      storageConfig.setValue(PARQUET_COMPRESSION_RATIO, String.valueOf(parquetCompressionRatio));
       return this;
     }
 
     public Builder parquetCompressionCodec(String parquetCompressionCodec) {
-      props.setProperty(PARQUET_COMPRESSION_CODEC, parquetCompressionCodec);
+      storageConfig.setValue(PARQUET_COMPRESSION_CODEC, parquetCompressionCodec);
       return this;
     }
 
     public Builder hfileCompressionAlgorithm(String hfileCompressionAlgorithm) {
-      props.setProperty(HFILE_COMPRESSION_ALGORITHM, hfileCompressionAlgorithm);
+      storageConfig.setValue(HFILE_COMPRESSION_ALGORITHM, hfileCompressionAlgorithm);
       return this;
     }
 
     public Builder logFileToParquetCompressionRatio(double logFileToParquetCompressionRatio) {
-      props.setProperty(LOGFILE_TO_PARQUET_COMPRESSION_RATIO, String.valueOf(logFileToParquetCompressionRatio));
+      storageConfig.setValue(LOGFILE_TO_PARQUET_COMPRESSION_RATIO, String.valueOf(logFileToParquetCompressionRatio));
       return this;
     }
 
     public Builder orcMaxFileSize(long maxFileSize) {
-      props.setProperty(ORC_FILE_MAX_BYTES, String.valueOf(maxFileSize));
+      storageConfig.setValue(ORC_FILE_MAX_BYTES, String.valueOf(maxFileSize));
       return this;
     }
 
     public Builder orcStripeSize(int orcStripeSize) {
-      props.setProperty(ORC_STRIPE_SIZE, String.valueOf(orcStripeSize));
+      storageConfig.setValue(ORC_STRIPE_SIZE, String.valueOf(orcStripeSize));
       return this;
     }
 
     public Builder orcBlockSize(int orcBlockSize) {
-      props.setProperty(ORC_BLOCK_SIZE, String.valueOf(orcBlockSize));
+      storageConfig.setValue(ORC_BLOCK_SIZE, String.valueOf(orcBlockSize));
       return this;
     }
 
     public Builder orcCompressionCodec(String orcCompressionCodec) {
-      props.setProperty(ORC_COMPRESSION_CODEC, orcCompressionCodec);
+      storageConfig.setValue(ORC_COMPRESSION_CODEC, orcCompressionCodec);
       return this;
     }
 
     public HoodieStorageConfig build() {
-      HoodieStorageConfig config = new HoodieStorageConfig(props);
-      setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES), PARQUET_FILE_MAX_BYTES,
-          DEFAULT_PARQUET_FILE_MAX_BYTES);
-      setDefaultOnCondition(props, !props.containsKey(PARQUET_BLOCK_SIZE_BYTES), PARQUET_BLOCK_SIZE_BYTES,
-          DEFAULT_PARQUET_BLOCK_SIZE_BYTES);
-      setDefaultOnCondition(props, !props.containsKey(PARQUET_PAGE_SIZE_BYTES), PARQUET_PAGE_SIZE_BYTES,
-          DEFAULT_PARQUET_PAGE_SIZE_BYTES);
-      setDefaultOnCondition(props, !props.containsKey(LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES),
-          LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES, DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES);
-      setDefaultOnCondition(props, !props.containsKey(LOGFILE_SIZE_MAX_BYTES), LOGFILE_SIZE_MAX_BYTES,
-          DEFAULT_LOGFILE_SIZE_MAX_BYTES);
-      setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_RATIO), PARQUET_COMPRESSION_RATIO,
-          DEFAULT_STREAM_COMPRESSION_RATIO);
-      setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_CODEC), PARQUET_COMPRESSION_CODEC,
-          DEFAULT_PARQUET_COMPRESSION_CODEC);
-      setDefaultOnCondition(props, !props.containsKey(LOGFILE_TO_PARQUET_COMPRESSION_RATIO),
-          LOGFILE_TO_PARQUET_COMPRESSION_RATIO, DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO);
-
-      setDefaultOnCondition(props, !props.containsKey(HFILE_BLOCK_SIZE_BYTES), HFILE_BLOCK_SIZE_BYTES,
-          DEFAULT_HFILE_BLOCK_SIZE_BYTES);
-      setDefaultOnCondition(props, !props.containsKey(HFILE_COMPRESSION_ALGORITHM), HFILE_COMPRESSION_ALGORITHM,
-          DEFAULT_HFILE_COMPRESSION_ALGORITHM);
-      setDefaultOnCondition(props, !props.containsKey(HFILE_FILE_MAX_BYTES), HFILE_FILE_MAX_BYTES,
-          DEFAULT_HFILE_FILE_MAX_BYTES);
-
-      setDefaultOnCondition(props, !props.containsKey(ORC_FILE_MAX_BYTES), ORC_FILE_MAX_BYTES,
-          DEFAULT_ORC_FILE_MAX_BYTES);
-      setDefaultOnCondition(props, !props.containsKey(ORC_STRIPE_SIZE), ORC_STRIPE_SIZE,
-          DEFAULT_ORC_STRIPE_SIZE);
-      setDefaultOnCondition(props, !props.containsKey(ORC_BLOCK_SIZE), ORC_BLOCK_SIZE,
-          DEFAULT_ORC_BLOCK_SIZE);
-      setDefaultOnCondition(props, !props.containsKey(ORC_COMPRESSION_CODEC), ORC_COMPRESSION_CODEC,
-          DEFAULT_ORC_COMPRESSION_CODEC);
-
-      return config;
+      storageConfig.setDefaults(HoodieStorageConfig.class.getName());
+      return storageConfig;
     }
   }
 
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 126e4f0..3e12691 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
@@ -17,7 +17,8 @@
 
 package org.apache.hudi.config;
 
-import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 
 import java.io.File;
 import java.io.FileReader;
@@ -27,24 +28,44 @@ import java.util.Properties;
 /**
  * Write callback related config.
  */
-public class HoodieWriteCommitCallbackConfig extends DefaultHoodieConfig {
+public class HoodieWriteCommitCallbackConfig extends HoodieConfig {
 
   public static final String CALLBACK_PREFIX = "hoodie.write.commit.callback.";
-  public static final String CALLBACK_ON = CALLBACK_PREFIX + "on";
-  public static final boolean DEFAULT_CALLBACK_ON = false;
 
-  public static final String CALLBACK_CLASS_PROP = CALLBACK_PREFIX + "class";
-  public static final String DEFAULT_CALLBACK_CLASS_PROP = "org.apache.hudi.callback.impl.HoodieWriteCommitHttpCallback";
+  public static final ConfigProperty<Boolean> CALLBACK_ON = ConfigProperty
+      .key(CALLBACK_PREFIX + "on")
+      .defaultValue(false)
+      .sinceVersion("0.6.0")
+      .withDocumentation("Turn callback on/off. off by default.");
+
+  public static final ConfigProperty<String> CALLBACK_CLASS_PROP = 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");
 
   // ***** HTTP callback configs *****
-  public static final String CALLBACK_HTTP_URL_PROP = CALLBACK_PREFIX + "http.url";
-  public static final String CALLBACK_HTTP_API_KEY = CALLBACK_PREFIX + "http.api.key";
-  public static final String DEFAULT_CALLBACK_HTTP_API_KEY = "hudi_write_commit_http_callback";
-  public static final String CALLBACK_HTTP_TIMEOUT_SECONDS = CALLBACK_PREFIX + "http.timeout.seconds";
-  public static final int DEFAULT_CALLBACK_HTTP_TIMEOUT_SECONDS = 3;
-
-  private HoodieWriteCommitCallbackConfig(Properties props) {
-    super(props);
+  public static final ConfigProperty<String> CALLBACK_HTTP_URL_PROP = ConfigProperty
+      .key(CALLBACK_PREFIX + "http.url")
+      .noDefaultValue()
+      .sinceVersion("0.6.0")
+      .withDocumentation("Callback host to be sent along with callback messages");
+
+  public static final ConfigProperty<String> CALLBACK_HTTP_API_KEY = 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
+      .key(CALLBACK_PREFIX + "http.timeout.seconds")
+      .defaultValue(3)
+      .sinceVersion("0.6.0")
+      .withDocumentation("Callback timeout in seconds. 3 by default");
+
+  private HoodieWriteCommitCallbackConfig() {
+    super();
   }
 
   public static HoodieWriteCommitCallbackConfig.Builder newBuilder() {
@@ -53,54 +74,48 @@ public class HoodieWriteCommitCallbackConfig extends DefaultHoodieConfig {
 
   public static class Builder {
 
-    private final Properties props = new Properties();
+    private final HoodieWriteCommitCallbackConfig writeCommitCallbackConfig = new HoodieWriteCommitCallbackConfig();
 
     public HoodieWriteCommitCallbackConfig.Builder fromFile(File propertiesFile) throws IOException {
       try (FileReader reader = new FileReader(propertiesFile)) {
-        this.props.load(reader);
+        this.writeCommitCallbackConfig.getProps().load(reader);
         return this;
       }
     }
 
     public HoodieWriteCommitCallbackConfig.Builder fromProperties(Properties props) {
-      this.props.putAll(props);
+      this.writeCommitCallbackConfig.getProps().putAll(props);
       return this;
     }
 
     public HoodieWriteCommitCallbackConfig.Builder writeCommitCallbackOn(String callbackOn) {
-      props.setProperty(CALLBACK_ON, callbackOn);
+      writeCommitCallbackConfig.setValue(CALLBACK_ON, callbackOn);
       return this;
     }
 
     public HoodieWriteCommitCallbackConfig.Builder withCallbackClass(String callbackClass) {
-      props.setProperty(CALLBACK_CLASS_PROP, callbackClass);
+      writeCommitCallbackConfig.setValue(CALLBACK_CLASS_PROP, callbackClass);
       return this;
     }
 
     public HoodieWriteCommitCallbackConfig.Builder withCallbackHttpUrl(String url) {
-      props.setProperty(CALLBACK_HTTP_URL_PROP, url);
+      writeCommitCallbackConfig.setValue(CALLBACK_HTTP_URL_PROP, url);
       return this;
     }
 
     public Builder withCallbackHttpTimeoutSeconds(String timeoutSeconds) {
-      props.setProperty(CALLBACK_HTTP_TIMEOUT_SECONDS, timeoutSeconds);
+      writeCommitCallbackConfig.setValue(CALLBACK_HTTP_TIMEOUT_SECONDS, timeoutSeconds);
       return this;
     }
 
     public Builder withCallbackHttpApiKey(String apiKey) {
-      props.setProperty(CALLBACK_HTTP_API_KEY, apiKey);
+      writeCommitCallbackConfig.setValue(CALLBACK_HTTP_API_KEY, apiKey);
       return this;
     }
 
     public HoodieWriteCommitCallbackConfig build() {
-      HoodieWriteCommitCallbackConfig config = new HoodieWriteCommitCallbackConfig(props);
-      setDefaultOnCondition(props, !props.containsKey(CALLBACK_ON), CALLBACK_ON, String.valueOf(DEFAULT_CALLBACK_ON));
-      setDefaultOnCondition(props, !props.containsKey(CALLBACK_CLASS_PROP), CALLBACK_CLASS_PROP, DEFAULT_CALLBACK_CLASS_PROP);
-      setDefaultOnCondition(props, !props.containsKey(CALLBACK_HTTP_API_KEY), CALLBACK_HTTP_API_KEY, DEFAULT_CALLBACK_HTTP_API_KEY);
-      setDefaultOnCondition(props, !props.containsKey(CALLBACK_HTTP_TIMEOUT_SECONDS), CALLBACK_HTTP_TIMEOUT_SECONDS,
-          String.valueOf(DEFAULT_CALLBACK_HTTP_TIMEOUT_SECONDS));
-
-      return config;
+      writeCommitCallbackConfig.setDefaults(HoodieWriteCommitCallbackConfig.class.getName());
+      return writeCommitCallbackConfig;
     }
   }
 
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 9e89e0e..1783535 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
@@ -22,9 +22,9 @@ import org.apache.hadoop.hbase.io.compress.Compression;
 import org.apache.hudi.client.WriteStatus;
 import org.apache.hudi.client.bootstrap.BootstrapMode;
 import org.apache.hudi.client.transaction.ConflictResolutionStrategy;
-import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.common.config.HoodieMetadataConfig;
-import org.apache.hudi.common.config.LockConfiguration;
 import org.apache.hudi.common.engine.EngineType;
 import org.apache.hudi.common.fs.ConsistencyGuardConfig;
 import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
@@ -58,115 +58,251 @@ import java.util.Properties;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
-import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_PROP;
-
 /**
  * Class storing configs for the HoodieWriteClient.
  */
 @Immutable
-public class HoodieWriteConfig extends DefaultHoodieConfig {
+public class HoodieWriteConfig extends HoodieConfig {
 
   private static final long serialVersionUID = 0L;
 
-  public static final String TABLE_NAME = "hoodie.table.name";
-  public static final String PRECOMBINE_FIELD_PROP = "hoodie.datasource.write.precombine.field";
-  public static final String WRITE_PAYLOAD_CLASS = "hoodie.datasource.write.payload.class";
-  public static final String DEFAULT_WRITE_PAYLOAD_CLASS = OverwriteWithLatestAvroPayload.class.getName();
-
-  public static final String KEYGENERATOR_CLASS_PROP = "hoodie.datasource.write.keygenerator.class";
-  public static final String KEYGENERATOR_TYPE_PROP = "hoodie.datasource.write.keygenerator.type";
-  public static final String DEFAULT_KEYGENERATOR_TYPE = KeyGeneratorType.SIMPLE.name();
-
-  public static final String DEFAULT_ROLLBACK_USING_MARKERS = "false";
-  public static final String ROLLBACK_USING_MARKERS = "hoodie.rollback.using.markers";
-  public static final String TIMELINE_LAYOUT_VERSION = "hoodie.timeline.layout.version";
-  public static final String BASE_PATH_PROP = "hoodie.base.path";
-  public static final String AVRO_SCHEMA = "hoodie.avro.schema";
-  public static final String AVRO_SCHEMA_VALIDATE = "hoodie.avro.schema.validate";
-  public static final String DEFAULT_AVRO_SCHEMA_VALIDATE = "false";
-  public static final String DEFAULT_PARALLELISM = "1500";
-  public static final String INSERT_PARALLELISM = "hoodie.insert.shuffle.parallelism";
-  public static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism";
-  public static final String BULKINSERT_USER_DEFINED_PARTITIONER_CLASS = "hoodie.bulkinsert.user.defined.partitioner.class";
-  public static final String BULKINSERT_INPUT_DATA_SCHEMA_DDL = "hoodie.bulkinsert.schema.ddl";
-  public static final String UPSERT_PARALLELISM = "hoodie.upsert.shuffle.parallelism";
-  public static final String DELETE_PARALLELISM = "hoodie.delete.shuffle.parallelism";
-  public static final String DEFAULT_ROLLBACK_PARALLELISM = "100";
-  public static final String ROLLBACK_PARALLELISM = "hoodie.rollback.parallelism";
-  public static final String WRITE_BUFFER_LIMIT_BYTES = "hoodie.write.buffer.limit.bytes";
-  public static final String DEFAULT_WRITE_BUFFER_LIMIT_BYTES = String.valueOf(4 * 1024 * 1024);
-  public static final String COMBINE_BEFORE_INSERT_PROP = "hoodie.combine.before.insert";
-  public static final String DEFAULT_COMBINE_BEFORE_INSERT = "false";
-  public static final String COMBINE_BEFORE_UPSERT_PROP = "hoodie.combine.before.upsert";
-  public static final String DEFAULT_COMBINE_BEFORE_UPSERT = "true";
-  public static final String COMBINE_BEFORE_DELETE_PROP = "hoodie.combine.before.delete";
-  public static final String DEFAULT_COMBINE_BEFORE_DELETE = "true";
-  public static final String WRITE_STATUS_STORAGE_LEVEL = "hoodie.write.status.storage.level";
-  public static final String DEFAULT_WRITE_STATUS_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
-  public static final String HOODIE_AUTO_COMMIT_PROP = "hoodie.auto.commit";
-  public static final String DEFAULT_HOODIE_AUTO_COMMIT = "true";
-
-  public static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class";
-  public static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName();
-  public static final String FINALIZE_WRITE_PARALLELISM = "hoodie.finalize.write.parallelism";
-  public static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = DEFAULT_PARALLELISM;
-  public static final String MARKERS_DELETE_PARALLELISM = "hoodie.markers.delete.parallelism";
-  public static final String DEFAULT_MARKERS_DELETE_PARALLELISM = "100";
-  public static final String BULKINSERT_SORT_MODE = "hoodie.bulkinsert.sort.mode";
-  public static final String DEFAULT_BULKINSERT_SORT_MODE = BulkInsertSortMode.GLOBAL_SORT
-      .toString();
-
-  public static final String EMBEDDED_TIMELINE_SERVER_ENABLED = "hoodie.embed.timeline.server";
-  public static final String DEFAULT_EMBEDDED_TIMELINE_SERVER_ENABLED = "true";
-  public static final String EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED = "hoodie.embed.timeline.server.reuse.enabled";
-  public static final String DEFAULT_EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED = "false";
-  public static final String EMBEDDED_TIMELINE_SERVER_PORT = "hoodie.embed.timeline.server.port";
-  public static final String DEFAULT_EMBEDDED_TIMELINE_SERVER_PORT = "0";
-  public static final String EMBEDDED_TIMELINE_SERVER_THREADS = "hoodie.embed.timeline.server.threads";
-  public static final String DEFAULT_EMBEDDED_TIMELINE_SERVER_THREADS = "-1";
-  public static final String EMBEDDED_TIMELINE_SERVER_COMPRESS_OUTPUT = "hoodie.embed.timeline.server.gzip";
-  public static final String DEFAULT_EMBEDDED_TIMELINE_COMPRESS_OUTPUT = "true";
-  public static final String EMBEDDED_TIMELINE_SERVER_USE_ASYNC = "hoodie.embed.timeline.server.async";
-  public static final String DEFAULT_EMBEDDED_TIMELINE_SERVER_ASYNC = "false";
-
-  public static final String FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP = "hoodie.fail.on.timeline.archiving";
-  public static final String DEFAULT_FAIL_ON_TIMELINE_ARCHIVING_ENABLED = "true";
-  // time between successive attempts to ensure written data's metadata is consistent on storage
-  public static final String INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP =
-      "hoodie.consistency.check.initial_interval_ms";
-  public static long DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = 2000L;
-
-  // max interval time
-  public static final String MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = "hoodie.consistency.check.max_interval_ms";
-  public static long DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS = 300000L;
-
-  // maximum number of checks, for consistency of written data. Will wait upto 256 Secs
-  public static final String MAX_CONSISTENCY_CHECKS_PROP = "hoodie.consistency.check.max_checks";
-  public static int DEFAULT_MAX_CONSISTENCY_CHECKS = 7;
-
-  // Data validation check performed during merges before actual commits
-  private static final String MERGE_DATA_VALIDATION_CHECK_ENABLED = "hoodie.merge.data.validation.enabled";
-  private static final String DEFAULT_MERGE_DATA_VALIDATION_CHECK_ENABLED = "false";
-
-  // Allow duplicates with inserts while merging with existing records
-  private static final String MERGE_ALLOW_DUPLICATE_ON_INSERTS = "hoodie.merge.allow.duplicate.on.inserts";
-  private static final String DEFAULT_MERGE_ALLOW_DUPLICATE_ON_INSERTS = "false";
-
-  public static final String CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP = "hoodie.client.heartbeat.interval_in_ms";
-  public static final Integer DEFAULT_CLIENT_HEARTBEAT_INTERVAL_IN_MS = 60 * 1000;
-
-  public static final String CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP = "hoodie.client.heartbeat.tolerable.misses";
-  public static final Integer DEFAULT_CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES = 2;
-  // Enable different concurrency support
-  public static final String WRITE_CONCURRENCY_MODE_PROP =
-      "hoodie.write.concurrency.mode";
-  public static final String DEFAULT_WRITE_CONCURRENCY_MODE = WriteConcurrencyMode.SINGLE_WRITER.name();
-
-  // Comma separated metadata key prefixes to override from latest commit during overlapping commits via multi writing
-  public static final String WRITE_META_KEY_PREFIXES_PROP =
-      "hoodie.write.meta.key.prefixes";
-  public static final String DEFAULT_WRITE_META_KEY_PREFIXES = "";
+  public static final ConfigProperty<String> TABLE_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<String> PRECOMBINE_FIELD_PROP = ConfigProperty
+      .key("hoodie.datasource.write.precombine.field")
+      .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(..)");
+
+  public static final ConfigProperty<String> WRITE_PAYLOAD_CLASS = 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_PROP = ConfigProperty
+      .key("hoodie.datasource.write.keygenerator.class")
+      .noDefaultValue()
+      .withDocumentation("Key generator class, that implements will extract the key out of incoming Row object");
+
+  public static final ConfigProperty<String> KEYGENERATOR_TYPE_PROP = ConfigProperty
+      .key("hoodie.datasource.write.keygenerator.type")
+      .defaultValue(KeyGeneratorType.SIMPLE.name())
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> ROLLBACK_USING_MARKERS = 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
+      .key("hoodie.timeline.layout.version")
+      .noDefaultValue()
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> BASE_PATH_PROP = ConfigProperty
+      .key("hoodie.base.path")
+      .noDefaultValue()
+      .withDocumentation("Base DFS path under which all the data partitions are created. "
+          + "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 the base directory.");
+
+  public static final ConfigProperty<String> AVRO_SCHEMA = ConfigProperty
+      .key("hoodie.avro.schema")
+      .noDefaultValue()
+      .withDocumentation("This is the current reader avro schema for the table. This is a string of the entire schema. "
+          + "HoodieWriteClient uses this schema to pass on to implementations of HoodieRecordPayload to convert "
+          + "from the source format to avro record. This is also used when re-writing records during an update.");
+
+  public static final ConfigProperty<String> AVRO_SCHEMA_VALIDATE = ConfigProperty
+      .key("hoodie.avro.schema.validate")
+      .defaultValue("false")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> INSERT_PARALLELISM = ConfigProperty
+      .key("hoodie.insert.shuffle.parallelism")
+      .defaultValue("1500")
+      .withDocumentation("Once data has been initially imported, this parallelism controls initial parallelism for reading input records. "
+          + "Ensure this value is high enough say: 1 partition for 1 GB of input data");
+
+  public static final ConfigProperty<String> BULKINSERT_PARALLELISM = ConfigProperty
+      .key("hoodie.bulkinsert.shuffle.parallelism")
+      .defaultValue("1500")
+      .withDocumentation("Bulk insert is meant to be used for large initial imports and this parallelism determines "
+          + "the initial number of files in your table. Tune this to achieve a desired optimal size during initial import.");
+
+  public static final ConfigProperty<String> BULKINSERT_USER_DEFINED_PARTITIONER_CLASS = ConfigProperty
+      .key("hoodie.bulkinsert.user.defined.partitioner.class")
+      .noDefaultValue()
+      .withDocumentation("If specified, this class will be used to re-partition input records before they are inserted.");
+
+  public static final ConfigProperty<String> BULKINSERT_INPUT_DATA_SCHEMA_DDL = ConfigProperty
+      .key("hoodie.bulkinsert.schema.ddl")
+      .noDefaultValue()
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> UPSERT_PARALLELISM = ConfigProperty
+      .key("hoodie.upsert.shuffle.parallelism")
+      .defaultValue("1500")
+      .withDocumentation("Once data has been initially imported, this parallelism controls initial parallelism for reading input records. "
+          + "Ensure this value is high enough say: 1 partition for 1 GB of input data");
+
+  public static final ConfigProperty<String> DELETE_PARALLELISM = ConfigProperty
+      .key("hoodie.delete.shuffle.parallelism")
+      .defaultValue("1500")
+      .withDocumentation("This parallelism is Used for “delete” operation while deduping or repartioning.");
+
+  public static final ConfigProperty<String> ROLLBACK_PARALLELISM = ConfigProperty
+      .key("hoodie.rollback.parallelism")
+      .defaultValue("100")
+      .withDocumentation("Determines the parallelism for rollback of commits.");
+
+  public static final ConfigProperty<String> WRITE_BUFFER_LIMIT_BYTES = ConfigProperty
+      .key("hoodie.write.buffer.limit.bytes")
+      .defaultValue(String.valueOf(4 * 1024 * 1024))
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> COMBINE_BEFORE_INSERT_PROP = ConfigProperty
+      .key("hoodie.combine.before.insert")
+      .defaultValue("false")
+      .withDocumentation("Flag which first combines the input RDD and merges multiple partial records into a single record "
+          + "before inserting or updating in DFS");
+
+  public static final ConfigProperty<String> COMBINE_BEFORE_UPSERT_PROP = ConfigProperty
+      .key("hoodie.combine.before.upsert")
+      .defaultValue("true")
+      .withDocumentation("Flag which first combines the input RDD and merges multiple partial records into a single record "
+          + "before inserting or updating in DFS");
+
+  public static final ConfigProperty<String> COMBINE_BEFORE_DELETE_PROP = ConfigProperty
+      .key("hoodie.combine.before.delete")
+      .defaultValue("true")
+      .withDocumentation("Flag which first combines the input RDD and merges multiple partial records into a single record "
+          + "before deleting in DFS");
+
+  public static final ConfigProperty<String> WRITE_STATUS_STORAGE_LEVEL = ConfigProperty
+      .key("hoodie.write.status.storage.level")
+      .defaultValue("MEMORY_AND_DISK_SER")
+      .withDocumentation("HoodieWriteClient.insert and HoodieWriteClient.upsert returns a persisted RDD[WriteStatus], "
+          + "this is because the Client can choose to inspect the WriteStatus and choose and commit or not based on the failures. "
+          + "This is a configuration for the storage level for this RDD");
+
+  public static final ConfigProperty<String> HOODIE_AUTO_COMMIT_PROP = ConfigProperty
+      .key("hoodie.auto.commit")
+      .defaultValue("true")
+      .withDocumentation("Should HoodieWriteClient autoCommit after insert and upsert. "
+          + "The client can choose to turn off auto-commit and commit on a “defined success condition”");
+
+  public static final ConfigProperty<String> HOODIE_WRITE_STATUS_CLASS_PROP = ConfigProperty
+      .key("hoodie.writestatus.class")
+      .defaultValue(WriteStatus.class.getName())
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> FINALIZE_WRITE_PARALLELISM = ConfigProperty
+      .key("hoodie.finalize.write.parallelism")
+      .defaultValue("1500")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> MARKERS_DELETE_PARALLELISM = ConfigProperty
+      .key("hoodie.markers.delete.parallelism")
+      .defaultValue("100")
+      .withDocumentation("Determines the parallelism for deleting marker files.");
+
+  public static final ConfigProperty<String> BULKINSERT_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 leveraged when user "
+          + "defined partitioner is not configured. Default is GLOBAL_SORT. 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
+      .key("hoodie.embed.timeline.server")
+      .defaultValue("true")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED = ConfigProperty
+      .key("hoodie.embed.timeline.server.reuse.enabled")
+      .defaultValue("false")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> EMBEDDED_TIMELINE_SERVER_PORT = ConfigProperty
+      .key("hoodie.embed.timeline.server.port")
+      .defaultValue("0")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> EMBEDDED_TIMELINE_SERVER_THREADS = ConfigProperty
+      .key("hoodie.embed.timeline.server.threads")
+      .defaultValue("-1")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> EMBEDDED_TIMELINE_SERVER_COMPRESS_OUTPUT = ConfigProperty
+      .key("hoodie.embed.timeline.server.gzip")
+      .defaultValue("true")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> EMBEDDED_TIMELINE_SERVER_USE_ASYNC = ConfigProperty
+      .key("hoodie.embed.timeline.server.async")
+      .defaultValue("false")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP = ConfigProperty
+      .key("hoodie.fail.on.timeline.archiving")
+      .defaultValue("true")
+      .withDocumentation("");
+
+  public static final ConfigProperty<Long> INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP = ConfigProperty
+      .key("hoodie.consistency.check.initial_interval_ms")
+      .defaultValue(2000L)
+      .withDocumentation("Time between successive attempts to ensure written data's metadata is consistent on storage");
+
+  public static final ConfigProperty<Long> MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = ConfigProperty
+      .key("hoodie.consistency.check.max_interval_ms")
+      .defaultValue(300000L)
+      .withDocumentation("Max interval time for consistency check");
+
+  public static final ConfigProperty<Integer> MAX_CONSISTENCY_CHECKS_PROP = ConfigProperty
+      .key("hoodie.consistency.check.max_checks")
+      .defaultValue(7)
+      .withDocumentation("Maximum number of checks, for consistency of written data. Will wait upto 256 Secs");
+
+  public static final ConfigProperty<String> MERGE_DATA_VALIDATION_CHECK_ENABLED = ConfigProperty
+      .key("hoodie.merge.data.validation.enabled")
+      .defaultValue("false")
+      .withDocumentation("Data validation check performed during merges before actual commits");
+
+  public static final ConfigProperty<String> MERGE_ALLOW_DUPLICATE_ON_INSERTS = ConfigProperty
+      .key("hoodie.merge.allow.duplicate.on.inserts")
+      .defaultValue("false")
+      .withDocumentation("Allow duplicates with inserts while merging with existing records");
+
+  public static final ConfigProperty<Integer> CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP = ConfigProperty
+      .key("hoodie.client.heartbeat.interval_in_ms")
+      .defaultValue(60 * 1000)
+      .withDocumentation("");
+
+  public static final ConfigProperty<Integer> CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP = ConfigProperty
+      .key("hoodie.client.heartbeat.tolerable.misses")
+      .defaultValue(2)
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> WRITE_CONCURRENCY_MODE_PROP = ConfigProperty
+      .key("hoodie.write.concurrency.mode")
+      .defaultValue(WriteConcurrencyMode.SINGLE_WRITER.name())
+      .withDocumentation("Enable different concurrency support");
+
+  public static final ConfigProperty<String> WRITE_META_KEY_PREFIXES_PROP = 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");
 
   /**
    * The specified write schema. In most case, we do not need set this parameter,
@@ -175,7 +311,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
    *
    * Currently the MergeIntoHoodieTableCommand use this to specify the write schema.
    */
-  public static final String WRITE_SCHEMA_PROP = "hoodie.write.schema";
+  public static final ConfigProperty<String> WRITE_SCHEMA_PROP = ConfigProperty
+      .key("hoodie.write.schema")
+      .noDefaultValue()
+      .withDocumentation("");
 
   /**
    * HUDI-858 : There are users who had been directly using RDD APIs and have relied on a behavior in 0.4.x to allow
@@ -187,12 +326,15 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
    * 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 String ALLOW_MULTI_WRITE_ON_SAME_INSTANT =
-      "_.hoodie.allow.multi.write.on.same.instant";
-  public static final String DEFAULT_ALLOW_MULTI_WRITE_ON_SAME_INSTANT = "false";
+  public static final ConfigProperty<String> ALLOW_MULTI_WRITE_ON_SAME_INSTANT = ConfigProperty
+      .key("_.hoodie.allow.multi.write.on.same.instant")
+      .defaultValue("false")
+      .withDocumentation("");
 
-  public static final String EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION = AVRO_SCHEMA + ".externalTransformation";
-  public static final String DEFAULT_EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION = "false";
+  public static final ConfigProperty<String> EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION = ConfigProperty
+      .key(AVRO_SCHEMA + ".externalTransformation")
+      .defaultValue("false")
+      .withDocumentation("");
 
   private ConsistencyGuardConfig consistencyGuardConfig;
 
@@ -208,8 +350,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
   /**
    * Use Spark engine by default.
    */
-  protected HoodieWriteConfig(Properties props) {
-    this(EngineType.SPARK, props);
+  protected HoodieWriteConfig() {
+    super();
+    this.engineType = EngineType.SPARK;
+    this.clientSpecifiedViewStorageConfig = null;
   }
 
   protected HoodieWriteConfig(EngineType engineType, Properties props) {
@@ -232,15 +376,15 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
    * base properties.
    */
   public String getBasePath() {
-    return props.getProperty(BASE_PATH_PROP);
+    return getString(BASE_PATH_PROP);
   }
 
   public String getSchema() {
-    return props.getProperty(AVRO_SCHEMA);
+    return getString(AVRO_SCHEMA);
   }
 
   public void setSchema(String schemaStr) {
-    props.setProperty(AVRO_SCHEMA, schemaStr);
+    setValue(AVRO_SCHEMA, schemaStr);
   }
 
   /**
@@ -251,34 +395,34 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
    * @return
    */
   public String getWriteSchema() {
-    if (props.containsKey(WRITE_SCHEMA_PROP)) {
-      return props.getProperty(WRITE_SCHEMA_PROP);
+    if (props.containsKey(WRITE_SCHEMA_PROP.key())) {
+      return getString(WRITE_SCHEMA_PROP);
     }
     return getSchema();
   }
 
   public boolean getAvroSchemaValidate() {
-    return Boolean.parseBoolean(props.getProperty(AVRO_SCHEMA_VALIDATE));
+    return getBoolean(AVRO_SCHEMA_VALIDATE);
   }
 
   public String getTableName() {
-    return props.getProperty(TABLE_NAME);
+    return getString(TABLE_NAME);
   }
 
   public String getPreCombineField() {
-    return props.getProperty(PRECOMBINE_FIELD_PROP);
+    return getString(PRECOMBINE_FIELD_PROP);
   }
 
   public String getWritePayloadClass() {
-    return props.getProperty(WRITE_PAYLOAD_CLASS);
+    return getString(WRITE_PAYLOAD_CLASS);
   }
 
   public String getKeyGeneratorClass() {
-    return props.getProperty(KEYGENERATOR_CLASS_PROP);
+    return getString(KEYGENERATOR_CLASS_PROP);
   }
 
   public Boolean shouldAutoCommit() {
-    return Boolean.parseBoolean(props.getProperty(HOODIE_AUTO_COMMIT_PROP));
+    return getBoolean(HOODIE_AUTO_COMMIT_PROP);
   }
 
   public Boolean shouldAssumeDatePartitioning() {
@@ -286,35 +430,35 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
   }
 
   public boolean shouldUseExternalSchemaTransformation() {
-    return Boolean.parseBoolean(props.getProperty(EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION));
+    return getBoolean(EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION);
   }
 
   public Integer getTimelineLayoutVersion() {
-    return Integer.parseInt(props.getProperty(TIMELINE_LAYOUT_VERSION));
+    return getInt(TIMELINE_LAYOUT_VERSION);
   }
 
   public int getBulkInsertShuffleParallelism() {
-    return Integer.parseInt(props.getProperty(BULKINSERT_PARALLELISM));
+    return getInt(BULKINSERT_PARALLELISM);
   }
 
   public String getUserDefinedBulkInsertPartitionerClass() {
-    return props.getProperty(BULKINSERT_USER_DEFINED_PARTITIONER_CLASS);
+    return getString(BULKINSERT_USER_DEFINED_PARTITIONER_CLASS);
   }
 
   public int getInsertShuffleParallelism() {
-    return Integer.parseInt(props.getProperty(INSERT_PARALLELISM));
+    return getInt(INSERT_PARALLELISM);
   }
 
   public int getUpsertShuffleParallelism() {
-    return Integer.parseInt(props.getProperty(UPSERT_PARALLELISM));
+    return getInt(UPSERT_PARALLELISM);
   }
 
   public int getDeleteShuffleParallelism() {
-    return Math.max(Integer.parseInt(props.getProperty(DELETE_PARALLELISM)), 1);
+    return Math.max(getInt(DELETE_PARALLELISM), 1);
   }
 
   public int getRollbackParallelism() {
-    return Integer.parseInt(props.getProperty(ROLLBACK_PARALLELISM));
+    return getInt(ROLLBACK_PARALLELISM);
   }
 
   public int getFileListingParallelism() {
@@ -322,92 +466,92 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
   }
 
   public boolean shouldRollbackUsingMarkers() {
-    return Boolean.parseBoolean(props.getProperty(ROLLBACK_USING_MARKERS));
+    return getBoolean(ROLLBACK_USING_MARKERS);
   }
 
   public int getWriteBufferLimitBytes() {
-    return Integer.parseInt(props.getProperty(WRITE_BUFFER_LIMIT_BYTES, DEFAULT_WRITE_BUFFER_LIMIT_BYTES));
+    return Integer.parseInt(getStringOrDefault(WRITE_BUFFER_LIMIT_BYTES));
   }
 
   public boolean shouldCombineBeforeInsert() {
-    return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_INSERT_PROP));
+    return getBoolean(COMBINE_BEFORE_INSERT_PROP);
   }
 
   public boolean shouldCombineBeforeUpsert() {
-    return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_UPSERT_PROP));
+    return getBoolean(COMBINE_BEFORE_UPSERT_PROP);
   }
 
   public boolean shouldCombineBeforeDelete() {
-    return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_DELETE_PROP));
+    return getBoolean(COMBINE_BEFORE_DELETE_PROP);
   }
 
   public boolean shouldAllowMultiWriteOnSameInstant() {
-    return Boolean.parseBoolean(props.getProperty(ALLOW_MULTI_WRITE_ON_SAME_INSTANT));
+    return getBoolean(ALLOW_MULTI_WRITE_ON_SAME_INSTANT);
   }
 
   public String getWriteStatusClassName() {
-    return props.getProperty(HOODIE_WRITE_STATUS_CLASS_PROP);
+    return getString(HOODIE_WRITE_STATUS_CLASS_PROP);
   }
 
   public int getFinalizeWriteParallelism() {
-    return Integer.parseInt(props.getProperty(FINALIZE_WRITE_PARALLELISM));
+    return getInt(FINALIZE_WRITE_PARALLELISM);
   }
 
   public int getMarkersDeleteParallelism() {
-    return Integer.parseInt(props.getProperty(MARKERS_DELETE_PARALLELISM));
+    return getInt(MARKERS_DELETE_PARALLELISM);
   }
 
   public boolean isEmbeddedTimelineServerEnabled() {
-    return Boolean.parseBoolean(props.getProperty(EMBEDDED_TIMELINE_SERVER_ENABLED));
+    return getBoolean(EMBEDDED_TIMELINE_SERVER_ENABLED);
   }
 
   public boolean isEmbeddedTimelineServerReuseEnabled() {
-    return Boolean.parseBoolean(props.getProperty(EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED));
+    return Boolean.parseBoolean(getStringOrDefault(EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED));
   }
 
   public int getEmbeddedTimelineServerPort() {
-    return Integer.parseInt(props.getProperty(EMBEDDED_TIMELINE_SERVER_PORT, DEFAULT_EMBEDDED_TIMELINE_SERVER_PORT));
+    return Integer.parseInt(getStringOrDefault(EMBEDDED_TIMELINE_SERVER_PORT));
   }
 
   public int getEmbeddedTimelineServerThreads() {
-    return Integer.parseInt(props.getProperty(EMBEDDED_TIMELINE_SERVER_THREADS, DEFAULT_EMBEDDED_TIMELINE_SERVER_THREADS));
+    return Integer.parseInt(getStringOrDefault(EMBEDDED_TIMELINE_SERVER_THREADS));
   }
 
   public boolean getEmbeddedTimelineServerCompressOutput() {
-    return Boolean.parseBoolean(props.getProperty(EMBEDDED_TIMELINE_SERVER_COMPRESS_OUTPUT, DEFAULT_EMBEDDED_TIMELINE_COMPRESS_OUTPUT));
+    return Boolean.parseBoolean(getStringOrDefault(EMBEDDED_TIMELINE_SERVER_COMPRESS_OUTPUT));
   }
 
   public boolean getEmbeddedTimelineServerUseAsync() {
-    return Boolean.parseBoolean(props.getProperty(EMBEDDED_TIMELINE_SERVER_USE_ASYNC, DEFAULT_EMBEDDED_TIMELINE_SERVER_ASYNC));
+    return Boolean.parseBoolean(getStringOrDefault(EMBEDDED_TIMELINE_SERVER_USE_ASYNC));
   }
 
   public boolean isFailOnTimelineArchivingEnabled() {
-    return Boolean.parseBoolean(props.getProperty(FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP));
+    return getBoolean(FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP);
   }
 
   public int getMaxConsistencyChecks() {
-    return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECKS_PROP));
+    return getInt(MAX_CONSISTENCY_CHECKS_PROP);
   }
 
   public int getInitialConsistencyCheckIntervalMs() {
-    return Integer.parseInt(props.getProperty(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
+    return getInt(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP);
   }
 
   public int getMaxConsistencyCheckIntervalMs() {
-    return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
+    return getInt(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP);
   }
 
   public BulkInsertSortMode getBulkInsertSortMode() {
-    String sortMode = props.getProperty(BULKINSERT_SORT_MODE);
+    String sortMode = getString(BULKINSERT_SORT_MODE);
     return BulkInsertSortMode.valueOf(sortMode.toUpperCase());
   }
 
   public boolean isMergeDataValidationCheckEnabled() {
-    return Boolean.parseBoolean(props.getProperty(MERGE_DATA_VALIDATION_CHECK_ENABLED));
+    return getBoolean(MERGE_DATA_VALIDATION_CHECK_ENABLED);
   }
 
   public boolean allowDuplicateInserts() {
-    return Boolean.parseBoolean(props.getProperty(MERGE_ALLOW_DUPLICATE_ON_INSERTS));
+    return getBoolean(MERGE_ALLOW_DUPLICATE_ON_INSERTS);
   }
 
   public EngineType getEngineType() {
@@ -418,99 +562,99 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
    * compaction properties.
    */
   public HoodieCleaningPolicy getCleanerPolicy() {
-    return HoodieCleaningPolicy.valueOf(props.getProperty(HoodieCompactionConfig.CLEANER_POLICY_PROP));
+    return HoodieCleaningPolicy.valueOf(getString(HoodieCompactionConfig.CLEANER_POLICY_PROP));
   }
 
   public int getCleanerFileVersionsRetained() {
-    return Integer.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED_PROP));
+    return getInt(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED_PROP);
   }
 
   public int getCleanerCommitsRetained() {
-    return Integer.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
+    return getInt(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP);
   }
 
   public int getMaxCommitsToKeep() {
-    return Integer.parseInt(props.getProperty(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP));
+    return getInt(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP);
   }
 
   public int getMinCommitsToKeep() {
-    return Integer.parseInt(props.getProperty(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP));
+    return getInt(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP);
   }
 
   public int getParquetSmallFileLimit() {
-    return Integer.parseInt(props.getProperty(HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT_BYTES));
+    return getInt(HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT_BYTES);
   }
 
   public double getRecordSizeEstimationThreshold() {
-    return Double.parseDouble(props.getProperty(HoodieCompactionConfig.RECORD_SIZE_ESTIMATION_THRESHOLD_PROP));
+    return getDouble(HoodieCompactionConfig.RECORD_SIZE_ESTIMATION_THRESHOLD_PROP);
   }
 
   public int getCopyOnWriteInsertSplitSize() {
-    return Integer.parseInt(props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE));
+    return getInt(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE);
   }
 
   public int getCopyOnWriteRecordSizeEstimate() {
-    return Integer.parseInt(props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE));
+    return getInt(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE);
   }
 
   public boolean shouldAutoTuneInsertSplits() {
-    return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS));
+    return getBoolean(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS);
   }
 
   public int getCleanerParallelism() {
-    return Integer.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_PARALLELISM));
+    return getInt(HoodieCompactionConfig.CLEANER_PARALLELISM);
   }
 
   public boolean isAutoClean() {
-    return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.AUTO_CLEAN_PROP));
+    return getBoolean(HoodieCompactionConfig.AUTO_CLEAN_PROP);
   }
 
   public boolean isAsyncClean() {
-    return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.ASYNC_CLEAN_PROP));
+    return getBoolean(HoodieCompactionConfig.ASYNC_CLEAN_PROP);
   }
 
   public boolean incrementalCleanerModeEnabled() {
-    return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.CLEANER_INCREMENTAL_MODE));
+    return getBoolean(HoodieCompactionConfig.CLEANER_INCREMENTAL_MODE);
   }
 
   public boolean inlineCompactionEnabled() {
-    return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_PROP));
+    return getBoolean(HoodieCompactionConfig.INLINE_COMPACT_PROP);
   }
 
   public CompactionTriggerStrategy getInlineCompactTriggerStrategy() {
-    return CompactionTriggerStrategy.valueOf(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_TRIGGER_STRATEGY_PROP));
+    return CompactionTriggerStrategy.valueOf(getString(HoodieCompactionConfig.INLINE_COMPACT_TRIGGER_STRATEGY_PROP));
   }
 
   public int getInlineCompactDeltaCommitMax() {
-    return Integer.parseInt(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP));
+    return getInt(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP);
   }
 
   public int getInlineCompactDeltaSecondsMax() {
-    return Integer.parseInt(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_TIME_DELTA_SECONDS_PROP));
+    return getInt(HoodieCompactionConfig.INLINE_COMPACT_TIME_DELTA_SECONDS_PROP);
   }
 
   public CompactionStrategy getCompactionStrategy() {
-    return ReflectionUtils.loadClass(props.getProperty(HoodieCompactionConfig.COMPACTION_STRATEGY_PROP));
+    return ReflectionUtils.loadClass(getString(HoodieCompactionConfig.COMPACTION_STRATEGY_PROP));
   }
 
   public Long getTargetIOPerCompactionInMB() {
-    return Long.parseLong(props.getProperty(HoodieCompactionConfig.TARGET_IO_PER_COMPACTION_IN_MB_PROP));
+    return getLong(HoodieCompactionConfig.TARGET_IO_PER_COMPACTION_IN_MB_PROP);
   }
 
   public Boolean getCompactionLazyBlockReadEnabled() {
-    return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP));
+    return getBoolean(HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP);
   }
 
   public Boolean getCompactionReverseLogReadEnabled() {
-    return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP));
+    return getBoolean(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP);
   }
 
   public boolean inlineClusteringEnabled() {
-    return Boolean.parseBoolean(props.getProperty(HoodieClusteringConfig.INLINE_CLUSTERING_PROP));
+    return getBoolean(HoodieClusteringConfig.INLINE_CLUSTERING_PROP);
   }
 
   public boolean isAsyncClusteringEnabled() {
-    return Boolean.parseBoolean(props.getProperty(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE_OPT_KEY));
+    return getBoolean(HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE_OPT_KEY);
   }
 
   public boolean isClusteringEnabled() {
@@ -519,150 +663,150 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
   }
 
   public int getInlineClusterMaxCommits() {
-    return Integer.parseInt(props.getProperty(HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT_PROP));
+    return getInt(HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT_PROP);
   }
 
   public String getPayloadClass() {
-    return props.getProperty(HoodieCompactionConfig.PAYLOAD_CLASS_PROP);
+    return getString(HoodieCompactionConfig.PAYLOAD_CLASS_PROP);
   }
 
   public int getTargetPartitionsPerDayBasedCompaction() {
-    return Integer.parseInt(props.getProperty(HoodieCompactionConfig.TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP));
+    return getInt(HoodieCompactionConfig.TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP);
   }
 
   public int getCommitArchivalBatchSize() {
-    return Integer.parseInt(props.getProperty(HoodieCompactionConfig.COMMITS_ARCHIVAL_BATCH_SIZE_PROP));
+    return getInt(HoodieCompactionConfig.COMMITS_ARCHIVAL_BATCH_SIZE_PROP);
   }
 
   public Boolean shouldCleanBootstrapBaseFile() {
-    return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.CLEANER_BOOTSTRAP_BASE_FILE_ENABLED));
+    return getBoolean(HoodieCompactionConfig.CLEANER_BOOTSTRAP_BASE_FILE_ENABLED);
   }
 
   public String getClusteringUpdatesStrategyClass() {
-    return props.getProperty(HoodieClusteringConfig.CLUSTERING_UPDATES_STRATEGY_PROP);
+    return getString(HoodieClusteringConfig.CLUSTERING_UPDATES_STRATEGY_PROP);
   }
 
   public HoodieFailedWritesCleaningPolicy getFailedWritesCleanPolicy() {
     return HoodieFailedWritesCleaningPolicy
-        .valueOf(props.getProperty(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY_PROP));
+        .valueOf(getString(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY_PROP));
   }
 
   /**
    * Clustering properties.
    */
   public String getClusteringPlanStrategyClass() {
-    return props.getProperty(HoodieClusteringConfig.CLUSTERING_PLAN_STRATEGY_CLASS);
+    return getString(HoodieClusteringConfig.CLUSTERING_PLAN_STRATEGY_CLASS);
   }
 
   public String getClusteringExecutionStrategyClass() {
-    return props.getProperty(HoodieClusteringConfig.CLUSTERING_EXECUTION_STRATEGY_CLASS);
+    return getString(HoodieClusteringConfig.CLUSTERING_EXECUTION_STRATEGY_CLASS);
   }
 
   public long getClusteringMaxBytesInGroup() {
-    return Long.parseLong(props.getProperty(HoodieClusteringConfig.CLUSTERING_MAX_BYTES_PER_GROUP));
+    return getLong(HoodieClusteringConfig.CLUSTERING_MAX_BYTES_PER_GROUP);
   }
 
   public long getClusteringSmallFileLimit() {
-    return Long.parseLong(props.getProperty(HoodieClusteringConfig.CLUSTERING_PLAN_SMALL_FILE_LIMIT));
+    return getLong(HoodieClusteringConfig.CLUSTERING_PLAN_SMALL_FILE_LIMIT);
   }
 
   public int getClusteringMaxNumGroups() {
-    return Integer.parseInt(props.getProperty(HoodieClusteringConfig.CLUSTERING_MAX_NUM_GROUPS));
+    return getInt(HoodieClusteringConfig.CLUSTERING_MAX_NUM_GROUPS);
   }
 
   public long getClusteringTargetFileMaxBytes() {
-    return Long.parseLong(props.getProperty(HoodieClusteringConfig.CLUSTERING_TARGET_FILE_MAX_BYTES));
+    return getLong(HoodieClusteringConfig.CLUSTERING_TARGET_FILE_MAX_BYTES);
   }
 
   public int getTargetPartitionsForClustering() {
-    return Integer.parseInt(props.getProperty(HoodieClusteringConfig.CLUSTERING_TARGET_PARTITIONS));
+    return getInt(HoodieClusteringConfig.CLUSTERING_TARGET_PARTITIONS);
   }
 
   public String getClusteringSortColumns() {
-    return props.getProperty(HoodieClusteringConfig.CLUSTERING_SORT_COLUMNS_PROPERTY);
+    return getString(HoodieClusteringConfig.CLUSTERING_SORT_COLUMNS_PROPERTY);
   }
 
   /**
    * index properties.
    */
   public HoodieIndex.IndexType getIndexType() {
-    return HoodieIndex.IndexType.valueOf(props.getProperty(HoodieIndexConfig.INDEX_TYPE_PROP));
+    return HoodieIndex.IndexType.valueOf(getString(HoodieIndexConfig.INDEX_TYPE_PROP));
   }
 
   public String getIndexClass() {
-    return props.getProperty(HoodieIndexConfig.INDEX_CLASS_PROP);
+    return getString(HoodieIndexConfig.INDEX_CLASS_PROP);
   }
 
   public int getBloomFilterNumEntries() {
-    return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES));
+    return getInt(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES);
   }
 
   public double getBloomFilterFPP() {
-    return Double.parseDouble(props.getProperty(HoodieIndexConfig.BLOOM_FILTER_FPP));
+    return getDouble(HoodieIndexConfig.BLOOM_FILTER_FPP);
   }
 
   public String getHbaseZkQuorum() {
-    return props.getProperty(HoodieHBaseIndexConfig.HBASE_ZKQUORUM_PROP);
+    return getString(HoodieHBaseIndexConfig.HBASE_ZKQUORUM_PROP);
   }
 
   public int getHbaseZkPort() {
-    return Integer.parseInt(props.getProperty(HoodieHBaseIndexConfig.HBASE_ZKPORT_PROP));
+    return getInt(HoodieHBaseIndexConfig.HBASE_ZKPORT_PROP);
   }
 
   public String getHBaseZkZnodeParent() {
-    return props.getProperty(HoodieIndexConfig.HBASE_ZK_ZNODEPARENT);
+    return getString(HoodieHBaseIndexConfig.HBASE_ZK_ZNODEPARENT);
   }
 
   public String getHbaseTableName() {
-    return props.getProperty(HoodieHBaseIndexConfig.HBASE_TABLENAME_PROP);
+    return getString(HoodieHBaseIndexConfig.HBASE_TABLENAME_PROP);
   }
 
   public int getHbaseIndexGetBatchSize() {
-    return Integer.parseInt(props.getProperty(HoodieHBaseIndexConfig.HBASE_GET_BATCH_SIZE_PROP));
+    return getInt(HoodieHBaseIndexConfig.HBASE_GET_BATCH_SIZE_PROP);
   }
 
   public Boolean getHBaseIndexRollbackSync() {
-    return Boolean.parseBoolean(props.getProperty(HoodieHBaseIndexConfig.HBASE_INDEX_ROLLBACK_SYNC));
+    return getBoolean(HoodieHBaseIndexConfig.HBASE_INDEX_ROLLBACK_SYNC);
   }
 
   public int getHbaseIndexPutBatchSize() {
-    return Integer.parseInt(props.getProperty(HoodieHBaseIndexConfig.HBASE_PUT_BATCH_SIZE_PROP));
+    return getInt(HoodieHBaseIndexConfig.HBASE_PUT_BATCH_SIZE_PROP);
   }
 
   public Boolean getHbaseIndexPutBatchSizeAutoCompute() {
-    return Boolean.valueOf(props.getProperty(HoodieHBaseIndexConfig.HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP));
+    return getBoolean(HoodieHBaseIndexConfig.HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP);
   }
 
   public String getHBaseQPSResourceAllocatorClass() {
-    return props.getProperty(HoodieHBaseIndexConfig.HBASE_INDEX_QPS_ALLOCATOR_CLASS);
+    return getString(HoodieHBaseIndexConfig.HBASE_INDEX_QPS_ALLOCATOR_CLASS);
   }
 
   public String getHBaseQPSZKnodePath() {
-    return props.getProperty(HoodieHBaseIndexConfig.HBASE_ZK_PATH_QPS_ROOT);
+    return getString(HoodieHBaseIndexConfig.HBASE_ZK_PATH_QPS_ROOT);
   }
 
   public String getHBaseZkZnodeSessionTimeout() {
-    return props.getProperty(HoodieHBaseIndexConfig.HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS);
+    return getString(HoodieHBaseIndexConfig.HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS);
   }
 
   public String getHBaseZkZnodeConnectionTimeout() {
-    return props.getProperty(HoodieHBaseIndexConfig.HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS);
+    return getString(HoodieHBaseIndexConfig.HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS);
   }
 
   public boolean getHBaseIndexShouldComputeQPSDynamically() {
-    return Boolean.parseBoolean(props.getProperty(HoodieHBaseIndexConfig.HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY));
+    return getBoolean(HoodieHBaseIndexConfig.HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY);
   }
 
   public int getHBaseIndexDesiredPutsTime() {
-    return Integer.parseInt(props.getProperty(HoodieHBaseIndexConfig.HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS));
+    return getInt(HoodieHBaseIndexConfig.HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS);
   }
 
   public String getBloomFilterType() {
-    return props.getProperty(HoodieIndexConfig.BLOOM_INDEX_FILTER_TYPE);
+    return getString(HoodieIndexConfig.BLOOM_INDEX_FILTER_TYPE);
   }
 
   public int getDynamicBloomFilterMaxNumEntries() {
-    return Integer.parseInt(props.getProperty(HoodieIndexConfig.HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES));
+    return getInt(HoodieIndexConfig.HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES);
   }
 
   /**
@@ -671,15 +815,15 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
    * the jobs would be (0.17) 1/6, 0.33 (2/6) and 0.5 (3/6) respectively.
    */
   public float getHbaseIndexQPSFraction() {
-    return Float.parseFloat(props.getProperty(HoodieHBaseIndexConfig.HBASE_QPS_FRACTION_PROP));
+    return getFloat(HoodieHBaseIndexConfig.HBASE_QPS_FRACTION_PROP);
   }
 
   public float getHBaseIndexMinQPSFraction() {
-    return Float.parseFloat(props.getProperty(HoodieHBaseIndexConfig.HBASE_MIN_QPS_FRACTION_PROP));
+    return getFloat(HoodieHBaseIndexConfig.HBASE_MIN_QPS_FRACTION_PROP);
   }
 
   public float getHBaseIndexMaxQPSFraction() {
-    return Float.parseFloat(props.getProperty(HoodieHBaseIndexConfig.HBASE_MAX_QPS_FRACTION_PROP));
+    return getFloat(HoodieHBaseIndexConfig.HBASE_MAX_QPS_FRACTION_PROP);
   }
 
   /**
@@ -687,239 +831,240 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
    * Hoodie jobs to an Hbase Region Server
    */
   public int getHbaseIndexMaxQPSPerRegionServer() {
-    return Integer.parseInt(props.getProperty(HoodieHBaseIndexConfig.HBASE_MAX_QPS_PER_REGION_SERVER_PROP));
+    return getInt(HoodieHBaseIndexConfig.HBASE_MAX_QPS_PER_REGION_SERVER_PROP);
   }
 
   public boolean getHbaseIndexUpdatePartitionPath() {
-    return Boolean.parseBoolean(props.getProperty(HoodieHBaseIndexConfig.HBASE_INDEX_UPDATE_PARTITION_PATH));
+    return getBoolean(HoodieHBaseIndexConfig.HBASE_INDEX_UPDATE_PARTITION_PATH);
   }
 
   public int getBloomIndexParallelism() {
-    return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PARALLELISM_PROP));
+    return getInt(HoodieIndexConfig.BLOOM_INDEX_PARALLELISM_PROP);
   }
 
   public boolean getBloomIndexPruneByRanges() {
-    return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PRUNE_BY_RANGES_PROP));
+    return getBoolean(HoodieIndexConfig.BLOOM_INDEX_PRUNE_BY_RANGES_PROP);
   }
 
   public boolean getBloomIndexUseCaching() {
-    return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_USE_CACHING_PROP));
+    return getBoolean(HoodieIndexConfig.BLOOM_INDEX_USE_CACHING_PROP);
   }
 
   public boolean useBloomIndexTreebasedFilter() {
-    return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_TREE_BASED_FILTER_PROP));
+    return getBoolean(HoodieIndexConfig.BLOOM_INDEX_TREE_BASED_FILTER_PROP);
   }
 
   public boolean useBloomIndexBucketizedChecking() {
-    return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_BUCKETIZED_CHECKING_PROP));
+    return getBoolean(HoodieIndexConfig.BLOOM_INDEX_BUCKETIZED_CHECKING_PROP);
   }
 
   public int getBloomIndexKeysPerBucket() {
-    return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_KEYS_PER_BUCKET_PROP));
+    return getInt(HoodieIndexConfig.BLOOM_INDEX_KEYS_PER_BUCKET_PROP);
   }
 
   public boolean getBloomIndexUpdatePartitionPath() {
-    return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_UPDATE_PARTITION_PATH));
+    return getBoolean(HoodieIndexConfig.BLOOM_INDEX_UPDATE_PARTITION_PATH);
   }
 
   public int getSimpleIndexParallelism() {
-    return Integer.parseInt(props.getProperty(HoodieIndexConfig.SIMPLE_INDEX_PARALLELISM_PROP));
+    return getInt(HoodieIndexConfig.SIMPLE_INDEX_PARALLELISM_PROP);
   }
 
   public boolean getSimpleIndexUseCaching() {
-    return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.SIMPLE_INDEX_USE_CACHING_PROP));
+    return getBoolean(HoodieIndexConfig.SIMPLE_INDEX_USE_CACHING_PROP);
   }
 
   public int getGlobalSimpleIndexParallelism() {
-    return Integer.parseInt(props.getProperty(HoodieIndexConfig.GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP));
+    return getInt(HoodieIndexConfig.GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP);
   }
 
   public boolean getGlobalSimpleIndexUpdatePartitionPath() {
-    return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.SIMPLE_INDEX_UPDATE_PARTITION_PATH));
+    return getBoolean(HoodieIndexConfig.SIMPLE_INDEX_UPDATE_PARTITION_PATH);
   }
 
   /**
    * storage properties.
    */
   public long getParquetMaxFileSize() {
-    return Long.parseLong(props.getProperty(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES));
+    return getLong(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES);
   }
 
   public int getParquetBlockSize() {
-    return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_BLOCK_SIZE_BYTES));
+    return getInt(HoodieStorageConfig.PARQUET_BLOCK_SIZE_BYTES);
   }
 
   public int getParquetPageSize() {
-    return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_PAGE_SIZE_BYTES));
+    return getInt(HoodieStorageConfig.PARQUET_PAGE_SIZE_BYTES);
   }
 
   public int getLogFileDataBlockMaxSize() {
-    return Integer.parseInt(props.getProperty(HoodieStorageConfig.LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES));
+    return getInt(HoodieStorageConfig.LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES);
   }
 
   public int getLogFileMaxSize() {
-    return Integer.parseInt(props.getProperty(HoodieStorageConfig.LOGFILE_SIZE_MAX_BYTES));
+    return getInt(HoodieStorageConfig.LOGFILE_SIZE_MAX_BYTES);
   }
 
   public double getParquetCompressionRatio() {
-    return Double.parseDouble(props.getProperty(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO));
+    return getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO);
   }
 
   public CompressionCodecName getParquetCompressionCodec() {
-    return CompressionCodecName.fromConf(props.getProperty(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC));
+    return CompressionCodecName.fromConf(getString(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC));
   }
 
   public double getLogFileToParquetCompressionRatio() {
-    return Double.parseDouble(props.getProperty(HoodieStorageConfig.LOGFILE_TO_PARQUET_COMPRESSION_RATIO));
+    return getDouble(HoodieStorageConfig.LOGFILE_TO_PARQUET_COMPRESSION_RATIO);
   }
 
   public long getHFileMaxFileSize() {
-    return Long.parseLong(props.getProperty(HoodieStorageConfig.HFILE_FILE_MAX_BYTES));
+    return getLong(HoodieStorageConfig.HFILE_FILE_MAX_BYTES);
   }
 
   public int getHFileBlockSize() {
-    return Integer.parseInt(props.getProperty(HoodieStorageConfig.HFILE_BLOCK_SIZE_BYTES));
+    return getInt(HoodieStorageConfig.HFILE_BLOCK_SIZE_BYTES);
   }
 
   public Compression.Algorithm getHFileCompressionAlgorithm() {
-    return Compression.Algorithm.valueOf(props.getProperty(HoodieStorageConfig.HFILE_COMPRESSION_ALGORITHM));
+    return Compression.Algorithm.valueOf(getString(HoodieStorageConfig.HFILE_COMPRESSION_ALGORITHM));
   }
 
   public long getOrcMaxFileSize() {
-    return Long.parseLong(props.getProperty(HoodieStorageConfig.ORC_FILE_MAX_BYTES));
+    return getLong(HoodieStorageConfig.ORC_FILE_MAX_BYTES);
   }
 
   public int getOrcStripeSize() {
-    return Integer.parseInt(props.getProperty(HoodieStorageConfig.ORC_STRIPE_SIZE));
+    return getInt(HoodieStorageConfig.ORC_STRIPE_SIZE);
   }
 
   public int getOrcBlockSize() {
-    return Integer.parseInt(props.getProperty(HoodieStorageConfig.ORC_BLOCK_SIZE));
+    return getInt(HoodieStorageConfig.ORC_BLOCK_SIZE);
   }
 
   public CompressionKind getOrcCompressionCodec() {
-    return CompressionKind.valueOf(props.getProperty(HoodieStorageConfig.ORC_COMPRESSION_CODEC));
+    return CompressionKind.valueOf(getString(HoodieStorageConfig.ORC_COMPRESSION_CODEC));
   }
 
   /**
    * metrics properties.
    */
   public boolean isMetricsOn() {
-    return Boolean.parseBoolean(props.getProperty(HoodieMetricsConfig.METRICS_ON));
+    return getBoolean(HoodieMetricsConfig.METRICS_ON);
   }
 
   public boolean isExecutorMetricsEnabled() {
-    return Boolean.parseBoolean(props.getProperty(HoodieMetricsConfig.ENABLE_EXECUTOR_METRICS, "false"));
+    return Boolean.parseBoolean(
+        getStringOrDefault(HoodieMetricsConfig.ENABLE_EXECUTOR_METRICS, "false"));
   }
 
   public MetricsReporterType getMetricsReporterType() {
-    return MetricsReporterType.valueOf(props.getProperty(HoodieMetricsConfig.METRICS_REPORTER_TYPE));
+    return MetricsReporterType.valueOf(getString(HoodieMetricsConfig.METRICS_REPORTER_TYPE));
   }
 
   public String getGraphiteServerHost() {
-    return props.getProperty(HoodieMetricsConfig.GRAPHITE_SERVER_HOST);
+    return getString(HoodieMetricsConfig.GRAPHITE_SERVER_HOST);
   }
 
   public int getGraphiteServerPort() {
-    return Integer.parseInt(props.getProperty(HoodieMetricsConfig.GRAPHITE_SERVER_PORT));
+    return getInt(HoodieMetricsConfig.GRAPHITE_SERVER_PORT);
   }
 
   public String getGraphiteMetricPrefix() {
-    return props.getProperty(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX);
+    return getString(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX);
   }
 
   public String getJmxHost() {
-    return props.getProperty(HoodieMetricsConfig.JMX_HOST);
+    return getString(HoodieMetricsConfig.JMX_HOST);
   }
 
   public String getJmxPort() {
-    return props.getProperty(HoodieMetricsConfig.JMX_PORT);
+    return getString(HoodieMetricsConfig.JMX_PORT);
   }
 
   public int getDatadogReportPeriodSeconds() {
-    return Integer.parseInt(props.getProperty(HoodieMetricsDatadogConfig.DATADOG_REPORT_PERIOD_SECONDS));
+    return getInt(HoodieMetricsDatadogConfig.DATADOG_REPORT_PERIOD_SECONDS);
   }
 
   public ApiSite getDatadogApiSite() {
-    return ApiSite.valueOf(props.getProperty(HoodieMetricsDatadogConfig.DATADOG_API_SITE));
+    return ApiSite.valueOf(getString(HoodieMetricsDatadogConfig.DATADOG_API_SITE));
   }
 
   public String getDatadogApiKey() {
-    if (props.containsKey(HoodieMetricsDatadogConfig.DATADOG_API_KEY)) {
-      return props.getProperty(HoodieMetricsDatadogConfig.DATADOG_API_KEY);
+    if (props.containsKey(HoodieMetricsDatadogConfig.DATADOG_API_KEY.key())) {
+      return getString(HoodieMetricsDatadogConfig.DATADOG_API_KEY);
     } else {
       Supplier<String> apiKeySupplier = ReflectionUtils.loadClass(
-          props.getProperty(HoodieMetricsDatadogConfig.DATADOG_API_KEY_SUPPLIER));
+          getString(HoodieMetricsDatadogConfig.DATADOG_API_KEY_SUPPLIER));
       return apiKeySupplier.get();
     }
   }
 
   public boolean getDatadogApiKeySkipValidation() {
-    return Boolean.parseBoolean(props.getProperty(HoodieMetricsDatadogConfig.DATADOG_API_KEY_SKIP_VALIDATION));
+    return getBoolean(HoodieMetricsDatadogConfig.DATADOG_API_KEY_SKIP_VALIDATION);
   }
 
   public int getDatadogApiTimeoutSeconds() {
-    return Integer.parseInt(props.getProperty(HoodieMetricsDatadogConfig.DATADOG_API_TIMEOUT_SECONDS));
+    return getInt(HoodieMetricsDatadogConfig.DATADOG_API_TIMEOUT_SECONDS);
   }
 
   public String getDatadogMetricPrefix() {
-    return props.getProperty(HoodieMetricsDatadogConfig.DATADOG_METRIC_PREFIX);
+    return getString(HoodieMetricsDatadogConfig.DATADOG_METRIC_PREFIX);
   }
 
   public String getDatadogMetricHost() {
-    return props.getProperty(HoodieMetricsDatadogConfig.DATADOG_METRIC_HOST);
+    return getString(HoodieMetricsDatadogConfig.DATADOG_METRIC_HOST);
   }
 
   public List<String> getDatadogMetricTags() {
-    return Arrays.stream(props.getProperty(
+    return Arrays.stream(getStringOrDefault(
         HoodieMetricsDatadogConfig.DATADOG_METRIC_TAGS, ",").split("\\s*,\\s*")).collect(Collectors.toList());
   }
 
   public String getMetricReporterClassName() {
-    return props.getProperty(HoodieMetricsConfig.METRICS_REPORTER_CLASS);
+    return getString(HoodieMetricsConfig.METRICS_REPORTER_CLASS);
   }
 
   public int getPrometheusPort() {
-    return Integer.parseInt(props.getProperty(HoodieMetricsPrometheusConfig.PROMETHEUS_PORT));
+    return getInt(HoodieMetricsPrometheusConfig.PROMETHEUS_PORT);
   }
 
   public String getPushGatewayHost() {
-    return props.getProperty(HoodieMetricsPrometheusConfig.PUSHGATEWAY_HOST);
+    return getString(HoodieMetricsPrometheusConfig.PUSHGATEWAY_HOST);
   }
 
   public int getPushGatewayPort() {
-    return Integer.parseInt(props.getProperty(HoodieMetricsPrometheusConfig.PUSHGATEWAY_PORT));
+    return getInt(HoodieMetricsPrometheusConfig.PUSHGATEWAY_PORT);
   }
 
   public int getPushGatewayReportPeriodSeconds() {
-    return Integer.parseInt(props.getProperty(HoodieMetricsPrometheusConfig.PUSHGATEWAY_REPORT_PERIOD_SECONDS));
+    return getInt(HoodieMetricsPrometheusConfig.PUSHGATEWAY_REPORT_PERIOD_SECONDS);
   }
 
   public boolean getPushGatewayDeleteOnShutdown() {
-    return Boolean.parseBoolean(props.getProperty(HoodieMetricsPrometheusConfig.PUSHGATEWAY_DELETE_ON_SHUTDOWN));
+    return getBoolean(HoodieMetricsPrometheusConfig.PUSHGATEWAY_DELETE_ON_SHUTDOWN);
   }
 
   public String getPushGatewayJobName() {
-    return props.getProperty(HoodieMetricsPrometheusConfig.PUSHGATEWAY_JOB_NAME);
+    return getString(HoodieMetricsPrometheusConfig.PUSHGATEWAY_JOB_NAME);
   }
 
   public boolean getPushGatewayRandomJobNameSuffix() {
-    return Boolean.parseBoolean(props.getProperty(HoodieMetricsPrometheusConfig.PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX));
+    return getBoolean(HoodieMetricsPrometheusConfig.PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX);
   }
 
   /**
    * memory configs.
    */
   public int getMaxDFSStreamBufferSize() {
-    return Integer.parseInt(props.getProperty(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP));
+    return getInt(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP);
   }
 
   public String getSpillableMapBasePath() {
-    return props.getProperty(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH_PROP);
+    return getString(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH_PROP);
   }
 
   public double getWriteStatusFailureFraction() {
-    return Double.parseDouble(props.getProperty(HoodieMemoryConfig.WRITESTATUS_FAILURE_FRACTION_PROP));
+    return getDouble(HoodieMemoryConfig.WRITESTATUS_FAILURE_FRACTION_PROP);
   }
 
   public ConsistencyGuardConfig getConsistencyGuardConfig() {
@@ -958,55 +1103,55 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
    * Commit call back configs.
    */
   public boolean writeCommitCallbackOn() {
-    return Boolean.parseBoolean(props.getProperty(HoodieWriteCommitCallbackConfig.CALLBACK_ON));
+    return getBoolean(HoodieWriteCommitCallbackConfig.CALLBACK_ON);
   }
 
   public String getCallbackClass() {
-    return props.getProperty(HoodieWriteCommitCallbackConfig.CALLBACK_CLASS_PROP);
+    return getString(HoodieWriteCommitCallbackConfig.CALLBACK_CLASS_PROP);
   }
 
   public String getBootstrapSourceBasePath() {
-    return props.getProperty(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP);
+    return getString(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP);
   }
 
   public String getBootstrapModeSelectorClass() {
-    return props.getProperty(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR);
+    return getString(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR);
   }
 
   public String getFullBootstrapInputProvider() {
-    return props.getProperty(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER);
+    return getString(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER);
   }
 
   public String getBootstrapKeyGeneratorClass() {
-    return props.getProperty(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS);
+    return getString(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS);
   }
 
   public String getBootstrapModeSelectorRegex() {
-    return props.getProperty(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR_REGEX);
+    return getString(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR_REGEX);
   }
 
   public BootstrapMode getBootstrapModeForRegexMatch() {
-    return BootstrapMode.valueOf(props.getProperty(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR_REGEX_MODE));
+    return BootstrapMode.valueOf(getString(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR_REGEX_MODE));
   }
 
   public String getBootstrapPartitionPathTranslatorClass() {
-    return props.getProperty(HoodieBootstrapConfig.BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS);
+    return getString(HoodieBootstrapConfig.BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS);
   }
 
   public int getBootstrapParallelism() {
-    return Integer.parseInt(props.getProperty(HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM));
+    return getInt(HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM);
   }
 
   public Long getMaxMemoryPerPartitionMerge() {
-    return Long.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP));
+    return getLong(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP);
   }
 
   public Long getHoodieClientHeartbeatIntervalInMs() {
-    return Long.valueOf(props.getProperty(CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP));
+    return getLong(CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP);
   }
 
   public Integer getHoodieClientHeartbeatTolerableMisses() {
-    return Integer.valueOf(props.getProperty(CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP));
+    return getInt(CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP);
   }
 
   /**
@@ -1021,27 +1166,27 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
   }
 
   public int getMetadataInsertParallelism() {
-    return Integer.parseInt(props.getProperty(HoodieMetadataConfig.METADATA_INSERT_PARALLELISM_PROP));
+    return getInt(HoodieMetadataConfig.METADATA_INSERT_PARALLELISM_PROP);
   }
 
   public int getMetadataCompactDeltaCommitMax() {
-    return Integer.parseInt(props.getProperty(HoodieMetadataConfig.METADATA_COMPACT_NUM_DELTA_COMMITS_PROP));
+    return getInt(HoodieMetadataConfig.METADATA_COMPACT_NUM_DELTA_COMMITS_PROP);
   }
 
   public boolean isMetadataAsyncClean() {
-    return Boolean.parseBoolean(props.getProperty(HoodieMetadataConfig.METADATA_ASYNC_CLEAN_PROP));
+    return getBoolean(HoodieMetadataConfig.METADATA_ASYNC_CLEAN_PROP);
   }
 
   public int getMetadataMaxCommitsToKeep() {
-    return Integer.parseInt(props.getProperty(HoodieMetadataConfig.MAX_COMMITS_TO_KEEP_PROP));
+    return getInt(HoodieMetadataConfig.MAX_COMMITS_TO_KEEP_PROP);
   }
 
   public int getMetadataMinCommitsToKeep() {
-    return Integer.parseInt(props.getProperty(HoodieMetadataConfig.MIN_COMMITS_TO_KEEP_PROP));
+    return getInt(HoodieMetadataConfig.MIN_COMMITS_TO_KEEP_PROP);
   }
 
   public int getMetadataCleanerCommitsRetained() {
-    return Integer.parseInt(props.getProperty(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED_PROP));
+    return getInt(HoodieMetadataConfig.CLEANER_COMMITS_RETAINED_PROP);
   }
 
   /**
@@ -1050,27 +1195,27 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
    */
 
   public String getLockProviderClass() {
-    return props.getProperty(HoodieLockConfig.LOCK_PROVIDER_CLASS_PROP);
+    return getString(HoodieLockConfig.LOCK_PROVIDER_CLASS_PROP);
   }
 
   public String getLockHiveDatabaseName() {
-    return props.getProperty(HIVE_DATABASE_NAME_PROP);
+    return getString(HoodieLockConfig.HIVE_DATABASE_NAME_PROP);
   }
 
   public String getLockHiveTableName() {
-    return props.getProperty(HIVE_TABLE_NAME_PROP);
+    return getString(HoodieLockConfig.HIVE_TABLE_NAME_PROP);
   }
 
   public ConflictResolutionStrategy getWriteConflictResolutionStrategy() {
-    return ReflectionUtils.loadClass(props.getProperty(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP));
+    return ReflectionUtils.loadClass(getString(HoodieLockConfig.WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP));
   }
 
   public Long getLockAcquireWaitTimeoutInMs() {
-    return Long.valueOf(props.getProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP));
+    return getLong(HoodieLockConfig.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP);
   }
 
   public WriteConcurrencyMode getWriteConcurrencyMode() {
-    return WriteConcurrencyMode.fromValue(props.getProperty(WRITE_CONCURRENCY_MODE_PROP));
+    return WriteConcurrencyMode.fromValue(getString(WRITE_CONCURRENCY_MODE_PROP));
   }
 
   public Boolean inlineTableServices() {
@@ -1078,12 +1223,12 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
   }
 
   public String getWriteMetaKeyPrefixes() {
-    return props.getProperty(WRITE_META_KEY_PREFIXES_PROP);
+    return getString(WRITE_META_KEY_PREFIXES_PROP);
   }
 
   public static class Builder {
 
-    protected final Properties props = new Properties();
+    protected final HoodieWriteConfig writeConfig = new HoodieWriteConfig();
     protected EngineType engineType = EngineType.SPARK;
     private boolean isIndexConfigSet = false;
     private boolean isStorageConfigSet = false;
@@ -1106,14 +1251,14 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
 
     public Builder fromFile(File propertiesFile) throws IOException {
       try (FileReader reader = new FileReader(propertiesFile)) {
-        this.props.load(reader);
+        this.writeConfig.getProps().load(reader);
         return this;
       }
     }
 
     public Builder fromInputStream(InputStream inputStream) throws IOException {
       try {
-        this.props.load(inputStream);
+        this.writeConfig.getProps().load(inputStream);
         return this;
       } finally {
         inputStream.close();
@@ -1121,365 +1266,309 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
     }
 
     public Builder withProps(Map kvprops) {
-      props.putAll(kvprops);
+      writeConfig.getProps().putAll(kvprops);
       return this;
     }
 
     public Builder withPath(String basePath) {
-      props.setProperty(BASE_PATH_PROP, basePath);
+      writeConfig.setValue(BASE_PATH_PROP, basePath);
       return this;
     }
 
     public Builder withSchema(String schemaStr) {
-      props.setProperty(AVRO_SCHEMA, schemaStr);
+      writeConfig.setValue(AVRO_SCHEMA, schemaStr);
       return this;
     }
 
     public Builder withAvroSchemaValidate(boolean enable) {
-      props.setProperty(AVRO_SCHEMA_VALIDATE, String.valueOf(enable));
+      writeConfig.setValue(AVRO_SCHEMA_VALIDATE, String.valueOf(enable));
       return this;
     }
 
     public Builder forTable(String tableName) {
-      props.setProperty(TABLE_NAME, tableName);
+      writeConfig.setValue(TABLE_NAME, tableName);
       return this;
     }
 
     public Builder withPreCombineField(String preCombineField) {
-      props.setProperty(PRECOMBINE_FIELD_PROP, preCombineField);
+      writeConfig.setValue(PRECOMBINE_FIELD_PROP, preCombineField);
       return this;
     }
 
     public Builder withWritePayLoad(String payload) {
-      props.setProperty(WRITE_PAYLOAD_CLASS, payload);
+      writeConfig.setValue(WRITE_PAYLOAD_CLASS, payload);
       return this;
     }
 
     public Builder withKeyGenerator(String keyGeneratorClass) {
-      props.setProperty(KEYGENERATOR_CLASS_PROP, keyGeneratorClass);
+      writeConfig.setValue(KEYGENERATOR_CLASS_PROP, keyGeneratorClass);
       return this;
     }
 
     public Builder withTimelineLayoutVersion(int version) {
-      props.setProperty(TIMELINE_LAYOUT_VERSION, String.valueOf(version));
+      writeConfig.setValue(TIMELINE_LAYOUT_VERSION, String.valueOf(version));
       return this;
     }
 
     public Builder withBulkInsertParallelism(int bulkInsertParallelism) {
-      props.setProperty(BULKINSERT_PARALLELISM, String.valueOf(bulkInsertParallelism));
+      writeConfig.setValue(BULKINSERT_PARALLELISM, String.valueOf(bulkInsertParallelism));
       return this;
     }
 
     public Builder withUserDefinedBulkInsertPartitionerClass(String className) {
-      props.setProperty(BULKINSERT_USER_DEFINED_PARTITIONER_CLASS, className);
+      writeConfig.setValue(BULKINSERT_USER_DEFINED_PARTITIONER_CLASS, className);
       return this;
     }
 
     public Builder withDeleteParallelism(int parallelism) {
-      props.setProperty(DELETE_PARALLELISM, String.valueOf(parallelism));
+      writeConfig.setValue(DELETE_PARALLELISM, String.valueOf(parallelism));
       return this;
     }
 
     public Builder withParallelism(int insertShuffleParallelism, int upsertShuffleParallelism) {
-      props.setProperty(INSERT_PARALLELISM, String.valueOf(insertShuffleParallelism));
-      props.setProperty(UPSERT_PARALLELISM, String.valueOf(upsertShuffleParallelism));
+      writeConfig.setValue(INSERT_PARALLELISM, String.valueOf(insertShuffleParallelism));
+      writeConfig.setValue(UPSERT_PARALLELISM, String.valueOf(upsertShuffleParallelism));
       return this;
     }
 
     public Builder withRollbackParallelism(int rollbackParallelism) {
-      props.setProperty(ROLLBACK_PARALLELISM, String.valueOf(rollbackParallelism));
+      writeConfig.setValue(ROLLBACK_PARALLELISM, String.valueOf(rollbackParallelism));
       return this;
     }
 
     public Builder withRollbackUsingMarkers(boolean rollbackUsingMarkers) {
-      props.setProperty(ROLLBACK_USING_MARKERS, String.valueOf(rollbackUsingMarkers));
+      writeConfig.setValue(ROLLBACK_USING_MARKERS, String.valueOf(rollbackUsingMarkers));
       return this;
     }
 
     public Builder withWriteBufferLimitBytes(int writeBufferLimit) {
-      props.setProperty(WRITE_BUFFER_LIMIT_BYTES, String.valueOf(writeBufferLimit));
+      writeConfig.setValue(WRITE_BUFFER_LIMIT_BYTES, String.valueOf(writeBufferLimit));
       return this;
     }
 
     public Builder combineInput(boolean onInsert, boolean onUpsert) {
-      props.setProperty(COMBINE_BEFORE_INSERT_PROP, String.valueOf(onInsert));
-      props.setProperty(COMBINE_BEFORE_UPSERT_PROP, String.valueOf(onUpsert));
+      writeConfig.setValue(COMBINE_BEFORE_INSERT_PROP, String.valueOf(onInsert));
+      writeConfig.setValue(COMBINE_BEFORE_UPSERT_PROP, String.valueOf(onUpsert));
       return this;
     }
 
     public Builder combineDeleteInput(boolean onDelete) {
-      props.setProperty(COMBINE_BEFORE_DELETE_PROP, String.valueOf(onDelete));
+      writeConfig.setValue(COMBINE_BEFORE_DELETE_PROP, String.valueOf(onDelete));
       return this;
     }
 
     public Builder withWriteStatusStorageLevel(String level) {
-      props.setProperty(WRITE_STATUS_STORAGE_LEVEL, level);
+      writeConfig.setValue(WRITE_STATUS_STORAGE_LEVEL, level);
       return this;
     }
 
     public Builder withIndexConfig(HoodieIndexConfig indexConfig) {
-      props.putAll(indexConfig.getProps());
+      writeConfig.getProps().putAll(indexConfig.getProps());
       isIndexConfigSet = true;
       return this;
     }
 
     public Builder withStorageConfig(HoodieStorageConfig storageConfig) {
-      props.putAll(storageConfig.getProps());
+      writeConfig.getProps().putAll(storageConfig.getProps());
       isStorageConfigSet = true;
       return this;
     }
 
     public Builder withCompactionConfig(HoodieCompactionConfig compactionConfig) {
-      props.putAll(compactionConfig.getProps());
+      writeConfig.getProps().putAll(compactionConfig.getProps());
       isCompactionConfigSet = true;
       return this;
     }
 
     public Builder withClusteringConfig(HoodieClusteringConfig clusteringConfig) {
-      props.putAll(clusteringConfig.getProps());
+      writeConfig.getProps().putAll(clusteringConfig.getProps());
       isClusteringConfigSet = true;
       return this;
     }
 
     public Builder withLockConfig(HoodieLockConfig lockConfig) {
-      props.putAll(lockConfig.getProps());
+      writeConfig.getProps().putAll(lockConfig.getProps());
       isLockConfigSet = true;
       return this;
     }
 
     public Builder withMetricsConfig(HoodieMetricsConfig metricsConfig) {
-      props.putAll(metricsConfig.getProps());
+      writeConfig.getProps().putAll(metricsConfig.getProps());
       isMetricsConfigSet = true;
       return this;
     }
 
     public Builder withMemoryConfig(HoodieMemoryConfig memoryConfig) {
-      props.putAll(memoryConfig.getProps());
+      writeConfig.getProps().putAll(memoryConfig.getProps());
       isMemoryConfigSet = true;
       return this;
     }
 
     public Builder withBootstrapConfig(HoodieBootstrapConfig bootstrapConfig) {
-      props.putAll(bootstrapConfig.getProps());
+      writeConfig.getProps().putAll(bootstrapConfig.getProps());
       isBootstrapConfigSet = true;
       return this;
     }
 
     public Builder withPayloadConfig(HoodiePayloadConfig payloadConfig) {
-      props.putAll(payloadConfig.getProps());
+      writeConfig.getProps().putAll(payloadConfig.getProps());
       isPayloadConfigSet = true;
       return this;
     }
 
     public Builder withMetadataConfig(HoodieMetadataConfig metadataConfig) {
-      props.putAll(metadataConfig.getProps());
+      writeConfig.getProps().putAll(metadataConfig.getProps());
       isMetadataConfigSet = true;
       return this;
     }
 
     public Builder withAutoCommit(boolean autoCommit) {
-      props.setProperty(HOODIE_AUTO_COMMIT_PROP, String.valueOf(autoCommit));
+      writeConfig.setValue(HOODIE_AUTO_COMMIT_PROP, String.valueOf(autoCommit));
       return this;
     }
 
     public Builder withWriteStatusClass(Class<? extends WriteStatus> writeStatusClass) {
-      props.setProperty(HOODIE_WRITE_STATUS_CLASS_PROP, writeStatusClass.getName());
+      writeConfig.setValue(HOODIE_WRITE_STATUS_CLASS_PROP, writeStatusClass.getName());
       return this;
     }
 
     public Builder withFileSystemViewConfig(FileSystemViewStorageConfig viewStorageConfig) {
-      props.putAll(viewStorageConfig.getProps());
+      writeConfig.getProps().putAll(viewStorageConfig.getProps());
       isViewConfigSet = true;
       return this;
     }
 
     public Builder withConsistencyGuardConfig(ConsistencyGuardConfig consistencyGuardConfig) {
-      props.putAll(consistencyGuardConfig.getProps());
+      writeConfig.getProps().putAll(consistencyGuardConfig.getProps());
       isConsistencyGuardSet = true;
       return this;
     }
 
     public Builder withCallbackConfig(HoodieWriteCommitCallbackConfig callbackConfig) {
-      props.putAll(callbackConfig.getProps());
+      writeConfig.getProps().putAll(callbackConfig.getProps());
       isCallbackConfigSet = true;
       return this;
     }
 
     public Builder withFinalizeWriteParallelism(int parallelism) {
-      props.setProperty(FINALIZE_WRITE_PARALLELISM, String.valueOf(parallelism));
+      writeConfig.setValue(FINALIZE_WRITE_PARALLELISM, String.valueOf(parallelism));
       return this;
     }
 
     public Builder withMarkersDeleteParallelism(int parallelism) {
-      props.setProperty(MARKERS_DELETE_PARALLELISM, String.valueOf(parallelism));
+      writeConfig.setValue(MARKERS_DELETE_PARALLELISM, String.valueOf(parallelism));
       return this;
     }
 
     public Builder withEmbeddedTimelineServerEnabled(boolean enabled) {
-      props.setProperty(EMBEDDED_TIMELINE_SERVER_ENABLED, String.valueOf(enabled));
+      writeConfig.setValue(EMBEDDED_TIMELINE_SERVER_ENABLED, String.valueOf(enabled));
       return this;
     }
 
     public Builder withEmbeddedTimelineServerReuseEnabled(boolean enabled) {
-      props.setProperty(EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED, String.valueOf(enabled));
+      writeConfig.setValue(EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED, String.valueOf(enabled));
       return this;
     }
 
     public Builder withEmbeddedTimelineServerPort(int port) {
-      props.setProperty(EMBEDDED_TIMELINE_SERVER_PORT, String.valueOf(port));
+      writeConfig.setValue(EMBEDDED_TIMELINE_SERVER_PORT, String.valueOf(port));
       return this;
     }
 
     public Builder withBulkInsertSortMode(String mode) {
-      props.setProperty(BULKINSERT_SORT_MODE, mode);
+      writeConfig.setValue(BULKINSERT_SORT_MODE, mode);
       return this;
     }
 
     public Builder withAllowMultiWriteOnSameInstant(boolean allow) {
-      props.setProperty(ALLOW_MULTI_WRITE_ON_SAME_INSTANT, String.valueOf(allow));
+      writeConfig.setValue(ALLOW_MULTI_WRITE_ON_SAME_INSTANT, String.valueOf(allow));
       return this;
     }
 
     public Builder withExternalSchemaTrasformation(boolean enabled) {
-      props.setProperty(EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION, String.valueOf(enabled));
+      writeConfig.setValue(EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION, String.valueOf(enabled));
       return this;
     }
 
     public Builder withMergeDataValidationCheckEnabled(boolean enabled) {
-      props.setProperty(MERGE_DATA_VALIDATION_CHECK_ENABLED, String.valueOf(enabled));
+      writeConfig.setValue(MERGE_DATA_VALIDATION_CHECK_ENABLED, String.valueOf(enabled));
       return this;
     }
 
     public Builder withMergeAllowDuplicateOnInserts(boolean routeInsertsToNewFiles) {
-      props.setProperty(MERGE_ALLOW_DUPLICATE_ON_INSERTS, String.valueOf(routeInsertsToNewFiles));
+      writeConfig.setValue(MERGE_ALLOW_DUPLICATE_ON_INSERTS, String.valueOf(routeInsertsToNewFiles));
       return this;
     }
 
     public Builder withHeartbeatIntervalInMs(Integer heartbeatIntervalInMs) {
-      props.setProperty(CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP, String.valueOf(heartbeatIntervalInMs));
+      writeConfig.setValue(CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP, String.valueOf(heartbeatIntervalInMs));
       return this;
     }
 
     public Builder withHeartbeatTolerableMisses(Integer heartbeatTolerableMisses) {
-      props.setProperty(CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP, String.valueOf(heartbeatTolerableMisses));
+      writeConfig.setValue(CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP, String.valueOf(heartbeatTolerableMisses));
       return this;
     }
 
     public Builder withWriteConcurrencyMode(WriteConcurrencyMode concurrencyMode) {
-      props.setProperty(WRITE_CONCURRENCY_MODE_PROP, concurrencyMode.value());
+      writeConfig.setValue(WRITE_CONCURRENCY_MODE_PROP, concurrencyMode.value());
       return this;
     }
 
     public Builder withWriteMetaKeyPrefixes(String writeMetaKeyPrefixes) {
-      props.setProperty(WRITE_META_KEY_PREFIXES_PROP, writeMetaKeyPrefixes);
+      writeConfig.setValue(WRITE_META_KEY_PREFIXES_PROP, writeMetaKeyPrefixes);
       return this;
     }
 
     public Builder withProperties(Properties properties) {
-      this.props.putAll(properties);
+      this.writeConfig.getProps().putAll(properties);
       return this;
     }
 
     protected void setDefaults() {
       // Check for mandatory properties
-      setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM, DEFAULT_PARALLELISM);
-      setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM), BULKINSERT_PARALLELISM,
-          DEFAULT_PARALLELISM);
-      setDefaultOnCondition(props, !props.containsKey(UPSERT_PARALLELISM), UPSERT_PARALLELISM, DEFAULT_PARALLELISM);
-      setDefaultOnCondition(props, !props.containsKey(DELETE_PARALLELISM), DELETE_PARALLELISM, DEFAULT_PARALLELISM);
-
-      setDefaultOnCondition(props, !props.containsKey(ROLLBACK_PARALLELISM), ROLLBACK_PARALLELISM,
-          DEFAULT_ROLLBACK_PARALLELISM);
-      setDefaultOnCondition(props, !props.containsKey(KEYGENERATOR_TYPE_PROP),
-          KEYGENERATOR_TYPE_PROP, DEFAULT_KEYGENERATOR_TYPE);
-      setDefaultOnCondition(props, !props.containsKey(WRITE_PAYLOAD_CLASS),
-          WRITE_PAYLOAD_CLASS, DEFAULT_WRITE_PAYLOAD_CLASS);
-      setDefaultOnCondition(props, !props.containsKey(ROLLBACK_USING_MARKERS), ROLLBACK_USING_MARKERS,
-          DEFAULT_ROLLBACK_USING_MARKERS);
-      setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP), COMBINE_BEFORE_INSERT_PROP,
-          DEFAULT_COMBINE_BEFORE_INSERT);
-      setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_UPSERT_PROP), COMBINE_BEFORE_UPSERT_PROP,
-          DEFAULT_COMBINE_BEFORE_UPSERT);
-      setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_DELETE_PROP), COMBINE_BEFORE_DELETE_PROP,
-          DEFAULT_COMBINE_BEFORE_DELETE);
-      setDefaultOnCondition(props, !props.containsKey(ALLOW_MULTI_WRITE_ON_SAME_INSTANT),
-          ALLOW_MULTI_WRITE_ON_SAME_INSTANT, DEFAULT_ALLOW_MULTI_WRITE_ON_SAME_INSTANT);
-      setDefaultOnCondition(props, !props.containsKey(WRITE_STATUS_STORAGE_LEVEL), WRITE_STATUS_STORAGE_LEVEL,
-          DEFAULT_WRITE_STATUS_STORAGE_LEVEL);
-      setDefaultOnCondition(props, !props.containsKey(HOODIE_AUTO_COMMIT_PROP), HOODIE_AUTO_COMMIT_PROP,
-          DEFAULT_HOODIE_AUTO_COMMIT);
-      setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP), HOODIE_WRITE_STATUS_CLASS_PROP,
-          DEFAULT_HOODIE_WRITE_STATUS_CLASS);
-      setDefaultOnCondition(props, !props.containsKey(FINALIZE_WRITE_PARALLELISM), FINALIZE_WRITE_PARALLELISM,
-          DEFAULT_FINALIZE_WRITE_PARALLELISM);
-      setDefaultOnCondition(props, !props.containsKey(MARKERS_DELETE_PARALLELISM), MARKERS_DELETE_PARALLELISM,
-          DEFAULT_MARKERS_DELETE_PARALLELISM);
-      setDefaultOnCondition(props, !props.containsKey(EMBEDDED_TIMELINE_SERVER_ENABLED),
-          EMBEDDED_TIMELINE_SERVER_ENABLED, DEFAULT_EMBEDDED_TIMELINE_SERVER_ENABLED);
-      setDefaultOnCondition(props, !props.containsKey(EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED),
-          EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED, DEFAULT_EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED);
-      setDefaultOnCondition(props, !props.containsKey(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
-          INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS));
-      setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
-          MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS));
-      setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECKS_PROP), MAX_CONSISTENCY_CHECKS_PROP,
-          String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECKS));
-      setDefaultOnCondition(props, !props.containsKey(FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP),
-          FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP, DEFAULT_FAIL_ON_TIMELINE_ARCHIVING_ENABLED);
-      setDefaultOnCondition(props, !props.containsKey(AVRO_SCHEMA_VALIDATE), AVRO_SCHEMA_VALIDATE, DEFAULT_AVRO_SCHEMA_VALIDATE);
-      setDefaultOnCondition(props, !props.containsKey(BULKINSERT_SORT_MODE),
-          BULKINSERT_SORT_MODE, DEFAULT_BULKINSERT_SORT_MODE);
-      setDefaultOnCondition(props, !props.containsKey(MERGE_DATA_VALIDATION_CHECK_ENABLED),
-          MERGE_DATA_VALIDATION_CHECK_ENABLED, DEFAULT_MERGE_DATA_VALIDATION_CHECK_ENABLED);
-      setDefaultOnCondition(props, !props.containsKey(MERGE_ALLOW_DUPLICATE_ON_INSERTS),
-          MERGE_ALLOW_DUPLICATE_ON_INSERTS, DEFAULT_MERGE_ALLOW_DUPLICATE_ON_INSERTS);
-      setDefaultOnCondition(props, !props.containsKey(CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP),
-          CLIENT_HEARTBEAT_INTERVAL_IN_MS_PROP, String.valueOf(DEFAULT_CLIENT_HEARTBEAT_INTERVAL_IN_MS));
-      setDefaultOnCondition(props, !props.containsKey(CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP),
-          CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES_PROP, String.valueOf(DEFAULT_CLIENT_HEARTBEAT_NUM_TOLERABLE_MISSES));
-      setDefaultOnCondition(props, !props.containsKey(WRITE_CONCURRENCY_MODE_PROP),
-          WRITE_CONCURRENCY_MODE_PROP, DEFAULT_WRITE_CONCURRENCY_MODE);
-      setDefaultOnCondition(props, !props.containsKey(WRITE_META_KEY_PREFIXES_PROP),
-          WRITE_META_KEY_PREFIXES_PROP, DEFAULT_WRITE_META_KEY_PREFIXES);
+      writeConfig.setDefaults(HoodieWriteConfig.class.getName());
       // Make sure the props is propagated
-      setDefaultOnCondition(props, !isIndexConfigSet, HoodieIndexConfig.newBuilder().withEngineType(engineType).fromProperties(props).build());
-      setDefaultOnCondition(props, !isStorageConfigSet, HoodieStorageConfig.newBuilder().fromProperties(props).build());
-      setDefaultOnCondition(props, !isCompactionConfigSet,
-          HoodieCompactionConfig.newBuilder().fromProperties(props).build());
-      setDefaultOnCondition(props, !isClusteringConfigSet,
-          HoodieClusteringConfig.newBuilder().fromProperties(props).build());
-      setDefaultOnCondition(props, !isMetricsConfigSet, HoodieMetricsConfig.newBuilder().fromProperties(props).build());
-      setDefaultOnCondition(props, !isBootstrapConfigSet,
-          HoodieBootstrapConfig.newBuilder().fromProperties(props).build());
-      setDefaultOnCondition(props, !isMemoryConfigSet, HoodieMemoryConfig.newBuilder().fromProperties(props).build());
-      setDefaultOnCondition(props, !isViewConfigSet,
-          FileSystemViewStorageConfig.newBuilder().fromProperties(props).build());
-      setDefaultOnCondition(props, !isConsistencyGuardSet,
-          ConsistencyGuardConfig.newBuilder().fromProperties(props).build());
-      setDefaultOnCondition(props, !isCallbackConfigSet,
-          HoodieWriteCommitCallbackConfig.newBuilder().fromProperties(props).build());
-      setDefaultOnCondition(props, !isPayloadConfigSet,
-          HoodiePayloadConfig.newBuilder().fromProperties(props).build());
-      setDefaultOnCondition(props, !isMetadataConfigSet,
-          HoodieMetadataConfig.newBuilder().fromProperties(props).build());
-      setDefaultOnCondition(props, !isLockConfigSet,
-          HoodieLockConfig.newBuilder().fromProperties(props).build());
-
-      setDefaultOnCondition(props, !props.containsKey(EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION),
-          EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION, DEFAULT_EXTERNAL_RECORD_AND_SCHEMA_TRANSFORMATION);
-      setDefaultOnCondition(props, !props.containsKey(TIMELINE_LAYOUT_VERSION), TIMELINE_LAYOUT_VERSION,
-          String.valueOf(TimelineLayoutVersion.CURR_VERSION));
+      writeConfig.setDefaultOnCondition(
+          !isIndexConfigSet, HoodieIndexConfig.newBuilder().withEngineType(engineType).fromProperties(
+              writeConfig.getProps()).build());
+      writeConfig.setDefaultOnCondition(!isStorageConfigSet, HoodieStorageConfig.newBuilder().fromProperties(
+          writeConfig.getProps()).build());
+      writeConfig.setDefaultOnCondition(!isCompactionConfigSet,
+          HoodieCompactionConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
+      writeConfig.setDefaultOnCondition(!isClusteringConfigSet,
+          HoodieClusteringConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
+      writeConfig.setDefaultOnCondition(!isMetricsConfigSet, HoodieMetricsConfig.newBuilder().fromProperties(
+          writeConfig.getProps()).build());
+      writeConfig.setDefaultOnCondition(!isBootstrapConfigSet,
+          HoodieBootstrapConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
+      writeConfig.setDefaultOnCondition(!isMemoryConfigSet, HoodieMemoryConfig.newBuilder().fromProperties(
+          writeConfig.getProps()).build());
+      writeConfig.setDefaultOnCondition(!isViewConfigSet,
+          FileSystemViewStorageConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
+      writeConfig.setDefaultOnCondition(!isConsistencyGuardSet,
+          ConsistencyGuardConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
+      writeConfig.setDefaultOnCondition(!isCallbackConfigSet,
+          HoodieWriteCommitCallbackConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
+      writeConfig.setDefaultOnCondition(!isPayloadConfigSet,
+          HoodiePayloadConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
+      writeConfig.setDefaultOnCondition(!isMetadataConfigSet,
+          HoodieMetadataConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
+      writeConfig.setDefaultOnCondition(!isLockConfigSet,
+          HoodieLockConfig.newBuilder().fromProperties(writeConfig.getProps()).build());
+
+      writeConfig.setDefaultValue(TIMELINE_LAYOUT_VERSION, String.valueOf(TimelineLayoutVersion.CURR_VERSION));
 
     }
 
     private void validate() {
-      String layoutVersion = props.getProperty(TIMELINE_LAYOUT_VERSION);
+      String layoutVersion = writeConfig.getString(TIMELINE_LAYOUT_VERSION);
       // Ensure Layout Version is good
       new TimelineLayoutVersion(Integer.parseInt(layoutVersion));
-      Objects.requireNonNull(props.getProperty(BASE_PATH_PROP));
-      if (props.getProperty(WRITE_CONCURRENCY_MODE_PROP)
+      Objects.requireNonNull(writeConfig.getString(BASE_PATH_PROP));
+      if (writeConfig.getString(WRITE_CONCURRENCY_MODE_PROP)
           .equalsIgnoreCase(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL.name())) {
-        ValidationUtils.checkArgument(props.getProperty(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY_PROP)
+        ValidationUtils.checkArgument(writeConfig.getString(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY_PROP)
             != HoodieFailedWritesCleaningPolicy.EAGER.name(), "To enable optimistic concurrency control, set hoodie.cleaner.policy.failed.writes=LAZY");
       }
     }
@@ -1488,8 +1577,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
       setDefaults();
       validate();
       // Build WriteConfig at the end
-      HoodieWriteConfig config = new HoodieWriteConfig(engineType, props);
-      return config;
+      return new HoodieWriteConfig(engineType, writeConfig.getProps());
     }
   }
 }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
index cbd77c6..cb9dcbd 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java
@@ -196,7 +196,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
   protected void initializeIncomingRecordsMap() {
     try {
       // Load the new records in a map
-      long memoryForMerge = IOUtils.getMaxMemoryPerPartitionMerge(taskContextSupplier, config.getProps());
+      long memoryForMerge = IOUtils.getMaxMemoryPerPartitionMerge(taskContextSupplier, config);
       LOG.info("MaxMemoryPerPartitionMerge => " + memoryForMerge);
       this.keyToNewRecords = new ExternalSpillableMap<>(memoryForMerge, config.getSpillableMapBasePath(),
           new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(tableSchema));
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java
index 6b5891c..6215df5 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/IOUtils.java
@@ -18,15 +18,12 @@
 
 package org.apache.hudi.io;
 
+import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.common.engine.EngineProperty;
 import org.apache.hudi.common.engine.TaskContextSupplier;
 import org.apache.hudi.common.util.Option;
 
-import java.util.Properties;
-
 import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
-import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION;
-import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE;
 import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MIN_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
 import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FOR_COMPACTION_PROP;
 import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP;
@@ -56,19 +53,19 @@ public class IOUtils {
     }
   }
 
-  public static long getMaxMemoryPerPartitionMerge(TaskContextSupplier context, Properties properties) {
-    if (properties.containsKey(MAX_MEMORY_FOR_MERGE_PROP)) {
-      return Long.parseLong(properties.getProperty(MAX_MEMORY_FOR_MERGE_PROP));
+  public static long getMaxMemoryPerPartitionMerge(TaskContextSupplier context, HoodieConfig hoodieConfig) {
+    if (hoodieConfig.contains(MAX_MEMORY_FOR_MERGE_PROP)) {
+      return hoodieConfig.getLong(MAX_MEMORY_FOR_MERGE_PROP);
     }
-    String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE);
+    String fraction = hoodieConfig.getStringOrDefault(MAX_MEMORY_FRACTION_FOR_MERGE_PROP);
     return getMaxMemoryAllowedForMerge(context, fraction);
   }
 
-  public static long getMaxMemoryPerCompaction(TaskContextSupplier context, Properties properties) {
-    if (properties.containsKey(MAX_MEMORY_FOR_COMPACTION_PROP)) {
-      return Long.parseLong(properties.getProperty(MAX_MEMORY_FOR_COMPACTION_PROP));
+  public static long getMaxMemoryPerCompaction(TaskContextSupplier context, HoodieConfig hoodieConfig) {
+    if (hoodieConfig.contains(MAX_MEMORY_FOR_COMPACTION_PROP)) {
+      return hoodieConfig.getLong(MAX_MEMORY_FOR_COMPACTION_PROP);
     }
-    String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION);
+    String fraction = hoodieConfig.getStringOrDefault(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP);
     return getMaxMemoryAllowedForMerge(context, fraction);
   }
 }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java
index 8020be8..52c0309 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/BaseKeyGenerator.java
@@ -35,10 +35,10 @@ public abstract class BaseKeyGenerator extends KeyGenerator {
 
   protected BaseKeyGenerator(TypedProperties config) {
     super(config);
-    this.encodePartitionPath = config.getBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY,
-        Boolean.parseBoolean(KeyGeneratorOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL));
-    this.hiveStylePartitioning = config.getBoolean(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY,
-        Boolean.parseBoolean(KeyGeneratorOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL));
+    this.encodePartitionPath = config.getBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key(),
+        Boolean.parseBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY.defaultValue()));
+    this.hiveStylePartitioning = config.getBoolean(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key(),
+        Boolean.parseBoolean(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.defaultValue()));
   }
 
   /**
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java
index edc1ad9..bb73c13 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/ComplexAvroKeyGenerator.java
@@ -32,9 +32,9 @@ public class ComplexAvroKeyGenerator extends BaseKeyGenerator {
 
   public ComplexAvroKeyGenerator(TypedProperties props) {
     super(props);
-    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
+    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key())
         .split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
-    this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY)
+    this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key())
         .split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
   }
 
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/CustomAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/CustomAvroKeyGenerator.java
index 3b927c9..da678a2 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/CustomAvroKeyGenerator.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/CustomAvroKeyGenerator.java
@@ -55,8 +55,8 @@ public class CustomAvroKeyGenerator extends BaseKeyGenerator {
 
   public CustomAvroKeyGenerator(TypedProperties props) {
     super(props);
-    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY).split(",")).map(String::trim).collect(Collectors.toList());
-    this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY).split(",")).map(String::trim).collect(Collectors.toList());
+    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()).split(",")).map(String::trim).collect(Collectors.toList());
+    this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key()).split(",")).map(String::trim).collect(Collectors.toList());
   }
 
   @Override
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/GlobalAvroDeleteKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/GlobalAvroDeleteKeyGenerator.java
index b074a25..aac28f5 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/GlobalAvroDeleteKeyGenerator.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/GlobalAvroDeleteKeyGenerator.java
@@ -35,7 +35,7 @@ public class GlobalAvroDeleteKeyGenerator extends BaseKeyGenerator {
 
   public GlobalAvroDeleteKeyGenerator(TypedProperties config) {
     super(config);
-    this.recordKeyFields = Arrays.asList(config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY).split(","));
+    this.recordKeyFields = Arrays.asList(config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()).split(","));
   }
 
   @Override
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 385b479..e8fc9c2 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
@@ -164,7 +164,7 @@ public class KeyGenUtils {
    */
   public static KeyGenerator createKeyGeneratorByClassName(TypedProperties props) throws IOException {
     KeyGenerator keyGenerator = null;
-    String keyGeneratorClass = props.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP, null);
+    String keyGeneratorClass = props.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP.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/NonpartitionedAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/NonpartitionedAvroKeyGenerator.java
index 5b117c5..feb3820 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/NonpartitionedAvroKeyGenerator.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/NonpartitionedAvroKeyGenerator.java
@@ -36,7 +36,7 @@ public class NonpartitionedAvroKeyGenerator extends BaseKeyGenerator {
 
   public NonpartitionedAvroKeyGenerator(TypedProperties props) {
     super(props);
-    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
+    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key())
         .split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
     this.partitionPathFields = EMPTY_PARTITION_FIELD_LIST;
   }
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java
index 59fe6be..2dab3dc 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/SimpleAvroKeyGenerator.java
@@ -29,8 +29,8 @@ import java.util.Collections;
 public class SimpleAvroKeyGenerator extends BaseKeyGenerator {
 
   public SimpleAvroKeyGenerator(TypedProperties props) {
-    this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY),
-        props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY));
+    this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()),
+        props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key()));
   }
 
   SimpleAvroKeyGenerator(TypedProperties props, String partitionPathField) {
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java
index 8d1c1a0..16b0c0f 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/TimestampBasedAvroKeyGenerator.java
@@ -88,8 +88,8 @@ public class TimestampBasedAvroKeyGenerator extends SimpleAvroKeyGenerator {
   }
 
   public TimestampBasedAvroKeyGenerator(TypedProperties config) throws IOException {
-    this(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY),
-        config.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY));
+    this(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()),
+        config.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key()));
   }
 
   TimestampBasedAvroKeyGenerator(TypedProperties config, String partitionPathField) throws IOException {
@@ -119,8 +119,8 @@ public class TimestampBasedAvroKeyGenerator extends SimpleAvroKeyGenerator {
       default:
         timeUnit = null;
     }
-    this.encodePartitionPath = config.getBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY,
-        Boolean.parseBoolean(KeyGeneratorOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL));
+    this.encodePartitionPath = config.getBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key(),
+        Boolean.parseBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY.defaultValue()));
   }
 
   @Override
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java
index da567e0..ad67562 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/keygen/constant/KeyGeneratorOptions.java
@@ -18,24 +18,33 @@
 
 package org.apache.hudi.keygen.constant;
 
+import org.apache.hudi.common.config.ConfigProperty;
+
 public class KeyGeneratorOptions {
 
-  /**
-   * Flag to indicate whether to use Hive style partitioning.
-   * If set true, the names of partition folders follow <partition_column_name>=<partition_value> format.
-   * By default false (the names of partition folders are only partition values)
-   */
-  public static final String URL_ENCODE_PARTITIONING_OPT_KEY = "hoodie.datasource.write.partitionpath.urlencode";
-  public static final String DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL = "false";
-  public static final String HIVE_STYLE_PARTITIONING_OPT_KEY = "hoodie.datasource.write.hive_style_partitioning";
-  public static final String DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL = "false";
-
-  /**
-   * Record key field. Value to be used as the `recordKey` component of `HoodieKey`. Actual value
-   * will be obtained by invoking .toString() on the field value. Nested fields can be specified using
-   * the dot notation eg: `a.b.c`
-   */
-  public static final String RECORDKEY_FIELD_OPT_KEY = "hoodie.datasource.write.recordkey.field";
-  public static final String PARTITIONPATH_FIELD_OPT_KEY = "hoodie.datasource.write.partitionpath.field";
+  public static final ConfigProperty<String> URL_ENCODE_PARTITIONING_OPT_KEY = ConfigProperty
+      .key("hoodie.datasource.write.partitionpath.urlencode")
+      .defaultValue("false")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HIVE_STYLE_PARTITIONING_OPT_KEY = ConfigProperty
+      .key("hoodie.datasource.write.hive_style_partitioning")
+      .defaultValue("false")
+      .withDocumentation("Flag to indicate whether to use Hive style partitioning.\n"
+          + "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 ConfigProperty<String> RECORDKEY_FIELD_OPT_KEY = ConfigProperty
+      .key("hoodie.datasource.write.recordkey.field")
+      .defaultValue("uuid")
+      .withDocumentation("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
+          + "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using\n"
+          + "the dot notation eg: `a.b.c`");
+
+  public static final ConfigProperty<String> PARTITIONPATH_FIELD_OPT_KEY = ConfigProperty
+      .key("hoodie.datasource.write.partitionpath.field")
+      .defaultValue("partitionpath")
+      .withDocumentation("Partition path field. Value to be used at the partitionPath component of HoodieKey. "
+          + "Actual value ontained by invoking .toString()");
 }
 
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 bdbf8b6..a9e9dbb 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
@@ -50,7 +50,7 @@ public class HoodieAvroKeyGeneratorFactory {
   private static KeyGenerator createAvroKeyGeneratorByType(TypedProperties props) throws IOException {
     // Use KeyGeneratorType.SIMPLE as default keyGeneratorType
     String keyGeneratorType =
-        props.getString(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP, KeyGeneratorType.SIMPLE.name());
+        props.getString(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP.key(), KeyGeneratorType.SIMPLE.name());
 
     KeyGeneratorType keyGeneratorTypeEnum;
     try {
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 034465d..4f1d79b 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.DEFAULT_ARCHIVELOG_FOLDER;
+import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP;
 import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX;
 import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP;
 
@@ -295,7 +295,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
     HoodieTableMetaClient.withPropertyBuilder()
       .setTableType(HoodieTableType.MERGE_ON_READ)
       .setTableName(tableName)
-      .setArchiveLogFolder(DEFAULT_ARCHIVELOG_FOLDER)
+      .setArchiveLogFolder(HOODIE_ARCHIVELOG_FOLDER_PROP.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 0ff2093..a90a07f 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
@@ -111,7 +111,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.DEFAULT_VIEW_SPILLABLE_DIR);
+        FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue());
 
     this.viewManager = FileSystemViewManager.createViewManager(context, config.getMetadataConfig(), config.getViewStorageConfig(), () -> 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 e2105a7..35dbd9e 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
@@ -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.useFileListingMetadata()) {
       try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(table.getContext(), config.getMetadataConfig(),
-          config.getBasePath(), FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR)) {
+          config.getBasePath(), FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue())) {
         Option<String> lastSyncedInstantTime = tableMetadata.getSyncedInstantTime();
 
         if (lastSyncedInstantTime.isPresent()) {
diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java
index 67bc1c2..7719948 100644
--- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java
+++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/AbstractUpgradeDowngrade.java
@@ -122,7 +122,7 @@ public abstract class AbstractUpgradeDowngrade {
 
     // Write out the current version in hoodie.properties.updated file
     metaClient.getTableConfig().setTableVersion(toVersion);
-    createUpdatedFile(metaClient.getTableConfig().getProperties());
+    createUpdatedFile(metaClient.getTableConfig().getProps());
 
     // because for different fs the fs.rename have different action,such as:
     // a) for hdfs : if propsFilePath already exist,fs.rename will not replace propsFilePath, but just return false
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java
index ed0b83b..c6a1527 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/FileSystemBasedLockProviderTestClass.java
@@ -30,9 +30,9 @@ import org.apache.hudi.exception.HoodieLockException;
 import java.io.IOException;
 import java.util.concurrent.TimeUnit;
 
-import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP;
+import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
 
 /**
  * This lock provider is used for testing purposes only. It provides a simple file system based lock using HDFS atomic
@@ -49,7 +49,7 @@ public class FileSystemBasedLockProviderTestClass implements LockProvider<String
 
   public FileSystemBasedLockProviderTestClass(final LockConfiguration lockConfiguration, final Configuration configuration) {
     this.lockConfiguration = lockConfiguration;
-    this.lockPath = lockConfiguration.getConfig().getString(FILESYSTEM_LOCK_PATH_PROP);
+    this.lockPath = lockConfiguration.getConfig().getString(FILESYSTEM_LOCK_PATH_PROP_KEY);
     this.fs = FSUtils.getFs(this.lockPath, configuration);
   }
 
@@ -75,8 +75,8 @@ public class FileSystemBasedLockProviderTestClass implements LockProvider<String
     try {
       int numRetries = 0;
       while (fs.exists(new Path(lockPath + "/" + LOCK_NAME))
-          && (numRetries <= lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP))) {
-        Thread.sleep(lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP));
+          && (numRetries <= lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY))) {
+        Thread.sleep(lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY));
       }
       acquireLock();
       return true;
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestZookeeperBasedLockProvider.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestZookeeperBasedLockProvider.java
index 5e36399..e9ab49a 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestZookeeperBasedLockProvider.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestZookeeperBasedLockProvider.java
@@ -34,12 +34,12 @@ import org.junit.jupiter.api.Test;
 import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 
-import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP;
-import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP;
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP_KEY;
+import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP_KEY;
 
 public class TestZookeeperBasedLockProvider {
 
@@ -64,14 +64,14 @@ public class TestZookeeperBasedLockProvider {
       }
     }
     Properties properties = new Properties();
-    properties.setProperty(ZK_BASE_PATH_PROP, basePath);
-    properties.setProperty(ZK_LOCK_KEY_PROP, key);
-    properties.setProperty(ZK_CONNECT_URL_PROP, server.getConnectString());
-    properties.setProperty(ZK_BASE_PATH_PROP, server.getTempDirectory().getAbsolutePath());
-    properties.setProperty(ZK_SESSION_TIMEOUT_MS_PROP, "10000");
-    properties.setProperty(ZK_CONNECTION_TIMEOUT_MS_PROP, "10000");
-    properties.setProperty(ZK_LOCK_KEY_PROP, "key");
-    properties.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP, "1000");
+    properties.setProperty(ZK_BASE_PATH_PROP_KEY, basePath);
+    properties.setProperty(ZK_LOCK_KEY_PROP_KEY, key);
+    properties.setProperty(ZK_CONNECT_URL_PROP_KEY, server.getConnectString());
+    properties.setProperty(ZK_BASE_PATH_PROP_KEY, server.getTempDirectory().getAbsolutePath());
+    properties.setProperty(ZK_SESSION_TIMEOUT_MS_PROP_KEY, "10000");
+    properties.setProperty(ZK_CONNECTION_TIMEOUT_MS_PROP_KEY, "10000");
+    properties.setProperty(ZK_LOCK_KEY_PROP_KEY, "key");
+    properties.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "1000");
     lockConfiguration = new LockConfiguration(properties);
   }
 
@@ -79,7 +79,7 @@ public class TestZookeeperBasedLockProvider {
   public void testAcquireLock() {
     ZookeeperBasedLockProvider zookeeperBasedLockProvider = new ZookeeperBasedLockProvider(lockConfiguration, client);
     Assertions.assertTrue(zookeeperBasedLockProvider.tryLock(lockConfiguration.getConfig()
-        .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS));
+        .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS));
     zookeeperBasedLockProvider.unlock();
   }
 
@@ -87,20 +87,20 @@ public class TestZookeeperBasedLockProvider {
   public void testUnLock() {
     ZookeeperBasedLockProvider zookeeperBasedLockProvider = new ZookeeperBasedLockProvider(lockConfiguration, client);
     Assertions.assertTrue(zookeeperBasedLockProvider.tryLock(lockConfiguration.getConfig()
-        .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS));
+        .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS));
     zookeeperBasedLockProvider.unlock();
     zookeeperBasedLockProvider.tryLock(lockConfiguration.getConfig()
-        .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS);
+        .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS);
   }
 
   @Test
   public void testReentrantLock() {
     ZookeeperBasedLockProvider zookeeperBasedLockProvider = new ZookeeperBasedLockProvider(lockConfiguration, client);
     Assertions.assertTrue(zookeeperBasedLockProvider.tryLock(lockConfiguration.getConfig()
-        .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS));
+        .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS));
     try {
       zookeeperBasedLockProvider.tryLock(lockConfiguration.getConfig()
-          .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP), TimeUnit.MILLISECONDS);
+          .getLong(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY), TimeUnit.MILLISECONDS);
       Assertions.fail();
     } catch (HoodieLockException e) {
       // expected
diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java
index 1f6a2a7..7661e1d 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/config/TestHoodieWriteConfig.java
@@ -40,9 +40,9 @@ public class TestHoodieWriteConfig {
   public void testPropertyLoading() throws IOException {
     Builder builder = HoodieWriteConfig.newBuilder().withPath("/tmp");
     Map<String, String> params = new HashMap<>(3);
-    params.put(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP, "1");
-    params.put(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP, "5");
-    params.put(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP, "2");
+    params.put(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP.key(), "1");
+    params.put(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP.key(), "5");
+    params.put(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP.key(), "2");
     ByteArrayOutputStream outStream = saveParamsIntoOutputStream(params);
     ByteArrayInputStream inputStream = new ByteArrayInputStream(outStream.toByteArray());
     try {
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 d69bc70..9fbe6d9 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
@@ -67,9 +67,9 @@ public class TestHoodieOrcReaderWriter {
   private HoodieOrcWriter createOrcWriter(Schema avroSchema) throws Exception {
     BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.00001, -1, BloomFilterTypeCode.SIMPLE.name());
     Configuration conf = new Configuration();
-    int orcStripSize = Integer.parseInt(HoodieStorageConfig.DEFAULT_ORC_STRIPE_SIZE);
-    int orcBlockSize = Integer.parseInt(HoodieStorageConfig.DEFAULT_ORC_BLOCK_SIZE);
-    int maxFileSize = Integer.parseInt(HoodieStorageConfig.DEFAULT_ORC_FILE_MAX_BYTES);
+    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());
     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/TestCreateAvroKeyGeneratorByTypeWithFactory.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/factory/TestCreateAvroKeyGeneratorByTypeWithFactory.java
index 3fb5a5e..5048314 100644
--- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/factory/TestCreateAvroKeyGeneratorByTypeWithFactory.java
+++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/keygen/factory/TestCreateAvroKeyGeneratorByTypeWithFactory.java
@@ -52,9 +52,9 @@ public class TestCreateAvroKeyGeneratorByTypeWithFactory {
   @BeforeEach
   public void init() {
     props = new TypedProperties();
-    props.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key");
-    props.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, "true");
-    props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp");
+    props.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key");
+    props.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key(), "true");
+    props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp");
 
     // for timestamp based key generator
     props.put("hoodie.deltastreamer.keygen.timebased.timestamp.type", "DATE_STRING");
@@ -70,7 +70,7 @@ public class TestCreateAvroKeyGeneratorByTypeWithFactory {
   @ParameterizedTest
   @MethodSource("configParams")
   public void testKeyGeneratorTypes(String keyGenType) throws IOException {
-    props.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP, keyGenType);
+    props.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP.key(), keyGenType);
     KeyGeneratorType keyType = KeyGeneratorType.valueOf(keyGenType);
 
     KeyGenerator keyGenerator = HoodieAvroKeyGeneratorFactory.createKeyGenerator(props);
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 406f788..8f32b79 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
@@ -38,38 +38,38 @@ public class TestHoodieAvroKeyGeneratorFactory {
     TypedProperties props = getCommonProps();
 
     // set KeyGenerator type only
-    props.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP, KeyGeneratorType.SIMPLE.name());
+    props.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP.key(), KeyGeneratorType.SIMPLE.name());
     KeyGenerator keyGenerator = HoodieAvroKeyGeneratorFactory.createKeyGenerator(props);
     Assertions.assertEquals(SimpleAvroKeyGenerator.class.getName(), keyGenerator.getClass().getName());
 
     // set KeyGenerator class only
     props = getCommonProps();
-    props.put(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP, SimpleAvroKeyGenerator.class.getName());
+    props.put(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP.key(), SimpleAvroKeyGenerator.class.getName());
     KeyGenerator keyGenerator2 = HoodieAvroKeyGeneratorFactory.createKeyGenerator(props);
     Assertions.assertEquals(SimpleAvroKeyGenerator.class.getName(), keyGenerator2.getClass().getName());
 
     // set both class name and keyGenerator type
-    props.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP, KeyGeneratorType.CUSTOM.name());
+    props.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP.key(), KeyGeneratorType.CUSTOM.name());
     KeyGenerator keyGenerator3 = HoodieAvroKeyGeneratorFactory.createKeyGenerator(props);
     // KEYGENERATOR_TYPE_PROP was overitten by KEYGENERATOR_CLASS_PROP
     Assertions.assertEquals(SimpleAvroKeyGenerator.class.getName(), keyGenerator3.getClass().getName());
 
     // set wrong class name
     final TypedProperties props2 = getCommonProps();
-    props2.put(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP, TestHoodieAvroKeyGeneratorFactory.class.getName());
+    props2.put(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP.key(), TestHoodieAvroKeyGeneratorFactory.class.getName());
     assertThrows(IOException.class, () -> HoodieAvroKeyGeneratorFactory.createKeyGenerator(props2));
 
     // set wrong keyGenerator type
     final TypedProperties props3 = getCommonProps();
-    props3.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP, "wrong_type");
+    props3.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP.key(), "wrong_type");
     assertThrows(HoodieKeyGeneratorException.class, () -> HoodieAvroKeyGeneratorFactory.createKeyGenerator(props3));
   }
 
   private TypedProperties getCommonProps() {
     TypedProperties properties = new TypedProperties();
-    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key");
-    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, "true");
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp");
+    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key");
+    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key(), "true");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp");
     return properties;
   }
 }
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 d0fa413..f4ecf42 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);
+    writeConfig.getProps().remove(HoodieMetricsDatadogConfig.DATADOG_METRIC_TAGS.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 694d44c..f02330e 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
@@ -88,7 +88,7 @@ public class HoodieWriteableTestTable extends HoodieTestTable {
         new AvroSchemaConverter().convert(schema), schema, filter);
     HoodieAvroParquetConfig config = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP,
         ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024,
-        new Configuration(), Double.parseDouble(HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO));
+        new Configuration(), Double.parseDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO.defaultValue()));
     try (HoodieParquetWriter writer = new HoodieParquetWriter(
         currentInstantTime,
         new Path(Paths.get(basePath, partition, fileName).toString()),
diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java
index 6a2d1bf..a5109dc 100644
--- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java
+++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java
@@ -112,7 +112,7 @@ public class HoodieFlinkMergeOnReadTableCompactor<T extends HoodieRecordPayload>
             HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION))
         .filterCompletedInstants().lastInstant().get().getTimestamp();
     // TODO(danny): make it configurable
-    long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new FlinkTaskContextSupplier(null), config.getProps());
+    long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new FlinkTaskContextSupplier(null), config);
     LOG.info("MaxMemoryPerCompaction => " + maxMemoryPerCompaction);
 
     List<String> logFiles = operation.getDeltaFileNames().stream().map(
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkRecentDaysClusteringPlanStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkRecentDaysClusteringPlanStrategy.java
index ed61c0c..63bcc69 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkRecentDaysClusteringPlanStrategy.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/SparkRecentDaysClusteringPlanStrategy.java
@@ -104,7 +104,7 @@ public class SparkRecentDaysClusteringPlanStrategy<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, getWriteConfig().getClusteringSortColumns());
+      params.put(CLUSTERING_SORT_COLUMNS_PROPERTY.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 07f9bc1..9ab40fc 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
@@ -69,10 +69,10 @@ public class SparkSortAndSizeExecutionStrategy<T extends HoodieRecordPayload<T>>
                                                 final String instantTime, final Map<String, String> strategyParams, final Schema schema) {
     LOG.info("Starting clustering for a group, parallelism:" + numOutputGroups + " commit:" + instantTime);
     Properties props = getWriteConfig().getProps();
-    props.put(HoodieWriteConfig.BULKINSERT_PARALLELISM, String.valueOf(numOutputGroups));
+    props.put(HoodieWriteConfig.BULKINSERT_PARALLELISM.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_PROP, Boolean.FALSE.toString());
-    props.put(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES, String.valueOf(getWriteConfig().getClusteringTargetFileMaxBytes()));
+    props.put(HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP.key(), Boolean.FALSE.toString());
+    props.put(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES.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);
@@ -82,8 +82,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)) {
-      return Option.of(new RDDCustomColumnsSortPartitioner(strategyParams.get(CLUSTERING_SORT_COLUMNS_PROPERTY).split(","),
+    if (strategyParams.containsKey(CLUSTERING_SORT_COLUMNS_PROPERTY.key())) {
+      return Option.of(new RDDCustomColumnsSortPartitioner(strategyParams.get(CLUSTERING_SORT_COLUMNS_PROPERTY.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 0fa75dc..9ca5a0c 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
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.client.utils;
 
+import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.config.HoodieIndexConfig;
 
 import org.apache.spark.storage.StorageLevel;
@@ -31,14 +32,14 @@ import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVE
  */
 public class SparkMemoryUtils {
   public static StorageLevel getWriteStatusStorageLevel(Properties properties) {
-    return StorageLevel.fromString(properties.getProperty(WRITE_STATUS_STORAGE_LEVEL));
+    return StorageLevel.fromString(new HoodieConfig(properties).getString(WRITE_STATUS_STORAGE_LEVEL));
   }
 
   public static StorageLevel getBloomIndexInputStorageLevel(Properties properties) {
-    return StorageLevel.fromString(properties.getProperty(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL));
+    return StorageLevel.fromString(new HoodieConfig(properties).getString(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL));
   }
 
   public static StorageLevel getSimpleIndexInputStorageLevel(Properties properties) {
-    return StorageLevel.fromString(properties.getProperty(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL));
+    return StorageLevel.fromString(new HoodieConfig(properties).getString(HoodieIndexConfig.SIMPLE_INDEX_INPUT_STORAGE_LEVEL));
   }
 }
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java
index 36c8345..74b28c8 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java
@@ -34,9 +34,9 @@ public class ComplexKeyGenerator extends BuiltinKeyGenerator {
 
   public ComplexKeyGenerator(TypedProperties props) {
     super(props);
-    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
+    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key())
         .split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
-    this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY)
+    this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key())
         .split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
     complexAvroKeyGenerator = new ComplexAvroKeyGenerator(props);
   }
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
index a2a3012..fd9b460 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/CustomKeyGenerator.java
@@ -49,8 +49,8 @@ public class CustomKeyGenerator extends BuiltinKeyGenerator {
 
   public CustomKeyGenerator(TypedProperties props) {
     super(props);
-    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY).split(",")).map(String::trim).collect(Collectors.toList());
-    this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY).split(",")).map(String::trim).collect(Collectors.toList());
+    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()).split(",")).map(String::trim).collect(Collectors.toList());
+    this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key()).split(",")).map(String::trim).collect(Collectors.toList());
     customAvroKeyGenerator = new CustomAvroKeyGenerator(props);
   }
 
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java
index 5c9a813..5ec732b 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/GlobalDeleteKeyGenerator.java
@@ -36,7 +36,7 @@ public class GlobalDeleteKeyGenerator extends BuiltinKeyGenerator {
   private final GlobalAvroDeleteKeyGenerator globalAvroDeleteKeyGenerator;
   public GlobalDeleteKeyGenerator(TypedProperties config) {
     super(config);
-    this.recordKeyFields = Arrays.asList(config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY).split(","));
+    this.recordKeyFields = Arrays.asList(config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()).split(","));
     globalAvroDeleteKeyGenerator = new GlobalAvroDeleteKeyGenerator(config);
   }
 
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java
index 277d2eb..6c83c0b 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/NonpartitionedKeyGenerator.java
@@ -37,7 +37,7 @@ public class NonpartitionedKeyGenerator extends BuiltinKeyGenerator {
 
   public NonpartitionedKeyGenerator(TypedProperties props) {
     super(props);
-    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
+    this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key())
         .split(",")).map(String::trim).collect(Collectors.toList());
     this.partitionPathFields = Collections.emptyList();
     nonpartitionedAvroKeyGenerator = new NonpartitionedAvroKeyGenerator(props);
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java
index 332686d..9e149f3 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/SimpleKeyGenerator.java
@@ -33,8 +33,8 @@ public class SimpleKeyGenerator extends BuiltinKeyGenerator {
   private final SimpleAvroKeyGenerator simpleAvroKeyGenerator;
 
   public SimpleKeyGenerator(TypedProperties props) {
-    this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY),
-        props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY));
+    this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()),
+        props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key()));
   }
 
   SimpleKeyGenerator(TypedProperties props, String partitionPathField) {
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java
index 859269c..72bca52 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/TimestampBasedKeyGenerator.java
@@ -38,8 +38,8 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
   private final TimestampBasedAvroKeyGenerator timestampBasedAvroKeyGenerator;
 
   public TimestampBasedKeyGenerator(TypedProperties config) throws IOException {
-    this(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY),
-        config.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY));
+    this(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()),
+        config.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key()));
   }
 
   TimestampBasedKeyGenerator(TypedProperties config, String partitionPathField) throws IOException {
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 4012184..902bdfb 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
@@ -54,7 +54,7 @@ public class HoodieSparkKeyGeneratorFactory {
   private static BuiltinKeyGenerator createKeyGeneratorByType(TypedProperties props) throws IOException {
     // Use KeyGeneratorType.SIMPLE as default keyGeneratorType
     String keyGeneratorType =
-        props.getString(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP, KeyGeneratorType.SIMPLE.name());
+        props.getString(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP.key(), KeyGeneratorType.SIMPLE.name());
 
     KeyGeneratorType keyGeneratorTypeEnum;
     try {
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java
index 9f6e39a..5a87ec0 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java
@@ -190,7 +190,7 @@ public class SparkExecuteClusteringCommitActionExecutor<T extends HoodieRecordPa
     return jsc.parallelize(clusteringOps, clusteringOps.size()).mapPartitions(clusteringOpsPartition -> {
       List<Iterator<HoodieRecord<? extends HoodieRecordPayload>>> recordIterators = new ArrayList<>();
       clusteringOpsPartition.forEachRemaining(clusteringOp -> {
-        long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new SparkTaskContextSupplier(), config.getProps());
+        long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new SparkTaskContextSupplier(), config);
         LOG.info("MaxMemoryPerCompaction run as part of clustering => " + maxMemoryPerCompaction);
         try {
           Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java
index 17c19ce..117be74 100644
--- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java
+++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java
@@ -122,7 +122,7 @@ public class HoodieSparkMergeOnReadTableCompactor<T extends HoodieRecordPayload>
         .getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION,
             HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION))
         .filterCompletedInstants().lastInstant().get().getTimestamp();
-    long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new SparkTaskContextSupplier(), config.getProps());
+    long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new SparkTaskContextSupplier(), config);
     LOG.info("MaxMemoryPerCompaction => " + maxMemoryPerCompaction);
 
     List<String> logFiles = operation.getDeltaFileNames().stream().map(
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java
index a56789b..c7599aa 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientMultiWriter.java
@@ -52,7 +52,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
-import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP;
+import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
@@ -80,7 +80,7 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
       setUpMORTestTable();
     }
     Properties properties = new Properties();
-    properties.setProperty(FILESYSTEM_LOCK_PATH_PROP, basePath + "/.hoodie/.locks");
+    properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks");
     HoodieWriteConfig cfg = getConfigBuilder()
         .withCompactionConfig(HoodieCompactionConfig.newBuilder()
             .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build())
@@ -131,7 +131,7 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
       setUpMORTestTable();
     }
     Properties properties = new Properties();
-    properties.setProperty(FILESYSTEM_LOCK_PATH_PROP, basePath + "/.hoodie/.locks");
+    properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks");
     // Disabling embedded timeline server, it doesn't work with multiwriter
     HoodieWriteConfig cfg = getConfigBuilder()
             .withCompactionConfig(HoodieCompactionConfig.newBuilder().withAutoClean(false)
@@ -240,7 +240,7 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
       setUpMORTestTable();
     }
     Properties properties = new Properties();
-    properties.setProperty(FILESYSTEM_LOCK_PATH_PROP, basePath + "/.hoodie/.locks");
+    properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks");
     HoodieWriteConfig cfg = getConfigBuilder()
         .withCompactionConfig(HoodieCompactionConfig.newBuilder().withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY)
             .withAutoClean(false).build())
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java
index 4e5aefd..e49723a 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java
@@ -116,7 +116,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_OPT_KEY;
-import static org.apache.hudi.config.HoodieClusteringConfig.DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS;
+import static org.apache.hudi.config.HoodieClusteringConfig.CLUSTERING_EXECUTION_STRATEGY_CLASS;
 import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -638,7 +638,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
       .setTimelineLayoutVersion(VERSION_0)
       .initTable(metaClient.getHadoopConf(), metaClient.getBasePath());
     // Set rollback to LAZY so no inflights are deleted
-    hoodieWriteConfig.getProps().put(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY_PROP,
+    hoodieWriteConfig.getProps().put(HoodieCompactionConfig.FAILED_WRITES_CLEANER_POLICY_PROP.key(),
         HoodieFailedWritesCleaningPolicy.LAZY.name());
     SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig);
 
@@ -791,7 +791,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
     final String testPartitionPath = "2016/09/26";
     dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath});
     Properties props = new Properties();
-    props.setProperty(ASYNC_CLUSTERING_ENABLE_OPT_KEY, "true");
+    props.setProperty(ASYNC_CLUSTERING_ENABLE_OPT_KEY.key(), "true");
     HoodieWriteConfig config = getSmallInsertWriteConfig(100,
         TRIP_EXAMPLE_SCHEMA, dataGen.getEstimatedFileSizeInBytes(150), props);
     SparkRDDWriteClient client = getHoodieWriteClient(config);
@@ -1985,7 +1985,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
 
   protected HoodieInstant createRequestedReplaceInstant(HoodieTableMetaClient metaClient, String clusterTime, List<FileSlice>[] fileSlices) throws IOException {
     HoodieClusteringPlan clusteringPlan =
-        ClusteringUtils.createClusteringPlan(DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS, STRATEGY_PARAMS, fileSlices, Collections.emptyMap());
+        ClusteringUtils.createClusteringPlan(CLUSTERING_EXECUTION_STRATEGY_CLASS.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/TestHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java
index e42b6ba..d5d66e7 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestHBaseIndex.java
@@ -470,9 +470,9 @@ public class TestHBaseIndex extends FunctionalTestHarness {
   public void testHbaseTagLocationForArchivedCommits() throws Exception {
     // Load to memory
     Map<String, String> params = new HashMap<String, String>();
-    params.put(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP, "1");
-    params.put(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP, "3");
-    params.put(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP, "2");
+    params.put(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP.key(), "1");
+    params.put(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP.key(), "3");
+    params.put(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP.key(), "2");
     HoodieWriteConfig config = getConfigBuilder(100, false, false).withProps(params).build();
 
     SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
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 e9f8b87..9f52a01 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.DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
+    HoodieWriteConfig config = getConfig(Option.of(HoodieHBaseIndexConfig.HBASE_INDEX_QPS_ALLOCATOR_CLASS.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/io/TestSparkIOUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestSparkIOUtils.java
index ffbf6d1..f6fb4e4 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestSparkIOUtils.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestSparkIOUtils.java
@@ -25,8 +25,6 @@ import org.apache.hudi.config.HoodieWriteConfig;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.io.TempDir;
 
-import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION;
-import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE;
 import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP;
 import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_MERGE_PROP;
 import static org.junit.jupiter.api.Assertions.assertEquals;
@@ -47,8 +45,8 @@ public class TestSparkIOUtils {
     HoodieMemoryConfig memoryConfig = HoodieMemoryConfig.newBuilder().withMaxMemoryMaxSize(mergeMaxSize, compactionMaxSize).build();
     HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(path).withMemoryConfig(memoryConfig).build();
 
-    assertEquals(mergeMaxSize, IOUtils.getMaxMemoryPerPartitionMerge(contextSupplier, config.getProps()));
-    assertEquals(compactionMaxSize, IOUtils.getMaxMemoryPerCompaction(contextSupplier, config.getProps()));
+    assertEquals(mergeMaxSize, IOUtils.getMaxMemoryPerPartitionMerge(contextSupplier, config));
+    assertEquals(compactionMaxSize, IOUtils.getMaxMemoryPerCompaction(contextSupplier, config));
   }
 
   @Test
@@ -57,13 +55,13 @@ public class TestSparkIOUtils {
 
     HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(path).build();
 
-    String compactionFraction = config.getProps().getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION);
+    String compactionFraction = config.getProps().getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP.key(), MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP.defaultValue());
     long compactionMaxSize = IOUtils.getMaxMemoryAllowedForMerge(contextSupplier, compactionFraction);
 
-    String mergeFraction = config.getProps().getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE);
+    String mergeFraction = config.getProps().getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP.key(), MAX_MEMORY_FRACTION_FOR_MERGE_PROP.defaultValue());
     long mergeMaxSize = IOUtils.getMaxMemoryAllowedForMerge(contextSupplier, mergeFraction);
 
-    assertEquals(mergeMaxSize, IOUtils.getMaxMemoryPerPartitionMerge(contextSupplier, config.getProps()));
-    assertEquals(compactionMaxSize, IOUtils.getMaxMemoryPerCompaction(contextSupplier, config.getProps()));
+    assertEquals(mergeMaxSize, IOUtils.getMaxMemoryPerPartitionMerge(contextSupplier, config));
+    assertEquals(compactionMaxSize, IOUtils.getMaxMemoryPerCompaction(contextSupplier, config));
   }
 }
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java
index 54f4ffa..76b83e1 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestComplexKeyGenerator.java
@@ -37,11 +37,11 @@ public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
   private TypedProperties getCommonProps(boolean getComplexRecordKey) {
     TypedProperties properties = new TypedProperties();
     if (getComplexRecordKey) {
-      properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key, pii_col");
+      properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key, pii_col");
     } else {
-      properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key");
+      properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key");
     }
-    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, "true");
+    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key(), "true");
     return properties;
   }
 
@@ -51,20 +51,20 @@ public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
 
   private TypedProperties getPropertiesWithoutRecordKeyProp() {
     TypedProperties properties = new TypedProperties();
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp");
     return properties;
   }
 
   private TypedProperties getWrongRecordKeyFieldProps() {
     TypedProperties properties = new TypedProperties();
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp");
-    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_wrong_key");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp");
+    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_wrong_key");
     return properties;
   }
 
   private TypedProperties getProps() {
     TypedProperties properties = getCommonProps(true);
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp,ts_ms");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp,ts_ms");
     return properties;
   }
 
@@ -100,8 +100,8 @@ public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
   @Test
   public void testSingleValueKeyGenerator() {
     TypedProperties properties = new TypedProperties();
-    properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key");
-    properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp");
+    properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key");
+    properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp");
     ComplexKeyGenerator compositeKeyGenerator = new ComplexKeyGenerator(properties);
     assertEquals(compositeKeyGenerator.getRecordKeyFields().size(), 1);
     assertEquals(compositeKeyGenerator.getPartitionPathFields().size(), 1);
@@ -117,8 +117,8 @@ public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
   @Test
   public void testMultipleValueKeyGenerator() {
     TypedProperties properties = new TypedProperties();
-    properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key,timestamp");
-    properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "rider,driver");
+    properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key,timestamp");
+    properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "rider,driver");
     ComplexKeyGenerator compositeKeyGenerator = new ComplexKeyGenerator(properties);
     assertEquals(compositeKeyGenerator.getRecordKeyFields().size(), 2);
     assertEquals(compositeKeyGenerator.getPartitionPathFields().size(), 2);
@@ -136,8 +136,8 @@ public class TestComplexKeyGenerator extends KeyGeneratorTestUtilities {
   @Test
   public void testMultipleValueKeyGeneratorNonPartitioned() {
     TypedProperties properties = new TypedProperties();
-    properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key,timestamp");
-    properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "");
+    properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key,timestamp");
+    properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "");
     ComplexKeyGenerator compositeKeyGenerator = new ComplexKeyGenerator(properties);
     assertEquals(compositeKeyGenerator.getRecordKeyFields().size(), 2);
     assertEquals(compositeKeyGenerator.getPartitionPathFields().size(), 0);
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 1e19a8a..1a424f5 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
@@ -47,57 +47,57 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
   private TypedProperties getCommonProps(boolean getComplexRecordKey, boolean useKeyGeneratorClassName) {
     TypedProperties properties = new TypedProperties();
     if (getComplexRecordKey) {
-      properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key, pii_col");
+      properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key, pii_col");
     } else {
-      properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key");
+      properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key");
     }
     if (useKeyGeneratorClassName) {
-      properties.put(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP, CustomKeyGenerator.class.getName());
+      properties.put(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP.key(), CustomKeyGenerator.class.getName());
     } else {
-      properties.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP, KeyGeneratorType.CUSTOM.name());
+      properties.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP.key(), KeyGeneratorType.CUSTOM.name());
     }
-    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, "true");
+    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key(), "true");
     return properties;
   }
 
   private TypedProperties getPropertiesForSimpleKeyGen(boolean useKeyGeneratorClassName) {
     TypedProperties properties = getCommonProps(false, useKeyGeneratorClassName);
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp:simple");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp:simple");
     return properties;
   }
 
   private TypedProperties getImproperPartitionFieldFormatProp(boolean useKeyGeneratorClassName) {
     TypedProperties properties = getCommonProps(false, useKeyGeneratorClassName);
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp");
     return properties;
   }
 
   private TypedProperties getInvalidPartitionKeyTypeProps(boolean useKeyGeneratorClassName) {
     TypedProperties properties = getCommonProps(false, useKeyGeneratorClassName);
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp:dummy");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp:dummy");
     return properties;
   }
 
   private TypedProperties getComplexRecordKeyWithSimplePartitionProps(boolean useKeyGeneratorClassName) {
     TypedProperties properties = getCommonProps(true, useKeyGeneratorClassName);
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp:simple");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp:simple");
     return properties;
   }
 
   private TypedProperties getComplexRecordKeyAndPartitionPathProps(boolean useKeyGeneratorClassName) {
     TypedProperties properties = getCommonProps(true, useKeyGeneratorClassName);
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp:simple,ts_ms:timestamp");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp:simple,ts_ms:timestamp");
     populateNecessaryPropsForTimestampBasedKeyGen(properties);
     return properties;
   }
 
   private TypedProperties getPropsWithoutRecordKeyFieldProps(boolean useKeyGeneratorClassName) {
     TypedProperties properties = new TypedProperties();
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp:simple");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp:simple");
     if (useKeyGeneratorClassName) {
-      properties.put(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP, CustomKeyGenerator.class.getName());
+      properties.put(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP.key(), CustomKeyGenerator.class.getName());
     } else {
-      properties.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP, KeyGeneratorType.CUSTOM.name());
+      properties.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP.key(), KeyGeneratorType.CUSTOM.name());
     }
     return properties;
   }
@@ -110,14 +110,14 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
 
   private TypedProperties getPropertiesForTimestampBasedKeyGen(boolean useKeyGeneratorClassName) {
     TypedProperties properties = getCommonProps(false, useKeyGeneratorClassName);
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "ts_ms:timestamp");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "ts_ms:timestamp");
     populateNecessaryPropsForTimestampBasedKeyGen(properties);
     return properties;
   }
 
   private TypedProperties getPropertiesForNonPartitionedKeyGen(boolean useKeyGeneratorClassName) {
     TypedProperties properties = getCommonProps(false, useKeyGeneratorClassName);
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "");
     return properties;
   }
 
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java
index 078101b..dd3cea4 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestGlobalDeleteKeyGenerator.java
@@ -34,29 +34,29 @@ public class TestGlobalDeleteKeyGenerator extends KeyGeneratorTestUtilities {
   private TypedProperties getCommonProps(boolean getComplexRecordKey) {
     TypedProperties properties = new TypedProperties();
     if (getComplexRecordKey) {
-      properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key,pii_col");
+      properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key,pii_col");
     } else {
-      properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key");
+      properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key");
     }
-    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, "true");
+    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key(), "true");
     return properties;
   }
 
   private TypedProperties getPropertiesWithoutRecordKeyProp() {
     TypedProperties properties = new TypedProperties();
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp");
     return properties;
   }
 
   private TypedProperties getWrongRecordKeyFieldProps() {
     TypedProperties properties = new TypedProperties();
-    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_wrong_key");
+    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_wrong_key");
     return properties;
   }
 
   private TypedProperties getProps() {
     TypedProperties properties = getCommonProps(true);
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp,ts_ms");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp,ts_ms");
     return properties;
   }
 
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java
index 4782caa..56d576e 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestNonpartitionedKeyGenerator.java
@@ -37,11 +37,11 @@ public class TestNonpartitionedKeyGenerator extends KeyGeneratorTestUtilities {
   private TypedProperties getCommonProps(boolean getComplexRecordKey) {
     TypedProperties properties = new TypedProperties();
     if (getComplexRecordKey) {
-      properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key, pii_col");
+      properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key, pii_col");
     } else {
-      properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key");
+      properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key");
     }
-    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, "true");
+    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key(), "true");
     return properties;
   }
 
@@ -51,19 +51,19 @@ public class TestNonpartitionedKeyGenerator extends KeyGeneratorTestUtilities {
 
   private TypedProperties getPropertiesWithPartitionPathProp() {
     TypedProperties properties = getCommonProps(true);
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp,ts_ms");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp,ts_ms");
     return properties;
   }
 
   private TypedProperties getPropertiesWithoutRecordKeyProp() {
     TypedProperties properties = new TypedProperties();
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp");
     return properties;
   }
 
   private TypedProperties getWrongRecordKeyFieldProps() {
     TypedProperties properties = new TypedProperties();
-    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_wrong_key");
+    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_wrong_key");
     return properties;
   }
 
@@ -78,7 +78,7 @@ public class TestNonpartitionedKeyGenerator extends KeyGeneratorTestUtilities {
     NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(properties);
     GenericRecord record = getRecord();
     Row row = KeyGeneratorTestUtilities.getRow(record);
-    Assertions.assertEquals(properties.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY), "timestamp,ts_ms");
+    Assertions.assertEquals(properties.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key()), "timestamp,ts_ms");
     Assertions.assertEquals(keyGenerator.getPartitionPath(row), "");
   }
 
@@ -101,8 +101,8 @@ public class TestNonpartitionedKeyGenerator extends KeyGeneratorTestUtilities {
   @Test
   public void testSingleValueKeyGeneratorNonPartitioned() {
     TypedProperties properties = new TypedProperties();
-    properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "timestamp");
-    properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "");
+    properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "timestamp");
+    properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "");
     NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(properties);
     assertEquals(keyGenerator.getRecordKeyFields().size(), 1);
     assertEquals(keyGenerator.getPartitionPathFields().size(), 0);
@@ -118,8 +118,8 @@ public class TestNonpartitionedKeyGenerator extends KeyGeneratorTestUtilities {
   @Test
   public void testMultipleValueKeyGeneratorNonPartitioned1() {
     TypedProperties properties = new TypedProperties();
-    properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "timestamp,driver");
-    properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "");
+    properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "timestamp,driver");
+    properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "");
     NonpartitionedKeyGenerator keyGenerator = new NonpartitionedKeyGenerator(properties);
     assertEquals(keyGenerator.getRecordKeyFields().size(), 2);
     assertEquals(keyGenerator.getPartitionPathFields().size(), 0);
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java
index 161b7c2..f9f2651 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestSimpleKeyGenerator.java
@@ -38,8 +38,8 @@ import static org.apache.hudi.keygen.KeyGenUtils.DEFAULT_PARTITION_PATH;
 public class TestSimpleKeyGenerator extends KeyGeneratorTestUtilities {
   private TypedProperties getCommonProps() {
     TypedProperties properties = new TypedProperties();
-    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key");
-    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, "true");
+    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key");
+    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key(), "true");
     return properties;
   }
 
@@ -49,40 +49,40 @@ public class TestSimpleKeyGenerator extends KeyGeneratorTestUtilities {
 
   private TypedProperties getPropertiesWithoutRecordKeyProp() {
     TypedProperties properties = new TypedProperties();
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp");
     return properties;
   }
 
   private TypedProperties getWrongRecordKeyFieldProps() {
     TypedProperties properties = new TypedProperties();
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp");
-    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_wrong_key");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp");
+    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_wrong_key");
     return properties;
   }
 
   private TypedProperties getWrongPartitionPathFieldProps() {
     TypedProperties properties = new TypedProperties();
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "_wrong_partition_path");
-    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "_wrong_partition_path");
+    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key");
     return properties;
   }
 
   private TypedProperties getComplexRecordKeyProp() {
     TypedProperties properties = new TypedProperties();
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp");
-    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key,pii_col");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp");
+    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key,pii_col");
     return properties;
   }
 
   private TypedProperties getProps() {
     TypedProperties properties = getCommonProps();
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp");
     return properties;
   }
 
   private TypedProperties getPropsWithNestedPartitionPathField() {
     TypedProperties properties = getCommonProps();
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "nested_col.prop1");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "nested_col.prop1");
     return properties;
   }
 
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java
index a839038..9820ac5 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/TestTimestampBasedKeyGenerator.java
@@ -59,9 +59,9 @@ public class TestTimestampBasedKeyGenerator {
         .generateAvroRecordFromJson(schema, 1, "001", "f1");
     baseRow = genericRecordToRow(baseRecord);
 
-    properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "field1");
-    properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "createTime");
-    properties.setProperty(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, "false");
+    properties.setProperty(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "field1");
+    properties.setProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "createTime");
+    properties.setProperty(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key(), "false");
   }
 
   private TypedProperties getBaseKeyConfig(String timestampType, String dateFormat, String timezone, String scalarType) {
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestCreateKeyGeneratorByTypeWithFactory.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestCreateKeyGeneratorByTypeWithFactory.java
index dac3d11..f7e5c43 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestCreateKeyGeneratorByTypeWithFactory.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/keygen/factory/TestCreateKeyGeneratorByTypeWithFactory.java
@@ -53,9 +53,9 @@ public class TestCreateKeyGeneratorByTypeWithFactory {
   @BeforeEach
   public void init() {
     props = new TypedProperties();
-    props.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key");
-    props.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, "true");
-    props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp");
+    props.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key");
+    props.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key(), "true");
+    props.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp");
 
     // for timestamp based key generator
     props.put("hoodie.deltastreamer.keygen.timebased.timestamp.type", "DATE_STRING");
@@ -71,7 +71,7 @@ public class TestCreateKeyGeneratorByTypeWithFactory {
   @ParameterizedTest
   @MethodSource("configParams")
   public void testKeyGeneratorTypes(String keyGenType) throws IOException {
-    props.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP, keyGenType);
+    props.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP.key(), keyGenType);
     KeyGeneratorType keyType = KeyGeneratorType.valueOf(keyGenType);
 
     KeyGenerator keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
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 bd868ff..204cb57 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
@@ -45,38 +45,38 @@ public class TestHoodieSparkKeyGeneratorFactory {
     TypedProperties props = getCommonProps();
 
     // set KeyGenerator type only
-    props.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP, KeyGeneratorType.SIMPLE.name());
+    props.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP.key(), KeyGeneratorType.SIMPLE.name());
     KeyGenerator keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
     Assertions.assertEquals(SimpleKeyGenerator.class.getName(), keyGenerator.getClass().getName());
 
     // set KeyGenerator class only
     props = getCommonProps();
-    props.put(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP, SimpleKeyGenerator.class.getName());
+    props.put(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP.key(), SimpleKeyGenerator.class.getName());
     KeyGenerator keyGenerator2 = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
     Assertions.assertEquals(SimpleKeyGenerator.class.getName(), keyGenerator2.getClass().getName());
 
     // set both class name and keyGenerator type
-    props.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP, KeyGeneratorType.CUSTOM.name());
+    props.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP.key(), KeyGeneratorType.CUSTOM.name());
     KeyGenerator keyGenerator3 = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props);
     // KEYGENERATOR_TYPE_PROP was overitten by KEYGENERATOR_CLASS_PROP
     Assertions.assertEquals(SimpleKeyGenerator.class.getName(), keyGenerator3.getClass().getName());
 
     // set wrong class name
     final TypedProperties props2 = getCommonProps();
-    props2.put(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP, TestHoodieSparkKeyGeneratorFactory.class.getName());
+    props2.put(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP.key(), TestHoodieSparkKeyGeneratorFactory.class.getName());
     assertThrows(IOException.class, () -> HoodieSparkKeyGeneratorFactory.createKeyGenerator(props2));
 
     // set wrong keyGenerator type
     final TypedProperties props3 = getCommonProps();
-    props3.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP, "wrong_type");
+    props3.put(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP.key(), "wrong_type");
     assertThrows(HoodieKeyGeneratorException.class, () -> HoodieSparkKeyGeneratorFactory.createKeyGenerator(props3));
   }
 
   private TypedProperties getCommonProps() {
     TypedProperties properties = new TypedProperties();
-    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "_row_key");
-    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, "true");
-    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY, "timestamp");
+    properties.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key(), "_row_key");
+    properties.put(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key(), "true");
+    properties.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "timestamp");
     return properties;
   }
 }
diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java
index dc9999e..3ae722f 100644
--- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java
+++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/metadata/TestHoodieBackedMetadata.java
@@ -578,7 +578,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
 
       // Reader should sync to all the completed instants
       HoodieTableMetadata metadata  = HoodieTableMetadata.create(context, client.getConfig().getMetadataConfig(),
-          client.getConfig().getBasePath(), FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR);
+          client.getConfig().getBasePath(), FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue());
       assertEquals(metadata.getSyncedInstantTime().get(), newCommitTime);
 
       // Remove the inflight instance holding back table sync
@@ -591,7 +591,7 @@ public class TestHoodieBackedMetadata extends HoodieClientTestHarness {
 
       // Reader should sync to all the completed instants
       metadata  = HoodieTableMetadata.create(context, client.getConfig().getMetadataConfig(),
-          client.getConfig().getBasePath(), FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR);
+          client.getConfig().getBasePath(), FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue());
       assertEquals(metadata.getSyncedInstantTime().get(), newCommitTime);
     }
 
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 e7cd5ce..32e0b43 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.DEFAULT_BASE_FILE_FORMAT.getFileExtension();
+  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.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 18ff136..84d0b03 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
@@ -132,7 +132,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
 
   @BeforeEach
   public void init() throws IOException {
-    init(HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT);
+    init(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue());
   }
 
   @AfterEach
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 156f215..fa5155f 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
@@ -275,7 +275,7 @@ public class TestHoodieCompactionStrategy {
     private final long size;
 
     public TestHoodieBaseFile(long size) {
-      super("/tmp/XYXYXYXYXYYX_11_20180918020003" + HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension());
+      super("/tmp/XYXYXYXYXYYX_11_20180918020003" + HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.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 4f01b34..62623b1 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
@@ -38,7 +38,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertIterableEquals;
 
 public class TestRollbackUtils {
-  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension();
+  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.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 e2416bb..882b495 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
@@ -24,6 +24,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hudi.client.SparkRDDWriteClient;
 import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.common.model.FileSlice;
 import org.apache.hudi.common.model.HoodieFileGroup;
 import org.apache.hudi.common.model.HoodieLogFile;
@@ -57,11 +58,10 @@ import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
-import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME;
+import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_TABLE_TYPE_PROP;
 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.junit.jupiter.api.Assertions.assertEquals;
@@ -111,7 +111,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_NAME, HoodieTableType.MERGE_ON_READ.name());
+      params.put(HOODIE_TABLE_TYPE_PROP.key(), HoodieTableType.MERGE_ON_READ.name());
       metaClient = HoodieTestUtils.init(dfs.getConf(), dfsBasePath, HoodieTableType.MERGE_ON_READ);
     }
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(false).withProps(params).build();
@@ -167,7 +167,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_NAME, HoodieTableType.MERGE_ON_READ.name());
+      params.put(HOODIE_TABLE_TYPE_PROP.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();
@@ -287,7 +287,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_NAME, HoodieTableType.MERGE_ON_READ.name());
+      params.put(HOODIE_TABLE_TYPE_PROP.key(), HoodieTableType.MERGE_ON_READ.name());
     }
     HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(enableMarkedBasedRollback).withProps(params).build();
     SparkRDDWriteClient client = getHoodieWriteClient(cfg);
@@ -397,7 +397,7 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
     metaClient.getTableConfig().setTableVersion(HoodieTableVersion.ONE);
     Path propertyFile = new Path(metaClient.getMetaPath() + "/" + HoodieTableConfig.HOODIE_PROPERTIES_FILE);
     try (FSDataOutputStream os = metaClient.getFs().create(propertyFile)) {
-      metaClient.getTableConfig().getProperties().store(os, "");
+      metaClient.getTableConfig().getProps().store(os, "");
     }
   }
 
@@ -414,9 +414,9 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
     Path propertyFile = new Path(metaClient.getMetaPath() + "/" + HoodieTableConfig.HOODIE_PROPERTIES_FILE);
     // Load the properties and verify
     FSDataInputStream fsDataInputStream = metaClient.getFs().open(propertyFile);
-    Properties prop = new Properties();
-    prop.load(fsDataInputStream);
+    HoodieConfig hoodieConfig = HoodieConfig.create(fsDataInputStream);
     fsDataInputStream.close();
-    assertEquals(Integer.toString(expectedVersion.versionCode()), prop.getProperty(HoodieTableConfig.HOODIE_TABLE_VERSION_PROP_NAME));
+    assertEquals(Integer.toString(expectedVersion.versionCode()), hoodieConfig
+        .getString(HoodieTableConfig.HOODIE_TABLE_VERSION_PROP));
   }
 }
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java
new file mode 100644
index 0000000..bacd3b9
--- /dev/null
+++ b/hudi-common/src/main/java/org/apache/hudi/common/config/ConfigProperty.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.config;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+
+import java.io.Serializable;
+import java.util.function.Function;
+import java.util.Objects;
+
+/**
+ * ConfigProperty describes a configuration property. It contains the configuration
+ * key, deprecated older versions of the key, and an optional default value for the configuration,
+ * configuration descriptions and also the an infer mechanism to infer the configuration value
+ * based on other configurations.
+ *
+ * @param <T> The type of the default value.
+ */
+public class ConfigProperty<T> implements Serializable {
+
+  private final String key;
+
+  private final T defaultValue;
+
+  private final String doc;
+
+  private final Option<String> sinceVersion;
+
+  private final Option<String> deprecatedVersion;
+
+  private final String[] alternatives;
+
+  // provide the ability to infer config value based on other configs
+  private final Option<Function<HoodieConfig, Option<T>>> inferFunction;
+
+  ConfigProperty(String key, T defaultValue, String doc, Option<String> sinceVersion,
+                 Option<String> deprecatedVersion, Option<Function<HoodieConfig, Option<T>>> inferFunc, String... alternatives) {
+    this.key = Objects.requireNonNull(key);
+    this.defaultValue = defaultValue;
+    this.doc = doc;
+    this.sinceVersion = sinceVersion;
+    this.deprecatedVersion = deprecatedVersion;
+    this.inferFunction = inferFunc;
+    this.alternatives = alternatives;
+  }
+
+  public String key() {
+    return key;
+  }
+
+  public T defaultValue() {
+    if (defaultValue == null) {
+      throw new HoodieException("There's no default value for this config");
+    }
+    return defaultValue;
+  }
+
+  public boolean hasDefaultValue() {
+    return defaultValue != null;
+  }
+
+  Option<Function<HoodieConfig, Option<T>>> getInferFunc() {
+    return inferFunction;
+  }
+
+  public String[] getAlternatives() {
+    return alternatives;
+  }
+
+  public ConfigProperty<T> withDocumentation(String doc) {
+    Objects.requireNonNull(doc);
+    return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, alternatives);
+  }
+
+  public ConfigProperty<T> withAlternatives(String... alternatives) {
+    Objects.requireNonNull(alternatives);
+    return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, alternatives);
+  }
+
+  public ConfigProperty<T> sinceVersion(String sinceVersion) {
+    Objects.requireNonNull(sinceVersion);
+    return new ConfigProperty<>(key, defaultValue, doc, Option.of(sinceVersion), deprecatedVersion, inferFunction, alternatives);
+  }
+
+  public ConfigProperty<T> deprecatedAfter(String deprecatedVersion) {
+    Objects.requireNonNull(deprecatedVersion);
+    return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, Option.of(deprecatedVersion), inferFunction, alternatives);
+  }
+
+  public ConfigProperty<T> withInferFunction(Function<HoodieConfig, Option<T>> inferFunction) {
+    Objects.requireNonNull(inferFunction);
+    return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, Option.of(inferFunction), alternatives);
+  }
+
+  /**
+   * Create a OptionBuilder with key.
+   *
+   * @param key The key of the option
+   * @return Return a OptionBuilder.
+   */
+  public static PropertyBuilder key(String key) {
+    Objects.requireNonNull(key);
+    return new PropertyBuilder(key);
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        "Key: '%s' , default: %s description: %s since version: %s deprecated after: %s)",
+        key, defaultValue, doc, sinceVersion.isPresent() ? sinceVersion.get() : "version is not defined",
+        deprecatedVersion.isPresent() ? deprecatedVersion.get() : "version is not defined");
+  }
+
+  /**
+   * The PropertyBuilder is used to build the ConfigProperty.
+   */
+  public static final class PropertyBuilder {
+
+    private final String key;
+
+    PropertyBuilder(String key) {
+      this.key = key;
+    }
+
+    public <T> ConfigProperty<T> defaultValue(T value) {
+      Objects.requireNonNull(value);
+      ConfigProperty<T> configProperty = new ConfigProperty<>(key, value, "", Option.empty(), Option.empty(), Option.empty());
+      return configProperty;
+    }
+
+    public ConfigProperty<String> noDefaultValue() {
+      ConfigProperty<String> configProperty = new ConfigProperty<>(key, null, "", Option.empty(),
+          Option.empty(), Option.empty());
+      return configProperty;
+    }
+  }
+}
\ No newline at end of file
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/DefaultHoodieConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/DefaultHoodieConfig.java
deleted file mode 100644
index e0766db..0000000
--- a/hudi-common/src/main/java/org/apache/hudi/common/config/DefaultHoodieConfig.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hudi.common.config;
-
-import java.io.Serializable;
-import java.util.Properties;
-
-/**
- * Default Way to load Hoodie config through a {@link java.util.Properties}.
- */
-public class DefaultHoodieConfig implements Serializable {
-
-  protected final Properties props;
-
-  public DefaultHoodieConfig(Properties props) {
-    this.props = props;
-  }
-
-  public static void setDefaultOnCondition(Properties props, boolean condition, String propName, String defaultValue) {
-    if (condition) {
-      props.setProperty(propName, defaultValue);
-    }
-  }
-
-  public static void setDefaultOnCondition(Properties props, boolean condition, DefaultHoodieConfig config) {
-    if (condition) {
-      props.putAll(config.getProps());
-    }
-  }
-
-  public Properties getProps() {
-    return props;
-  }
-
-}
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java
new file mode 100644
index 0000000..97c6f49
--- /dev/null
+++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java
@@ -0,0 +1,177 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.config;
+
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.Modifier;
+import java.util.Arrays;
+import java.util.Properties;
+
+/**
+ * This class deals with {@link ConfigProperty} and provides get/set functionalities.
+ */
+public class HoodieConfig implements Serializable {
+
+  private static final Logger LOG = LogManager.getLogger(HoodieConfig.class);
+
+  public static HoodieConfig create(FSDataInputStream inputStream) throws IOException {
+    HoodieConfig config = new HoodieConfig();
+    config.props.load(inputStream);
+    return config;
+  }
+
+  protected Properties props;
+
+  public HoodieConfig() {
+    this.props = new Properties();
+  }
+
+  public HoodieConfig(Properties props) {
+    this.props = props;
+  }
+
+  public <T> void setValue(ConfigProperty<T> cfg, String val) {
+    props.setProperty(cfg.key(), val);
+  }
+
+  public <T> void setDefaultValue(ConfigProperty<T> configProperty) {
+    if (!contains(configProperty)) {
+      Option<T> inferValue = Option.empty();
+      if (configProperty.getInferFunc().isPresent()) {
+        inferValue = configProperty.getInferFunc().get().apply(this);
+      }
+      props.setProperty(configProperty.key(), inferValue.isPresent() ? inferValue.get().toString() : configProperty.defaultValue().toString());
+    }
+  }
+
+  public <T> void setDefaultValue(ConfigProperty<T> configProperty, T defaultVal) {
+    if (!contains(configProperty)) {
+      props.setProperty(configProperty.key(), defaultVal.toString());
+    }
+  }
+
+  public <T> boolean contains(ConfigProperty<T> configProperty) {
+    if (props.containsKey(configProperty.key())) {
+      return true;
+    }
+    return Arrays.stream(configProperty.getAlternatives()).anyMatch(props::containsKey);
+  }
+
+  private <T> Option<Object> getRawValue(ConfigProperty<T> configProperty) {
+    if (props.containsKey(configProperty.key())) {
+      return Option.ofNullable(props.get(configProperty.key()));
+    }
+    for (String alternative : configProperty.getAlternatives()) {
+      if (props.containsKey(alternative)) {
+        LOG.warn(String.format("The configuration key '%s' has been deprecated "
+                + "and may be removed in the future. Please use the new key '%s' instead.",
+            alternative, configProperty.key()));
+        return Option.ofNullable(props.get(alternative));
+      }
+    }
+    return Option.empty();
+  }
+
+  protected void setDefaults(String configClassName) {
+    Class<?> configClass = ReflectionUtils.getClass(configClassName);
+    Arrays.stream(configClass.getDeclaredFields())
+        .filter(f -> Modifier.isStatic(f.getModifiers()))
+        .filter(f -> f.getType().isAssignableFrom(ConfigProperty.class))
+        .forEach(f -> {
+          try {
+            ConfigProperty<?> cfgProp = (ConfigProperty<?>) f.get("null");
+            if (cfgProp.hasDefaultValue()) {
+              setDefaultValue(cfgProp);
+            }
+          } catch (IllegalAccessException e) {
+            e.printStackTrace();
+          }
+        });
+  }
+
+  public <T> String getString(ConfigProperty<T> configProperty) {
+    Option<Object> rawValue = getRawValue(configProperty);
+    return rawValue.map(Object::toString).orElse(null);
+  }
+
+  public String getString(String key) {
+    return props.getProperty(key);
+  }
+
+  public <T> Integer getInt(ConfigProperty<T> configProperty) {
+    Option<Object> rawValue = getRawValue(configProperty);
+    return rawValue.map(v -> Integer.parseInt(v.toString())).orElse(null);
+  }
+
+  public <T> Boolean getBoolean(ConfigProperty<T> configProperty) {
+    Option<Object> rawValue = getRawValue(configProperty);
+    return rawValue.map(v -> Boolean.parseBoolean(v.toString())).orElse(null);
+  }
+
+  public <T> Long getLong(ConfigProperty<T> configProperty) {
+    Option<Object> rawValue = getRawValue(configProperty);
+    return rawValue.map(v -> Long.parseLong(v.toString())).orElse(null);
+  }
+
+  public <T> Float getFloat(ConfigProperty<T> configProperty) {
+    Option<Object> rawValue = getRawValue(configProperty);
+    return rawValue.map(v -> Float.parseFloat(v.toString())).orElse(null);
+  }
+
+  public <T> Double getDouble(ConfigProperty<T> configProperty) {
+    Option<Object> rawValue = getRawValue(configProperty);
+    return rawValue.map(v -> Double.parseDouble(v.toString())).orElse(null);
+  }
+
+  public <T> String getStringOrDefault(ConfigProperty<T> configProperty) {
+    return getStringOrDefault(configProperty, configProperty.defaultValue().toString());
+  }
+
+  public <T> String getStringOrDefault(ConfigProperty<T> configProperty, String defaultVal) {
+    Option<Object> rawValue = getRawValue(configProperty);
+    return rawValue.map(Object::toString).orElse(defaultVal);
+  }
+
+  public Properties getProps() {
+    return props;
+  }
+
+  public void setDefaultOnCondition(boolean condition, HoodieConfig config) {
+    if (condition) {
+      props.putAll(config.getProps());
+    }
+  }
+
+  public <T> String getStringOrThrow(ConfigProperty<T> configProperty, String errorMessage) throws HoodieException {
+    Option<Object> rawValue = getRawValue(configProperty);
+    if (rawValue.isPresent()) {
+      return rawValue.get().toString();
+    } else {
+      throw new HoodieException(errorMessage);
+    }
+  }
+}
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 3a6b4f3..c5f7ff1 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
@@ -28,57 +28,96 @@ import java.util.Properties;
  * Configurations used by the HUDI Metadata Table.
  */
 @Immutable
-public final class HoodieMetadataConfig extends DefaultHoodieConfig {
+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 String METADATA_ENABLE_PROP = METADATA_PREFIX + ".enable";
-  public static final boolean DEFAULT_METADATA_ENABLE = false;
+  public static final ConfigProperty<Boolean> METADATA_ENABLE_PROP = ConfigProperty
+      .key(METADATA_PREFIX + ".enable")
+      .defaultValue(false)
+      .sinceVersion("0.7.0")
+      .withDocumentation("Enable the internal Metadata Table which stores table level file listings");
 
   // Validate contents of Metadata Table on each access against the actual filesystem
-  public static final String METADATA_VALIDATE_PROP = METADATA_PREFIX + ".validate";
-  public static final boolean DEFAULT_METADATA_VALIDATE = false;
+  public static final ConfigProperty<Boolean> METADATA_VALIDATE_PROP = ConfigProperty
+      .key(METADATA_PREFIX + ".validate")
+      .defaultValue(false)
+      .sinceVersion("0.7.0")
+      .withDocumentation("Validate contents of Metadata Table on each access against the actual listings from DFS");
+
   public static final boolean DEFAULT_METADATA_ENABLE_FOR_READERS = false;
 
   // Enable metrics for internal Metadata Table
-  public static final String METADATA_METRICS_ENABLE_PROP = METADATA_PREFIX + ".metrics.enable";
-  public static final boolean DEFAULT_METADATA_METRICS_ENABLE = false;
+  public static final ConfigProperty<Boolean> METADATA_METRICS_ENABLE_PROP = ConfigProperty
+      .key(METADATA_PREFIX + ".metrics.enable")
+      .defaultValue(false)
+      .sinceVersion("0.7.0")
+      .withDocumentation("");
 
   // Parallelism for inserts
-  public static final String METADATA_INSERT_PARALLELISM_PROP = METADATA_PREFIX + ".insert.parallelism";
-  public static final int DEFAULT_METADATA_INSERT_PARALLELISM = 1;
+  public static final ConfigProperty<Integer> METADATA_INSERT_PARALLELISM_PROP = ConfigProperty
+      .key(METADATA_PREFIX + ".insert.parallelism")
+      .defaultValue(1)
+      .sinceVersion("0.7.0")
+      .withDocumentation("Parallelism to use when writing to the metadata table");
 
   // Async clean
-  public static final String METADATA_ASYNC_CLEAN_PROP = METADATA_PREFIX + ".clean.async";
-  public static final boolean DEFAULT_METADATA_ASYNC_CLEAN = false;
+  public static final ConfigProperty<Boolean> METADATA_ASYNC_CLEAN_PROP = 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 String METADATA_COMPACT_NUM_DELTA_COMMITS_PROP = METADATA_PREFIX + ".compact.max.delta.commits";
-  public static final int DEFAULT_METADATA_COMPACT_NUM_DELTA_COMMITS = 24;
+  public static final ConfigProperty<Integer> METADATA_COMPACT_NUM_DELTA_COMMITS_PROP = 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 String MIN_COMMITS_TO_KEEP_PROP = METADATA_PREFIX + ".keep.min.commits";
-  public static final int DEFAULT_MIN_COMMITS_TO_KEEP = 20;
-  public static final String MAX_COMMITS_TO_KEEP_PROP = METADATA_PREFIX + ".keep.max.commits";
-  public static final int DEFAULT_MAX_COMMITS_TO_KEEP = 30;
+  public static final ConfigProperty<Integer> MIN_COMMITS_TO_KEEP_PROP = 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
+      .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 String CLEANER_COMMITS_RETAINED_PROP = METADATA_PREFIX + ".cleaner.commits.retained";
-  public static final int DEFAULT_CLEANER_COMMITS_RETAINED = 3;
+  public static final ConfigProperty<Integer> CLEANER_COMMITS_RETAINED_PROP = ConfigProperty
+      .key(METADATA_PREFIX + ".cleaner.commits.retained")
+      .defaultValue(3)
+      .sinceVersion("0.7.0")
+      .withDocumentation("");
 
   // Regex to filter out matching directories during bootstrap
-  public static final String DIRECTORY_FILTER_REGEX = METADATA_PREFIX + ".dir.filter.regex";
-  public static final String DEFAULT_DIRECTORY_FILTER_REGEX = "";
-
-  public static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = "hoodie.assume.date.partitioning";
-  public static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false";
-
-  public static final String FILE_LISTING_PARALLELISM_PROP = "hoodie.file.listing.parallelism";
-  public static final int DEFAULT_FILE_LISTING_PARALLELISM = 1500;
-
-  private HoodieMetadataConfig(Properties props) {
-    super(props);
+  public static final ConfigProperty<String> DIRECTORY_FILTER_REGEX = ConfigProperty
+      .key(METADATA_PREFIX + ".dir.filter.regex")
+      .defaultValue("")
+      .sinceVersion("0.7.0")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HOODIE_ASSUME_DATE_PARTITIONING_PROP = ConfigProperty
+      .key("hoodie.assume.date.partitioning")
+      .defaultValue("false")
+      .sinceVersion("0.7.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
+      .key("hoodie.file.listing.parallelism")
+      .defaultValue(1500)
+      .sinceVersion("0.7.0")
+      .withDocumentation("");
+
+  private HoodieMetadataConfig() {
+    super();
   }
 
   public static HoodieMetadataConfig.Builder newBuilder() {
@@ -86,128 +125,104 @@ public final class HoodieMetadataConfig extends DefaultHoodieConfig {
   }
 
   public int getFileListingParallelism() {
-    return Math.max(Integer.parseInt(props.getProperty(HoodieMetadataConfig.FILE_LISTING_PARALLELISM_PROP)), 1);
+    return Math.max(getInt(HoodieMetadataConfig.FILE_LISTING_PARALLELISM_PROP), 1);
   }
 
   public Boolean shouldAssumeDatePartitioning() {
-    return Boolean.parseBoolean(props.getProperty(HoodieMetadataConfig.HOODIE_ASSUME_DATE_PARTITIONING_PROP));
+    return getBoolean(HoodieMetadataConfig.HOODIE_ASSUME_DATE_PARTITIONING_PROP);
   }
 
   public boolean useFileListingMetadata() {
-    return Boolean.parseBoolean(props.getProperty(METADATA_ENABLE_PROP));
+    return getBoolean(METADATA_ENABLE_PROP);
   }
 
   public boolean validateFileListingMetadata() {
-    return Boolean.parseBoolean(props.getProperty(METADATA_VALIDATE_PROP));
+    return getBoolean(METADATA_VALIDATE_PROP);
   }
 
   public boolean enableMetrics() {
-    return Boolean.parseBoolean(props.getProperty(METADATA_METRICS_ENABLE_PROP));
+    return getBoolean(METADATA_METRICS_ENABLE_PROP);
   }
 
   public String getDirectoryFilterRegex() {
-    return props.getProperty(DIRECTORY_FILTER_REGEX);
+    return getString(DIRECTORY_FILTER_REGEX);
   }
 
   public static class Builder {
 
-    private final Properties props = new Properties();
+    private final HoodieMetadataConfig metadataConfig = new HoodieMetadataConfig();
 
     public Builder fromFile(File propertiesFile) throws IOException {
       try (FileReader reader = new FileReader(propertiesFile)) {
-        this.props.load(reader);
+        this.metadataConfig.getProps().load(reader);
         return this;
       }
     }
 
     public Builder fromProperties(Properties props) {
-      this.props.putAll(props);
+      this.metadataConfig.getProps().putAll(props);
       return this;
     }
 
     public Builder enable(boolean enable) {
-      props.setProperty(METADATA_ENABLE_PROP, String.valueOf(enable));
+      metadataConfig.setValue(METADATA_ENABLE_PROP, String.valueOf(enable));
       return this;
     }
 
     public Builder enableMetrics(boolean enableMetrics) {
-      props.setProperty(METADATA_METRICS_ENABLE_PROP, String.valueOf(enableMetrics));
+      metadataConfig.setValue(METADATA_METRICS_ENABLE_PROP, String.valueOf(enableMetrics));
       return this;
     }
 
     public Builder validate(boolean validate) {
-      props.setProperty(METADATA_VALIDATE_PROP, String.valueOf(validate));
+      metadataConfig.setValue(METADATA_VALIDATE_PROP, String.valueOf(validate));
       return this;
     }
 
     public Builder withInsertParallelism(int parallelism) {
-      props.setProperty(METADATA_INSERT_PARALLELISM_PROP, String.valueOf(parallelism));
+      metadataConfig.setValue(METADATA_INSERT_PARALLELISM_PROP, String.valueOf(parallelism));
       return this;
     }
 
     public Builder withAsyncClean(boolean asyncClean) {
-      props.setProperty(METADATA_ASYNC_CLEAN_PROP, String.valueOf(asyncClean));
+      metadataConfig.setValue(METADATA_ASYNC_CLEAN_PROP, String.valueOf(asyncClean));
       return this;
     }
 
     public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) {
-      props.setProperty(METADATA_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(maxNumDeltaCommitsBeforeCompaction));
+      metadataConfig.setValue(METADATA_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(maxNumDeltaCommitsBeforeCompaction));
       return this;
     }
 
     public Builder archiveCommitsWith(int minToKeep, int maxToKeep) {
-      props.setProperty(MIN_COMMITS_TO_KEEP_PROP, String.valueOf(minToKeep));
-      props.setProperty(MAX_COMMITS_TO_KEEP_PROP, String.valueOf(maxToKeep));
+      metadataConfig.setValue(MIN_COMMITS_TO_KEEP_PROP, String.valueOf(minToKeep));
+      metadataConfig.setValue(MAX_COMMITS_TO_KEEP_PROP, String.valueOf(maxToKeep));
       return this;
     }
 
     public Builder retainCommits(int commitsRetained) {
-      props.setProperty(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
+      metadataConfig.setValue(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
       return this;
     }
 
     public Builder withFileListingParallelism(int parallelism) {
-      props.setProperty(FILE_LISTING_PARALLELISM_PROP, String.valueOf(parallelism));
+      metadataConfig.setValue(FILE_LISTING_PARALLELISM_PROP, String.valueOf(parallelism));
       return this;
     }
 
     public Builder withAssumeDatePartitioning(boolean assumeDatePartitioning) {
-      props.setProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP, String.valueOf(assumeDatePartitioning));
+      metadataConfig.setValue(HOODIE_ASSUME_DATE_PARTITIONING_PROP, String.valueOf(assumeDatePartitioning));
       return this;
     }
 
     public Builder withDirectoryFilterRegex(String regex) {
-      props.setProperty(DIRECTORY_FILTER_REGEX, regex);
+      metadataConfig.setValue(DIRECTORY_FILTER_REGEX, regex);
       return this;
     }
 
     public HoodieMetadataConfig build() {
-      HoodieMetadataConfig config = new HoodieMetadataConfig(props);
-      setDefaultOnCondition(props, !props.containsKey(METADATA_ENABLE_PROP), METADATA_ENABLE_PROP,
-          String.valueOf(DEFAULT_METADATA_ENABLE));
-      setDefaultOnCondition(props, !props.containsKey(METADATA_METRICS_ENABLE_PROP), METADATA_METRICS_ENABLE_PROP,
-          String.valueOf(DEFAULT_METADATA_METRICS_ENABLE));
-      setDefaultOnCondition(props, !props.containsKey(METADATA_VALIDATE_PROP), METADATA_VALIDATE_PROP,
-          String.valueOf(DEFAULT_METADATA_VALIDATE));
-      setDefaultOnCondition(props, !props.containsKey(METADATA_INSERT_PARALLELISM_PROP), METADATA_INSERT_PARALLELISM_PROP,
-          String.valueOf(DEFAULT_METADATA_INSERT_PARALLELISM));
-      setDefaultOnCondition(props, !props.containsKey(METADATA_ASYNC_CLEAN_PROP), METADATA_ASYNC_CLEAN_PROP,
-          String.valueOf(DEFAULT_METADATA_ASYNC_CLEAN));
-      setDefaultOnCondition(props, !props.containsKey(METADATA_COMPACT_NUM_DELTA_COMMITS_PROP),
-          METADATA_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(DEFAULT_METADATA_COMPACT_NUM_DELTA_COMMITS));
-      setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP), CLEANER_COMMITS_RETAINED_PROP,
-          String.valueOf(DEFAULT_CLEANER_COMMITS_RETAINED));
-      setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP_PROP), MAX_COMMITS_TO_KEEP_PROP,
-          String.valueOf(DEFAULT_MAX_COMMITS_TO_KEEP));
-      setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP_PROP), MIN_COMMITS_TO_KEEP_PROP,
-          String.valueOf(DEFAULT_MIN_COMMITS_TO_KEEP));
-      setDefaultOnCondition(props, !props.containsKey(FILE_LISTING_PARALLELISM_PROP), FILE_LISTING_PARALLELISM_PROP,
-          String.valueOf(DEFAULT_FILE_LISTING_PARALLELISM));
-      setDefaultOnCondition(props, !props.containsKey(HOODIE_ASSUME_DATE_PARTITIONING_PROP),
-          HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING);
-      setDefaultOnCondition(props, !props.containsKey(DIRECTORY_FILTER_REGEX), DIRECTORY_FILTER_REGEX,
-          DEFAULT_DIRECTORY_FILTER_REGEX);
-      return config;
+      metadataConfig.setDefaults(HoodieMetadataConfig.class.getName());
+      return metadataConfig;
     }
   }
 }
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 2a1eea3..8f16149 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
@@ -28,36 +28,51 @@ import java.util.Properties;
 public class LockConfiguration implements Serializable {
 
   public static final String LOCK_PREFIX = "hoodie.write.lock.";
-  public static final String LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP = LOCK_PREFIX + "wait_time_ms_between_retry";
+
+  public static final String LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY = LOCK_PREFIX + "wait_time_ms_between_retry";
   public static final String DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS = String.valueOf(5000L);
-  public static final String LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP = LOCK_PREFIX + "max_wait_time_ms_between_retry";
-  public static final String DEFAULT_LOCK_ACQUIRE_MAX_RETRY_WAIT_TIME_IN_MILLIS = String.valueOf(5000L);
-  public static final String LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP = LOCK_PREFIX + "client.wait_time_ms_between_retry";
-  public static final String DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS = String.valueOf(10000L);
-  public static final String LOCK_ACQUIRE_NUM_RETRIES_PROP = LOCK_PREFIX + "num_retries";
+
+  public static final String LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP_KEY = LOCK_PREFIX + "max_wait_time_ms_between_retry";
+
+  public static final String LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY = LOCK_PREFIX + "client.wait_time_ms_between_retry";
+
+  public static final String LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY = LOCK_PREFIX + "num_retries";
   public static final String DEFAULT_LOCK_ACQUIRE_NUM_RETRIES = String.valueOf(3);
-  public static final String LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP = LOCK_PREFIX + "client.num_retries";
-  public static final String DEFAULT_LOCK_ACQUIRE_CLIENT_NUM_RETRIES = String.valueOf(0);
-  public static final String LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP = LOCK_PREFIX + "wait_time_ms";
-  public static final int DEFAULT_ACQUIRE_LOCK_WAIT_TIMEOUT_MS = 60 * 1000;
+
+  public static final String LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY = LOCK_PREFIX + "client.num_retries";
+
+  public static final String LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY = LOCK_PREFIX + "wait_time_ms";
+
   // configs for file system based locks. NOTE: This only works for DFS with atomic create/delete operation
   public static final String FILESYSTEM_BASED_LOCK_PROPERTY_PREFIX = LOCK_PREFIX + "filesystem.";
-  public static final String FILESYSTEM_LOCK_PATH_PROP = FILESYSTEM_BASED_LOCK_PROPERTY_PREFIX + "path";
+
+  public static final String FILESYSTEM_LOCK_PATH_PROP_KEY = FILESYSTEM_BASED_LOCK_PROPERTY_PREFIX + "path";
+
   // configs for metastore based locks
   public static final String HIVE_METASTORE_LOCK_PROPERTY_PREFIX = LOCK_PREFIX + "hivemetastore.";
-  public static final String HIVE_DATABASE_NAME_PROP = HIVE_METASTORE_LOCK_PROPERTY_PREFIX + "database";
-  public static final String HIVE_TABLE_NAME_PROP = HIVE_METASTORE_LOCK_PROPERTY_PREFIX + "table";
-  public static final String HIVE_METASTORE_URI_PROP = HIVE_METASTORE_LOCK_PROPERTY_PREFIX + "uris";
+
+  public static final String HIVE_DATABASE_NAME_PROP_KEY = HIVE_METASTORE_LOCK_PROPERTY_PREFIX + "database";
+
+  public static final String HIVE_TABLE_NAME_PROP_KEY = HIVE_METASTORE_LOCK_PROPERTY_PREFIX + "table";
+
+  public static final String HIVE_METASTORE_URI_PROP_KEY = HIVE_METASTORE_LOCK_PROPERTY_PREFIX + "uris";
+
   // Zookeeper configs for zk based locks
   public static final String ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX = LOCK_PREFIX + "zookeeper.";
-  public static final String ZK_BASE_PATH_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "base_path";
-  public static final String ZK_SESSION_TIMEOUT_MS_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "session_timeout_ms";
+
+  public static final String ZK_BASE_PATH_PROP_KEY = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "base_path";
+
+  public static final String ZK_SESSION_TIMEOUT_MS_PROP_KEY = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "session_timeout_ms";
   public static final int DEFAULT_ZK_SESSION_TIMEOUT_MS = 60 * 1000;
-  public static final String ZK_CONNECTION_TIMEOUT_MS_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "connection_timeout_ms";
+
+  public static final String ZK_CONNECTION_TIMEOUT_MS_PROP_KEY = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "connection_timeout_ms";
   public static final int DEFAULT_ZK_CONNECTION_TIMEOUT_MS = 15 * 1000;
-  public static final String ZK_CONNECT_URL_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "url";
-  public static final String ZK_PORT_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "port";
-  public static final String ZK_LOCK_KEY_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "lock_key";
+
+  public static final String ZK_CONNECT_URL_PROP_KEY = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "url";
+
+  public static final String ZK_PORT_PROP_KEY = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "port";
+
+  public static final String ZK_LOCK_KEY_PROP_KEY = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "lock_key";
 
   private final TypedProperties 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 e55fb24..d7acf61 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
@@ -18,7 +18,8 @@
 
 package org.apache.hudi.common.fs;
 
-import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 
 import java.io.File;
 import java.io.FileReader;
@@ -28,34 +29,53 @@ import java.util.Properties;
 /**
  * The consistency guard relevant config options.
  */
-public class ConsistencyGuardConfig extends DefaultHoodieConfig {
-
-  private static final String CONSISTENCY_CHECK_ENABLED_PROP = "hoodie.consistency.check.enabled";
-  private static final String DEFAULT_CONSISTENCY_CHECK_ENABLED = "false";
+public class ConsistencyGuardConfig extends HoodieConfig {
 
   // time between successive attempts to ensure written data's metadata is consistent on storage
-  private static final String INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP =
-      "hoodie.consistency.check.initial_interval_ms";
-  private static long DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = 400L;
+  @Deprecated
+  public static final ConfigProperty<String> CONSISTENCY_CHECK_ENABLED_PROP = ConfigProperty
+      .key("hoodie.consistency.check.enabled")
+      .defaultValue("false")
+      .sinceVersion("0.5.0")
+      .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
+      .key("hoodie.consistency.check.initial_interval_ms")
+      .defaultValue(400L)
+      .sinceVersion("0.5.0")
+      .withDocumentation("");
 
   // max interval time
-  private static final String MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = "hoodie.consistency.check.max_interval_ms";
-  private static long DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS = 20000L;
+  public static final ConfigProperty<Long> MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = ConfigProperty
+      .key("hoodie.consistency.check.max_interval_ms")
+      .defaultValue(20000L)
+      .sinceVersion("0.5.0")
+      .withDocumentation("");
 
   // maximum number of checks, for consistency of written data. Will wait upto 140 Secs
-  private static final String MAX_CONSISTENCY_CHECKS_PROP = "hoodie.consistency.check.max_checks";
-  private static int DEFAULT_MAX_CONSISTENCY_CHECKS = 6;
+  public static final ConfigProperty<Integer> MAX_CONSISTENCY_CHECKS_PROP = ConfigProperty
+      .key("hoodie.consistency.check.max_checks")
+      .defaultValue(6)
+      .sinceVersion("0.5.0")
+      .withDocumentation("");
 
   // sleep time for OptimisticConsistencyGuard
-  private static final String OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP = "hoodie.optimistic.consistency.guard.sleep_time_ms";
-  private static long DEFAULT_OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP = 500L;
+  public static final ConfigProperty<Long> OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP = ConfigProperty
+      .key("hoodie.optimistic.consistency.guard.sleep_time_ms")
+      .defaultValue(500L)
+      .sinceVersion("0.6.0")
+      .withDocumentation("");
 
   // config to enable OptimisticConsistencyGuard in finalizeWrite instead of FailSafeConsistencyGuard
-  private static final String ENABLE_OPTIMISTIC_CONSISTENCY_GUARD = "_hoodie.optimistic.consistency.guard.enable";
-  private static boolean DEFAULT_ENABLE_OPTIMISTIC_CONSISTENCY_GUARD = true;
-
-  public ConsistencyGuardConfig(Properties props) {
-    super(props);
+  public static final ConfigProperty<Boolean> ENABLE_OPTIMISTIC_CONSISTENCY_GUARD_PROP = ConfigProperty
+      .key("_hoodie.optimistic.consistency.guard.enable")
+      .defaultValue(true)
+      .sinceVersion("0.6.0")
+      .withDocumentation("");
+
+  private ConsistencyGuardConfig() {
+    super();
   }
 
   public static ConsistencyGuardConfig.Builder newBuilder() {
@@ -63,27 +83,27 @@ public class ConsistencyGuardConfig extends DefaultHoodieConfig {
   }
 
   public boolean isConsistencyCheckEnabled() {
-    return Boolean.parseBoolean(props.getProperty(CONSISTENCY_CHECK_ENABLED_PROP));
+    return getBoolean(CONSISTENCY_CHECK_ENABLED_PROP);
   }
 
   public int getMaxConsistencyChecks() {
-    return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECKS_PROP));
+    return getInt(MAX_CONSISTENCY_CHECKS_PROP);
   }
 
   public int getInitialConsistencyCheckIntervalMs() {
-    return Integer.parseInt(props.getProperty(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
+    return getInt(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP);
   }
 
   public int getMaxConsistencyCheckIntervalMs() {
-    return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
+    return getInt(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP);
   }
 
   public long getOptimisticConsistencyGuardSleepTimeMs() {
-    return Long.parseLong(props.getProperty(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP));
+    return getLong(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP);
   }
 
   public boolean shouldEnableOptimisticConsistencyGuard() {
-    return Boolean.parseBoolean(props.getProperty(ENABLE_OPTIMISTIC_CONSISTENCY_GUARD));
+    return getBoolean(ENABLE_OPTIMISTIC_CONSISTENCY_GUARD_PROP);
   }
 
   /**
@@ -91,65 +111,53 @@ public class ConsistencyGuardConfig extends DefaultHoodieConfig {
    */
   public static class Builder {
 
-    private final Properties props = new Properties();
+    private final ConsistencyGuardConfig consistencyGuardConfig = new ConsistencyGuardConfig();
 
     public Builder fromFile(File propertiesFile) throws IOException {
       try (FileReader reader = new FileReader(propertiesFile)) {
-        props.load(reader);
+        consistencyGuardConfig.getProps().load(reader);
         return this;
       }
     }
 
     public Builder fromProperties(Properties props) {
-      this.props.putAll(props);
+      this.consistencyGuardConfig.getProps().putAll(props);
       return this;
     }
 
     public Builder withConsistencyCheckEnabled(boolean enabled) {
-      props.setProperty(CONSISTENCY_CHECK_ENABLED_PROP, String.valueOf(enabled));
+      consistencyGuardConfig.setValue(CONSISTENCY_CHECK_ENABLED_PROP, String.valueOf(enabled));
       return this;
     }
 
     public Builder withInitialConsistencyCheckIntervalMs(int initialIntevalMs) {
-      props.setProperty(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(initialIntevalMs));
+      consistencyGuardConfig.setValue(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(initialIntevalMs));
       return this;
     }
 
     public Builder withMaxConsistencyCheckIntervalMs(int maxIntervalMs) {
-      props.setProperty(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(maxIntervalMs));
+      consistencyGuardConfig.setValue(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(maxIntervalMs));
       return this;
     }
 
     public Builder withMaxConsistencyChecks(int maxConsistencyChecks) {
-      props.setProperty(MAX_CONSISTENCY_CHECKS_PROP, String.valueOf(maxConsistencyChecks));
+      consistencyGuardConfig.setValue(MAX_CONSISTENCY_CHECKS_PROP, String.valueOf(maxConsistencyChecks));
       return this;
     }
 
     public Builder withOptimisticConsistencyGuardSleepTimeMs(long sleepTimeMs) {
-      props.setProperty(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP, String.valueOf(sleepTimeMs));
+      consistencyGuardConfig.setValue(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP, String.valueOf(sleepTimeMs));
       return this;
     }
 
     public Builder withEnableOptimisticConsistencyGuard(boolean enableOptimisticConsistencyGuard) {
-      props.setProperty(ENABLE_OPTIMISTIC_CONSISTENCY_GUARD, String.valueOf(enableOptimisticConsistencyGuard));
+      consistencyGuardConfig.setValue(ENABLE_OPTIMISTIC_CONSISTENCY_GUARD_PROP, String.valueOf(enableOptimisticConsistencyGuard));
       return this;
     }
 
     public ConsistencyGuardConfig build() {
-      setDefaultOnCondition(props, !props.containsKey(CONSISTENCY_CHECK_ENABLED_PROP), CONSISTENCY_CHECK_ENABLED_PROP,
-          DEFAULT_CONSISTENCY_CHECK_ENABLED);
-      setDefaultOnCondition(props, !props.containsKey(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
-          INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS));
-      setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
-          MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS));
-      setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECKS_PROP), MAX_CONSISTENCY_CHECKS_PROP,
-          String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECKS));
-      setDefaultOnCondition(props, !props.containsKey(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP),
-          OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP,  String.valueOf(DEFAULT_OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP));
-      setDefaultOnCondition(props, !props.containsKey(ENABLE_OPTIMISTIC_CONSISTENCY_GUARD),
-          ENABLE_OPTIMISTIC_CONSISTENCY_GUARD,
-          String.valueOf(DEFAULT_ENABLE_OPTIMISTIC_CONSISTENCY_GUARD));
-      return new ConsistencyGuardConfig(props);
+      consistencyGuardConfig.setDefaults(ConsistencyGuardConfig.class.getName());
+      return consistencyGuardConfig;
     }
   }
 }
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 2fbe19f..8669fb3 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
@@ -130,7 +130,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.DEFAULT_BASE_FILE_FORMAT.getFileExtension());
+        HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension());
   }
 
   public static String makeDataFileName(String instantTime, String writeToken, String fileId, String fileExtension) {
@@ -142,7 +142,8 @@ public class FSUtils {
   }
 
   public static String maskWithoutFileId(String instantTime, int taskPartitionId) {
-    return String.format("*_%s_%s%s", taskPartitionId, instantTime, HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension());
+    return String.format("*_%s_%s%s", taskPartitionId, instantTime, HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP
+        .defaultValue().getFileExtension());
   }
 
   public static String getCommitFromCommitFile(String commitFileName) {
@@ -259,7 +260,7 @@ public class FSUtils {
         .withAssumeDatePartitioning(assumeDatePartitioning)
         .build();
     try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr,
-        FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR)) {
+        FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue())) {
       return tableMetadata.getAllPartitionPaths();
     } catch (Exception e) {
       throw new HoodieException("Error fetching partition paths from metadata table", e);
@@ -269,7 +270,7 @@ public class FSUtils {
   public static List<String> getAllPartitionPaths(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
                                                   String basePathStr) {
     try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr,
-        FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR)) {
+        FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue())) {
       return tableMetadata.getAllPartitionPaths();
     } catch (Exception e) {
       throw new HoodieException("Error fetching partition paths from metadata table", e);
@@ -279,7 +280,7 @@ public class FSUtils {
   public static FileStatus[] getFilesInPartition(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
                                                  String basePathStr, Path partitionPath) {
     try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext,
-        metadataConfig, basePathStr, FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR)) {
+        metadataConfig, basePathStr, FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue())) {
       return tableMetadata.getAllFilesInPartition(partitionPath);
     } catch (Exception e) {
       throw new HoodieException("Error get files in partition: " + partitionPath, e);
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java b/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java
index 1cf32f1..86ccf67 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/model/DefaultHoodieRecordPayload.java
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.common.model;
 
+import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.common.util.Option;
 
 import org.apache.avro.Schema;
@@ -55,6 +56,7 @@ public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload {
     if (recordBytes.length == 0) {
       return Option.empty();
     }
+    HoodieConfig hoodieConfig = new HoodieConfig(properties);
     GenericRecord incomingRecord = bytesToAvro(recordBytes, schema);
 
     // Null check is needed here to support schema evolution. The record in storage may be from old schema where
@@ -66,7 +68,8 @@ public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload {
     /*
      * We reached a point where the value is disk is older than the incoming record.
      */
-    eventTime = Option.ofNullable(getNestedFieldVal(incomingRecord, properties.getProperty(HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP), true));
+    eventTime = Option.ofNullable(getNestedFieldVal(incomingRecord, hoodieConfig
+        .getString(HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP_KEY), true));
 
     /*
      * Now check if the incoming record is a delete record.
@@ -99,9 +102,9 @@ public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload {
      * and need to be dealt with separately.
      */
     Object persistedOrderingVal = getNestedFieldVal((GenericRecord) currentValue,
-        properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP), true);
+        properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), true);
     Comparable incomingOrderingVal = (Comparable) getNestedFieldVal((GenericRecord) incomingRecord,
-        properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP), false);
+        properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), false);
     return persistedOrderingVal == null || ((Comparable) persistedOrderingVal).compareTo(incomingOrderingVal) <= 0;
   }
 }
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 48dde2a..d6067f2 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
@@ -30,14 +30,12 @@ public class HoodiePayloadProps {
    *
    * @see DefaultHoodieRecordPayload
    */
-  public static final String PAYLOAD_ORDERING_FIELD_PROP = "hoodie.payload.ordering.field";
-  public static String DEFAULT_PAYLOAD_ORDERING_FIELD_VAL = "ts";
+  public static final String PAYLOAD_ORDERING_FIELD_PROP_KEY = "hoodie.payload.ordering.field";
 
   /**
    * Property for payload event time field; to be used to extract source event time info.
    *
    * @see DefaultHoodieRecordPayload
    */
-  public static final String PAYLOAD_EVENT_TIME_FIELD_PROP = "hoodie.payload.event.time.field";
-  public static String DEFAULT_PAYLOAD_EVENT_TIME_FIELD_VAL = "ts";
+  public static final String PAYLOAD_EVENT_TIME_FIELD_PROP_KEY = "hoodie.payload.event.time.field";
 }
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 6670739..3a48c92 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
@@ -18,10 +18,10 @@
 
 package org.apache.hudi.common.table;
 
-import java.util.Arrays;
-import org.apache.avro.Schema;
 import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
 import org.apache.hudi.common.bootstrap.index.NoOpBootstrapIndex;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.common.model.HoodieFileFormat;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
@@ -30,6 +30,7 @@ import org.apache.hudi.common.util.Option;
 import org.apache.hudi.common.util.ValidationUtils;
 import org.apache.hudi.exception.HoodieIOException;
 
+import org.apache.avro.Schema;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -39,6 +40,7 @@ import org.apache.log4j.Logger;
 
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.Arrays;
 import java.util.Date;
 import java.util.Map;
 import java.util.Properties;
@@ -51,54 +53,104 @@ import java.util.stream.Collectors;
  * @see HoodieTableMetaClient
  * @since 0.3.0
  */
-public class HoodieTableConfig implements Serializable {
+public class HoodieTableConfig extends HoodieConfig implements Serializable {
 
   private static final Logger LOG = LogManager.getLogger(HoodieTableConfig.class);
 
   public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
-  public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name";
-  public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type";
-  public static final String HOODIE_TABLE_VERSION_PROP_NAME = "hoodie.table.version";
-  public static final String HOODIE_TABLE_PRECOMBINE_FIELD = "hoodie.table.precombine.field";
-  public static final String HOODIE_TABLE_PARTITION_COLUMNS = "hoodie.table.partition.columns";
-  public static final String HOODIE_TABLE_RECORDKEY_FIELDS = "hoodie.table.recordkey.fields";
-  public static final String HOODIE_TABLE_CREATE_SCHEMA = "hoodie.table.create.schema";
-
-  @Deprecated
-  public static final String HOODIE_RO_FILE_FORMAT_PROP_NAME = "hoodie.table.ro.file.format";
-  @Deprecated
-  public static final String HOODIE_RT_FILE_FORMAT_PROP_NAME = "hoodie.table.rt.file.format";
-  public static final String HOODIE_BASE_FILE_FORMAT_PROP_NAME = "hoodie.table.base.file.format";
-  public static final String HOODIE_LOG_FILE_FORMAT_PROP_NAME = "hoodie.table.log.file.format";
-  public static final String HOODIE_TIMELINE_LAYOUT_VERSION = "hoodie.timeline.layout.version";
-  public static final String HOODIE_PAYLOAD_CLASS_PROP_NAME = "hoodie.compaction.payload.class";
-  public static final String HOODIE_ARCHIVELOG_FOLDER_PROP_NAME = "hoodie.archivelog.folder";
-  public static final String HOODIE_BOOTSTRAP_INDEX_ENABLE = "hoodie.bootstrap.index.enable";
-  public static final String HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME = "hoodie.bootstrap.index.class";
-  public static final String HOODIE_BOOTSTRAP_BASE_PATH = "hoodie.bootstrap.base.path";
-
-  public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
-  public static final HoodieTableVersion DEFAULT_TABLE_VERSION = HoodieTableVersion.ZERO;
-  public static final HoodieFileFormat DEFAULT_BASE_FILE_FORMAT = HoodieFileFormat.PARQUET;
-  public static final HoodieFileFormat DEFAULT_LOG_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG;
-  public static final String DEFAULT_PAYLOAD_CLASS = OverwriteWithLatestAvroPayload.class.getName();
-  public static final String NO_OP_BOOTSTRAP_INDEX_CLASS = NoOpBootstrapIndex.class.getName();
-  public static final String DEFAULT_BOOTSTRAP_INDEX_CLASS = HFileBootstrapIndex.class.getName();
-  public static final String DEFAULT_ARCHIVELOG_FOLDER = "archived";
 
-  private Properties props;
+  public static final ConfigProperty<String> HOODIE_TABLE_NAME_PROP = 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
+      .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
+      .key("hoodie.table.version")
+      .defaultValue(HoodieTableVersion.ZERO)
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HOODIE_TABLE_PRECOMBINE_FIELD_PROP = ConfigProperty
+      .key("hoodie.table.precombine.field")
+      .noDefaultValue()
+      .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(..)");
+
+  public static final ConfigProperty<String> HOODIE_TABLE_PARTITION_COLUMNS_PROP = ConfigProperty
+      .key("hoodie.table.partition.columns")
+      .noDefaultValue()
+      .withDocumentation("Partition path field. Value to be used at the partitionPath component of HoodieKey. "
+          + "Actual value ontained by invoking .toString()");
+
+  public static final ConfigProperty<String> HOODIE_TABLE_RECORDKEY_FIELDS = ConfigProperty
+      .key("hoodie.table.recordkey.fields")
+      .noDefaultValue()
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HOODIE_TABLE_CREATE_SCHEMA = ConfigProperty
+      .key("hoodie.table.create.schema")
+      .noDefaultValue()
+      .withDocumentation("");
+
+  public static final ConfigProperty<HoodieFileFormat> HOODIE_BASE_FILE_FORMAT_PROP = ConfigProperty
+      .key("hoodie.table.base.file.format")
+      .defaultValue(HoodieFileFormat.PARQUET)
+      .withAlternatives("hoodie.table.ro.file.format")
+      .withDocumentation("");
+
+  public static final ConfigProperty<HoodieFileFormat> HOODIE_LOG_FILE_FORMAT_PROP = ConfigProperty
+      .key("hoodie.table.log.file.format")
+      .defaultValue(HoodieFileFormat.HOODIE_LOG)
+      .withAlternatives("hoodie.table.rt.file.format")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HOODIE_TIMELINE_LAYOUT_VERSION_PROP = ConfigProperty
+      .key("hoodie.timeline.layout.version")
+      .noDefaultValue()
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HOODIE_PAYLOAD_CLASS_PROP = ConfigProperty
+      .key("hoodie.compaction.payload.class")
+      .defaultValue(OverwriteWithLatestAvroPayload.class.getName())
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HOODIE_ARCHIVELOG_FOLDER_PROP = ConfigProperty
+      .key("hoodie.archivelog.folder")
+      .defaultValue("archived")
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HOODIE_BOOTSTRAP_INDEX_ENABLE_PROP = ConfigProperty
+      .key("hoodie.bootstrap.index.enable")
+      .noDefaultValue()
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HOODIE_BOOTSTRAP_INDEX_CLASS_PROP = ConfigProperty
+      .key("hoodie.bootstrap.index.class")
+      .defaultValue(HFileBootstrapIndex.class.getName())
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> HOODIE_BOOTSTRAP_BASE_PATH_PROP = 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 String NO_OP_BOOTSTRAP_INDEX_CLASS = NoOpBootstrapIndex.class.getName();
 
   public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName) {
-    Properties props = new Properties();
+    super();
     Path propertyPath = new Path(metaPath, HOODIE_PROPERTIES_FILE);
     LOG.info("Loading table properties from " + propertyPath);
     try {
       try (FSDataInputStream inputStream = fs.open(propertyPath)) {
         props.load(inputStream);
       }
-      if (props.containsKey(HOODIE_PAYLOAD_CLASS_PROP_NAME) && payloadClassName != null
-          && !props.getProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME).equals(payloadClassName)) {
-        props.setProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName);
+      if (contains(HOODIE_PAYLOAD_CLASS_PROP) && payloadClassName != null
+          && !getString(HOODIE_PAYLOAD_CLASS_PROP).equals(payloadClassName)) {
+        setValue(HOODIE_PAYLOAD_CLASS_PROP, payloadClassName);
         try (FSDataOutputStream outputStream = fs.create(propertyPath)) {
           props.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
         }
@@ -106,21 +158,16 @@ public class HoodieTableConfig implements Serializable {
     } catch (IOException e) {
       throw new HoodieIOException("Could not load Hoodie properties from " + propertyPath, e);
     }
-    this.props = props;
-    ValidationUtils.checkArgument(props.containsKey(HOODIE_TABLE_TYPE_PROP_NAME) && props.containsKey(HOODIE_TABLE_NAME_PROP_NAME),
+    ValidationUtils.checkArgument(contains(HOODIE_TABLE_TYPE_PROP) && contains(HOODIE_TABLE_NAME_PROP),
         "hoodie.properties file seems invalid. Please check for left over `.updated` files if any, manually copy it to hoodie.properties and retry");
   }
 
-  public HoodieTableConfig(Properties props) {
-    this.props = props;
-  }
-
   /**
-   * For serailizing and de-serializing.
+   * For serializing and de-serializing.
    *
-   * @deprecated
    */
   public HoodieTableConfig() {
+    super();
   }
 
   /**
@@ -131,30 +178,26 @@ public class HoodieTableConfig implements Serializable {
     if (!fs.exists(metadataFolder)) {
       fs.mkdirs(metadataFolder);
     }
+    HoodieConfig hoodieConfig = new HoodieConfig(properties);
     Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
     try (FSDataOutputStream outputStream = fs.create(propertyPath)) {
-      if (!properties.containsKey(HOODIE_TABLE_NAME_PROP_NAME)) {
-        throw new IllegalArgumentException(HOODIE_TABLE_NAME_PROP_NAME + " property needs to be specified");
-      }
-      if (!properties.containsKey(HOODIE_TABLE_TYPE_PROP_NAME)) {
-        properties.setProperty(HOODIE_TABLE_TYPE_PROP_NAME, DEFAULT_TABLE_TYPE.name());
+      if (!hoodieConfig.contains(HOODIE_TABLE_NAME_PROP)) {
+        throw new IllegalArgumentException(HOODIE_TABLE_NAME_PROP.key() + " property needs to be specified");
       }
-      if (properties.getProperty(HOODIE_TABLE_TYPE_PROP_NAME).equals(HoodieTableType.MERGE_ON_READ.name())
-          && !properties.containsKey(HOODIE_PAYLOAD_CLASS_PROP_NAME)) {
-        properties.setProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS);
+      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);
       }
-      if (!properties.containsKey(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME)) {
-        properties.setProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER);
-      }
-      if (!properties.containsKey(HOODIE_TIMELINE_LAYOUT_VERSION)) {
+      hoodieConfig.setDefaultValue(HOODIE_ARCHIVELOG_FOLDER_PROP);
+      if (!hoodieConfig.contains(HOODIE_TIMELINE_LAYOUT_VERSION_PROP)) {
         // Use latest Version as default unless forced by client
-        properties.setProperty(HOODIE_TIMELINE_LAYOUT_VERSION, TimelineLayoutVersion.CURR_VERSION.toString());
+        hoodieConfig.setValue(HOODIE_TIMELINE_LAYOUT_VERSION_PROP, TimelineLayoutVersion.CURR_VERSION.toString());
       }
-      if (properties.containsKey(HOODIE_BOOTSTRAP_BASE_PATH) && !properties.containsKey(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME)) {
+      if (hoodieConfig.contains(HOODIE_BOOTSTRAP_BASE_PATH_PROP)) {
         // Use the default bootstrap index class.
-        properties.setProperty(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME, getDefaultBootstrapIndexClass(properties));
+        hoodieConfig.setDefaultValue(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP, getDefaultBootstrapIndexClass(properties));
       }
-      properties.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
+      hoodieConfig.getProps().store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
     }
   }
 
@@ -162,15 +205,12 @@ public class HoodieTableConfig implements Serializable {
    * Read the table type from the table properties and if not found, return the default.
    */
   public HoodieTableType getTableType() {
-    if (props.containsKey(HOODIE_TABLE_TYPE_PROP_NAME)) {
-      return HoodieTableType.valueOf(props.getProperty(HOODIE_TABLE_TYPE_PROP_NAME));
-    }
-    return DEFAULT_TABLE_TYPE;
+    return HoodieTableType.valueOf(getStringOrDefault(HOODIE_TABLE_TYPE_PROP));
   }
 
   public Option<TimelineLayoutVersion> getTimelineLayoutVersion() {
-    return props.containsKey(HOODIE_TIMELINE_LAYOUT_VERSION)
-        ? Option.of(new TimelineLayoutVersion(Integer.valueOf(props.getProperty(HOODIE_TIMELINE_LAYOUT_VERSION))))
+    return contains(HOODIE_TIMELINE_LAYOUT_VERSION_PROP)
+        ? Option.of(new TimelineLayoutVersion(getInt(HOODIE_TIMELINE_LAYOUT_VERSION_PROP)))
         : Option.empty();
   }
 
@@ -178,13 +218,13 @@ public class HoodieTableConfig implements Serializable {
    * @return the hoodie.table.version from hoodie.properties file.
    */
   public HoodieTableVersion getTableVersion() {
-    return props.containsKey(HOODIE_TABLE_VERSION_PROP_NAME)
-        ? HoodieTableVersion.versionFromCode(Integer.parseInt(props.getProperty(HOODIE_TABLE_VERSION_PROP_NAME)))
-        : DEFAULT_TABLE_VERSION;
+    return contains(HOODIE_TABLE_VERSION_PROP)
+        ? HoodieTableVersion.versionFromCode(getInt(HOODIE_TABLE_VERSION_PROP))
+        : HOODIE_TABLE_VERSION_PROP.defaultValue();
   }
 
   public void setTableVersion(HoodieTableVersion tableVersion) {
-    props.put(HOODIE_TABLE_VERSION_PROP_NAME, Integer.toString(tableVersion.versionCode()));
+    setValue(HOODIE_TABLE_VERSION_PROP, Integer.toString(tableVersion.versionCode()));
   }
 
   /**
@@ -193,17 +233,17 @@ public class HoodieTableConfig 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 props.getProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS).replace("com.uber.hoodie",
+    return getStringOrDefault(HOODIE_PAYLOAD_CLASS_PROP).replace("com.uber.hoodie",
         "org.apache.hudi");
   }
 
   public String getPreCombineField() {
-    return props.getProperty(HOODIE_TABLE_PRECOMBINE_FIELD);
+    return getString(HOODIE_TABLE_PRECOMBINE_FIELD_PROP);
   }
 
   public Option<String[]> getPartitionColumns() {
-    if (props.containsKey(HOODIE_TABLE_PARTITION_COLUMNS)) {
-      return Option.of(Arrays.stream(props.getProperty(HOODIE_TABLE_PARTITION_COLUMNS).split(","))
+    if (contains(HOODIE_TABLE_PARTITION_COLUMNS_PROP)) {
+      return Option.of(Arrays.stream(getString(HOODIE_TABLE_PARTITION_COLUMNS_PROP).split(","))
         .filter(p -> p.length() > 0).collect(Collectors.toList()).toArray(new String[]{}));
     }
     return Option.empty();
@@ -215,24 +255,24 @@ public class HoodieTableConfig 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 props.getProperty(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME, getDefaultBootstrapIndexClass(props));
+    return getStringOrDefault(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP, getDefaultBootstrapIndexClass(props));
   }
 
   public static String getDefaultBootstrapIndexClass(Properties props) {
-    String defaultClass = DEFAULT_BOOTSTRAP_INDEX_CLASS;
-    if ("false".equalsIgnoreCase(props.getProperty(HOODIE_BOOTSTRAP_INDEX_ENABLE))) {
+    String defaultClass = HOODIE_BOOTSTRAP_INDEX_CLASS_PROP.defaultValue();
+    if ("false".equalsIgnoreCase(props.getProperty(HOODIE_BOOTSTRAP_INDEX_ENABLE_PROP.key()))) {
       defaultClass = NO_OP_BOOTSTRAP_INDEX_CLASS;
     }
     return defaultClass;
   }
 
   public Option<String> getBootstrapBasePath() {
-    return Option.ofNullable(props.getProperty(HOODIE_BOOTSTRAP_BASE_PATH));
+    return Option.ofNullable(getString(HOODIE_BOOTSTRAP_BASE_PATH_PROP));
   }
 
   public Option<Schema> getTableCreateSchema() {
-    if (props.containsKey(HOODIE_TABLE_CREATE_SCHEMA)) {
-      return Option.of(new Schema.Parser().parse(props.getProperty(HOODIE_TABLE_CREATE_SCHEMA)));
+    if (contains(HOODIE_TABLE_CREATE_SCHEMA)) {
+      return Option.of(new Schema.Parser().parse(getString(HOODIE_TABLE_CREATE_SCHEMA)));
     } else {
       return Option.empty();
     }
@@ -242,7 +282,7 @@ public class HoodieTableConfig implements Serializable {
    * Read the table name.
    */
   public String getTableName() {
-    return props.getProperty(HOODIE_TABLE_NAME_PROP_NAME);
+    return getString(HOODIE_TABLE_NAME_PROP);
   }
 
   /**
@@ -251,13 +291,7 @@ public class HoodieTableConfig implements Serializable {
    * @return HoodieFileFormat for the base file Storage format
    */
   public HoodieFileFormat getBaseFileFormat() {
-    if (props.containsKey(HOODIE_BASE_FILE_FORMAT_PROP_NAME)) {
-      return HoodieFileFormat.valueOf(props.getProperty(HOODIE_BASE_FILE_FORMAT_PROP_NAME));
-    }
-    if (props.containsKey(HOODIE_RO_FILE_FORMAT_PROP_NAME)) {
-      return HoodieFileFormat.valueOf(props.getProperty(HOODIE_RO_FILE_FORMAT_PROP_NAME));
-    }
-    return DEFAULT_BASE_FILE_FORMAT;
+    return HoodieFileFormat.valueOf(getStringOrDefault(HOODIE_BASE_FILE_FORMAT_PROP));
   }
 
   /**
@@ -266,28 +300,18 @@ public class HoodieTableConfig implements Serializable {
    * @return HoodieFileFormat for the log Storage format
    */
   public HoodieFileFormat getLogFileFormat() {
-    if (props.containsKey(HOODIE_LOG_FILE_FORMAT_PROP_NAME)) {
-      return HoodieFileFormat.valueOf(props.getProperty(HOODIE_LOG_FILE_FORMAT_PROP_NAME));
-    }
-    if (props.containsKey(HOODIE_RT_FILE_FORMAT_PROP_NAME)) {
-      return HoodieFileFormat.valueOf(props.getProperty(HOODIE_RT_FILE_FORMAT_PROP_NAME));
-    }
-    return DEFAULT_LOG_FILE_FORMAT;
+    return HoodieFileFormat.valueOf(getStringOrDefault(HOODIE_LOG_FILE_FORMAT_PROP));
   }
 
   /**
    * Get the relative path of archive log folder under metafolder, for this table.
    */
   public String getArchivelogFolder() {
-    return props.getProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER);
+    return getStringOrDefault(HOODIE_ARCHIVELOG_FOLDER_PROP);
   }
 
-  public Map<String, String> getProps() {
+  public Map<String, String> propsMap() {
     return props.entrySet().stream()
         .collect(Collectors.toMap(e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getValue())));
   }
-
-  public Properties getProperties() {
-    return props;
-  }
 }
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 3285a00..4c0b845 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
@@ -18,6 +18,7 @@
 
 package org.apache.hudi.common.table;
 
+import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.common.config.SerializableConfiguration;
 import org.apache.hudi.common.fs.ConsistencyGuardConfig;
 import org.apache.hudi.common.fs.FSUtils;
@@ -225,7 +226,7 @@ public class HoodieTableMetaClient implements Serializable {
    */
   public String getArchivePath() {
     String archiveFolder = tableConfig.getArchivelogFolder();
-    if (archiveFolder.equals(HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER)) {
+    if (archiveFolder.equals(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue())) {
       return getMetaPath();
     } else {
       return getMetaPath() + "/" + archiveFolder;
@@ -330,8 +331,7 @@ public class HoodieTableMetaClient implements Serializable {
     }
 
     // if anything other than default archive log folder is specified, create that too
-    String archiveLogPropVal = props.getProperty(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME,
-        HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER);
+    String archiveLogPropVal = new HoodieConfig(props).getStringOrDefault(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP);
     if (!StringUtils.isNullOrEmpty(archiveLogPropVal)) {
       Path archiveLogDir = new Path(metaPathDir, archiveLogPropVal);
       if (!fs.exists(archiveLogDir)) {
@@ -683,47 +683,47 @@ public class HoodieTableMetaClient implements Serializable {
     }
 
     public PropertyBuilder fromProperties(Properties properties) {
-      if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME)) {
-        setTableName(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME));
+      HoodieConfig hoodieConfig = new HoodieConfig(properties);
+      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_NAME_PROP)) {
+        setTableName(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_NAME_PROP));
       }
-      if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME)) {
-        setTableType(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME));
+      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP)) {
+        setTableType(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP));
       }
-      if (properties.containsKey(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME)) {
+      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP)) {
         setArchiveLogFolder(
-            properties.getProperty(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME));
+            hoodieConfig.getString(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP));
       }
-      if (properties.containsKey(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME)) {
+      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP)) {
         setPayloadClassName(
-            properties.getProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME));
+            hoodieConfig.getString(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP));
       }
-      if (properties.containsKey(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION)) {
-        setTimelineLayoutVersion(Integer
-            .parseInt(properties.getProperty(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION)));
+      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION_PROP)) {
+        setTimelineLayoutVersion(hoodieConfig.getInt(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION_PROP));
       }
-      if (properties.containsKey(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME)) {
+      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP)) {
         setBaseFileFormat(
-            properties.getProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME));
+            hoodieConfig.getString(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP));
       }
-      if (properties.containsKey(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME)) {
+      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP)) {
         setBootstrapIndexClass(
-            properties.getProperty(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME));
+            hoodieConfig.getString(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP));
       }
-      if (properties.containsKey(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH)) {
-        setBootstrapBasePath(properties.getProperty(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH));
+      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP)) {
+        setBootstrapBasePath(hoodieConfig.getString(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP));
       }
-      if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD)) {
-        setPreCombineField(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD));
+      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP)) {
+        setPreCombineField(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP));
       }
-      if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS)) {
+      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS_PROP)) {
         setPartitionColumns(
-            properties.getProperty(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS));
+            hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS_PROP));
       }
-      if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS)) {
-        setRecordKeyFields(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS));
+      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS)) {
+        setRecordKeyFields(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS));
       }
-      if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA)) {
-        setTableCreateSchema(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA));
+      if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA)) {
+        setTableCreateSchema(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA));
       }
       return this;
     }
@@ -732,55 +732,53 @@ public class HoodieTableMetaClient implements Serializable {
       ValidationUtils.checkArgument(tableType != null, "tableType is null");
       ValidationUtils.checkArgument(tableName != null, "tableName is null");
 
-      Properties properties = new Properties();
-      properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
-      properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
-      properties.setProperty(HoodieTableConfig.HOODIE_TABLE_VERSION_PROP_NAME,
+      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,
           String.valueOf(HoodieTableVersion.current().versionCode()));
       if (tableType == HoodieTableType.MERGE_ON_READ && payloadClassName != null) {
-        properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName);
+        tableConfig.setValue(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP, payloadClassName);
       }
 
       if (null != tableCreateSchema) {
-        properties.put(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA, tableCreateSchema);
+        tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA, tableCreateSchema);
       }
 
       if (!StringUtils.isNullOrEmpty(archiveLogFolder)) {
-        properties.put(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, archiveLogFolder);
+        tableConfig.setValue(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP, archiveLogFolder);
       } else {
-        properties.setProperty(HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER, HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER);
+        tableConfig.setDefaultValue(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP);
       }
 
       if (null != timelineLayoutVersion) {
-        properties.put(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION,
+        tableConfig.setValue(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION_PROP,
             String.valueOf(timelineLayoutVersion));
       }
 
       if (null != baseFileFormat) {
-        properties.setProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME,
-            baseFileFormat.toUpperCase());
+        tableConfig.setValue(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP, baseFileFormat.toUpperCase());
       }
 
       if (null != bootstrapIndexClass) {
-        properties
-          .put(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME, bootstrapIndexClass);
+        tableConfig.setValue(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP, bootstrapIndexClass);
       }
 
       if (null != bootstrapBasePath) {
-        properties.put(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH, bootstrapBasePath);
+        tableConfig.setValue(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP, bootstrapBasePath);
       }
 
       if (null != preCombineField) {
-        properties.put(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD, preCombineField);
+        tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP, preCombineField);
       }
 
       if (null != partitionColumns) {
-        properties.put(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS, partitionColumns);
+        tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS_PROP, partitionColumns);
       }
       if (null != recordKeyFields) {
-        properties.put(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS, recordKeyFields);
+        tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS, recordKeyFields);
       }
-      return properties;
+      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 f967d43..c62b3c5 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
@@ -18,7 +18,8 @@
 
 package org.apache.hudi.common.table.view;
 
-import org.apache.hudi.common.config.DefaultHoodieConfig;
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
 import org.apache.hudi.common.util.ValidationUtils;
 
 import java.io.File;
@@ -29,136 +30,156 @@ import java.util.Properties;
 /**
  * File System View Storage Configurations.
  */
-public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
+public class FileSystemViewStorageConfig extends HoodieConfig {
 
   // Property Names
-  public static final String FILESYSTEM_VIEW_STORAGE_TYPE = "hoodie.filesystem.view.type";
-  public static final FileSystemViewStorageType DEFAULT_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY;
-
-  public static final String FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = "hoodie.filesystem.view.incr.timeline.sync.enable";
-  public static final String DEFAULT_FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = "false";
-
-  public static final String FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE = "hoodie.filesystem.view.secondary.type";
-  public static final FileSystemViewStorageType DEFAULT_SECONDARY_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY;
-
-  public static final String FILESYSTEM_VIEW_REMOTE_HOST = "hoodie.filesystem.view.remote.host";
-  public static final String DEFUALT_REMOTE_VIEW_SERVER_HOST = "localhost";
-
-  public static final String FILESYSTEM_VIEW_REMOTE_PORT = "hoodie.filesystem.view.remote.port";
-  public static final Integer DEFAULT_REMOTE_VIEW_SERVER_PORT = 26754;
-
-  public static final String FILESYSTEM_VIEW_SPILLABLE_DIR = "hoodie.filesystem.view.spillable.dir";
-  public static final String DEFAULT_VIEW_SPILLABLE_DIR = "/tmp/view_map/";
-
-  public static final String FILESYSTEM_VIEW_SPILLABLE_MEM = "hoodie.filesystem.view.spillable.mem";
-  private static final Long DEFAULT_MAX_MEMORY_FOR_VIEW = 100 * 1024 * 1024L; // 100 MB
-
-  public static final String FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION =
-      "hoodie.filesystem.view.spillable.compaction.mem.fraction";
-  private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION = 0.01;
-
-  public static final String FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION =
-      "hoodie.filesystem.view.spillable.bootstrap.base.file.mem.fraction";
-
-  public static final String FILESYSTEM_VIEW_REPLACED_MEM_FRACTION =
-      "hoodie.filesystem.view.spillable.replaced.mem.fraction";
-  private static final Double DEFAULT_MEM_FRACTION_FOR_REPLACED_FILEGROUPS = 0.01;
-
-  public static final String FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION =
-      "hoodie.filesystem.view.spillable.clustering.mem.fraction";
-  private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_CLUSTERING_FILEGROUPS = 0.01;
-
-  private static final String ROCKSDB_BASE_PATH_PROP = "hoodie.filesystem.view.rocksdb.base.path";
-  public static final String DEFAULT_ROCKSDB_BASE_PATH = "/tmp/hoodie_timeline_rocksdb";
-
-  public static final String FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS =
-      "hoodie.filesystem.view.remote.timeout.secs";
-  public static final Integer DEFAULT_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = 5 * 60; // 5 min
-
-  private static final Double DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE = 0.05;
+  public static final ConfigProperty<FileSystemViewStorageType> FILESYSTEM_VIEW_STORAGE_TYPE = ConfigProperty
+      .key("hoodie.filesystem.view.type")
+      .defaultValue(FileSystemViewStorageType.MEMORY)
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = ConfigProperty
+      .key("hoodie.filesystem.view.incr.timeline.sync.enable")
+      .defaultValue("false")
+      .withDocumentation("");
+
+  public static final ConfigProperty<FileSystemViewStorageType> FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE = ConfigProperty
+      .key("hoodie.filesystem.view.secondary.type")
+      .defaultValue(FileSystemViewStorageType.MEMORY)
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> FILESYSTEM_VIEW_REMOTE_HOST = ConfigProperty
+      .key("hoodie.filesystem.view.remote.host")
+      .defaultValue("localhost")
+      .withDocumentation("");
+
+  public static final ConfigProperty<Integer> FILESYSTEM_VIEW_REMOTE_PORT = ConfigProperty
+      .key("hoodie.filesystem.view.remote.port")
+      .defaultValue(26754)
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> FILESYSTEM_VIEW_SPILLABLE_DIR = ConfigProperty
+      .key("hoodie.filesystem.view.spillable.dir")
+      .defaultValue("/tmp/view_map/")
+      .withDocumentation("");
+
+  public static final ConfigProperty<Long> FILESYSTEM_VIEW_SPILLABLE_MEM = ConfigProperty
+      .key("hoodie.filesystem.view.spillable.mem")
+      .defaultValue(100 * 1024 * 1024L) // 100 MB
+      .withDocumentation("");
+
+  public static final ConfigProperty<Double> FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION = ConfigProperty
+      .key("hoodie.filesystem.view.spillable.compaction.mem.fraction")
+      .defaultValue(0.8)
+      .withDocumentation("");
+
+  public static final ConfigProperty<Double> FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION = ConfigProperty
+      .key("hoodie.filesystem.view.spillable.bootstrap.base.file.mem.fraction")
+      .defaultValue(0.05)
+      .withDocumentation("");
+
+  public static final ConfigProperty<Double> FILESYSTEM_VIEW_REPLACED_MEM_FRACTION = ConfigProperty
+      .key("hoodie.filesystem.view.spillable.replaced.mem.fraction")
+      .defaultValue(0.01)
+      .withDocumentation("");
+
+  public static final ConfigProperty<Double> FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION = ConfigProperty
+      .key("hoodie.filesystem.view.spillable.clustering.mem.fraction")
+      .defaultValue(0.01)
+      .withDocumentation("");
+
+  public static final ConfigProperty<String> ROCKSDB_BASE_PATH_PROP = ConfigProperty
+      .key("hoodie.filesystem.view.rocksdb.base.path")
+      .defaultValue("/tmp/hoodie_timeline_rocksdb")
+      .withDocumentation("");
+
+  public static final ConfigProperty<Integer> FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = ConfigProperty
+      .key("hoodie.filesystem.view.remote.timeout.secs")
+      .defaultValue(5 * 60) // 5 min
+      .withDocumentation("");
 
   /**
    * Configs to control whether backup needs to be configured if clients were not able to reach
    * timeline service.
    */
-  public static final String REMOTE_BACKUP_VIEW_HANDLER_ENABLE =
-      "hoodie.filesystem.remote.backup.view.enable";
-  // Need to be disabled only for tests.
-  public static final String DEFAULT_REMOTE_BACKUP_VIEW_HANDLER_ENABLE = "true";
+  public static final ConfigProperty<String> REMOTE_BACKUP_VIEW_HANDLER_ENABLE = ConfigProperty
+      .key("hoodie.filesystem.remote.backup.view.enable")
+      .defaultValue("true") // Need to be disabled only for tests.
+      .withDocumentation("");
 
   public static FileSystemViewStorageConfig.Builder newBuilder() {
     return new Builder();
   }
 
-  private FileSystemViewStorageConfig(Properties props) {
-    super(props);
+  private FileSystemViewStorageConfig() {
+    super();
   }
 
   public FileSystemViewStorageType getStorageType() {
-    return FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_VIEW_STORAGE_TYPE));
+    return FileSystemViewStorageType.valueOf(getString(FILESYSTEM_VIEW_STORAGE_TYPE));
   }
 
   public boolean isIncrementalTimelineSyncEnabled() {
-    return Boolean.parseBoolean(props.getProperty(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE));
+    return getBoolean(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE);
   }
 
   public String getRemoteViewServerHost() {
-    return props.getProperty(FILESYSTEM_VIEW_REMOTE_HOST);
+    return getString(FILESYSTEM_VIEW_REMOTE_HOST);
   }
 
   public Integer getRemoteViewServerPort() {
-    return Integer.parseInt(props.getProperty(FILESYSTEM_VIEW_REMOTE_PORT));
+    return getInt(FILESYSTEM_VIEW_REMOTE_PORT);
   }
 
   public Integer getRemoteTimelineClientTimeoutSecs() {
-    return Integer.parseInt(props.getProperty(FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS));
+    return getInt(FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS);
   }
 
   public long getMaxMemoryForFileGroupMap() {
-    long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
+    long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
     return totalMemory - getMaxMemoryForPendingCompaction() - getMaxMemoryForBootstrapBaseFile();
   }
 
   public long getMaxMemoryForPendingCompaction() {
-    long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
-    return new Double(totalMemory * Double.parseDouble(props.getProperty(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION)))
+    long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
+    return new Double(totalMemory * getDouble(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION))
         .longValue();
   }
 
   public long getMaxMemoryForBootstrapBaseFile() {
-    long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
+    long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
     long reservedForExternalDataFile =
-        new Double(totalMemory * Double.parseDouble(props.getProperty(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION)))
+        new Double(totalMemory * getDouble(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION))
             .longValue();
     return reservedForExternalDataFile;
   }
 
   public long getMaxMemoryForReplacedFileGroups() {
-    long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
-    return new Double(totalMemory * Double.parseDouble(props.getProperty(FILESYSTEM_VIEW_REPLACED_MEM_FRACTION)))
+    long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
+    return new Double(totalMemory * getDouble(FILESYSTEM_VIEW_REPLACED_MEM_FRACTION))
         .longValue();
   }
 
   public long getMaxMemoryForPendingClusteringFileGroups() {
-    long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
-    return new Double(totalMemory * Double.parseDouble(props.getProperty(FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION)))
+    long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
+    return new Double(totalMemory * getDouble(FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION))
         .longValue();
   }
 
   public String getSpillableDir() {
-    return props.getProperty(FILESYSTEM_VIEW_SPILLABLE_DIR);
+    return getString(FILESYSTEM_VIEW_SPILLABLE_DIR);
   }
 
   public FileSystemViewStorageType getSecondaryStorageType() {
-    return FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
+    return FileSystemViewStorageType.valueOf(getString(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
   }
 
   public boolean shouldEnableBackupForRemoteFileSystemView() {
-    return Boolean.parseBoolean(props.getProperty(REMOTE_BACKUP_VIEW_HANDLER_ENABLE));
+    return getBoolean(REMOTE_BACKUP_VIEW_HANDLER_ENABLE);
   }
 
   public String getRocksdbBasePath() {
-    return props.getProperty(ROCKSDB_BASE_PATH_PROP);
+    return getString(ROCKSDB_BASE_PATH_PROP);
   }
 
   /**
@@ -166,118 +187,87 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
    */
   public static class Builder {
 
-    private final Properties props = new Properties();
+    private final FileSystemViewStorageConfig fileSystemViewStorageConfig = new FileSystemViewStorageConfig();
 
     public Builder fromFile(File propertiesFile) throws IOException {
       try (FileReader reader = new FileReader(propertiesFile)) {
-        props.load(reader);
+        fileSystemViewStorageConfig.getProps().load(reader);
         return this;
       }
     }
 
     public Builder fromProperties(Properties props) {
-      this.props.putAll(props);
+      this.fileSystemViewStorageConfig.getProps().putAll(props);
       return this;
     }
 
     public Builder withStorageType(FileSystemViewStorageType storageType) {
-      props.setProperty(FILESYSTEM_VIEW_STORAGE_TYPE, storageType.name());
+      fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_STORAGE_TYPE, storageType.name());
       return this;
     }
 
     public Builder withSecondaryStorageType(FileSystemViewStorageType storageType) {
-      props.setProperty(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE, storageType.name());
+      fileSystemViewStorageConfig.setValue(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE, storageType.name());
       return this;
     }
 
     public Builder withIncrementalTimelineSync(boolean enableIncrTimelineSync) {
-      props.setProperty(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE, Boolean.toString(enableIncrTimelineSync));
+      fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE, Boolean.toString(enableIncrTimelineSync));
       return this;
     }
 
     public Builder withRemoteServerHost(String remoteServerHost) {
-      props.setProperty(FILESYSTEM_VIEW_REMOTE_HOST, remoteServerHost);
+      fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_REMOTE_HOST, remoteServerHost);
       return this;
     }
 
     public Builder withRemoteServerPort(Integer remoteServerPort) {
-      props.setProperty(FILESYSTEM_VIEW_REMOTE_PORT, remoteServerPort.toString());
+      fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_REMOTE_PORT, remoteServerPort.toString());
       return this;
     }
 
     public Builder withMaxMemoryForView(Long maxMemoryForView) {
-      props.setProperty(FILESYSTEM_VIEW_SPILLABLE_MEM, maxMemoryForView.toString());
+      fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_SPILLABLE_MEM, maxMemoryForView.toString());
       return this;
     }
 
     public Builder withRemoteTimelineClientTimeoutSecs(Long timelineClientTimeoutSecs) {
-      props.setProperty(FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS, timelineClientTimeoutSecs.toString());
+      fileSystemViewStorageConfig.setValue(FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS, timelineClientTimeoutSecs.toString());
       return this;
     }
 
     public Builder withMemFractionForPendingCompaction(Double memFractionForPendingCompaction) {
-      props.setProperty(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, memFractionForPendingCompaction.toString());
+      fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, memFractionForPendingCompaction.toString());
       return this;
     }
 
     public Builder withMemFractionForExternalDataFile(Double memFractionForExternalDataFile) {
-      props.setProperty(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION, memFractionForExternalDataFile.toString());
+      fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION, memFractionForExternalDataFile.toString());
       return this;
     }
 
     public Builder withBaseStoreDir(String baseStorePath) {
-      props.setProperty(FILESYSTEM_VIEW_SPILLABLE_DIR, baseStorePath);
+      fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_SPILLABLE_DIR, baseStorePath);
       return this;
     }
 
     public Builder withRocksDBPath(String basePath) {
-      props.setProperty(ROCKSDB_BASE_PATH_PROP, basePath);
+      fileSystemViewStorageConfig.setValue(ROCKSDB_BASE_PATH_PROP, basePath);
       return this;
     }
 
     public Builder withEnableBackupForRemoteFileSystemView(boolean enable) {
-      props.setProperty(REMOTE_BACKUP_VIEW_HANDLER_ENABLE, Boolean.toString(enable));
+      fileSystemViewStorageConfig.setValue(REMOTE_BACKUP_VIEW_HANDLER_ENABLE, Boolean.toString(enable));
       return this;
     }
 
     public FileSystemViewStorageConfig build() {
-      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_STORAGE_TYPE), FILESYSTEM_VIEW_STORAGE_TYPE,
-          DEFAULT_VIEW_STORAGE_TYPE.name());
-      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE),
-          FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE, DEFAULT_FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE);
-      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE),
-          FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE, DEFAULT_SECONDARY_VIEW_STORAGE_TYPE.name());
-      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_REMOTE_HOST), FILESYSTEM_VIEW_REMOTE_HOST,
-          DEFUALT_REMOTE_VIEW_SERVER_HOST);
-      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_REMOTE_PORT), FILESYSTEM_VIEW_REMOTE_PORT,
-          DEFAULT_REMOTE_VIEW_SERVER_PORT.toString());
-
-      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_SPILLABLE_DIR), FILESYSTEM_VIEW_SPILLABLE_DIR,
-          DEFAULT_VIEW_SPILLABLE_DIR);
-      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_SPILLABLE_MEM), FILESYSTEM_VIEW_SPILLABLE_MEM,
-          DEFAULT_MAX_MEMORY_FOR_VIEW.toString());
-      setDefaultOnCondition(props, !props.containsKey(FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS),
-          FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS, DEFAULT_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS.toString());
-      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION),
-          FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION.toString());
-      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION),
-          FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION, DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE.toString());
-      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_REPLACED_MEM_FRACTION),
-          FILESYSTEM_VIEW_REPLACED_MEM_FRACTION, DEFAULT_MEM_FRACTION_FOR_REPLACED_FILEGROUPS.toString());
-      setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION),
-          FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION, DEFAULT_MEM_FRACTION_FOR_PENDING_CLUSTERING_FILEGROUPS.toString());
-
-      setDefaultOnCondition(props, !props.containsKey(ROCKSDB_BASE_PATH_PROP), ROCKSDB_BASE_PATH_PROP,
-          DEFAULT_ROCKSDB_BASE_PATH);
-
-      setDefaultOnCondition(props, !props.containsKey(REMOTE_BACKUP_VIEW_HANDLER_ENABLE),
-          REMOTE_BACKUP_VIEW_HANDLER_ENABLE, DEFAULT_REMOTE_BACKUP_VIEW_HANDLER_ENABLE);
-
+      fileSystemViewStorageConfig.setDefaults(FileSystemViewStorageConfig.class.getName());
       // Validations
-      FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_VIEW_STORAGE_TYPE));
-      FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
-      ValidationUtils.checkArgument(Integer.parseInt(props.getProperty(FILESYSTEM_VIEW_REMOTE_PORT)) > 0);
-      return new FileSystemViewStorageConfig(props);
+      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);
+      return fileSystemViewStorageConfig;
     }
   }
 
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 36c9cef..a3d0e2d 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.DEFAULT_VIEW_SPILLABLE_DIR);
+        FileSystemViewStorageConfig.FILESYSTEM_VIEW_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 b0adeac..391435e 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
@@ -91,12 +91,10 @@ public class TestBootstrapIndex extends HoodieCommonTestHarness {
 
   @Test
   public void testNoOpBootstrapIndex() throws IOException {
-    Map<String, String> props = metaClient.getTableConfig().getProps();
-    props.put(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_ENABLE, "false");
+    Properties props = metaClient.getTableConfig().getProps();
+    props.put(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_ENABLE_PROP.key(), "false");
     Properties properties = new Properties();
-    for (Map.Entry<String, String> prop : props.entrySet()) {
-      properties.setProperty(prop.getKey(), prop.getValue());
-    }
+    properties.putAll(props);
     HoodieTableConfig.createHoodieProperties(metaClient.getFs(), new Path(metaClient.getMetaPath()), properties);
 
     metaClient = HoodieTableMetaClient.builder().setConf(metaClient.getHadoopConf()).setBasePath(basePath).build();
@@ -187,7 +185,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.DEFAULT_BASE_FILE_FORMAT.getFileExtension();
+        String sourceFileName = idx + HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.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/config/TestConfigProperty.java b/hudi-common/src/test/java/org/apache/hudi/common/config/TestConfigProperty.java
new file mode 100644
index 0000000..5ca48f1
--- /dev/null
+++ b/hudi-common/src/test/java/org/apache/hudi/common/config/TestConfigProperty.java
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.config;
+
+import org.apache.hudi.common.util.Option;
+import org.junit.jupiter.api.Test;
+
+import java.util.Properties;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class TestConfigProperty extends HoodieConfig {
+
+  public static ConfigProperty<String> FAKE_STRING_CONFIG = ConfigProperty
+      .key("test.fake.string.config")
+      .defaultValue("1")
+      .withAlternatives("test.fake.string.alternative.config")
+      .withDocumentation("Fake config only for testing");
+
+  public static ConfigProperty<String> FAKE_BOOLEAN_CONFIG = ConfigProperty
+      .key("test.fake.boolean.config")
+      .defaultValue("false")
+      .withDocumentation("Fake config only for testing");
+
+  public static ConfigProperty<Integer> FAKE_INTEGER_CONFIG = ConfigProperty
+      .key("test.fake.integer.config")
+      .defaultValue(0)
+      .withInferFunction(p -> {
+        if (p.contains(FAKE_STRING_CONFIG) && p.getString(FAKE_STRING_CONFIG).equals("5")) {
+          return Option.of(100);
+        }
+        return Option.empty();
+      })
+      .withDocumentation("Fake config only for testing");
+
+  @Test
+  public void testGetTypedValue() {
+    HoodieConfig hoodieConfig = new HoodieConfig();
+    assertNull(hoodieConfig.getInt(FAKE_STRING_CONFIG));
+    hoodieConfig.setValue(FAKE_STRING_CONFIG, "5");
+    assertEquals(5, hoodieConfig.getInt(FAKE_STRING_CONFIG));
+
+    assertNull(hoodieConfig.getBoolean(FAKE_BOOLEAN_CONFIG));
+    hoodieConfig.setValue(FAKE_BOOLEAN_CONFIG, "true");
+    assertEquals(true, hoodieConfig.getBoolean(FAKE_BOOLEAN_CONFIG));
+  }
+
+  @Test
+  public void testGetOrDefault() {
+    Properties props = new Properties();
+    props.put("test.unknown.config", "abc");
+    HoodieConfig hoodieConfig = new HoodieConfig(props);
+    assertEquals("1", hoodieConfig.getStringOrDefault(FAKE_STRING_CONFIG));
+    assertEquals("2", hoodieConfig.getStringOrDefault(FAKE_STRING_CONFIG, "2"));
+  }
+
+  @Test
+  public void testAlternatives() {
+    Properties props = new Properties();
+    props.put("test.fake.string.alternative.config", "1");
+    HoodieConfig hoodieConfig = new HoodieConfig(props);
+    assertTrue(hoodieConfig.contains(FAKE_STRING_CONFIG));
+    assertEquals("1", hoodieConfig.getString(FAKE_STRING_CONFIG));
+  }
+
+  @Test
+  public void testInference() {
+    HoodieConfig hoodieConfig1 = new HoodieConfig();
+    hoodieConfig1.setDefaultValue(FAKE_INTEGER_CONFIG);
+    assertEquals(0, hoodieConfig1.getInt(FAKE_INTEGER_CONFIG));
+
+    HoodieConfig hoodieConfig2 = new HoodieConfig();
+    hoodieConfig2.setValue(FAKE_STRING_CONFIG, "5");
+    hoodieConfig2.setDefaultValue(FAKE_INTEGER_CONFIG);
+    assertEquals(100, hoodieConfig2.getInt(FAKE_INTEGER_CONFIG));
+  }
+
+  @Test
+  public void testSetDefaults() {
+    setDefaults(this.getClass().getName());
+    assertEquals(3, getProps().size());
+  }
+}
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 37b87ed..5a73667 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
@@ -57,7 +57,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.DEFAULT_BASE_FILE_FORMAT.getFileExtension();
+  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension();
 
   @Rule
   public final EnvironmentVariables environmentVariables = new EnvironmentVariables();
diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java
index 4c43903..5be0961 100644
--- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java
+++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestDefaultHoodieRecordPayload.java
@@ -54,8 +54,8 @@ public class TestDefaultHoodieRecordPayload {
         new Schema.Field("_hoodie_is_deleted", Schema.create(Type.BOOLEAN), "", false)
     ));
     props = new Properties();
-    props.setProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP, "ts");
-    props.setProperty(HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP, "ts");
+    props.setProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY, "ts");
+    props.setProperty(HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP_KEY, "ts");
   }
 
   @Test
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 68f441b..8393de5 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
@@ -34,7 +34,7 @@ public class TestHoodieDeltaWriteStat {
   @Test
   public void testBaseFileAndLogFiles() {
     HoodieDeltaWriteStat writeStat = new HoodieDeltaWriteStat();
-    String baseFile = "file1" + HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension();
+    String baseFile = "file1" + HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.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 369a063..5e73d18 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.DEFAULT_BASE_FILE_FORMAT.getFileExtension()));
+        "file0001" + HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.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 211bb8f..6bf2e9f 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
@@ -60,7 +60,7 @@ import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serial
 public class FileCreateUtils {
 
   private static final String WRITE_TOKEN = "1-0-1";
-  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension();
+  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.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 943a255..046fc9e 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.DEFAULT_BASE_FILE_FORMAT.getFileExtension());
+    return listAllBaseFiles(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.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 d5d5cb2..cf83eac 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, bootstrapBasePath);
+    props.setProperty(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP.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_NAME, tableName);
+    properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP.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_NAME, baseFileFormat.toString());
+    properties.setProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.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 2251a4c..49659d1 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
@@ -37,7 +37,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.DEFAULT_BASE_FILE_FORMAT.getFileExtension();
+  private static final String BASE_FILE_EXTENSION = HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.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 27accad..ebd7c9e 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
@@ -76,10 +76,10 @@ object HoodieDataSourceExample {
     val df = spark.read.json(spark.sparkContext.parallelize(inserts, 1))
     df.write.format("org.apache.hudi").
         options(getQuickstartWriteConfigs).
-        option(PRECOMBINE_FIELD_OPT_KEY, "ts").
-        option(RECORDKEY_FIELD_OPT_KEY, "uuid").
-        option(PARTITIONPATH_FIELD_OPT_KEY, "partitionpath").
-        option(TABLE_NAME, tableName).
+        option(PRECOMBINE_FIELD_OPT_KEY.key, "ts").
+        option(RECORDKEY_FIELD_OPT_KEY.key, "uuid").
+        option(PARTITIONPATH_FIELD_OPT_KEY.key, "partitionpath").
+        option(TABLE_NAME.key, tableName).
         mode(Overwrite).
         save(tablePath)
   }
@@ -121,10 +121,10 @@ object HoodieDataSourceExample {
     val df = spark.read.json(spark.sparkContext.parallelize(updates, 1))
     df.write.format("org.apache.hudi").
         options(getQuickstartWriteConfigs).
-        option(PRECOMBINE_FIELD_OPT_KEY, "ts").
-        option(RECORDKEY_FIELD_OPT_KEY, "uuid").
-        option(PARTITIONPATH_FIELD_OPT_KEY, "partitionpath").
-        option(TABLE_NAME, tableName).
+        option(PRECOMBINE_FIELD_OPT_KEY.key, "ts").
+        option(RECORDKEY_FIELD_OPT_KEY.key, "uuid").
+        option(PARTITIONPATH_FIELD_OPT_KEY.key, "partitionpath").
+        option(TABLE_NAME.key, tableName).
         mode(Append).
         save(tablePath)
   }
@@ -143,8 +143,8 @@ object HoodieDataSourceExample {
     val incViewDF = spark.
         read.
         format("org.apache.hudi").
-        option(QUERY_TYPE_OPT_KEY, QUERY_TYPE_INCREMENTAL_OPT_VAL).
-        option(BEGIN_INSTANTTIME_OPT_KEY, beginTime).
+        option(QUERY_TYPE_OPT_KEY.key, QUERY_TYPE_INCREMENTAL_OPT_VAL).
+        option(BEGIN_INSTANTTIME_OPT_KEY.key, beginTime).
         load(tablePath)
     incViewDF.createOrReplaceTempView("hudi_incr_table")
     spark.sql("select `_hoodie_commit_time`, fare, begin_lon, begin_lat, ts from hudi_incr_table where fare > 20.0").show()
@@ -163,9 +163,9 @@ object HoodieDataSourceExample {
 
     //incrementally query data
     val incViewDF = spark.read.format("org.apache.hudi").
-        option(QUERY_TYPE_OPT_KEY, QUERY_TYPE_INCREMENTAL_OPT_VAL).
-        option(BEGIN_INSTANTTIME_OPT_KEY, beginTime).
-        option(END_INSTANTTIME_OPT_KEY, endTime).
+        option(QUERY_TYPE_OPT_KEY.key, QUERY_TYPE_INCREMENTAL_OPT_VAL).
+        option(BEGIN_INSTANTTIME_OPT_KEY.key, beginTime).
+        option(END_INSTANTTIME_OPT_KEY.key, endTime).
         load(tablePath)
     incViewDF.createOrReplaceTempView("hudi_incr_table")
     spark.sql("select `_hoodie_commit_time`, fare, begin_lon, begin_lat, ts from  hudi_incr_table where fare > 20.0").show()
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 bf1fc0b..0f95c49 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
@@ -173,7 +173,7 @@ public class FlinkOptions {
   //  Write Options
   // ------------------------------------------------------------------------
   public static final ConfigOption<String> TABLE_NAME = ConfigOptions
-      .key(HoodieWriteConfig.TABLE_NAME)
+      .key(HoodieWriteConfig.TABLE_NAME.key())
       .stringType()
       .noDefaultValue()
       .withDescription("Table name to register to Hive metastore");
@@ -240,7 +240,7 @@ public class FlinkOptions {
           + "By default true (in favor of streaming progressing over data integrity)");
 
   public static final ConfigOption<String> RECORD_KEY_FIELD = ConfigOptions
-      .key(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
+      .key(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key())
       .stringType()
       .defaultValue("uuid")
       .withDescription("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
@@ -248,7 +248,7 @@ public class FlinkOptions {
           + "the dot notation eg: `a.b.c`");
 
   public static final ConfigOption<String> PARTITION_PATH_FIELD = ConfigOptions
-      .key(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY)
+      .key(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key())
       .stringType()
       .defaultValue("")
       .withDescription("Partition path field. Value to be used at the `partitionPath` component of `HoodieKey`.\n"
@@ -269,13 +269,13 @@ public class FlinkOptions {
           + "By default false (the names of partition folders are only partition values)");
 
   public static final ConfigOption<String> KEYGEN_CLASS = ConfigOptions
-      .key(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP)
+      .key(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP.key())
       .stringType()
       .defaultValue("")
       .withDescription("Key generator class, that implements will extract the key out of incoming record");
 
   public static final ConfigOption<String> KEYGEN_TYPE = ConfigOptions
-      .key(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP)
+      .key(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP.key())
       .stringType()
       .defaultValue(KeyGeneratorType.SIMPLE.name())
       .withDescription("Key generator type, that implements will extract the key out of incoming record");
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 d73b300..7530f08 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
@@ -56,7 +56,7 @@ import java.util.List;
 import java.util.Locale;
 import java.util.Properties;
 
-import static org.apache.hudi.common.table.HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER;
+import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP;
 
 /**
  * Utilities for Flink stream read and write.
@@ -209,7 +209,7 @@ public class StreamerUtil {
           .setTableType(conf.getString(FlinkOptions.TABLE_TYPE))
           .setTableName(conf.getString(FlinkOptions.TABLE_NAME))
           .setPayloadClassName(conf.getString(FlinkOptions.PAYLOAD_CLASS))
-          .setArchiveLogFolder(DEFAULT_ARCHIVELOG_FOLDER)
+          .setArchiveLogFolder(HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue())
           .setTimelineLayoutVersion(1)
           .initTable(hadoopConf, basePath);
       LOG.info("Table initialized under base path {}", basePath);
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 26fbdda..f4bddbd 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
@@ -71,7 +71,6 @@ import java.util.function.Function;
 import java.util.stream.Collectors;
 
... 5038 lines suppressed ...