You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by GitBox <gi...@apache.org> on 2020/03/07 07:54:05 UTC

[GitHub] [carbondata] niuge01 opened a new pull request #3661: [WIP] Support materialized view

niuge01 opened a new pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661
 
 
    ### Why is this PR needed?
    1. Support create materialized view on all table, include parquest table, orc table, hive table and carbon table.
    2. Materialized view DDL is common in databases, carbondata should change its materialized view related SQL syntax as other database.
    3. Materialized view support mutil-tenant.
    
    ### What changes were proposed in this PR?
    1. Define materialized view related commands:  CREATE MATERIALIZED VIEW, DROP MATERIALIZED VIEW, REFRESH MATERIALIZED VIEW and SHOW MATERIALIZED VIEW.
    2. Move materialized view schema files to each database directory.
    3. Support create materialized view on all table, remove carbon table related check.
       
    ### Does this PR introduce any user interface change?
    - Yes. (please explain the change and update document)
   
    ### Is any new testcase added?
    - Yes
   
       
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601554019
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/811/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401356054
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/IndexMeta.java
 ##########
 @@ -29,24 +29,24 @@
 import org.apache.commons.lang3.StringUtils;
 
 /**
- * Metadata of the datamap, set by DataMap developer
+ * Metadata of the datamap, set by Index developer
  */
-@InterfaceAudience.Developer("DataMap")
+@InterfaceAudience.Developer("Index")
 @InterfaceStability.Evolving
