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/08/30 08:55:45 UTC

carbondata git commit: [CARBONDATA-1401] fix duplicate issue in ListInfo

Repository: carbondata
Updated Branches:
  refs/heads/master 4f7487dec -> c77221227


[CARBONDATA-1401] fix duplicate issue in ListInfo

This closes#1276


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

Branch: refs/heads/master
Commit: c77221227e4ae8613e739c0e65572bf5d3a54fa9
Parents: 4f7487d
Author: lionelcao <wh...@gmail.com>
Authored: Mon Aug 21 14:43:35 2017 +0800
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Wed Aug 30 14:25:21 2017 +0530

----------------------------------------------------------------------
 .../partition/TestDDLForPartitionTable.scala         | 15 ++++++++++++++-
 .../apache/carbondata/spark/util/CommonUtil.scala    |  9 ++++++++-
 .../spark/sql/catalyst/CarbonDDLSqlParser.scala      |  1 +
 3 files changed, 23 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7722122/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 4ae5ece..561ebf5 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
@@ -17,7 +17,6 @@
 
 package org.apache.carbondata.spark.testsuite.partition
 
-import org.apache.spark.sql.test.TestQueryExecutor
 import org.scalatest.BeforeAndAfterAll
 import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.metadata.CarbonMetadata
@@ -118,6 +117,20 @@ class TestDDLForPartitionTable  extends QueryTest with BeforeAndAfterAll {
     assert(partitionInfo.getListInfo.get(2).get(1).equals("3"))
   }
 
+  test("create partition table: list partition with duplicate value") {
+    intercept[Exception] { sql(
+      """
+        | CREATE TABLE default.listTableError (empno int, empname String, designation String, doj Timestamp,
+        |  workgroupcategoryname String, deptno int, deptname String,
+        |  projectcode int, projectjoindate Timestamp, projectenddate Timestamp,attendance int,
+        |  utilization int,salary int)
+        | PARTITIONED BY (workgroupcategory string)
+        | STORED BY 'org.apache.carbondata.format'
+        | TBLPROPERTIES('PARTITION_TYPE'='LIST',
+        |  'LIST_INFO'='0, 1, (2, 3, 1)')
+      """.stripMargin) }
+  }
+
   test("test exception if partition column is dropped") {
     sql("drop table if exists test")
     sql(

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7722122/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 6f5c85b..37aefcb 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
@@ -167,7 +167,7 @@ object CommonUtil {
 
   /**
    * 1. If partitioned by clause exists, then partition_type should be defined
-   * 2. If partition_type is Hash, then number_of_partitions should be defined
+   * 2. If partition_type is Hash, then num_partitions should be defined
    * 3. If partition_type is List, then list_info should be defined
    * 4. If partition_type is Range, then range_info should be defined
    * 5. Only support single level partition for now
@@ -369,6 +369,13 @@ object CommonUtil {
     }
   }
 
+  def validateListInfo(listInfo: List[List[String]]): Unit = {
+    val list = listInfo.flatten
+    if (list.distinct.size != list.size) {
+      sys.error("Duplicate elements defined in LIST_INFO!")
+    }
+  }
+
   def validateFields(key: String, fields: Seq[Field]): Boolean = {
     var isValid: Boolean = false
     fields.foreach { field =>

http://git-wip-us.apache.org/repos/asf/carbondata/blob/c7722122/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
index 9aef6c9..03aac20 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
@@ -394,6 +394,7 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
       if (tableProperties.get(CarbonCommonConstants.LIST_INFO).isDefined) {
         val originListInfo = tableProperties.get(CarbonCommonConstants.LIST_INFO).get
         listInfo = PartitionUtils.getListInfo(originListInfo)
+        CommonUtil.validateListInfo(listInfo)
       }
       val cols : ArrayBuffer[ColumnSchema] = new ArrayBuffer[ColumnSchema]()
       partitionCols.foreach(partition_col => {