[GitHub] [carbondata] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-05-02 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r280418332
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/DataMapProvider.java
 ##
 @@ -106,13 +122,206 @@ public void initData() { }
* 1. after datamap creation and if `autoRefreshDataMap` is set to true
* 2. user manually trigger REBUILD DATAMAP command
*/
-  public abstract void rebuild() throws IOException, NoSuchDataMapException;
+  public boolean rebuild() throws IOException, NoSuchDataMapException {
+String newLoadName = "";
+String segmentMap = "";
+AbsoluteTableIdentifier dataMapTableAbsoluteTableIdentifier = 
AbsoluteTableIdentifier
+.from(dataMapSchema.getRelationIdentifier().getTablePath(),
 
 Review comment:
   Check for `getRelationIdentifier` and return false directly if it is null.


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-29 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r279248051
 
 

 ##
 File path: 
datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVDataMapProvider.scala
 ##
 @@ -91,29 +115,280 @@ class MVDataMapProvider(
   val queryPlan = SparkSQLUtil.execute(
 sparkSession.sql(updatedQuery).queryExecution.analyzed,
 sparkSession).drop("preAgg")
-  val header = logicalPlan.output.map(_.name).mkString(",")
+  var isOverwriteTable = false
+  val isFullRefresh =
+if (null != dataMapSchema.getProperties.get("full_refresh")) {
+  dataMapSchema.getProperties.get("full_refresh").toBoolean
+} else {
+  false
+}
+  if (isFullRefresh) {
+isOverwriteTable = true
+  }
+  val dataMapTable = CarbonTable
+.buildFromTablePath(identifier.getTableName,
+  identifier.getDatabaseName,
+  identifier.getTablePath,
+  identifier.getTableId)
+  // Get new load for datamap by loading specified main table segments
+  val newDataMapLoad = incrementalBuild(isOverwriteTable,
+dataMapTable.getAbsoluteTableIdentifier,
+dataMapTable.getMetadataPath)
+  if(newDataMapLoad.isEmpty) {
+return false
+  }
+  val header = 
dataMapTable.getTableInfo.getFactTable.getListOfColumns.asScala
+.filter { column =>
+  !column.getColumnName
+
.equalsIgnoreCase(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE)
+}.sortBy(_.getSchemaOrdinal).map(_.getColumnName).mkString(",")
   val loadCommand = CarbonLoadDataCommand(
 databaseNameOp = Some(identifier.getDatabaseName),
 tableName = identifier.getTableName,
 factPathFromUser = null,
 dimFilesPath = Seq(),
 options = scala.collection.immutable.Map("fileheader" -> header),
-isOverwriteTable = true,
+isOverwriteTable,
 inputSqlString = null,
 dataFrame = Some(queryPlan),
 updateModel = None,
 tableInfoOp = None,
-internalOptions = Map.empty,
+internalOptions = Map("mergedSegmentName" -> newDataMapLoad),
 partition = Map.empty)
 
-  SparkSQLUtil.execute(loadCommand, sparkSession)
+  try {
+SparkSQLUtil.execute(loadCommand, sparkSession)
+  } catch {
+case ex: Exception =>
+  DataMapStatusManager.disableDataMap(dataMapSchema.getDataMapName)
+  LOGGER.error("Data Load failed for DataMap: ", ex)
+  return false
+  } finally {
+unsetMainTableSegments()
+  }
 }
+true
   }
 
+  /**
+   * This method builds new load for datamap by loading specified segment data 
and returns
+   * new load name for datamap table load
+   */
   @throws[IOException]
-  override def incrementalBuild(
-  segmentIds: Array[String]): Unit = {
-throw new UnsupportedOperationException
+  override def incrementalBuild(isOverwriteTable: Boolean,
+  dataMapTableAbsoluteTableIdentifier: AbsoluteTableIdentifier,
+  dataMapTableMetadataPath: String): String = {
+var loadMetaDataDetails = 
SegmentStatusManager.readLoadMetadata(dataMapTableMetadataPath)
+var newLoadName : String = ""
+var segmentMap: String = ""
+val segmentStatusManager = new 
SegmentStatusManager(dataMapTableAbsoluteTableIdentifier)
+// Acquire table status lock to handle concurrent dataloading
+val carbonLock: ICarbonLock = segmentStatusManager.getTableStatusLock
 
 Review comment:
   Please move it to the parent class and add abstract method 
rebuildInteral(String loadName, Map >)


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-29 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r279247186
 
 

 ##
 File path: 
datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVDataMapProvider.scala
 ##
 @@ -91,29 +115,280 @@ class MVDataMapProvider(
   val queryPlan = SparkSQLUtil.execute(
 sparkSession.sql(updatedQuery).queryExecution.analyzed,
 sparkSession).drop("preAgg")
-  val header = logicalPlan.output.map(_.name).mkString(",")
+  var isOverwriteTable = false
+  val isFullRefresh =
+if (null != dataMapSchema.getProperties.get("full_refresh")) {
+  dataMapSchema.getProperties.get("full_refresh").toBoolean
+} else {
+  false
+}
+  if (isFullRefresh) {
+isOverwriteTable = true
+  }
+  val dataMapTable = CarbonTable
+.buildFromTablePath(identifier.getTableName,
+  identifier.getDatabaseName,
+  identifier.getTablePath,
+  identifier.getTableId)
+  // Get new load for datamap by loading specified main table segments
+  val newDataMapLoad = incrementalBuild(isOverwriteTable,
+dataMapTable.getAbsoluteTableIdentifier,
+dataMapTable.getMetadataPath)
+  if(newDataMapLoad.isEmpty) {
+return false
+  }
+  val header = 
dataMapTable.getTableInfo.getFactTable.getListOfColumns.asScala
+.filter { column =>
+  !column.getColumnName
+
.equalsIgnoreCase(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE)
+}.sortBy(_.getSchemaOrdinal).map(_.getColumnName).mkString(",")
   val loadCommand = CarbonLoadDataCommand(
 databaseNameOp = Some(identifier.getDatabaseName),
 tableName = identifier.getTableName,
 factPathFromUser = null,
 dimFilesPath = Seq(),
 options = scala.collection.immutable.Map("fileheader" -> header),
-isOverwriteTable = true,
+isOverwriteTable,
 inputSqlString = null,
 dataFrame = Some(queryPlan),
 updateModel = None,
 tableInfoOp = None,
-internalOptions = Map.empty,
+internalOptions = Map("mergedSegmentName" -> newDataMapLoad),
 partition = Map.empty)
 
-  SparkSQLUtil.execute(loadCommand, sparkSession)
+  try {
+SparkSQLUtil.execute(loadCommand, sparkSession)
+  } catch {
+case ex: Exception =>
+  DataMapStatusManager.disableDataMap(dataMapSchema.getDataMapName)
+  LOGGER.error("Data Load failed for DataMap: ", ex)
+  return false
+  } finally {
+unsetMainTableSegments()
+  }
 }
+true
   }
 
+  /**
+   * This method builds new load for datamap by loading specified segment data 
and returns
+   * new load name for datamap table load
+   */
   @throws[IOException]
-  override def incrementalBuild(
-  segmentIds: Array[String]): Unit = {
-throw new UnsupportedOperationException
+  override def incrementalBuild(isOverwriteTable: Boolean,
 
 Review comment:
   PLease remove from interface as it is doing nothing


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-29 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r279248549
 
 

 ##
 File path: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mv/MVListeners.scala
 ##
 @@ -0,0 +1,137 @@
+/*
+* 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.mv
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.command.AlterTableModel
+import 
org.apache.spark.sql.execution.command.management.CarbonAlterTableCompactionCommand
+
+import org.apache.carbondata.core.datamap.DataMapStoreManager
+import org.apache.carbondata.core.datamap.status.DataMapStatusManager
+import 
org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.MV
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.datamap.DataMapManager
+import org.apache.carbondata.events.{AlterTableCompactionPreStatusUpdateEvent, 
DeleteFromTablePostEvent, Event, OperationContext, OperationEventListener, 
UpdateTablePostEvent}
+import 
org.apache.carbondata.processing.loading.events.LoadEvents.LoadTablePostExecutionEvent
+import org.apache.carbondata.processing.merger.CompactionType
+
+/**
+ * Listener to trigger compaction on mv datamap after main table compaction
+ */
+object AlterMVtableCompactionPostListener extends OperationEventListener {
 
 Review comment:
   Don't mention about above MV here


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-29 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r279248488
 
 

 ##
 File path: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForDeleteCommand.scala
 ##
 @@ -115,6 +118,12 @@ private[sql] case class CarbonProjectForDeleteCommand(
 throw new Exception(executorErrors.errorMsg)
   }
 
+  if (CarbonTable.hasMVDataMap(carbonTable)) {
 
 Review comment:
   Just check  whether it is child datamap


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-29 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r279248407
 
 

 ##
 File path: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
 ##
 @@ -70,6 +71,18 @@ case class CarbonCleanFilesCommand(
 isInternalCleanCall = true)
   }.toList
   cleanFileCommands.foreach(_.processMetadata(sparkSession))
+} else if (CarbonTable.hasMVDataMap(carbonTable)) {
+  val allDataMapSchemas = DataMapStoreManager.getInstance
+.getDataMapSchemasOfTable(carbonTable).asScala
+.filter(dataMapSchema => 
dataMapSchema.getProviderName.equalsIgnoreCase(MV.toString))
 
 Review comment:
   Please don't mention the MV here, 


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-29 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r279244910
 
 

 ##
 File path: 
datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVDataMapProvider.scala
 ##
 @@ -91,29 +115,280 @@ class MVDataMapProvider(
   val queryPlan = SparkSQLUtil.execute(
 sparkSession.sql(updatedQuery).queryExecution.analyzed,
 sparkSession).drop("preAgg")
-  val header = logicalPlan.output.map(_.name).mkString(",")
+  var isOverwriteTable = false
+  val isFullRefresh =
+if (null != dataMapSchema.getProperties.get("full_refresh")) {
+  dataMapSchema.getProperties.get("full_refresh").toBoolean
+} else {
+  false
+}
+  if (isFullRefresh) {
+isOverwriteTable = true
+  }
+  val dataMapTable = CarbonTable
+.buildFromTablePath(identifier.getTableName,
+  identifier.getDatabaseName,
+  identifier.getTablePath,
+  identifier.getTableId)
+  // Get new load for datamap by loading specified main table segments
+  val newDataMapLoad = incrementalBuild(isOverwriteTable,
+dataMapTable.getAbsoluteTableIdentifier,
+dataMapTable.getMetadataPath)
+  if(newDataMapLoad.isEmpty) {
+return false
+  }
+  val header = 
dataMapTable.getTableInfo.getFactTable.getListOfColumns.asScala
+.filter { column =>
+  !column.getColumnName
+
.equalsIgnoreCase(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE)
+}.sortBy(_.getSchemaOrdinal).map(_.getColumnName).mkString(",")
   val loadCommand = CarbonLoadDataCommand(
 databaseNameOp = Some(identifier.getDatabaseName),
 tableName = identifier.getTableName,
 factPathFromUser = null,
 dimFilesPath = Seq(),
 options = scala.collection.immutable.Map("fileheader" -> header),
-isOverwriteTable = true,
+isOverwriteTable,
 inputSqlString = null,
 dataFrame = Some(queryPlan),
 updateModel = None,
 tableInfoOp = None,
-internalOptions = Map.empty,
+internalOptions = Map("mergedSegmentName" -> newDataMapLoad),
 partition = Map.empty)
 
-  SparkSQLUtil.execute(loadCommand, sparkSession)
+  try {
+SparkSQLUtil.execute(loadCommand, sparkSession)
+  } catch {
+case ex: Exception =>
+  DataMapStatusManager.disableDataMap(dataMapSchema.getDataMapName)
+  LOGGER.error("Data Load failed for DataMap: ", ex)
+  return false
+  } finally {
+unsetMainTableSegments()
+  }
 }
+true
   }
 
+  /**
+   * This method builds new load for datamap by loading specified segment data 
and returns
+   * new load name for datamap table load
+   */
   @throws[IOException]
-  override def incrementalBuild(
-  segmentIds: Array[String]): Unit = {
-throw new UnsupportedOperationException
+  override def incrementalBuild(isOverwriteTable: Boolean,
+  dataMapTableAbsoluteTableIdentifier: AbsoluteTableIdentifier,
+  dataMapTableMetadataPath: String): String = {
+var loadMetaDataDetails = 
SegmentStatusManager.readLoadMetadata(dataMapTableMetadataPath)
 
 Review comment:
   Please don't read multiple times, read inside the lock once only


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-29 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r279243688
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
 ##
 @@ -122,9 +136,99 @@ public static void dropDataMap(String dataMapName) throws 
IOException, NoSuchDat
 }
   }
 
-  private static DataMapSchema getDataMapSchema(String dataMapName)
+  public static DataMapSchema getDataMapSchema(String dataMapName)
   throws IOException, NoSuchDataMapException {
 return DataMapStoreManager.getInstance().getDataMapSchema(dataMapName);
   }
 
+  /**
+   * This method will remove all segments of dataMap table in case of 
Insert-Overwrite/Update/Delete
+   * operations on main table
+   *
+   * @param allDataMapSchemas of main carbon table
+   * @throws IOException
+   */
+  public static void truncateDataMap(List allDataMapSchemas) 
throws IOException {
+for (DataMapSchema datamapschema : allDataMapSchemas) {
+  if (null != datamapschema.getRelationIdentifier() && 
!datamapschema.isIndexDataMap()) {
+RelationIdentifier dataMapRelationIdentifier = 
datamapschema.getRelationIdentifier();
+SegmentStatusManager segmentStatusManager = new 
SegmentStatusManager(AbsoluteTableIdentifier
+.from(dataMapRelationIdentifier.getTablePath(),
+dataMapRelationIdentifier.getDatabaseName(),
+dataMapRelationIdentifier.getTableName()));
+ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
+try {
+  if (carbonLock.lockWithRetries()) {
+LOGGER.info(
+"Acquired lock for table" + 
dataMapRelationIdentifier.getDatabaseName() + "."
++ dataMapRelationIdentifier.getTableName() + " for table 
status updation");
+String metaDataPath =
+
CarbonTablePath.getMetadataPath(dataMapRelationIdentifier.getTablePath());
+LoadMetadataDetails[] loadMetadataDetails =
+SegmentStatusManager.readLoadMetadata(metaDataPath);
+for (LoadMetadataDetails entry : loadMetadataDetails) {
+  entry.setSegmentStatus(SegmentStatus.MARKED_FOR_DELETE);
+}
+SegmentStatusManager.writeLoadDetailsIntoFile(
+
CarbonTablePath.getTableStatusFilePath(dataMapRelationIdentifier.getTablePath()),
+loadMetadataDetails);
+  } else {
+LOGGER.error("Not able to acquire the lock for Table status 
updation for table "
++ dataMapRelationIdentifier.getDatabaseName() + "." + 
dataMapRelationIdentifier
+.getTableName());
+  }
+} finally {
+  if (carbonLock.unlock()) {
+LOGGER.info("Table unlocked successfully after table status 
updation"
++ dataMapRelationIdentifier.getDatabaseName() + "." + 
dataMapRelationIdentifier
+.getTableName());
+  } else {
+LOGGER.error("Unable to unlock Table lock for table" + 
dataMapRelationIdentifier
+.getDatabaseName() + "." + 
dataMapRelationIdentifier.getTableName()
++ " during table status updation");
+  }
+}
+  }
+}
+  }
+
+  /**
+   * This method checks if main table and datamap table are synchronised or 
not. If synchronised
+   * return true to enable the datamap
+   * @param dataMapSchema of datamap to be disabled or enabled
+   * @return flag to enable or disable datamap
+   * @throws IOException
+   */
+  public static boolean canDataMapBeEnabled(DataMapSchema dataMapSchema) 
throws IOException {
 
 Review comment:
   This needs to be taken inside the lock of datamap status otherwise in 
concurrent scenarios one might enable the datamap which is intended to be 
disabled. So it should be in the same transaction. 


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-29 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r279242487
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
 ##
 @@ -122,9 +136,99 @@ public static void dropDataMap(String dataMapName) throws 
IOException, NoSuchDat
 }
   }
 
-  private static DataMapSchema getDataMapSchema(String dataMapName)
+  public static DataMapSchema getDataMapSchema(String dataMapName)
   throws IOException, NoSuchDataMapException {
 return DataMapStoreManager.getInstance().getDataMapSchema(dataMapName);
   }
 
+  /**
+   * This method will remove all segments of dataMap table in case of 
Insert-Overwrite/Update/Delete
+   * operations on main table
+   *
+   * @param allDataMapSchemas of main carbon table
+   * @throws IOException
+   */
+  public static void truncateDataMap(List allDataMapSchemas) 
throws IOException {
+for (DataMapSchema datamapschema : allDataMapSchemas) {
+  if (null != datamapschema.getRelationIdentifier() && 
!datamapschema.isIndexDataMap()) {
+RelationIdentifier dataMapRelationIdentifier = 
datamapschema.getRelationIdentifier();
+SegmentStatusManager segmentStatusManager = new 
SegmentStatusManager(AbsoluteTableIdentifier
+.from(dataMapRelationIdentifier.getTablePath(),
+dataMapRelationIdentifier.getDatabaseName(),
+dataMapRelationIdentifier.getTableName()));
+ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
+try {
+  if (carbonLock.lockWithRetries()) {
+LOGGER.info(
+"Acquired lock for table" + 
dataMapRelationIdentifier.getDatabaseName() + "."
++ dataMapRelationIdentifier.getTableName() + " for table 
status updation");
+String metaDataPath =
+
CarbonTablePath.getMetadataPath(dataMapRelationIdentifier.getTablePath());
+LoadMetadataDetails[] loadMetadataDetails =
+SegmentStatusManager.readLoadMetadata(metaDataPath);
+for (LoadMetadataDetails entry : loadMetadataDetails) {
+  entry.setSegmentStatus(SegmentStatus.MARKED_FOR_DELETE);
+}
+SegmentStatusManager.writeLoadDetailsIntoFile(
+
CarbonTablePath.getTableStatusFilePath(dataMapRelationIdentifier.getTablePath()),
+loadMetadataDetails);
+  } else {
+LOGGER.error("Not able to acquire the lock for Table status 
updation for table "
++ dataMapRelationIdentifier.getDatabaseName() + "." + 
dataMapRelationIdentifier
+.getTableName());
+  }
+} finally {
+  if (carbonLock.unlock()) {
+LOGGER.info("Table unlocked successfully after table status 
updation"
++ dataMapRelationIdentifier.getDatabaseName() + "." + 
dataMapRelationIdentifier
+.getTableName());
+  } else {
+LOGGER.error("Unable to unlock Table lock for table" + 
dataMapRelationIdentifier
+.getDatabaseName() + "." + 
dataMapRelationIdentifier.getTableName()
++ " during table status updation");
+  }
+}
+  }
+}
+  }
+
+  /**
+   * This method checks if main table and datamap table are synchronised or 
not. If synchronised
+   * return true to enable the datamap
+   * @param dataMapSchema of datamap to be disabled or enabled
+   * @return flag to enable or disable datamap
+   * @throws IOException
+   */
+  public static boolean canDataMapBeEnabled(DataMapSchema dataMapSchema) 
throws IOException {
+boolean isDataMapInSync = true;
+String metaDataPath =
+
CarbonTablePath.getMetadataPath(dataMapSchema.getRelationIdentifier().getTablePath());
+LoadMetadataDetails[] dataMapLoadMetadataDetails =
+SegmentStatusManager.readLoadMetadata(metaDataPath);
+Map> dataMapSegmentMap = new HashMap<>();
+for (LoadMetadataDetails loadMetadataDetail : dataMapLoadMetadataDetails) {
+  Map> segmentMap =
+  
DataMapSegmentStatusUtil.getSegmentMap(loadMetadataDetail.getExtraInfo());
+  if (dataMapSegmentMap.isEmpty()) {
+dataMapSegmentMap.putAll(segmentMap);
+  } else {
+for (Map.Entry> entry : segmentMap.entrySet()) {
+  dataMapSegmentMap.get(entry.getKey()).addAll(entry.getValue());
 
 Review comment:
   What if the key is not available in the map? 


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-28 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r279241397
 
 

 ##
 File path: 
processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
 ##
 @@ -351,6 +357,27 @@ public static boolean 
updateLoadMetadataWithMergeStatus(List

[GitHub] [carbondata] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-28 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r279241157
 
 

 ##
 File path: 
processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
 ##
 @@ -351,6 +357,27 @@ public static boolean 
updateLoadMetadataWithMergeStatus(List

[GitHub] [carbondata] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-28 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r279241328
 
 

 ##
 File path: 
processing/src/main/java/org/apache/carbondata/processing/merger/CarbonDataMergerUtil.java
 ##
 @@ -351,6 +357,27 @@ public static boolean 
updateLoadMetadataWithMergeStatus(List

[GitHub] [carbondata] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-26 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278910691
 
 

 ##
 File path: 
processing/src/main/java/org/apache/carbondata/processing/util/CarbonLoaderUtil.java
 ##
 @@ -304,6 +305,11 @@ public static boolean 
recordNewLoadMetadata(LoadMetadataDetails newMetaEntry,
 }
 indexToOverwriteNewMetaEntry++;
   }
+  if 
(DataMapUtil.isMVdatamapTable(loadModel.getCarbonDataLoadSchema().getCarbonTable()))
 {
 
 Review comment:
   No need to check, please 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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-26 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278910282
 
 

 ##
 File path: 
processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModel.java
 ##
 @@ -983,4 +988,12 @@ public void setScaleFactor(int scaleFactor) {
   public int getScaleFactor() {
 return scaleFactor;
   }
+
+  public String getExtraInfo() {
 
 Review comment:
   NO need, please remove 


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-26 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278910188
 
 

 ##
 File path: 
integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mv/MVListeners.scala
 ##
 @@ -0,0 +1,140 @@
+/*
+* 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.mv
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.execution.command.AlterTableModel
+import 
org.apache.spark.sql.execution.command.management.CarbonAlterTableCompactionCommand
+
+import org.apache.carbondata.core.datamap.DataMapStoreManager
+import org.apache.carbondata.core.datamap.status.DataMapStatusManager
+import 
org.apache.carbondata.core.metadata.schema.datamap.DataMapClassProvider.MV
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.datamap.DataMapManager
+import org.apache.carbondata.events.{AlterTableCompactionPreStatusUpdateEvent, 
DeleteFromTablePostEvent, Event, OperationContext, OperationEventListener, 
UpdateTablePostEvent}
+import 
org.apache.carbondata.processing.loading.events.LoadEvents.LoadTablePostExecutionEvent
+import org.apache.carbondata.processing.merger.CompactionType
+
+/**
+ * Listener to trigger compaction on mv datamap after main table compaction
+ */
+object AlterMVtableCompactionPostListener extends OperationEventListener {
+  /**
+   * Called on a specified event occurrence
+   *
+   * @param event
+   * @param operationContext
+   */
+  override def onEvent(event: Event, operationContext: OperationContext): Unit 
= {
+val compactionEvent = 
event.asInstanceOf[AlterTableCompactionPreStatusUpdateEvent]
+val carbonTable = compactionEvent.carbonTable
+val compactionType = compactionEvent.carbonMergerMapping.campactionType
+if (compactionType == CompactionType.CUSTOM) {
+  return
+}
+val carbonLoadModel = compactionEvent.carbonLoadModel
+val sparkSession = compactionEvent.sparkSession
+if (CarbonTable.hasMVDataMap(carbonTable)) {
+  val allDataMapSchemas = DataMapStoreManager.getInstance
+.getDataMapSchemasOfTable(carbonTable).asScala
+.filter(dataMapSchema => 
dataMapSchema.getProviderName.equalsIgnoreCase(MV.toString))
+  allDataMapSchemas.foreach { dataMapSchema =>
+  val childRelationIdentifier = dataMapSchema.getRelationIdentifier
+  val alterTableModel = 
AlterTableModel(Some(childRelationIdentifier.getDatabaseName),
+childRelationIdentifier.getTableName,
+None,
+compactionType.toString,
+Some(System.currentTimeMillis()),
+"")
+  operationContext.setProperty(
+dataMapSchema.getRelationIdentifier.getDatabaseName + "_" +
+dataMapSchema.getRelationIdentifier.getTableName + "_Segment",
+carbonLoadModel.getSegmentId)
+  CarbonAlterTableCompactionCommand(alterTableModel, operationContext 
= operationContext)
+.run(sparkSession)
+  }
+}
+  }
+}
+
+/**
+ * Listener to trigger data load on mv datamap after main table data load
+ */
+object LoadPostMVListener extends OperationEventListener {
+  /**
+   * Called on a specified event occurrence
+   *
+   * @param event
+   * @param operationContext
+   */
+  override def onEvent(event: Event, operationContext: OperationContext): Unit 
= {
+val sparkSession = SparkSession.getActiveSession.get
+val carbonTable: CarbonTable =
+  event match {
+case event: LoadTablePostExecutionEvent =>
+  val carbonLoadModelOption = Some(event.getCarbonLoadModel)
+  if (carbonLoadModelOption.isDefined) {
+val carbonLoadModel = carbonLoadModelOption.get
+carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
+  } else {
+null
+  }
+case event: UpdateTablePostEvent =>
+  val table = Some(event.carbonTable)
+  if (table.isDefined) {
+table.get
+  } else {
+null
+  }
+case 

[GitHub] [carbondata] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-26 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278907341
 
 

 ##
 File path: 
datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVDataMapProvider.scala
 ##
 @@ -91,22 +108,255 @@ class MVDataMapProvider(
   val queryPlan = SparkSQLUtil.execute(
 sparkSession.sql(updatedQuery).queryExecution.analyzed,
 sparkSession).drop("preAgg")
-  val header = logicalPlan.output.map(_.name).mkString(",")
+  var isOverwriteTable = false
+  val isFullRebuild =
+if (null != dataMapSchema.getProperties.get("full_refresh")) {
+  dataMapSchema.getProperties.get("full_refresh").toBoolean
+} else {
+  false
+}
+  if (isFullRebuild) {
+isOverwriteTable = true
+  }
+  queryPlan.queryExecution.optimizedPlan transformDown {
+case join@Join(l1, l2, jointype, condition) =>
+  // TODO: Support Incremental loading for multiple tables with join - 
CARBONDATA-3340
+  isOverwriteTable = true
+  join
+  }
+  val dataMapTable = CarbonTable
+.buildFromTablePath(identifier.getTableName,
+  identifier.getDatabaseName,
+  identifier.getTablePath,
+  identifier.getTableId)
+  if (dataMapSchema.isLazy) {
+// check if rebuild to datamap is already in progress and throw 
exception
+val loadMetaDataDetails = SegmentStatusManager
+  .readLoadMetadata(dataMapTable.getMetadataPath)
+if (loadMetaDataDetails.nonEmpty) {
+  for (loadMetaDetail <- loadMetaDataDetails) {
+if ((loadMetaDetail.getSegmentStatus == 
SegmentStatus.INSERT_IN_PROGRESS ||
+ loadMetaDetail.getSegmentStatus == 
SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS) &&
+SegmentStatusManager
+  .isLoadInProgress(dataMapTable.getAbsoluteTableIdentifier,
+loadMetaDetail.getLoadName)) {
+  throw new RuntimeException(
+"Rebuild to datamap " + dataMapSchema.getDataMapName + " is 
already in progress")
+}
+  }
+}
+  }
+  var segmentMap: String = ""
+  if (!isOverwriteTable) {
+// Set main table segments to load for incremental data loading.
+// Compare main table segments info with datamap table segment map 
info and load only newly
+// added segment from main table to datamap table
+val (isIncrementLoad, segmentMapping) = 
setSegmentsBasedOnMapping(dataMapTable
+  .getMetadataPath, dataMapTable.getAbsoluteTableIdentifier)
+if (!isIncrementLoad) {
+  return
+}
+segmentMap = segmentMapping
+  } else {
+val segmentMapping = new util.HashMap[String, java.util.List[String]]()
+val relationIdentifiers = dataMapSchema.getParentTables.asScala
+for (relationIdentifier <- relationIdentifiers) {
+  val mainTableSegmentList = 
getMainTableValidSegmentList(relationIdentifier)
+  segmentMapping.put(relationIdentifier.getTableName, 
mainTableSegmentList)
+}
+segmentMap = new Gson().toJson(segmentMapping)
+  }
+
+  val header = 
dataMapTable.getTableInfo.getFactTable.getListOfColumns.asScala
+.filter { column =>
+  !column.getColumnName
+
.equalsIgnoreCase(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE)
+}.sortBy(_.getSchemaOrdinal).map(_.getColumnName).mkString(",")
   val loadCommand = CarbonLoadDataCommand(
 databaseNameOp = Some(identifier.getDatabaseName),
 tableName = identifier.getTableName,
 factPathFromUser = null,
 dimFilesPath = Seq(),
 options = scala.collection.immutable.Map("fileheader" -> header),
-isOverwriteTable = true,
+isOverwriteTable,
 inputSqlString = null,
 dataFrame = Some(queryPlan),
 updateModel = None,
 tableInfoOp = None,
-internalOptions = Map.empty,
+internalOptions = 
Map(CarbonCommonConstants.DATAMAP_MAINTABLE_SEGMENTMAP -> segmentMap),
 partition = Map.empty)
 
-  SparkSQLUtil.execute(loadCommand, sparkSession)
+  try {
+SparkSQLUtil.execute(loadCommand, sparkSession)
+  } catch {
+case ex: Exception =>
+  DataMapStatusManager.disableDataMap(dataMapSchema.getDataMapName)
+  LOGGER.error("Data Load failed for DataMap: ", ex)
+  } finally {
+unsetMainTableSegments()
+  }
+}
+  }
+
+  /**
+   * 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 

[GitHub] [carbondata] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-26 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278905116
 
 

 ##
 File path: 
datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVDataMapProvider.scala
 ##
 @@ -55,6 +68,10 @@ class MVDataMapProvider(
   }
 
   override def initData(): Unit = {
+if (!dataMapSchema.isLazy) {
+  rebuild()
+  DataMapStatusManager.enableDataMap(dataMapSchema.getDataMapName)
 
 Review comment:
   During enabling of MV datamap we should check the mapping and make sure the 
parent and child datamaps are completely synchronized. 


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-26 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278904133
 
 

 ##
 File path: 
datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVDataMapProvider.scala
 ##
 @@ -91,22 +108,255 @@ class MVDataMapProvider(
   val queryPlan = SparkSQLUtil.execute(
 sparkSession.sql(updatedQuery).queryExecution.analyzed,
 sparkSession).drop("preAgg")
-  val header = logicalPlan.output.map(_.name).mkString(",")
+  var isOverwriteTable = false
+  val isFullRebuild =
+if (null != dataMapSchema.getProperties.get("full_refresh")) {
+  dataMapSchema.getProperties.get("full_refresh").toBoolean
+} else {
+  false
+}
+  if (isFullRebuild) {
+isOverwriteTable = true
+  }
+  queryPlan.queryExecution.optimizedPlan transformDown {
+case join@Join(l1, l2, jointype, condition) =>
+  // TODO: Support Incremental loading for multiple tables with join - 
CARBONDATA-3340
+  isOverwriteTable = true
+  join
+  }
+  val dataMapTable = CarbonTable
+.buildFromTablePath(identifier.getTableName,
+  identifier.getDatabaseName,
+  identifier.getTablePath,
+  identifier.getTableId)
+  if (dataMapSchema.isLazy) {
 
 Review comment:
   Please take a tablestatus lock on the table ,
 1. Check the mapping for which segments needs to be loaded, during the 
check you need consider inprogress segments. 
   2. Create a tablestatus entry to as inprogress along with extra info.
   3. If it is lazy and any load in progress just return exception.
   4. Pass the segmentid to the load as mergedName .
   5. During updating tablestatus again you can get the old inprogress segment 
and update it and make entry gain.


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-26 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278891244
 
 

 ##
 File path: 
datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVDataMapProvider.scala
 ##
 @@ -91,22 +108,255 @@ class MVDataMapProvider(
   val queryPlan = SparkSQLUtil.execute(
 sparkSession.sql(updatedQuery).queryExecution.analyzed,
 sparkSession).drop("preAgg")
-  val header = logicalPlan.output.map(_.name).mkString(",")
+  var isOverwriteTable = false
+  val isFullRebuild =
+if (null != dataMapSchema.getProperties.get("full_refresh")) {
+  dataMapSchema.getProperties.get("full_refresh").toBoolean
+} else {
+  false
+}
+  if (isFullRebuild) {
+isOverwriteTable = true
+  }
+  queryPlan.queryExecution.optimizedPlan transformDown {
+case join@Join(l1, l2, jointype, condition) =>
 
 Review comment:
   Better check it during the creation of datamap, no need to check for every 
load


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-26 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278889392
 
 

 ##
 File path: 
datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVDataMapProvider.scala
 ##
 @@ -55,6 +68,10 @@ class MVDataMapProvider(
   }
 
   override def initData(): Unit = {
+if (!dataMapSchema.isLazy) {
+  rebuild()
 
 Review comment:
   Return the boolean whether success or not, after that enable the datamap


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-26 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278886149
 
 

 ##
 File path: 
datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVAnalyzerRule.scala
 ##
 @@ -136,4 +139,37 @@ class MVAnalyzerRule(sparkSession: SparkSession) extends 
Rule[LogicalPlan] {
 }
 catalogs
   }
+
+  /**
+   * Check if any segments are set for main table for Query. If any segments 
are set, then
+   * skip mv datamap table for query
+   */
+  def isSetMainTableSegments(datasets: Array[SummaryDataset],
 
 Review comment:
   No need to check for `datasets` just check for `catalogs` is enough


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-26 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278884436
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
 ##
 @@ -122,9 +136,54 @@ public static void dropDataMap(String dataMapName) throws 
IOException, NoSuchDat
 }
   }
 
-  private static DataMapSchema getDataMapSchema(String dataMapName)
+  public static DataMapSchema getDataMapSchema(String dataMapName)
   throws IOException, NoSuchDataMapException {
 return DataMapStoreManager.getInstance().getDataMapSchema(dataMapName);
   }
 
+  /**
+   * This method will remove all segments of dataMap table in case of 
Insert-Overwrite/Update/Delete
+   * operations on main table
+   */
+  public static void cleanDataMap(List allDataMapSchemas) 
throws IOException {
 
 Review comment:
   Better rename as truncateDataMaps


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-26 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278883574
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
 ##
 @@ -122,9 +136,54 @@ public static void dropDataMap(String dataMapName) throws 
IOException, NoSuchDat
 }
   }
 
-  private static DataMapSchema getDataMapSchema(String dataMapName)
+  public static DataMapSchema getDataMapSchema(String dataMapName)
   throws IOException, NoSuchDataMapException {
 return DataMapStoreManager.getInstance().getDataMapSchema(dataMapName);
   }
 
+  /**
+   * This method will remove all segments of dataMap table in case of 
Insert-Overwrite/Update/Delete
+   * operations on main table
+   */
+  public static void cleanDataMap(List allDataMapSchemas) 
throws IOException {
+for (DataMapSchema datamapschema : allDataMapSchemas) {
+  if (datamapschema.getProviderName()
+  .equalsIgnoreCase(DataMapClassProvider.MV.getShortName())) {
 
 Review comment:
   No need to check for MV datamap, check for getRelationIdentifier


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-26 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278882539
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/statusmanager/LoadMetadataDetails.java
 ##
 @@ -133,6 +133,11 @@ public String getPartitionCount() {
 return partitionCount;
   }
 
+  /**
+   * extraInfo will contain segment mapping Information for mv table
 
 Review comment:
   It is not just for 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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-24 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278130624
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
 ##
 @@ -127,4 +150,115 @@ private static DataMapSchema getDataMapSchema(String 
dataMapName)
 return DataMapStoreManager.getInstance().getDataMapSchema(dataMapName);
   }
 
+  /**
+   * Returns valid segment list for a given RelationIdentifier
+   *
+   * @param relationIdentifier
+   * @return
+   * @throws IOException
+   */
+  public static List getSegmentList(RelationIdentifier 
relationIdentifier)
+  throws IOException {
+List segmentList = new ArrayList<>();
+AbsoluteTableIdentifier absoluteTableIdentifier =
+AbsoluteTableIdentifier.from(relationIdentifier.getTablePath());
+List validSegments =
+new 
SegmentStatusManager(absoluteTableIdentifier).getValidAndInvalidSegments()
+.getValidSegments();
+for (Segment segment : validSegments) {
+  segmentList.add(segment.getSegmentNo());
+}
+return segmentList;
+  }
+
+  /**
+   * This method will delete segment folders of the mv datamap table and 
update the
+   * datamapSegmentStatus map in case of Insert-Overwrite/Update operation on 
main table
+   */
+  public static void cleanMVdatamap(CarbonTable carbonTable) throws 
IOException {
+List allDataMapSchemas =
+
DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable);
+for (DataMapSchema datamapschema : allDataMapSchemas) {
+  if (datamapschema.getProviderName()
+  .equalsIgnoreCase(DataMapClassProvider.MV.getShortName())) {
+CarbonTable datamapTable = CarbonTable
+
.buildFromTablePath(datamapschema.getRelationIdentifier().getTableName(),
+datamapschema.getRelationIdentifier().getDatabaseName(),
+datamapschema.getRelationIdentifier().getTablePath(),
+datamapschema.getRelationIdentifier().getTableId());
+SegmentStatusManager segmentStatusManager =
+new 
SegmentStatusManager(datamapTable.getAbsoluteTableIdentifier());
+ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
+try {
+  if (carbonLock.lockWithRetries()) {
+LOGGER.info(
+"Acquired lock for table" + datamapTable.getDatabaseName() + 
"." + datamapTable
+.getTableName() + " for table status updation");
+LoadMetadataDetails[] loadMetadataDetails =
+
SegmentStatusManager.readLoadMetadata(datamapTable.getMetadataPath());
+List staleFolders = new ArrayList<>();
+for (LoadMetadataDetails entry : loadMetadataDetails) {
+  entry.setSegmentStatus(SegmentStatus.MARKED_FOR_DELETE);
+  // For insert overwrite, we will delete the old segment folder 
immediately
+  // So collect the old segments here
+  String segmentPath = CarbonTablePath
+  
.getSegmentPath(datamapschema.getRelationIdentifier().getTablePath(),
+  entry.getLoadName());
+  if (FileFactory.isFileExist(segmentPath, 
FileFactory.getFileType(segmentPath))) {
+staleFolders.add(FileFactory.getCarbonFile(segmentPath));
+  }
+}
+SegmentStatusManager.writeLoadDetailsIntoFile(CarbonTablePath
+
.getTableStatusFilePath(datamapschema.getRelationIdentifier().getTablePath()),
+loadMetadataDetails);
+// Update datamapSegmentStatus map for mv
+storageProvider.clearSegmentMapping(datamapschema);
+// Delete all old stale segment folders
+for (CarbonFile staleFolder : staleFolders) {
+  try {
+CarbonUtil.deleteFoldersAndFiles(staleFolder);
+  } catch (IOException | InterruptedException e) {
+LOGGER.error("Failed to delete stale folder: " + 
e.getMessage(), e);
+  }
+}
+  }
+} finally {
+  if (carbonLock.unlock()) {
+LOGGER.info("Table unlocked successfully after table status 
updation" + datamapTable
+.getDatabaseName() + "." + datamapTable.getTableName());
+  } else {
+LOGGER.error(
+"Unable to unlock Table lock for table" + 
datamapTable.getDatabaseName() + "."
++ datamapTable.getTableName() + " during table status 
updation");
+  }
+}
+  }
+}
+  }
+
+  public static void updateDataMapSegmentStatusAfterCompaction(CarbonTable 
carbonTable)
+  throws IOException, NoSuchDataMapException {
+DataMapSchema dataMapSchema = getDataMapSchema(carbonTable.getTableName()
+.substring(0, 

[GitHub] [carbondata] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-24 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278130234
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
 ##
 @@ -127,4 +150,115 @@ private static DataMapSchema getDataMapSchema(String 
dataMapName)
 return DataMapStoreManager.getInstance().getDataMapSchema(dataMapName);
   }
 
+  /**
+   * Returns valid segment list for a given RelationIdentifier
+   *
+   * @param relationIdentifier
+   * @return
+   * @throws IOException
+   */
+  public static List getSegmentList(RelationIdentifier 
relationIdentifier)
+  throws IOException {
+List segmentList = new ArrayList<>();
+AbsoluteTableIdentifier absoluteTableIdentifier =
+AbsoluteTableIdentifier.from(relationIdentifier.getTablePath());
+List validSegments =
+new 
SegmentStatusManager(absoluteTableIdentifier).getValidAndInvalidSegments()
+.getValidSegments();
+for (Segment segment : validSegments) {
+  segmentList.add(segment.getSegmentNo());
+}
+return segmentList;
+  }
+
+  /**
+   * This method will delete segment folders of the mv datamap table and 
update the
+   * datamapSegmentStatus map in case of Insert-Overwrite/Update operation on 
main table
+   */
+  public static void cleanMVdatamap(CarbonTable carbonTable) throws 
IOException {
+List allDataMapSchemas =
+
DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable);
+for (DataMapSchema datamapschema : allDataMapSchemas) {
+  if (datamapschema.getProviderName()
+  .equalsIgnoreCase(DataMapClassProvider.MV.getShortName())) {
+CarbonTable datamapTable = CarbonTable
+
.buildFromTablePath(datamapschema.getRelationIdentifier().getTableName(),
+datamapschema.getRelationIdentifier().getDatabaseName(),
+datamapschema.getRelationIdentifier().getTablePath(),
+datamapschema.getRelationIdentifier().getTableId());
+SegmentStatusManager segmentStatusManager =
+new 
SegmentStatusManager(datamapTable.getAbsoluteTableIdentifier());
+ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
+try {
+  if (carbonLock.lockWithRetries()) {
+LOGGER.info(
+"Acquired lock for table" + datamapTable.getDatabaseName() + 
"." + datamapTable
+.getTableName() + " for table status updation");
+LoadMetadataDetails[] loadMetadataDetails =
+
SegmentStatusManager.readLoadMetadata(datamapTable.getMetadataPath());
+List staleFolders = new ArrayList<>();
+for (LoadMetadataDetails entry : loadMetadataDetails) {
+  entry.setSegmentStatus(SegmentStatus.MARKED_FOR_DELETE);
+  // For insert overwrite, we will delete the old segment folder 
immediately
+  // So collect the old segments here
+  String segmentPath = CarbonTablePath
+  
.getSegmentPath(datamapschema.getRelationIdentifier().getTablePath(),
+  entry.getLoadName());
+  if (FileFactory.isFileExist(segmentPath, 
FileFactory.getFileType(segmentPath))) {
+staleFolders.add(FileFactory.getCarbonFile(segmentPath));
+  }
+}
+SegmentStatusManager.writeLoadDetailsIntoFile(CarbonTablePath
+
.getTableStatusFilePath(datamapschema.getRelationIdentifier().getTablePath()),
+loadMetadataDetails);
+// Update datamapSegmentStatus map for mv
+storageProvider.clearSegmentMapping(datamapschema);
+// Delete all old stale segment folders
+for (CarbonFile staleFolder : staleFolders) {
+  try {
+CarbonUtil.deleteFoldersAndFiles(staleFolder);
+  } catch (IOException | InterruptedException e) {
+LOGGER.error("Failed to delete stale folder: " + 
e.getMessage(), e);
+  }
+}
+  }
+} finally {
+  if (carbonLock.unlock()) {
+LOGGER.info("Table unlocked successfully after table status 
updation" + datamapTable
+.getDatabaseName() + "." + datamapTable.getTableName());
+  } else {
+LOGGER.error(
+"Unable to unlock Table lock for table" + 
datamapTable.getDatabaseName() + "."
++ datamapTable.getTableName() + " during table status 
updation");
+  }
+}
+  }
+}
+  }
+
+  public static void updateDataMapSegmentStatusAfterCompaction(CarbonTable 
carbonTable)
 
 Review comment:
   Please pass dataMapSchema and compacted segment id, don't pass table


This is an 

[GitHub] [carbondata] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-24 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278125365
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
 ##
 @@ -127,4 +150,115 @@ private static DataMapSchema getDataMapSchema(String 
dataMapName)
 return DataMapStoreManager.getInstance().getDataMapSchema(dataMapName);
   }
 
+  /**
+   * Returns valid segment list for a given RelationIdentifier
+   *
+   * @param relationIdentifier
+   * @return
+   * @throws IOException
+   */
+  public static List getSegmentList(RelationIdentifier 
relationIdentifier)
+  throws IOException {
+List segmentList = new ArrayList<>();
+AbsoluteTableIdentifier absoluteTableIdentifier =
+AbsoluteTableIdentifier.from(relationIdentifier.getTablePath());
+List validSegments =
+new 
SegmentStatusManager(absoluteTableIdentifier).getValidAndInvalidSegments()
+.getValidSegments();
+for (Segment segment : validSegments) {
+  segmentList.add(segment.getSegmentNo());
+}
+return segmentList;
+  }
+
+  /**
+   * This method will delete segment folders of the mv datamap table and 
update the
+   * datamapSegmentStatus map in case of Insert-Overwrite/Update operation on 
main table
+   */
+  public static void cleanMVdatamap(CarbonTable carbonTable) throws 
IOException {
 
 Review comment:
   Please name `carbonTable` as `mainTable`


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-24 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278124943
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
 ##
 @@ -127,4 +150,115 @@ private static DataMapSchema getDataMapSchema(String 
dataMapName)
 return DataMapStoreManager.getInstance().getDataMapSchema(dataMapName);
   }
 
+  /**
+   * Returns valid segment list for a given RelationIdentifier
+   *
+   * @param relationIdentifier
+   * @return
+   * @throws IOException
+   */
+  public static List getSegmentList(RelationIdentifier 
relationIdentifier)
+  throws IOException {
+List segmentList = new ArrayList<>();
+AbsoluteTableIdentifier absoluteTableIdentifier =
+AbsoluteTableIdentifier.from(relationIdentifier.getTablePath());
+List validSegments =
+new 
SegmentStatusManager(absoluteTableIdentifier).getValidAndInvalidSegments()
+.getValidSegments();
+for (Segment segment : validSegments) {
+  segmentList.add(segment.getSegmentNo());
+}
+return segmentList;
+  }
+
+  /**
+   * This method will delete segment folders of the mv datamap table and 
update the
+   * datamapSegmentStatus map in case of Insert-Overwrite/Update operation on 
main table
+   */
+  public static void cleanMVdatamap(CarbonTable carbonTable) throws 
IOException {
+List allDataMapSchemas =
+
DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable);
+for (DataMapSchema datamapschema : allDataMapSchemas) {
+  if (datamapschema.getProviderName()
+  .equalsIgnoreCase(DataMapClassProvider.MV.getShortName())) {
+CarbonTable datamapTable = CarbonTable
+
.buildFromTablePath(datamapschema.getRelationIdentifier().getTableName(),
+datamapschema.getRelationIdentifier().getDatabaseName(),
+datamapschema.getRelationIdentifier().getTablePath(),
+datamapschema.getRelationIdentifier().getTableId());
+SegmentStatusManager segmentStatusManager =
+new 
SegmentStatusManager(datamapTable.getAbsoluteTableIdentifier());
+ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
+try {
+  if (carbonLock.lockWithRetries()) {
+LOGGER.info(
+"Acquired lock for table" + datamapTable.getDatabaseName() + 
"." + datamapTable
+.getTableName() + " for table status updation");
+LoadMetadataDetails[] loadMetadataDetails =
+
SegmentStatusManager.readLoadMetadata(datamapTable.getMetadataPath());
+List staleFolders = new ArrayList<>();
+for (LoadMetadataDetails entry : loadMetadataDetails) {
+  entry.setSegmentStatus(SegmentStatus.MARKED_FOR_DELETE);
+  // For insert overwrite, we will delete the old segment folder 
immediately
+  // So collect the old segments here
+  String segmentPath = CarbonTablePath
+  
.getSegmentPath(datamapschema.getRelationIdentifier().getTablePath(),
+  entry.getLoadName());
+  if (FileFactory.isFileExist(segmentPath, 
FileFactory.getFileType(segmentPath))) {
+staleFolders.add(FileFactory.getCarbonFile(segmentPath));
+  }
+}
+SegmentStatusManager.writeLoadDetailsIntoFile(CarbonTablePath
+
.getTableStatusFilePath(datamapschema.getRelationIdentifier().getTablePath()),
+loadMetadataDetails);
+// Update datamapSegmentStatus map for mv
+storageProvider.clearSegmentMapping(datamapschema);
 
 Review comment:
   THis method should belong to here


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-24 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278124174
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
 ##
 @@ -127,4 +150,115 @@ private static DataMapSchema getDataMapSchema(String 
dataMapName)
 return DataMapStoreManager.getInstance().getDataMapSchema(dataMapName);
   }
 
+  /**
+   * Returns valid segment list for a given RelationIdentifier
+   *
+   * @param relationIdentifier
+   * @return
+   * @throws IOException
+   */
+  public static List getSegmentList(RelationIdentifier 
relationIdentifier)
+  throws IOException {
+List segmentList = new ArrayList<>();
+AbsoluteTableIdentifier absoluteTableIdentifier =
+AbsoluteTableIdentifier.from(relationIdentifier.getTablePath());
+List validSegments =
+new 
SegmentStatusManager(absoluteTableIdentifier).getValidAndInvalidSegments()
+.getValidSegments();
+for (Segment segment : validSegments) {
+  segmentList.add(segment.getSegmentNo());
+}
+return segmentList;
+  }
+
+  /**
+   * This method will delete segment folders of the mv datamap table and 
update the
+   * datamapSegmentStatus map in case of Insert-Overwrite/Update operation on 
main table
+   */
+  public static void cleanMVdatamap(CarbonTable carbonTable) throws 
IOException {
+List allDataMapSchemas =
+
DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable);
+for (DataMapSchema datamapschema : allDataMapSchemas) {
+  if (datamapschema.getProviderName()
+  .equalsIgnoreCase(DataMapClassProvider.MV.getShortName())) {
+CarbonTable datamapTable = CarbonTable
+
.buildFromTablePath(datamapschema.getRelationIdentifier().getTableName(),
+datamapschema.getRelationIdentifier().getDatabaseName(),
+datamapschema.getRelationIdentifier().getTablePath(),
+datamapschema.getRelationIdentifier().getTableId());
+SegmentStatusManager segmentStatusManager =
+new 
SegmentStatusManager(datamapTable.getAbsoluteTableIdentifier());
+ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
+try {
+  if (carbonLock.lockWithRetries()) {
+LOGGER.info(
+"Acquired lock for table" + datamapTable.getDatabaseName() + 
"." + datamapTable
+.getTableName() + " for table status updation");
+LoadMetadataDetails[] loadMetadataDetails =
+
SegmentStatusManager.readLoadMetadata(datamapTable.getMetadataPath());
+List staleFolders = new ArrayList<>();
+for (LoadMetadataDetails entry : loadMetadataDetails) {
+  entry.setSegmentStatus(SegmentStatus.MARKED_FOR_DELETE);
+  // For insert overwrite, we will delete the old segment folder 
immediately
+  // So collect the old segments here
+  String segmentPath = CarbonTablePath
+  
.getSegmentPath(datamapschema.getRelationIdentifier().getTablePath(),
+  entry.getLoadName());
+  if (FileFactory.isFileExist(segmentPath, 
FileFactory.getFileType(segmentPath))) {
+staleFolders.add(FileFactory.getCarbonFile(segmentPath));
 
 Review comment:
   Please don't remove immediately as some queries might be running on it. Let 
it remove during next time load


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-24 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278121252
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
 ##
 @@ -127,4 +150,115 @@ private static DataMapSchema getDataMapSchema(String 
dataMapName)
 return DataMapStoreManager.getInstance().getDataMapSchema(dataMapName);
   }
 
+  /**
+   * Returns valid segment list for a given RelationIdentifier
+   *
+   * @param relationIdentifier
+   * @return
+   * @throws IOException
+   */
+  public static List getSegmentList(RelationIdentifier 
relationIdentifier)
+  throws IOException {
+List segmentList = new ArrayList<>();
+AbsoluteTableIdentifier absoluteTableIdentifier =
+AbsoluteTableIdentifier.from(relationIdentifier.getTablePath());
+List validSegments =
+new 
SegmentStatusManager(absoluteTableIdentifier).getValidAndInvalidSegments()
+.getValidSegments();
+for (Segment segment : validSegments) {
+  segmentList.add(segment.getSegmentNo());
+}
+return segmentList;
+  }
+
+  /**
+   * This method will delete segment folders of the mv datamap table and 
update the
+   * datamapSegmentStatus map in case of Insert-Overwrite/Update operation on 
main table
+   */
+  public static void cleanMVdatamap(CarbonTable carbonTable) throws 
IOException {
+List allDataMapSchemas =
+
DataMapStoreManager.getInstance().getDataMapSchemasOfTable(carbonTable);
+for (DataMapSchema datamapschema : allDataMapSchemas) {
+  if (datamapschema.getProviderName()
+  .equalsIgnoreCase(DataMapClassProvider.MV.getShortName())) {
+CarbonTable datamapTable = CarbonTable
+
.buildFromTablePath(datamapschema.getRelationIdentifier().getTableName(),
 
 Review comment:
   No need to create a table to get the `AbsoluteTableIdentifier`.


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-24 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278120867
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
 ##
 @@ -127,4 +150,115 @@ private static DataMapSchema getDataMapSchema(String 
dataMapName)
 return DataMapStoreManager.getInstance().getDataMapSchema(dataMapName);
   }
 
+  /**
+   * Returns valid segment list for a given RelationIdentifier
+   *
+   * @param relationIdentifier
+   * @return
+   * @throws IOException
+   */
+  public static List getSegmentList(RelationIdentifier 
relationIdentifier)
+  throws IOException {
+List segmentList = new ArrayList<>();
+AbsoluteTableIdentifier absoluteTableIdentifier =
+AbsoluteTableIdentifier.from(relationIdentifier.getTablePath());
+List validSegments =
+new 
SegmentStatusManager(absoluteTableIdentifier).getValidAndInvalidSegments()
+.getValidSegments();
+for (Segment segment : validSegments) {
+  segmentList.add(segment.getSegmentNo());
+}
+return segmentList;
+  }
+
+  /**
+   * This method will delete segment folders of the mv datamap table and 
update the
+   * datamapSegmentStatus map in case of Insert-Overwrite/Update operation on 
main table
+   */
+  public static void cleanMVdatamap(CarbonTable carbonTable) throws 
IOException {
 
 Review comment:
   It is just not MV so please rename as cleanDatamap


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-24 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278118380
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
 ##
 @@ -127,4 +150,115 @@ private static DataMapSchema getDataMapSchema(String 
dataMapName)
 return DataMapStoreManager.getInstance().getDataMapSchema(dataMapName);
   }
 
+  /**
+   * Returns valid segment list for a given RelationIdentifier
+   *
+   * @param relationIdentifier
+   * @return
+   * @throws IOException
+   */
+  public static List getSegmentList(RelationIdentifier 
relationIdentifier)
+  throws IOException {
+List segmentList = new ArrayList<>();
+AbsoluteTableIdentifier absoluteTableIdentifier =
+AbsoluteTableIdentifier.from(relationIdentifier.getTablePath());
+List validSegments =
+new 
SegmentStatusManager(absoluteTableIdentifier).getValidAndInvalidSegments()
+.getValidSegments();
+for (Segment segment : validSegments) {
+  segmentList.add(segment.getSegmentNo());
+}
+return segmentList;
+  }
+
+  /**
+   * This method will delete segment folders of the mv datamap table and 
update the
+   * datamapSegmentStatus map in case of Insert-Overwrite/Update operation on 
main table
 
 Review comment:
   Even support for delete as well


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-24 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278117503
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapStatusManager.java
 ##
 @@ -127,4 +150,115 @@ private static DataMapSchema getDataMapSchema(String 
dataMapName)
 return DataMapStoreManager.getInstance().getDataMapSchema(dataMapName);
   }
 
+  /**
+   * Returns valid segment list for a given RelationIdentifier
+   *
+   * @param relationIdentifier
+   * @return
+   * @throws IOException
+   */
+  public static List getSegmentList(RelationIdentifier 
relationIdentifier)
 
 Review comment:
   It shouldn't exist in this class, please move to some utility


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-24 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278069323
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DiskBasedDataMapStatusProvider.java
 ##
 @@ -196,4 +207,278 @@ private static ICarbonLock getDataMapStatusLock() {
 
.getSystemLevelCarbonLockObj(CarbonProperties.getInstance().getSystemFolderLocation(),
 LockUsage.DATAMAP_STATUS_LOCK);
   }
+
+  /**
+   * Reads and returns dataMapSegmentStatusDetail
+   *
+   * @param dataMapSchema
+   * @throws IOException
+   */
+  public DataMapSegmentStatusDetail getDataMapSegmentStatus(DataMapSchema 
dataMapSchema)
+  throws IOException {
+String statusPath = 
getDatamapSegmentStatusFile(dataMapSchema.getDataMapName());
+Gson gsonObjectToRead = new Gson();
+DataInputStream dataInputStream = null;
+BufferedReader buffReader = null;
+InputStreamReader inStream = null;
+DataMapSegmentStatusDetail dataMapSegmentStatusDetail;
+try {
+  if (!FileFactory.isFileExist(statusPath)) {
+return new DataMapSegmentStatusDetail();
+  }
+  dataInputStream =
+  FileFactory.getDataInputStream(statusPath, 
FileFactory.getFileType(statusPath));
+  inStream = new InputStreamReader(dataInputStream,
+  Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+  buffReader = new BufferedReader(inStream);
+  dataMapSegmentStatusDetail =
+  gsonObjectToRead.fromJson(buffReader, 
DataMapSegmentStatusDetail.class);
+} catch (IOException e) {
+  LOG.error("Failed to read datamap segment status", e);
+  throw e;
+} finally {
+  CarbonUtil.closeStreams(buffReader, inStream, dataInputStream);
+}
+
+if (null == dataMapSegmentStatusDetail) {
+  return new DataMapSegmentStatusDetail();
+}
+return dataMapSegmentStatusDetail;
+  }
+
+  /**
+   * After each data load to mv datamap, update the segment status mapping. 
Get the new load name
+   * from datamap table loadMetaDetails and map newly loaded main table 
segments against the datamap
+   * table new load entry
+   *
+   * @param dataMapSchema
+   * @throws IOException
+   */
+  public void updateSegmentMapping(DataMapSchema dataMapSchema) throws 
IOException {
+DataMapSegmentStatusDetail dataMapSegmentStatus = 
getDataMapSegmentStatus(dataMapSchema);
+List relationIdentifiers = 
dataMapSchema.getParentTables();
+CarbonTable dataMapTable = CarbonTable
+
.buildFromTablePath(dataMapSchema.getRelationIdentifier().getTableName(),
+dataMapSchema.getRelationIdentifier().getDatabaseName(),
+dataMapSchema.getRelationIdentifier().getTablePath(),
+dataMapSchema.getRelationIdentifier().getTableId());
+LoadMetadataDetails[] loadMetadataDetails =
+SegmentStatusManager.readLoadMetadata(dataMapTable.getMetadataPath());
+if (loadMetadataDetails.length != 0) {
+  String newLoadKey;
+  if (!dataMapSegmentStatus.getSegmentMapping().isEmpty()) {
+for (LoadMetadataDetails entry : loadMetadataDetails) {
+  if (entry.getSegmentStatus() == SegmentStatus.MARKED_FOR_DELETE
+  || entry.getSegmentStatus() == SegmentStatus.COMPACTED) {
+//In case of main table or datamap compaction, remove invalid 
entries
+
dataMapSegmentStatus.getSegmentMapping().remove(entry.getLoadName());
+  }
+}
+  } else {
+dataMapSegmentStatus.setDataMapName(dataMapSchema.getDataMapName());
+  }
+  newLoadKey = loadMetadataDetails[loadMetadataDetails.length - 
1].getLoadName();
+  Map> mainTableSegmentMap = new HashMap<>();
+  for (RelationIdentifier relationIdentifier : relationIdentifiers) {
+List validMainTableSegmentList =
+DataMapStatusManager.getSegmentList(relationIdentifier);
+List datamapTableSegmentList =
+getDataMapSegmentsFromMapping(dataMapSegmentStatus, 
relationIdentifier);
+// Compare main table and datamap table valid segment list and collect 
newly loaded segments
+// from main table to datamap table
+validMainTableSegmentList.removeAll(datamapTableSegmentList);
+mainTableSegmentMap.put(relationIdentifier.getTableName(), 
validMainTableSegmentList);
+  }
+  dataMapSegmentStatus.getSegmentMapping().put(newLoadKey, 
mainTableSegmentMap);
+  
dataMapSegmentStatus.setSegmentMapping(dataMapSegmentStatus.getSegmentMapping());
+  writeToSegmentStatusFile(dataMapSegmentStatus, 
dataMapSchema.getDataMapName());
+}
+  }
+
+  /**
+   * write datamap to mainTbale segment mapping details
+   *
+   * @param dataMapSegmentStatus
+   * @param dataMapName
+   * @throws IOException
+   */
+  private void writeToSegmentStatusFile(DataMapSegmentStatusDetail 
dataMapSegmentStatus,
+ 

[GitHub] [carbondata] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-24 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278068979
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DiskBasedDataMapStatusProvider.java
 ##
 @@ -196,4 +207,278 @@ private static ICarbonLock getDataMapStatusLock() {
 
.getSystemLevelCarbonLockObj(CarbonProperties.getInstance().getSystemFolderLocation(),
 LockUsage.DATAMAP_STATUS_LOCK);
   }
+
+  /**
+   * Reads and returns dataMapSegmentStatusDetail
+   *
+   * @param dataMapSchema
+   * @throws IOException
+   */
+  public DataMapSegmentStatusDetail getDataMapSegmentStatus(DataMapSchema 
dataMapSchema)
+  throws IOException {
+String statusPath = 
getDatamapSegmentStatusFile(dataMapSchema.getDataMapName());
+Gson gsonObjectToRead = new Gson();
+DataInputStream dataInputStream = null;
+BufferedReader buffReader = null;
+InputStreamReader inStream = null;
+DataMapSegmentStatusDetail dataMapSegmentStatusDetail;
+try {
+  if (!FileFactory.isFileExist(statusPath)) {
+return new DataMapSegmentStatusDetail();
+  }
+  dataInputStream =
+  FileFactory.getDataInputStream(statusPath, 
FileFactory.getFileType(statusPath));
+  inStream = new InputStreamReader(dataInputStream,
+  Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+  buffReader = new BufferedReader(inStream);
+  dataMapSegmentStatusDetail =
+  gsonObjectToRead.fromJson(buffReader, 
DataMapSegmentStatusDetail.class);
+} catch (IOException e) {
+  LOG.error("Failed to read datamap segment status", e);
+  throw e;
+} finally {
+  CarbonUtil.closeStreams(buffReader, inStream, dataInputStream);
+}
+
+if (null == dataMapSegmentStatusDetail) {
+  return new DataMapSegmentStatusDetail();
+}
+return dataMapSegmentStatusDetail;
+  }
+
+  /**
+   * After each data load to mv datamap, update the segment status mapping. 
Get the new load name
+   * from datamap table loadMetaDetails and map newly loaded main table 
segments against the datamap
+   * table new load entry
+   *
+   * @param dataMapSchema
+   * @throws IOException
+   */
+  public void updateSegmentMapping(DataMapSchema dataMapSchema) throws 
IOException {
 
 Review comment:
   Segment mapping should be passed explicitly after the finish of datamap 
load. Don't read the main table status file to get 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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-24 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278068300
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DiskBasedDataMapStatusProvider.java
 ##
 @@ -196,4 +207,278 @@ private static ICarbonLock getDataMapStatusLock() {
 
.getSystemLevelCarbonLockObj(CarbonProperties.getInstance().getSystemFolderLocation(),
 LockUsage.DATAMAP_STATUS_LOCK);
   }
+
+  /**
+   * Reads and returns dataMapSegmentStatusDetail
+   *
+   * @param dataMapSchema
+   * @throws IOException
+   */
+  public DataMapSegmentStatusDetail getDataMapSegmentStatus(DataMapSchema 
dataMapSchema)
+  throws IOException {
+String statusPath = 
getDatamapSegmentStatusFile(dataMapSchema.getDataMapName());
+Gson gsonObjectToRead = new Gson();
+DataInputStream dataInputStream = null;
+BufferedReader buffReader = null;
+InputStreamReader inStream = null;
+DataMapSegmentStatusDetail dataMapSegmentStatusDetail;
+try {
+  if (!FileFactory.isFileExist(statusPath)) {
+return new DataMapSegmentStatusDetail();
+  }
+  dataInputStream =
+  FileFactory.getDataInputStream(statusPath, 
FileFactory.getFileType(statusPath));
+  inStream = new InputStreamReader(dataInputStream,
+  Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+  buffReader = new BufferedReader(inStream);
+  dataMapSegmentStatusDetail =
+  gsonObjectToRead.fromJson(buffReader, 
DataMapSegmentStatusDetail.class);
+} catch (IOException e) {
+  LOG.error("Failed to read datamap segment status", e);
+  throw e;
+} finally {
+  CarbonUtil.closeStreams(buffReader, inStream, dataInputStream);
+}
+
+if (null == dataMapSegmentStatusDetail) {
+  return new DataMapSegmentStatusDetail();
+}
+return dataMapSegmentStatusDetail;
+  }
+
+  /**
+   * After each data load to mv datamap, update the segment status mapping. 
Get the new load name
+   * from datamap table loadMetaDetails and map newly loaded main table 
segments against the datamap
+   * table new load entry
+   *
+   * @param dataMapSchema
+   * @throws IOException
+   */
+  public void updateSegmentMapping(DataMapSchema dataMapSchema) throws 
IOException {
 
 Review comment:
   Update segment mapping should not be this class API


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-24 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278066086
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DiskBasedDataMapStatusProvider.java
 ##
 @@ -196,4 +207,278 @@ private static ICarbonLock getDataMapStatusLock() {
 
.getSystemLevelCarbonLockObj(CarbonProperties.getInstance().getSystemFolderLocation(),
 LockUsage.DATAMAP_STATUS_LOCK);
   }
+
+  /**
+   * Reads and returns dataMapSegmentStatusDetail
+   *
+   * @param dataMapSchema
+   * @throws IOException
+   */
+  public DataMapSegmentStatusDetail getDataMapSegmentStatus(DataMapSchema 
dataMapSchema)
+  throws IOException {
+String statusPath = 
getDatamapSegmentStatusFile(dataMapSchema.getDataMapName());
+Gson gsonObjectToRead = new Gson();
+DataInputStream dataInputStream = null;
+BufferedReader buffReader = null;
+InputStreamReader inStream = null;
+DataMapSegmentStatusDetail dataMapSegmentStatusDetail;
+try {
+  if (!FileFactory.isFileExist(statusPath)) {
+return new DataMapSegmentStatusDetail();
+  }
+  dataInputStream =
+  FileFactory.getDataInputStream(statusPath, 
FileFactory.getFileType(statusPath));
+  inStream = new InputStreamReader(dataInputStream,
+  Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+  buffReader = new BufferedReader(inStream);
+  dataMapSegmentStatusDetail =
+  gsonObjectToRead.fromJson(buffReader, 
DataMapSegmentStatusDetail.class);
+} catch (IOException e) {
+  LOG.error("Failed to read datamap segment status", e);
+  throw e;
+} finally {
+  CarbonUtil.closeStreams(buffReader, inStream, dataInputStream);
+}
+
+if (null == dataMapSegmentStatusDetail) {
+  return new DataMapSegmentStatusDetail();
+}
+return dataMapSegmentStatusDetail;
+  }
+
+  /**
+   * After each data load to mv datamap, update the segment status mapping. 
Get the new load name
+   * from datamap table loadMetaDetails and map newly loaded main table 
segments against the datamap
+   * table new load entry
+   *
+   * @param dataMapSchema
+   * @throws IOException
+   */
+  public void updateSegmentMapping(DataMapSchema dataMapSchema) throws 
IOException {
+DataMapSegmentStatusDetail dataMapSegmentStatus = 
getDataMapSegmentStatus(dataMapSchema);
+List relationIdentifiers = 
dataMapSchema.getParentTables();
+CarbonTable dataMapTable = CarbonTable
+
.buildFromTablePath(dataMapSchema.getRelationIdentifier().getTableName(),
+dataMapSchema.getRelationIdentifier().getDatabaseName(),
+dataMapSchema.getRelationIdentifier().getTablePath(),
+dataMapSchema.getRelationIdentifier().getTableId());
+LoadMetadataDetails[] loadMetadataDetails =
+SegmentStatusManager.readLoadMetadata(dataMapTable.getMetadataPath());
+if (loadMetadataDetails.length != 0) {
+  String newLoadKey;
+  if (!dataMapSegmentStatus.getSegmentMapping().isEmpty()) {
+for (LoadMetadataDetails entry : loadMetadataDetails) {
+  if (entry.getSegmentStatus() == SegmentStatus.MARKED_FOR_DELETE
+  || entry.getSegmentStatus() == SegmentStatus.COMPACTED) {
+//In case of main table or datamap compaction, remove invalid 
entries
+
dataMapSegmentStatus.getSegmentMapping().remove(entry.getLoadName());
+  }
+}
+  } else {
+dataMapSegmentStatus.setDataMapName(dataMapSchema.getDataMapName());
+  }
+  newLoadKey = loadMetadataDetails[loadMetadataDetails.length - 
1].getLoadName();
+  Map> mainTableSegmentMap = new HashMap<>();
+  for (RelationIdentifier relationIdentifier : relationIdentifiers) {
+List validMainTableSegmentList =
+DataMapStatusManager.getSegmentList(relationIdentifier);
+List datamapTableSegmentList =
+getDataMapSegmentsFromMapping(dataMapSegmentStatus, 
relationIdentifier);
+// Compare main table and datamap table valid segment list and collect 
newly loaded segments
+// from main table to datamap table
+validMainTableSegmentList.removeAll(datamapTableSegmentList);
+mainTableSegmentMap.put(relationIdentifier.getTableName(), 
validMainTableSegmentList);
+  }
+  dataMapSegmentStatus.getSegmentMapping().put(newLoadKey, 
mainTableSegmentMap);
+  
dataMapSegmentStatus.setSegmentMapping(dataMapSegmentStatus.getSegmentMapping());
+  writeToSegmentStatusFile(dataMapSegmentStatus, 
dataMapSchema.getDataMapName());
+}
+  }
+
+  /**
+   * write datamap to mainTbale segment mapping details
+   *
+   * @param dataMapSegmentStatus
+   * @param dataMapName
+   * @throws IOException
+   */
+  private void writeToSegmentStatusFile(DataMapSegmentStatusDetail 
dataMapSegmentStatus,
+ 

[GitHub] [carbondata] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-24 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278065357
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapSegmentStatusDetail.java
 ##
 @@ -0,0 +1,64 @@
+/*
+ * 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.datamap.status;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+
+/**
+ * This class holds information about datamap segments to main table segments 
mapping
+ */
+@InterfaceAudience.Internal public class DataMapSegmentStatusDetail implements 
Serializable {
+
+  private String dataMapName;
+
+  private Map>> segmentMapping = new 
HashMap<>();
 
 Review comment:
   Create one object for every load. 


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-24 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278065438
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapSegmentStatusDetail.java
 ##
 @@ -0,0 +1,64 @@
+/*
+ * 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.datamap.status;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+
+/**
+ * This class holds information about datamap segments to main table segments 
mapping
+ */
+@InterfaceAudience.Internal public class DataMapSegmentStatusDetail implements 
Serializable {
+
+  private String dataMapName;
+
+  private Map>> segmentMapping = new 
HashMap<>();
+
+  public DataMapSegmentStatusDetail(String dataMapName,
 
 Review comment:
   Remove if not used


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-24 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r278063741
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/status/DataMapSegmentStatusDetail.java
 ##
 @@ -0,0 +1,64 @@
+/*
+ * 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.datamap.status;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+
+/**
+ * This class holds information about datamap segments to main table segments 
mapping
+ */
+@InterfaceAudience.Internal public class DataMapSegmentStatusDetail implements 
Serializable {
+
+  private String dataMapName;
 
 Review comment:
   Not required this name inside


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] ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] Support Incremental DataLoad for MV Datamap[with single parent table]

2019-04-23 Thread GitBox
ravipesala commented on a change in pull request #3179: [CARBONDATA-3338] 
Support Incremental DataLoad for MV Datamap[with single parent table]
URL: https://github.com/apache/carbondata/pull/3179#discussion_r277959926
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/datamap/DataMapUtil.java
 ##
 @@ -159,4 +159,12 @@ private static DistributableDataMapFormat 
createDataMapJob(CarbonTable carbonTab
 return ssm.getValidAndInvalidSegments();
   }
 
+  /**
+   * Check if carbonTable is mv datamap table
+   */
+  public static Boolean isMVdatamapTable(CarbonTable carbonTable) {
+String isMVdatamapTable =
+
carbonTable.getTableInfo().getFactTable().getTableProperties().get("isMVdatamapTable");
 
 Review comment:
   please move this `isMVdatamapTable` to constant


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