You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ra...@apache.org on 2017/09/09 13:24:06 UTC

carbondata git commit: [CARBONDATA-1420] Fixed bug for creation of partitioned table with date datatype column

Repository: carbondata
Updated Branches:
  refs/heads/master 435ea26eb -> 252c3e335


[CARBONDATA-1420] Fixed bug for creation of partitioned table with date datatype column

Added code to fetch the default date and timestamp format if not specified in carbon.properties.
Added exception case when partition values are not compatible with datatype of partition column.
Added test case for fetching default timestamp and date formats in case of create partition table command.

This closes #1319


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

Branch: refs/heads/master
Commit: 252c3e335e69fc0342de825c38a01b51cf0330a9
Parents: 435ea26
Author: Geetika Gupta <ge...@knoldus.in>
Authored: Mon Sep 4 16:51:18 2017 +0530
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Sat Sep 9 18:53:52 2017 +0530

----------------------------------------------------------------------
 .../partition/TestDDLForPartitionTable.scala    |  32 +--
 ...ForPartitionTableWithDefaultProperties.scala | 207 +++++++++++++++++++
 .../carbondata/spark/util/CommonUtil.scala      |  15 +-
 3 files changed, 235 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/252c3e33/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
index 561ebf5..8083fde 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
@@ -197,7 +197,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
           | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
         """.stripMargin)
     }
-    assert(exception_test_list_int.getMessage.contains("Invalid partition definition"))
+    assert(exception_test_list_int.getMessage.contains("Invalid Partition Values"))
 
     sql("DROP TABLE IF EXISTS test_list_small")
     val exception_test_list_small: Exception = intercept[Exception] {
@@ -208,7 +208,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
           | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
         """.stripMargin)
     }
-    assert(exception_test_list_small.getMessage.contains("Invalid partition definition"))
+    assert(exception_test_list_small.getMessage.contains("Invalid Partition Values"))
 
     sql("DROP TABLE IF EXISTS test_list_float")
     val exception_test_list_float: Exception = intercept[Exception] {
@@ -219,7 +219,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
           | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
         """.stripMargin)
     }
-    assert(exception_test_list_float.getMessage.contains("Invalid partition definition"))
+    assert(exception_test_list_float.getMessage.contains("Invalid Partition Values"))
 
     sql("DROP TABLE IF EXISTS test_list_double")
     val exception_test_list_double: Exception = intercept[Exception] {
@@ -230,7 +230,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
           | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
         """.stripMargin)
     }
-    assert(exception_test_list_double.getMessage.contains("Invalid partition definition"))
+    assert(exception_test_list_double.getMessage.contains("Invalid Partition Values"))
 
     sql("DROP TABLE IF EXISTS test_list_bigint")
     val exception_test_list_bigint: Exception = intercept[Exception] {
@@ -241,7 +241,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
           | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
         """.stripMargin)
     }
-    assert(exception_test_list_bigint.getMessage.contains("Invalid partition definition"))
+    assert(exception_test_list_bigint.getMessage.contains("Invalid Partition Values"))
 
     sql("DROP TABLE IF EXISTS test_list_date")
     val exception_test_list_date: Exception = intercept[Exception] {
@@ -252,7 +252,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
           | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
         """.stripMargin)
     }
-    assert(exception_test_list_date.getMessage.contains("Invalid partition definition"))
+    assert(exception_test_list_date.getMessage.contains("Invalid Partition Values"))
 
     sql("DROP TABLE IF EXISTS test_list_timestamp")
     val exception_test_list_timestamp: Exception = intercept[Exception] {
@@ -263,7 +263,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
           | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='abc,def')
         """.stripMargin)
     }
-    assert(exception_test_list_timestamp.getMessage.contains("Invalid partition definition"))
+    assert(exception_test_list_timestamp.getMessage.contains("Invalid Partition Values"))
 
     sql("DROP TABLE IF EXISTS test_list_decimal")
     val exception_test_list_decimal: Exception = intercept[Exception] {
@@ -274,7 +274,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
           | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='23.23111,2.32')
         """.stripMargin)
     }
