You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ja...@apache.org on 2018/11/06 12:28:48 UTC

[1/2] carbondata git commit: [CARBONDATA-3038] Refactor dynamic configuration

Repository: carbondata
Updated Branches:
  refs/heads/master 35fb9409d -> 7c1d68ed6


http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/core/src/main/java/org/apache/carbondata/core/util/path/HDFSLeaseUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/path/HDFSLeaseUtils.java b/core/src/main/java/org/apache/carbondata/core/util/path/HDFSLeaseUtils.java
index 833ed8b..1a10f46 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/path/HDFSLeaseUtils.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/path/HDFSLeaseUtils.java
@@ -37,13 +37,6 @@ import org.apache.log4j.Logger;
  */
 public class HDFSLeaseUtils {
 
-  private static final int CARBON_LEASE_RECOVERY_RETRY_COUNT_MIN = 1;
-  private static final int CARBON_LEASE_RECOVERY_RETRY_COUNT_MAX = 50;
-  private static final String CARBON_LEASE_RECOVERY_RETRY_COUNT_DEFAULT = "5";
-  private static final int CARBON_LEASE_RECOVERY_RETRY_INTERVAL_MIN = 1000;
-  private static final int CARBON_LEASE_RECOVERY_RETRY_INTERVAL_MAX = 10000;
-  private static final String CARBON_LEASE_RECOVERY_RETRY_INTERVAL_DEFAULT = "1000";
-
   /**
    * LOGGER
    */
@@ -164,22 +157,26 @@ public class HDFSLeaseUtils {
   private static int getLeaseRecoveryRetryCount() {
     String retryMaxAttempts = CarbonProperties.getInstance()
         .getProperty(CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_COUNT,
-            CARBON_LEASE_RECOVERY_RETRY_COUNT_DEFAULT);
+            CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_COUNT_DEFAULT);
     int retryCount = 0;
     try {
       retryCount = Integer.parseInt(retryMaxAttempts);
-      if (retryCount < CARBON_LEASE_RECOVERY_RETRY_COUNT_MIN
-          || retryCount > CARBON_LEASE_RECOVERY_RETRY_COUNT_MAX) {
-        retryCount = Integer.parseInt(CARBON_LEASE_RECOVERY_RETRY_COUNT_DEFAULT);
+      if (retryCount < CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_COUNT_MIN
+          || retryCount > CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_COUNT_MAX) {
+        retryCount = Integer.parseInt(
+            CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_COUNT_DEFAULT);
         LOGGER.warn(
-            "value configured for " + CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_COUNT
-                + " is not in allowed range. Allowed range is >="
-                + CARBON_LEASE_RECOVERY_RETRY_COUNT_MIN + " and <="
-                + CARBON_LEASE_RECOVERY_RETRY_COUNT_MAX + ". Therefore considering default value: "
-                + retryCount);
+            String.format("value configured for %s is not in allowed range. Allowed range " +
+                    "is >= %d and <= %d. Therefore considering default value: %d",
+                CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_COUNT,
+                CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_COUNT_MIN,
+                CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_COUNT_MAX,
+                retryCount
+            ));
       }
     } catch (NumberFormatException ne) {
-      retryCount = Integer.parseInt(CARBON_LEASE_RECOVERY_RETRY_COUNT_DEFAULT);
+      retryCount = Integer.parseInt(
+          CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_COUNT_DEFAULT);
       LOGGER.warn("value configured for " + CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_COUNT
           + " is incorrect. Therefore considering default value: " + retryCount);
     }
@@ -189,22 +186,24 @@ public class HDFSLeaseUtils {
   private static int getLeaseRecoveryRetryInterval() {
     String retryMaxAttempts = CarbonProperties.getInstance()
         .getProperty(CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_INTERVAL,
-            CARBON_LEASE_RECOVERY_RETRY_INTERVAL_DEFAULT);
+            CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_INTERVAL_DEFAULT);
     int retryCount = 0;
     try {
       retryCount = Integer.parseInt(retryMaxAttempts);
-      if (retryCount < CARBON_LEASE_RECOVERY_RETRY_INTERVAL_MIN
-          || retryCount > CARBON_LEASE_RECOVERY_RETRY_INTERVAL_MAX) {
-        retryCount = Integer.parseInt(CARBON_LEASE_RECOVERY_RETRY_INTERVAL_DEFAULT);
+      if (retryCount < CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_INTERVAL_MIN
+          || retryCount > CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_INTERVAL_MAX) {
+        retryCount = Integer.parseInt(
+            CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_INTERVAL_DEFAULT);
         LOGGER.warn(
             "value configured for " + CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_INTERVAL
                 + " is not in allowed range. Allowed range is >="
-                + CARBON_LEASE_RECOVERY_RETRY_INTERVAL_MIN + " and <="
-                + CARBON_LEASE_RECOVERY_RETRY_INTERVAL_MAX
+                + CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_INTERVAL_MIN + " and <="
+                + CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_INTERVAL_MAX
                 + ". Therefore considering default value (ms): " + retryCount);
       }
     } catch (NumberFormatException ne) {
-      retryCount = Integer.parseInt(CARBON_LEASE_RECOVERY_RETRY_INTERVAL_DEFAULT);
+      retryCount = Integer.parseInt(
+          CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_INTERVAL_DEFAULT);
       LOGGER.warn(
           "value configured for " + CarbonCommonConstants.CARBON_LEASE_RECOVERY_RETRY_INTERVAL
               + " is incorrect. Therefore considering default value (ms): " + retryCount);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/core/src/test/java/org/apache/carbondata/core/CarbonPropertiesValidationTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/CarbonPropertiesValidationTest.java b/core/src/test/java/org/apache/carbondata/core/CarbonPropertiesValidationTest.java
index 7cc665e..b2b03cd 100644
--- a/core/src/test/java/org/apache/carbondata/core/CarbonPropertiesValidationTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/CarbonPropertiesValidationTest.java
@@ -24,7 +24,6 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.util.CarbonProperties;
 
 import junit.framework.TestCase;
-import org.junit.Assert;
 import org.junit.Test;
 
 /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/core/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/DateDirectDictionaryGeneratorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/DateDirectDictionaryGeneratorTest.java b/core/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/DateDirectDictionaryGeneratorTest.java
index 7390556..78ee6d9 100644
--- a/core/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/DateDirectDictionaryGeneratorTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/keygenerator/directdictionary/DateDirectDictionaryGeneratorTest.java
@@ -18,7 +18,6 @@ package org.apache.carbondata.core.keygenerator.directdictionary;
 
 import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.keygenerator.directdictionary.timestamp.TimeStampDirectDictionaryGenerator;
-import org.apache.carbondata.core.metadata.datatype.DataType;
 import org.apache.carbondata.core.metadata.datatype.DataTypes;
 
 import org.junit.After;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/docs/configuration-parameters.md
----------------------------------------------------------------------
diff --git a/docs/configuration-parameters.md b/docs/configuration-parameters.md
index 5a4dea6..c210cae 100644
--- a/docs/configuration-parameters.md
+++ b/docs/configuration-parameters.md
@@ -121,8 +121,8 @@ This section provides the details of all the configurations required for the Car
 | carbon.max.executor.lru.cache.size | -1 | Maximum memory **(in MB)** upto which the executor process can cache the data (BTree and reverse dictionary values).Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted. **NOTE:** If this parameter is not configured, then the value of ***carbon.max.driver.lru.cache.size*** will be used. |
 | max.query.execution.time | 60 | Maximum time allowed for one query to be executed. The value is in minutes. |
 | carbon.enableMinMax | true | CarbonData maintains the metadata which enables to prune unnecessary files from being scanned as per the query conditions. To achieve pruning, Min,Max of each column is maintined.Based on the filter condition in the query, certain data can be skipped from scanning by matching the filter value against the min,max values of the column(s) present in that carbondata file. This pruning enhances query performance significantly. |
-| carbon.dynamicallocation.schedulertimeout | 5 | CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData. To determine the number of tasks that can be scheduled, knowing the count of active executors is necessary. When dynamic allocation is enabled on a YARN based spark cluster, executor processes are shutdown if no request is received for a particular amount of time. The executors are brought up when the requet is received again. This configuration specifies the maximum time (unit in seconds) the carbon scheduler can wait for executor to be active. Minimum value is 5 sec and maximum value is 15 sec.**NOTE: **Waiting for longer time leads to slow query response time.Moreover it might be possible that YARN is not able to start the executors and waiting is not beneficial. |
-| carbon.scheduler.minregisteredresourcesratio | 0.8 | Specifies the minimum resource (executor) ratio needed for starting the block distribution. The default value is 0.8, which indicates 80% of the requested resource is allocated for starting block distribution. The minimum value is 0.1 min and the maximum value is 1.0. |
+| carbon.dynamical.location.scheduler.timeout | 5 | CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData. To determine the number of tasks that can be scheduled, knowing the count of active executors is necessary. When dynamic allocation is enabled on a YARN based spark cluster, executor processes are shutdown if no request is received for a particular amount of time. The executors are brought up when the requet is received again. This configuration specifies the maximum time (unit in seconds) the carbon scheduler can wait for executor to be active. Minimum value is 5 sec and maximum value is 15 sec.**NOTE: **Waiting for longer time leads to slow query response time.Moreover it might be possible that YARN is not able to start the executors and waiting is not beneficial. |
+| carbon.scheduler.min.registered.resources.ratio | 0.8 | Specifies the minimum resource (executor) ratio needed for starting the block distribution. The default value is 0.8, which indicates 80% of the requested resource is allocated for starting block distribution. The minimum value is 0.1 min and the maximum value is 1.0. |
 | carbon.search.enabled (Alpha Feature) | false | If set to true, it will use CarbonReader to do distributed scan directly instead of using compute framework like spark, thus avoiding limitation of compute framework like SQL optimizer and task scheduling overhead. |
 | carbon.search.query.timeout | 10s | Time within which the result is expected from the workers, beyond which the query is terminated |
 | carbon.search.scan.thread | num of cores available in worker node | Number of cores to be used in each worker for performing scan. |
@@ -135,7 +135,7 @@ This section provides the details of all the configurations required for the Car
 | carbon.custom.block.distribution | false | CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData. When this configuration is true, CarbonData would distribute the available blocks to be scanned among the available number of cores.For Example:If there are 10 blocks to be scanned and only 3 tasks can be run(only 3 executor cores available in the cluster), CarbonData would combine blocks as 4,3,3 and give it to 3 tasks to run. **NOTE:** When this configuration is false, as per the ***carbon.task.distribution*** configuration, each block/blocklet would be given to each task. |
 | enable.query.statistics | false | CarbonData has extensive logging which would be useful for debugging issues related to performance or hard to locate issues. This configuration when made ***true*** would log additional query statistics information to more accurately locate the issues being debugged.**NOTE:** Enabling this would log more debug information to log files, there by increasing the log files size significantly in short span of time.It is advised to configure the log files size, retention of log files parameters in log4j properties appropriately. Also extensive logging is an increased IO operation and hence over all query performance might get reduced. Therefore it is recommended to enable this configuration only for the duration of debugging. |
 | enable.unsafe.in.query.processing | false | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations. This configuration enables to use unsafe functions in CarbonData while scanning the  data during query. |
-| carbon.query.validate.directqueryondatamap | true | CarbonData supports creating pre-aggregate table datamaps as an independent tables. For some debugging purposes, it might be required to directly query from such datamap tables. This configuration allows to query on such datamaps. |
+| carbon.query.validate.direct.query.on.datamap | true | CarbonData supports creating pre-aggregate table datamaps as an independent tables. For some debugging purposes, it might be required to directly query from such datamap tables. This configuration allows to query on such datamaps. |
 | carbon.heap.memory.pooling.threshold.bytes | 1048576 | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations. Using unsafe, memory can be allocated on Java Heap or off heap. This configuration controls the allocation mechanism on Java HEAP.If the heap memory allocations of the given size is greater or equal than this value,it should go through the pooling mechanism.But if set this size to -1, it should not go through the pooling mechanism.Default value is 1048576(1MB, the same as Spark).Value to be specified in bytes. |
 
 ## Data Mutation Configuration
@@ -204,12 +204,12 @@ RESET
 | carbon.options.bad.record.path            | Specifies the HDFS path where bad records needs to be stored. |
 | carbon.custom.block.distribution          | Specifies whether to use the Spark or Carbon block distribution feature.**NOTE: **Refer to [Query Configuration](#query-configuration)#carbon.custom.block.distribution for more details on CarbonData scheduler. |
 | enable.unsafe.sort                        | Specifies whether to use unsafe sort during data loading. Unsafe sort reduces the garbage collection during data load operation, resulting in better performance. |
-| carbon.options.dateformat                 | Specifies the data format of the date columns in the data being loaded |
-| carbon.options.timestampformat            | Specifies the timestamp format of the time stamp columns in the data being loaded |
+| carbon.options.date.format                 | Specifies the data format of the date columns in the data being loaded |
+| carbon.options.timestamp.format            | Specifies the timestamp format of the time stamp columns in the data being loaded |
 | carbon.options.sort.scope                 | Specifies how the current data load should be sorted with. **NOTE:** Refer to [Data Loading Configuration](#data-loading-configuration)#carbon.sort.scope for detailed information. |
 | carbon.options.global.sort.partitions     |                                                              |
 | carbon.options.serialization.null.format  | Default Null value representation in the data being loaded. **NOTE:** Refer to [Data Loading Configuration](#data-loading-configuration)#carbon.options.serialization.null.format for detailed information. |
-| carbon.query.directQueryOnDataMap.enabled | Specifies whether datamap can be queried directly. This is useful for debugging purposes.**NOTE: **Refer to [Query Configuration](#query-configuration)#carbon.query.validate.directqueryondatamap for detailed information. |
+| carbon.query.directQueryOnDataMap.enabled | Specifies whether datamap can be queried directly. This is useful for debugging purposes.**NOTE: **Refer to [Query Configuration](#query-configuration)#carbon.query.validate.direct.query.on.datamap for detailed information. |
 
 **Examples:**
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/HorizontalCompactionTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/HorizontalCompactionTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/HorizontalCompactionTestCase.scala
index af1846b..24c8f17 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/HorizontalCompactionTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/HorizontalCompactionTestCase.scala
@@ -47,7 +47,7 @@ class HorizontalCompactionTestCase extends QueryTest with BeforeAndAfterAll {
     sql(
       s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/update01.csv' INTO TABLE iud4.update_01 OPTIONS('BAD_RECORDS_LOGGER_ENABLE' = 'FALSE', 'BAD_RECORDS_ACTION' = 'FORCE') """)
     CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.isHorizontalCompactionEnabled, "true")
+      .addProperty(CarbonCommonConstants.CARBON_HORIZONTAL_COMPACTION_ENABLE, "true")
   }
 
 
@@ -389,7 +389,7 @@ class HorizontalCompactionTestCase extends QueryTest with BeforeAndAfterAll {
     sql("use default")
     sql("drop database if exists iud4 cascade")
     CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.isHorizontalCompactionEnabled , "true")
+      .addProperty(CarbonCommonConstants.CARBON_HORIZONTAL_COMPACTION_ENABLE , "true")
     sql("""drop table if exists t_carbn01""")
     sql("""drop table if exists customer1""")
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
index cc8abe4..50fdd0c 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/iud/UpdateCarbonTableTestCase.scala
@@ -46,7 +46,7 @@ class UpdateCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql("""CREATE TABLE iud.update_01(imei string,age int,task bigint,num double,level decimal(10,3),name string)STORED BY 'org.apache.carbondata.format' """)
     sql(s"""LOAD DATA LOCAL INPATH '$resourcesPath/IUD/update01.csv' INTO TABLE iud.update_01 OPTIONS('BAD_RECORDS_LOGGER_ENABLE' = 'FALSE', 'BAD_RECORDS_ACTION' = 'FORCE') """)
     CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.isHorizontalCompactionEnabled , "true")
+      .addProperty(CarbonCommonConstants.CARBON_HORIZONTAL_COMPACTION_ENABLE , "true")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.ENABLE_VECTOR_READER , "true")
   }
@@ -457,7 +457,7 @@ class UpdateCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
 
   test("""CARBONDATA-1445 carbon.update.persist.enable=false it will fail to update data""") {
     CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.isPersistEnabled, "false")
+      .addProperty(CarbonCommonConstants.CARBON_UPDATE_PERSIST_ENABLE, "false")
     import sqlContext.implicits._
     val df = sqlContext.sparkContext.parallelize(0 to 50)
       .map(x => ("a", x.toString, (x % 2).toString, x, x.toLong, x * 2))
@@ -486,7 +486,7 @@ class UpdateCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
       """).show()
     assert(sql("select stringField1 from study_carbondata where stringField2 = '1_test'").collect().length == 1)
     CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.isPersistEnabled, "true")
+      .addProperty(CarbonCommonConstants.CARBON_UPDATE_PERSIST_ENABLE, "true")
     sql("DROP TABLE IF EXISTS study_carbondata ")
   }
 
@@ -555,7 +555,7 @@ class UpdateCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
   }
   test("Update operation on carbon table with persist false") {
     CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.isPersistEnabled, "false")
+      .addProperty(CarbonCommonConstants.CARBON_UPDATE_PERSIST_ENABLE, "false")
     sql("drop database if exists carbon1 cascade")
     sql(s"create database carbon1 location '$dblocation'")
     sql("use carbon1")
@@ -571,8 +571,8 @@ class UpdateCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     )
     sql("drop table carbontable")
     CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.isPersistEnabled,
-        CarbonCommonConstants.defaultValueIsPersistEnabled)
+      .addProperty(CarbonCommonConstants.CARBON_UPDATE_PERSIST_ENABLE,
+        CarbonCommonConstants.CARBON_UPDATE_PERSIST_ENABLE_DEFAULT)
   }
 
   test("partition test update operation with 0 rows updation.") {
@@ -776,7 +776,7 @@ class UpdateCarbonTableTestCase extends QueryTest with BeforeAndAfterAll {
     sql("use default")
     sql("drop database  if exists iud cascade")
     CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.isHorizontalCompactionEnabled , "true")
+      .addProperty(CarbonCommonConstants.CARBON_HORIZONTAL_COMPACTION_ENABLE , "true")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.ENABLE_VECTOR_READER , "true")
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/integration/spark-common/src/main/scala/org/apache/spark/util/CarbonReflectionUtils.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/util/CarbonReflectionUtils.scala b/integration/spark-common/src/main/scala/org/apache/spark/util/CarbonReflectionUtils.scala
index 0055e87..af9cc2d 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/util/CarbonReflectionUtils.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/util/CarbonReflectionUtils.scala
@@ -196,7 +196,7 @@ object CarbonReflectionUtils {
       sparkSession: SparkSession): AstBuilder = {
     val className = sparkSession.sparkContext.conf.get(
       CarbonCommonConstants.CARBON_SQLASTBUILDER_CLASSNAME,
-      "org.apache.spark.sql.hive.CarbonSqlAstBuilder")
+      CarbonCommonConstants.CARBON_SQLASTBUILDER_CLASSNAME_DEFAULT)
     createObject(className,
       conf,
       sqlParser, sparkSession)._1.asInstanceOf[AstBuilder]

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
index 6905aa2..5fbe82e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
@@ -208,7 +208,8 @@ case class CarbonLoadDataCommand(
       carbonLoadModel.setFactFilePath(factPath)
       carbonLoadModel.setCarbonTransactionalTable(table.getTableInfo.isTransactionalTable)
       carbonLoadModel.setAggLoadRequest(
-        internalOptions.getOrElse(CarbonCommonConstants.IS_INTERNAL_LOAD_CALL, "false").toBoolean)
+        internalOptions.getOrElse(CarbonCommonConstants.IS_INTERNAL_LOAD_CALL,
+          CarbonCommonConstants.IS_INTERNAL_LOAD_CALL_DEFAULT).toBoolean)
       carbonLoadModel.setSegmentId(internalOptions.getOrElse("mergedSegmentName", ""))
       val columnCompressor = table.getTableInfo.getFactTable.getTableProperties.asScala
         .getOrElse(CarbonCommonConstants.COMPRESSOR,

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
index 21070be..37ab04f 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDropTableCommand.scala
@@ -183,7 +183,8 @@ case class CarbonDropTableCommand(
         CarbonUtil.deleteFoldersAndFilesSilent(file)
       }
       // Delete lock directory if external lock path is specified.
-      if (CarbonProperties.getInstance.getProperty(CarbonCommonConstants.LOCK_PATH, "").toLowerCase
+      if (CarbonProperties.getInstance.getProperty(CarbonCommonConstants.LOCK_PATH,
+        CarbonCommonConstants.LOCK_PATH_DEFAULT).toLowerCase
         .nonEmpty) {
         val tableLockPath = CarbonLockFactory
           .getLockpath(carbonTable.getCarbonTableIdentifier.getTableId)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
index 04b65e8..16d974e 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
@@ -47,8 +47,6 @@ import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.datamap.{TextMatch, TextMatchLimit, TextMatchMaxDocUDF, TextMatchUDF}
 import org.apache.carbondata.spark.CarbonAliasDecoderRelation
 import org.apache.carbondata.spark.rdd.CarbonScanRDD
-import org.apache.carbondata.spark.util.CarbonScalaUtil
-
 
 /**
  * Carbon specific optimization for late decode (convert dictionary key to value as late as
@@ -57,8 +55,6 @@ import org.apache.carbondata.spark.util.CarbonScalaUtil
 private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
   val PUSHED_FILTERS = "PushedFilters"
 
-  val vectorPushRowFilters = CarbonProperties.getInstance().isPushRowFiltersForVector
-
   /*
   Spark 2.3.1 plan there can be case of multiple projections like below
   Project [substring(name, 1, 2)#124, name#123, tupleId#117, cast(rand(-6778822102499951904)#125
@@ -307,6 +303,7 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
     // applying the filter in spark's side. So we should disable vectorPushRowFilters option
     // in case of filters on global dictionary.
     val hasDictionaryFilterCols = hasFilterOnDictionaryColumn(filterSet, table)
+    val vectorPushRowFilters = CarbonProperties.getInstance().isPushRowFiltersForVector
     if (projects.map(_.toAttribute) == projects &&
         projectSet.size == projects.size &&
         filterSet.subsetOf(projectSet)) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
index 186e39e..c681b62 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
@@ -112,7 +112,7 @@ object CarbonSetCommand {
         sessionParams.addProperty(key.toLowerCase, value)
       } else {
         throw new MalformedCarbonCommandException("property should be in " +
-          "\" carbon.datamap.visible.<database_name>.<table_name>.<database_name>" +
+          "\" carbon.datamap.visible.<database_name>.<table_name>.<datamap_name>" +
           " = <true/false> \" format")
       }
     } else if (key.startsWith(CarbonCommonConstants.CARBON_LOAD_DATAMAPS_PARALLEL)) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
index a7fc3f8..fc139a6 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
@@ -1190,8 +1190,9 @@ public final class CarbonDataMergerUtil {
    */
   public static boolean isHorizontalCompactionEnabled() {
     if ((CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.isHorizontalCompactionEnabled,
-            CarbonCommonConstants.defaultIsHorizontalCompactionEnabled)).equalsIgnoreCase("true")) {
+        .getProperty(CarbonCommonConstants.CARBON_HORIZONTAL_COMPACTION_ENABLE,
+            CarbonCommonConstants.CARBON_HORIZONTAL_COMPACTION_ENABLE_DEFAULT))
+        .equalsIgnoreCase("true")) {
       return true;
     } else {
       return false;

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
index f1e1d9e..3e4e31f 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
@@ -591,7 +591,8 @@ public final class CarbonLoaderUtil {
       } else {
         LOGGER.warn("Invalid load_min_size_inmb value found: " + expectedMinSizePerNode
             + ", only int value greater than 0 is supported.");
-        iexpectedMinSizePerNode = CarbonCommonConstants.CARBON_LOAD_MIN_SIZE_DEFAULT;
+        iexpectedMinSizePerNode = Integer.parseInt(
+            CarbonCommonConstants.CARBON_LOAD_MIN_SIZE_INMB_DEFAULT);
       }
       // If the average expected size for each node greater than load min size,
       // then fall back to default strategy


[2/2] carbondata git commit: [CARBONDATA-3038] Refactor dynamic configuration

Posted by ja...@apache.org.
[CARBONDATA-3038] Refactor dynamic configuration

This closes #2875


Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/7c1d68ed
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/7c1d68ed
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/7c1d68ed

Branch: refs/heads/master
Commit: 7c1d68ed62a62d425e57e60613b22bdedd0a94ed
Parents: 35fb940
Author: xubo245 <xu...@huawei.com>
Authored: Thu Nov 1 23:12:05 2018 +0800
Committer: Jacky Li <ja...@qq.com>
Committed: Tue Nov 6 20:28:15 2018 +0800

----------------------------------------------------------------------
 .../carbondata/core/cache/CacheProvider.java    |   3 +-
 .../core/constants/CarbonCommonConstants.java   | 578 ++++++++++++++-----
 .../constants/CarbonLoadOptionConstants.java    |  44 +-
 .../constants/CarbonV3DataFormatConstants.java  |   4 +-
 .../core/locks/CarbonLockFactory.java           |   2 +-
 .../carbondata/core/locks/CarbonLockUtil.java   |   2 +-
 .../core/memory/HeapMemoryAllocator.java        |   3 +-
 .../core/memory/UnsafeMemoryManager.java        |   3 +-
 .../carbondata/core/util/CarbonProperties.java  |  26 +-
 .../carbondata/core/util/CarbonProperty.java    |  28 -
 .../carbondata/core/util/SessionParams.java     |  13 +-
 .../core/util/annotations/CarbonProperty.java   |  37 ++
 .../core/util/path/HDFSLeaseUtils.java          |  47 +-
 .../core/CarbonPropertiesValidationTest.java    |   1 -
 .../DateDirectDictionaryGeneratorTest.java      |   1 -
 docs/configuration-parameters.md                |  12 +-
 .../iud/HorizontalCompactionTestCase.scala      |   4 +-
 .../iud/UpdateCarbonTableTestCase.scala         |  14 +-
 .../spark/util/CarbonReflectionUtils.scala      |   2 +-
 .../management/CarbonLoadDataCommand.scala      |   3 +-
 .../command/table/CarbonDropTableCommand.scala  |   3 +-
 .../strategy/CarbonLateDecodeStrategy.scala     |   5 +-
 .../execution/command/CarbonHiveCommands.scala  |   2 +-
 .../processing/merger/CarbonDataMergerUtil.java |   5 +-
 .../processing/util/CarbonLoaderUtil.java       |   3 +-
 25 files changed, 574 insertions(+), 271 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java b/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java
index d809c25..99b1693 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/CacheProvider.java
@@ -152,7 +152,8 @@ public class CacheProvider {
    */
   private void createLRULevelCacheInstance() {
     boolean isDriver = Boolean.parseBoolean(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.IS_DRIVER_INSTANCE, "false"));
+        .getProperty(CarbonCommonConstants.IS_DRIVER_INSTANCE,
+            CarbonCommonConstants.IS_DRIVER_INSTANCE_DEFAULT));
     if (isDriver) {
       carbonLRUCache = new CarbonLRUCache(CarbonCommonConstants.CARBON_MAX_DRIVER_LRU_CACHE_SIZE,
           CarbonCommonConstants.CARBON_MAX_LRU_CACHE_SIZE_DEFAULT);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index 9484bb4..fc26404 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -20,7 +20,7 @@ package org.apache.carbondata.core.constants;
 import java.nio.charset.Charset;
 
 import org.apache.carbondata.common.annotations.InterfaceStability;
-import org.apache.carbondata.core.util.CarbonProperty;
+import org.apache.carbondata.core.util.annotations.CarbonProperty;
 
 public final class CarbonCommonConstants {
 
@@ -43,71 +43,80 @@ public final class CarbonCommonConstants {
   /**
    * blocklet size in carbon file
    */
+  @CarbonProperty
   public static final String BLOCKLET_SIZE = "carbon.blocklet.size";
 
   /**
    * default blocklet size
    */
   public static final String BLOCKLET_SIZE_DEFAULT_VAL = "120000";
+
   /**
    * min blocklet size
    */
   public static final int BLOCKLET_SIZE_MIN_VAL = 2000;
+
   /**
    * max blocklet size
    */
   public static final int BLOCKLET_SIZE_MAX_VAL = 12000000;
+
   /**
    * default block size in MB
    */
   public static final String BLOCK_SIZE_DEFAULT_VAL = "1024";
+
   /**
    * min block size in MB
    */
   public static final int BLOCK_SIZE_MIN_VAL = 1;
+
   /**
    * max block size in MB
    */
   public static final int BLOCK_SIZE_MAX_VAL = 2048;
+
   /**
    * carbon properties file path
    */
-  public static final String CARBON_PROPERTIES_FILE_PATH = "../../../conf/carbon.properties";
+  @CarbonProperty
+  public static final String CARBON_PROPERTIES_FILE_PATH = "carbon.properties.filepath";
+
+  /**
+   * default carbon properties file path
+   */
+  public static final String CARBON_PROPERTIES_FILE_PATH_DEFAULT =
+      "../../../conf/carbon.properties";
+
   /**
    * CARBON_DDL_BASE_HDFS_URL
    */
   @CarbonProperty
   public static final String CARBON_DDL_BASE_HDFS_URL = "carbon.ddl.base.hdfs.url";
+
   /**
    * CARBON_BADRECORDS_LOCATION
    */
   @CarbonProperty
   public static final String CARBON_BADRECORDS_LOC = "carbon.badRecords.location";
+
   /**
    * CARBON_BADRECORDS_LOCATION_DEFAULT
    */
   public static final String CARBON_BADRECORDS_LOC_DEFAULT_VAL = "";
-  /**
-   * CARBON_TIMESTAMP_DEFAULT_FORMAT
-   */
-  public static final String CARBON_TIMESTAMP_DEFAULT_FORMAT = "yyyy-MM-dd HH:mm:ss";
+
   /**
    * Property for specifying the format of TIMESTAMP data type column.
-   * e.g. yyyy/MM/dd HH:mm:ss, or using CARBON_TIMESTAMP_DEFAULT_FORMAT
+   * e.g. yyyy/MM/dd HH:mm:ss, or using default value
    */
   @CarbonProperty
   public static final String CARBON_TIMESTAMP_FORMAT = "carbon.timestamp.format";
 
   /**
-   * CARBON_DATE_DEFAULT_FORMAT
-   */
-  public static final String CARBON_DATE_DEFAULT_FORMAT = "yyyy-MM-dd";
-  /**
-   * Property for specifying the format of DATE data type column.
-   * e.g. yyyy/MM/dd , or using CARBON_DATE_DEFAULT_FORMAT
+   * default value
    */
-  @CarbonProperty
-  public static final String CARBON_DATE_FORMAT = "carbon.date.format";
+  public static final String CARBON_TIMESTAMP_DEFAULT_FORMAT = "yyyy-MM-dd HH:mm:ss";
+
   /**
    * CARBON_TIMESTAMP
    */
@@ -117,56 +126,79 @@ public final class CarbonCommonConstants {
    * CARBON_TIMESTAMP
    */
   public static final String CARBON_TIMESTAMP_MILLIS = "dd-MM-yyyy HH:mm:ss:SSS";
+
+  /**
+   * Property for specifying the format of DATE data type column.
+   * e.g. yyyy/MM/dd , or using default value
+   */
+  @CarbonProperty
+  public static final String CARBON_DATE_FORMAT = "carbon.date.format";
+
+  /**
+   * default value
+   */
+  public static final String CARBON_DATE_DEFAULT_FORMAT = "yyyy-MM-dd";
+
   /**
-   * compressor for writing/reading carbondata file
+   * compressor for writing/reading CarbonData file
    */
+  @CarbonProperty
   public static final String COMPRESSOR = "carbon.column.compressor";
 
   /**
    * default compressor is snappy
    */
   public static final String DEFAULT_COMPRESSOR = "snappy";
+
   /**
-   * ZOOKEEPER_ENABLE_LOCK if this is set to true then zookeeper will be used to handle locking
+   * ZOOKEEPER_ENABLE_LOCK if this is set to true then zookeeper
+   * will be used to handle locking
    * mechanism of carbon
    */
   @CarbonProperty
   public static final String LOCK_TYPE = "carbon.lock.type";
 
   /**
+   * ZOOKEEPER_ENABLE_DEFAULT the default value for zookeeper will be true for carbon
+   */
+  public static final String LOCK_TYPE_DEFAULT = "LOCALLOCK";
+
+  /**
    * Specifies the path where the lock files have to be created.
    * By default, lock files are created in table path.
    */
   @CarbonProperty
   public static final String LOCK_PATH = "carbon.lock.path";
 
-  /**
-   * ZOOKEEPER_ENABLE_DEFAULT the default value for zookeeper will be true for carbon
-   */
-  public static final String LOCK_TYPE_DEFAULT = "LOCALLOCK";
+  public static final String LOCK_PATH_DEFAULT = "";
 
   /**
    * ZOOKEEPER_LOCATION this is the location in zookeeper file system where locks are created.
    * mechanism of carbon
    */
   public static final String ZOOKEEPER_LOCATION = "/CarbonLocks";
+
   /**
    * xxhash algorithm property for hashmap
    */
+  @CarbonProperty
   public static final String ENABLE_XXHASH = "carbon.enableXXHash";
 
   /**
-   * xxhash algorithm property for hashmap. Default value false
+   * xxhash algorithm property for hashmap Default value false
    */
   public static final String ENABLE_XXHASH_DEFAULT = "true";
+
   /**
    * System property to enable or disable local dictionary generation
    */
+  @CarbonProperty
   public static final String LOCAL_DICTIONARY_SYSTEM_ENABLE = "carbon.local.dictionary.enable";
 
   /**
    * System property to enable or disable decoder based local dictionary fallback
    */
+  @CarbonProperty
   public static final String LOCAL_DICTIONARY_DECODER_BASED_FALLBACK =
       "carbon.local.dictionary.decoder.fallback";
 
@@ -174,14 +206,19 @@ public final class CarbonCommonConstants {
    * System property to enable or disable decoder based local dictionary fallback default value
    */
   public static final String LOCAL_DICTIONARY_DECODER_BASED_FALLBACK_DEFAULT = "true";
+
   /**
    * zookeeper url key
    */
+  @CarbonProperty
   public static final String ZOOKEEPER_URL = "spark.deploy.zookeeper.url";
+
   /**
    * carbon data file version property
    */
+  @CarbonProperty
   public static final String CARBON_DATA_FILE_VERSION = "carbon.data.file.version";
+
   @CarbonProperty
   public static final String SPARK_SCHEMA_STRING_LENGTH_THRESHOLD =
       "spark.sql.sources.schemaStringLengthThreshold";
@@ -196,6 +233,7 @@ public final class CarbonCommonConstants {
    */
   public static final String CARBON_BAD_RECORDS_ACTION_DEFAULT = "FAIL";
 
+  @CarbonProperty
   public static final String ENABLE_HIVE_SCHEMA_META_STORE = "spark.carbon.hive.schema.store";
 
   public static final String ENABLE_HIVE_SCHEMA_META_STORE_DEFAULT = "false";
@@ -204,44 +242,69 @@ public final class CarbonCommonConstants {
    * There is more often that in production uses different drivers for load and queries. So in case
    * of load driver user should set this property to enable loader specific clean up.
    */
+  @CarbonProperty
   public static final String DATA_MANAGEMENT_DRIVER = "spark.carbon.datamanagement.driver";
 
   public static final String DATA_MANAGEMENT_DRIVER_DEFAULT = "true";
 
+  @CarbonProperty
   public static final String CARBON_SESSIONSTATE_CLASSNAME = "spark.carbon.sessionstate.classname";
 
   /**
    * This property will be used to configure the sqlastbuilder class.
    */
+  @CarbonProperty
   public static final String CARBON_SQLASTBUILDER_CLASSNAME =
       "spark.carbon.sqlastbuilder.classname";
+
+  public static final String CARBON_SQLASTBUILDER_CLASSNAME_DEFAULT =
+      "org.apache.spark.sql.hive.CarbonSqlAstBuilder";
+
   @CarbonProperty
   public static final String CARBON_LEASE_RECOVERY_RETRY_COUNT =
       "carbon.lease.recovery.retry.count";
+
+  public static final String CARBON_LEASE_RECOVERY_RETRY_COUNT_DEFAULT = "5";
+
+  public static final int CARBON_LEASE_RECOVERY_RETRY_COUNT_MIN = 1;
+
+  public static final int CARBON_LEASE_RECOVERY_RETRY_COUNT_MAX = 50;
+
   @CarbonProperty
   public static final String CARBON_LEASE_RECOVERY_RETRY_INTERVAL =
       "carbon.lease.recovery.retry.interval";
 
+  public static final String CARBON_LEASE_RECOVERY_RETRY_INTERVAL_DEFAULT = "1000";
+
+  public static final int CARBON_LEASE_RECOVERY_RETRY_INTERVAL_MIN = 1000;
+
+  public static final int CARBON_LEASE_RECOVERY_RETRY_INTERVAL_MAX = 10000;
+
   @CarbonProperty
   public static final String CARBON_SECURE_DICTIONARY_SERVER =
       "carbon.secure.dictionary.server";
 
   public static final String CARBON_SECURE_DICTIONARY_SERVER_DEFAULT = "true";
+
   /**
    * ENABLE_CALCULATE_DATA_INDEX_SIZE
    */
-  @CarbonProperty public static final String ENABLE_CALCULATE_SIZE = "carbon.enable.calculate.size";
+  @CarbonProperty
+  public static final String ENABLE_CALCULATE_SIZE = "carbon.enable.calculate.size";
 
   /**
-   * DEFAULT_ENABLE_CALCULATE_DATA_INDEX_SIZE
+   * default value of ENABLE_CALCULATE_DATA_INDEX_SIZE
    */
   public static final String DEFAULT_ENABLE_CALCULATE_SIZE = "true";
+
   /**
    * this will be used to skip / ignore empty lines while loading
    */
-  @CarbonProperty public static final String CARBON_SKIP_EMPTY_LINE = "carbon.skip.empty.line";
+  @CarbonProperty
+  public static final String CARBON_SKIP_EMPTY_LINE = "carbon.skip.empty.line";
 
   public static final String CARBON_SKIP_EMPTY_LINE_DEFAULT = "false";
+
   /**
    * Currently the segment lock files are not deleted immediately when unlock,
    * this value indicates the number of hours the segment lock files will be preserved.
@@ -250,7 +313,9 @@ public final class CarbonCommonConstants {
   public static final String CARBON_SEGMENT_LOCK_FILES_PRESERVE_HOURS =
       "carbon.segment.lock.files.preserve.hours";
 
-  // default value is 2 days
+  /**
+   * default value is 2 days
+   */
   public static final String CARBON_SEGMENT_LOCK_FILES_PRESERVE_HOURS_DEFAULT = "48";
 
   /**
@@ -272,7 +337,9 @@ public final class CarbonCommonConstants {
   /**
    * System older location to store system level data like datamap schema and status files.
    */
+  @CarbonProperty
   public static final String CARBON_SYSTEM_FOLDER_LOCATION = "carbon.system.folder.location";
+
   /**
    * It is internal configuration and used only for test purpose.
    * It will merge the carbon index files with in the segment to single segment.
@@ -282,12 +349,15 @@ public final class CarbonCommonConstants {
       "carbon.merge.index.in.segment";
 
   public static final String CARBON_MERGE_INDEX_IN_SEGMENT_DEFAULT = "true";
+
   /**
    * property to be used for specifying the max byte limit for string/varchar data type till
    * where storing min/max in data file will be considered
    */
+  @CarbonProperty
   public static final String CARBON_MINMAX_ALLOWED_BYTE_COUNT =
       "carbon.minmax.allowed.byte.count";
+
   public static final String CARBON_MINMAX_ALLOWED_BYTE_COUNT_DEFAULT = "200";
 
   //////////////////////////////////////////////////////////////////////////////////////////
@@ -295,15 +365,19 @@ public final class CarbonCommonConstants {
   //////////////////////////////////////////////////////////////////////////////////////////
   // Table level property is the table property for Carbon table
 
-  // Flat folder support on table. when it is true all carbondata files store directly under table
+  // Flat folder support on table. when it is true all CarbonData files store directly under table
   // path instead of sub folders.
   public static final String FLAT_FOLDER = "flat_folder";
+
   /**
-   * DEFAULT_FLAT_FOLDER
+   * FLAT_FOLDER default value
    */
   public static final String DEFAULT_FLAT_FOLDER = "false";
+
   public static final String DICTIONARY_EXCLUDE = "dictionary_exclude";
+
   public static final String DICTIONARY_INCLUDE = "dictionary_include";
+
   public static final String LONG_STRING_COLUMNS = "long_string_columns";
 
   /**
@@ -315,6 +389,7 @@ public final class CarbonCommonConstants {
    * default value for local dictionary generation
    */
   public static final String LOCAL_DICTIONARY_ENABLE_DEFAULT = "true";
+
   /**
    * Threshold value for local dictionary
    */
@@ -346,6 +421,12 @@ public final class CarbonCommonConstants {
   public static final String LOCAL_DICTIONARY_EXCLUDE = "local_dictionary_exclude";
 
   /**
+   * DMPROPERTY for Index DataMap, like lucene, bloomfilter DataMap,
+   * to indicate a list of column name to be indexed
+   */
+  public static final String INDEX_COLUMNS = "INDEX_COLUMNS";
+
+  /**
    * key for dictionary path
    */
   public static final String DICTIONARY_PATH = "dictionary_path";
@@ -371,14 +452,17 @@ public final class CarbonCommonConstants {
   public static final String TABLE_COMPACTION_PRESERVE_SEGMENTS = "compaction_preserve_segments";
   // table property name of allowed compaction days while compaction
   public static final String TABLE_ALLOWED_COMPACTION_DAYS = "allowed_compaction_days";
+
   /**
    * property to be specified for caching min/max of required columns
    */
   public static final String COLUMN_META_CACHE = "column_meta_cache";
+
   /**
    * property to be specified for caching level (Block/Blocket)
    */
   public static final String CACHE_LEVEL = "cache_level";
+
   /**
    * default value for cache level
    */
@@ -391,42 +475,42 @@ public final class CarbonCommonConstants {
   /**
    * Number of cores to be used while loading
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String NUM_CORES_LOADING = "carbon.number.of.cores.while.loading";
+
   /**
    * default value of number of cores to be used
    */
   public static final String NUM_CORES_DEFAULT_VAL = "2";
+
   /**
    * Number of cores to be used while compacting
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String NUM_CORES_COMPACTING = "carbon.number.of.cores.while.compacting";
+
   /**
    * Number of cores to be used while alter partition
    */
+  @CarbonProperty
   public static final String NUM_CORES_ALT_PARTITION = "carbon.number.of.cores.while.altPartition";
-  /**
-   * default carbon sort size
-   */
-  public static final String SORT_SIZE_DEFAULT_VAL = "100000";
-  /**
-   * min carbon sort size
-   */
-  public static final int SORT_SIZE_MIN_VAL = 1000;
+
   /**
    * BYTEBUFFER_SIZE
    */
   public static final int BYTEBUFFER_SIZE = 24 * 1024;
+
   /**
    * SORT_TEMP_FILE_LOCATION
    */
   public static final String SORT_TEMP_FILE_LOCATION = "sortrowtmp";
+
   /**
    * SORT_INTERMEDIATE_FILES_LIMIT
    */
   @CarbonProperty
   public static final String SORT_INTERMEDIATE_FILES_LIMIT = "carbon.sort.intermediate.files.limit";
+
   /**
    * SORT_INTERMEDIATE_FILES_LIMIT_DEFAULT_VALUE
    */
@@ -435,53 +519,75 @@ public final class CarbonCommonConstants {
   public static final int SORT_INTERMEDIATE_FILES_LIMIT_MIN = 2;
 
   public static final int SORT_INTERMEDIATE_FILES_LIMIT_MAX = 50;
+
   /**
    * SORT_FILE_WRITE_BUFFER_SIZE
    */
   @CarbonProperty
   public static final String CARBON_SORT_FILE_WRITE_BUFFER_SIZE =
       "carbon.sort.file.write.buffer.size";
-  public static final int CARBON_SORT_FILE_WRITE_BUFFER_SIZE_MIN = 10240;
-  public static final int CARBON_SORT_FILE_WRITE_BUFFER_SIZE_MAX = 10485760;
+
   /**
    * SORT_FILE_WRITE_BUFFER_SIZE_DEFAULT_VALUE
    */
   public static final String CARBON_SORT_FILE_WRITE_BUFFER_SIZE_DEFAULT_VALUE = "16384";
+
+  public static final int CARBON_SORT_FILE_WRITE_BUFFER_SIZE_MIN = 10240;
+
+  public static final int CARBON_SORT_FILE_WRITE_BUFFER_SIZE_MAX = 10485760;
+
   /**
    * CSV_READ_BUFFER_SIZE
    */
   @CarbonProperty
   public static final String CSV_READ_BUFFER_SIZE = "carbon.csv.read.buffersize.byte";
+
   /**
    * CSV_READ_BUFFER_SIZE
+   * default value is 1mb
    */
-  public static final String CSV_READ_BUFFER_SIZE_DEFAULT = "1048576"; // 1mb
+  public static final String CSV_READ_BUFFER_SIZE_DEFAULT = "1048576";
+
   /**
-   * min value for csv read buffer size
+   * min value for csv read buffer size, 10 kb
    */
-  public static final int CSV_READ_BUFFER_SIZE_MIN = 10240; //10 kb
+  public static final int CSV_READ_BUFFER_SIZE_MIN = 10240;
+
   /**
-   * max value for csv read buffer size
+   * max value for csv read buffer size, 10 mb
    */
-  public static final int CSV_READ_BUFFER_SIZE_MAX = 10485760; // 10 mb
+  public static final int CSV_READ_BUFFER_SIZE_MAX = 10485760;
+
   /**
    * CARBON_MERGE_SORT_READER_THREAD
    */
   @CarbonProperty
   public static final String CARBON_MERGE_SORT_READER_THREAD = "carbon.merge.sort.reader.thread";
+
   /**
-   * CARBON_MERGE_SORT_READER_THREAD_DEFAULTVALUE
+   * CARBON_MERGE_SORT_READER_THREAD DEFAULT value
    */
   public static final String CARBON_MERGE_SORT_READER_THREAD_DEFAULTVALUE = "3";
+
   /**
    * TIME_STAT_UTIL_TYPE
    */
   @CarbonProperty
   public static final String ENABLE_DATA_LOADING_STATISTICS = "enable.data.loading.statistics";
+
   /**
    * TIME_STAT_UTIL_TYPE_DEFAULT
    */
+
   public static final String ENABLE_DATA_LOADING_STATISTICS_DEFAULT = "false";
+
+  /**
+   * NUMBER_OF_TRIES_FOR_LOAD_METADATA_LOCK
+   */
+  @CarbonProperty
+  public static final String NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK =
+      "carbon.concurrent.lock.retries";
+
   /**
    * NUMBER_OF_TRIES_FOR_LOAD_METADATA_LOCK.
    *
@@ -491,75 +597,83 @@ public final class CarbonCommonConstants {
    * Example: Concurrent loads will use this to wait to acquire the table status lock.
    */
   public static final int NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK_DEFAULT = 100;
-  /**
-   * MAX_TIMEOUT_FOR_LOAD_METADATA_LOCK
-   *
-   * * Example: Concurrent loads will use this to wait to acquire the table status lock.
-   */
-  public static final int MAX_TIMEOUT_FOR_CONCURRENT_LOCK_DEFAULT = 1;
-  /**
-   * NUMBER_OF_TRIES_FOR_LOAD_METADATA_LOCK
-   */
-  @CarbonProperty
-  public static final String NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK =
-      "carbon.concurrent.lock.retries";
+
   /**
    * MAX_TIMEOUT_FOR_LOAD_METADATA_LOCK
    */
   @CarbonProperty
   public static final String MAX_TIMEOUT_FOR_CONCURRENT_LOCK =
       "carbon.concurrent.lock.retry.timeout.sec";
+
   /**
-   * NUMBER_OF_TRIES_FOR_CARBON_LOCK
-   */
-  public static final int NUMBER_OF_TRIES_FOR_CARBON_LOCK_DEFAULT = 3;
-  /**
-   * MAX_TIMEOUT_FOR_CARBON_LOCK
+   * MAX_TIMEOUT_FOR_LOAD_METADATA_LOCK
+   *
+   * Example: Concurrent loads will use this to wait to acquire the table status lock.
    */
-  public static final int MAX_TIMEOUT_FOR_CARBON_LOCK_DEFAULT = 5;
+  public static final int MAX_TIMEOUT_FOR_CONCURRENT_LOCK_DEFAULT = 1;
+
   /**
    * NUMBER_OF_TRIES_FOR_CARBON_LOCK
    */
   @CarbonProperty
   public static final String NUMBER_OF_TRIES_FOR_CARBON_LOCK =
       "carbon.lock.retries";
+
+  /**
+   * NUMBER_OF_TRIES_FOR_CARBON_LOCK
+   */
+  public static final int NUMBER_OF_TRIES_FOR_CARBON_LOCK_DEFAULT = 3;
+
   /**
    * MAX_TIMEOUT_FOR_CARBON_LOCK
    */
   @CarbonProperty
   public static final String MAX_TIMEOUT_FOR_CARBON_LOCK =
       "carbon.lock.retry.timeout.sec";
+
+  /**
+   * MAX_TIMEOUT_FOR_CARBON_LOCK
+   */
+  public static final int MAX_TIMEOUT_FOR_CARBON_LOCK_DEFAULT = 5;
+
   /**
    * CARBON_PREFETCH_BUFFERSIZE
    */
   @CarbonProperty
   public static final String CARBON_PREFETCH_BUFFERSIZE = "carbon.prefetch.buffersize";
+
   /**
    * CARBON_PREFETCH_BUFFERSIZE DEFAULT VALUE
    */
   public static final String CARBON_PREFETCH_BUFFERSIZE_DEFAULT = "1000";
+
   /**
    * CARBON_PREFETCH_IN_MERGE
    */
   public static final boolean CARBON_PREFETCH_IN_MERGE_VALUE = false;
+
   /**
    * ENABLE_AUTO_LOAD_MERGE
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String ENABLE_AUTO_LOAD_MERGE = "carbon.enable.auto.load.merge";
+
   /**
-   * DEFAULT_ENABLE_AUTO_LOAD_MERGE
+   * DEFAULT value of ENABLE_AUTO_LOAD_MERGE
    */
   public static final String DEFAULT_ENABLE_AUTO_LOAD_MERGE = "false";
+
   /**
    * maximum dictionary chunk size that can be kept in memory while writing dictionary file
    */
   @CarbonProperty
   public static final String DICTIONARY_ONE_CHUNK_SIZE = "carbon.dictionary.chunk.size";
+
   /**
    * dictionary chunk default size
    */
   public static final String DICTIONARY_ONE_CHUNK_SIZE_DEFAULT = "10000";
+
   /**
    *  Dictionary Server Worker Threads
    */
@@ -570,16 +684,18 @@ public final class CarbonCommonConstants {
    *  Dictionary Server Worker Threads
    */
   public static final String DICTIONARY_WORKER_THREADS_DEFAULT = "1";
+
   /**
    * Size of Major Compaction in MBs
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_MAJOR_COMPACTION_SIZE = "carbon.major.compaction.size";
 
   /**
    * By default size of major compaction in MBs.
    */
   public static final String DEFAULT_CARBON_MAJOR_COMPACTION_SIZE = "1024";
+
   /**
    * This property is used to tell how many segments to be preserved from merging.
    */
@@ -606,6 +722,7 @@ public final class CarbonCommonConstants {
   /**
    * space reserved for writing block meta data in carbon data file
    */
+  @CarbonProperty
   public static final String CARBON_BLOCK_META_RESERVED_SPACE =
       "carbon.block.meta.size.reserved.percentage";
 
@@ -613,13 +730,16 @@ public final class CarbonCommonConstants {
    * default value for space reserved for writing block meta data in carbon data file
    */
   public static final String CARBON_BLOCK_META_RESERVED_SPACE_DEFAULT = "10";
+
   /**
    * this variable is to enable/disable prefetch of data during merge sort while
    * reading data from sort temp files
    */
   @CarbonProperty
   public static final String CARBON_MERGE_SORT_PREFETCH = "carbon.merge.sort.prefetch";
+
   public static final String CARBON_MERGE_SORT_PREFETCH_DEFAULT = "true";
+
   /**
    * If we are executing insert into query from source table using select statement
    * & loading the same source table concurrently, when select happens on source table
@@ -629,13 +749,12 @@ public final class CarbonCommonConstants {
    * will be false because no need to persist the dataframe in all cases. If user want
    * to run load and insert queries on source table concurrently then user can enable this flag
    */
-  @CarbonProperty
   @InterfaceStability.Evolving
+  @CarbonProperty
   public static final String CARBON_INSERT_PERSIST_ENABLED = "carbon.insert.persist.enable";
 
   /**
    * by default rdd will not be persisted in the insert case.
-
    */
   public static final String CARBON_INSERT_PERSIST_ENABLED_DEFAULT = "false";
 
@@ -643,8 +762,8 @@ public final class CarbonCommonConstants {
    * Which storage level to persist dataset when insert into data
    * with 'carbon.insert.persist.enable'='true'
    */
-  @CarbonProperty
   @InterfaceStability.Evolving
+  @CarbonProperty
   public static final String CARBON_INSERT_STORAGE_LEVEL =
       "carbon.insert.storage.level";
 
@@ -659,10 +778,11 @@ public final class CarbonCommonConstants {
    * http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence.
    */
   public static final String CARBON_INSERT_STORAGE_LEVEL_DEFAULT = "MEMORY_AND_DISK";
+
   /**
    * Number of unmerged segments to be merged.
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String COMPACTION_SEGMENT_LEVEL_THRESHOLD =
       "carbon.compaction.level.threshold";
 
@@ -678,6 +798,7 @@ public final class CarbonCommonConstants {
   @CarbonProperty
   public static final String UPDATE_DELTAFILE_COUNT_THRESHOLD_IUD_COMPACTION =
       "carbon.horizontal.update.compaction.threshold";
+
   /**
    * Default count of segments which act as a threshold for IUD compaction merge.
    */
@@ -690,10 +811,12 @@ public final class CarbonCommonConstants {
   @CarbonProperty
   public static final String DELETE_DELTAFILE_COUNT_THRESHOLD_IUD_COMPACTION =
       "carbon.horizontal.delete.compaction.threshold";
+
   /**
    * Default count of segments which act as a threshold for IUD compaction merge.
    */
   public static final String DEFAULT_DELETE_DELTAFILE_COUNT_THRESHOLD_IUD_COMPACTION = "1";
+
   /**
    * @Deprecated : This property has been deprecated.
    * Property for enabling system level compaction lock.1 compaction can run at once.
@@ -706,44 +829,48 @@ public final class CarbonCommonConstants {
    * at once.
    */
   public static final String DEFAULT_ENABLE_CONCURRENT_COMPACTION = "true";
+
   /**
    * This batch size is used to send rows from load step to another step in batches.
    */
+  @CarbonProperty
   public static final String DATA_LOAD_BATCH_SIZE = "DATA_LOAD_BATCH_SIZE";
 
   /**
    * Default size of data load batch size.
    */
   public static final String DATA_LOAD_BATCH_SIZE_DEFAULT = "1000";
+
   /**
    * to determine to use the rdd persist or not.
    */
-  @CarbonProperty
   @InterfaceStability.Evolving
-  public static final String isPersistEnabled = "carbon.update.persist.enable";
+  @CarbonProperty
+  public static final String CARBON_UPDATE_PERSIST_ENABLE = "carbon.update.persist.enable";
 
   /**
-   * for enabling or disabling Horizontal Compaction.
+   * by default rdd will be persisted in the update case.
    */
-  @CarbonProperty
-  public static final String isHorizontalCompactionEnabled = "carbon.horizontal.compaction.enable";
+  public static final String CARBON_UPDATE_PERSIST_ENABLE_DEFAULT = "true";
 
   /**
-   * Default value for HorizontalCompaction is true.
+   * for enabling or disabling Horizontal Compaction.
    */
-  public static final String defaultIsHorizontalCompactionEnabled = "true";
+  @CarbonProperty
+  public static final String CARBON_HORIZONTAL_COMPACTION_ENABLE =
+      "carbon.horizontal.compaction.enable";
 
   /**
-   * by default rdd will be persisted in the update case.
+   * Default value for HorizontalCompaction is true.
    */
-  public static final String defaultValueIsPersistEnabled = "true";
+  public static final String CARBON_HORIZONTAL_COMPACTION_ENABLE_DEFAULT = "true";
 
   /**
    * Which storage level to persist dataset when updating data
    * with 'carbon.update.persist.enable'='true'
    */
-  @CarbonProperty
   @InterfaceStability.Evolving
+  @CarbonProperty
   public static final String CARBON_UPDATE_STORAGE_LEVEL =
       "carbon.update.storage.level";
 
@@ -758,9 +885,11 @@ public final class CarbonCommonConstants {
    * http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence.
    */
   public static final String CARBON_UPDATE_STORAGE_LEVEL_DEFAULT = "MEMORY_AND_DISK";
+
   /**
    * to enable unsafe column page
    */
+  @CarbonProperty
   public static final String ENABLE_UNSAFE_COLUMN_PAGE = "enable.unsafe.columnpage";
 
   /**
@@ -771,7 +900,7 @@ public final class CarbonCommonConstants {
   /**
    * to enable offheap sort
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String ENABLE_UNSAFE_SORT = "enable.unsafe.sort";
 
   /**
@@ -782,27 +911,33 @@ public final class CarbonCommonConstants {
   /**
    * to enable offheap sort
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String ENABLE_OFFHEAP_SORT = "enable.offheap.sort";
 
   /**
    * to enable offheap sort
    */
   public static final String ENABLE_OFFHEAP_SORT_DEFAULT = "true";
+
   @CarbonProperty
   public static final String ENABLE_INMEMORY_MERGE_SORT = "enable.inmemory.merge.sort";
 
   public static final String ENABLE_INMEMORY_MERGE_SORT_DEFAULT = "false";
+
+  @CarbonProperty
   public static final String OFFHEAP_SORT_CHUNK_SIZE_IN_MB = "offheap.sort.chunk.size.inmb";
 
   public static final String OFFHEAP_SORT_CHUNK_SIZE_IN_MB_DEFAULT = "64";
+
   @CarbonProperty
   public static final String UNSAFE_WORKING_MEMORY_IN_MB = "carbon.unsafe.working.memory.in.mb";
+
   public static final String UNSAFE_WORKING_MEMORY_IN_MB_DEFAULT = "512";
 
   @CarbonProperty
   public static final String UNSAFE_DRIVER_WORKING_MEMORY_IN_MB =
       "carbon.unsafe.driver.working.memory.in.mb";
+
   /**
    * Sorts the data in batches and writes the batch data to store with index file.
    */
@@ -825,16 +960,19 @@ public final class CarbonCommonConstants {
    */
   @CarbonProperty
   public static final String LOAD_BATCH_SORT_SIZE_INMB = "carbon.load.batch.sort.size.inmb";
+
   public static final String LOAD_BATCH_SORT_SIZE_INMB_DEFAULT = "0";
-  @CarbonProperty
+
   /**
    * The Number of partitions to use when shuffling data for sort. If user don't configurate or
    * configurate it less than 1, it uses the number of map tasks as reduce tasks. In general, we
    * recommend 2-3 tasks per CPU core in your cluster.
    */
+  @CarbonProperty
   public static final String LOAD_GLOBAL_SORT_PARTITIONS = "carbon.load.global.sort.partitions";
 
   public static final String LOAD_GLOBAL_SORT_PARTITIONS_DEFAULT = "0";
+
   /*
    * carbon dictionary server port
    */
@@ -855,6 +993,7 @@ public final class CarbonCommonConstants {
    * default value for prefetch data while loading.
    */
   public static final String USE_PREFETCH_WHILE_LOADING_DEFAULT = "false";
+
   /**
    * for loading, whether to use yarn's local dir the main purpose is to avoid single disk hot spot
    */
@@ -877,11 +1016,12 @@ public final class CarbonCommonConstants {
    * Specially, empty means that Carbondata will not compress the sort temp files.
    */
   public static final String CARBON_SORT_TEMP_COMPRESSOR_DEFAULT = "SNAPPY";
+
   /**
    * Which storage level to persist rdd when sort_scope=global_sort
    */
-  @CarbonProperty
   @InterfaceStability.Evolving
+  @CarbonProperty
   public static final String CARBON_GLOBAL_SORT_RDD_STORAGE_LEVEL =
       "carbon.global.sort.rdd.storage.level";
 
@@ -897,7 +1037,7 @@ public final class CarbonCommonConstants {
   /**
    * property for configuring parallelism per segment when doing an update. Increase this
    * value will avoid data screw problem for a large segment.
-   * Refer to CARBONDATA-1373 for more details.
+   * Refer to CarbonData-1373 for more details.
    */
   @CarbonProperty
   public static final String CARBON_UPDATE_SEGMENT_PARALLELISM =
@@ -907,15 +1047,17 @@ public final class CarbonCommonConstants {
    * In default we will not optimize the update
    */
   public static final String CARBON_UPDATE_SEGMENT_PARALLELISM_DEFAULT = "1";
+
   /**
    * The property to configure the mdt file folder path, earlier it was pointing to the
    * fixed carbon store path. This is needed in case of the federation setup when user removes
-   * the fixedtorepath namesevice
+   * the fixed store path nameService
    */
   @CarbonProperty
   public static final String CARBON_UPDATE_SYNC_FOLDER = "carbon.update.sync.folder";
 
   public static final String CARBON_UPDATE_SYNC_FOLDER_DEFAULT = "/tmp/carbondata";
+
   /**
    * Configures the parser/writer to limit the length of displayed contents being parsed/written
    * in the exception message when an error occurs.
@@ -932,41 +1074,59 @@ public final class CarbonCommonConstants {
   public static final String HANDOFF_SIZE = "carbon.streaming.segment.max.size";
 
   /**
+   * the default handoff size of streaming segment, the unit is byte
+   */
+  public static final long HANDOFF_SIZE_DEFAULT = 1024L * 1024 * 1024;
+
+  /**
+   * the min handoff size of streaming segment, the unit is byte
+   */
+  public static final long HANDOFF_SIZE_MIN = 1024L * 1024 * 64;
+
+  /**
    * enable auto handoff streaming segment
    */
   @CarbonProperty
   public static final String ENABLE_AUTO_HANDOFF = "carbon.streaming.auto.handoff.enabled";
 
   public static final String ENABLE_AUTO_HANDOFF_DEFAULT = "true";
+
   /**
    * Enabling page level reader for compaction reduces the memory usage while compacting more
    * number of segments. It allows reading only page by page instead of reaing whole blocklet to
    * memory.
    */
-  @CarbonProperty
   @InterfaceStability.Evolving
+  @CarbonProperty
   public static final String CARBON_ENABLE_PAGE_LEVEL_READER_IN_COMPACTION =
       "carbon.enable.page.level.reader.in.compaction";
 
-  // Note: If this property is set to true it can impact compaction performance as IO will increase
+  /**
+   * Default value
+   * Note: If this property is set to true it can impact compaction performance as IO will increase
+   */
   public static final String CARBON_ENABLE_PAGE_LEVEL_READER_IN_COMPACTION_DEFAULT = "false";
 
   @CarbonProperty
   public static final String CARBON_SORT_STORAGE_INMEMORY_IN_MB =
       "carbon.sort.storage.inmemory.size.inmb";
+
   public static final int CARBON_SORT_STORAGE_INMEMORY_IN_MB_DEFAULT = 512;
+
   /*
    * whether to enable prefetch for rowbatch to enhance row reconstruction during compaction
    */
   @CarbonProperty
   public static final String CARBON_COMPACTION_PREFETCH_ENABLE =
       "carbon.compaction.prefetch.enable";
+
   public static final String CARBON_COMPACTION_PREFETCH_ENABLE_DEFAULT = "false";
 
   /**
    * compression mode used by lucene for index writing, this conf will be passed to lucene writer
    * while writing index files.
    */
+  @CarbonProperty
   public static final String CARBON_LUCENE_COMPRESSION_MODE = "carbon.lucene.compression.mode";
 
   /**
@@ -975,16 +1135,38 @@ public final class CarbonCommonConstants {
    * make it less and not the index writing speed.
    */
   public static final String CARBON_LUCENE_COMPRESSION_MODE_DEFAULT = "speed";
+
   /**
    * The node loads the smallest amount of data
    */
   @CarbonProperty
   public static final String CARBON_LOAD_MIN_SIZE_INMB = "load_min_size_inmb";
+
+  /**
+   * the node minimum load data default value
+   */
   public static final String CARBON_LOAD_MIN_SIZE_INMB_DEFAULT = "0";
+
+  @CarbonProperty
+  public static final String IN_MEMORY_FOR_SORT_DATA_IN_MB = "sort.inmemory.size.inmb";
+
+  public static final String IN_MEMORY_FOR_SORT_DATA_IN_MB_DEFAULT = "1024";
+
   /**
-   *  the node minimum load data default value
+   * carbon sort size
    */
-  public static final int CARBON_LOAD_MIN_SIZE_DEFAULT = 256;
+  @CarbonProperty
+  public static final String SORT_SIZE = "carbon.sort.size";
+
+  /**
+   * default carbon sort size
+   */
+  public static final String SORT_SIZE_DEFAULT_VAL = "100000";
+
+  /**
+   * min carbon sort size
+   */
+  public static final int SORT_SIZE_MIN_VAL = 1000;
 
   //////////////////////////////////////////////////////////////////////////////////////////
   // Query parameter start here
@@ -993,25 +1175,31 @@ public final class CarbonCommonConstants {
   /**
    * set the segment ids to query from the table
    */
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_INPUT_SEGMENTS = "carbon.input.segments.";
+
   /**
    * ENABLE_QUERY_STATISTICS
    */
   @CarbonProperty
   public static final String ENABLE_QUERY_STATISTICS = "enable.query.statistics";
+
   /**
    * ENABLE_QUERY_STATISTICS_DEFAULT
    */
   public static final String ENABLE_QUERY_STATISTICS_DEFAULT = "false";
+
   /**
    * MAX_QUERY_EXECUTION_TIME
    */
   @CarbonProperty
   public static final String MAX_QUERY_EXECUTION_TIME = "max.query.execution.time";
+
   /**
    * MAX_QUERY_EXECUTION_TIME
    */
   public static final int DEFAULT_MAX_QUERY_EXECUTION_TIME = 60;
+
   /**
    * The batch size of records which returns to client.
    */
@@ -1019,21 +1207,25 @@ public final class CarbonCommonConstants {
   public static final String DETAIL_QUERY_BATCH_SIZE = "carbon.detail.batch.size";
 
   public static final int DETAIL_QUERY_BATCH_SIZE_DEFAULT = 100;
+
   /**
    * max driver lru cache size upto which lru cache will be loaded in memory
    */
   @CarbonProperty
   public static final String CARBON_MAX_DRIVER_LRU_CACHE_SIZE = "carbon.max.driver.lru.cache.size";
+
   /**
    * max executor lru cache size upto which lru cache will be loaded in memory
    */
   @CarbonProperty
   public static final String CARBON_MAX_EXECUTOR_LRU_CACHE_SIZE =
       "carbon.max.executor.lru.cache.size";
+
   /**
    * max lru cache size default value in MB
    */
   public static final String CARBON_MAX_LRU_CACHE_SIZE_DEFAULT = "-1";
+
   /**
    * property to enable min max during filter query
    */
@@ -1044,7 +1236,8 @@ public final class CarbonCommonConstants {
    * default value to enable min or max during filter query execution
    */
   public static final String MIN_MAX_DEFAULT_VALUE = "true";
-  @CarbonProperty
+
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String ENABLE_VECTOR_READER = "carbon.enable.vector.reader";
 
   public static final String ENABLE_VECTOR_READER_DEFAULT = "true";
@@ -1055,20 +1248,24 @@ public final class CarbonCommonConstants {
   @CarbonProperty
   public static final String IS_DRIVER_INSTANCE = "is.driver.instance";
 
+  public static final String IS_DRIVER_INSTANCE_DEFAULT = "false";
+
   /**
    * property for enabling unsafe based query processing
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String ENABLE_UNSAFE_IN_QUERY_EXECUTION = "enable.unsafe.in.query.processing";
 
   /**
    * default property of unsafe processing
    */
   public static final String ENABLE_UNSAFE_IN_QUERY_EXECUTION_DEFAULTVALUE = "false";
-  @CarbonProperty
+
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_CUSTOM_BLOCK_DISTRIBUTION = "carbon.custom.block.distribution";
+
   /**
-   * This property defines how the tasks are splitted/combined and launch spark tasks during query
+   * This property defines how the tasks are split/combined and launch spark tasks during query
    */
   @CarbonProperty
   public static final String CARBON_TASK_DISTRIBUTION = "carbon.task.distribution";
@@ -1097,36 +1294,45 @@ public final class CarbonCommonConstants {
    * Default task distribution.
    */
   public static final String CARBON_TASK_DISTRIBUTION_DEFAULT = CARBON_TASK_DISTRIBUTION_BLOCK;
+
   /**
    * this will be used to pass bitset value in filter to another filter for
    * faster execution of filter query
    */
+  @CarbonProperty
   public static final String BITSET_PIPE_LINE = "carbon.use.bitset.pipe.line";
+
   public static final String BITSET_PIPE_LINE_DEFAULT = "true";
+
   /**
    * minimum required registered resource for starting block distribution
    */
   @CarbonProperty
   public static final String CARBON_SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO =
-      "carbon.scheduler.minregisteredresourcesratio";
+      "carbon.scheduler.min.registered.resources.ratio";
+
   /**
    * default minimum required registered resource for starting block distribution
    */
   public static final String CARBON_SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO_DEFAULT = "0.8d";
+
   /**
    * minimum required registered resource for starting block distribution
    */
   public static final double CARBON_SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO_MIN = 0.1d;
+
   /**
    * max minimum required registered resource for starting block distribution
    */
   public static final double CARBON_SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO_MAX = 1.0d;
+
   /**
    * To define how much time scheduler should wait for the
    * resource in dynamic allocation.
    */
+  @CarbonProperty
   public static final String CARBON_DYNAMIC_ALLOCATION_SCHEDULER_TIMEOUT =
-      "carbon.dynamicallocation.schedulertimeout";
+      "carbon.dynamical.location.scheduler.timeout";
 
   /**
    * default scheduler wait time
@@ -1149,13 +1355,17 @@ public final class CarbonCommonConstants {
    * have been started
    */
   public static final int CARBON_DYNAMIC_ALLOCATION_SCHEDULER_THREAD_SLEEP_TIME = 250;
+
   /**
    * It allows queries on hive metastore directly along with filter information, otherwise first
    * fetches all partitions from hive and apply filters on it.
    */
+  @CarbonProperty
   public static final String CARBON_READ_PARTITION_HIVE_DIRECT =
       "carbon.read.partition.hive.direct";
+
   public static final String CARBON_READ_PARTITION_HIVE_DIRECT_DEFAULT = "true";
+
   /**
    * If the heap memory allocations of the given size is greater or equal than this value,
    * it should go through the pooling mechanism.
@@ -1166,14 +1376,16 @@ public final class CarbonCommonConstants {
   @CarbonProperty
   public static final String CARBON_HEAP_MEMORY_POOLING_THRESHOLD_BYTES =
       "carbon.heap.memory.pooling.threshold.bytes";
+
   public static final String CARBON_HEAP_MEMORY_POOLING_THRESHOLD_BYTES_DEFAULT = "1048576";
+
   /**
    * If set to true, will use CarbonReader to do distributed scan directly instead of using
    * compute framework like spark, thus avoiding limitation of compute framework like SQL
    * optimizer and task scheduling overhead.
    */
-  @CarbonProperty
   @InterfaceStability.Unstable
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_SEARCH_MODE_ENABLE = "carbon.search.enabled";
 
   public static final String CARBON_SEARCH_MODE_ENABLE_DEFAULT = "false";
@@ -1181,8 +1393,8 @@ public final class CarbonCommonConstants {
   /**
    * It's timeout threshold of carbon search query
    */
-  @CarbonProperty
   @InterfaceStability.Unstable
+  @CarbonProperty
   public static final String CARBON_SEARCH_QUERY_TIMEOUT = "carbon.search.query.timeout";
 
   /**
@@ -1194,8 +1406,8 @@ public final class CarbonCommonConstants {
    * The size of thread pool used for reading files in Work for search mode. By default,
    * it is number of cores in Worker
    */
-  @CarbonProperty
   @InterfaceStability.Unstable
+  @CarbonProperty
   public static final String CARBON_SEARCH_MODE_SCAN_THREAD = "carbon.search.scan.thread";
 
   /**
@@ -1203,8 +1415,8 @@ public final class CarbonCommonConstants {
    * If Master failed to start service with this port, it will try to increment the port number
    * and try to bind again, until it is success
    */
-  @CarbonProperty
   @InterfaceStability.Unstable
+  @CarbonProperty
   public static final String CARBON_SEARCH_MODE_MASTER_PORT = "carbon.search.master.port";
 
   public static final String CARBON_SEARCH_MODE_MASTER_PORT_DEFAULT = "10020";
@@ -1214,8 +1426,8 @@ public final class CarbonCommonConstants {
    * If Worker failed to start service with this port, it will try to increment the port number
    * and try to bind again, until it is success
    */
-  @CarbonProperty
   @InterfaceStability.Unstable
+  @CarbonProperty
   public static final String CARBON_SEARCH_MODE_WORKER_PORT = "carbon.search.worker.port";
 
   public static final String CARBON_SEARCH_MODE_WORKER_PORT_DEFAULT = "10021";
@@ -1227,11 +1439,23 @@ public final class CarbonCommonConstants {
    *
    * If user does not set this value, by default it is 10 * number of cores in Worker
    */
-  @CarbonProperty
   @InterfaceStability.Unstable
+  @CarbonProperty
   public static final String CARBON_SEARCH_MODE_WORKER_WORKLOAD_LIMIT =
       "carbon.search.worker.workload.limit";
 
+  /**
+   * When enabled complete row filters will be handled by carbon in case of vector.
+   * If it is disabled then only page level pruning will be done by carbon and row level filtering
+   * will be done by spark for vector.
+   * There is no change in flow for non-vector based queries.
+   */
+  @CarbonProperty(dynamicConfigurable = true)
+  public static final String CARBON_PUSH_ROW_FILTERS_FOR_VECTOR =
+      "carbon.push.rowfilters.for.vector";
+
+  public static final String CARBON_PUSH_ROW_FILTERS_FOR_VECTOR_DEFAULT = "false";
+
   //////////////////////////////////////////////////////////////////////////////////////////
   // Datamap parameter start here
   //////////////////////////////////////////////////////////////////////////////////////////
@@ -1241,45 +1465,55 @@ public final class CarbonCommonConstants {
    * that the query on 'dbName.table' will not use the datamap 'dmName'
    */
   @InterfaceStability.Unstable
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_DATAMAP_VISIBLE = "carbon.datamap.visible.";
 
   /**
    * Fetch and validate the segments.
    * Used for aggregate table load as segment validation is not required.
    */
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String VALIDATE_CARBON_INPUT_SEGMENTS = "validate.carbon.input.segments.";
 
   /**
    * Whether load/insert command is fired internally or by the user.
    * Used to block load/insert on pre-aggregate if fired by user
    */
+  @CarbonProperty
   public static final String IS_INTERNAL_LOAD_CALL = "is.internal.load.call";
+
+  public static final String IS_INTERNAL_LOAD_CALL_DEFAULT = "false";
+
+  @CarbonProperty
   public static final String USE_DISTRIBUTED_DATAMAP = "carbon.enable.distributed.datamap";
+
   public static final String USE_DISTRIBUTED_DATAMAP_DEFAULT = "false";
-  @CarbonProperty
+
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String SUPPORT_DIRECT_QUERY_ON_DATAMAP =
       "carbon.query.directQueryOnDataMap.enabled";
+
   public static final String SUPPORT_DIRECT_QUERY_ON_DATAMAP_DEFAULTVALUE = "false";
 
   @CarbonProperty
   public static final String VALIDATE_DIRECT_QUERY_ON_DATAMAP =
-      "carbon.query.validate.directqueryondatamap";
+      "carbon.query.validate.direct.query.on.datamap";
+
   public static final String VALIDATE_DIRECT_QUERY_ON_DATAMAP_DEFAULTVALUE = "true";
+
   @CarbonProperty
   public static final String CARBON_SHOW_DATAMAPS = "carbon.query.show.datamaps";
 
   public static final String CARBON_SHOW_DATAMAPS_DEFAULT = "true";
-  /**
-   * DMPROPERTY for Index DataMap, like lucene, bloomfilter DataMap,
-   * to indicate a list of column name to be indexed
-   */
-  public static final String INDEX_COLUMNS = "INDEX_COLUMNS";
 
   // Property to enable parallel datamap loading for a table
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_LOAD_DATAMAPS_PARALLEL = "carbon.load.datamaps.parallel.";
+
   // by default lucene will not store or create index for stop words like "is","the", if this
   // property is set to true lucene will index for stop words also and gives result for the filter
   // with stop words(example: TEXT_MATCH('description':'the'))
+  @CarbonProperty
   public static final String CARBON_LUCENE_INDEX_STOP_WORDS = "carbon.lucene.index.stop.words";
 
   public static final String CARBON_LUCENE_INDEX_STOP_WORDS_DEFAULT = "false";
@@ -1292,34 +1526,42 @@ public final class CarbonCommonConstants {
    * surrogate value of null
    */
   public static final int DICT_VALUE_NULL = 1;
+
   /**
    * surrogate value of null for direct dictionary
    */
   public static final int DIRECT_DICT_VALUE_NULL = 1;
+
   /**
    * integer size in bytes
    */
   public static final int INT_SIZE_IN_BYTE = 4;
+
   /**
    * short size in bytes
    */
   public static final int SHORT_SIZE_IN_BYTE = 2;
+
   /**
    * DOUBLE size in bytes
    */
   public static final int DOUBLE_SIZE_IN_BYTE = 8;
+
   /**
    * LONG size in bytes
    */
   public static final int LONG_SIZE_IN_BYTE = 8;
+
   /**
    * byte to KB conversion factor
    */
   public static final int BYTE_TO_KB_CONVERSION_FACTOR = 1024;
+
   /**
    * CARDINALITY_INCREMENT_DEFAULT_VALUE
    */
   public static final int CARDINALITY_INCREMENT_VALUE_DEFAULT_VAL = 10;
+
   /**
    * Load Folder Name
    */
@@ -1338,18 +1580,22 @@ public final class CarbonCommonConstants {
   public static final String S3N_PREFIX = "s3n://";
 
   public static final String S3A_PREFIX = "s3a://";
+
   /**
    * Access Key for s3n
    */
   public static final String S3N_ACCESS_KEY = "fs.s3n.awsAccessKeyId";
+
   /**
    * Secret Key for s3n
    */
   public static final String S3N_SECRET_KEY = "fs.s3n.awsSecretAccessKey";
+
   /**
    * Access Key for s3
    */
   public static final String S3_ACCESS_KEY = "fs.s3.awsAccessKeyId";
+
   /**
    * Secret Key for s3
    */
@@ -1360,40 +1606,49 @@ public final class CarbonCommonConstants {
    */
   @CarbonProperty
   public static final String FS_DEFAULT_FS = "fs.defaultFS";
+
   /**
    * Average constant
    */
   public static final String AVERAGE = "avg";
+
   /**
    * Count constant
    */
   public static final String COUNT = "count";
+
   /**
    * SUM
    */
   public static final String SUM = "sum";
+
   /**
    * MEMBER_DEFAULT_VAL
    */
   public static final String MEMBER_DEFAULT_VAL = "@NU#LL$!";
 
+
   /**
    * default charset to be used for reading and writing
    */
   public static final String DEFAULT_CHARSET = "UTF-8";
+
   /**
    * MEMBER_DEFAULT_VAL_ARRAY
    */
   public static final byte[] MEMBER_DEFAULT_VAL_ARRAY =
       MEMBER_DEFAULT_VAL.getBytes(Charset.forName(DEFAULT_CHARSET));
+
   /**
    * Empty byte array
    */
   public static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
+
   /**
    * FILE STATUS IN-PROGRESS
    */
   public static final String FILE_INPROGRESS_STATUS = ".inprogress";
+
   /**
    * FACT_FILE_EXT
    */
@@ -1408,14 +1663,17 @@ public final class CarbonCommonConstants {
    * UPDATE_DELTA_FILE_EXT
    */
   public static final String UPDATE_DELTA_FILE_EXT = FACT_FILE_EXT;
+
   /**
    * MERGERD_EXTENSION
    */
   public static final String MERGERD_EXTENSION = ".merge";
+
   /**
    * CSV_READ_COPIES
    */
   public static final String DEFAULT_NUMBER_CORES = "2";
+
   /**
    * CSV_FILE_EXTENSION
    */
@@ -1430,66 +1688,79 @@ public final class CarbonCommonConstants {
    * COLON_SPC_CHARACTER
    */
   public static final String COLON_SPC_CHARACTER = ":!@#COLON#@!:";
+
   /**
    * HASH_SPC_CHARATER
    */
   public static final String HASH_SPC_CHARACTER = "#!@:HASH:@!#";
+
   /**
-   * SEMICOLON_SPC_CHARATER
+   * SEMICOLON_SPC_CHARACTER
    */
   public static final String SEMICOLON_SPC_CHARACTER = ";#!@:SEMIC:@!#;";
+
   /**
-   * AMPERSAND_SPC_CHARATER
+   * AMPERSAND_SPC_CHARACTER
    */
   public static final String AMPERSAND_SPC_CHARACTER = "&#!@:AMPER:@!#&";
+
   /**
-   * ATTHERATE_SPC_CHARATER
+   * ATTHERATE_SPC_CHARACTER
    */
   public static final String COMA_SPC_CHARACTER = ",#!:COMA:!#,";
+
   /**
    * HYPHEN_SPC_CHARACTER
    */
   public static final String HYPHEN_SPC_CHARACTER = "-#!:HYPHEN:!#-";
+
   /**
    * SORT_TEMP_FILE_EXT
    */
   public static final String SORT_TEMP_FILE_EXT = ".sorttemp";
+
   /**
    * DEFAULT_COLLECTION_SIZE
    */
   public static final int DEFAULT_COLLECTION_SIZE = 16;
+
   /**
    * DIMENSION_SPLIT_VALUE_IN_COLUMNAR_DEFAULTVALUE
    */
   public static final String DIMENSION_SPLIT_VALUE_IN_COLUMNAR_DEFAULTVALUE = "1";
-  /**
-   * IS_FULLY_FILLED_BITS_DEFAULT_VALUE
-   */
+
   public static final String IS_FULLY_FILLED_BITS_DEFAULT_VALUE = "true";
+
   /**
    * CONSTANT_SIZE_TEN
    */
   public static final int CONSTANT_SIZE_TEN = 10;
+
   /**
    * COMMA
    */
   public static final String COMMA = ",";
+
   /**
    * UNDERSCORE
    */
   public static final String UNDERSCORE = "_";
+
   /**
    * POINT
    */
   public static final String POINT = ".";
+
   /**
    * Windows File separator
    */
   public static final String WINDOWS_FILE_SEPARATOR = "\\";
+
   /**
    * File separator
    */
   public static final String FILE_SEPARATOR = "/";
+
   /**
    * ARRAY separator
    */
@@ -1501,10 +1772,12 @@ public final class CarbonCommonConstants {
   public static final String STRUCT = "struct";
   public static final String MAP = "map";
   public static final String FROM = "from";
+
   /**
    * TABLE UPDATE STATUS FILENAME
    */
   public static final String TABLEUPDATESTATUS_FILENAME = "tableupdatestatus";
+
   /**
    * SPILL_OVER_DISK_PATH
    */
@@ -1514,14 +1787,17 @@ public final class CarbonCommonConstants {
   public static final String CARBON_IMPLICIT_COLUMN_TUPLEID = "tupleId";
   public static final String CARBON_IMPLICIT_COLUMN_SEGMENTID = "segId";
   public static final String POSITION_REFERENCE = "positionReference";
+
   /**
    * implicit column which will be added to each carbon table
    */
   public static final String POSITION_ID = "positionId";
+
   /**
    * TEMPWRITEFILEEXTENSION
    */
   public static final String TEMPWRITEFILEEXTENSION = ".write";
+
   /**
    * default charset class to be used for reading and writing
    */
@@ -1544,14 +1820,17 @@ public final class CarbonCommonConstants {
    * default load time of the segment
    */
   public static final long SEGMENT_LOAD_TIME_DEFAULT = -1;
+
   /**
    * default name of data base
    */
   public static final String DATABASE_DEFAULT_NAME = "default";
+
   /**
    * 16 mb size
    */
   public static final long CARBON_16MB = 16 * 1024 * 1024;
+
   /**
    * 256 mb size
    */
@@ -1576,11 +1855,13 @@ public final class CarbonCommonConstants {
    * S3LOCK TYPE
    */
   public static final String CARBON_LOCK_TYPE_S3 = "S3LOCK";
+
   /**
    * Invalid filter member log string
    */
   public static final String FILTER_INVALID_MEMBER =
       " Invalid Record(s) are present while filter evaluation. ";
+
   /**
    * default location of the carbon metastore db
    */
@@ -1605,10 +1886,12 @@ public final class CarbonCommonConstants {
    * Indicates alter partition
    */
   public static final String ALTER_PARTITION_KEY_WORD = "ALTER_PARTITION";
+
   /**
    * hdfs temporary directory key
    */
   public static final String HDFS_TEMP_LOCATION = "hadoop.tmp.dir";
+
   /**
    * File created in case of minor compaction request
    */
@@ -1618,10 +1901,12 @@ public final class CarbonCommonConstants {
    * File created in case of major compaction request
    */
   public static final String majorCompactionRequiredFile = "compactionRequired_major";
+
   /**
    * Compaction system level lock folder.
    */
   public static final String SYSTEM_LEVEL_COMPACTION_LOCK_FOLDER = "SystemCompactionLock";
+
   /**
    * Index file name will end with this extension when update.
    */
@@ -1659,15 +1944,19 @@ public final class CarbonCommonConstants {
   /**
    * data file version header
    */
+  @Deprecated
   public static final String CARBON_DATA_VERSION_HEADER = "CARBONDATAVERSION#";
+
   /**
    * Maximum no of column supported
    */
   public static final int DEFAULT_MAX_NUMBER_OF_COLUMNS = 20000;
-  public static final String MINOR = "minor";
 
+  public static final String MINOR = "minor";
   public static final String MAJOR = "major";
+
   public static final int DICTIONARY_DEFAULT_CARDINALITY = 1;
+
   /**
    * this will be used to provide comment for table
    */
@@ -1677,6 +1966,7 @@ public final class CarbonCommonConstants {
    * this will be used to provide comment for table
    */
   public static final String COLUMN_COMMENT = "comment";
+
   /*
    * The total size of carbon data
    */
@@ -1691,15 +1981,7 @@ public final class CarbonCommonConstants {
   public static final String TABLE_INDEX_SIZE = "Table Index Size";
 
   public static final String LAST_UPDATE_TIME = "Last Update Time";
-  /**
-   * the min handoff size of streaming segment, the unit is byte
-   */
-  public static final long HANDOFF_SIZE_MIN = 1024L * 1024 * 64;
 
-  /**
-   * the default handoff size of streaming segment, the unit is byte
-   */
-  public static final long HANDOFF_SIZE_DEFAULT = 1024L * 1024 * 1024;
   // As Short data type is used for storing the length of a column during data processing hence
   // the maximum characters that can be supported should be less than Short max value
   public static final int MAX_CHARS_PER_COLUMN_DEFAULT = 32000;
@@ -1710,32 +1992,20 @@ public final class CarbonCommonConstants {
   public static final int CARBON_MINMAX_ALLOWED_BYTE_COUNT_MAX = 1000;
 
   /**
-   * Written by detail to be written in carbondata footer for better maintanability
+   * Written by detail to be written in CarbonData footer for better maintanability
    */
   public static final String CARBON_WRITTEN_BY_FOOTER_INFO = "written_by";
 
   /**
-   * carbondata project version used while writing the carbondata file
+   * CarbonData project version used while writing the CarbonData file
    */
   public static final String CARBON_WRITTEN_VERSION = "version";
 
   /**
-   * property to set the appName of who is going to write the carbondata
+   * property to set the appName of who is going to write the CarbonData
    */
   public static final String CARBON_WRITTEN_BY_APPNAME = "carbon.writtenby.app.name";
 
-  /**
-   * When enabled complete row filters will be handled by carbon in case of vector.
-   * If it is disabled then only page level pruning will be done by carbon and row level filtering
-   * will be done by spark for vector.
-   * There is no change in flow for non-vector based queries.
-   */
-  @CarbonProperty
-  public static final String CARBON_PUSH_ROW_FILTERS_FOR_VECTOR =
-      "carbon.push.rowfilters.for.vector";
-
-  public static final String CARBON_PUSH_ROW_FILTERS_FOR_VECTOR_DEFAULT = "false";
-
   //////////////////////////////////////////////////////////////////////////////////////////
   // Unused constants and parameters start here
   //////////////////////////////////////////////////////////////////////////////////////////
@@ -1744,89 +2014,101 @@ public final class CarbonCommonConstants {
    * BYTE_ENCODING
    */
   public static final String BYTE_ENCODING = "ISO-8859-1";
+
   /**
    * measure meta data file name
    */
   public static final String MEASURE_METADATA_FILE_NAME = "/msrMetaData_";
+
   /**
    * DUMMY aggregation function
    */
   public static final String DUMMY = "dummy";
+
   /**
    * Bytes for string 0, it is used in codegen in case of null values.
    */
   public static final byte[] ZERO_BYTE_ARRAY = "0".getBytes(Charset.forName(DEFAULT_CHARSET));
+
   /**
    * HIERARCHY_FILE_EXTENSION
    */
   public static final String HIERARCHY_FILE_EXTENSION = ".hierarchy";
+
   /**
    * CARBON_RESULT_SIZE_DEFAULT
    */
   public static final String LEVEL_FILE_EXTENSION = ".level";
+
   /**
    * MEASUREMETADATA_FILE_EXT
    */
   public static final String MEASUREMETADATA_FILE_EXT = ".msrmetadata";
+
   /**
    * Comment for <code>TYPE_MYSQL</code>
    */
   public static final String TYPE_MYSQL = "MYSQL";
+
   /**
    * Comment for <code>TYPE_MSSQL</code>
    */
   public static final String TYPE_MSSQL = "MSSQL";
+
   /**
    * Comment for <code>TYPE_ORACLE</code>
    */
   public static final String TYPE_ORACLE = "ORACLE";
+
   /**
    * Comment for <code>TYPE_SYBASE</code>
    */
   public static final String TYPE_SYBASE = "SYBASE";
+
   /**
    * BAD_RECORD_KEY_VALUE
    */
   public static final String BAD_RECORD_KEY = "BADRECORD";
+
   /**
    * Default value of number of cores to be used for block sort
    */
   public static final String NUM_CORES_BLOCK_SORT_DEFAULT_VAL = "7";
+
   /**
    * Max value of number of cores to be used for block sort
    */
   public static final int NUM_CORES_BLOCK_SORT_MAX_VAL = 12;
+
   /**
    * Min value of number of cores to be used for block sort
    */
   public static final int NUM_CORES_BLOCK_SORT_MIN_VAL = 1;
+
   /**
    * LEVEL_METADATA_FILE
    */
   public static final String LEVEL_METADATA_FILE = "levelmetadata_";
+
   /**
    * DASH
    */
   public static final String DASH = "-";
+
   /**
    * FACT_UPDATE_EXTENSION.
    */
   public static final String FACT_UPDATE_EXTENSION = ".carbondata_update";
   public static final String FACT_DELETE_EXTENSION = "_delete";
+
   /**
    * MARKED_FOR_UPDATION
    */
   public static final String FACT_FILE_UPDATED = "update";
-  public static final String IN_MEMORY_FOR_SORT_DATA_IN_MB = "sort.inmemory.size.inmb";
-  public static final String IN_MEMORY_FOR_SORT_DATA_IN_MB_DEFAULT = "1024";
+
   /**
    * default value in size for cache size of bloom filter datamap.
    */
   public static final String CARBON_QUERY_DATAMAP_BLOOM_CACHE_SIZE_DEFAULT_VAL = "512";
-  /**
-   * carbon sort size
-   */
-  @Deprecated
-  @CarbonProperty
-  public static final String SORT_SIZE = "carbon.sort.size";
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
index 0d98cf4..46be8d8 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonLoadOptionConstants.java
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.core.constants;
 
-import org.apache.carbondata.core.util.CarbonProperty;
+import org.apache.carbondata.core.util.annotations.CarbonProperty;
 
 /**
  * Load options constant
@@ -26,76 +26,87 @@ public final class CarbonLoadOptionConstants {
   /**
    * option to enable and disable the logger
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE =
       "carbon.options.bad.records.logger.enable";
 
   public static String CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE_DEFAULT = "false";
+
   /**
    * property to pass the bad records action
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_OPTIONS_BAD_RECORDS_ACTION =
       "carbon.options.bad.records.action";
+
   /**
    * load option to specify weather empty data to be treated as bad record
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD =
       "carbon.options.is.empty.data.bad.record";
+
   public static final String CARBON_OPTIONS_IS_EMPTY_DATA_BAD_RECORD_DEFAULT = "false";
 
   /**
    * option to specify whether to skip empty lines in load
    */
-  @CarbonProperty public static final String CARBON_OPTIONS_SKIP_EMPTY_LINE =
+  @CarbonProperty(dynamicConfigurable = true)
+  public static final String CARBON_OPTIONS_SKIP_EMPTY_LINE =
       "carbon.options.is.empty.data.bad.record";
 
   /**
    * option to specify the dateFormat in load option for all date columns in table
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_OPTIONS_DATEFORMAT =
-      "carbon.options.dateformat";
+      "carbon.options.date.format";
+
   public static final String CARBON_OPTIONS_DATEFORMAT_DEFAULT = "";
 
   /**
    * option to specify the timestampFormat in load option for all timestamp columns in table
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_OPTIONS_TIMESTAMPFORMAT =
-          "carbon.options.timestampformat";
+          "carbon.options.timestamp.format";
+
   public static final String CARBON_OPTIONS_TIMESTAMPFORMAT_DEFAULT = "";
+
   /**
    * option to specify the sort_scope
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_OPTIONS_SORT_SCOPE =
       "carbon.options.sort.scope";
+
   /**
    * option to specify the batch sort size inmb
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_OPTIONS_BATCH_SORT_SIZE_INMB =
       "carbon.options.batch.sort.size.inmb";
+
   /**
    * Option to enable/ disable single_pass
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_OPTIONS_SINGLE_PASS =
       "carbon.options.single.pass";
+
   public static final String CARBON_OPTIONS_SINGLE_PASS_DEFAULT = "false";
 
   /**
    * specify bad record path option
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_OPTIONS_BAD_RECORD_PATH =
       "carbon.options.bad.record.path";
+
   /**
    * specify bad record path option
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_OPTIONS_GLOBAL_SORT_PARTITIONS =
       "carbon.options.global.sort.partitions";
 
@@ -103,7 +114,7 @@ public final class CarbonLoadOptionConstants {
    * specify serialization null format, it is used describe which character in side the csv file
    * is treated as null.
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   public static final String CARBON_OPTIONS_SERIALIZATION_NULL_FORMAT =
       "carbon.options.serialization.null.format";
 
@@ -123,6 +134,7 @@ public final class CarbonLoadOptionConstants {
   @CarbonProperty
   public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION
       = "carbon.load.skewedDataOptimization.enabled";
+
   public static final String ENABLE_CARBON_LOAD_SKEWED_DATA_OPTIMIZATION_DEFAULT = "false";
 
   /**
@@ -138,6 +150,7 @@ public final class CarbonLoadOptionConstants {
   @CarbonProperty
   public static final String ENABLE_CARBON_LOAD_DIRECT_WRITE_TO_STORE_PATH
       = "carbon.load.directWriteToStorePath.enabled";
+
   public static final String ENABLE_CARBON_LOAD_DIRECT_WRITE_TO_STORE_PATH_DEFAULT = "false";
 
   /**
@@ -150,5 +163,6 @@ public final class CarbonLoadOptionConstants {
   @CarbonProperty
   public static final String CARBON_LOAD_SORT_MEMORY_SPILL_PERCENTAGE
       = "carbon.load.sortmemory.spill.percentage";
+
   public static final String CARBON_LOAD_SORT_MEMORY_SPILL_PERCENTAGE_DEFAULT = "0";
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/core/src/main/java/org/apache/carbondata/core/constants/CarbonV3DataFormatConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonV3DataFormatConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonV3DataFormatConstants.java
index e888986..89c6adc 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonV3DataFormatConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonV3DataFormatConstants.java
@@ -16,7 +16,7 @@
  */
 package org.apache.carbondata.core.constants;
 
-import org.apache.carbondata.core.util.CarbonProperty;
+import org.apache.carbondata.core.util.annotations.CarbonProperty;
 
 /**
  * Constants for V3 data format
@@ -26,7 +26,7 @@ public interface CarbonV3DataFormatConstants {
   /**
    * each blocklet group size in mb
    */
-  @CarbonProperty
+  @CarbonProperty(dynamicConfigurable = true)
   String BLOCKLET_SIZE_IN_MB = "carbon.blockletgroup.size.in.mb";
 
   /**

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java
index b98ebb7..574746a 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockFactory.java
@@ -41,7 +41,7 @@ public class CarbonLockFactory {
   private static String lockTypeConfigured;
 
   private static String lockPath = CarbonProperties.getInstance()
-      .getProperty(CarbonCommonConstants.LOCK_PATH, "")
+      .getProperty(CarbonCommonConstants.LOCK_PATH, CarbonCommonConstants.LOCK_PATH_DEFAULT)
       .toLowerCase();
 
   static {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
index 3d86587..89ccbd0 100644
--- a/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/locks/CarbonLockUtil.java
@@ -125,7 +125,7 @@ public class CarbonLockUtil {
         CarbonProperties.getInstance().getSegmentLockFilesPreserveHours();
     AbsoluteTableIdentifier absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier();
     String lockFilesDir = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.LOCK_PATH, "");
+        .getProperty(CarbonCommonConstants.LOCK_PATH, CarbonCommonConstants.LOCK_PATH_DEFAULT);
     if (StringUtils.isEmpty(lockFilesDir)) {
       lockFilesDir = CarbonTablePath.getLockFilesDirPath(absoluteTableIdentifier.getTablePath());
     } else {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
index 58162da..d5c4833 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/HeapMemoryAllocator.java
@@ -41,7 +41,8 @@ public class HeapMemoryAllocator implements MemoryAllocator {
   public HeapMemoryAllocator() {
     poolingThresholdBytes = CarbonProperties.getInstance().getHeapMemoryPoolingThresholdBytes();
     boolean isDriver = Boolean.parseBoolean(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.IS_DRIVER_INSTANCE, "false"));
+        .getProperty(CarbonCommonConstants.IS_DRIVER_INSTANCE,
+            CarbonCommonConstants.IS_DRIVER_INSTANCE_DEFAULT));
     // if set 'poolingThresholdBytes' to -1 or the object creation call is in driver,
     // it should not go through the pooling mechanism.
     if (poolingThresholdBytes == -1 || isDriver) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java b/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
index 7ccbc3f..725f003 100644
--- a/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
+++ b/core/src/main/java/org/apache/carbondata/core/memory/UnsafeMemoryManager.java
@@ -49,7 +49,8 @@ public class UnsafeMemoryManager {
       // check if driver unsafe memory is configured and JVM process is in driver. In that case
       // initialize unsafe memory configured for driver
       boolean isDriver = Boolean.parseBoolean(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.IS_DRIVER_INSTANCE, "false"));
+          .getProperty(CarbonCommonConstants.IS_DRIVER_INSTANCE,
+              CarbonCommonConstants.IS_DRIVER_INSTANCE_DEFAULT));
       boolean initializedWithUnsafeDriverMemory = false;
       if (isDriver) {
         configuredWorkingMemorySize = CarbonProperties.getInstance()

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
index 191a33e..fd42822 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
@@ -35,6 +35,7 @@ import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
 import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
 import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
+import org.apache.carbondata.core.util.annotations.CarbonProperty;
 
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.BLOCKLET_SIZE;
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION;
@@ -706,9 +707,9 @@ public final class CarbonProperties {
    * memory
    */
   private void loadProperties() {
-    String property = System.getProperty("carbon.properties.filepath");
+    String property = System.getProperty(CarbonCommonConstants.CARBON_PROPERTIES_FILE_PATH);
     if (null == property) {
-      property = CarbonCommonConstants.CARBON_PROPERTIES_FILE_PATH;
+      property = CarbonCommonConstants.CARBON_PROPERTIES_FILE_PATH_DEFAULT;
     }
     File file = new File(property);
     LOGGER.info("Property file path: " + file.getAbsolutePath());
@@ -722,17 +723,22 @@ public final class CarbonProperties {
       }
     } catch (FileNotFoundException e) {
       LOGGER.error(
-          "The file: " + CarbonCommonConstants.CARBON_PROPERTIES_FILE_PATH + " does not exist");
+          "The file: " + FileFactory.getCarbonFile(CarbonCommonConstants
+              .CARBON_PROPERTIES_FILE_PATH_DEFAULT).getAbsolutePath()
+              + " does not exist");
     } catch (IOException e) {
       LOGGER.error(
-          "Error while reading the file: " + CarbonCommonConstants.CARBON_PROPERTIES_FILE_PATH);
+          "Error while reading the file: "
+              + FileFactory.getCarbonFile(CarbonCommonConstants
+              .CARBON_PROPERTIES_FILE_PATH_DEFAULT).getAbsolutePath());
     } finally {
       if (null != fis) {
         try {
           fis.close();
         } catch (IOException e) {
           LOGGER.error("Error while closing the file stream for file: "
-              + CarbonCommonConstants.CARBON_PROPERTIES_FILE_PATH);
+              + FileFactory.getCarbonFile(CarbonCommonConstants
+              .CARBON_PROPERTIES_FILE_PATH_DEFAULT).getAbsolutePath());
         }
       }
     }
@@ -1189,15 +1195,15 @@ public final class CarbonProperties {
    * @return boolean
    */
   public boolean isPersistUpdateDataset() {
-    String isPersistEnabled = getProperty(CarbonCommonConstants.isPersistEnabled,
-            CarbonCommonConstants.defaultValueIsPersistEnabled);
+    String isPersistEnabled = getProperty(CarbonCommonConstants.CARBON_UPDATE_PERSIST_ENABLE,
+            CarbonCommonConstants.CARBON_UPDATE_PERSIST_ENABLE_DEFAULT);
     boolean validatePersistEnabled = CarbonUtil.validateBoolean(isPersistEnabled);
     if (!validatePersistEnabled) {
-      LOGGER.warn("The " + CarbonCommonConstants.isPersistEnabled
+      LOGGER.warn("The " + CarbonCommonConstants.CARBON_UPDATE_PERSIST_ENABLE
           + " configuration value is invalid. It will use default value("
-          + CarbonCommonConstants.defaultValueIsPersistEnabled
+          + CarbonCommonConstants.CARBON_UPDATE_PERSIST_ENABLE_DEFAULT
           + ").");
-      isPersistEnabled = CarbonCommonConstants.defaultValueIsPersistEnabled;
+      isPersistEnabled = CarbonCommonConstants.CARBON_UPDATE_PERSIST_ENABLE_DEFAULT;
     }
     return isPersistEnabled.equalsIgnoreCase("true");
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/core/src/main/java/org/apache/carbondata/core/util/CarbonProperty.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperty.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperty.java
deleted file mode 100644
index 6d453a6..0000000
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperty.java
+++ /dev/null
@@ -1,28 +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.carbondata.core.util;
-
-import java.lang.annotation.Retention;
-import java.lang.annotation.RetentionPolicy;
-
-/**
- * CarbonProperty annotation
- */
-@Retention(RetentionPolicy.RUNTIME)
-public @interface CarbonProperty {
-}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java b/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
index 027e6cb..2181107 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
@@ -32,17 +32,7 @@ import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
 import org.apache.carbondata.core.datamap.DataMapStoreManager;
 import org.apache.carbondata.core.exception.InvalidConfigurationException;
 
-import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION;
-import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_MAJOR_COMPACTION_SIZE;
-import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_SEARCH_MODE_ENABLE;
-import static org.apache.carbondata.core.constants.CarbonCommonConstants.COMPACTION_SEGMENT_LEVEL_THRESHOLD;
-import static org.apache.carbondata.core.constants.CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE;
-import static org.apache.carbondata.core.constants.CarbonCommonConstants.ENABLE_OFFHEAP_SORT;
-import static org.apache.carbondata.core.constants.CarbonCommonConstants.ENABLE_UNSAFE_IN_QUERY_EXECUTION;
-import static org.apache.carbondata.core.constants.CarbonCommonConstants.ENABLE_UNSAFE_SORT;
-import static org.apache.carbondata.core.constants.CarbonCommonConstants.ENABLE_VECTOR_READER;
-import static org.apache.carbondata.core.constants.CarbonCommonConstants.NUM_CORES_COMPACTING;
-import static org.apache.carbondata.core.constants.CarbonCommonConstants.NUM_CORES_LOADING;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.*;
 import static org.apache.carbondata.core.constants.CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_ACTION;
 import static org.apache.carbondata.core.constants.CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORDS_LOGGER_ENABLE;
 import static org.apache.carbondata.core.constants.CarbonLoadOptionConstants.CARBON_OPTIONS_BAD_RECORD_PATH;
@@ -167,6 +157,7 @@ public class SessionParams implements Serializable, Cloneable {
       case ENABLE_VECTOR_READER:
       case ENABLE_UNSAFE_IN_QUERY_EXECUTION:
       case ENABLE_AUTO_LOAD_MERGE:
+      case CARBON_PUSH_ROW_FILTERS_FOR_VECTOR:
         isValid = CarbonUtil.validateBoolean(value);
         if (!isValid) {
           throw new InvalidConfigurationException("Invalid value " + value + " for key " + key);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/7c1d68ed/core/src/main/java/org/apache/carbondata/core/util/annotations/CarbonProperty.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/util/annotations/CarbonProperty.java b/core/src/main/java/org/apache/carbondata/core/util/annotations/CarbonProperty.java
new file mode 100644
index 0000000..a859572
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/util/annotations/CarbonProperty.java
@@ -0,0 +1,37 @@
+/*
+ * 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.carbondata.core.util.annotations;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * Carbon property that can be dynamic configure
+ * it can be used set command to configure in beeline
+ */
+@Retention(RetentionPolicy.RUNTIME)
+public @interface CarbonProperty {
+  /**
+   * default value is false, it means this property isn't dynamic configurable.
+   * if set the value as true, it means this property can be dynamic configurable,
+   * but still need support it when validate key and value
+   *
+   * @return
+   */
+  boolean dynamicConfigurable() default false;
+}