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

2020-03-23 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-23 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-23 Thread GitBox
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 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


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

2020-03-23 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-23 Thread GitBox
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 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 {

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

2020-03-23 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-23 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-23 Thread GitBox
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 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 {

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

2020-03-23 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-23 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-23 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-23 Thread GitBox
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 

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

2020-03-23 Thread GitBox
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 segments, 
List
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 validSegments, List 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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-23 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-23 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-23 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-23 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-23 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-23 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-23 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-20 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-20 Thread GitBox
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 cgDataMaps;
-  private List fgDataMaps;
+  private List cgIndexes;
+  private List 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 visibleDataMaps =
-DataMapStoreManager.getInstance().getAllVisibleDataMap(carbonTable);
+List 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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-20 Thread GitBox
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 dataMapCatalogs = null;
+  private Map 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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-20 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-20 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-20 Thread GitBox
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
+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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-20 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-20 Thread GitBox
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> getAllDataMaps() {
+  public Map> getTableIndexForAllTables() {
 return allDataMaps;
   }
 
   /**
* Contains the list of datamaps for each table.
*/
-  private Map> allDataMaps = new 
ConcurrentHashMap<>();
+  private Map> 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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-03-20 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services