-    assert(exception_test_list_decimal.getMessage.contains("Invalid partition definition"))
+    assert(exception_test_list_decimal.getMessage.contains("Invalid Partition Values"))
   }
 
   test("test exception when values in range_info can not match partition column type") {
@@ -287,7 +287,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
           | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
         """.stripMargin)
     }
-    assert(exception_test_range_int.getMessage.contains("Invalid partition definition"))
+    assert(exception_test_range_int.getMessage.contains("Invalid Partition Values"))
 
     sql("DROP TABLE IF EXISTS test_range_smallint")
     val exception_test_range_smallint: Exception = intercept[Exception] {
@@ -298,7 +298,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
           | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
         """.stripMargin)
     }
-    assert(exception_test_range_smallint.getMessage.contains("Invalid partition definition"))
+    assert(exception_test_range_smallint.getMessage.contains("Invalid Partition Values"))
 
     sql("DROP TABLE IF EXISTS test_range_float")
     val exception_test_range_float: Exception = intercept[Exception] {
@@ -309,7 +309,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
           | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
         """.stripMargin)
     }
-    assert(exception_test_range_float.getMessage.contains("Invalid partition definition"))
+    assert(exception_test_range_float.getMessage.contains("Invalid Partition Values"))
 
     sql("DROP TABLE IF EXISTS test_range_double")
     val exception_test_range_double: Exception = intercept[Exception] {
@@ -320,7 +320,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
           | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
         """.stripMargin)
     }
-    assert(exception_test_range_double.getMessage.contains("Invalid partition definition"))
+    assert(exception_test_range_double.getMessage.contains("Invalid Partition Values"))
 
     sql("DROP TABLE IF EXISTS test_range_bigint")
     val exception_test_range_bigint: Exception = intercept[Exception] {
@@ -331,7 +331,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
           | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
         """.stripMargin)
     }
-    assert(exception_test_range_bigint.getMessage.contains("Invalid partition definition"))
+    assert(exception_test_range_bigint.getMessage.contains("Invalid Partition Values"))
 
     sql("DROP TABLE IF EXISTS test_range_date")
     val exception_test_range_date: Exception = intercept[Exception] {
@@ -342,7 +342,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
           | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
         """.stripMargin)
     }
-    assert(exception_test_range_date.getMessage.contains("Invalid partition definition"))
+    assert(exception_test_range_date.getMessage.contains("Invalid Partition Values"))
 
     sql("DROP TABLE IF EXISTS test_range_timestamp")
     val exception_test_range_timestamp: Exception = intercept[Exception] {
@@ -353,7 +353,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
           | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
         """.stripMargin)
     }
-    assert(exception_test_range_timestamp.getMessage.contains("Invalid partition definition"))
+    assert(exception_test_range_timestamp.getMessage.contains("Invalid Partition Values"))
 
     sql("DROP TABLE IF EXISTS test_range_decimal")
     val exception_test_range_decimal: Exception = intercept[Exception] {
@@ -364,7 +364,7 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
           | TBLPROPERTIES('PARTITION_TYPE'='RANGE', 'RANGE_INFO'='abc,def')
         """.stripMargin)
     }