-public class DataMapMeta {
+public class IndexMeta {
 
 Review comment:
   we will do it in a new pr.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r400679356
 
 

 ##########
 File path: docs/index-developer-guide.md
 ##########
 @@ -24,11 +24,11 @@ Currently, there are two types of DataMap supported:
 2. MVDataMap: DataMap that leverages Materialized View to accelerate olap style query, like SPJG query (select, predicate, join, groupby). Preaggregate, timeseries and mv DataMap belong to this type of DataMaps.
 
 Review comment:
   Yes, it will be changed in a new pr.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601584465
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2520/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-606426926
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/887/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395463202
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/IndexChooser.java
 ##########
 @@ -59,76 +59,76 @@
  *   the datamap which has fewer columns that is the first datamap.
  */
 @InterfaceAudience.Internal
-public class DataMapChooser {
+public class IndexChooser {
 
   private CarbonTable carbonTable;
-  private List<TableDataMap> cgDataMaps;
-  private List<TableDataMap> fgDataMaps;
+  private List<TableIndex> cgIndexes;
+  private List<TableIndex> fgIndexes;
 
-  public DataMapChooser(CarbonTable carbonTable) throws IOException {
+  public IndexChooser(CarbonTable carbonTable) throws IOException {
     this.carbonTable = carbonTable;
     // read all datamaps for this table and populate CG and FG datamap list
-    List<TableDataMap> visibleDataMaps =
-        DataMapStoreManager.getInstance().getAllVisibleDataMap(carbonTable);
+    List<TableIndex> visibleDataMaps =
 
 Review comment:
   Change variable name to `visibleIndexes`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-605770150
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/883/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401380863
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/CarbonExtensions.scala
 ##########
 @@ -76,6 +66,11 @@ case class CarbonOptimizerRule(session: SparkSession) extends Rule[LogicalPlan]
       self.synchronized {
         if (notAdded) {
           notAdded = false
+
+          session.udf.register(MaterializedViewUDF.DUMMY_FUNCTION, () => "")
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r399963004
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
 ##########
 @@ -163,6 +165,17 @@ private DataMapStoreManager() {
     return provider.retrieveAllSchemas();
   }
 
+  /**
+   * Return first match of the specified index name in table
+   *
+   */
+  public Optional<DataMapSchema> getIndexInTable(CarbonTable carbonTable, String indexName)
 
 Review comment:
   DataMapSchema need to change to IndexSchema ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-606428588
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2595/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395462101
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -2265,6 +2265,11 @@ private CarbonCommonConstants() {
    */
   public static final String PARENT_TABLES = "parent_tables";
 
+  /**
+   * This property will be used to store table name's associated with mv
+   */
 
 Review comment:
   Can reuse `PARENT_TABLES` which serves the same purpose

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-602569626
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/836/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r399971835
 
 

 ##########
 File path: integration/spark/src/test/scala/org/apache/carbondata/view/MaterializedViewTest.scala
 ##########
 @@ -0,0 +1,194 @@
+package org.apache.carbondata.view
+
+import java.io.File
+
+import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException
+import org.apache.spark.sql.catalyst.catalog.HiveTableRelation
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.spark.exception.ProcessMetaDataException
+
+class MaterializedViewTest extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+    drop()
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
+    val projectPath = new File(this.getClass.getResource("/").getPath + "../../../../")
+      .getCanonicalPath.replaceAll("\\\\", "/")
+    val integrationPath = s"$projectPath/integration"
+    val resourcesPath = s"$integrationPath/spark/src/test/resources"
+    sql(
+      """
+        | CREATE TABLE fact_table (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)
+        | STORED AS carbondata
+      """.stripMargin)
+    sql(s"""LOAD DATA local inpath '$resourcesPath/data_big.csv' INTO TABLE fact_table OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
+    sql(s"""LOAD DATA local inpath '$resourcesPath/data_big.csv' INTO TABLE fact_table OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
+  }
+
+  test("test create mv on hive table") {
+    sql("drop materialized view if exists mv1")
+    sql("drop table if exists source")
+    sql("create table source as select * from fact_table")
+    sql("create materialized view mv1 as select empname, deptname, avg(salary) from source group by empname, deptname")
 
 Review comment:
   Now that hive mv and our mv syntax is same. what is the impcat ?  On hive table even if we want to create hive mv. Does it create carbon mv table ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596309863
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2387/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396306214
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/status/MVSegmentStatusUtil.java
 ##########
 @@ -32,7 +32,7 @@
 /**
  * Utility class to get updated segment mapping for datamap table
  */
-public class DataMapSegmentStatusUtil {
+public class MVSegmentStatusUtil {
 
 Review comment:
   Can move `getSegmentMap` method from this class to `MaterializedViewManager` and delete this class

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395968369
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -2265,6 +2265,11 @@ private CarbonCommonConstants() {
    */
   public static final String PARENT_TABLES = "parent_tables";
 
+  /**
+   * This property will be used to store table name's associated with mv
+   */
 
 Review comment:
   the PARENT_TABLES field will be delete when clean the old mv implementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396974320
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala
 ##########
 @@ -136,11 +137,18 @@ private[sql] case class CarbonProjectForDeleteCommand(
       val allDataMapSchemas = DataMapStoreManager.getInstance
         .getDataMapSchemasOfTable(carbonTable).asScala
         .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                 !dataMapSchema.isIndexDataMap).asJava
+                                 !dataMapSchema.isIndex).asJava
 
 Review comment:
   Yes, will be delete when clean the old mv implementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596059399
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2363/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396349479
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
 ##########
 @@ -1011,14 +1015,25 @@ object CarbonDataRDDFactory {
       throw new Exception(errorMessage)
     } else {
       DataMapStatusManager.disableAllLazyDataMaps(carbonTable)
+      val viewManager = MaterializedViewManagerInSpark.get(session)
+      val viewSchemas = new util.ArrayList[MaterializedViewSchema]()
+      for (viewSchema <- viewManager.getSchemasOnTable(carbonTable).asScala) {
+        if (viewSchema.isRefreshOnManual) {
+          viewSchemas.add(viewSchema)
+        }
+      }
+      viewManager.setStatus(viewSchemas, MaterializedViewStatus.DISABLED)
       if (overwriteTable) {
         val allDataMapSchemas = DataMapStoreManager.getInstance
           .getDataMapSchemasOfTable(carbonTable).asScala
           .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                   !dataMapSchema.isIndexDataMap).asJava
+                                   !dataMapSchema.isIndex).asJava
 
 Review comment:
   Can remove code from line:1027 to 1033, as it is added for mv

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-600483365
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/794/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395968330
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/IndexChooser.java
 ##########
 @@ -59,76 +59,76 @@
  *   the datamap which has fewer columns that is the first datamap.
  */
 @InterfaceAudience.Internal
-public class DataMapChooser {
+public class IndexChooser {
 
   private CarbonTable carbonTable;
-  private List<TableDataMap> cgDataMaps;
-  private List<TableDataMap> fgDataMaps;
+  private List<TableIndex> cgIndexes;
+  private List<TableIndex> fgIndexes;
 
-  public DataMapChooser(CarbonTable carbonTable) throws IOException {
+  public IndexChooser(CarbonTable carbonTable) throws IOException {
     this.carbonTable = carbonTable;
     // read all datamaps for this table and populate CG and FG datamap list
-    List<TableDataMap> visibleDataMaps =
-        DataMapStoreManager.getInstance().getAllVisibleDataMap(carbonTable);
+    List<TableIndex> visibleDataMaps =
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396368302
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/listeners/MVListeners.scala
 ##########
 @@ -67,7 +67,7 @@ object AlterDataMaptableCompactionPostListener extends OperationEventListener {
     val allDataMapSchemas = DataMapStoreManager.getInstance
       .getDataMapSchemasOfTable(carbonTable).asScala
       .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                               !dataMapSchema.isIndexDataMap)
+                               !dataMapSchema.isIndex)
 
 Review comment:
   Can delete this class

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395460695
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 ##########
 @@ -885,6 +894,16 @@ public boolean isChildTableForMV() {
         .get(CarbonCommonConstants.PARENT_TABLES).isEmpty();
   }
 
+  /**
+   * Return true if this table is a MV table (child table of other table)
+   */
+  public boolean isMaterializedView() {
+    return tableInfo.getFactTable().getTableProperties()
+        .get(CarbonCommonConstants.ASSOCIATED_TABLES) != null &&
+        !tableInfo.getFactTable().getTableProperties()
+        .get(CarbonCommonConstants.ASSOCIATED_TABLES).isEmpty();
 
 Review comment:
   It is confusing  with isMVTable/isMaterializedView. I think both are same. Please check and use one 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396975897
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
 ##########
 @@ -101,6 +100,12 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     loadDataNew | explainPlan | alterTableColumnRenameAndModifyDataType |
     alterTableAddColumns
 
+  protected lazy val materializedViewCommands: Parser[LogicalPlan] =
 
 Review comment:
   MVParser will be delete when clean the old mv implementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-603103936
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/838/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596316531
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2388/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-603005146
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/837/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396367410
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/listeners/DropCacheEventListeners.scala
 ##########
 @@ -31,31 +32,28 @@ import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
 import org.apache.carbondata.events.{DropTableCacheEvent, Event, OperationContext, OperationEventListener}
 
-object DropCacheDataMapEventListener extends OperationEventListener {
+object DropCacheMVEventListener extends OperationEventListener {
 
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
 
   /**
    * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
    */
   override protected def onEvent(event: Event, operationContext: OperationContext): Unit = {
     event match {
       case dropCacheEvent: DropTableCacheEvent =>
         val carbonTable = dropCacheEvent.carbonTable
         val sparkSession = dropCacheEvent.sparkSession
         val internalCall = dropCacheEvent.internalCall
-        if (carbonTable.isChildTableForMV && !internalCall) {
+        if (carbonTable.isMaterializedView && !internalCall) {
           throw new UnsupportedOperationException("Operation not allowed on child table.")
         }
 
         if (carbonTable.hasMVCreated) {
           val childrenSchemas = DataMapStoreManager.getInstance
             .getDataMapSchemasOfTable(carbonTable).asScala
             .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                     !dataMapSchema.isIndexDataMap)
+                                     !dataMapSchema.isIndex)
 
 Review comment:
   Remove this if check code added for mv, as drop cache handled in DropCacheCommand

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396363582
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala
 ##########
 @@ -136,11 +137,18 @@ private[sql] case class CarbonProjectForDeleteCommand(
       val allDataMapSchemas = DataMapStoreManager.getInstance
         .getDataMapSchemasOfTable(carbonTable).asScala
         .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                 !dataMapSchema.isIndexDataMap).asJava
+                                 !dataMapSchema.isIndex).asJava
 
 Review comment:
   Can remove code from line 137 to 143 added for mv, after this refactory

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396974363
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
 ##########
 @@ -203,10 +204,18 @@ private[sql] case class CarbonProjectForUpdateCommand(
       val allDataMapSchemas = DataMapStoreManager.getInstance
         .getDataMapSchemasOfTable(carbonTable).asScala
         .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                 !dataMapSchema.isIndexDataMap).asJava
+                                 !dataMapSchema.isIndex).asJava
 
 Review comment:
   Yes, will be delete when clean the old mv implementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r393458926
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatus.java
 ##########
 @@ -18,7 +18,7 @@
 package org.apache.carbondata.core.datamap.status;
 
 /**
- * DataMap status
+ * Index status
  */
 public enum DataMapStatus {
 
 Review comment:
   better to rename the file itself

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r399970121
 
 

 ##########
 File path: docs/index-developer-guide.md
 ##########
 @@ -24,11 +24,11 @@ Currently, there are two types of DataMap supported:
 2. MVDataMap: DataMap that leverages Materialized View to accelerate olap style query, like SPJG query (select, predicate, join, groupby). Preaggregate, timeseries and mv DataMap belong to this type of DataMaps.
 
 Review comment:
   Also documents [*.md] files need to be updated for all the changes of this PR
   Example, index-developer-guide.md has many datamap words in it still
   
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r399971835
 
 

 ##########
 File path: integration/spark/src/test/scala/org/apache/carbondata/view/MaterializedViewTest.scala
 ##########
 @@ -0,0 +1,194 @@
+package org.apache.carbondata.view
+
+import java.io.File
+
+import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException
+import org.apache.spark.sql.catalyst.catalog.HiveTableRelation
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.spark.exception.ProcessMetaDataException
+
+class MaterializedViewTest extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+    drop()
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
+    val projectPath = new File(this.getClass.getResource("/").getPath + "../../../../")
+      .getCanonicalPath.replaceAll("\\\\", "/")
+    val integrationPath = s"$projectPath/integration"
+    val resourcesPath = s"$integrationPath/spark/src/test/resources"
+    sql(
+      """
+        | CREATE TABLE fact_table (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)
+        | STORED AS carbondata
+      """.stripMargin)
+    sql(s"""LOAD DATA local inpath '$resourcesPath/data_big.csv' INTO TABLE fact_table OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
+    sql(s"""LOAD DATA local inpath '$resourcesPath/data_big.csv' INTO TABLE fact_table OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
+  }
+
+  test("test create mv on hive table") {
+    sql("drop materialized view if exists mv1")
+    sql("drop table if exists source")
+    sql("create table source as select * from fact_table")
+    sql("create materialized view mv1 as select empname, deptname, avg(salary) from source group by empname, deptname")
 
 Review comment:
   Now that hive mv and our mv syntax is same. what is the impact ?  On hive table even if we want to create hive mv. Does it create carbon mv table ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401323108
 
 

 ##########
 File path: mv/plan/src/main/scala/org/apache/carbondata/mv/plans/util/BirdcageOptimizer.scala
 ##########
 @@ -124,8 +123,9 @@ object BirdcageOptimizer extends RuleExecutor[LogicalPlan] {
     //    Batch("LocalRelation", fixedPoint,
     //      ConvertToLocalRelation,
     //      PropagateEmptyRelation) ::
-    Batch(
-      "OptimizeCodegen", Once, CarbonToSparkAdapter.getOptimizeCodegenRule(): _*) ::
+    // As per SPARK-22520 OptimizeCodegen is removed in 2.3.1
 
 Review comment:
   remove it if not required

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401311666
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
 ##########
 @@ -160,29 +161,29 @@ object CarbonEnv {
   def initListeners(): Unit = {
     OperationListenerBus.getInstance()
       .addListener(classOf[IndexServerLoadEvent], PrePrimingEventListener)
-      .addListener(classOf[LoadTablePreExecutionEvent], LoadMVTablePreListener)
-      .addListener(classOf[AlterTableCompactionPreStatusUpdateEvent],
-        AlterDataMaptableCompactionPostListener)
+//      .addListener(classOf[LoadTablePreExecutionEvent], LoadMVTablePreListener)
 
 Review comment:
    if code is not needed, remove it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-603732720
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/847/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-599918677
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/777/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401378493
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/view/MaterializedViewManagerInSpark.scala
 ##########
 @@ -0,0 +1,62 @@
+/*
+ * 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.view
+
+import java.util
+
+import org.apache.spark.sql.{CarbonUtils, SparkSession}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.view.MaterializedViewManager
+
+class MaterializedViewManagerInSpark(session: SparkSession) extends MaterializedViewManager {
+  override def getDatabases: util.List[String] = {
+    CarbonUtils.threadSet(CarbonCommonConstants.DISABLE_SQL_REWRITE, "true")
+    try {
+      val databaseList = session.catalog.listDatabases()
+      val databaseNameList = new util.ArrayList[String]()
+      for (database <- databaseList.collect()) {
+        databaseNameList.add(database.name)
+      }
+      databaseNameList
+    } finally {
+      CarbonUtils.threadUnset(CarbonCommonConstants.DISABLE_SQL_REWRITE)
+    }
+  }
+}
+
+object MaterializedViewManagerInSpark {
+
+  private val MANAGER_MAP_BY_SESSION =
+    new util.HashMap[SparkSession, MaterializedViewManagerInSpark]()
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401377408
 
 

 ##########
 File path: mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/Harmonizer.scala
 ##########
 @@ -72,8 +69,8 @@ object HarmonizeDimensionTable extends Rule[ModularPlan] with PredicateHelper {
       case s@Select(_, _, _, _, jedges, fact :: dims, _, _, _, _) if
       jedges.forall(e => e.joinType == LeftOuter || e.joinType == Inner) &&
       fact.isInstanceOf[ModularRelation] &&
-      dims.filterNot(_.isInstanceOf[modular.LeafNode]).nonEmpty &&
-      dims.forall(d => (d.isInstanceOf[ModularRelation] || HarmonizedRelation.canHarmonize(d))) => {
+      !dims.forall(_.isInstanceOf[modular.LeafNode]) &&
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-602018425
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2532/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-598539924
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2450/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601582126
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/813/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401339656
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 ##########
 @@ -876,6 +876,15 @@ public boolean hasMVCreated() throws IOException {
         schema.getProviderName().equalsIgnoreCase(DataMapClassProvider.MV.toString()));
   }
 
+  /**
+   * Return true if this table is a MV table (child table of other table)
+   */
+  public boolean isMVTable() {
+    String parentTables = tableInfo.getFactTable().getTableProperties()
+        .get(CarbonCommonConstants.PARENT_TABLES);
 
 Review comment:
   How about changing it to MV_ON_TABlES?
   What is the difference with method isChildTableForMV/isMaterializedView
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395475971
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
 ##########
 @@ -1595,6 +1595,24 @@ public String getSystemFolderLocation() {
     return systemLocation + CarbonCommonConstants.FILE_SEPARATOR + "_system";
   }
 
+  /**
+   * Get the configured system folder location.
+   * @return
+   */
+  public String getSystemFolderLocation(String databaseName) {
+    String systemLocation = CarbonProperties.getInstance()
 
 Review comment:
   Can extract common code from getSystemFolderLocation() and getSystemFolderLocation(String databaseName) to new method

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601554134
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2517/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-600048981
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2489/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401303776
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/events/DropTableEvents.scala
 ##########
 @@ -30,7 +30,8 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 case class DropTablePreEvent(
     carbonTable: CarbonTable,
     ifExistsSet: Boolean,
-    sparkSession: SparkSession)
+    sparkSession: SparkSession,
+    isInternalCall: Boolean = false)
 
 Review comment:
   what does 'isInternalCall' mean?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-597024732
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2408/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596418009
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2395/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596414333
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/689/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-607158608
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2603/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r399968880
 
 

 ##########
 File path: dev/findbugs-exclude.xml
 ##########
 @@ -59,7 +59,7 @@
     <Bug pattern="STCAL_INVOKE_ON_STATIC_DATE_FORMAT_INSTANCE"/>
   </Match>
   <Match>
-    <Class name="org.apache.carbondata.core.datamap.DistributableDataMapFormat"/>
+    <Class name="org.apache.carbondata.core.datamap.IndexInputFormat"/>
 
 Review comment:
   Also documents [*.md] files need to be updated for all the changes of this PR

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-599920532
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2484/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-600982507
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/801/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-597955250
 
 
   retest this please

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on issue #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on issue #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-607194573
 
 
   LGTM

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r399969838
 
 

 ##########
 File path: dev/findbugs-exclude.xml
 ##########
 @@ -59,7 +59,7 @@
     <Bug pattern="STCAL_INVOKE_ON_STATIC_DATE_FORMAT_INSTANCE"/>
   </Match>
   <Match>
-    <Class name="org.apache.carbondata.core.datamap.DistributableDataMapFormat"/>
+    <Class name="org.apache.carbondata.core.datamap.IndexInputFormat"/>
 
 Review comment:
   Example, index-developer-guide.md has many datamap words in it still

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-597971616
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/724/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396957216
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapClassProvider.java
 ##########
 @@ -18,26 +18,20 @@
 package org.apache.carbondata.core.metadata.schema.datamap;
 
 /**
- * type for create datamap
- * The syntax of datamap creation is as follows.
- * CREATE DATAMAP IF NOT EXISTS dataMapName ON TABLE tableName USING 'DataMapClassProvider'
- * DMPROPERTIES('KEY'='VALUE') AS SELECT COUNT(COL1) FROM tableName
- *
  * Please refer {{org.apache.spark.sql.parser.CarbonSpark2SqlParser}}
  */
-
 public enum DataMapClassProvider {
-  LUCENE("org.apache.carbondata.datamap.lucene.LuceneFineGrainDataMapFactory", "lucene"),
-  BLOOMFILTER("org.apache.carbondata.datamap.bloom.BloomCoarseGrainDataMapFactory", "bloomfilter"),
+  LUCENE("org.apache.carbondata.datamap.lucene.LuceneFineGrainIndexFactory", "lucene"),
+  BLOOMFILTER("org.apache.carbondata.datamap.bloom.BloomCoarseGrainIndexFactory", "bloomfilter"),
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601004166
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/802/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396333133
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/view/MaterializedViewProvider.java
 ##########
 @@ -0,0 +1,573 @@
+/*
+ * 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.core.view;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperationFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
+import org.apache.carbondata.core.fileoperations.FileWriteOperation;
+import org.apache.carbondata.core.locks.CarbonLockFactory;
+import org.apache.carbondata.core.locks.CarbonLockUtil;
+import org.apache.carbondata.core.locks.ICarbonLock;
+import org.apache.carbondata.core.locks.LockUsage;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatus;
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+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;
+import org.apache.log4j.Logger;
+
+/**
+ * Stores mv schema in disk as json format
+ */
+@InterfaceAudience.Internal
+public class MaterializedViewProvider {
+
+  private static final Logger LOG = LogServiceFactory.getLogService(
+      MaterializedViewProvider.class.getCanonicalName());
+
+  private static final String STATUS_FILE_NAME = "mv_status";
+
+  private final String storeLocation;
+
+  private final Map<String, SchemaProvider> schemaProviders = new ConcurrentHashMap<>();
+
+  private MaterializedViewProvider(String storeLocation) {
+    this.storeLocation = storeLocation;
+  }
+
+  public static MaterializedViewProvider get() {
+    String storeLocation =
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION);
+    if (storeLocation == null) {
+      throw new RuntimeException(
+          "Property [" + CarbonCommonConstants.STORE_LOCATION + "] is not set.");
+    }
+    return new MaterializedViewProvider(storeLocation);
+  }
+
+  private static String getSchemaPath(String schemaRoot, String viewName) {
+    return schemaRoot + CarbonCommonConstants.FILE_SEPARATOR + "mv_schema." + viewName;
+  }
+
+  private SchemaProvider getSchemaProvider(String databaseName) {
+    String databaseNameUpper = databaseName.toUpperCase();
+    SchemaProvider schemaProvider = this.schemaProviders.get(databaseNameUpper);
+    if (schemaProvider == null) {
+      synchronized (this.schemaProviders) {
+        schemaProvider = this.schemaProviders.get(databaseNameUpper);
+        if (schemaProvider == null) {
+          String databaseLocation;
+          if (databaseNameUpper.equalsIgnoreCase("default")) {
+            databaseLocation = CarbonUtil.checkAndAppendHDFSUrl(this.storeLocation);
+          } else {
+            databaseLocation = CarbonUtil.checkAndAppendHDFSUrl(this.storeLocation +
+                CarbonCommonConstants.FILE_SEPARATOR + databaseName + ".db");
+          }
+          if (!FileFactory.getCarbonFile(databaseLocation).exists()) {
+            return null;
+          }
+          schemaProvider = new SchemaProvider(databaseLocation);
+          this.schemaProviders.put(databaseNameUpper, schemaProvider);
+        }
+      }
+    }
+    return schemaProvider;
+  }
+
+  public MaterializedViewSchema getSchema(MaterializedViewManager viewManager,
+      String databaseName, String viewName) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      return null;
+    }
+    return schemaProvider.retrieveSchema(viewManager, viewName);
+  }
+
+  List<MaterializedViewSchema> getSchemas(MaterializedViewManager viewManager,
+      String databaseName, CarbonTable carbonTable) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      return Collections.emptyList();
+    } else {
+      return schemaProvider.retrieveSchemas(viewManager, carbonTable);
+    }
+  }
+
+  List<MaterializedViewSchema> getSchemas(MaterializedViewManager viewManager,
+      String databaseName) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      return Collections.emptyList();
+    } else {
+      return schemaProvider.retrieveAllSchemas(viewManager);
+    }
+  }
+
+  void saveSchema(MaterializedViewManager viewManager, String databaseName,
+      MaterializedViewSchema viewSchema) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      throw new IOException("Database [" + databaseName + "] is not found.");
+    }
+    schemaProvider.saveSchema(viewManager, viewSchema);
+  }
+
+  public void dropSchema(String databaseName, String viewName)
+      throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      throw new IOException("Materialized view with name " + databaseName + "." + viewName +
+          " does not exists in storage");
+    }
+    schemaProvider.dropSchema(viewName);
+  }
+
+  private String getStatusFileName(String databaseName) {
+    if (databaseName.equalsIgnoreCase("default")) {
+      return this.storeLocation +
+          CarbonCommonConstants.FILE_SEPARATOR + "_system" +
+          CarbonCommonConstants.FILE_SEPARATOR + STATUS_FILE_NAME;
+    } else {
+      return this.storeLocation +
+          CarbonCommonConstants.FILE_SEPARATOR + databaseName + ".db" +
+          CarbonCommonConstants.FILE_SEPARATOR + "_system" +
+          CarbonCommonConstants.FILE_SEPARATOR + STATUS_FILE_NAME;
+    }
+  }
+
+  public List<MaterializedViewStatusDetail> getStatusDetails(String databaseName)
+      throws IOException {
+    String statusPath = this.getStatusFileName(databaseName);
+    Gson gsonObjectToRead = new Gson();
+    DataInputStream dataInputStream = null;
+    BufferedReader buffReader = null;
+    InputStreamReader inStream = null;
+    MaterializedViewStatusDetail[] statusDetails;
+    try {
+      if (!FileFactory.isFileExist(statusPath)) {
+        return Collections.emptyList();
+      }
+      dataInputStream = FileFactory.getDataInputStream(statusPath);
+      inStream = new InputStreamReader(dataInputStream,
+          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+      buffReader = new BufferedReader(inStream);
+      statusDetails = gsonObjectToRead.fromJson(buffReader,
+          MaterializedViewStatusDetail[].class);
+    } catch (IOException e) {
+      LOG.error("Failed to read datamap status", e);
+      throw e;
+    } finally {
+      CarbonUtil.closeStreams(buffReader, inStream, dataInputStream);
+    }
+    // if status details is null, return empty array
+    if (null == statusDetails) {
+      return Collections.emptyList();
+    }
+    return Arrays.asList(statusDetails);
+  }
+
+  private static ICarbonLock getStatusLock(String databaseName) {
+    return CarbonLockFactory
+        .getSystemLevelCarbonLockObj(
+            CarbonProperties.getInstance().getSystemFolderLocation(databaseName),
+            LockUsage.MATERIALIZED_VIEW_STATUS_LOCK);
+  }
+
+  /**
+   * Update or add the status of passed mvs with the given mv status. If the mv status
+   * given is enabled/disabled then updates/adds the mv, in case of drop it just removes it
+   * from the file.
+   * This method always overwrites the old file.
+   * @param schemaList schemas of which are need to be updated in mv status
+   * @param status  status to be updated for the mv schemas
+   */
+  public void updateStatus(List<MaterializedViewSchema> schemaList,
+      MaterializedViewStatus status) throws IOException {
+    if (schemaList == null || schemaList.size() == 0) {
+      // There is nothing to update
+      return;
+    }
+    final Map<String, List<MaterializedViewSchema>> schemasMapByDatabase = new HashMap<>();
+    for (MaterializedViewSchema schema : schemaList) {
+      String databaseName = schema.getIdentifier().getDatabaseName().toLowerCase();
+      List<MaterializedViewSchema> schemas = schemasMapByDatabase.get(databaseName);
+      if (schemas == null) {
+        schemas = new ArrayList<>();
+        schemasMapByDatabase.put(databaseName, schemas);
+      }
+      schemas.add(schema);
+    }
+    for (Map.Entry<String, List<MaterializedViewSchema>> entry : schemasMapByDatabase.entrySet()) {
+      this.updateStatus(entry.getKey(), entry.getValue(), status);
+    }
+  }
+
+  private void updateStatus(String databaseName, List<MaterializedViewSchema> schemaList,
+      MaterializedViewStatus status) throws IOException {
+    ICarbonLock carbonTableStatusLock = getStatusLock(databaseName);
+    boolean locked = false;
+    try {
+      locked = carbonTableStatusLock.lockWithRetries();
+      if (locked) {
+        LOG.info("Materialized view status lock has been successfully acquired.");
+        if (status == MaterializedViewStatus.ENABLED) {
+          // Enable mv only if mv tables and main table are in sync
+          if (!isViewCanBeEnabled(schemaList.get(0))) {
+            return;
+          }
+        }
+        List<MaterializedViewStatusDetail> statusDetailList =
+            new ArrayList<>(getStatusDetails(databaseName));
+        List<MaterializedViewStatusDetail> changedStatusDetails = new ArrayList<>();
+        List<MaterializedViewStatusDetail> newStatusDetails = new ArrayList<>();
+        for (MaterializedViewSchema schema : schemaList) {
+          boolean exists = false;
+          for (MaterializedViewStatusDetail statusDetail : statusDetailList) {
+            if (statusDetail.getIdentifier().equals(schema.getIdentifier())) {
+              statusDetail.setStatus(status);
+              changedStatusDetails.add(statusDetail);
+              exists = true;
+            }
+          }
+          if (!exists) {
+            newStatusDetails
+                .add(new MaterializedViewStatusDetail(schema.getIdentifier(),
+                    status));
+          }
+        }
+        // Add the newly added datamaps to the list.
+        if (newStatusDetails.size() > 0 &&
+            status != MaterializedViewStatus.DROPPED) {
+          statusDetailList.addAll(newStatusDetails);
+        }
+        // In case of dropped datamap, just remove from the list.
+        if (status == MaterializedViewStatus.DROPPED) {
+          statusDetailList.removeAll(changedStatusDetails);
+        }
+        writeLoadDetailsIntoFile(
+            this.getStatusFileName(databaseName),
+            statusDetailList.toArray(
+                new MaterializedViewStatusDetail[statusDetailList.size()]));
+      } else {
+        String errorMsg = "Upadating datamapstatus is failed due to another process taken the lock"
+            + " for updating it";
+        LOG.error(errorMsg);
+        throw new IOException(errorMsg + " Please try after some time.");
+      }
+    } finally {
+      if (locked) {
+        CarbonLockUtil.fileUnlock(carbonTableStatusLock, LockUsage.DATAMAP_STATUS_LOCK);
+      }
+    }
+  }
+
+  /**
+   * writes mv status details
+   */
+  private static void writeLoadDetailsIntoFile(String location,
+      MaterializedViewStatusDetail[] statusDetails) throws IOException {
+    FileFactory.touchFile(FileFactory.getCarbonFile(location),
+        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
+    AtomicFileOperations fileWrite = AtomicFileOperationFactory.getAtomicFileOperations(location);
+    BufferedWriter brWriter = null;
+    DataOutputStream dataOutputStream = null;
+    Gson gsonObjectToWrite = new Gson();
+    // write the updated data into the mv status file.
+    try {
+      dataOutputStream = fileWrite.openForWrite(FileWriteOperation.OVERWRITE);
+      brWriter = new BufferedWriter(new OutputStreamWriter(dataOutputStream,
+          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
+
+      String metadataInstance = gsonObjectToWrite.toJson(statusDetails);
+      brWriter.write(metadataInstance);
+    } catch (IOException ioe) {
+      LOG.error("Error message: " + ioe.getLocalizedMessage());
+      fileWrite.setFailed();
+      throw ioe;
+    } finally {
+      if (null != brWriter) {
+        brWriter.flush();
+      }
+      CarbonUtil.closeStreams(brWriter);
+      fileWrite.close();
+    }
+
+  }
+
+  /**
+   * This method checks if main table and mv table are synchronised or not. If synchronised
+   * return true to enable the mv
+   *
+   * @param schema of mv to be disabled or enabled
+   * @return flag to enable or disable mv
+   * @throws IOException
+   */
+  private static boolean isViewCanBeEnabled(MaterializedViewSchema schema)
+      throws IOException {
+    if (!schema.isRefreshIncremental()) {
+      return true;
+    }
+    boolean isViewCanBeEnabled = true;
+    String viewMetadataPath =
+        CarbonTablePath.getMetadataPath(schema.getIdentifier().getTablePath());
+    LoadMetadataDetails[] viewLoadMetadataDetails =
+        SegmentStatusManager.readLoadMetadata(viewMetadataPath);
+    Map<String, List<String>> viewSegmentMap = new HashMap<>();
+    for (LoadMetadataDetails loadMetadataDetail : viewLoadMetadataDetails) {
+      if (loadMetadataDetail.getSegmentStatus() == SegmentStatus.SUCCESS) {
+        Map<String, List<String>> segmentMap =
+            new Gson().fromJson(loadMetadataDetail.getExtraInfo(), Map.class);
+        if (viewSegmentMap.isEmpty()) {
+          viewSegmentMap.putAll(segmentMap);
+        } else {
+          for (Map.Entry<String, List<String>> entry : segmentMap.entrySet()) {
+            if (null != viewSegmentMap.get(entry.getKey())) {
+              viewSegmentMap.get(entry.getKey()).addAll(entry.getValue());
+            }
+          }
+        }
+      }
+    }
+    List<RelationIdentifier> associatedTables = schema.getAssociatedTables();
+    for (RelationIdentifier associatedTable : associatedTables) {
+      List<String> associatedTableSegmentList =
+          SegmentStatusManager.getValidSegmentList(associatedTable);
+      if (!associatedTableSegmentList.isEmpty()) {
+        if (viewSegmentMap.isEmpty()) {
+          isViewCanBeEnabled = false;
+        } else {
+          isViewCanBeEnabled = viewSegmentMap.get(
+              associatedTable.getDatabaseName() + CarbonCommonConstants.POINT +
+                  associatedTable.getTableName()).containsAll(associatedTableSegmentList);
+        }
+      }
+    }
+    return isViewCanBeEnabled;
+  }
+
+  /**
+   * Data map schema provider of a database.
+   */
+  private static final class SchemaProvider {
+
+    private String systemDirectory;
+
+    private String schemaIndexFilePath;
 
 Review comment:
   Can change the variable name, as it indicates schema modified file path and can variable description

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601533651
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2513/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-598126802
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/730/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395489746
 
 

 ##########
 File path: mv/core/src/main/scala/org/apache/carbondata/mv/extension/MVHelper.scala
 ##########
 @@ -631,7 +630,7 @@ object MVHelper {
       val newLongStringColumn = longStringColumn.get.split(",").map(_.trim).map { colName =>
         val newColName = parentTable.getTableName.toLowerCase() + "_" + colName
         if (!fieldNames.contains(newColName)) {
-          throw new MalformedDataMapCommandException(
+          throw new MalformedIndexCommandException(
 
 Review comment:
   Change exception to mv

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396975897
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
 ##########
 @@ -101,6 +100,12 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     loadDataNew | explainPlan | alterTableColumnRenameAndModifyDataType |
     alterTableAddColumns
 
+  protected lazy val materializedViewCommands: Parser[LogicalPlan] =
 
 Review comment:
   MVParser never used now, MVParser will be delete when clean the old mv implementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396366570
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
 ##########
 @@ -65,20 +67,30 @@ case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[Logica
       val projList = Seq(UnresolvedAlias(UnresolvedStar(alias.map(Seq(_)))), tupleId)
       val carbonTable = CarbonEnv.getCarbonTable(table.tableIdentifier)(sparkSession)
       if (carbonTable != null) {
+        val viewManager = MaterializedViewManagerInSpark.get(sparkSession)
+        val viewSchemas = viewManager.getSchemasOnTable(carbonTable)
+        if (!viewSchemas.isEmpty) {
+          viewSchemas.asScala.foreach { schema =>
+            viewManager.setStatus(
+              schema.getIdentifier,
+              MaterializedViewStatus.DISABLED
+            )
+          }
+        }
         val indexSchemas = DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable)
         if (carbonTable.hasMVCreated) {
           val allDataMapSchemas = DataMapStoreManager.getInstance
             .getDataMapSchemasOfTable(carbonTable).asScala
             .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                     !dataMapSchema.isIndexDataMap).asJava
+                                     !dataMapSchema.isIndex).asJava
 
 Review comment:
   remove this if check code

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401357265
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 ##########
 @@ -876,6 +876,15 @@ public boolean hasMVCreated() throws IOException {
         schema.getProviderName().equalsIgnoreCase(DataMapClassProvider.MV.toString()));
   }
 
+  /**
+   * Return true if this table is a MV table (child table of other table)
+   */
+  public boolean isMVTable() {
+    String parentTables = tableInfo.getFactTable().getTableProperties()
+        .get(CarbonCommonConstants.PARENT_TABLES);
 
 Review comment:
   the 'isChildTableForMV' is a method of old mv implementation, it will be removed when clean old mv implementation, we will do it in a new pr.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-598198315
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2440/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396953401
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/IndexUtil.java
 ##########
 @@ -185,38 +185,37 @@ public static void pruneSegments(List<Segment> segments, List<ExtendedBlocklet>
    Loads the datamaps in parallel by utilizing executor
    *
    @param carbonTable
-   @param dataMapExprWrapper
+   @param indexExprWrapper
    @param validSegments
    @param partitionsToPrune
    @throws IOException
    */
-  public static void loadDataMaps(CarbonTable carbonTable, DataMapExprWrapper dataMapExprWrapper,
+  public static void loadDataMaps(CarbonTable carbonTable, IndexExprWrapper indexExprWrapper,
       List<Segment> validSegments, List<PartitionSpec> partitionsToPrune) throws IOException {
     if (!CarbonProperties.getInstance()
         .isDistributedPruningEnabled(carbonTable.getDatabaseName(), carbonTable.getTableName())
-        && BlockletDataMapUtil.loadDataMapsParallel(carbonTable)) {
-      String clsName = "org.apache.spark.sql.secondaryindex.Jobs.SparkBlockletDataMapLoaderJob";
-      DataMapJob dataMapJob = (DataMapJob) createDataMapJob(clsName);
-      String className =
-          "org.apache.spark.sql.secondaryindex.Jobs.DistributableBlockletDataMapLoader";
+        && BlockletIndexUtil.loadDataMapsParallel(carbonTable)) {
+      String clsName = "org.apache.carbondata.spark.rdd.SparkBlockletDataMapLoaderJob";
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401355488
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/events/DropTableEvents.scala
 ##########
 @@ -30,7 +30,8 @@ import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 case class DropTablePreEvent(
     carbonTable: CarbonTable,
     ifExistsSet: Boolean,
-    sparkSession: SparkSession)
+    sparkSession: SparkSession,
+    isInternalCall: Boolean = false)
 
 Review comment:
   'isInternalCall' is mean trigger by other command, not by user directly.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-597535058
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/715/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396354903
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/view/MaterializedViewRefresher.scala
 ##########
 @@ -0,0 +1,398 @@
+/*
+ * 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.view
+
+import java._
+import java.io.IOException
+
+import scala.collection.JavaConverters._
+
+import com.google.gson.Gson
+import org.apache.log4j.Logger
+import org.apache.spark.sql.{CarbonUtils, SparkSession}
+import org.apache.spark.sql.execution.command.management.CarbonInsertIntoCommand
+import org.apache.spark.sql.parser.MaterializedViewQueryParser
+
+import org.apache.carbondata.common.exceptions.sql.NoSuchMaterializedViewException
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.locks.ICarbonLock
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, RelationIdentifier}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.core.view.{MaterializedViewSchema, MaterializedViewStatus}
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
+
+class MaterializedViewRefresher{
+
+}
+
+object MaterializedViewRefresher {
+
+  private val LOGGER: Logger = LogServiceFactory.getLogService(
+    classOf[MaterializedViewRefresher].getCanonicalName)
+
+  /**
+   * Refresh the mv by loading all existing data from associated table
+   * This is called when refreshing the mv when
+   * 1. after mv creation and no "WITH DEFERRED REBUILD" defined
+   * 2. user manually trigger REFRESH MATERIALIZED VIEW command
+   */
+  @throws[IOException]
+  @throws[NoSuchMaterializedViewException]
+  def refresh(viewSchema: MaterializedViewSchema, session: SparkSession): Boolean = {
+    var newLoadName: String = ""
+    var segmentMap: String = ""
+    val viewTable: CarbonTable = CarbonTable.buildFromTablePath(
+      viewSchema.getIdentifier.getTableName,
+      viewSchema.getIdentifier.getDatabaseName,
+      viewSchema.getIdentifier.getTablePath,
+      viewSchema.getIdentifier.getTableId)
+    val viewIdentifier = viewSchema.getIdentifier
+    val viewTableIdentifier = viewTable.getAbsoluteTableIdentifier
+    // Clean up the old invalid segment data before creating a new entry for new load.
+    SegmentStatusManager.deleteLoadsAndUpdateMetadata(viewTable, false, null)
+    val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(viewTableIdentifier)
+    // Acquire table status lock to handle concurrent dataloading
+    val lock: ICarbonLock = segmentStatusManager.getTableStatusLock
+    val segmentMapping: util.Map[String, util.List[String]] =
+      new util.HashMap[String, util.List[String]]
+    val viewManager = MaterializedViewManagerInSpark.get(session)
+    try if (lock.lockWithRetries) {
+      LOGGER.info("Acquired lock for mv " + viewIdentifier + " for table status updation")
+      val viewTableMetadataPath: String =
+        CarbonTablePath.getMetadataPath(viewIdentifier.getTablePath)
+      val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTableMetadataPath)
+      val loadMetadataDetailList: util.List[LoadMetadataDetails] =
+        new util.ArrayList[LoadMetadataDetails](CarbonCommonConstants.DEFAULT_COLLECTION_SIZE)
+      // Mark for delete all stale loadMetadetail
+      for (loadMetadataDetail <- loadMetadataDetails) {
+        if (((loadMetadataDetail.getSegmentStatus eq SegmentStatus.INSERT_IN_PROGRESS) ||
+             (loadMetadataDetail.getSegmentStatus eq SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS)) &&
+            loadMetadataDetail.getVisibility.equalsIgnoreCase("false")) {
+          loadMetadataDetail.setSegmentStatus(SegmentStatus.MARKED_FOR_DELETE)
+        }
+        loadMetadataDetailList.add(loadMetadataDetail)
+      }
+      if (viewSchema.isRefreshOnManual) {
+        // check if rebuild to mv is already in progress and throw exception
+        if (loadMetadataDetails.nonEmpty) {
+          for (loadMetaDetail <- loadMetadataDetails) {
+            if (((loadMetaDetail.getSegmentStatus eq SegmentStatus.INSERT_IN_PROGRESS) ||
+                 (loadMetaDetail.getSegmentStatus eq SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS)) &&
+                SegmentStatusManager.isLoadInProgress(viewTableIdentifier,
+                  loadMetaDetail.getLoadName)) {
+              throw new RuntimeException(
+                "Rebuild to materialized view " + viewSchema.getIdentifier.getTableName +
+                  " is already in progress")
+            }
+          }
+        }
+      }
+      if (viewSchema.isRefreshIncremental) {
+        if (!getSpecificSegmentsTobeLoaded(viewSchema, segmentMapping, loadMetadataDetailList)) {
+          return false
+        }
+      } else {
+          // set segment mapping only for carbondata table
+          val associatedTableIds =
+            viewSchema.getAssociatedTables.asScala.filter(_.isCarbonDataTable)
+          for (associatedTableId <- associatedTableIds) {
+            val associatedTableSegmentList: util.List[String] =
+              SegmentStatusManager.getValidSegmentList(associatedTableId)
+            if (associatedTableSegmentList.isEmpty) {
+              return false
+            }
+            segmentMapping.put(associatedTableId.toString, associatedTableSegmentList)
+          }
+        }
+      segmentMap = new Gson().toJson(segmentMapping)
+      // To handle concurrent dataloading to mv, create new loadMetaEntry and
+      // set segmentMap to new loadMetaEntry and pass new segmentId with load command
+      val loadMetadataDetail: LoadMetadataDetails = new LoadMetadataDetails
+      val segmentId: String = String.valueOf(
+        SegmentStatusManager.createNewSegmentId(loadMetadataDetails))
+      loadMetadataDetail.setLoadName(segmentId)
+      loadMetadataDetail.setSegmentStatus(SegmentStatus.INSERT_IN_PROGRESS)
+      loadMetadataDetail.setExtraInfo(segmentMap)
+      loadMetadataDetailList.add(loadMetadataDetail)
+      newLoadName = segmentId
+      SegmentStatusManager.writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(
+        viewSchema.getIdentifier.getTablePath),
+        loadMetadataDetailList.toArray(new Array[LoadMetadataDetails](loadMetadataDetailList
+          .size)))
+    } else {
+      LOGGER.error("Not able to acquire the lock for Table status updation for table " +
+                   viewSchema.getIdentifier.getDatabaseName + "." +
+                   viewSchema.getIdentifier.getTableName)
+      viewManager.setStatus(viewSchema.getIdentifier, MaterializedViewStatus.DISABLED)
+      return false
+    } finally {
+      if (lock.unlock) {
+        LOGGER.info("Table unlocked successfully after table status updation" +
+                    viewSchema.getIdentifier.getDatabaseName + "." +
+                    viewSchema.getIdentifier.getTableName)
+      } else {
+        LOGGER.error("Unable to unlock Table lock for table" +
+                     viewSchema.getIdentifier.getDatabaseName + "." +
+                     viewSchema.getIdentifier.getTableName +
+                     " during table status updation")
+      }
+    }
+    refreshInternal(viewManager, viewSchema, viewTable, newLoadName, segmentMapping, session)
+  }
+
+  @throws[IOException]
+  private def refreshInternal(
+      viewManager: MaterializedViewManagerInSpark,
+      viewSchema: MaterializedViewSchema,
+      viewTable: CarbonTable,
+      newLoadName: String,
+      segmentMap: java.util.Map[String, java.util.List[String]],
+      session: SparkSession): Boolean = {
+    val query = viewSchema.getQuery
+    if (query != null) {
+      val viewIdentifier = viewSchema.getIdentifier
+      val updatedQuery = MaterializedViewQueryParser.getQuery(query, session)
+      val isFullRefresh = !viewSchema.isRefreshIncremental
+      // Set specified segments for incremental load
+      val segmentMapIterator = segmentMap.entrySet().iterator()
+      while (segmentMapIterator.hasNext) {
+        val entry = segmentMapIterator.next()
+        setInputSegments(entry.getKey, entry.getValue)
+      }
+      val header = viewTable.getTableInfo.getFactTable.getListOfColumns.asScala
+        .filter { column =>
+          !column.getColumnName
+            .equalsIgnoreCase(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE)
+        }.sortBy(_.getSchemaOrdinal).map(_.getColumnName).mkString(",")
+      val insertIntoCommand = CarbonInsertIntoCommand(
+        databaseNameOp = Some(viewIdentifier.getDatabaseName),
+        tableName = viewIdentifier.getTableName,
+        options = scala.collection.immutable.Map("fileheader" -> header),
+        isFullRefresh,
+        logicalPlan = updatedQuery.queryExecution.analyzed,
+        tableInfo = viewTable.getTableInfo,
+        internalOptions = Map("mergedSegmentName" -> newLoadName,
+          CarbonCommonConstants.IS_INTERNAL_LOAD_CALL -> "true"),
+        partition = Map.empty)
+      try {
+        insertIntoCommand.run(session)
+      } catch {
+        case exception: Exception =>
+          // If load to dataMap table fails, disable the dataMap and if newLoad is still
+          // in INSERT_IN_PROGRESS state, mark for delete the newLoad and update table status file
+          viewManager.setStatus(viewSchema.getIdentifier, MaterializedViewStatus.DISABLED)
+          LOGGER.error("Data Load failed for DataMap: ", exception)
+          CarbonLoaderUtil.updateTableStatusInCaseOfFailure(
+            newLoadName,
+            viewTable.getAbsoluteTableIdentifier,
+            viewTable.getTableName,
+            viewTable.getDatabaseName,
+            viewTable.getTablePath,
+            viewTable.getMetadataPath)
+          throw exception
+      } finally {
+        unsetInputSegments(viewSchema)
+      }
+    }
+    true
+  }
+
+  /**
+   * This method will compare mainTable and dataMapTable segment List and loads only newly added
+   * segment from main table to dataMap table.
+   * In case if mainTable is compacted, then based on dataMap to mainTables segmentMapping, dataMap
+   * will be loaded
+   * Eg:
+   * case 1: Consider mainTableSegmentList: {0, 1, 2}, dataMapToMainTable segmentMap:
+   * { 0 -> 0, 1-> 1,2}. If (1, 2) segments of main table are compacted to 1.1 and new segment (3)
+   * is loaded to main table, then mainTableSegmentList will be updated to{0, 1.1, 3}.
+   * In this case, segment (1) of dataMap table will be marked for delete, and new segment
+   * {2 -> 1.1, 3} will be loaded to dataMap table
+   * case 2: Consider mainTableSegmentList: {0, 1, 2, 3}, dataMapToMainTable segmentMap:
+   * { 0 -> 0,1,2, 1-> 3}. If (1, 2) segments of main table are compacted to 1.1 and new segment
+   * (4) is loaded to main table, then mainTableSegmentList will be updated to {0, 1.1, 3, 4}.
+   * In this case, segment (0) of dataMap table will be marked for delete and segment (0) of
+   * main table will be added to validSegmentList which needs to be loaded again. Now, new dataMap
+   * table segment (2) with main table segmentList{2 -> 1.1, 4, 0} will be loaded to dataMap table.
+   * dataMapToMainTable segmentMap will be updated to {1 -> 3, 2 -> 1.1, 4, 0} after rebuild
+   */
+  @throws[IOException]
+  private def getSpecificSegmentsTobeLoaded(schema: MaterializedViewSchema,
+      segmentMapping: util.Map[String, util.List[String]],
+      listOfLoadFolderDetails: util.List[LoadMetadataDetails]): Boolean = {
+    val relationIdentifiers: util.List[RelationIdentifier] = schema.getAssociatedTables
+    // invalidDataMapSegmentList holds segment list which needs to be marked for delete
+    val invalidDataMapSegmentList: util.HashSet[String] = new util.HashSet[String]
 
 Review comment:
   Can check and rename all variables to mv

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395427793
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -460,7 +460,7 @@ private CarbonCommonConstants() {
   public static final String LOCAL_DICTIONARY_EXCLUDE = "local_dictionary_exclude";
 
   /**
-   * DMPROPERTY for Index DataMap, like lucene, bloomfilter DataMap,
+   * DMPROPERTY for Index Index, like lucene, bloomfilter Index,
 
 Review comment:
   remove repeated Index

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396974855
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/DDLHelper.scala
 ##########
 @@ -379,13 +380,17 @@ object DDLHelper {
           throw new MalformedCarbonCommandException(
             "Streaming property value is incorrect")
         }
-        if (carbonTable.hasMVCreated) {
+        if (carbonTable.isMaterializedView) {
+          throw new MalformedCarbonCommandException(
+            "Datamap table does not support set streaming property")
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-597094204
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2411/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395968044
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 ##########
 @@ -885,6 +894,16 @@ public boolean isChildTableForMV() {
         .get(CarbonCommonConstants.PARENT_TABLES).isEmpty();
   }
 
+  /**
+   * Return true if this table is a MV table (child table of other table)
+   */
+  public boolean isMaterializedView() {
+    return tableInfo.getFactTable().getTableProperties()
+        .get(CarbonCommonConstants.ASSOCIATED_TABLES) != null &&
+        !tableInfo.getFactTable().getTableProperties()
+        .get(CarbonCommonConstants.ASSOCIATED_TABLES).isEmpty();
 
 Review comment:
   Yes, both are same,  the isMVTable method will be delete when clean the old mv implementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396366706
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
 ##########
 @@ -207,16 +219,26 @@ case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[Logica
         val projList = Seq(UnresolvedAlias(UnresolvedStar(alias.map(Seq(_)))), tupleId)
         val carbonTable = CarbonEnv.getCarbonTable(table.tableIdentifier)(sparkSession)
         if (carbonTable != null) {
-          if (carbonTable.isChildTableForMV) {
+          if (carbonTable.isMaterializedView) {
             throw new UnsupportedOperationException(
               "Delete operation is not supported for datamap table")
           }
+          val viewManager = MaterializedViewManagerInSpark.get(sparkSession)
+          val viewSchemas = viewManager.getSchemasOnTable(carbonTable)
+          if (!viewSchemas.isEmpty) {
+            viewSchemas.asScala.foreach { schema =>
+              viewManager.setStatus(
+                schema.getIdentifier,
+                MaterializedViewStatus.DISABLED
+              )
+            }
+          }
           val indexSchemas = DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable)
           if (carbonTable.hasMVCreated) {
             val allDataMapSchemas = DataMapStoreManager.getInstance
               .getDataMapSchemasOfTable(carbonTable).asScala
               .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                       !dataMapSchema.isIndexDataMap).asJava
+                                       !dataMapSchema.isIndex).asJava
 
 Review comment:
   remove this if check code

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396975509
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
 ##########
 @@ -207,16 +219,26 @@ case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[Logica
         val projList = Seq(UnresolvedAlias(UnresolvedStar(alias.map(Seq(_)))), tupleId)
         val carbonTable = CarbonEnv.getCarbonTable(table.tableIdentifier)(sparkSession)
         if (carbonTable != null) {
-          if (carbonTable.isChildTableForMV) {
+          if (carbonTable.isMaterializedView) {
             throw new UnsupportedOperationException(
               "Delete operation is not supported for datamap table")
           }
+          val viewManager = MaterializedViewManagerInSpark.get(sparkSession)
+          val viewSchemas = viewManager.getSchemasOnTable(carbonTable)
+          if (!viewSchemas.isEmpty) {
+            viewSchemas.asScala.foreach { schema =>
+              viewManager.setStatus(
+                schema.getIdentifier,
+                MaterializedViewStatus.DISABLED
+              )
+            }
+          }
           val indexSchemas = DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable)
           if (carbonTable.hasMVCreated) {
             val allDataMapSchemas = DataMapStoreManager.getInstance
               .getDataMapSchemasOfTable(carbonTable).asScala
               .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                       !dataMapSchema.isIndexDataMap).asJava
+                                       !dataMapSchema.isIndex).asJava
 
 Review comment:
   Yes, will be delete when clean the old mv implementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-603018451
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2544/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401342546
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
 ##########
 @@ -30,7 +30,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonCommonConstantsInternal, CarbonLoadOptionConstants}
 import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.exception.InvalidConfigurationException
-import org.apache.carbondata.core.util.{BlockletDataMapUtil, CarbonProperties, CarbonUtil, SessionParams}
+import org.apache.carbondata.core.util.{BlockletIndexUtil, CarbonProperties, CarbonUtil, SessionParams}
 
 Review comment:
   remove unused import

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r400681805
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/IndexChooser.java
 ##########
 @@ -59,76 +59,76 @@
  *   the datamap which has fewer columns that is the first datamap.
  */
 @InterfaceAudience.Internal
-public class DataMapChooser {
+public class IndexChooser {
 
   private CarbonTable carbonTable;
-  private List<TableDataMap> cgDataMaps;
-  private List<TableDataMap> fgDataMaps;
+  private List<TableIndex> cgIndexes;
+  private List<TableIndex> fgIndexes;
 
-  public DataMapChooser(CarbonTable carbonTable) throws IOException {
+  public IndexChooser(CarbonTable carbonTable) throws IOException {
     this.carbonTable = carbonTable;
-    // read all datamaps for this table and populate CG and FG datamap list
-    List<TableDataMap> visibleDataMaps =
-        DataMapStoreManager.getInstance().getAllVisibleDataMap(carbonTable);
+    // read all indexes for this table and populate CG and FG index list
+    List<TableIndex> visibleIndexes =
+        DataMapStoreManager.getInstance().getAllVisibleIndexes(carbonTable);
     Map<String, DataMapStatusDetail> map = DataMapStatusManager.readDataMapStatusMap();
-    cgDataMaps = new ArrayList<>(visibleDataMaps.size());
-    fgDataMaps = new ArrayList<>(visibleDataMaps.size());
-    for (TableDataMap visibleDataMap : visibleDataMaps) {
-      DataMapStatusDetail status = map.get(visibleDataMap.getDataMapSchema().getDataMapName());
+    cgIndexes = new ArrayList<>(visibleIndexes.size());
+    fgIndexes = new ArrayList<>(visibleIndexes.size());
+    for (TableIndex visibleIndex : visibleIndexes) {
+      DataMapStatusDetail status = map.get(visibleIndex.getDataMapSchema().getDataMapName());
       if (status != null && status.isEnabled()) {
-        DataMapLevel level = visibleDataMap.getDataMapFactory().getDataMapLevel();
-        if (level == DataMapLevel.CG) {
-          cgDataMaps.add(visibleDataMap);
+        IndexLevel level = visibleIndex.getIndexFactory().getDataMapLevel();
+        if (level == IndexLevel.CG) {
+          cgIndexes.add(visibleIndex);
         } else {
-          fgDataMaps.add(visibleDataMap);
+          fgIndexes.add(visibleIndex);
         }
       }
     }
   }
 
   /**
-   * Return a chosen datamap based on input filter. See {@link DataMapChooser}
+   * Return a chosen datamap based on input filter. See {@link IndexChooser}
    */
-  public DataMapExprWrapper choose(FilterResolverIntf filter) {
+  public IndexExprWrapper choose(FilterResolverIntf filter) {
     if (filter != null) {
       Expression expression = filter.getFilterExpression();
       // First check for FG datamaps if any exist
-      ExpressionTuple tuple = selectDataMap(expression, fgDataMaps, filter);
-      if (tuple.dataMapExprWrapper == null) {
+      ExpressionTuple tuple = selectDataMap(expression, fgIndexes, filter);
+      if (tuple.indexExprWrapper == null) {
         // Check for CG datamap
-        tuple = selectDataMap(expression, cgDataMaps, filter);
+        tuple = selectDataMap(expression, cgIndexes, filter);
       }
-      if (tuple.dataMapExprWrapper != null) {
-        return tuple.dataMapExprWrapper;
+      if (tuple.indexExprWrapper != null) {
+        return tuple.indexExprWrapper;
       }
     }
     // Return the default datamap if no other datamap exists.
-    return new DataMapExprWrapperImpl(
-        DataMapStoreManager.getInstance().getDefaultDataMap(carbonTable), filter);
+    return new IndexExprWrapperImpl(
+        DataMapStoreManager.getInstance().getDefaultIndex(carbonTable), filter);
   }
 
   /**
-   * Return a chosen FG datamap based on input filter. See {@link DataMapChooser}
+   * Return a chosen FG datamap based on input filter. See {@link IndexChooser}
    */
-  public DataMapExprWrapper chooseFGDataMap(FilterResolverIntf resolverIntf) {
-    return chooseDataMap(DataMapLevel.FG, resolverIntf);
+  public IndexExprWrapper chooseFGDataMap(FilterResolverIntf resolverIntf) {
+    return chooseDataMap(IndexLevel.FG, resolverIntf);
   }
 
   /**
-   * Return a chosen CG datamap based on input filter. See {@link DataMapChooser}
+   * Return a chosen CG datamap based on input filter. See {@link IndexChooser}
    */
-  public DataMapExprWrapper chooseCGDataMap(FilterResolverIntf resolverIntf) {
-    return chooseDataMap(DataMapLevel.CG, resolverIntf);
+  public IndexExprWrapper chooseCGDataMap(FilterResolverIntf resolverIntf) {
+    return chooseDataMap(IndexLevel.CG, resolverIntf);
   }
 
-  DataMapExprWrapper chooseDataMap(DataMapLevel level, FilterResolverIntf resolverIntf) {
+  IndexExprWrapper chooseDataMap(IndexLevel level, FilterResolverIntf resolverIntf) {
     if (resolverIntf != null) {
       Expression expression = resolverIntf.getFilterExpression();
-      List<TableDataMap> datamaps = level == DataMapLevel.CG ? cgDataMaps : fgDataMaps;
+      List<TableIndex> datamaps = level == IndexLevel.CG ? cgIndexes : fgIndexes;
 
 Review comment:
   Yes, we will do it in a new pr.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on issue #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on issue #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-607202895
 
 
   > many examples has create datamap
   
   Yes, we will clean all 'datamap' word in new pr.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401322301
 
 

 ##########
 File path: mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/Harmonizer.scala
 ##########
 @@ -72,8 +69,8 @@ object HarmonizeDimensionTable extends Rule[ModularPlan] with PredicateHelper {
       case s@Select(_, _, _, _, jedges, fact :: dims, _, _, _, _) if
       jedges.forall(e => e.joinType == LeftOuter || e.joinType == Inner) &&
       fact.isInstanceOf[ModularRelation] &&
-      dims.filterNot(_.isInstanceOf[modular.LeafNode]).nonEmpty &&
-      dims.forall(d => (d.isInstanceOf[ModularRelation] || HarmonizedRelation.canHarmonize(d))) => {
+      !dims.forall(_.isInstanceOf[modular.LeafNode]) &&
 
 Review comment:
   dims.exists(!_.isInstanceOf[modular.LeafNode]

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-602496537
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2542/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401311390
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/view/MaterializedViewManagerInSpark.scala
 ##########
 @@ -0,0 +1,62 @@
+/*
+ * 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.view
+
+import java.util
+
+import org.apache.spark.sql.{CarbonUtils, SparkSession}
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.view.MaterializedViewManager
+
+class MaterializedViewManagerInSpark(session: SparkSession) extends MaterializedViewManager {
+  override def getDatabases: util.List[String] = {
+    CarbonUtils.threadSet(CarbonCommonConstants.DISABLE_SQL_REWRITE, "true")
+    try {
+      val databaseList = session.catalog.listDatabases()
+      val databaseNameList = new util.ArrayList[String]()
+      for (database <- databaseList.collect()) {
+        databaseNameList.add(database.name)
+      }
+      databaseNameList
+    } finally {
+      CarbonUtils.threadUnset(CarbonCommonConstants.DISABLE_SQL_REWRITE)
+    }
+  }
+}
+
+object MaterializedViewManagerInSpark {
+
+  private val MANAGER_MAP_BY_SESSION =
+    new util.HashMap[SparkSession, MaterializedViewManagerInSpark]()
 
 Review comment:
   please clean the spark session when it closed.
   check SparkSqlAdapter.addSparkSessionListener, should do same with  CarbonEnv.carbonEnvMap

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-600054558
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/784/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-597972795
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2432/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-607165296
 
 
   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/896/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601495102
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2512/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596648854
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2399/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596641952
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/693/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401415307
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SIRebuildSegmentRunner.scala
 ##########
 @@ -35,32 +34,24 @@ import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandExcepti
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
 import org.apache.carbondata.core.metadata.{CarbonTableIdentifier, ColumnarFormatVersion}
-import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
 import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
 
-case class SIRebuildSegmentCommand(
-  alterTableModel: AlterTableModel,
-  tableInfoOp: Option[TableInfo] = None,
-  operationContext: OperationContext = new OperationContext)
-  extends AtomicRunnableCommand {
+case class SIRebuildSegmentRunner(
 
 Review comment:
   we will do it in a new pr.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r399968880
 
 

 ##########
 File path: dev/findbugs-exclude.xml
 ##########
 @@ -59,7 +59,7 @@
     <Bug pattern="STCAL_INVOKE_ON_STATIC_DATE_FORMAT_INSTANCE"/>
   </Match>
   <Match>
-    <Class name="org.apache.carbondata.core.datamap.DistributableDataMapFormat"/>
+    <Class name="org.apache.carbondata.core.datamap.IndexInputFormat"/>
 
 Review comment:
   Also documents [*.md] files need to be updated

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401356033
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/IndexChooser.java
 ##########
 @@ -269,14 +269,14 @@ private void extractColumnExpression(Expression expression,
     }
   }
 
-  private TableDataMap chooseDataMap(List<TableDataMap> allDataMap,
+  private TableIndex chooseDataMap(List<TableIndex> allDataMap,
 
 Review comment:
   we will do it in a new pr.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] asfgit closed pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596337774
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/683/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401316964
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatus.java
 ##########
 @@ -18,7 +18,7 @@
 package org.apache.carbondata.core.datamap.status;
 
 /**
- * DataMap status
+ * Index status
  */
 public enum DataMapStatus {
 
 Review comment:
   strongly agree with @ajantha-bhat 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401318776
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
 ##########
 @@ -76,21 +77,19 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
   }
 
 
-  protected lazy val start: Parser[LogicalPlan] = startCommand | extendedSparkSyntax
+  protected lazy val start: Parser[LogicalPlan] =
+    startCommand | extendedSparkSyntax | materializedViewCommands
 
 Review comment:
   append materializedViewCommands to startCommand

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395474972
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
 ##########
 @@ -1005,8 +1005,8 @@ public int getNumberOfSegmentsToBePreserved() {
   }
 
   public void print() {
-    LOGGER.info("------Using Carbon.properties --------");
-    LOGGER.info(carbonProperties.toString());
+    LOGGER.error("------Using Carbon.properties --------");
 
 Review comment:
   Why error?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r400681680
 
 

 ##########
 File path: integration/spark/src/test/scala/org/apache/carbondata/view/MaterializedViewTest.scala
 ##########
 @@ -0,0 +1,194 @@
+package org.apache.carbondata.view
+
+import java.io.File
+
+import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException
+import org.apache.spark.sql.catalyst.catalog.HiveTableRelation
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.spark.exception.ProcessMetaDataException
+
+class MaterializedViewTest extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+    drop()
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
+    val projectPath = new File(this.getClass.getResource("/").getPath + "../../../../")
+      .getCanonicalPath.replaceAll("\\\\", "/")
+    val integrationPath = s"$projectPath/integration"
+    val resourcesPath = s"$integrationPath/spark/src/test/resources"
+    sql(
+      """
+        | CREATE TABLE fact_table (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)
+        | STORED AS carbondata
+      """.stripMargin)
+    sql(s"""LOAD DATA local inpath '$resourcesPath/data_big.csv' INTO TABLE fact_table OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
+    sql(s"""LOAD DATA local inpath '$resourcesPath/data_big.csv' INTO TABLE fact_table OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
+  }
+
+  test("test create mv on hive table") {
+    sql("drop materialized view if exists mv1")
+    sql("drop table if exists source")
+    sql("create table source as select * from fact_table")
+    sql("create materialized view mv1 as select empname, deptname, avg(salary) from source group by empname, deptname")
 
 Review comment:
   Currently,  all mv tables which created by CREATE MATERIALIZED VIEW statement is carbon mv table.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401336799
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/view/MaterializedViewCatalog.java
 ##########
 @@ -0,0 +1,47 @@
+/*
 
 Review comment:
   how about move core/src/main/java/org/apache/carbondata/core/view to mv-plan module

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401320625
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
 ##########
 @@ -101,6 +100,12 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     loadDataNew | explainPlan | alterTableColumnRenameAndModifyDataType |
     alterTableAddColumns
 
+  protected lazy val materializedViewCommands: Parser[LogicalPlan] =
 
 Review comment:
   agree with @Indhumathi27, MVParser(MV module) should  Keep independent from CarbonParser(integeration/spark module).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r399972261
 
 

 ##########
 File path: integration/spark/src/test/scala/org/apache/carbondata/view/MaterializedViewTest.scala
 ##########
 @@ -0,0 +1,194 @@
+package org.apache.carbondata.view
+
+import java.io.File
+
+import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException
+import org.apache.spark.sql.catalyst.catalog.HiveTableRelation
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.test.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.spark.exception.ProcessMetaDataException
+
+class MaterializedViewTest extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+    drop()
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
+    val projectPath = new File(this.getClass.getResource("/").getPath + "../../../../")
+      .getCanonicalPath.replaceAll("\\\\", "/")
+    val integrationPath = s"$projectPath/integration"
+    val resourcesPath = s"$integrationPath/spark/src/test/resources"
+    sql(
+      """
+        | CREATE TABLE fact_table (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)
+        | STORED AS carbondata
+      """.stripMargin)
+    sql(s"""LOAD DATA local inpath '$resourcesPath/data_big.csv' INTO TABLE fact_table OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
+    sql(s"""LOAD DATA local inpath '$resourcesPath/data_big.csv' INTO TABLE fact_table OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '"')""")
+  }
+
+  test("test create mv on hive table") {
+    sql("drop materialized view if exists mv1")
+    sql("drop table if exists source")
+    sql("create table source as select * from fact_table")
+    sql("create materialized view mv1 as select empname, deptname, avg(salary) from source group by empname, deptname")
 
 Review comment:
   do we support all the feature supported in hive mv ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396975555
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/listeners/DropCacheEventListeners.scala
 ##########
 @@ -31,31 +32,28 @@ import org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider
 import org.apache.carbondata.core.metadata.schema.table.DataMapSchema
 import org.apache.carbondata.events.{DropTableCacheEvent, Event, OperationContext, OperationEventListener}
 
-object DropCacheDataMapEventListener extends OperationEventListener {
+object DropCacheMVEventListener extends OperationEventListener {
 
-  val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
 
   /**
    * Called on a specified event occurrence
-   *
-   * @param event
-   * @param operationContext
    */
   override protected def onEvent(event: Event, operationContext: OperationContext): Unit = {
     event match {
       case dropCacheEvent: DropTableCacheEvent =>
         val carbonTable = dropCacheEvent.carbonTable
         val sparkSession = dropCacheEvent.sparkSession
         val internalCall = dropCacheEvent.internalCall
-        if (carbonTable.isChildTableForMV && !internalCall) {
+        if (carbonTable.isMaterializedView && !internalCall) {
           throw new UnsupportedOperationException("Operation not allowed on child table.")
         }
 
         if (carbonTable.hasMVCreated) {
           val childrenSchemas = DataMapStoreManager.getInstance
             .getDataMapSchemasOfTable(carbonTable).asScala
             .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                     !dataMapSchema.isIndexDataMap)
+                                     !dataMapSchema.isIndex)
 
 Review comment:
   Yes, will be delete when clean the old mv implementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395462533
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
 ##########
 @@ -65,14 +66,14 @@
 
   private static DataMapStoreManager instance = new DataMapStoreManager();
 
-  public Map<String, List<TableDataMap>> getAllDataMaps() {
+  public Map<String, List<TableIndex>> getTableIndexForAllTables() {
     return allDataMaps;
   }
 
   /**
    * Contains the list of datamaps for each table.
    */
-  private Map<String, List<TableDataMap>> allDataMaps = new ConcurrentHashMap<>();
+  private Map<String, List<TableIndex>> allDataMaps = new ConcurrentHashMap<>();
 
 Review comment:
   change variable name to index and change variable description

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601636956
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/815/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-605770848
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2591/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-598538475
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/742/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396303230
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapSyncStatus.java
 ##########
 @@ -36,7 +36,7 @@
 /**
  * Interface to check whether datamap can be enabled
  */
-@InterfaceAudience.Developer("DataMap")
+@InterfaceAudience.Developer("Index")
 
 Review comment:
   Rename className from `DataMapSyncStatus` to `MVSyncStatus`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-600375453
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/786/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401356180
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
 ##########
 @@ -76,21 +77,19 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
   }
 
 
-  protected lazy val start: Parser[LogicalPlan] = startCommand | extendedSparkSyntax
+  protected lazy val start: Parser[LogicalPlan] =
+    startCommand | extendedSparkSyntax | materializedViewCommands
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396301531
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/IndexUtil.java
 ##########
 @@ -185,38 +185,37 @@ public static void pruneSegments(List<Segment> segments, List<ExtendedBlocklet>
    Loads the datamaps in parallel by utilizing executor
    *
    @param carbonTable
-   @param dataMapExprWrapper
+   @param indexExprWrapper
    @param validSegments
    @param partitionsToPrune
    @throws IOException
    */
-  public static void loadDataMaps(CarbonTable carbonTable, DataMapExprWrapper dataMapExprWrapper,
+  public static void loadDataMaps(CarbonTable carbonTable, IndexExprWrapper indexExprWrapper,
       List<Segment> validSegments, List<PartitionSpec> partitionsToPrune) throws IOException {
     if (!CarbonProperties.getInstance()
         .isDistributedPruningEnabled(carbonTable.getDatabaseName(), carbonTable.getTableName())
-        && BlockletDataMapUtil.loadDataMapsParallel(carbonTable)) {
-      String clsName = "org.apache.spark.sql.secondaryindex.Jobs.SparkBlockletDataMapLoaderJob";
-      DataMapJob dataMapJob = (DataMapJob) createDataMapJob(clsName);
-      String className =
-          "org.apache.spark.sql.secondaryindex.Jobs.DistributableBlockletDataMapLoader";
+        && BlockletIndexUtil.loadDataMapsParallel(carbonTable)) {
+      String clsName = "org.apache.carbondata.spark.rdd.SparkBlockletDataMapLoaderJob";
 
 Review comment:
   Please change the className and classPath for `SparkBlockletDataMapLoaderJob` and `DistributableBlockletDataMapLoader`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-600958408
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/800/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395491786
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/Jobs/BlockletIndexInputFormate.java
 ##########
 @@ -56,52 +56,52 @@
 /**
  * class to load blocklet data map
  */
-public class DistributableBlockletDataMapLoader
-    extends FileInputFormat<TableBlockIndexUniqueIdentifier, BlockletDataMapDetailsWithSchema>
+public class BlockletIndexInputFormate
 
 Review comment:
   Rename the class

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601998318
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/822/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401314214
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/AbstractIndexJob.java
 ##########
 @@ -28,15 +28,15 @@
 /**
  * abstract class for data map job
 
 Review comment:
   Change the comment content.
   If we change 'datamap' in the file name into 'index', better to change package name at same time.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-602012966
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/825/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396358616
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/listeners/ShowCacheEventListeners.scala
 ##########
 @@ -40,7 +40,7 @@ object ShowCachePreMVEventListener extends OperationEventListener {
       case showTableCacheEvent: ShowTableCacheEvent =>
         val carbonTable = showTableCacheEvent.carbonTable
         val internalCall = showTableCacheEvent.internalCall
-        if (carbonTable.isChildTableForMV && !internalCall) {
+        if (carbonTable.isMaterializedView && !internalCall) {
 
 Review comment:
   Show cache is not handled for mv after refactory. Please check

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-602401942
 
 
   please rebase

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396975357
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
 ##########
 @@ -65,20 +67,30 @@ case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[Logica
       val projList = Seq(UnresolvedAlias(UnresolvedStar(alias.map(Seq(_)))), tupleId)
       val carbonTable = CarbonEnv.getCarbonTable(table.tableIdentifier)(sparkSession)
       if (carbonTable != null) {
+        val viewManager = MaterializedViewManagerInSpark.get(sparkSession)
+        val viewSchemas = viewManager.getSchemasOnTable(carbonTable)
+        if (!viewSchemas.isEmpty) {
+          viewSchemas.asScala.foreach { schema =>
+            viewManager.setStatus(
+              schema.getIdentifier,
+              MaterializedViewStatus.DISABLED
+            )
+          }
+        }
         val indexSchemas = DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable)
         if (carbonTable.hasMVCreated) {
           val allDataMapSchemas = DataMapStoreManager.getInstance
             .getDataMapSchemasOfTable(carbonTable).asScala
             .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                     !dataMapSchema.isIndexDataMap).asJava
+                                     !dataMapSchema.isIndex).asJava
 
 Review comment:
   Yes, will be delete when clean the old mv implementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596984889
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/700/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-597025003
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/701/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-602032307
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2533/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395968302
 
 

 ##########
 File path: mv/core/src/main/scala/org/apache/carbondata/mv/extension/MVHelper.scala
 ##########
 @@ -631,7 +630,7 @@ object MVHelper {
       val newLongStringColumn = longStringColumn.get.split(",").map(_.trim).map { colName =>
         val newColName = parentTable.getTableName.toLowerCase() + "_" + colName
         if (!fieldNames.contains(newColName)) {
-          throw new MalformedDataMapCommandException(
+          throw new MalformedIndexCommandException(
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396362694
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
 ##########
 @@ -65,7 +66,7 @@ case class CarbonCleanFilesCommand(
       val allDataMapSchemas = DataMapStoreManager.getInstance
         .getDataMapSchemasOfTable(carbonTable).asScala
         .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                 !dataMapSchema.isIndexDataMap)
+                                 !dataMapSchema.isIndex)
 
 Review comment:
   Can remove this if check for mv

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat edited a comment on issue #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
ajantha-bhat edited a comment on issue #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-607197936
 
 
   @niuge01 : Look for datamap in *.md file. Still, many examples has create datamap syntax.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596985390
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2407/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-600983385
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2506/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r399968880
 
 

 ##########
 File path: dev/findbugs-exclude.xml
 ##########
 @@ -59,7 +59,7 @@
     <Bug pattern="STCAL_INVOKE_ON_STATIC_DATE_FORMAT_INSTANCE"/>
   </Match>
   <Match>
-    <Class name="org.apache.carbondata.core.datamap.DistributableDataMapFormat"/>
+    <Class name="org.apache.carbondata.core.datamap.IndexInputFormat"/>
 
 Review comment:
   Also documents [*.md] files need to be updated for all the changes of this PR

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-597624322
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/720/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395463541
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
 ##########
 @@ -82,7 +83,7 @@
   /**
    * Contains the datamap catalog for each datamap provider.
    */
-  private Map<String, DataMapCatalog> dataMapCatalogs = null;
+  private Map<String, MVCatalog> mvCatalogMap = null;
 
 Review comment:
   Change variable description also

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401315843
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/IndexChooser.java
 ##########
 @@ -269,14 +269,14 @@ private void extractColumnExpression(Expression expression,
     }
   }
 
-  private TableDataMap chooseDataMap(List<TableDataMap> allDataMap,
+  private TableIndex chooseDataMap(List<TableIndex> allDataMap,
 
 Review comment:
   better to change all DataMap to index once.
   if not, code will be hard to read

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-603733216
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2554/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-600052588
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2490/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401355792
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/AbstractIndexJob.java
 ##########
 @@ -28,15 +28,15 @@
 /**
  * abstract class for data map job
 
 Review comment:
   OK, we will do it in a new pr.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396329129
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/view/MaterializedViewProvider.java
 ##########
 @@ -0,0 +1,573 @@
+/*
+ * 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.core.view;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperationFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
+import org.apache.carbondata.core.fileoperations.FileWriteOperation;
+import org.apache.carbondata.core.locks.CarbonLockFactory;
+import org.apache.carbondata.core.locks.CarbonLockUtil;
+import org.apache.carbondata.core.locks.ICarbonLock;
+import org.apache.carbondata.core.locks.LockUsage;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatus;
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+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;
+import org.apache.log4j.Logger;
+
+/**
+ * Stores mv schema in disk as json format
+ */
+@InterfaceAudience.Internal
+public class MaterializedViewProvider {
+
+  private static final Logger LOG = LogServiceFactory.getLogService(
+      MaterializedViewProvider.class.getCanonicalName());
+
+  private static final String STATUS_FILE_NAME = "mv_status";
+
+  private final String storeLocation;
+
+  private final Map<String, SchemaProvider> schemaProviders = new ConcurrentHashMap<>();
+
+  private MaterializedViewProvider(String storeLocation) {
+    this.storeLocation = storeLocation;
+  }
+
+  public static MaterializedViewProvider get() {
+    String storeLocation =
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION);
+    if (storeLocation == null) {
+      throw new RuntimeException(
+          "Property [" + CarbonCommonConstants.STORE_LOCATION + "] is not set.");
+    }
+    return new MaterializedViewProvider(storeLocation);
+  }
+
+  private static String getSchemaPath(String schemaRoot, String viewName) {
+    return schemaRoot + CarbonCommonConstants.FILE_SEPARATOR + "mv_schema." + viewName;
+  }
+
+  private SchemaProvider getSchemaProvider(String databaseName) {
+    String databaseNameUpper = databaseName.toUpperCase();
+    SchemaProvider schemaProvider = this.schemaProviders.get(databaseNameUpper);
+    if (schemaProvider == null) {
+      synchronized (this.schemaProviders) {
+        schemaProvider = this.schemaProviders.get(databaseNameUpper);
+        if (schemaProvider == null) {
+          String databaseLocation;
+          if (databaseNameUpper.equalsIgnoreCase("default")) {
+            databaseLocation = CarbonUtil.checkAndAppendHDFSUrl(this.storeLocation);
+          } else {
+            databaseLocation = CarbonUtil.checkAndAppendHDFSUrl(this.storeLocation +
+                CarbonCommonConstants.FILE_SEPARATOR + databaseName + ".db");
+          }
+          if (!FileFactory.getCarbonFile(databaseLocation).exists()) {
+            return null;
+          }
+          schemaProvider = new SchemaProvider(databaseLocation);
+          this.schemaProviders.put(databaseNameUpper, schemaProvider);
+        }
+      }
+    }
+    return schemaProvider;
+  }
+
+  public MaterializedViewSchema getSchema(MaterializedViewManager viewManager,
+      String databaseName, String viewName) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      return null;
+    }
+    return schemaProvider.retrieveSchema(viewManager, viewName);
+  }
+
+  List<MaterializedViewSchema> getSchemas(MaterializedViewManager viewManager,
+      String databaseName, CarbonTable carbonTable) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      return Collections.emptyList();
+    } else {
+      return schemaProvider.retrieveSchemas(viewManager, carbonTable);
+    }
+  }
+
+  List<MaterializedViewSchema> getSchemas(MaterializedViewManager viewManager,
+      String databaseName) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      return Collections.emptyList();
+    } else {
+      return schemaProvider.retrieveAllSchemas(viewManager);
+    }
+  }
+
+  void saveSchema(MaterializedViewManager viewManager, String databaseName,
+      MaterializedViewSchema viewSchema) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      throw new IOException("Database [" + databaseName + "] is not found.");
+    }
+    schemaProvider.saveSchema(viewManager, viewSchema);
+  }
+
+  public void dropSchema(String databaseName, String viewName)
+      throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      throw new IOException("Materialized view with name " + databaseName + "." + viewName +
+          " does not exists in storage");
+    }
+    schemaProvider.dropSchema(viewName);
+  }
+
+  private String getStatusFileName(String databaseName) {
+    if (databaseName.equalsIgnoreCase("default")) {
+      return this.storeLocation +
+          CarbonCommonConstants.FILE_SEPARATOR + "_system" +
+          CarbonCommonConstants.FILE_SEPARATOR + STATUS_FILE_NAME;
+    } else {
+      return this.storeLocation +
+          CarbonCommonConstants.FILE_SEPARATOR + databaseName + ".db" +
+          CarbonCommonConstants.FILE_SEPARATOR + "_system" +
+          CarbonCommonConstants.FILE_SEPARATOR + STATUS_FILE_NAME;
+    }
+  }
+
+  public List<MaterializedViewStatusDetail> getStatusDetails(String databaseName)
+      throws IOException {
+    String statusPath = this.getStatusFileName(databaseName);
+    Gson gsonObjectToRead = new Gson();
+    DataInputStream dataInputStream = null;
+    BufferedReader buffReader = null;
+    InputStreamReader inStream = null;
+    MaterializedViewStatusDetail[] statusDetails;
+    try {
+      if (!FileFactory.isFileExist(statusPath)) {
+        return Collections.emptyList();
+      }
+      dataInputStream = FileFactory.getDataInputStream(statusPath);
+      inStream = new InputStreamReader(dataInputStream,
+          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+      buffReader = new BufferedReader(inStream);
+      statusDetails = gsonObjectToRead.fromJson(buffReader,
+          MaterializedViewStatusDetail[].class);
+    } catch (IOException e) {
+      LOG.error("Failed to read datamap status", e);
+      throw e;
+    } finally {
+      CarbonUtil.closeStreams(buffReader, inStream, dataInputStream);
+    }
+    // if status details is null, return empty array
+    if (null == statusDetails) {
+      return Collections.emptyList();
+    }
+    return Arrays.asList(statusDetails);
+  }
+
+  private static ICarbonLock getStatusLock(String databaseName) {
+    return CarbonLockFactory
+        .getSystemLevelCarbonLockObj(
+            CarbonProperties.getInstance().getSystemFolderLocation(databaseName),
+            LockUsage.MATERIALIZED_VIEW_STATUS_LOCK);
+  }
+
+  /**
+   * Update or add the status of passed mvs with the given mv status. If the mv status
+   * given is enabled/disabled then updates/adds the mv, in case of drop it just removes it
+   * from the file.
+   * This method always overwrites the old file.
+   * @param schemaList schemas of which are need to be updated in mv status
+   * @param status  status to be updated for the mv schemas
+   */
+  public void updateStatus(List<MaterializedViewSchema> schemaList,
+      MaterializedViewStatus status) throws IOException {
+    if (schemaList == null || schemaList.size() == 0) {
+      // There is nothing to update
+      return;
+    }
+    final Map<String, List<MaterializedViewSchema>> schemasMapByDatabase = new HashMap<>();
+    for (MaterializedViewSchema schema : schemaList) {
+      String databaseName = schema.getIdentifier().getDatabaseName().toLowerCase();
+      List<MaterializedViewSchema> schemas = schemasMapByDatabase.get(databaseName);
+      if (schemas == null) {
+        schemas = new ArrayList<>();
+        schemasMapByDatabase.put(databaseName, schemas);
+      }
+      schemas.add(schema);
+    }
+    for (Map.Entry<String, List<MaterializedViewSchema>> entry : schemasMapByDatabase.entrySet()) {
+      this.updateStatus(entry.getKey(), entry.getValue(), status);
+    }
+  }
+
+  private void updateStatus(String databaseName, List<MaterializedViewSchema> schemaList,
+      MaterializedViewStatus status) throws IOException {
+    ICarbonLock carbonTableStatusLock = getStatusLock(databaseName);
+    boolean locked = false;
+    try {
+      locked = carbonTableStatusLock.lockWithRetries();
+      if (locked) {
+        LOG.info("Materialized view status lock has been successfully acquired.");
+        if (status == MaterializedViewStatus.ENABLED) {
+          // Enable mv only if mv tables and main table are in sync
+          if (!isViewCanBeEnabled(schemaList.get(0))) {
+            return;
+          }
+        }
+        List<MaterializedViewStatusDetail> statusDetailList =
+            new ArrayList<>(getStatusDetails(databaseName));
+        List<MaterializedViewStatusDetail> changedStatusDetails = new ArrayList<>();
+        List<MaterializedViewStatusDetail> newStatusDetails = new ArrayList<>();
+        for (MaterializedViewSchema schema : schemaList) {
+          boolean exists = false;
+          for (MaterializedViewStatusDetail statusDetail : statusDetailList) {
+            if (statusDetail.getIdentifier().equals(schema.getIdentifier())) {
+              statusDetail.setStatus(status);
+              changedStatusDetails.add(statusDetail);
+              exists = true;
+            }
+          }
+          if (!exists) {
+            newStatusDetails
+                .add(new MaterializedViewStatusDetail(schema.getIdentifier(),
+                    status));
+          }
+        }
+        // Add the newly added datamaps to the list.
+        if (newStatusDetails.size() > 0 &&
+            status != MaterializedViewStatus.DROPPED) {
+          statusDetailList.addAll(newStatusDetails);
+        }
+        // In case of dropped datamap, just remove from the list.
+        if (status == MaterializedViewStatus.DROPPED) {
+          statusDetailList.removeAll(changedStatusDetails);
+        }
+        writeLoadDetailsIntoFile(
+            this.getStatusFileName(databaseName),
+            statusDetailList.toArray(
+                new MaterializedViewStatusDetail[statusDetailList.size()]));
+      } else {
+        String errorMsg = "Upadating datamapstatus is failed due to another process taken the lock"
+            + " for updating it";
+        LOG.error(errorMsg);
+        throw new IOException(errorMsg + " Please try after some time.");
+      }
+    } finally {
+      if (locked) {
+        CarbonLockUtil.fileUnlock(carbonTableStatusLock, LockUsage.DATAMAP_STATUS_LOCK);
+      }
+    }
+  }
+
+  /**
+   * writes mv status details
+   */
+  private static void writeLoadDetailsIntoFile(String location,
+      MaterializedViewStatusDetail[] statusDetails) throws IOException {
+    FileFactory.touchFile(FileFactory.getCarbonFile(location),
+        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
+    AtomicFileOperations fileWrite = AtomicFileOperationFactory.getAtomicFileOperations(location);
+    BufferedWriter brWriter = null;
+    DataOutputStream dataOutputStream = null;
+    Gson gsonObjectToWrite = new Gson();
+    // write the updated data into the mv status file.
+    try {
+      dataOutputStream = fileWrite.openForWrite(FileWriteOperation.OVERWRITE);
+      brWriter = new BufferedWriter(new OutputStreamWriter(dataOutputStream,
+          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
+
+      String metadataInstance = gsonObjectToWrite.toJson(statusDetails);
+      brWriter.write(metadataInstance);
+    } catch (IOException ioe) {
+      LOG.error("Error message: " + ioe.getLocalizedMessage());
+      fileWrite.setFailed();
+      throw ioe;
+    } finally {
+      if (null != brWriter) {
+        brWriter.flush();
+      }
+      CarbonUtil.closeStreams(brWriter);
+      fileWrite.close();
+    }
+
+  }
+
+  /**
+   * This method checks if main table and mv table are synchronised or not. If synchronised
+   * return true to enable the mv
+   *
+   * @param schema of mv to be disabled or enabled
+   * @return flag to enable or disable mv
+   * @throws IOException
+   */
+  private static boolean isViewCanBeEnabled(MaterializedViewSchema schema)
+      throws IOException {
+    if (!schema.isRefreshIncremental()) {
+      return true;
+    }
+    boolean isViewCanBeEnabled = true;
+    String viewMetadataPath =
+        CarbonTablePath.getMetadataPath(schema.getIdentifier().getTablePath());
+    LoadMetadataDetails[] viewLoadMetadataDetails =
+        SegmentStatusManager.readLoadMetadata(viewMetadataPath);
+    Map<String, List<String>> viewSegmentMap = new HashMap<>();
+    for (LoadMetadataDetails loadMetadataDetail : viewLoadMetadataDetails) {
+      if (loadMetadataDetail.getSegmentStatus() == SegmentStatus.SUCCESS) {
+        Map<String, List<String>> segmentMap =
+            new Gson().fromJson(loadMetadataDetail.getExtraInfo(), Map.class);
+        if (viewSegmentMap.isEmpty()) {
+          viewSegmentMap.putAll(segmentMap);
+        } else {
+          for (Map.Entry<String, List<String>> entry : segmentMap.entrySet()) {
+            if (null != viewSegmentMap.get(entry.getKey())) {
+              viewSegmentMap.get(entry.getKey()).addAll(entry.getValue());
+            }
+          }
+        }
+      }
+    }
+    List<RelationIdentifier> associatedTables = schema.getAssociatedTables();
+    for (RelationIdentifier associatedTable : associatedTables) {
+      List<String> associatedTableSegmentList =
+          SegmentStatusManager.getValidSegmentList(associatedTable);
+      if (!associatedTableSegmentList.isEmpty()) {
+        if (viewSegmentMap.isEmpty()) {
+          isViewCanBeEnabled = false;
+        } else {
+          isViewCanBeEnabled = viewSegmentMap.get(
+              associatedTable.getDatabaseName() + CarbonCommonConstants.POINT +
+                  associatedTable.getTableName()).containsAll(associatedTableSegmentList);
+        }
+      }
+    }
+    return isViewCanBeEnabled;
+  }
+
+  /**
+   * Data map schema provider of a database.
+   */
+  private static final class SchemaProvider {
+
+    private String systemDirectory;
+
+    private String schemaIndexFilePath;
+
+    private long lastModifiedTime;
+
+    private Set<MaterializedViewSchema> schemas = new HashSet<>();
+
+    SchemaProvider(String databaseLocation) {
+      final String systemDirectory =
+          databaseLocation + CarbonCommonConstants.FILE_SEPARATOR + "_system";
+      this.systemDirectory = systemDirectory;
+      this.schemaIndexFilePath = systemDirectory + CarbonCommonConstants.FILE_SEPARATOR +
+          "mv_schema_index";
+    }
+
+    void saveSchema(MaterializedViewManager viewManager, MaterializedViewSchema viewSchema)
+        throws IOException {
+      BufferedWriter brWriter = null;
+      DataOutputStream dataOutputStream = null;
+      Gson gsonObjectToWrite = new Gson();
+      String schemaPath =
+          getSchemaPath(this.systemDirectory, viewSchema.getIdentifier().getTableName());
+      if (FileFactory.isFileExist(schemaPath)) {
 
 Review comment:
   Can remove this check, as if it is already present in CarbonCreateMaterializedViewCommand:78

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r399961269
 
 

 ##########
 File path: common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedIndexCommandException.java
 ##########
 @@ -21,21 +21,21 @@
 import org.apache.carbondata.common.annotations.InterfaceStability;
 
 /**
- * This exception will be thrown when Datamap related SQL statement is invalid
+ * This exception will be thrown when index related SQL statement is invalid
 
 Review comment:
   @niuge01 and @jackylk : As per my understanding. After this PR. There won't be datamap word in the code. It is either index or materialized view. But **many package names still has datamap** [exmaple CgDatamap, FgDatamap] and **files also exist with datamap name**. [Example DataMapStatus, DataMapStatusManager] 
   Do we need to completly remove if weare doing this change.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r400679356
 
 

 ##########
 File path: docs/index-developer-guide.md
 ##########
 @@ -24,11 +24,11 @@ Currently, there are two types of DataMap supported:
 2. MVDataMap: DataMap that leverages Materialized View to accelerate olap style query, like SPJG query (select, predicate, join, groupby). Preaggregate, timeseries and mv DataMap belong to this type of DataMaps.
 
 Review comment:
   Yes, we will do it in a new pr.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-598010636
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2434/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401357702
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala
 ##########
 @@ -30,7 +30,7 @@ import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.constants.{CarbonCommonConstants, CarbonCommonConstantsInternal, CarbonLoadOptionConstants}
 import org.apache.carbondata.core.datamap.DataMapStoreManager
 import org.apache.carbondata.core.exception.InvalidConfigurationException
-import org.apache.carbondata.core.util.{BlockletDataMapUtil, CarbonProperties, CarbonUtil, SessionParams}
+import org.apache.carbondata.core.util.{BlockletIndexUtil, CarbonProperties, CarbonUtil, SessionParams}
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401314484
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/DataMapProvider.java
 ##########
 @@ -48,20 +48,20 @@
 import org.apache.log4j.Logger;
 
 /**
- * DataMap is a accelerator for certain type of query. Developer can add new DataMap
+ * Index is a accelerator for certain type of query. Developer can add new Index
  * implementation to improve query performance.
  *
- * Currently two types of DataMap are supported
+ * Currently two types of Index are supported
  * <ol>
- *   <li> MVDataMap: materialized view type of DataMap to accelerate olap style query,
+ *   <li> MVDataMap: materialized view type of Index to accelerate olap style query,
  * like SPJG query (select, predicate, join, groupby) </li>
- *   <li> DataMap: index type of DataMap to accelerate filter query </li>
+ *   <li> Index: index type of Index to accelerate filter query </li>
  * </ol>
  *
  * <p>
  * In following command <br>
  * {@code CREATE DATAMAP dm ON TABLE main USING 'provider'}, <br>
 
 Review comment:
   do we still have this SQL syntax?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r399965545
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapSyncStatus.java
 ##########
 @@ -36,7 +36,7 @@
 /**
  * Interface to check whether datamap can be enabled
  */
-@InterfaceAudience.Developer("DataMap")
+@InterfaceAudience.Developer("Index")
 
 Review comment:
   please delete old mv also in this PR

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596306009
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2386/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601989343
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2528/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401325282
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SIRebuildSegmentRunner.scala
 ##########
 @@ -35,32 +34,24 @@ import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandExcepti
 import org.apache.carbondata.common.logging.LogServiceFactory
 import org.apache.carbondata.core.locks.{CarbonLockFactory, LockUsage}
 import org.apache.carbondata.core.metadata.{CarbonTableIdentifier, ColumnarFormatVersion}
-import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, TableInfo}
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
 import org.apache.carbondata.core.util.CarbonUtil
 import org.apache.carbondata.events.{OperationContext, OperationListenerBus}
 
-case class SIRebuildSegmentCommand(
-  alterTableModel: AlterTableModel,
-  tableInfoOp: Option[TableInfo] = None,
-  operationContext: OperationContext = new OperationContext)
-  extends AtomicRunnableCommand {
+case class SIRebuildSegmentRunner(
 
 Review comment:
   how about SIRebuildSegmentExecutor?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-604229889
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/852/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r399963875
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/IndexChooser.java
 ##########
 @@ -59,76 +59,76 @@
  *   the datamap which has fewer columns that is the first datamap.
  */
 @InterfaceAudience.Internal
-public class DataMapChooser {
+public class IndexChooser {
 
   private CarbonTable carbonTable;
-  private List<TableDataMap> cgDataMaps;
-  private List<TableDataMap> fgDataMaps;
+  private List<TableIndex> cgIndexes;
+  private List<TableIndex> fgIndexes;
 
-  public DataMapChooser(CarbonTable carbonTable) throws IOException {
+  public IndexChooser(CarbonTable carbonTable) throws IOException {
     this.carbonTable = carbonTable;
-    // read all datamaps for this table and populate CG and FG datamap list
-    List<TableDataMap> visibleDataMaps =
-        DataMapStoreManager.getInstance().getAllVisibleDataMap(carbonTable);
+    // read all indexes for this table and populate CG and FG index list
+    List<TableIndex> visibleIndexes =
+        DataMapStoreManager.getInstance().getAllVisibleIndexes(carbonTable);
 
 Review comment:
   DataMapStoreManager need to be changed to IndexStoreManager ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-603810828
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/848/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-600019926
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/782/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-600403910
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2493/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401356901
 
 

 ##########
 File path: mv/plan/src/main/scala/org/apache/carbondata/mv/plans/util/BirdcageOptimizer.scala
 ##########
 @@ -124,8 +123,9 @@ object BirdcageOptimizer extends RuleExecutor[LogicalPlan] {
     //    Batch("LocalRelation", fixedPoint,
     //      ConvertToLocalRelation,
     //      PropagateEmptyRelation) ::
-    Batch(
-      "OptimizeCodegen", Once, CarbonToSparkAdapter.getOptimizeCodegenRule(): _*) ::
+    // As per SPARK-22520 OptimizeCodegen is removed in 2.3.1
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r400626624
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/IndexChooser.java
 ##########
 @@ -59,76 +59,76 @@
  *   the datamap which has fewer columns that is the first datamap.
  */
 @InterfaceAudience.Internal
-public class DataMapChooser {
+public class IndexChooser {
 
   private CarbonTable carbonTable;
-  private List<TableDataMap> cgDataMaps;
-  private List<TableDataMap> fgDataMaps;
+  private List<TableIndex> cgIndexes;
+  private List<TableIndex> fgIndexes;
 
-  public DataMapChooser(CarbonTable carbonTable) throws IOException {
+  public IndexChooser(CarbonTable carbonTable) throws IOException {
     this.carbonTable = carbonTable;
-    // read all datamaps for this table and populate CG and FG datamap list
-    List<TableDataMap> visibleDataMaps =
-        DataMapStoreManager.getInstance().getAllVisibleDataMap(carbonTable);
+    // read all indexes for this table and populate CG and FG index list
+    List<TableIndex> visibleIndexes =
+        DataMapStoreManager.getInstance().getAllVisibleIndexes(carbonTable);
 
 Review comment:
   Yes, it will be changed in a new pr.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395968334
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
 ##########
 @@ -65,14 +66,14 @@
 
   private static DataMapStoreManager instance = new DataMapStoreManager();
 
-  public Map<String, List<TableDataMap>> getAllDataMaps() {
+  public Map<String, List<TableIndex>> getTableIndexForAllTables() {
     return allDataMaps;
   }
 
   /**
    * Contains the list of datamaps for each table.
    */
-  private Map<String, List<TableDataMap>> allDataMaps = new ConcurrentHashMap<>();
+  private Map<String, List<TableIndex>> allDataMaps = new ConcurrentHashMap<>();
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-602486731
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/835/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396956769
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/status/MVSegmentStatusUtil.java
 ##########
 @@ -32,7 +32,7 @@
 /**
  * Utility class to get updated segment mapping for datamap table
  */
-public class DataMapSegmentStatusUtil {
+public class MVSegmentStatusUtil {
 
 Review comment:
   This method never used by new mv implementation, and will be delete when clean the old mv implementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601494914
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/807/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601010873
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2507/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396955981
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/dev/DataMapSyncStatus.java
 ##########
 @@ -36,7 +36,7 @@
 /**
  * Interface to check whether datamap can be enabled
  */
-@InterfaceAudience.Developer("DataMap")
+@InterfaceAudience.Developer("Index")
 
 Review comment:
   This class will be delete when clean the old mv implementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395968322
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
 ##########
 @@ -82,7 +83,7 @@
   /**
    * Contains the datamap catalog for each datamap provider.
    */
-  private Map<String, DataMapCatalog> dataMapCatalogs = null;
+  private Map<String, MVCatalog> mvCatalogMap = null;
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401356003
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/DataMapProvider.java
 ##########
 @@ -48,20 +48,20 @@
 import org.apache.log4j.Logger;
 
 /**
- * DataMap is a accelerator for certain type of query. Developer can add new DataMap
+ * Index is a accelerator for certain type of query. Developer can add new Index
  * implementation to improve query performance.
  *
- * Currently two types of DataMap are supported
+ * Currently two types of Index are supported
  * <ol>
- *   <li> MVDataMap: materialized view type of DataMap to accelerate olap style query,
+ *   <li> MVDataMap: materialized view type of Index to accelerate olap style query,
  * like SPJG query (select, predicate, join, groupby) </li>
- *   <li> DataMap: index type of DataMap to accelerate filter query </li>
+ *   <li> Index: index type of Index to accelerate filter query </li>
  * </ol>
  *
  * <p>
  * In following command <br>
  * {@code CREATE DATAMAP dm ON TABLE main USING 'provider'}, <br>
 
 Review comment:
   No, we will do it in a new pr.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-602026183
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/826/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401355696
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/CarbonEnv.scala
 ##########
 @@ -160,29 +161,29 @@ object CarbonEnv {
   def initListeners(): Unit = {
     OperationListenerBus.getInstance()
       .addListener(classOf[IndexServerLoadEvent], PrePrimingEventListener)
-      .addListener(classOf[LoadTablePreExecutionEvent], LoadMVTablePreListener)
-      .addListener(classOf[AlterTableCompactionPreStatusUpdateEvent],
-        AlterDataMaptableCompactionPostListener)
+//      .addListener(classOf[LoadTablePreExecutionEvent], LoadMVTablePreListener)
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596071430
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2365/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r400626587
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/DataMapStoreManager.java
 ##########
 @@ -163,6 +165,17 @@ private DataMapStoreManager() {
     return provider.retrieveAllSchemas();
   }
 
+  /**
+   * Return first match of the specified index name in table
+   *
+   */
+  public Optional<DataMapSchema> getIndexInTable(CarbonTable carbonTable, String indexName)
 
 Review comment:
   Yes, it will be changed in a new pr.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-600486663
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2499/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596059054
 
 
   please test this

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396965813
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
 ##########
 @@ -1011,14 +1015,25 @@ object CarbonDataRDDFactory {
       throw new Exception(errorMessage)
     } else {
       DataMapStatusManager.disableAllLazyDataMaps(carbonTable)
+      val viewManager = MaterializedViewManagerInSpark.get(session)
+      val viewSchemas = new util.ArrayList[MaterializedViewSchema]()
+      for (viewSchema <- viewManager.getSchemasOnTable(carbonTable).asScala) {
+        if (viewSchema.isRefreshOnManual) {
+          viewSchemas.add(viewSchema)
+        }
+      }
+      viewManager.setStatus(viewSchemas, MaterializedViewStatus.DISABLED)
       if (overwriteTable) {
         val allDataMapSchemas = DataMapStoreManager.getInstance
           .getDataMapSchemasOfTable(carbonTable).asScala
           .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                   !dataMapSchema.isIndexDataMap).asJava
+                                   !dataMapSchema.isIndex).asJava
 
 Review comment:
   Will be delete when clean the old mv implementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-600375732
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2492/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395968488
 
 

 ##########
 File path: common/src/main/java/org/apache/carbondata/common/exceptions/sql/NoSuchMaterializedViewException.java
 ##########
 @@ -0,0 +1,40 @@
+/*
+ * 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.common.exceptions.sql;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+
+/**
+ * This exception will be thrown if mv is not found when executing mv
+ * related SQL statement
+ */
+@InterfaceAudience.User
+@InterfaceStability.Stable
+public class NoSuchMaterializedViewException extends MalformedCarbonCommandException {
+
+  /**
+   * default serial version ID.
+   */
+  private static final long serialVersionUID = 1L;
+
+  public NoSuchMaterializedViewException(String databaseName, String mvName) {
+    super("Datamap with name " + databaseName + "." + mvName + " does not exist");
 
 Review comment:
   keep the same style with NoSuchIndexException and NoSuchStreamException

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-598195910
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/732/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396965973
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/carbondata/view/MaterializedViewRefresher.scala
 ##########
 @@ -0,0 +1,398 @@
+/*
+ * 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.view
+
+import java._
+import java.io.IOException
+
+import scala.collection.JavaConverters._
+
+import com.google.gson.Gson
+import org.apache.log4j.Logger
+import org.apache.spark.sql.{CarbonUtils, SparkSession}
+import org.apache.spark.sql.execution.command.management.CarbonInsertIntoCommand
+import org.apache.spark.sql.parser.MaterializedViewQueryParser
+
+import org.apache.carbondata.common.exceptions.sql.NoSuchMaterializedViewException
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.locks.ICarbonLock
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, RelationIdentifier}
+import org.apache.carbondata.core.statusmanager.{LoadMetadataDetails, SegmentStatus, SegmentStatusManager}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.core.view.{MaterializedViewSchema, MaterializedViewStatus}
+import org.apache.carbondata.processing.util.CarbonLoaderUtil
+
+class MaterializedViewRefresher{
+
+}
+
+object MaterializedViewRefresher {
+
+  private val LOGGER: Logger = LogServiceFactory.getLogService(
+    classOf[MaterializedViewRefresher].getCanonicalName)
+
+  /**
+   * Refresh the mv by loading all existing data from associated table
+   * This is called when refreshing the mv when
+   * 1. after mv creation and no "WITH DEFERRED REBUILD" defined
+   * 2. user manually trigger REFRESH MATERIALIZED VIEW command
+   */
+  @throws[IOException]
+  @throws[NoSuchMaterializedViewException]
+  def refresh(viewSchema: MaterializedViewSchema, session: SparkSession): Boolean = {
+    var newLoadName: String = ""
+    var segmentMap: String = ""
+    val viewTable: CarbonTable = CarbonTable.buildFromTablePath(
+      viewSchema.getIdentifier.getTableName,
+      viewSchema.getIdentifier.getDatabaseName,
+      viewSchema.getIdentifier.getTablePath,
+      viewSchema.getIdentifier.getTableId)
+    val viewIdentifier = viewSchema.getIdentifier
+    val viewTableIdentifier = viewTable.getAbsoluteTableIdentifier
+    // Clean up the old invalid segment data before creating a new entry for new load.
+    SegmentStatusManager.deleteLoadsAndUpdateMetadata(viewTable, false, null)
+    val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(viewTableIdentifier)
+    // Acquire table status lock to handle concurrent dataloading
+    val lock: ICarbonLock = segmentStatusManager.getTableStatusLock
+    val segmentMapping: util.Map[String, util.List[String]] =
+      new util.HashMap[String, util.List[String]]
+    val viewManager = MaterializedViewManagerInSpark.get(session)
+    try if (lock.lockWithRetries) {
+      LOGGER.info("Acquired lock for mv " + viewIdentifier + " for table status updation")
+      val viewTableMetadataPath: String =
+        CarbonTablePath.getMetadataPath(viewIdentifier.getTablePath)
+      val loadMetadataDetails = SegmentStatusManager.readLoadMetadata(viewTableMetadataPath)
+      val loadMetadataDetailList: util.List[LoadMetadataDetails] =
+        new util.ArrayList[LoadMetadataDetails](CarbonCommonConstants.DEFAULT_COLLECTION_SIZE)
+      // Mark for delete all stale loadMetadetail
+      for (loadMetadataDetail <- loadMetadataDetails) {
+        if (((loadMetadataDetail.getSegmentStatus eq SegmentStatus.INSERT_IN_PROGRESS) ||
+             (loadMetadataDetail.getSegmentStatus eq SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS)) &&
+            loadMetadataDetail.getVisibility.equalsIgnoreCase("false")) {
+          loadMetadataDetail.setSegmentStatus(SegmentStatus.MARKED_FOR_DELETE)
+        }
+        loadMetadataDetailList.add(loadMetadataDetail)
+      }
+      if (viewSchema.isRefreshOnManual) {
+        // check if rebuild to mv is already in progress and throw exception
+        if (loadMetadataDetails.nonEmpty) {
+          for (loadMetaDetail <- loadMetadataDetails) {
+            if (((loadMetaDetail.getSegmentStatus eq SegmentStatus.INSERT_IN_PROGRESS) ||
+                 (loadMetaDetail.getSegmentStatus eq SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS)) &&
+                SegmentStatusManager.isLoadInProgress(viewTableIdentifier,
+                  loadMetaDetail.getLoadName)) {
+              throw new RuntimeException(
+                "Rebuild to materialized view " + viewSchema.getIdentifier.getTableName +
+                  " is already in progress")
+            }
+          }
+        }
+      }
+      if (viewSchema.isRefreshIncremental) {
+        if (!getSpecificSegmentsTobeLoaded(viewSchema, segmentMapping, loadMetadataDetailList)) {
+          return false
+        }
+      } else {
+          // set segment mapping only for carbondata table
+          val associatedTableIds =
+            viewSchema.getAssociatedTables.asScala.filter(_.isCarbonDataTable)
+          for (associatedTableId <- associatedTableIds) {
+            val associatedTableSegmentList: util.List[String] =
+              SegmentStatusManager.getValidSegmentList(associatedTableId)
+            if (associatedTableSegmentList.isEmpty) {
+              return false
+            }
+            segmentMapping.put(associatedTableId.toString, associatedTableSegmentList)
+          }
+        }
+      segmentMap = new Gson().toJson(segmentMapping)
+      // To handle concurrent dataloading to mv, create new loadMetaEntry and
+      // set segmentMap to new loadMetaEntry and pass new segmentId with load command
+      val loadMetadataDetail: LoadMetadataDetails = new LoadMetadataDetails
+      val segmentId: String = String.valueOf(
+        SegmentStatusManager.createNewSegmentId(loadMetadataDetails))
+      loadMetadataDetail.setLoadName(segmentId)
+      loadMetadataDetail.setSegmentStatus(SegmentStatus.INSERT_IN_PROGRESS)
+      loadMetadataDetail.setExtraInfo(segmentMap)
+      loadMetadataDetailList.add(loadMetadataDetail)
+      newLoadName = segmentId
+      SegmentStatusManager.writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(
+        viewSchema.getIdentifier.getTablePath),
+        loadMetadataDetailList.toArray(new Array[LoadMetadataDetails](loadMetadataDetailList
+          .size)))
+    } else {
+      LOGGER.error("Not able to acquire the lock for Table status updation for table " +
+                   viewSchema.getIdentifier.getDatabaseName + "." +
+                   viewSchema.getIdentifier.getTableName)
+      viewManager.setStatus(viewSchema.getIdentifier, MaterializedViewStatus.DISABLED)
+      return false
+    } finally {
+      if (lock.unlock) {
+        LOGGER.info("Table unlocked successfully after table status updation" +
+                    viewSchema.getIdentifier.getDatabaseName + "." +
+                    viewSchema.getIdentifier.getTableName)
+      } else {
+        LOGGER.error("Unable to unlock Table lock for table" +
+                     viewSchema.getIdentifier.getDatabaseName + "." +
+                     viewSchema.getIdentifier.getTableName +
+                     " during table status updation")
+      }
+    }
+    refreshInternal(viewManager, viewSchema, viewTable, newLoadName, segmentMapping, session)
+  }
+
+  @throws[IOException]
+  private def refreshInternal(
+      viewManager: MaterializedViewManagerInSpark,
+      viewSchema: MaterializedViewSchema,
+      viewTable: CarbonTable,
+      newLoadName: String,
+      segmentMap: java.util.Map[String, java.util.List[String]],
+      session: SparkSession): Boolean = {
+    val query = viewSchema.getQuery
+    if (query != null) {
+      val viewIdentifier = viewSchema.getIdentifier
+      val updatedQuery = MaterializedViewQueryParser.getQuery(query, session)
+      val isFullRefresh = !viewSchema.isRefreshIncremental
+      // Set specified segments for incremental load
+      val segmentMapIterator = segmentMap.entrySet().iterator()
+      while (segmentMapIterator.hasNext) {
+        val entry = segmentMapIterator.next()
+        setInputSegments(entry.getKey, entry.getValue)
+      }
+      val header = viewTable.getTableInfo.getFactTable.getListOfColumns.asScala
+        .filter { column =>
+          !column.getColumnName
+            .equalsIgnoreCase(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE)
+        }.sortBy(_.getSchemaOrdinal).map(_.getColumnName).mkString(",")
+      val insertIntoCommand = CarbonInsertIntoCommand(
+        databaseNameOp = Some(viewIdentifier.getDatabaseName),
+        tableName = viewIdentifier.getTableName,
+        options = scala.collection.immutable.Map("fileheader" -> header),
+        isFullRefresh,
+        logicalPlan = updatedQuery.queryExecution.analyzed,
+        tableInfo = viewTable.getTableInfo,
+        internalOptions = Map("mergedSegmentName" -> newLoadName,
+          CarbonCommonConstants.IS_INTERNAL_LOAD_CALL -> "true"),
+        partition = Map.empty)
+      try {
+        insertIntoCommand.run(session)
+      } catch {
+        case exception: Exception =>
+          // If load to dataMap table fails, disable the dataMap and if newLoad is still
+          // in INSERT_IN_PROGRESS state, mark for delete the newLoad and update table status file
+          viewManager.setStatus(viewSchema.getIdentifier, MaterializedViewStatus.DISABLED)
+          LOGGER.error("Data Load failed for DataMap: ", exception)
+          CarbonLoaderUtil.updateTableStatusInCaseOfFailure(
+            newLoadName,
+            viewTable.getAbsoluteTableIdentifier,
+            viewTable.getTableName,
+            viewTable.getDatabaseName,
+            viewTable.getTablePath,
+            viewTable.getMetadataPath)
+          throw exception
+      } finally {
+        unsetInputSegments(viewSchema)
+      }
+    }
+    true
+  }
+
+  /**
+   * This method will compare mainTable and dataMapTable segment List and loads only newly added
+   * segment from main table to dataMap table.
+   * In case if mainTable is compacted, then based on dataMap to mainTables segmentMapping, dataMap
+   * will be loaded
+   * Eg:
+   * case 1: Consider mainTableSegmentList: {0, 1, 2}, dataMapToMainTable segmentMap:
+   * { 0 -> 0, 1-> 1,2}. If (1, 2) segments of main table are compacted to 1.1 and new segment (3)
+   * is loaded to main table, then mainTableSegmentList will be updated to{0, 1.1, 3}.
+   * In this case, segment (1) of dataMap table will be marked for delete, and new segment
+   * {2 -> 1.1, 3} will be loaded to dataMap table
+   * case 2: Consider mainTableSegmentList: {0, 1, 2, 3}, dataMapToMainTable segmentMap:
+   * { 0 -> 0,1,2, 1-> 3}. If (1, 2) segments of main table are compacted to 1.1 and new segment
+   * (4) is loaded to main table, then mainTableSegmentList will be updated to {0, 1.1, 3, 4}.
+   * In this case, segment (0) of dataMap table will be marked for delete and segment (0) of
+   * main table will be added to validSegmentList which needs to be loaded again. Now, new dataMap
+   * table segment (2) with main table segmentList{2 -> 1.1, 4, 0} will be loaded to dataMap table.
+   * dataMapToMainTable segmentMap will be updated to {1 -> 3, 2 -> 1.1, 4, 0} after rebuild
+   */
+  @throws[IOException]
+  private def getSpecificSegmentsTobeLoaded(schema: MaterializedViewSchema,
+      segmentMapping: util.Map[String, util.List[String]],
+      listOfLoadFolderDetails: util.List[LoadMetadataDetails]): Boolean = {
+    val relationIdentifiers: util.List[RelationIdentifier] = schema.getAssociatedTables
+    // invalidDataMapSegmentList holds segment list which needs to be marked for delete
+    val invalidDataMapSegmentList: util.HashSet[String] = new util.HashSet[String]
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601982938
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/821/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-600958786
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2505/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396965321
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/view/MaterializedViewProvider.java
 ##########
 @@ -0,0 +1,573 @@
+/*
+ * 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.core.view;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperationFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
+import org.apache.carbondata.core.fileoperations.FileWriteOperation;
+import org.apache.carbondata.core.locks.CarbonLockFactory;
+import org.apache.carbondata.core.locks.CarbonLockUtil;
+import org.apache.carbondata.core.locks.ICarbonLock;
+import org.apache.carbondata.core.locks.LockUsage;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatus;
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+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;
+import org.apache.log4j.Logger;
+
+/**
+ * Stores mv schema in disk as json format
+ */
+@InterfaceAudience.Internal
+public class MaterializedViewProvider {
+
+  private static final Logger LOG = LogServiceFactory.getLogService(
+      MaterializedViewProvider.class.getCanonicalName());
+
+  private static final String STATUS_FILE_NAME = "mv_status";
+
+  private final String storeLocation;
+
+  private final Map<String, SchemaProvider> schemaProviders = new ConcurrentHashMap<>();
+
+  private MaterializedViewProvider(String storeLocation) {
+    this.storeLocation = storeLocation;
+  }
+
+  public static MaterializedViewProvider get() {
+    String storeLocation =
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION);
+    if (storeLocation == null) {
+      throw new RuntimeException(
+          "Property [" + CarbonCommonConstants.STORE_LOCATION + "] is not set.");
+    }
+    return new MaterializedViewProvider(storeLocation);
+  }
+
+  private static String getSchemaPath(String schemaRoot, String viewName) {
+    return schemaRoot + CarbonCommonConstants.FILE_SEPARATOR + "mv_schema." + viewName;
+  }
+
+  private SchemaProvider getSchemaProvider(String databaseName) {
+    String databaseNameUpper = databaseName.toUpperCase();
+    SchemaProvider schemaProvider = this.schemaProviders.get(databaseNameUpper);
+    if (schemaProvider == null) {
+      synchronized (this.schemaProviders) {
+        schemaProvider = this.schemaProviders.get(databaseNameUpper);
+        if (schemaProvider == null) {
+          String databaseLocation;
+          if (databaseNameUpper.equalsIgnoreCase("default")) {
+            databaseLocation = CarbonUtil.checkAndAppendHDFSUrl(this.storeLocation);
+          } else {
+            databaseLocation = CarbonUtil.checkAndAppendHDFSUrl(this.storeLocation +
+                CarbonCommonConstants.FILE_SEPARATOR + databaseName + ".db");
+          }
+          if (!FileFactory.getCarbonFile(databaseLocation).exists()) {
+            return null;
+          }
+          schemaProvider = new SchemaProvider(databaseLocation);
+          this.schemaProviders.put(databaseNameUpper, schemaProvider);
+        }
+      }
+    }
+    return schemaProvider;
+  }
+
+  public MaterializedViewSchema getSchema(MaterializedViewManager viewManager,
+      String databaseName, String viewName) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      return null;
+    }
+    return schemaProvider.retrieveSchema(viewManager, viewName);
+  }
+
+  List<MaterializedViewSchema> getSchemas(MaterializedViewManager viewManager,
+      String databaseName, CarbonTable carbonTable) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      return Collections.emptyList();
+    } else {
+      return schemaProvider.retrieveSchemas(viewManager, carbonTable);
+    }
+  }
+
+  List<MaterializedViewSchema> getSchemas(MaterializedViewManager viewManager,
+      String databaseName) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      return Collections.emptyList();
+    } else {
+      return schemaProvider.retrieveAllSchemas(viewManager);
+    }
+  }
+
+  void saveSchema(MaterializedViewManager viewManager, String databaseName,
+      MaterializedViewSchema viewSchema) throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      throw new IOException("Database [" + databaseName + "] is not found.");
+    }
+    schemaProvider.saveSchema(viewManager, viewSchema);
+  }
+
+  public void dropSchema(String databaseName, String viewName)
+      throws IOException {
+    SchemaProvider schemaProvider = this.getSchemaProvider(databaseName);
+    if (schemaProvider == null) {
+      throw new IOException("Materialized view with name " + databaseName + "." + viewName +
+          " does not exists in storage");
+    }
+    schemaProvider.dropSchema(viewName);
+  }
+
+  private String getStatusFileName(String databaseName) {
+    if (databaseName.equalsIgnoreCase("default")) {
+      return this.storeLocation +
+          CarbonCommonConstants.FILE_SEPARATOR + "_system" +
+          CarbonCommonConstants.FILE_SEPARATOR + STATUS_FILE_NAME;
+    } else {
+      return this.storeLocation +
+          CarbonCommonConstants.FILE_SEPARATOR + databaseName + ".db" +
+          CarbonCommonConstants.FILE_SEPARATOR + "_system" +
+          CarbonCommonConstants.FILE_SEPARATOR + STATUS_FILE_NAME;
+    }
+  }
+
+  public List<MaterializedViewStatusDetail> getStatusDetails(String databaseName)
+      throws IOException {
+    String statusPath = this.getStatusFileName(databaseName);
+    Gson gsonObjectToRead = new Gson();
+    DataInputStream dataInputStream = null;
+    BufferedReader buffReader = null;
+    InputStreamReader inStream = null;
+    MaterializedViewStatusDetail[] statusDetails;
+    try {
+      if (!FileFactory.isFileExist(statusPath)) {
+        return Collections.emptyList();
+      }
+      dataInputStream = FileFactory.getDataInputStream(statusPath);
+      inStream = new InputStreamReader(dataInputStream,
+          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+      buffReader = new BufferedReader(inStream);
+      statusDetails = gsonObjectToRead.fromJson(buffReader,
+          MaterializedViewStatusDetail[].class);
+    } catch (IOException e) {
+      LOG.error("Failed to read datamap status", e);
+      throw e;
+    } finally {
+      CarbonUtil.closeStreams(buffReader, inStream, dataInputStream);
+    }
+    // if status details is null, return empty array
+    if (null == statusDetails) {
+      return Collections.emptyList();
+    }
+    return Arrays.asList(statusDetails);
+  }
+
+  private static ICarbonLock getStatusLock(String databaseName) {
+    return CarbonLockFactory
+        .getSystemLevelCarbonLockObj(
+            CarbonProperties.getInstance().getSystemFolderLocation(databaseName),
+            LockUsage.MATERIALIZED_VIEW_STATUS_LOCK);
+  }
+
+  /**
+   * Update or add the status of passed mvs with the given mv status. If the mv status
+   * given is enabled/disabled then updates/adds the mv, in case of drop it just removes it
+   * from the file.
+   * This method always overwrites the old file.
+   * @param schemaList schemas of which are need to be updated in mv status
+   * @param status  status to be updated for the mv schemas
+   */
+  public void updateStatus(List<MaterializedViewSchema> schemaList,
+      MaterializedViewStatus status) throws IOException {
+    if (schemaList == null || schemaList.size() == 0) {
+      // There is nothing to update
+      return;
+    }
+    final Map<String, List<MaterializedViewSchema>> schemasMapByDatabase = new HashMap<>();
+    for (MaterializedViewSchema schema : schemaList) {
+      String databaseName = schema.getIdentifier().getDatabaseName().toLowerCase();
+      List<MaterializedViewSchema> schemas = schemasMapByDatabase.get(databaseName);
+      if (schemas == null) {
+        schemas = new ArrayList<>();
+        schemasMapByDatabase.put(databaseName, schemas);
+      }
+      schemas.add(schema);
+    }
+    for (Map.Entry<String, List<MaterializedViewSchema>> entry : schemasMapByDatabase.entrySet()) {
+      this.updateStatus(entry.getKey(), entry.getValue(), status);
+    }
+  }
+
+  private void updateStatus(String databaseName, List<MaterializedViewSchema> schemaList,
+      MaterializedViewStatus status) throws IOException {
+    ICarbonLock carbonTableStatusLock = getStatusLock(databaseName);
+    boolean locked = false;
+    try {
+      locked = carbonTableStatusLock.lockWithRetries();
+      if (locked) {
+        LOG.info("Materialized view status lock has been successfully acquired.");
+        if (status == MaterializedViewStatus.ENABLED) {
+          // Enable mv only if mv tables and main table are in sync
+          if (!isViewCanBeEnabled(schemaList.get(0))) {
+            return;
+          }
+        }
+        List<MaterializedViewStatusDetail> statusDetailList =
+            new ArrayList<>(getStatusDetails(databaseName));
+        List<MaterializedViewStatusDetail> changedStatusDetails = new ArrayList<>();
+        List<MaterializedViewStatusDetail> newStatusDetails = new ArrayList<>();
+        for (MaterializedViewSchema schema : schemaList) {
+          boolean exists = false;
+          for (MaterializedViewStatusDetail statusDetail : statusDetailList) {
+            if (statusDetail.getIdentifier().equals(schema.getIdentifier())) {
+              statusDetail.setStatus(status);
+              changedStatusDetails.add(statusDetail);
+              exists = true;
+            }
+          }
+          if (!exists) {
+            newStatusDetails
+                .add(new MaterializedViewStatusDetail(schema.getIdentifier(),
+                    status));
+          }
+        }
+        // Add the newly added datamaps to the list.
+        if (newStatusDetails.size() > 0 &&
+            status != MaterializedViewStatus.DROPPED) {
+          statusDetailList.addAll(newStatusDetails);
+        }
+        // In case of dropped datamap, just remove from the list.
+        if (status == MaterializedViewStatus.DROPPED) {
+          statusDetailList.removeAll(changedStatusDetails);
+        }
+        writeLoadDetailsIntoFile(
+            this.getStatusFileName(databaseName),
+            statusDetailList.toArray(
+                new MaterializedViewStatusDetail[statusDetailList.size()]));
+      } else {
+        String errorMsg = "Upadating datamapstatus is failed due to another process taken the lock"
+            + " for updating it";
+        LOG.error(errorMsg);
+        throw new IOException(errorMsg + " Please try after some time.");
+      }
+    } finally {
+      if (locked) {
+        CarbonLockUtil.fileUnlock(carbonTableStatusLock, LockUsage.DATAMAP_STATUS_LOCK);
+      }
+    }
+  }
+
+  /**
+   * writes mv status details
+   */
+  private static void writeLoadDetailsIntoFile(String location,
+      MaterializedViewStatusDetail[] statusDetails) throws IOException {
+    FileFactory.touchFile(FileFactory.getCarbonFile(location),
+        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
+    AtomicFileOperations fileWrite = AtomicFileOperationFactory.getAtomicFileOperations(location);
+    BufferedWriter brWriter = null;
+    DataOutputStream dataOutputStream = null;
+    Gson gsonObjectToWrite = new Gson();
+    // write the updated data into the mv status file.
+    try {
+      dataOutputStream = fileWrite.openForWrite(FileWriteOperation.OVERWRITE);
+      brWriter = new BufferedWriter(new OutputStreamWriter(dataOutputStream,
+          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
+
+      String metadataInstance = gsonObjectToWrite.toJson(statusDetails);
+      brWriter.write(metadataInstance);
+    } catch (IOException ioe) {
+      LOG.error("Error message: " + ioe.getLocalizedMessage());
+      fileWrite.setFailed();
+      throw ioe;
+    } finally {
+      if (null != brWriter) {
+        brWriter.flush();
+      }
+      CarbonUtil.closeStreams(brWriter);
+      fileWrite.close();
+    }
+
+  }
+
+  /**
+   * This method checks if main table and mv table are synchronised or not. If synchronised
+   * return true to enable the mv
+   *
+   * @param schema of mv to be disabled or enabled
+   * @return flag to enable or disable mv
+   * @throws IOException
+   */
+  private static boolean isViewCanBeEnabled(MaterializedViewSchema schema)
+      throws IOException {
+    if (!schema.isRefreshIncremental()) {
+      return true;
+    }
+    boolean isViewCanBeEnabled = true;
+    String viewMetadataPath =
+        CarbonTablePath.getMetadataPath(schema.getIdentifier().getTablePath());
+    LoadMetadataDetails[] viewLoadMetadataDetails =
+        SegmentStatusManager.readLoadMetadata(viewMetadataPath);
+    Map<String, List<String>> viewSegmentMap = new HashMap<>();
+    for (LoadMetadataDetails loadMetadataDetail : viewLoadMetadataDetails) {
+      if (loadMetadataDetail.getSegmentStatus() == SegmentStatus.SUCCESS) {
+        Map<String, List<String>> segmentMap =
+            new Gson().fromJson(loadMetadataDetail.getExtraInfo(), Map.class);
+        if (viewSegmentMap.isEmpty()) {
+          viewSegmentMap.putAll(segmentMap);
+        } else {
+          for (Map.Entry<String, List<String>> entry : segmentMap.entrySet()) {
+            if (null != viewSegmentMap.get(entry.getKey())) {
+              viewSegmentMap.get(entry.getKey()).addAll(entry.getValue());
+            }
+          }
+        }
+      }
+    }
+    List<RelationIdentifier> associatedTables = schema.getAssociatedTables();
+    for (RelationIdentifier associatedTable : associatedTables) {
+      List<String> associatedTableSegmentList =
+          SegmentStatusManager.getValidSegmentList(associatedTable);
+      if (!associatedTableSegmentList.isEmpty()) {
+        if (viewSegmentMap.isEmpty()) {
+          isViewCanBeEnabled = false;
+        } else {
+          isViewCanBeEnabled = viewSegmentMap.get(
+              associatedTable.getDatabaseName() + CarbonCommonConstants.POINT +
+                  associatedTable.getTableName()).containsAll(associatedTableSegmentList);
+        }
+      }
+    }
+    return isViewCanBeEnabled;
+  }
+
+  /**
+   * Data map schema provider of a database.
+   */
+  private static final class SchemaProvider {
+
+    private String systemDirectory;
+
+    private String schemaIndexFilePath;
+
+    private long lastModifiedTime;
+
+    private Set<MaterializedViewSchema> schemas = new HashSet<>();
+
+    SchemaProvider(String databaseLocation) {
+      final String systemDirectory =
+          databaseLocation + CarbonCommonConstants.FILE_SEPARATOR + "_system";
+      this.systemDirectory = systemDirectory;
+      this.schemaIndexFilePath = systemDirectory + CarbonCommonConstants.FILE_SEPARATOR +
+          "mv_schema_index";
+    }
+
+    void saveSchema(MaterializedViewManager viewManager, MaterializedViewSchema viewSchema)
+        throws IOException {
+      BufferedWriter brWriter = null;
+      DataOutputStream dataOutputStream = null;
+      Gson gsonObjectToWrite = new Gson();
+      String schemaPath =
+          getSchemaPath(this.systemDirectory, viewSchema.getIdentifier().getTableName());
+      if (FileFactory.isFileExist(schemaPath)) {
 
 Review comment:
   MaterializedViewManager.createSchema is a public interface, so it should ensuring independence and integrity itself.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396313702
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/datamap/DataMapClassProvider.java
 ##########
 @@ -18,26 +18,20 @@
 package org.apache.carbondata.core.metadata.schema.datamap;
 
 /**
- * type for create datamap
- * The syntax of datamap creation is as follows.
- * CREATE DATAMAP IF NOT EXISTS dataMapName ON TABLE tableName USING 'DataMapClassProvider'
- * DMPROPERTIES('KEY'='VALUE') AS SELECT COUNT(COL1) FROM tableName
- *
  * Please refer {{org.apache.spark.sql.parser.CarbonSpark2SqlParser}}
  */
-
 public enum DataMapClassProvider {
-  LUCENE("org.apache.carbondata.datamap.lucene.LuceneFineGrainDataMapFactory", "lucene"),
-  BLOOMFILTER("org.apache.carbondata.datamap.bloom.BloomCoarseGrainDataMapFactory", "bloomfilter"),
+  LUCENE("org.apache.carbondata.datamap.lucene.LuceneFineGrainIndexFactory", "lucene"),
+  BLOOMFILTER("org.apache.carbondata.datamap.bloom.BloomCoarseGrainIndexFactory", "bloomfilter"),
 
 Review comment:
   Please change the package for lucene and bloom classes

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401341097
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 ##########
 @@ -885,6 +894,16 @@ public boolean isChildTableForMV() {
         .get(CarbonCommonConstants.PARENT_TABLES).isEmpty();
   }
 
+  /**
+   * Return true if this table is a MV table (child table of other table)
+   */
+  public boolean isMaterializedView() {
+    return tableInfo.getFactTable().getTableProperties()
+        .get(CarbonCommonConstants.ASSOCIATED_TABLES) != null &&
+        !tableInfo.getFactTable().getTableProperties()
+        .get(CarbonCommonConstants.ASSOCIATED_TABLES).isEmpty();
 
 Review comment:
   please remove redundancy code.
   how about use MV_ON_TABLES OR MV_RELATED_TABLES instead of  ASSOCIATED_TABLES ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-600401864
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/787/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596059310
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/658/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-603813251
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2555/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-598009564
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/726/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-597631577
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2427/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596310499
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/681/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-598128690
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2438/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396975590
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/listeners/MVListeners.scala
 ##########
 @@ -67,7 +67,7 @@ object AlterDataMaptableCompactionPostListener extends OperationEventListener {
     val allDataMapSchemas = DataMapStoreManager.getInstance
       .getDataMapSchemasOfTable(carbonTable).asScala
       .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                               !dataMapSchema.isIndexDataMap)
+                               !dataMapSchema.isIndex)
 
 Review comment:
   Yes, will be delete when clean the old mv implementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-597091621
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/704/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601189004
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/805/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-604235438
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2559/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401312862
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/CarbonExtensions.scala
 ##########
 @@ -76,6 +66,11 @@ case class CarbonOptimizerRule(session: SparkSession) extends Rule[LogicalPlan]
       self.synchronized {
         if (notAdded) {
           notAdded = false
+
+          session.udf.register(MaterializedViewUDF.DUMMY_FUNCTION, () => "")
 
 Review comment:
   move code to CarbonEnv.init

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r399961269
 
 

 ##########
 File path: common/src/main/java/org/apache/carbondata/common/exceptions/sql/MalformedIndexCommandException.java
 ##########
 @@ -21,21 +21,21 @@
 import org.apache.carbondata.common.annotations.InterfaceStability;
 
 /**
- * This exception will be thrown when Datamap related SQL statement is invalid
+ * This exception will be thrown when index related SQL statement is invalid
 
 Review comment:
   @niuge01 and @jackylk : As per my understanding. After this PR. There won't be datamap word in the code. It is either index or materialized view. But **many package names still has datamap** [exmaple CgDatamap, FgDatamap] and files also exist with datamap name. [Example DataMapStatus, DataMapStatusManager] 
   Do we need to completly remove if weare doing this change.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396370616
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
 ##########
 @@ -101,6 +100,12 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     loadDataNew | explainPlan | alterTableColumnRenameAndModifyDataType |
     alterTableAddColumns
 
+  protected lazy val materializedViewCommands: Parser[LogicalPlan] =
 
 Review comment:
   As already a new Parser => `MVParser` is already present. Why it is added?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401356360
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
 ##########
 @@ -101,6 +100,12 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     loadDataNew | explainPlan | alterTableColumnRenameAndModifyDataType |
     alterTableAddColumns
 
+  protected lazy val materializedViewCommands: Parser[LogicalPlan] =
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-597534422
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2421/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596065067
 
 
   retest this please

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401316305
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/IndexMeta.java
 ##########
 @@ -29,24 +29,24 @@
 import org.apache.commons.lang3.StringUtils;
 
 /**
- * Metadata of the datamap, set by DataMap developer
+ * Metadata of the datamap, set by Index developer
  */
-@InterfaceAudience.Developer("DataMap")
+@InterfaceAudience.Developer("Index")
 @InterfaceStability.Evolving
-public class DataMapMeta {
+public class IndexMeta {
 
 Review comment:
   if we change the class name, better to change the content also.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601632452
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2522/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596339810
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2389/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596086163
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/663/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r399883644
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
 ##########
 @@ -1595,6 +1595,24 @@ public String getSystemFolderLocation() {
     return systemLocation + CarbonCommonConstants.FILE_SEPARATOR + "_system";
   }
 
+  /**
+   * Get the configured system folder location.
+   * @return
+   */
+  public String getSystemFolderLocation(String databaseName) {
+    String systemLocation = CarbonProperties.getInstance()
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396365929
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/DDLHelper.scala
 ##########
 @@ -379,13 +380,17 @@ object DDLHelper {
           throw new MalformedCarbonCommandException(
             "Streaming property value is incorrect")
         }
-        if (carbonTable.hasMVCreated) {
+        if (carbonTable.isMaterializedView) {
+          throw new MalformedCarbonCommandException(
+            "Datamap table does not support set streaming property")
 
 Review comment:
   Change the exception to mv

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r399962377
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -2265,6 +2265,11 @@ private CarbonCommonConstants() {
    */
   public static final String PARENT_TABLES = "parent_tables";
 
+  /**
+   * This property will be used to store table name's associated with mv
+   */
 
 Review comment:
   why not clean in this PR itself as it is meant for MV refactoring ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
QiangCai commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401320051
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
 ##########
 @@ -101,6 +100,12 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     loadDataNew | explainPlan | alterTableColumnRenameAndModifyDataType |
     alterTableAddColumns
 
+  protected lazy val materializedViewCommands: Parser[LogicalPlan] =
 
 Review comment:
   At some places, we use MaterializedView, but at some other places, we use MV.
   What's the rule? Better to unify to use "MV" word in code(beside of SQL syntax)
   
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601194047
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2510/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396392817
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/CarbonCreateIndexCommand.scala
 ##########
 @@ -0,0 +1,195 @@
+/*
+ * 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.spark.sql.execution.command.index
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.secondaryindex.command.IndexModel
+
+import org.apache.carbondata.common.exceptions.sql.{MalformedCarbonCommandException, MalformedIndexCommandException, NoSuchIndexException}
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datamap.DataMapStoreManager
+import org.apache.carbondata.core.datamap.status.DataMapStatusManager
+import org.apache.carbondata.core.metadata.ColumnarFormatVersion
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.schema.datamap.{DataMapClassProvider, DataMapProperty}
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, DataMapSchema}
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.datamap.IndexProvider
+import org.apache.carbondata.events._
+
+/**
+ * Below command class will be used to create index on table
+ * and updating the parent table about the index information
+ */
+case class CarbonCreateIndexCommand(
+    indexModel: IndexModel,
+    indexProviderName: String,
+    properties: Map[String, String],
+    ifNotExistsSet: Boolean = false,
+    var deferredRebuild: Boolean = false)
+  extends AtomicRunnableCommand {
+
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+  private var provider: IndexProvider = _
+  private var parentTable: CarbonTable = _
+  private var dataMapSchema: DataMapSchema = _
+
+  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
+    // since streaming segment does not support building index yet,
+    // so streaming table does not support create index
+    parentTable = CarbonEnv.getCarbonTable(indexModel.dbName, indexModel.tableName)(sparkSession)
+    val indexName = indexModel.indexName
+
+    setAuditTable(parentTable)
+    setAuditInfo(Map("provider" -> indexProviderName, "indexName" -> indexName) ++ properties)
+
+    if (!parentTable.getTableInfo.isTransactionalTable) {
+      throw new MalformedCarbonCommandException("Unsupported operation on non transactional table")
+    }
+
+    if (DataMapStoreManager.getInstance().isDataMapExist(parentTable.getTableId, indexName)) {
+      if (!ifNotExistsSet) {
+        throw new NoSuchIndexException(indexName)
 
 Review comment:
   should throw index exists exception

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on issue #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on issue #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-607197936
 
 
   @niuge01 : Look for datamap in *.md file. Still many exmaples has create datamap syntax.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-606019193
 
 
   @niuge01 Please describe what all index related changes are done in this PR also

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 edited a comment on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 edited a comment on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-606019193
 
 
   @niuge01 @jackylk  Please describe what all index related changes are done in this PR also

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596086285
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2369/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-596065822
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/659/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-600021017
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2488/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396326086
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/view/MaterializedViewProvider.java
 ##########
 @@ -0,0 +1,573 @@
+/*
+ * 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.core.view;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperationFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
+import org.apache.carbondata.core.fileoperations.FileWriteOperation;
+import org.apache.carbondata.core.locks.CarbonLockFactory;
+import org.apache.carbondata.core.locks.CarbonLockUtil;
+import org.apache.carbondata.core.locks.ICarbonLock;
+import org.apache.carbondata.core.locks.LockUsage;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatus;
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+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;
+import org.apache.log4j.Logger;
+
+/**
+ * Stores mv schema in disk as json format
+ */
+@InterfaceAudience.Internal
+public class MaterializedViewProvider {
+
+  private static final Logger LOG = LogServiceFactory.getLogService(
+      MaterializedViewProvider.class.getCanonicalName());
+
+  private static final String STATUS_FILE_NAME = "mv_status";
+
+  private final String storeLocation;
+
+  private final Map<String, SchemaProvider> schemaProviders = new ConcurrentHashMap<>();
+
+  private MaterializedViewProvider(String storeLocation) {
+    this.storeLocation = storeLocation;
+  }
+
+  public static MaterializedViewProvider get() {
+    String storeLocation =
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION);
+    if (storeLocation == null) {
+      throw new RuntimeException(
+          "Property [" + CarbonCommonConstants.STORE_LOCATION + "] is not set.");
+    }
+    return new MaterializedViewProvider(storeLocation);
+  }
+
+  private static String getSchemaPath(String schemaRoot, String viewName) {
+    return schemaRoot + CarbonCommonConstants.FILE_SEPARATOR + "mv_schema." + viewName;
+  }
+
+  private SchemaProvider getSchemaProvider(String databaseName) {
+    String databaseNameUpper = databaseName.toUpperCase();
+    SchemaProvider schemaProvider = this.schemaProviders.get(databaseNameUpper);
+    if (schemaProvider == null) {
+      synchronized (this.schemaProviders) {
+        schemaProvider = this.schemaProviders.get(databaseNameUpper);
+        if (schemaProvider == null) {
+          String databaseLocation;
+          if (databaseNameUpper.equalsIgnoreCase("default")) {
 
 Review comment:
   Use CarbonCommonConstants.DATABASE_DEFAULT_NAME

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r400663521
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/datamap/IndexChooser.java
 ##########
 @@ -59,76 +59,76 @@
  *   the datamap which has fewer columns that is the first datamap.
  */
 @InterfaceAudience.Internal
-public class DataMapChooser {
+public class IndexChooser {
 
   private CarbonTable carbonTable;
-  private List<TableDataMap> cgDataMaps;
-  private List<TableDataMap> fgDataMaps;
+  private List<TableIndex> cgIndexes;
+  private List<TableIndex> fgIndexes;
 
-  public DataMapChooser(CarbonTable carbonTable) throws IOException {
+  public IndexChooser(CarbonTable carbonTable) throws IOException {
     this.carbonTable = carbonTable;
-    // read all datamaps for this table and populate CG and FG datamap list
-    List<TableDataMap> visibleDataMaps =
-        DataMapStoreManager.getInstance().getAllVisibleDataMap(carbonTable);
+    // read all indexes for this table and populate CG and FG index list
+    List<TableIndex> visibleIndexes =
+        DataMapStoreManager.getInstance().getAllVisibleIndexes(carbonTable);
     Map<String, DataMapStatusDetail> map = DataMapStatusManager.readDataMapStatusMap();
-    cgDataMaps = new ArrayList<>(visibleDataMaps.size());
-    fgDataMaps = new ArrayList<>(visibleDataMaps.size());
-    for (TableDataMap visibleDataMap : visibleDataMaps) {
-      DataMapStatusDetail status = map.get(visibleDataMap.getDataMapSchema().getDataMapName());
+    cgIndexes = new ArrayList<>(visibleIndexes.size());
+    fgIndexes = new ArrayList<>(visibleIndexes.size());
+    for (TableIndex visibleIndex : visibleIndexes) {
+      DataMapStatusDetail status = map.get(visibleIndex.getDataMapSchema().getDataMapName());
       if (status != null && status.isEnabled()) {
-        DataMapLevel level = visibleDataMap.getDataMapFactory().getDataMapLevel();
-        if (level == DataMapLevel.CG) {
-          cgDataMaps.add(visibleDataMap);
+        IndexLevel level = visibleIndex.getIndexFactory().getDataMapLevel();
+        if (level == IndexLevel.CG) {
+          cgIndexes.add(visibleIndex);
         } else {
-          fgDataMaps.add(visibleDataMap);
+          fgIndexes.add(visibleIndex);
         }
       }
     }
   }
 
   /**
-   * Return a chosen datamap based on input filter. See {@link DataMapChooser}
+   * Return a chosen datamap based on input filter. See {@link IndexChooser}
    */
-  public DataMapExprWrapper choose(FilterResolverIntf filter) {
+  public IndexExprWrapper choose(FilterResolverIntf filter) {
     if (filter != null) {
       Expression expression = filter.getFilterExpression();
       // First check for FG datamaps if any exist
-      ExpressionTuple tuple = selectDataMap(expression, fgDataMaps, filter);
-      if (tuple.dataMapExprWrapper == null) {
+      ExpressionTuple tuple = selectDataMap(expression, fgIndexes, filter);
+      if (tuple.indexExprWrapper == null) {
         // Check for CG datamap
-        tuple = selectDataMap(expression, cgDataMaps, filter);
+        tuple = selectDataMap(expression, cgIndexes, filter);
       }
-      if (tuple.dataMapExprWrapper != null) {
-        return tuple.dataMapExprWrapper;
+      if (tuple.indexExprWrapper != null) {
+        return tuple.indexExprWrapper;
       }
     }
     // Return the default datamap if no other datamap exists.
-    return new DataMapExprWrapperImpl(
-        DataMapStoreManager.getInstance().getDefaultDataMap(carbonTable), filter);
+    return new IndexExprWrapperImpl(
+        DataMapStoreManager.getInstance().getDefaultIndex(carbonTable), filter);
   }
 
   /**
-   * Return a chosen FG datamap based on input filter. See {@link DataMapChooser}
+   * Return a chosen FG datamap based on input filter. See {@link IndexChooser}
    */
-  public DataMapExprWrapper chooseFGDataMap(FilterResolverIntf resolverIntf) {
-    return chooseDataMap(DataMapLevel.FG, resolverIntf);
+  public IndexExprWrapper chooseFGDataMap(FilterResolverIntf resolverIntf) {
+    return chooseDataMap(IndexLevel.FG, resolverIntf);
   }
 
   /**
-   * Return a chosen CG datamap based on input filter. See {@link DataMapChooser}
+   * Return a chosen CG datamap based on input filter. See {@link IndexChooser}
    */
-  public DataMapExprWrapper chooseCGDataMap(FilterResolverIntf resolverIntf) {
-    return chooseDataMap(DataMapLevel.CG, resolverIntf);
+  public IndexExprWrapper chooseCGDataMap(FilterResolverIntf resolverIntf) {
+    return chooseDataMap(IndexLevel.CG, resolverIntf);
   }
 
-  DataMapExprWrapper chooseDataMap(DataMapLevel level, FilterResolverIntf resolverIntf) {
+  IndexExprWrapper chooseDataMap(IndexLevel level, FilterResolverIntf resolverIntf) {
     if (resolverIntf != null) {
       Expression expression = resolverIntf.getFilterExpression();
-      List<TableDataMap> datamaps = level == DataMapLevel.CG ? cgDataMaps : fgDataMaps;
+      List<TableIndex> datamaps = level == IndexLevel.CG ? cgIndexes : fgIndexes;
 
 Review comment:
   change variable name also to indexes

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396976400
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/CarbonCreateIndexCommand.scala
 ##########
 @@ -0,0 +1,195 @@
+/*
+ * 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.spark.sql.execution.command.index
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.execution.command._
+import org.apache.spark.sql.secondaryindex.command.IndexModel
+
+import org.apache.carbondata.common.exceptions.sql.{MalformedCarbonCommandException, MalformedIndexCommandException, NoSuchIndexException}
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datamap.DataMapStoreManager
+import org.apache.carbondata.core.datamap.status.DataMapStatusManager
+import org.apache.carbondata.core.metadata.ColumnarFormatVersion
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.schema.datamap.{DataMapClassProvider, DataMapProperty}
+import org.apache.carbondata.core.metadata.schema.table.{CarbonTable, DataMapSchema}
+import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.apache.carbondata.datamap.IndexProvider
+import org.apache.carbondata.events._
+
+/**
+ * Below command class will be used to create index on table
+ * and updating the parent table about the index information
+ */
+case class CarbonCreateIndexCommand(
+    indexModel: IndexModel,
+    indexProviderName: String,
+    properties: Map[String, String],
+    ifNotExistsSet: Boolean = false,
+    var deferredRebuild: Boolean = false)
+  extends AtomicRunnableCommand {
+
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+  private var provider: IndexProvider = _
+  private var parentTable: CarbonTable = _
+  private var dataMapSchema: DataMapSchema = _
+
+  override def processMetadata(sparkSession: SparkSession): Seq[Row] = {
+    // since streaming segment does not support building index yet,
+    // so streaming table does not support create index
+    parentTable = CarbonEnv.getCarbonTable(indexModel.dbName, indexModel.tableName)(sparkSession)
+    val indexName = indexModel.indexName
+
+    setAuditTable(parentTable)
+    setAuditInfo(Map("provider" -> indexProviderName, "indexName" -> indexName) ++ properties)
+
+    if (!parentTable.getTableInfo.isTransactionalTable) {
+      throw new MalformedCarbonCommandException("Unsupported operation on non transactional table")
+    }
+
+    if (DataMapStoreManager.getInstance().isDataMapExist(parentTable.getTableId, indexName)) {
+      if (!ifNotExistsSet) {
+        throw new NoSuchIndexException(indexName)
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396958744
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/view/MaterializedViewProvider.java
 ##########
 @@ -0,0 +1,573 @@
+/*
+ * 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.core.view;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFileFilter;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperationFactory;
+import org.apache.carbondata.core.fileoperations.AtomicFileOperations;
+import org.apache.carbondata.core.fileoperations.FileWriteOperation;
+import org.apache.carbondata.core.locks.CarbonLockFactory;
+import org.apache.carbondata.core.locks.CarbonLockUtil;
+import org.apache.carbondata.core.locks.ICarbonLock;
+import org.apache.carbondata.core.locks.LockUsage;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+import org.apache.carbondata.core.metadata.schema.table.RelationIdentifier;
+import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
+import org.apache.carbondata.core.statusmanager.SegmentStatus;
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
+import org.apache.carbondata.core.util.CarbonProperties;
+import org.apache.carbondata.core.util.CarbonUtil;
+import org.apache.carbondata.core.util.path.CarbonTablePath;
+
+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;
+import org.apache.log4j.Logger;
+
+/**
+ * Stores mv schema in disk as json format
+ */
+@InterfaceAudience.Internal
+public class MaterializedViewProvider {
+
+  private static final Logger LOG = LogServiceFactory.getLogService(
+      MaterializedViewProvider.class.getCanonicalName());
+
+  private static final String STATUS_FILE_NAME = "mv_status";
+
+  private final String storeLocation;
+
+  private final Map<String, SchemaProvider> schemaProviders = new ConcurrentHashMap<>();
+
+  private MaterializedViewProvider(String storeLocation) {
+    this.storeLocation = storeLocation;
+  }
+
+  public static MaterializedViewProvider get() {
+    String storeLocation =
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION);
+    if (storeLocation == null) {
+      throw new RuntimeException(
+          "Property [" + CarbonCommonConstants.STORE_LOCATION + "] is not set.");
+    }
+    return new MaterializedViewProvider(storeLocation);
+  }
+
+  private static String getSchemaPath(String schemaRoot, String viewName) {
+    return schemaRoot + CarbonCommonConstants.FILE_SEPARATOR + "mv_schema." + viewName;
+  }
+
+  private SchemaProvider getSchemaProvider(String databaseName) {
+    String databaseNameUpper = databaseName.toUpperCase();
+    SchemaProvider schemaProvider = this.schemaProviders.get(databaseNameUpper);
+    if (schemaProvider == null) {
+      synchronized (this.schemaProviders) {
+        schemaProvider = this.schemaProviders.get(databaseNameUpper);
+        if (schemaProvider == null) {
+          String databaseLocation;
+          if (databaseNameUpper.equalsIgnoreCase("default")) {
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-601527815
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/808/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396363915
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
 ##########
 @@ -203,10 +204,18 @@ private[sql] case class CarbonProjectForUpdateCommand(
       val allDataMapSchemas = DataMapStoreManager.getInstance
         .getDataMapSchemasOfTable(carbonTable).asScala
         .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                 !dataMapSchema.isIndexDataMap).asJava
+                                 !dataMapSchema.isIndex).asJava
 
 Review comment:
   Can remove code from line 204 to 210 added for mv, after this refactory

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r398280930
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/DiskBasedDMSchemaStorageProvider.java
 ##########
 @@ -172,17 +172,17 @@ public void dropSchema(String dataMapName)
       throws IOException {
     String schemaPath = getSchemaPath(storePath, dataMapName);
     if (!FileFactory.isFileExist(schemaPath)) {
-      throw new IOException("DataMap with name " + dataMapName + " does not exists in storage");
+      throw new IOException("Index with name " + dataMapName + " does not exists in storage");
     }
 
-    LOG.info(String.format("Trying to delete DataMap %s schema", dataMapName));
+    LOG.info(String.format("Trying to delete Index %s schema", dataMapName));
 
 Review comment:
   ```suggestion
       LOG.info(String.format("Trying to delete Index %s schema", indexName));
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-602567797
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2543/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-603136100
 
 
   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2545/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395968122
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
 ##########
 @@ -1005,8 +1005,8 @@ public int getNumberOfSegmentsToBePreserved() {
   }
 
   public void print() {
-    LOGGER.info("------Using Carbon.properties --------");
-    LOGGER.info(carbonProperties.toString());
+    LOGGER.error("------Using Carbon.properties --------");
 
 Review comment:
   This is test code, will change it back to info

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401355336
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -460,7 +460,7 @@ private CarbonCommonConstants() {
   public static final String LOCAL_DICTIONARY_EXCLUDE = "local_dictionary_exclude";
 
   /**
-   * DMPROPERTY for Index DataMap, like lucene, bloomfilter DataMap,
+   * DMPROPERTY for Index Index, like lucene, bloomfilter Index,
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [CARBONDATA-3704] Support create materialized view on all type table, and make mv support mutil-tenant.
URL: https://github.com/apache/carbondata/pull/3661#discussion_r401377301
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/view/MaterializedViewCatalog.java
 ##########
 @@ -0,0 +1,47 @@
+/*
 
 Review comment:
   we can start a new work to extract mv code to a single module after this pr.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r399969838
 
 

 ##########
 File path: dev/findbugs-exclude.xml
 ##########
 @@ -59,7 +59,7 @@
     <Bug pattern="STCAL_INVOKE_ON_STATIC_DATE_FORMAT_INSTANCE"/>
   </Match>
   <Match>
-    <Class name="org.apache.carbondata.core.datamap.DistributableDataMapFormat"/>
+    <Class name="org.apache.carbondata.core.datamap.IndexInputFormat"/>
 
 Review comment:
   Example, index-developer-guide.md has many datamap words in it still

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r396974227
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
 ##########
 @@ -65,7 +66,7 @@ case class CarbonCleanFilesCommand(
       val allDataMapSchemas = DataMapStoreManager.getInstance
         .getDataMapSchemasOfTable(carbonTable).asScala
         .filter(dataMapSchema => null != dataMapSchema.getRelationIdentifier &&
-                                 !dataMapSchema.isIndexDataMap)
+                                 !dataMapSchema.isIndex)
 
 Review comment:
   Yes, will be delete when clean the old mv implementation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#issuecomment-600048447
 
 
   Build Failed  with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/783/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
niuge01 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395969702
 
 

 ##########
 File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/Jobs/BlockletIndexInputFormate.java
 ##########
 @@ -56,52 +56,52 @@
 /**
  * class to load blocklet data map
  */
-public class DistributableBlockletDataMapLoader
-    extends FileInputFormat<TableBlockIndexUniqueIdentifier, BlockletDataMapDetailsWithSchema>
+public class BlockletIndexInputFormate
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #3661: [WIP] Support materialized view
URL: https://github.com/apache/carbondata/pull/3661#discussion_r395461796
 
 

 ##########
 File path: common/src/main/java/org/apache/carbondata/common/exceptions/sql/NoSuchMaterializedViewException.java
 ##########
 @@ -0,0 +1,40 @@
+/*
+ * 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.common.exceptions.sql;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.common.annotations.InterfaceStability;
+
+/**
+ * This exception will be thrown if mv is not found when executing mv
+ * related SQL statement
+ */
+@InterfaceAudience.User
+@InterfaceStability.Stable
+public class NoSuchMaterializedViewException extends MalformedCarbonCommandException {
+
+  /**
+   * default serial version ID.
+   */
+  private static final long serialVersionUID = 1L;
+
+  public NoSuchMaterializedViewException(String databaseName, String mvName) {
+    super("Datamap with name " + databaseName + "." + mvName + " does not exist");
 
 Review comment:
   Should be Materialised view not found

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services