[CARBONDATA-2530][MV] Disable the MV datamaps after main table load

Problem:
MV datamaps are not disabled after the main table load is done. So the wrong 
data is displaying.
Solution:
Disable the MV datamaps after main table load success.

This closes #2474


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

Branch: refs/heads/branch-1.4
Commit: 3cde160255fcf16fffe392c1ef5ad1d592be1cab
Parents: 5d055a4
Author: ravipesala <[email protected]>
Authored: Sun Jul 15 17:45:43 2018 +0530
Committer: ravipesala <[email protected]>
Committed: Tue Jul 31 00:10:41 2018 +0530

----------------------------------------------------------------------
 .../datamap/status/DataMapStatusManager.java     |  4 +++-
 .../table/DiskBasedDMSchemaStorageProvider.java  |  9 ++++++++-
 .../mv/rewrite/SummaryDatasetCatalog.scala       |  2 +-
 .../carbondata/mv/rewrite/MVCreateTestCase.scala | 19 +++++++++++++++++++
 4 files changed, 31 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/3cde1602/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
 
b/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
index d0ff589..c2f97ea 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
@@ -95,7 +95,9 @@ public class DataMapStatusManager {
         DataMapStoreManager.getInstance().getDataMapSchemasOfTable(table);
     List<DataMapSchema> dataMapToBeDisabled = new 
ArrayList<>(allDataMapSchemas.size());
     for (DataMapSchema dataMap : allDataMapSchemas) {
-      if (dataMap.isLazy()) {
+      // TODO all non datamaps like MV is now supports only lazy. Once the 
support is made the
+      // following check can be removed.
+      if (dataMap.isLazy() || !dataMap.isIndexDataMap()) {
         dataMapToBeDisabled.add(dataMap);
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3cde1602/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DiskBasedDMSchemaStorageProvider.java
----------------------------------------------------------------------
diff --git 
a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DiskBasedDMSchemaStorageProvider.java
 
b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DiskBasedDMSchemaStorageProvider.java
index 7c1be26..b8149cc 100644
--- 
a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DiskBasedDMSchemaStorageProvider.java
+++ 
b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DiskBasedDMSchemaStorageProvider.java
@@ -39,6 +39,7 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.util.CarbonUtil;
 
 import com.google.gson.Gson;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 
@@ -109,7 +110,13 @@ public class DiskBasedDMSchemaStorageProvider implements 
DataMapSchemaStoragePro
     for (DataMapSchema dataMapSchema : this.dataMapSchemas) {
       List<RelationIdentifier> parentTables = dataMapSchema.getParentTables();
       for (RelationIdentifier identifier : parentTables) {
-        if 
(identifier.getTableId().equalsIgnoreCase(carbonTable.getTableId())) {
+        if (StringUtils.isNotEmpty(identifier.getTableId())) {
+          if 
(identifier.getTableId().equalsIgnoreCase(carbonTable.getTableId())) {
+            dataMapSchemas.add(dataMapSchema);
+            break;
+          }
+        } else if 
(identifier.getTableName().equalsIgnoreCase(carbonTable.getTableName()) &&
+            
identifier.getDatabaseName().equalsIgnoreCase(carbonTable.getDatabaseName())) {
           dataMapSchemas.add(dataMapSchema);
           break;
         }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3cde1602/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/rewrite/SummaryDatasetCatalog.scala
----------------------------------------------------------------------
diff --git 
a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/rewrite/SummaryDatasetCatalog.scala
 
b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/rewrite/SummaryDatasetCatalog.scala
index 1072e96..3f64c40 100644
--- 
a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/rewrite/SummaryDatasetCatalog.scala
+++ 
b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/rewrite/SummaryDatasetCatalog.scala
@@ -235,7 +235,7 @@ private[mv] class SummaryDatasetCatalog(sparkSession: 
SparkSession)
       //  ****not sure what enabledDataSets is used for ****
       //  can enable/disable datamap move to other place ?
       //    val feasible = enabledDataSets.filter { x =>
-      val feasible = summaryDatasets.filter { x =>
+      val feasible = enabledDataSets.filter { x =>
         (x.signature, sig) match {
           case (Some(sig1), Some(sig2)) =>
             if (sig1.groupby && sig2.groupby && 
sig1.datasets.subsetOf(sig2.datasets)) {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/3cde1602/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/MVCreateTestCase.scala
----------------------------------------------------------------------
diff --git 
a/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/MVCreateTestCase.scala
 
b/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/MVCreateTestCase.scala
index 01884c5..4056f92 100644
--- 
a/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/MVCreateTestCase.scala
+++ 
b/datamap/mv/core/src/test/scala/org/apache/carbondata/mv/rewrite/MVCreateTestCase.scala
@@ -731,6 +731,25 @@ class MVCreateTestCase extends QueryTest with 
BeforeAndAfterAll {
     sql("drop datamap if exists MV_order")
   }
 
+  test("jira carbondata-2530") {
+
+    sql("drop table if exists test1")
+    sql("drop datamap if exists datamv2")
+    sql("create table test1( name string,country string,age int,salary int) 
stored by 'carbondata'")
+    sql("insert into test1 select 'name1','USA',12,23")
+    sql("create datamap datamv2 using 'mv' as select country,sum(salary) from 
test1 group by country")
+    sql("rebuild datamap datamv2")
+    val frame = sql("select country,sum(salary) from test1 where country='USA' 
group by country")
+    val analyzed = frame.queryExecution.analyzed
+    assert(verifyMVDataMap(analyzed, "datamv2"))
+    sql("insert into test1 select 'name1','USA',12,23")
+    val frame1 = sql("select country,sum(salary) from test1 where 
country='USA' group by country")
+    val analyzed1 = frame1.queryExecution.analyzed
+    assert(!verifyMVDataMap(analyzed1, "datamv2"))
+    sql("drop datamap if exists datamv2")
+    sql("drop table if exists test1")
+  }
+
   def verifyMVDataMap(logicalPlan: LogicalPlan, dataMapName: String): Boolean 
= {
     val tables = logicalPlan collect {
       case l: LogicalRelation => l.catalogTable.get

Reply via email to