-    assert(exception_test_range_decimal.getMessage.contains("Invalid partition definition"))
+    assert(exception_test_range_decimal.getMessage.contains("Invalid Partition Values"))
   }
 
   override def afterAll = {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/252c3e33/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
new file mode 100644
index 0000000..7359b53
--- /dev/null
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
@@ -0,0 +1,207 @@
+/*
+ * 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.spark.testsuite.partition
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.metadata.CarbonMetadata
+import org.apache.carbondata.core.metadata.datatype.DataType
+import org.apache.carbondata.core.metadata.encoder.Encoding
+import org.apache.carbondata.core.metadata.schema.partition.PartitionType
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+class TestDDLForPartitionTableWithDefaultProperties  extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll = {
+    dropTable
+    }
+
+  test("create partition table: hash partition") {
+    sql(
+      """
+        | CREATE TABLE default.hashTable (empname String, designation String, doj Timestamp,
+        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
+        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
+        |  utilization int,salary int)
+        | PARTITIONED BY (empno int)
+        | STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='3')
+      """.stripMargin)
+
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default_hashTable")
+    val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName)
+    assert(partitionInfo != null)
+    assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("empno"))
+    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataType.INT)
+    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 0)
+    assert(partitionInfo.getPartitionType ==  PartitionType.HASH)
+    assert(partitionInfo.getNumPartitions == 3)
+  }
+
+  test("create partition table: range partition") {
+    sql(
+      """
+        | CREATE TABLE default.rangeTable (empno int, empname String, designation String,
+        |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
+        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
+        |  utilization int,salary int)
+        | PARTITIONED BY (doj Timestamp)
+        | STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
+        |  'RANGE_INFO'='2017-06-11 00:00:02, 2017-06-13 23:59:59')
+      """.stripMargin)
+
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default_rangeTable")
+    val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName)
+    assert(partitionInfo != null)
+    assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("doj"))
+    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataType.TIMESTAMP)
+    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 3)
+    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(0) == Encoding.DICTIONARY)
+    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(1) == Encoding.DIRECT_DICTIONARY)
+    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(2) == Encoding.INVERTED_INDEX)
+    assert(partitionInfo.getPartitionType == PartitionType.RANGE)
+    assert(partitionInfo.getRangeInfo.size == 2)
+    assert(partitionInfo.getRangeInfo.get(0).equals("2017-06-11 00:00:02"))
+    assert(partitionInfo.getRangeInfo.get(1).equals("2017-06-13 23:59:59"))
+  }
+
+  test("create partition table: list partition with timestamp datatype") {
+    sql(
+      """
+        | CREATE TABLE default.listTable (empno int, empname String, designation String, doj Timestamp,
+        |  workgroupcategoryname String, deptno int, deptname String,
+        |  projectcode int, projectjoindate Timestamp, attendance int,
+        |  utilization int,salary int)
+        | PARTITIONED BY (projectenddate Timestamp)
+        | STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
+        |  'LIST_INFO'='2017-06-11 00:00:02, 2017-06-13 23:59:59')
+      """.stripMargin)
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default_listTable")
+    val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName)
+    assert(partitionInfo != null)
+    assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("projectenddate"))
+    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataType.TIMESTAMP)
+    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 3)
+    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(0) == Encoding.DICTIONARY)
+    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(1) == Encoding.DIRECT_DICTIONARY)
+    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(2) == Encoding.INVERTED_INDEX)
+    assert(partitionInfo.getPartitionType == PartitionType.LIST)
+    assert(partitionInfo.getListInfo.size == 2)
+    assert(partitionInfo.getListInfo.get(0).size == 1)
+    assert(partitionInfo.getListInfo.get(0).get(0).equals("2017-06-11 00:00:02"))
+    assert(partitionInfo.getListInfo.get(1).size == 1)
+    assert(partitionInfo.getListInfo.get(1).get(0).equals("2017-06-13 23:59:59"))
+  }
+
+  test("create partition table: list partition with date datatype") {
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_DATE_FORMAT, "yyyy-MM-dd")
+
+    sql(
+      """
+        | CREATE TABLE default.listTableDate (empno int, empname String, designation String, doj Timestamp,
+        |  workgroupcategoryname String, deptno int, deptname String,
+        |  projectcode int, projectjoindate Timestamp, attendance int,
+        |  utilization int,salary int)
+        | PARTITIONED BY (projectenddate date)
+        | STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
+        |  'LIST_INFO'='2017-06-11 , 2017-06-13')
+      """.stripMargin)
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default_listTableDate")
+    val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getFactTableName)
+    assert(partitionInfo != null)
+    assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("projectenddate"))
+    assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataType.DATE)
+    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.size == 3)
+    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(0) == Encoding.DICTIONARY)
+    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(1) == Encoding.DIRECT_DICTIONARY)
+    assert(partitionInfo.getColumnSchemaList.get(0).getEncodingList.get(2) == Encoding.INVERTED_INDEX)
+    assert(partitionInfo.getPartitionType == PartitionType.LIST)
+    assert(partitionInfo.getListInfo.size == 2)
+    assert(partitionInfo.getListInfo.get(0).size == 1)
+    assert(partitionInfo.getListInfo.get(0).get(0).equals("2017-06-11"))
+    assert(partitionInfo.getListInfo.get(1).size == 1)
+    assert(partitionInfo.getListInfo.get(1).get(0).equals("2017-06-13"))
+  }
+
+  test("test exception when values in list_info can not match partition column type") {
+    sql("DROP TABLE IF EXISTS test_list_int")
+    val exception_test_list_int: Exception = intercept[Exception] {
+      sql(
+        """
+          | CREATE TABLE test_list_int(col1 INT, col2 STRING)
+          | PARTITIONED BY (col3 INT) STORED BY 'carbondata'
+          | TBLPROPERTIES('PARTITION_TYPE'='LIST', 'LIST_INFO'='1,2,(abc,efg)')
+        """.stripMargin)
+    }
+    assert(exception_test_list_int.getMessage.contains("Invalid Partition Values"))
+  }
+
+  test("test exception when partition values in rangeTable are in group ") {
+    sql("DROP TABLE IF EXISTS rangeTable")
+    val exception_test_list_int: Exception = intercept[Exception] {
+      sql(
+        """
+          |CREATE TABLE default.rangeTable (empno int, empname String, designation String,
+          |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
+          |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
+          |  utilization int,salary int)
+          | PARTITIONED BY (doj Timestamp)
+          | STORED BY 'org.apache.carbondata.format'
+          | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
+          |  'RANGE_INFO'='2017-06-11 00:00:02, (2017-06-13 23:59:59, 2017-09-13 23:45:59)')
+        """.stripMargin)
+    }
+    assert(exception_test_list_int.getMessage.contains("Invalid Partition Values"))
+  }
+
+  test("test exception when values in rangeTable does not match partition column type") {
+    sql("DROP TABLE IF EXISTS rangeTable")
+    val exception_test_list_int: Exception = intercept[Exception] {
+      sql(
+        """
+          |CREATE TABLE default.rangeTable (empno int, empname String, designation String,
+          |  workgroupcategory int, workgroupcategoryname String, deptno int, deptname String,
+          |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
+          |  utilization int,salary int)
+          | PARTITIONED BY (doj Timestamp)
+          | STORED BY 'org.apache.carbondata.format'
+          | TBLPROPERTIES('PARTITION_TYPE'='RANGE',
+          |  'RANGE_INFO'='2017-06-11 00:00:02, abc, 2017-09-13 23:45:59')
+        """.stripMargin)
+    }
+    assert(exception_test_list_int.getMessage.contains("Invalid Partition Values"))
+  }
+
+
+  override def afterAll = {
+    dropTable
+  }
+
+  def dropTable = {
+    sql("drop table if exists hashTable")
+    sql("drop table if exists rangeTable")
+    sql("drop table if exists listTable")
+    sql("drop table if exists listTableDate")
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/carbondata/blob/252c3e33/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
index 4f4faff..c67806e 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
@@ -242,11 +242,13 @@ object CommonUtil {
         value.matches(pattern)
       case "timestamptype" =>
         val timeStampFormat = new SimpleDateFormat(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT))
+          .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+            CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT))
         scala.util.Try(timeStampFormat.parse(value)).isSuccess
       case "datetype" =>
         val dateFormat = new SimpleDateFormat(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT))
+          .getProperty(CarbonCommonConstants.CARBON_DATE_FORMAT,
+            CarbonCommonConstants.CARBON_DATE_DEFAULT_FORMAT))
         scala.util.Try(dateFormat.parse(value)).isSuccess
       case others =>
        if (others != null && others.startsWith("char")) {
@@ -303,8 +305,15 @@ object CommonUtil {
       case _ =>
         validateTypeConvertForSpark2(partitionerField, value)
     }
-    result
+
+    if(!result) {
+      throw new MalformedCarbonCommandException(s"Invalid Partition Values for partition " +
+        s"column: ${partitionerField.partitionColumn}")
+    } else {
+      result
+    }
   }
+
   /**
    * To verify the range info is in correct order
    * @param rangeInfo