carbondata-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ravipes...@apache.org
Subject [34/50] [abbrv] carbondata git commit: [CARBONDATA-1929][Validation]carbon property configuration validation + Fixed test case
Date Tue, 09 Jan 2018 04:02:02 GMT
[CARBONDATA-1929][Validation]carbon property configuration validation + Fixed test case

Added validation for below parameter:
carbon.timestamp.format
carbon.date.format
carbon.sort.file.write.buffer.size (minValue = 10 KB, maxValue=10MB, defaultValue =16 KB )
carbon.sort.intermediate.files.limit (minValue = 2, maxValue=50, defaultValue =20 )

This closes #1718


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

Branch: refs/heads/branch-1.3
Commit: bcf3ca3feda544dcbc1b5c096b98369c8a27f4e3
Parents: 7a4bd22
Author: mohammadshahidkhan <mohdshahidkhan1987@gmail.com>
Authored: Fri Dec 22 16:52:44 2017 +0530
Committer: manishgupta88 <tomanishgupta18@gmail.com>
Committed: Fri Jan 5 10:36:32 2018 +0530

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |   6 +
 .../carbondata/core/util/CarbonProperties.java  | 318 ++++++++++++++-----
 .../core/CarbonPropertiesValidationTest.java    |  66 +++-
 .../testsuite/datamap/DataMapWriterSuite.scala  |  20 +-
 4 files changed, 318 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcf3ca3f/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 2d1e4f9..b58f42a 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
