You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ku...@apache.org on 2018/04/20 10:50:17 UTC

carbondata git commit: [CARBONDATA-2334] Added Property enabling user to block partitioning of Pre-Aggregate table

Repository: carbondata
Updated Branches:
  refs/heads/master 3beaa0e29 -> a9d5e9dec


[CARBONDATA-2334] Added Property enabling user to block partitioning of Pre-Aggregate table

While creating PreAgg table, in DMPROPERTIES, 'partitioning'='false' will disable partitioning of the Pre-Aggregate table

This closes #2157


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

Branch: refs/heads/master
Commit: a9d5e9dec117c4cb432f90dd16d31aad77cb9308
Parents: 3beaa0e
Author: praveenmeenakshi56 <pr...@gmail.com>
Authored: Wed Apr 11 14:13:14 2018 +0530
Committer: kumarvishal09 <ku...@gmail.com>
Committed: Fri Apr 20 16:15:54 2018 +0530

----------------------------------------------------------------------
 .../testsuite/preaggregate/TestPreAggCreateCommand.scala  |  2 +-
 .../StandardPartitionWithPreaggregateTestCase.scala       | 10 ++++++++++
 .../org/apache/carbondata/datamap/DataMapProperty.java    |  1 +
 .../carbondata/datamap/PreAggregateDataMapProvider.java   |  7 ++++---
 .../command/preaaggregate/PreAggregateTableHelper.scala   |  5 ++++-
 5 files changed, 20 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/a9d5e9de/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
index 7cb1adf..57b3b8f 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/preaggregate/TestPreAggCreateCommand.scala
@@ -274,7 +274,7 @@ class TestPreAggCreateCommand extends QueryTest with BeforeAndAfterAll {
            | GROUP BY dob,name
        """.stripMargin)
     }
-    assert(e.getMessage.contains("Only 'path' dmproperty is allowed for this datamap"))
+    assert(e.getMessage.contains("Only 'path' and 'partitioning' dmproperties are allowed for this datamap"))
     sql("DROP TABLE IF EXISTS maintabletime")
   }
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a9d5e9de/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionWithPreaggregateTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionWithPreaggregateTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionWithPreaggregateTestCase.scala
index ce264a4..489d5b1 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionWithPreaggregateTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/standardpartition/StandardPartitionWithPreaggregateTestCase.scala
@@ -514,6 +514,16 @@ class StandardPartitionWithPreaggregateTestCase extends QueryTest with BeforeAnd
     assert(sql("show datamap on table partitiontable").collect().head.get(0).toString.equalsIgnoreCase("ag1"))
     sql("drop datamap ag1 on table partitiontable")
   }
+  
+  test("test blocking partitioning of Pre-Aggregate table") {
+    sql("drop table if exists updatetime_8")
+    sql("create table updatetime_8" +
+      "(countryid smallint,hs_len smallint,minstartdate string,startdate string,newdate string,minnewdate string) partitioned by (imex smallint) stored by 'carbondata' tblproperties('sort_scope'='global_sort','sort_columns'='countryid,imex,hs_len,minstartdate,startdate,newdate,minnewdate','table_blocksize'='256')")
+    sql("create datamap ag on table updatetime_8 using 'preaggregate' dmproperties('partitioning'='false') as select imex,sum(hs_len) from updatetime_8 group by imex")
+    val carbonTable = CarbonEnv.getCarbonTable(Some("partition_preaggregate"), "updatetime_8_ag")(sqlContext.sparkSession)
+    assert(!carbonTable.isHivePartitionTable)
+    sql("drop table if exists updatetime_8")
+  }
 
   def preAggTableValidator(plan: LogicalPlan, actualTableName: String) : Unit = {
     var isValidPlan = false

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a9d5e9de/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProperty.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProperty.java b/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProperty.java
index 0cf0d04..043acb1 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProperty.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/datamap/DataMapProperty.java
@@ -28,5 +28,6 @@ public class DataMapProperty {
   /**
    * Used to specify the store location of the datamap
    */
+  public static final String PARTITIONING = "partitioning";
   public static final String PATH = "path";
 }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a9d5e9de/integration/spark2/src/main/java/org/apache/carbondata/datamap/PreAggregateDataMapProvider.java
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/java/org/apache/carbondata/datamap/PreAggregateDataMapProvider.java b/integration/spark2/src/main/java/org/apache/carbondata/datamap/PreAggregateDataMapProvider.java
index 835b31c..b612f47 100644
--- a/integration/spark2/src/main/java/org/apache/carbondata/datamap/PreAggregateDataMapProvider.java
+++ b/integration/spark2/src/main/java/org/apache/carbondata/datamap/PreAggregateDataMapProvider.java
@@ -52,10 +52,11 @@ public class PreAggregateDataMapProvider implements DataMapProvider {
   private void validateDmProperty(DataMapSchema dataMapSchema)
       throws MalformedDataMapCommandException {
     if (!dataMapSchema.getProperties().isEmpty()) {
-      if (dataMapSchema.getProperties().size() > 1 ||
-          !dataMapSchema.getProperties().containsKey(DataMapProperty.PATH)) {
+      if (dataMapSchema.getProperties().size() > 2 || (
+              !dataMapSchema.getProperties().containsKey(DataMapProperty.PATH) &&
+                      !dataMapSchema.getProperties().containsKey(DataMapProperty.PARTITIONING))) {
         throw new MalformedDataMapCommandException(
-            "Only 'path' dmproperty is allowed for this datamap");
+                "Only 'path' and 'partitioning' dmproperties are allowed for this datamap");
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/a9d5e9de/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala
index 4c6883a..2862d96 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateTableHelper.scala
@@ -67,7 +67,10 @@ case class PreAggregateTableHelper(
     val partitionInfo = parentTable.getPartitionInfo
     val fields = fieldRelationMap.keySet.toSeq
     val tableProperties = mutable.Map[String, String]()
-    val parentPartitionColumns = if (parentTable.isHivePartitionTable) {
+    val usePartitioning = dataMapProperties.getOrDefault("partitioning", "true").toBoolean
+    val parentPartitionColumns = if (!usePartitioning) {
+      Seq.empty
+    } else if (parentTable.isHivePartitionTable) {
       partitionInfo.getColumnSchemaList.asScala.map(_.getColumnName)
     } else {
       Seq()