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/17 06:33:51 UTC
carbondata git commit: [CARBONDATA-2346] Added fix for NULL error
while dropping partition with multiple Pre-Aggregate tables
Repository: carbondata
Updated Branches:
refs/heads/master ecd6c0c54 -> b0e6fbea2
[CARBONDATA-2346] Added fix for NULL error while dropping partition with multiple Pre-Aggregate tables
1. While dropping Partitions when multiple Pre-Aggregate tables are created,
partitions to be dropped was considered null (getParentColumnTableRelations = null)
2. Fixed issue of failure to Create Pre-Aggregate table with count(*) as
column list was considered null
This closes #2170
Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/b0e6fbea
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/b0e6fbea
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/b0e6fbea
Branch: refs/heads/master
Commit: b0e6fbea2cbba3dbf7df621bbe5194c9a209c56b
Parents: ecd6c0c
Author: praveenmeenakshi56 <pr...@gmail.com>
Authored: Fri Apr 13 16:01:35 2018 +0530
Committer: kumarvishal <ku...@gmail.com>
Committed: Tue Apr 17 14:32:20 2018 +0800
----------------------------------------------------------------------
...ndardPartitionWithPreaggregateTestCase.scala | 40 ++++++++++++++++++++
.../preaaggregate/PreAggregateListeners.scala | 1 +
.../preaaggregate/PreAggregateTableHelper.scala | 4 +-
3 files changed, 44 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b0e6fbea/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 02c3158..ce264a4 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
@@ -475,6 +475,46 @@ class StandardPartitionWithPreaggregateTestCase extends QueryTest with BeforeAnd
}
}
+ test("test dropping partition which has already been deleted") {
+ sql("drop table if exists partitiontable")
+ sql("create table partitiontable(id int,name string) partitioned by (email string) " +
+ "stored by 'carbondata' tblproperties('sort_scope'='global_sort')")
+ sql("insert into table partitiontable select 1,'huawei','abc'")
+ sql("create datamap ag1 on table partitiontable using 'preaggregate' as select count(email),id" +
+ " from partitiontable group by id")
+ sql("create datamap ag2 on table partitiontable using 'preaggregate' as select sum(email),name" +
+ " from partitiontable group by name")
+ sql("create datamap ag3 on table partitiontable using 'preaggregate' as select max(email),name" +
+ " from partitiontable group by name")
+ sql("create datamap ag4 on table partitiontable using 'preaggregate' as select min(email),name" +
+ " from partitiontable group by name")
+ sql("create datamap ag5 on table partitiontable using 'preaggregate' as select avg(email),name" +
+ " from partitiontable group by name")
+ sql("alter table partitiontable add partition (email='def')")
+ sql("insert into table partitiontable select 1,'huawei','def'")
+ sql("drop datamap ag1 on table partitiontable")
+ sql("drop datamap ag2 on table partitiontable")
+ sql("drop datamap ag3 on table partitiontable")
+ sql("drop datamap ag4 on table partitiontable")
+ sql("drop datamap ag5 on table partitiontable")
+ sql("alter table partitiontable drop partition(email='def')")
+ assert(intercept[Exception] {
+ sql("alter table partitiontable drop partition(email='def')")
+ }.getMessage.contains("No partition is dropped. One partition spec 'Map(email -> def)' does not exist in table 'partitiontable' database 'partition_preaggregate'"))
+ }
+
+ test("test Pre-Aggregate table creation with count(*) on Partition table") {
+ sql("drop table if exists partitiontable")
+ sql("create table partitiontable(id int,name string) partitioned by (email string) " +
+ "stored by 'carbondata' tblproperties('sort_scope'='global_sort')")
+ sql("insert into table partitiontable select 1,'huawei','abc'")
+ sql("create datamap ag1 on table partitiontable using 'preaggregate' as select count(*),id" +
+ " from partitiontable group by id")
+ sql("insert into table partitiontable select 1,'huawei','def'")
+ assert(sql("show datamap on table partitiontable").collect().head.get(0).toString.equalsIgnoreCase("ag1"))
+ sql("drop datamap ag1 on table partitiontable")
+ }
+
def preAggTableValidator(plan: LogicalPlan, actualTableName: String) : Unit = {
var isValidPlan = false
plan.transform {
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b0e6fbea/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
index ce4e6fa..c86fe5a 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateListeners.scala
@@ -207,6 +207,7 @@ object AlterTableDropPartitionMetaListener extends OperationEventListener{
partition =>
childColumns.exists { childColumn =>
childColumn.getAggFunction.isEmpty &&
+ childColumn.getParentColumnTableRelations != null &&
childColumn.getParentColumnTableRelations.asScala.head.getColumnName.
equals(partition)
}
http://git-wip-us.apache.org/repos/asf/carbondata/blob/b0e6fbea/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 1f9b2b4..4c6883a 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
@@ -76,7 +76,9 @@ case class PreAggregateTableHelper(
val partitionerFields = fieldRelationMap.collect {
case (field, dataMapField) if parentPartitionColumns
.exists(parentCol =>
- parentCol.equals(dataMapField.columnTableRelationList.get.head.parentColumnName) &&
+ /* For count(*) while Pre-Aggregate table creation,columnTableRelationList was null */
+ dataMapField.columnTableRelationList.getOrElse(Seq()).nonEmpty &&
+ parentCol.equals(dataMapField.columnTableRelationList.get.head.parentColumnName) &&
dataMapField.aggregateFunction.isEmpty) =>
(PartitionerField(field.name.get,
field.dataType,