@@ -297,6 +297,10 @@ public final class CarbonCommonConstants {
    * SORT_INTERMEDIATE_FILES_LIMIT_DEFAULT_VALUE
    */
   public static final String SORT_INTERMEDIATE_FILES_LIMIT_DEFAULT_VALUE = "20";
+
+  public static final int SORT_INTERMEDIATE_FILES_LIMIT_MIN = 2;
+
+  public static final int SORT_INTERMEDIATE_FILES_LIMIT_MAX = 50;
   /**
    * BAD_RECORD_KEY_VALUE
    */
@@ -325,6 +329,8 @@ public final class CarbonCommonConstants {
   @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
    */

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcf3ca3f/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 8042cfa..a918611 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
@@ -22,6 +22,7 @@ import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.reflect.Field;
+import java.text.SimpleDateFormat;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
@@ -34,6 +35,27 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants;
 import org.apache.carbondata.core.constants.CarbonLoadOptionConstants;
 import org.apache.carbondata.core.constants.CarbonV3DataFormatConstants;
 import org.apache.carbondata.core.metadata.ColumnarFormatVersion;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.BLOCKLET_SIZE;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_COMBINE_SMALL_INPUT_FILES;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_DATA_FILE_VERSION;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_DATE_FORMAT;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_EXECUTOR_STARTUP_TIMEOUT;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_SORT_FILE_WRITE_BUFFER_SIZE;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.CSV_READ_BUFFER_SIZE;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.ENABLE_AUTO_HANDOFF;
+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.HANDOFF_SIZE;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.LOCK_TYPE;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.NUM_CORES;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.NUM_CORES_BLOCK_SORT;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.SORT_SIZE;
+import static org.apache.carbondata.core.constants.CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB;
+import static org.apache.carbondata.core.constants.CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO;
 
 import org.apache.hadoop.conf.Configuration;
 
@@ -81,16 +103,87 @@ public final class CarbonProperties {
   }
 
   /**
+   * This method is to validate only a specific key added to carbonProperties using addProperty
+   *
+   * @param key
+   */
+  private void validateAndLoadDefaultProperties(String key) {
+    switch (key) {
+      case BLOCKLET_SIZE:
+        validateBlockletSize();
+        break;
+      case NUM_CORES:
+        validateNumCores();
+        break;
+      case NUM_CORES_BLOCK_SORT:
+        validateNumCoresBlockSort();
+        break;
+      case SORT_SIZE:
+        validateSortSize();
+        break;
+      case CARBON_DATA_FILE_VERSION:
+        validateCarbonDataFileVersion();
+        break;
+      case CARBON_EXECUTOR_STARTUP_TIMEOUT:
+        validateExecutorStartUpTime();
+        break;
+      case CARBON_PREFETCH_BUFFERSIZE:
+        validatePrefetchBufferSize();
+        break;
+      case BLOCKLET_SIZE_IN_MB:
+        validateBlockletGroupSizeInMB();
+        break;
+      case NUMBER_OF_COLUMN_TO_READ_IN_IO:
+        validateNumberOfColumnPerIORead();
+        break;
+      case ENABLE_UNSAFE_SORT:
+        validateEnableUnsafeSort();
+        break;
+      case CARBON_CUSTOM_BLOCK_DISTRIBUTION:
+        validateCustomBlockDistribution();
+        break;
+      case ENABLE_VECTOR_READER:
+        validateEnableVectorReader();
+        break;
+      case CSV_READ_BUFFER_SIZE:
+        validateCarbonCSVReadBufferSizeByte();
+        break;
+      case HANDOFF_SIZE:
+        validateHandoffSize();
+        break;
+      case CARBON_COMBINE_SMALL_INPUT_FILES:
+        validateCombineSmallInputFiles();
+        break;
+      // The method validate the validity of configured carbon.timestamp.format value
+      // and reset to default value if validation fail
+      case CARBON_TIMESTAMP_FORMAT:
+        validateTimeFormatKey(CARBON_TIMESTAMP_FORMAT,
+            CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
+        break;
+      // The method validate the validity of configured carbon.date.format value
+      // and reset to default value if validation fail
+      case CARBON_DATE_FORMAT:
+        validateTimeFormatKey(CARBON_DATE_FORMAT, CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT);
+        break;
+      case CARBON_SORT_FILE_WRITE_BUFFER_SIZE:
+        validateSortFileWriteBufferSize();
+        break;
+      case SORT_INTERMEDIATE_FILES_LIMIT:
+        validateSortIntermediateFilesLimit();
+        break;
+      case ENABLE_AUTO_HANDOFF:
+        validateHandoffSize();
+        break;
+      // TODO : Validation for carbon.lock.type should be handled for addProperty flow
+      default:
+        // none
+    }
+  }
+  /**
    * This method validates the loaded properties and loads default
    * values in case of wrong values.
    */
   private void validateAndLoadDefaultProperties() {
-    try {
-      initPropertySet();
-    } catch (IllegalAccessException e) {
-      LOGGER.error("Illelagal access to declared field" + e.getMessage());
-    }
-
     validateBlockletSize();
     validateNumCores();
     validateNumCoresBlockSort();
@@ -107,40 +200,107 @@ public final class CarbonProperties {
     validateCarbonCSVReadBufferSizeByte();
     validateHandoffSize();
     validateCombineSmallInputFiles();
+    // The method validate the validity of configured carbon.timestamp.format value
+    // and reset to default value if validation fail
+    validateTimeFormatKey(CARBON_TIMESTAMP_FORMAT,
+        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
+    // The method validate the validity of configured carbon.date.format value
+    // and reset to default value if validation fail
+    validateTimeFormatKey(CARBON_DATE_FORMAT,
+        CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT);
+    validateSortFileWriteBufferSize();
+    validateSortIntermediateFilesLimit();
     validateEnableAutoHandoff();
   }
 
-  private void validateCarbonCSVReadBufferSizeByte() {
-    String csvReadBufferSizeStr =
-        carbonProperties.getProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE);
-    if (null != csvReadBufferSizeStr) {
+  /**
+   * Sort intermediate file size validation and if not valid then reset to the default value
+   */
+  private void validateSortIntermediateFilesLimit() {
+    validateRange(SORT_INTERMEDIATE_FILES_LIMIT,
+        CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT_DEFAULT_VALUE,
+        CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT_MIN,
+        CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT_MAX);
+  }
+
+  /**
+   *
+   * @param key
+   * @param defaultValue default value for the given key
+   * @param minValue Minimum value for the given key
+   * @param maxValue Max value for the given key
+   */
+  private void validateRange(String key, String defaultValue, int minValue, int maxValue)
{
+    String fileBufferSize = carbonProperties
+        .getProperty(key, defaultValue);
+    if (null != fileBufferSize) {
       try {
-        int bufferSize = Integer.parseInt(csvReadBufferSizeStr);
-        if (bufferSize < CarbonCommonConstants.CSV_READ_BUFFER_SIZE_MIN
-            || bufferSize > CarbonCommonConstants.CSV_READ_BUFFER_SIZE_MAX) {
-          LOGGER.warn("The value \"" + csvReadBufferSizeStr + "\" configured for key "
-              + CarbonCommonConstants.CSV_READ_BUFFER_SIZE
+        int bufferSize = Integer.parseInt(fileBufferSize);
+
+        if (bufferSize < minValue
+            || bufferSize > maxValue) {
+          LOGGER.warn("The value \"" + fileBufferSize + "\" configured for key "
+              + key
               + "\" is not in range. Valid range is (byte) \""
-              + CarbonCommonConstants.CSV_READ_BUFFER_SIZE_MIN + " to \""
-              + CarbonCommonConstants.CSV_READ_BUFFER_SIZE_MAX + ". Using the default value
\""
-              + CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT);
-          carbonProperties.setProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE,
-              CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT);
+              + minValue + " to \""
+              + maxValue +
+              ". Using the default value \""
+              + defaultValue);
+          carbonProperties.setProperty(key,
+              defaultValue);
         }
       } catch (NumberFormatException nfe) {
-        LOGGER.warn("The value \"" + csvReadBufferSizeStr + "\" configured for key "
-            + CarbonCommonConstants.CSV_READ_BUFFER_SIZE
+        LOGGER.warn("The value \"" + fileBufferSize + "\" configured for key "
+            + key
             + "\" is invalid. Using the default value \""
-            + CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT);
-        carbonProperties.setProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE,
-            CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT);
+            + defaultValue);
+        carbonProperties.setProperty(key,
+            defaultValue);
       }
     }
   }
 
+  /**
+   * validate carbon.sort.file.write.buffer.size and if not valid then reset to the default
value
+   */
+  private void validateSortFileWriteBufferSize() {
+    validateRange(CARBON_SORT_FILE_WRITE_BUFFER_SIZE,
+        CarbonCommonConstants.CARBON_SORT_FILE_WRITE_BUFFER_SIZE_DEFAULT_VALUE,
+        CarbonCommonConstants.CARBON_SORT_FILE_WRITE_BUFFER_SIZE_MIN,
+        CarbonCommonConstants.CARBON_SORT_FILE_WRITE_BUFFER_SIZE_MAX);
+  }
+
+  /**
+   * The method validate the validity of configured carbon.date.format value
+   * and reset to default value if validation fail
+   */
+  private void validateTimeFormatKey(String key, String defaultValue) {
+    String dateFormat = carbonProperties
+        .getProperty(key, defaultValue);
+    try {
+      new SimpleDateFormat(dateFormat);
+    } catch (Exception e) {
+      LOGGER.warn("The value \"" + dateFormat + "\" configured for key "
+          + key
+          + "\" is invalid. Using the default value \""
+          + key);
+      carbonProperties.setProperty(key, defaultValue);
+    }
+  }
+
+  /**
+   * The method value csv read buffer size and if not valid then reset to the default value
+   */
+  private void validateCarbonCSVReadBufferSizeByte() {
+    validateRange(CSV_READ_BUFFER_SIZE,
+        CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT,
+        CarbonCommonConstants.CSV_READ_BUFFER_SIZE_MIN,
+        CarbonCommonConstants.CSV_READ_BUFFER_SIZE_MAX);
+  }
+
   private void validateLockType() {
     String lockTypeConfigured = carbonProperties
-        .getProperty(CarbonCommonConstants.LOCK_TYPE, CarbonCommonConstants.LOCK_TYPE_DEFAULT);
+        .getProperty(LOCK_TYPE, CarbonCommonConstants.LOCK_TYPE_DEFAULT);
     switch (lockTypeConfigured.toUpperCase()) {
       // if user is setting the lock type as CARBON_LOCK_TYPE_ZOOKEEPER then no need to validate
       // else validate based on the file system type for LOCAL file system lock will be
@@ -167,67 +327,67 @@ public final class CarbonProperties {
         .startsWith(CarbonCommonConstants.ALLUXIOURL_PREFIX))
         && !CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS.equalsIgnoreCase(lockTypeConfigured))
{
       LOGGER.warn("The value \"" + lockTypeConfigured + "\" configured for key "
-          + CarbonCommonConstants.LOCK_TYPE + " is invalid for current file system. "
+          + LOCK_TYPE + " is invalid for current file system. "
           + "Use the default value " + CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS + " instead.");
-      carbonProperties.setProperty(CarbonCommonConstants.LOCK_TYPE,
+      carbonProperties.setProperty(LOCK_TYPE,
           CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS);
     } else if (null != defaultFs && defaultFs.startsWith(CarbonCommonConstants.LOCAL_FILE_PREFIX)
         && !CarbonCommonConstants.CARBON_LOCK_TYPE_LOCAL.equalsIgnoreCase(lockTypeConfigured))
{
-      carbonProperties.setProperty(CarbonCommonConstants.LOCK_TYPE,
+      carbonProperties.setProperty(LOCK_TYPE,
           CarbonCommonConstants.CARBON_LOCK_TYPE_LOCAL);
       LOGGER.warn("The value \"" + lockTypeConfigured + "\" configured for key "
-          + CarbonCommonConstants.LOCK_TYPE + " is invalid for current file system. "
+          + LOCK_TYPE + " is invalid for current file system. "
           + "Use the default value " + CarbonCommonConstants.CARBON_LOCK_TYPE_LOCAL + " instead.");
     }
   }
 
   private void validateEnableVectorReader() {
     String vectorReaderStr =
-        carbonProperties.getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER);
+        carbonProperties.getProperty(ENABLE_VECTOR_READER);
     boolean isValidBooleanValue = CarbonUtil.validateBoolean(vectorReaderStr);
     if (!isValidBooleanValue) {
       LOGGER.warn("The enable vector reader value \"" + vectorReaderStr
           + "\" is invalid. Using the default value \""
           + CarbonCommonConstants.ENABLE_VECTOR_READER_DEFAULT);
-      carbonProperties.setProperty(CarbonCommonConstants.ENABLE_VECTOR_READER,
+      carbonProperties.setProperty(ENABLE_VECTOR_READER,
           CarbonCommonConstants.ENABLE_VECTOR_READER_DEFAULT);
     }
   }
 
   private void validateCustomBlockDistribution() {
     String customBlockDistributionStr =
-        carbonProperties.getProperty(CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION);
+        carbonProperties.getProperty(CARBON_CUSTOM_BLOCK_DISTRIBUTION);
     boolean isValidBooleanValue = CarbonUtil.validateBoolean(customBlockDistributionStr);
     if (!isValidBooleanValue) {
       LOGGER.warn("The custom block distribution value \"" + customBlockDistributionStr
           + "\" is invalid. Using the default value \""
           + CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION_DEFAULT);
-      carbonProperties.setProperty(CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION,
+      carbonProperties.setProperty(CARBON_CUSTOM_BLOCK_DISTRIBUTION,
           CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION_DEFAULT);
     }
   }
 
   private void validateCombineSmallInputFiles() {
     String combineSmallInputFilesStr =
-        carbonProperties.getProperty(CarbonCommonConstants.CARBON_COMBINE_SMALL_INPUT_FILES);
+        carbonProperties.getProperty(CARBON_COMBINE_SMALL_INPUT_FILES);
     boolean isValidBooleanValue = CarbonUtil.validateBoolean(combineSmallInputFilesStr);
     if (!isValidBooleanValue) {
       LOGGER.warn("The combine small files value \"" + combineSmallInputFilesStr
           + "\" is invalid. Using the default value \""
           + CarbonCommonConstants.CARBON_COMBINE_SMALL_INPUT_FILES_DEFAULT);
-      carbonProperties.setProperty(CarbonCommonConstants.CARBON_COMBINE_SMALL_INPUT_FILES,
+      carbonProperties.setProperty(CARBON_COMBINE_SMALL_INPUT_FILES,
           CarbonCommonConstants.CARBON_COMBINE_SMALL_INPUT_FILES_DEFAULT);
     }
   }
 
   private void validateEnableUnsafeSort() {
-    String unSafeSortStr = carbonProperties.getProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT);
+    String unSafeSortStr = carbonProperties.getProperty(ENABLE_UNSAFE_SORT);
     boolean isValidBooleanValue = CarbonUtil.validateBoolean(unSafeSortStr);
     if (!isValidBooleanValue) {
       LOGGER.warn("The enable unsafe sort value \"" + unSafeSortStr
           + "\" is invalid. Using the default value \""
           + CarbonCommonConstants.ENABLE_UNSAFE_SORT_DEFAULT);
-      carbonProperties.setProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT,
+      carbonProperties.setProperty(ENABLE_UNSAFE_SORT,
           CarbonCommonConstants.ENABLE_UNSAFE_SORT_DEFAULT);
     }
   }
@@ -255,10 +415,10 @@ public final class CarbonProperties {
 
   private void validatePrefetchBufferSize() {
     String prefetchBufferSizeStr =
-        carbonProperties.getProperty(CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE);
+        carbonProperties.getProperty(CARBON_PREFETCH_BUFFERSIZE);
 
     if (null == prefetchBufferSizeStr || prefetchBufferSizeStr.length() == 0) {
-      carbonProperties.setProperty(CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE,
+      carbonProperties.setProperty(CARBON_PREFETCH_BUFFERSIZE,
           CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE_DEFAULT);
     } else {
       try {
@@ -267,16 +427,16 @@ public final class CarbonProperties {
         LOGGER.info("The prefetch buffer size value \"" + prefetchBufferSizeStr
             + "\" is invalid. Using the default value \""
             + CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE_DEFAULT + "\"");
-        carbonProperties.setProperty(CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE,
+        carbonProperties.setProperty(CARBON_PREFETCH_BUFFERSIZE,
             CarbonCommonConstants.CARBON_PREFETCH_BUFFERSIZE_DEFAULT);
       }
     }
   }
 
   private void validateHandoffSize() {
-    String handoffSizeStr = carbonProperties.getProperty(CarbonCommonConstants.HANDOFF_SIZE);
+    String handoffSizeStr = carbonProperties.getProperty(HANDOFF_SIZE);
     if (null == handoffSizeStr || handoffSizeStr.length() == 0) {
-      carbonProperties.setProperty(CarbonCommonConstants.HANDOFF_SIZE,
+      carbonProperties.setProperty(HANDOFF_SIZE,
           "" + CarbonCommonConstants.HANDOFF_SIZE_DEFAULT);
     } else {
       try {
@@ -285,14 +445,14 @@ public final class CarbonProperties {
           LOGGER.info("The streaming segment max size configured value " + handoffSizeStr
+
               " is invalid. Using the default value "
               + CarbonCommonConstants.HANDOFF_SIZE_DEFAULT);
-          carbonProperties.setProperty(CarbonCommonConstants.HANDOFF_SIZE,
+          carbonProperties.setProperty(HANDOFF_SIZE,
               "" + CarbonCommonConstants.HANDOFF_SIZE_DEFAULT);
         }
       } catch (NumberFormatException e) {
         LOGGER.info("The streaming segment max size value \"" + handoffSizeStr
             + "\" is invalid. Using the default value \""
             + CarbonCommonConstants.HANDOFF_SIZE_DEFAULT + "\"");
-        carbonProperties.setProperty(CarbonCommonConstants.HANDOFF_SIZE,
+        carbonProperties.setProperty(HANDOFF_SIZE,
             "" + CarbonCommonConstants.HANDOFF_SIZE_DEFAULT);
       }
     }
@@ -300,13 +460,13 @@ public final class CarbonProperties {
 
   private void validateEnableAutoHandoff() {
     String enableAutoHandoffStr =
-        carbonProperties.getProperty(CarbonCommonConstants.ENABLE_AUTO_HANDOFF);
+        carbonProperties.getProperty(ENABLE_AUTO_HANDOFF);
     boolean isValid = CarbonUtil.validateBoolean(enableAutoHandoffStr);
     if (!isValid) {
       LOGGER.warn("The enable auto handoff value \"" + enableAutoHandoffStr
           + "\" is invalid. Using the default value \""
           + CarbonCommonConstants.ENABLE_AUTO_HANDOFF_DEFAULT);
-      carbonProperties.setProperty(CarbonCommonConstants.ENABLE_AUTO_HANDOFF,
+      carbonProperties.setProperty(ENABLE_AUTO_HANDOFF,
           CarbonCommonConstants.ENABLE_AUTO_HANDOFF_DEFAULT);
     }
   }
@@ -316,7 +476,7 @@ public final class CarbonProperties {
    */
   private void validateBlockletGroupSizeInMB() {
     String numberOfPagePerBlockletColumnString = carbonProperties
-        .getProperty(CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB,
+        .getProperty(BLOCKLET_SIZE_IN_MB,
             CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE);
     try {
       short numberOfPagePerBlockletColumn = Short.parseShort(numberOfPagePerBlockletColumnString);
@@ -324,18 +484,18 @@ public final class CarbonProperties {
         LOGGER.info("Blocklet Size Configured value \"" + numberOfPagePerBlockletColumnString
             + "\" is invalid. Using the default value \""
             + CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE);
-        carbonProperties.setProperty(CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB,
+        carbonProperties.setProperty(BLOCKLET_SIZE_IN_MB,
             CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE);
       }
     } catch (NumberFormatException e) {
       LOGGER.info("Blocklet Size Configured value \"" + numberOfPagePerBlockletColumnString
           + "\" is invalid. Using the default value \""
           + CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE);
-      carbonProperties.setProperty(CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB,
+      carbonProperties.setProperty(BLOCKLET_SIZE_IN_MB,
           CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE);
     }
     LOGGER.info("Blocklet Size Configured value is \"" + carbonProperties
-        .getProperty(CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB,
+        .getProperty(BLOCKLET_SIZE_IN_MB,
             CarbonV3DataFormatConstants.BLOCKLET_SIZE_IN_MB_DEFAULT_VALUE));
   }
 
@@ -344,7 +504,7 @@ public final class CarbonProperties {
    */
   private void validateNumberOfColumnPerIORead() {
     String numberofColumnPerIOString = carbonProperties
-        .getProperty(CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO,
+        .getProperty(NUMBER_OF_COLUMN_TO_READ_IN_IO,
             CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO_DEFAULTVALUE);
     try {
       short numberofColumnPerIO = Short.parseShort(numberofColumnPerIOString);
@@ -353,14 +513,14 @@ public final class CarbonProperties {
         LOGGER.info("The Number Of pages per blocklet column value \"" + numberofColumnPerIOString
             + "\" is invalid. Using the default value \""
             + CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO_DEFAULTVALUE);
-        carbonProperties.setProperty(CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO,
+        carbonProperties.setProperty(NUMBER_OF_COLUMN_TO_READ_IN_IO,
             CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO_DEFAULTVALUE);
       }
     } catch (NumberFormatException e) {
       LOGGER.info("The Number Of pages per blocklet column value \"" + numberofColumnPerIOString
           + "\" is invalid. Using the default value \""
           + CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO_DEFAULTVALUE);
-      carbonProperties.setProperty(CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO,
+      carbonProperties.setProperty(NUMBER_OF_COLUMN_TO_READ_IN_IO,
           CarbonV3DataFormatConstants.NUMBER_OF_COLUMN_TO_READ_IN_IO_DEFAULTVALUE);
     }
   }
@@ -369,7 +529,7 @@ public final class CarbonProperties {
    * This method validates the blocklet size
    */
   private void validateBlockletSize() {
-    String blockletSizeStr = carbonProperties.getProperty(CarbonCommonConstants.BLOCKLET_SIZE,
+    String blockletSizeStr = carbonProperties.getProperty(BLOCKLET_SIZE,
         CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL);
     try {
       int blockletSize = Integer.parseInt(blockletSizeStr);
@@ -379,14 +539,14 @@ public final class CarbonProperties {
         LOGGER.info("The blocklet size value \"" + blockletSizeStr
             + "\" is invalid. Using the default value \""
             + CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL);
-        carbonProperties.setProperty(CarbonCommonConstants.BLOCKLET_SIZE,
+        carbonProperties.setProperty(BLOCKLET_SIZE,
             CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL);
       }
     } catch (NumberFormatException e) {
       LOGGER.info("The blocklet size value \"" + blockletSizeStr
           + "\" is invalid. Using the default value \""
           + CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL);
-      carbonProperties.setProperty(CarbonCommonConstants.BLOCKLET_SIZE,
+      carbonProperties.setProperty(BLOCKLET_SIZE,
           CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL);
     }
   }
@@ -396,7 +556,7 @@ public final class CarbonProperties {
    */
   private void validateNumCores() {
     String numCoresStr = carbonProperties
-        .getProperty(CarbonCommonConstants.NUM_CORES, CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
+        .getProperty(NUM_CORES, CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
     try {
       int numCores = Integer.parseInt(numCoresStr);
 
@@ -405,14 +565,14 @@ public final class CarbonProperties {
         LOGGER.info(
             "The num Cores  value \"" + numCoresStr + "\" is invalid. Using the default value
\""
                 + CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
-        carbonProperties.setProperty(CarbonCommonConstants.NUM_CORES,
+        carbonProperties.setProperty(NUM_CORES,
             CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
       }
     } catch (NumberFormatException e) {
       LOGGER.info(
           "The num Cores  value \"" + numCoresStr + "\" is invalid. Using the default value
\""
               + CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
-      carbonProperties.setProperty(CarbonCommonConstants.NUM_CORES,
+      carbonProperties.setProperty(NUM_CORES,
           CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
     }
   }
@@ -421,7 +581,7 @@ public final class CarbonProperties {
    * This method validates the number cores specified for mdk block sort
    */
   private void validateNumCoresBlockSort() {
-    String numCoresStr = carbonProperties.getProperty(CarbonCommonConstants.NUM_CORES_BLOCK_SORT,
+    String numCoresStr = carbonProperties.getProperty(NUM_CORES_BLOCK_SORT,
         CarbonCommonConstants.NUM_CORES_BLOCK_SORT_DEFAULT_VAL);
     try {
       int numCores = Integer.parseInt(numCoresStr);
@@ -431,14 +591,14 @@ public final class CarbonProperties {
         LOGGER.info("The num cores value \"" + numCoresStr
             + "\" for block sort is invalid. Using the default value \""
             + CarbonCommonConstants.NUM_CORES_BLOCK_SORT_DEFAULT_VAL);
-        carbonProperties.setProperty(CarbonCommonConstants.NUM_CORES_BLOCK_SORT,
+        carbonProperties.setProperty(NUM_CORES_BLOCK_SORT,
             CarbonCommonConstants.NUM_CORES_BLOCK_SORT_DEFAULT_VAL);
       }
     } catch (NumberFormatException e) {
       LOGGER.info("The num cores value \"" + numCoresStr
           + "\" for block sort is invalid. Using the default value \""
           + CarbonCommonConstants.NUM_CORES_BLOCK_SORT_DEFAULT_VAL);
-      carbonProperties.setProperty(CarbonCommonConstants.NUM_CORES_BLOCK_SORT,
+      carbonProperties.setProperty(NUM_CORES_BLOCK_SORT,
           CarbonCommonConstants.NUM_CORES_BLOCK_SORT_DEFAULT_VAL);
     }
   }
@@ -448,7 +608,7 @@ public final class CarbonProperties {
    */
   private void validateSortSize() {
     String sortSizeStr = carbonProperties
-        .getProperty(CarbonCommonConstants.SORT_SIZE, CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL);
+        .getProperty(SORT_SIZE, CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL);
     try {
       int sortSize = Integer.parseInt(sortSizeStr);
 
@@ -456,14 +616,14 @@ public final class CarbonProperties {
         LOGGER.info(
             "The batch size value \"" + sortSizeStr + "\" is invalid. Using the default value
\""
                 + CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL);
-        carbonProperties.setProperty(CarbonCommonConstants.SORT_SIZE,
+        carbonProperties.setProperty(SORT_SIZE,
             CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL);
       }
     } catch (NumberFormatException e) {
       LOGGER.info(
           "The batch size value \"" + sortSizeStr + "\" is invalid. Using the default value
\""
               + CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL);
-      carbonProperties.setProperty(CarbonCommonConstants.SORT_SIZE,
+      carbonProperties.setProperty(SORT_SIZE,
           CarbonCommonConstants.SORT_SIZE_DEFAULT_VAL);
     }
   }
@@ -474,10 +634,10 @@ public final class CarbonProperties {
    */
   private void validateCarbonDataFileVersion() {
     String carbondataFileVersionString =
-        carbonProperties.getProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION);
+        carbonProperties.getProperty(CARBON_DATA_FILE_VERSION);
     if (carbondataFileVersionString == null) {
       // use default property if user does not specify version property
-      carbonProperties.setProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION,
+      carbonProperties.setProperty(CARBON_DATA_FILE_VERSION,
           CarbonCommonConstants.CARBON_DATA_FILE_DEFAULT_VERSION);
     } else {
       try {
@@ -487,12 +647,12 @@ public final class CarbonProperties {
         LOGGER.warn("Specified file version property is invalid: " + carbondataFileVersionString
             + ". Using " + CarbonCommonConstants.CARBON_DATA_FILE_DEFAULT_VERSION
             + " as default file version");
-        carbonProperties.setProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION,
+        carbonProperties.setProperty(CARBON_DATA_FILE_VERSION,
             CarbonCommonConstants.CARBON_DATA_FILE_DEFAULT_VERSION);
       }
     }
     LOGGER.info("Carbon Current data file version: " + carbonProperties
-        .setProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION,
+        .setProperty(CARBON_DATA_FILE_VERSION,
             CarbonCommonConstants.CARBON_DATA_FILE_DEFAULT_VERSION));
   }
 
@@ -533,6 +693,11 @@ public final class CarbonProperties {
     }
 
     print();
+    try {
+      initPropertySet();
+    } catch (IllegalAccessException e) {
+      LOGGER.error("Illegal access to declared field" + e.getMessage());
+    }
   }
 
   /**
@@ -605,6 +770,9 @@ public final class CarbonProperties {
   public CarbonProperties addProperty(String key, String value) {
     carbonProperties.setProperty(key, value);
     addedProperty.put(key, value);
+    // the method will validate the added property
+    // if the added property is not valid then will reset to default value.
+    validateAndLoadDefaultProperties(key.toLowerCase());
     return this;
   }
 
@@ -613,7 +781,7 @@ public final class CarbonProperties {
   }
 
   public ColumnarFormatVersion getFormatVersion() {
-    String versionStr = getInstance().getProperty(CarbonCommonConstants.CARBON_DATA_FILE_VERSION);
+    String versionStr = getInstance().getProperty(CARBON_DATA_FILE_VERSION);
     if (versionStr == null) {
       return getDefaultFormatVersion();
     } else {
@@ -796,7 +964,7 @@ public final class CarbonProperties {
     try {
       handoffSize = Long.parseLong(
           CarbonProperties.getInstance().getProperty(
-              CarbonCommonConstants.HANDOFF_SIZE,
+              HANDOFF_SIZE,
               "" + CarbonCommonConstants.HANDOFF_SIZE_DEFAULT
           )
       );
@@ -808,7 +976,7 @@ public final class CarbonProperties {
 
   public boolean isEnableAutoHandoff() {
     String enableAutoHandoffStr = CarbonProperties.getInstance().getProperty(
-        CarbonCommonConstants.ENABLE_AUTO_HANDOFF,
+        ENABLE_AUTO_HANDOFF,
         CarbonCommonConstants.ENABLE_AUTO_HANDOFF_DEFAULT);
     return enableAutoHandoffStr.equalsIgnoreCase("true");
   }
@@ -836,7 +1004,7 @@ public final class CarbonProperties {
     int executorStartUpTimeOut = 0;
     try {
       executorStartUpTimeOut = Integer.parseInt(carbonProperties
-          .getProperty(CarbonCommonConstants.CARBON_EXECUTOR_STARTUP_TIMEOUT,
+          .getProperty(CARBON_EXECUTOR_STARTUP_TIMEOUT,
               CarbonCommonConstants.CARBON_EXECUTOR_WAITING_TIMEOUT_DEFAULT));
       // If value configured by user is more than max value of time out then consider the
max value
       if (executorStartUpTimeOut > CarbonCommonConstants.CARBON_EXECUTOR_WAITING_TIMEOUT_MAX)
{
@@ -846,7 +1014,7 @@ public final class CarbonProperties {
       executorStartUpTimeOut =
           Integer.parseInt(CarbonCommonConstants.CARBON_EXECUTOR_WAITING_TIMEOUT_DEFAULT);
     }
-    carbonProperties.setProperty(CarbonCommonConstants.CARBON_EXECUTOR_STARTUP_TIMEOUT,
+    carbonProperties.setProperty(CARBON_EXECUTOR_STARTUP_TIMEOUT,
         String.valueOf(executorStartUpTimeOut));
     LOGGER.info("Executor start up wait time: " + executorStartUpTimeOut);
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcf3ca3f/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 27a9a6d..ae29b03 100644
--- a/core/src/test/java/org/apache/carbondata/core/CarbonPropertiesValidationTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/CarbonPropertiesValidationTest.java
@@ -60,7 +60,7 @@ public class CarbonPropertiesValidationTest extends TestCase {
     validateMethodType.invoke(carbonProperties);
     String valueAfterValidation =
         carbonProperties.getProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT);
-    assertTrue(!valueBeforeValidation.equals(valueAfterValidation));
+    assertTrue(valueBeforeValidation.equals(valueAfterValidation));
     assertTrue(
         CarbonCommonConstants.ENABLE_UNSAFE_SORT_DEFAULT.equalsIgnoreCase(valueAfterValidation));
   }
@@ -76,7 +76,7 @@ public class CarbonPropertiesValidationTest extends TestCase {
     validateMethodType.invoke(carbonProperties);
     String valueAfterValidation =
         carbonProperties.getProperty(CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION);
-    assertTrue(!valueBeforeValidation.equals(valueAfterValidation));
+    assertTrue(valueBeforeValidation.equals(valueAfterValidation));
     assertTrue(CarbonCommonConstants.CARBON_CUSTOM_BLOCK_DISTRIBUTION_DEFAULT
         .equalsIgnoreCase(valueAfterValidation));
   }
@@ -92,7 +92,7 @@ public class CarbonPropertiesValidationTest extends TestCase {
     validateMethodType.invoke(carbonProperties);
     String valueAfterValidation =
         carbonProperties.getProperty(CarbonCommonConstants.ENABLE_VECTOR_READER);
-    assertTrue(!valueBeforeValidation.equals(valueAfterValidation));
+    assertTrue(valueBeforeValidation.equals(valueAfterValidation));
     assertTrue(
         CarbonCommonConstants.ENABLE_VECTOR_READER_DEFAULT.equalsIgnoreCase(valueAfterValidation));
   }
@@ -108,7 +108,7 @@ public class CarbonPropertiesValidationTest extends TestCase {
     validateMethodType.invoke(carbonProperties);
     String valueAfterValidation =
         carbonProperties.getProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE);
-    assertTrue(!valueBeforeValidation.equals(valueAfterValidation));
+    assertTrue(valueBeforeValidation.equals(valueAfterValidation));
     assertTrue(
         CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT.equalsIgnoreCase(valueAfterValidation));
   }
@@ -124,7 +124,7 @@ public class CarbonPropertiesValidationTest extends TestCase {
     validateMethodType.invoke(carbonProperties);
     String valueAfterValidation =
         carbonProperties.getProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE);
-    assertTrue(!valueBeforeValidation.equals(valueAfterValidation));
+    assertTrue(valueBeforeValidation.equals(valueAfterValidation));
     assertTrue(
         CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT.equalsIgnoreCase(valueAfterValidation));
     carbonProperties.addProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE, "10240");
@@ -140,7 +140,7 @@ public class CarbonPropertiesValidationTest extends TestCase {
     validateMethodType.invoke(carbonProperties);
     valueAfterValidation =
         carbonProperties.getProperty(CarbonCommonConstants.CSV_READ_BUFFER_SIZE);
-    assertTrue(!valueBeforeValidation.equals(valueAfterValidation));
+    assertTrue(valueBeforeValidation.equals(valueAfterValidation));
     assertTrue(
         CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT.equalsIgnoreCase(valueAfterValidation));
   }
@@ -151,4 +151,58 @@ public class CarbonPropertiesValidationTest extends TestCase {
     carbonProperties.addProperty(CarbonCommonConstants.HANDOFF_SIZE, "" + newSize);
     assertEquals(newSize, carbonProperties.getHandoffSize());
   }
+
+  @Test public void testValidateTimeStampFormat()
+      throws NoSuchMethodException, InvocationTargetException, IllegalAccessException {
+    Method validateMethodType = carbonProperties.getClass()
+        .getDeclaredMethod("validateTimeFormatKey", new Class[] { String.class, String.class
});
+    validateMethodType.setAccessible(true);
+    carbonProperties.addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "agdgaJIASDG667");
+    String valueBeforeValidation =
+        carbonProperties.getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT);
+    validateMethodType.invoke(carbonProperties, CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
+    String valueAfterValidation =
+        carbonProperties.getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT);
+    assertTrue(valueBeforeValidation.equals(valueAfterValidation));
+    assertTrue(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT
+        .equalsIgnoreCase(valueAfterValidation));
+    carbonProperties.addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+        "yyyy-MM-dd hh:mm:ss");
+    validateMethodType.invoke(carbonProperties, CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+        CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
+    assertEquals("yyyy-MM-dd hh:mm:ss",
+        carbonProperties.getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT));
+  }
+
+  @Test public void testValidateSortFileWriteBufferSize()
+      throws NoSuchMethodException, InvocationTargetException, IllegalAccessException {
+    Method validateMethodType =
+        carbonProperties.getClass().getDeclaredMethod("validateSortFileWriteBufferSize");
+    validateMethodType.setAccessible(true);
+    carbonProperties.addProperty(CarbonCommonConstants.CARBON_SORT_FILE_WRITE_BUFFER_SIZE,
"test");
+    String valueBeforeValidation =
+        carbonProperties.getProperty(CarbonCommonConstants.CARBON_SORT_FILE_WRITE_BUFFER_SIZE);
+    validateMethodType.invoke(carbonProperties);
+    String valueAfterValidation =
+        carbonProperties.getProperty(CarbonCommonConstants.CARBON_SORT_FILE_WRITE_BUFFER_SIZE);
+    assertTrue(valueBeforeValidation.equals(valueAfterValidation));
+    assertTrue(CarbonCommonConstants.CARBON_SORT_FILE_WRITE_BUFFER_SIZE_DEFAULT_VALUE
+        .equalsIgnoreCase(valueAfterValidation));
+  }
+  @Test public void testValidateSortIntermediateFilesLimit()
+      throws NoSuchMethodException, InvocationTargetException, IllegalAccessException {
+    Method validateMethodType =
+        carbonProperties.getClass().getDeclaredMethod("validateSortIntermediateFilesLimit");
+    validateMethodType.setAccessible(true);
+    carbonProperties.addProperty(CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT, "test");
+    String valueBeforeValidation =
+        carbonProperties.getProperty(CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT);
+    validateMethodType.invoke(carbonProperties);
+    String valueAfterValidation =
+        carbonProperties.getProperty(CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT);
+    assertTrue(valueBeforeValidation.equals(valueAfterValidation));
+    assertTrue(CarbonCommonConstants.SORT_INTERMEDIATE_FILES_LIMIT_DEFAULT_VALUE
+        .equalsIgnoreCase(valueAfterValidation));
+  }
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/bcf3ca3f/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
index d9f119c..74216ac 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/datamap/DataMapWriterSuite.scala
@@ -138,6 +138,8 @@ class DataMapWriterSuite extends QueryTest with BeforeAndAfterAll {
 
     assert(DataMapWriterSuite.callbackSeq.head.contains("block start"))
     assert(DataMapWriterSuite.callbackSeq.last.contains("block end"))
+    // corrected test case the min "carbon.blockletgroup.size.in.mb" size could not be less
than
+    // 64 MB
     assert(
       DataMapWriterSuite.callbackSeq.slice(1, DataMapWriterSuite.callbackSeq.length - 1)
== Seq(
         "blocklet start 0",
@@ -145,17 +147,13 @@ class DataMapWriterSuite extends QueryTest with BeforeAndAfterAll {
         "add page data: blocklet 0, page 1",
         "add page data: blocklet 0, page 2",
         "add page data: blocklet 0, page 3",
-        "blocklet end: 0",
-        "blocklet start 1",
-        "add page data: blocklet 1, page 0",
-        "add page data: blocklet 1, page 1",
-        "add page data: blocklet 1, page 2",
-        "add page data: blocklet 1, page 3",
-        "blocklet end: 1",
-        "blocklet start 2",
-        "add page data: blocklet 2, page 0",
-        "add page data: blocklet 2, page 1",
-        "blocklet end: 2"
+        "add page data: blocklet 0, page 4",
+        "add page data: blocklet 0, page 5",
+        "add page data: blocklet 0, page 6",
+        "add page data: blocklet 0, page 7",
+        "add page data: blocklet 0, page 8",
+        "add page data: blocklet 0, page 9",
+        "blocklet end: 0"
       ))
     DataMapWriterSuite.callbackSeq = Seq()
   }


Mime
View raw message