[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-03-09 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590246840



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -1133,17 +1133,23 @@ public static void deleteSegment(String tablePath, 
Segment segment,
 List indexOrMergeFiles = 
fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
 FileFactory.getConfiguration());
 Map> indexFilesMap = fileStore.getIndexFilesMap();
+List deletedFiles = new ArrayList<>();

Review comment:
   We need  to print every file when it is deleted so as to check any 
unnecessary deletion does not happen, the whole purpose is defeated if we 
delete a file and do not log it's name





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-03-09 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590203265



##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
##
@@ -37,11 +38,24 @@ case class CarbonCleanFilesCommand(
 databaseNameOp: Option[String],
 tableName: String,
 options: Map[String, String] = Map.empty,
+dryRun: Boolean,
 isInternalCleanCall: Boolean = false)
   extends DataCommand {
 
   val LOGGER: Logger = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
 
+  override def output: Seq[AttributeReference] = {
+if (dryRun) {
+  Seq(
+AttributeReference("Size Freed", LongType, nullable = false)(),
+AttributeReference("Trash Data Remaining", LongType, nullable = 
false)())
+} else {
+  Seq(
+AttributeReference("Size Freed", LongType, nullable = false)(),
+AttributeReference("Trash Data Remaining", LongType, nullable = 
false)())
+}

Review comment:
   Changed the logic, now both the blocks are different





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-03-09 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590192598



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##
@@ -112,13 +141,91 @@ object DataTrashManager {
   carbonTable: CarbonTable,
   isForceDelete: Boolean,
   cleanStaleInProgress: Boolean,
-  partitionSpecsOption: Option[Seq[PartitionSpec]]): Unit = {
+  partitionSpecsOption: Option[Seq[PartitionSpec]]): Seq[Long] = {
 val partitionSpecs = partitionSpecsOption.map(_.asJava).orNull
-SegmentStatusManager.deleteLoadsAndUpdateMetadata(carbonTable,
+val sizeStatistics = 
SegmentStatusManager.deleteLoadsAndUpdateMetadata(carbonTable,
   isForceDelete, partitionSpecs, cleanStaleInProgress, true)
 if (carbonTable.isHivePartitionTable && partitionSpecsOption.isDefined) {
   SegmentFileStore.cleanSegments(carbonTable, partitionSpecs, 
isForceDelete)
 }
+sizeStatistics
+  }
+
+  private def dryRunOnExpiredSegments(
+  carbonTable: CarbonTable,
+  isForceDelete: Boolean,
+  cleanStaleInProgress: Boolean,
+  partitionSpecsOption: Option[Seq[PartitionSpec]]): Seq[Long] = {
+var sizeFreed: Long = 0
+var trashSizeRemaining: Long = 0
+val loadMetadataDetails = 
SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+if (SegmentStatusManager.isLoadDeletionRequired(loadMetadataDetails)) {
+  loadMetadataDetails.foreach { oneLoad =>
+val segmentFilePath = 
CarbonTablePath.getSegmentFilePath(carbonTable.getTablePath,
+  oneLoad.getSegmentFile)
+if (DeleteLoadFolders.canDeleteThisLoad(oneLoad, isForceDelete, 
cleanStaleInProgress)) {
+  // No need to consider physical data for external segments, only 
consider metadata.
+  if (oneLoad.getPath() == null || 
oneLoad.getPath().equalsIgnoreCase("NA")) {
+if (!carbonTable.isHivePartitionTable) {
+  sizeFreed += 
FileFactory.getDirectorySize(CarbonTablePath.getSegmentPath(carbonTable
+  .getTablePath, oneLoad.getLoadName))
+} else {
+  sizeFreed += partitionTableSegmentSize(carbonTable, oneLoad, 
loadMetadataDetails,
+partitionSpecsOption)
+}
+  }
+  sizeFreed += FileFactory.getCarbonFile(segmentFilePath).getSize
+} else {
+  if (SegmentStatusManager.isExpiredSegment(oneLoad, carbonTable
+  .getAbsoluteTableIdentifier)) {
+if (!carbonTable.isHivePartitionTable) {
+  trashSizeRemaining += 
FileFactory.getDirectorySize(CarbonTablePath.getSegmentPath(

Review comment:
   changed logic





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-03-09 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590192283



##
File path: 
core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##
@@ -1297,4 +1359,37 @@ public static TableStatusReturnTuple 
separateVisibleAndInvisibleSegments(
   return new HashMap<>(0);
 }
   }
+
+  public static long partitionTableSegmentSize(CarbonTable carbonTable, 
LoadMetadataDetails

Review comment:
   done, changed this method completely, now reading the data using 
tablestatus by reading the data and the index size





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-03-09 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590191260



##
File path: 
core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##
@@ -1072,7 +1097,22 @@ public static void 
deleteLoadsAndUpdateMetadata(CarbonTable carbonTable, boolean
 isUpdateRequired(isForceDeletion, carbonTable,
 identifier, details, cleanStaleInprogress);
 if (!tuple2.isUpdateRequired) {
-  return;
+  try {
+for (LoadMetadataDetails oneLoad : details) {
+  if (isExpiredSegment(oneLoad, 
carbonTable.getAbsoluteTableIdentifier())) {
+if (!carbonTable.isHivePartitionTable()) {
+  trashSizeRemaining += 
FileFactory.getDirectorySize(CarbonTablePath
+.getSegmentPath(carbonTable.getTablePath(), 
oneLoad.getLoadName()));
+} else {
+  trashSizeRemaining += 
partitionTableSegmentSize(carbonTable, oneLoad,
+details, partitionSpecs);
+}
+  }
+}
+  } catch (Exception e) {
+LOG.error("Unable to calculate size of garbage data", e);
+  }
+  return new long[]{sizeFreed, trashSizeRemaining};

Review comment:
   changed code, dry run is separate from the current clean files code and 
not mixed with the existing code





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-03-09 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590189418



##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
##
@@ -513,12 +513,13 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
 
   protected lazy val cleanFiles: Parser[LogicalPlan] =
 CLEAN ~> FILES ~> FOR ~> TABLE ~> (ident <~ ".").? ~ ident ~
-  (OPTIONS ~> "(" ~> repsep(options, ",") <~ ")").? <~ opt(";") ^^ {
-  case databaseName ~ tableName ~ optionList =>
+  (OPTIONS ~> "(" ~> repsep(options, ",") <~ ")").? ~ opt(DRYRUN) <~ 
opt(";") ^^ {

Review comment:
   changed, added it in the option





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-03-09 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590187262



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##
@@ -87,13 +101,28 @@ object DataTrashManager {
 }
   }
 
-  private def checkAndCleanTrashFolder(carbonTable: CarbonTable, 
isForceDelete: Boolean): Unit = {
+  def cleanFilesDryRunOperation (
+  carbonTable: CarbonTable,
+  isForceDelete: Boolean,
+  cleanStaleInProgress: Boolean,
+  partitionSpecs: Option[Seq[PartitionSpec]] = None): Seq[Long] = {
+// get size freed from the trash folder
+val trashFolderSizeStats = checkAndCleanTrashFolder(carbonTable, 
isForceDelete, isDryRun = true)
+// get size that will be deleted (MFD, COmpacted, Inprogress segments)
+val expiredSegmentsSizeStats = dryRunOnExpiredSegments(carbonTable, 
isForceDelete,
+  cleanStaleInProgress, partitionSpecs)
+Seq(trashFolderSizeStats.head + expiredSegmentsSizeStats.head, 
trashFolderSizeStats(1) +
+expiredSegmentsSizeStats(1))
+  }
+
+  private def checkAndCleanTrashFolder(carbonTable: CarbonTable, 
isForceDelete: Boolean,
+  isDryRun: Boolean): Seq[Long] = {

Review comment:
   dry run option can be ran along force option, in the force option we 
will have to delete all the data of the trash folder immediately. So, it will 
just return the complete trash size in case of dryrun = true and forceDelete = 
true, without deleting anything





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-03-09 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590186070



##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFilesCommandPartitionTable.scala
##
@@ -186,7 +193,9 @@ class TestCleanFilesCommandPartitionTable extends QueryTest 
with BeforeAndAfterA
 
removeSegmentEntryFromTableStatusFile(CarbonEnv.getCarbonTable(Some("default"), 
"cleantest")(
   sqlContext.sparkSession), "2")
 
-sql(s"CLEAN FILES FOR TABLE CLEANTEST").show()
+val df1 = sql(s"CLEAN FILES FOR TABLE CLEANTEST DRYRUN")

Review comment:
   added code to check the segment size in one of the test case and 
checking it with clean files result and the dry run result. The test case will 
only pass when all three result are the same





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-03-09 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590185386



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##
@@ -112,13 +141,91 @@ object DataTrashManager {
   carbonTable: CarbonTable,
   isForceDelete: Boolean,
   cleanStaleInProgress: Boolean,
-  partitionSpecsOption: Option[Seq[PartitionSpec]]): Unit = {
+  partitionSpecsOption: Option[Seq[PartitionSpec]]): Seq[Long] = {
 val partitionSpecs = partitionSpecsOption.map(_.asJava).orNull
-SegmentStatusManager.deleteLoadsAndUpdateMetadata(carbonTable,
+val sizeStatistics = 
SegmentStatusManager.deleteLoadsAndUpdateMetadata(carbonTable,
   isForceDelete, partitionSpecs, cleanStaleInProgress, true)
 if (carbonTable.isHivePartitionTable && partitionSpecsOption.isDefined) {
   SegmentFileStore.cleanSegments(carbonTable, partitionSpecs, 
isForceDelete)
 }
+sizeStatistics
+  }
+
+  private def dryRunOnExpiredSegments(
+  carbonTable: CarbonTable,
+  isForceDelete: Boolean,
+  cleanStaleInProgress: Boolean,
+  partitionSpecsOption: Option[Seq[PartitionSpec]]): Seq[Long] = {
+var sizeFreed: Long = 0
+var trashSizeRemaining: Long = 0
+val loadMetadataDetails = 
SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+if (SegmentStatusManager.isLoadDeletionRequired(loadMetadataDetails)) {
+  loadMetadataDetails.foreach { oneLoad =>
+val segmentFilePath = 
CarbonTablePath.getSegmentFilePath(carbonTable.getTablePath,
+  oneLoad.getSegmentFile)
+if (DeleteLoadFolders.canDeleteThisLoad(oneLoad, isForceDelete, 
cleanStaleInProgress)) {
+  // No need to consider physical data for external segments, only 
consider metadata.
+  if (oneLoad.getPath() == null || 
oneLoad.getPath().equalsIgnoreCase("NA")) {
+if (!carbonTable.isHivePartitionTable) {
+  sizeFreed += 
FileFactory.getDirectorySize(CarbonTablePath.getSegmentPath(carbonTable
+  .getTablePath, oneLoad.getLoadName))
+} else {
+  sizeFreed += partitionTableSegmentSize(carbonTable, oneLoad, 
loadMetadataDetails,
+partitionSpecsOption)
+}
+  }
+  sizeFreed += FileFactory.getCarbonFile(segmentFilePath).getSize
+} else {
+  if (SegmentStatusManager.isExpiredSegment(oneLoad, carbonTable
+  .getAbsoluteTableIdentifier)) {
+if (!carbonTable.isHivePartitionTable) {
+  trashSizeRemaining += 
FileFactory.getDirectorySize(CarbonTablePath.getSegmentPath(
+  carbonTable.getTablePath, oneLoad.getLoadName))
+} else {
+  trashSizeRemaining += partitionTableSegmentSize(carbonTable, 
oneLoad,
+  loadMetadataDetails, partitionSpecsOption)
+}
+trashSizeRemaining += 
FileFactory.getCarbonFile(segmentFilePath).getSize
+  }
+}
+  }
+}
+Seq(sizeFreed, trashSizeRemaining)
+  }
+
+  def partitionTableSegmentSize( carbonTable: CarbonTable, oneLoad: 
LoadMetadataDetails,
+  loadMetadataDetails: Array[LoadMetadataDetails], partitionSpecsOption:
+  Option[Seq[PartitionSpec]]) : Long = {
+var segmentSize: Long = 0

Review comment:
   changed





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-03-09 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590185083



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/events/CleanFilesEvents.scala
##
@@ -26,5 +26,6 @@ case class CleanFilesPreEvent(carbonTable: CarbonTable, 
sparkSession: SparkSessi
 case class CleanFilesPostEvent(
 carbonTable: CarbonTable,
 sparkSession: SparkSession,
-options: Map[String, String])
+options: Map[String, String],
+dryRun: Boolean)

Review comment:
   changed





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-03-09 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590184901



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -1112,11 +1112,15 @@ public static void cleanSegments(CarbonTable table,
 cleanSegments(table, details, partitionSpecs, forceDelete);
   }
 
-  public static void deleteSegmentFile(String tablePath, Segment segment) 
throws Exception {
+  public static long deleteSegmentFile(String tablePath, Segment segment) 
throws Exception {
 String segmentFilePath =
 CarbonTablePath.getSegmentFilePath(tablePath, 
segment.getSegmentFileName());
 // Deletes the physical segment file
-FileFactory.deleteFile(segmentFilePath);
+CarbonFile carbonSegmentFile = FileFactory.getCarbonFile(segmentFilePath);
+long sizeFreed  = carbonSegmentFile.getSize();
+FileFactory.deleteFile(carbonSegmentFile);

Review comment:
   changed





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-03-09 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590184637



##
File path: 
core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##
@@ -1297,4 +1356,37 @@ public static TableStatusReturnTuple 
separateVisibleAndInvisibleSegments(
   return new HashMap<>(0);
 }
   }
+
+  public static long partitionTableSegmentSize(CarbonTable carbonTable, 
LoadMetadataDetails
+  oneLoad, LoadMetadataDetails[] loadMetadataDetails, List
+  partitionSpecs) throws Exception {
+long size = 0;
+SegmentFileStore fileStore = new 
SegmentFileStore(carbonTable.getTablePath(), oneLoad
+.getSegmentFile());
+List indexOrMergeFiles = 
fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
+FileFactory.getConfiguration());
+Map> indexFilesMap = fileStore.getIndexFilesMap();

Review comment:
   instead of this, using the metadata to read the data and the index size, 
it is a very fast operation and will not require any file reading other than 
the tablestatus file.

##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFilesCommandPartitionTable.scala
##
@@ -65,14 +65,19 @@ class TestCleanFilesCommandPartitionTable extends QueryTest 
with BeforeAndAfterA
 loadData()
 sql(s"""ALTER TABLE CLEANTEST COMPACT "MINOR" """)
 loadData()
+sql(s"CLEAN FILES FOR TABLE cleantest DRYRUN").show()
+sql(s"CLEAN FILES FOR TABLE cleantest").show()

Review comment:
   changed





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-03-09 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590183560



##
File path: 
core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##
@@ -1125,13 +1165,32 @@ public static void 
deleteLoadsAndUpdateMetadata(CarbonTable carbonTable, boolean
 CarbonLockUtil.fileUnlock(carbonTableStatusLock, 
LockUsage.TABLE_STATUS_LOCK);
   }
   if (updateCompletionStatus) {
-DeleteLoadFolders
+long[] cleanFileSizeFreed = DeleteLoadFolders
 .physicalFactAndMeasureMetadataDeletion(carbonTable, 
newAddedLoadHistoryList,
   isForceDeletion, partitionSpecs, cleanStaleInprogress);
+sizeFreed += cleanFileSizeFreed[0];
+trashSizeRemaining += cleanFileSizeFreed[1];
+  }
+}
+  } else {
+try {
+  for (LoadMetadataDetails oneLoad : metadataDetails) {
+if (isExpiredSegment(oneLoad, 
carbonTable.getAbsoluteTableIdentifier())) {
+  if (!carbonTable.isHivePartitionTable()) {
+trashSizeRemaining += 
FileFactory.getDirectorySize(CarbonTablePath.getSegmentPath(
+  carbonTable.getTablePath(), oneLoad.getLoadName()));
+  } else {
+trashSizeRemaining += partitionTableSegmentSize(carbonTable, 
oneLoad,
+  metadataDetails, partitionSpecs);
+  }
+}
   }
+} catch (Exception e) {
+  LOG.error("Unable to calculate size of garbage data", e);
 }
   }
 }
+return new long[]{sizeFreed, trashSizeRemaining};

Review comment:
   changed logic, removed dry run flow with current clean files flow and 
made a different flow to handle that part





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-03-09 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590182511



##
File path: 
core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
##
@@ -297,6 +297,10 @@ public static boolean deleteFile(String filePath) throws 
IOException {
 return getCarbonFile(filePath).deleteFile();
   }
 
+  public static boolean deleteFile(CarbonFile carbonFile) throws IOException {

Review comment:
   removed





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-03-09 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590142619



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##
@@ -87,13 +106,53 @@ object DataTrashManager {
 }
   }
 
-  private def checkAndCleanTrashFolder(carbonTable: CarbonTable, 
isForceDelete: Boolean): Unit = {
+  /**
+   * Checks the size of the segment files as well as datafiles, this method is 
used before and after
+   * clean files operation to check how much space is actually freed, during 
the operation.
+   */
+  def getSizeSnapshot(carbonTable: CarbonTable): Long = {
+val metadataDetails = 
SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+var size: Long = 0
+val segmentFileLocation = 
CarbonTablePath.getSegmentFilesLocation(carbonTable.getTablePath)
+if (FileFactory.isFileExist(segmentFileLocation)) {
+  size += FileFactory.getDirectorySize(segmentFileLocation)
+}
+metadataDetails.foreach(oneLoad =>
+  if (oneLoad.getVisibility.toBoolean) {
+size += calculateSegmentSizeForOneLoad(carbonTable, oneLoad, 
metadataDetails)
+  }
+)
+size
+  }
+
+  /**
+   * Method to handle the Clean files dry run operation
+   */
+  def cleanFilesDryRunOperation (
+  carbonTable: CarbonTable,
+  isForceDelete: Boolean,
+  cleanStaleInProgress: Boolean,
+  showStats: Boolean): (Long, Long) = {
+// get size freed from the trash folder
+val trashFolderSizeStats = checkAndCleanTrashFolder(carbonTable, 
isForceDelete,
+isDryRun = true, showStats)
+// get size that will be deleted (MFD, COmpacted, Inprogress segments)
+val expiredSegmentsSizeStats = dryRunOnExpiredSegments(carbonTable, 
isForceDelete,
+  cleanStaleInProgress)
+(trashFolderSizeStats._1 + expiredSegmentsSizeStats._1, 
trashFolderSizeStats._2 +
+expiredSegmentsSizeStats._2)
+  }
+
+  private def checkAndCleanTrashFolder(carbonTable: CarbonTable, 
isForceDelete: Boolean,
+  isDryRun: Boolean, showStats: Boolean): (Long, Long) = {
 if (isForceDelete) {
   // empty the trash folder
-  TrashUtil.emptyTrash(carbonTable.getTablePath)
+  val a = TrashUtil.emptyTrash(carbonTable.getTablePath, isDryRun, 
showStats)

Review comment:
   done

##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##
@@ -87,13 +106,53 @@ object DataTrashManager {
 }
   }
 
-  private def checkAndCleanTrashFolder(carbonTable: CarbonTable, 
isForceDelete: Boolean): Unit = {
+  /**
+   * Checks the size of the segment files as well as datafiles, this method is 
used before and after
+   * clean files operation to check how much space is actually freed, during 
the operation.
+   */
+  def getSizeSnapshot(carbonTable: CarbonTable): Long = {
+val metadataDetails = 
SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+var size: Long = 0
+val segmentFileLocation = 
CarbonTablePath.getSegmentFilesLocation(carbonTable.getTablePath)
+if (FileFactory.isFileExist(segmentFileLocation)) {
+  size += FileFactory.getDirectorySize(segmentFileLocation)
+}
+metadataDetails.foreach(oneLoad =>
+  if (oneLoad.getVisibility.toBoolean) {
+size += calculateSegmentSizeForOneLoad(carbonTable, oneLoad, 
metadataDetails)
+  }
+)
+size
+  }
+
+  /**
+   * Method to handle the Clean files dry run operation
+   */
+  def cleanFilesDryRunOperation (
+  carbonTable: CarbonTable,
+  isForceDelete: Boolean,
+  cleanStaleInProgress: Boolean,
+  showStats: Boolean): (Long, Long) = {
+// get size freed from the trash folder
+val trashFolderSizeStats = checkAndCleanTrashFolder(carbonTable, 
isForceDelete,
+isDryRun = true, showStats)
+// get size that will be deleted (MFD, COmpacted, Inprogress segments)
+val expiredSegmentsSizeStats = dryRunOnExpiredSegments(carbonTable, 
isForceDelete,
+  cleanStaleInProgress)
+(trashFolderSizeStats._1 + expiredSegmentsSizeStats._1, 
trashFolderSizeStats._2 +
+expiredSegmentsSizeStats._2)
+  }
+
+  private def checkAndCleanTrashFolder(carbonTable: CarbonTable, 
isForceDelete: Boolean,
+  isDryRun: Boolean, showStats: Boolean): (Long, Long) = {
 if (isForceDelete) {
   // empty the trash folder
-  TrashUtil.emptyTrash(carbonTable.getTablePath)
+  val a = TrashUtil.emptyTrash(carbonTable.getTablePath, isDryRun, 
showStats)
+  (a.head, a(1))
 } else {
   // clear trash based on timestamp
-  TrashUtil.deleteExpiredDataFromTrash(carbonTable.getTablePath)
+  val a = TrashUtil.deleteExpiredDataFromTrash(carbonTable.getTablePath, 
isDryRun, showStats)

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the 

[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-03-08 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r589996949



##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
##
@@ -41,6 +43,26 @@ case class CarbonCleanFilesCommand(
   extends DataCommand {
 
   val LOGGER: Logger = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+  val isDryRun: Boolean = options.getOrElse("dryrun", "false").toBoolean
+  var showStats: Boolean = options.getOrElse("statistics", "true").toBoolean
+  if (isInternalCleanCall) {
+showStats = false
+  }
+
+  override def output: Seq[AttributeReference] = {
+if (isDryRun) {
+  // dry run operation
+  Seq(
+AttributeReference("Size Freed", StringType, nullable = false)(),

Review comment:
   Okay i will change to "Size to be freed". Trash data remaining is both 
the trash data inside and outside the trash folder





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-22 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r580105024



##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
##
@@ -41,6 +43,26 @@ case class CarbonCleanFilesCommand(
   extends DataCommand {
 
   val LOGGER: Logger = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+  val isDryRun: Boolean = options.getOrElse("dryrun", "false").toBoolean
+  var showStats: Boolean = options.getOrElse("statistics", "true").toBoolean
+  if (isInternalCleanCall) {
+showStats = false
+  }
+
+  override def output: Seq[AttributeReference] = {
+if (isDryRun) {
+  // dry run operation
+  Seq(
+AttributeReference("Size Freed", StringType, nullable = false)(),

Review comment:
   @ajantha-bhat 
   Clean files with stats: 
   +--+
   |Size Freed|
   +--+
   |  7 KB |
   +--+
   
   Dry Run:
   
   +--++
   |Size Freed|Trash Data Remaining|
   +--++
   |  7 KB|  0 Byte|
   +--+-+





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-22 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r580105024



##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
##
@@ -41,6 +43,26 @@ case class CarbonCleanFilesCommand(
   extends DataCommand {
 
   val LOGGER: Logger = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+  val isDryRun: Boolean = options.getOrElse("dryrun", "false").toBoolean
+  var showStats: Boolean = options.getOrElse("statistics", "true").toBoolean
+  if (isInternalCleanCall) {
+showStats = false
+  }
+
+  override def output: Seq[AttributeReference] = {
+if (isDryRun) {
+  // dry run operation
+  Seq(
+AttributeReference("Size Freed", StringType, nullable = false)(),

Review comment:
   @ajantha-bhat 
   Clean files with stats: 
   +--+
   |Size Freed|
   +--+
   |  7 KB|
   +--+
   
   Dry Run:
   
   +--++
   |Size Freed|Trash Data Remaining|
   +--++
   |  7 KB|  0 Byte|
   +--++





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-22 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r580051926



##
File path: 
integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFileCommand.scala
##
@@ -466,6 +485,39 @@ class TestCleanFileCommand extends QueryTest with 
BeforeAndAfterAll {
 CarbonCommonConstants.CARBON_CLEAN_FILES_FORCE_ALLOWED_DEFAULT)
 }
 
+  test("Test clean files after delete command") {
+CarbonProperties.getInstance()
+  .addProperty(CarbonCommonConstants.CARBON_CLEAN_FILES_FORCE_ALLOWED, 
"true")
+sql("drop table if exists cleantest")
+sql(
+  """
+| CREATE TABLE cleantest (empname String, designation String, doj 
Timestamp,
+|  workgroupcategory int, workgroupcategoryname String, deptno int, 
deptname String,
+|  projectcode int, projectjoindate Timestamp, projectenddate 
Date,attendance int,
+|  utilization int,salary int, empno int)
+| STORED AS carbondata
+  """.stripMargin)
+sql(
+  s"""LOAD DATA local inpath '$resourcesPath/data.csv' INTO TABLE 
cleantest OPTIONS
+ |('DELIMITER'= ',', 'QUOTECHAR'= '"')""".stripMargin)
+val table = CarbonEnv.getCarbonTable(None, "cleantest") 
(sqlContext.sparkSession)
+sql("delete from cleantest where deptno='10'")
+sql(s"""Delete from table cleantest where segment.id in(0)""")
+
+var dryRun = sql(s"CLEAN FILES FOR TABLE cleantest 
OPTIONS('dryrun'='true')").collect()
+var cleanFiles = sql(s"CLEAN FILES FOR TABLE cleantest").collect()
+assert(cleanFiles(0).get(0) == dryRun(0).get(0))
+dryRun = sql(s"CLEAN FILES FOR TABLE cleantest 
OPTIONS('dryrun'='true','force'='true')")
+  .collect()
+cleanFiles = sql(s"CLEAN FILES FOR TABLE cleantest 
OPTIONS('force'='true')").collect()
+assert(cleanFiles(0).get(0) == dryRun(0).get(0))

Review comment:
   done added





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-21 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r580044526



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##
@@ -72,11 +74,26 @@ object DataTrashManager {
   carbonDeleteSegmentLock = CarbonLockUtil.getLockObject(carbonTable
 .getAbsoluteTableIdentifier, LockUsage.DELETE_SEGMENT_LOCK, 
deleteSegmentErrorMsg)
   // step 1: check and clean trash folder
-  checkAndCleanTrashFolder(carbonTable, isForceDelete)
+  // trashFolderSizeStats(0) contains the size that is freed/or can be 
freed and
+  // trashFolderSizeStats(1) contains the size of remaining data in the 
trash folder
+  val trashFolderSizeStats = checkAndCleanTrashFolder(carbonTable, 
isForceDelete,
+  isDryRun = false, showStatistics)
   // step 2: move stale segments which are not exists in metadata into 
.Trash
   moveStaleSegmentsToTrash(carbonTable)
   // step 3: clean expired segments(MARKED_FOR_DELETE, Compacted, In 
Progress)
-  checkAndCleanExpiredSegments(carbonTable, isForceDelete, 
cleanStaleInProgress, partitionSpecs)
+  // Since calculating the the size before and after clean files can be a 
costly operation
+  // have exposed an option where user can change this behaviour.
+  if (showStatistics) {
+val sizeBeforeCleaning = getSizeScreenshot(carbonTable)

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-21 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r580044175



##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
##
@@ -41,6 +43,26 @@ case class CarbonCleanFilesCommand(
   extends DataCommand {
 
   val LOGGER: Logger = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+  val isDryRun: Boolean = options.getOrElse("dryrun", "false").toBoolean
+  var showStats: Boolean = options.getOrElse("statistics", "true").toBoolean
+  if (isInternalCleanCall) {
+showStats = false
+  }
+
+  override def output: Seq[AttributeReference] = {
+if (isDryRun) {
+  // dry run operation
+  Seq(
+AttributeReference("Size Freed", StringType, nullable = false)(),

Review comment:
   I am using convertBytesToReadable before printing it, in that it will 
add the necessary size value(Kb/Mb etc) in the result itself, so no need to add 
it in the Attribute Reference. Yes this is both inside and outside trash folder





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-21 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r580001530



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##
@@ -87,13 +104,51 @@ object DataTrashManager {
 }
   }
 
-  private def checkAndCleanTrashFolder(carbonTable: CarbonTable, 
isForceDelete: Boolean): Unit = {
+  /**
+   * Checks the size of the segment files as well as datafiles, this method is 
used before and after
+   * clean files operation to check how much space is actually freed, during 
the operation.
+   */
+  def getSizeScreenshot(carbonTable: CarbonTable): Long = {
+val metadataDetails = 
SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+var size: Long = 0
+val segmentFileLocation = 
CarbonTablePath.getSegmentFilesLocation(carbonTable.getTablePath)
+if (FileFactory.isFileExist(segmentFileLocation)) {
+  size += FileFactory.getDirectorySize(segmentFileLocation)
+}
+metadataDetails.foreach(oneLoad =>
+  if (oneLoad.getVisibility.toBoolean) {
+size += calculateSegmentSizeForOneLoad(carbonTable, oneLoad, 
metadataDetails)
+  }
+)
+size
+  }
+
+  /**
+   * Method to handle the Clean files dry run operation
+   */
+  def cleanFilesDryRunOperation (
+  carbonTable: CarbonTable,
+  isForceDelete: Boolean,
+  cleanStaleInProgress: Boolean,
+  showStats: Boolean): Seq[Long] = {
+// get size freed from the trash folder
+val trashFolderSizeStats = checkAndCleanTrashFolder(carbonTable, 
isForceDelete,
+isDryRun = true, showStats)
+// get size that will be deleted (MFD, COmpacted, Inprogress segments)
+val expiredSegmentsSizeStats = dryRunOnExpiredSegments(carbonTable, 
isForceDelete,
+  cleanStaleInProgress)
+Seq(trashFolderSizeStats.head + expiredSegmentsSizeStats.head, 
trashFolderSizeStats(1) +
+expiredSegmentsSizeStats(1))
+  }
+
+  private def checkAndCleanTrashFolder(carbonTable: CarbonTable, 
isForceDelete: Boolean,
+  isDryRun: Boolean, showStats: Boolean): Seq[Long] = {

Review comment:
   done

##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##
@@ -121,6 +176,78 @@ object DataTrashManager {
 }
   }
 
+  /**
+   * Does Clean files dry run operation on the expired segments. Returns the 
size freed
+   * during that clean files operation and also shows the remaining trash 
size, which can be
+   * cleaned after those segments are expired
+   */
+  private def dryRunOnExpiredSegments(
+  carbonTable: CarbonTable,
+  isForceDelete: Boolean,
+  cleanStaleInProgress: Boolean): Seq[Long] = {

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-21 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r579996841



##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
##
@@ -41,6 +43,26 @@ case class CarbonCleanFilesCommand(
   extends DataCommand {
 
   val LOGGER: Logger = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+  val isDryRun: Boolean = options.getOrElse("dryrun", "false").toBoolean
+  var showStats: Boolean = options.getOrElse("statistics", "true").toBoolean
+  if (isInternalCleanCall) {
+showStats = false
+  }
+
+  override def output: Seq[AttributeReference] = {
+if (isDryRun) {
+  // dry run operation
+  Seq(
+AttributeReference("Size Freed", StringType, nullable = false)(),
+AttributeReference("Trash Data Remaining", StringType, nullable = 
false)())
+} else if (!isDryRun && showStats) {

Review comment:
   Yes, it was a redundant check, removed 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-21 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r579996496



##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
##
@@ -41,6 +43,26 @@ case class CarbonCleanFilesCommand(
   extends DataCommand {
 
   val LOGGER: Logger = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+  val isDryRun: Boolean = options.getOrElse("dryrun", "false").toBoolean
+  var showStats: Boolean = options.getOrElse("statistics", "true").toBoolean

Review comment:
   Yeah, good idea

##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
##
@@ -41,6 +43,26 @@ case class CarbonCleanFilesCommand(
   extends DataCommand {
 
   val LOGGER: Logger = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+  val isDryRun: Boolean = options.getOrElse("dryrun", "false").toBoolean
+  var showStats: Boolean = options.getOrElse("statistics", "true").toBoolean

Review comment:
   Yeah, good idea. Edited





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-21 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r579996248



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##
@@ -121,6 +176,78 @@ object DataTrashManager {
 }
   }
 
+  /**
+   * Does Clean files dry run operation on the expired segments. Returns the 
size freed
+   * during that clean files operation and also shows the remaining trash 
size, which can be
+   * cleaned after those segments are expired
+   */
+  private def dryRunOnExpiredSegments(
+  carbonTable: CarbonTable,
+  isForceDelete: Boolean,
+  cleanStaleInProgress: Boolean): Seq[Long] = {
+var sizeFreed: Long = 0
+var trashSizeRemaining: Long = 0
+val loadMetadataDetails = 
SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+if (SegmentStatusManager.isLoadDeletionRequired(loadMetadataDetails)) {
+  loadMetadataDetails.foreach { oneLoad =>
+val segmentFilePath = 
CarbonTablePath.getSegmentFilePath(carbonTable.getTablePath,
+  oneLoad.getSegmentFile)
+if (DeleteLoadFolders.canDeleteThisLoad(oneLoad, isForceDelete, 
cleanStaleInProgress)) {
+  // No need to consider physical data for external segments, only 
consider metadata.
+  if (oneLoad.getPath() == null || 
oneLoad.getPath().equalsIgnoreCase("NA")) {
+sizeFreed += calculateSegmentSizeForOneLoad(carbonTable, oneLoad, 
loadMetadataDetails)
+  }
+  sizeFreed += FileFactory.getCarbonFile(segmentFilePath).getSize
+} else {
+  if (SegmentStatusManager.isExpiredSegment(oneLoad, carbonTable
+  .getAbsoluteTableIdentifier)) {
+trashSizeRemaining += calculateSegmentSizeForOneLoad(carbonTable, 
oneLoad,
+loadMetadataDetails)
+trashSizeRemaining += 
FileFactory.getCarbonFile(segmentFilePath).getSize
+  }
+}
+  }
+}
+Seq(sizeFreed, trashSizeRemaining)
+  }
+
+  /**
+   * calculates the segment size based of a segment
+   */
+  def calculateSegmentSizeForOneLoad( carbonTable: CarbonTable, oneLoad: 
LoadMetadataDetails,
+loadMetadataDetails: Array[LoadMetadataDetails]) : Long = {
+var size : Long = 0
+if (oneLoad.getDataSize!= null && !oneLoad.getDataSize.isEmpty) {

Review comment:
   Okay

##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##
@@ -121,6 +176,78 @@ object DataTrashManager {
 }
   }
 
+  /**
+   * Does Clean files dry run operation on the expired segments. Returns the 
size freed
+   * during that clean files operation and also shows the remaining trash 
size, which can be
+   * cleaned after those segments are expired
+   */
+  private def dryRunOnExpiredSegments(
+  carbonTable: CarbonTable,
+  isForceDelete: Boolean,
+  cleanStaleInProgress: Boolean): Seq[Long] = {
+var sizeFreed: Long = 0
+var trashSizeRemaining: Long = 0
+val loadMetadataDetails = 
SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+if (SegmentStatusManager.isLoadDeletionRequired(loadMetadataDetails)) {
+  loadMetadataDetails.foreach { oneLoad =>
+val segmentFilePath = 
CarbonTablePath.getSegmentFilePath(carbonTable.getTablePath,
+  oneLoad.getSegmentFile)
+if (DeleteLoadFolders.canDeleteThisLoad(oneLoad, isForceDelete, 
cleanStaleInProgress)) {
+  // No need to consider physical data for external segments, only 
consider metadata.
+  if (oneLoad.getPath() == null || 
oneLoad.getPath().equalsIgnoreCase("NA")) {
+sizeFreed += calculateSegmentSizeForOneLoad(carbonTable, oneLoad, 
loadMetadataDetails)
+  }
+  sizeFreed += FileFactory.getCarbonFile(segmentFilePath).getSize
+} else {
+  if (SegmentStatusManager.isExpiredSegment(oneLoad, carbonTable
+  .getAbsoluteTableIdentifier)) {
+trashSizeRemaining += calculateSegmentSizeForOneLoad(carbonTable, 
oneLoad,
+loadMetadataDetails)
+trashSizeRemaining += 
FileFactory.getCarbonFile(segmentFilePath).getSize
+  }
+}
+  }
+}
+Seq(sizeFreed, trashSizeRemaining)
+  }
+
+  /**
+   * calculates the segment size based of a segment
+   */
+  def calculateSegmentSizeForOneLoad( carbonTable: CarbonTable, oneLoad: 
LoadMetadataDetails,
+loadMetadataDetails: Array[LoadMetadataDetails]) : Long = {
+var size : Long = 0
+if (oneLoad.getDataSize!= null && !oneLoad.getDataSize.isEmpty) {

Review comment:
   yeah





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:

[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-21 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r579995778



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -1161,10 +1167,12 @@ private static void 
deletePhysicalPartition(List partitionSpecs,
 boolean exists = pathExistsInPartitionSpec(partitionSpecs, location);
 if (!exists) {
   
FileFactory.deleteAllCarbonFilesOfDir(FileFactory.getCarbonFile(location.toString()));
+  LOGGER.info("Deleted the mergeindex file: " + location.toString());

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-21 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r579995710



##
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##
@@ -150,48 +151,92 @@ public static void copyFilesToTrash(List 
filesToCopy,
 
   /**
* The below method deletes timestamp subdirectories in the trash folder 
which have expired as
-   * per the user defined retention time
+   * per the user defined retention time. It return an array where the first 
element has the size
+   * freed from the trash folder and the second element has the remaining size 
in the trash folder
*/
-  public static void deleteExpiredDataFromTrash(String tablePath) {
+  public static long[] deleteExpiredDataFromTrash(String tablePath, Boolean 
isDryRun,
+  Boolean showStats) {
 CarbonFile trashFolder = FileFactory.getCarbonFile(CarbonTablePath
 .getTrashFolderPath(tablePath));
+long sizeFreed = 0;
+long trashFolderSize = 0;
 // Deleting the timestamp based subdirectories in the trashfolder by the 
given timestamp.
 try {
   if (trashFolder.isFileExist()) {
+if (isDryRun || showStats) {
+  trashFolderSize = 
FileFactory.getDirectorySize(trashFolder.getAbsolutePath());
+}
 CarbonFile[] timestampFolderList = trashFolder.listFiles();
+List filesToDelete = new ArrayList<>();
 for (CarbonFile timestampFolder : timestampFolderList) {
   // If the timeStamp at which the timeStamp subdirectory has expired 
as per the user
   // defined value, delete the complete timeStamp subdirectory
-  if (timestampFolder.isDirectory() && 
isTrashRetentionTimeoutExceeded(Long
-  .parseLong(timestampFolder.getName( {
-FileFactory.deleteAllCarbonFilesOfDir(timestampFolder);
-LOGGER.info("Timestamp subfolder from the Trash folder deleted: " 
+ timestampFolder
+  if 
(isTrashRetentionTimeoutExceeded(Long.parseLong(timestampFolder.getName( {
+if (timestampFolder.isDirectory()) {

Review comment:
   technically it won't, just added a fail safe in case someone changes 
something in future.





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-21 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r579995522



##
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##
@@ -150,48 +151,92 @@ public static void copyFilesToTrash(List 
filesToCopy,
 
   /**
* The below method deletes timestamp subdirectories in the trash folder 
which have expired as
-   * per the user defined retention time
+   * per the user defined retention time. It return an array where the first 
element has the size
+   * freed from the trash folder and the second element has the remaining size 
in the trash folder
*/
-  public static void deleteExpiredDataFromTrash(String tablePath) {
+  public static long[] deleteExpiredDataFromTrash(String tablePath, Boolean 
isDryRun,
+  Boolean showStats) {
 CarbonFile trashFolder = FileFactory.getCarbonFile(CarbonTablePath
 .getTrashFolderPath(tablePath));
+long sizeFreed = 0;
+long trashFolderSize = 0;
 // Deleting the timestamp based subdirectories in the trashfolder by the 
given timestamp.
 try {
   if (trashFolder.isFileExist()) {
+if (isDryRun || showStats) {
+  trashFolderSize = 
FileFactory.getDirectorySize(trashFolder.getAbsolutePath());
+}
 CarbonFile[] timestampFolderList = trashFolder.listFiles();
+List filesToDelete = new ArrayList<>();
 for (CarbonFile timestampFolder : timestampFolderList) {
   // If the timeStamp at which the timeStamp subdirectory has expired 
as per the user
   // defined value, delete the complete timeStamp subdirectory
-  if (timestampFolder.isDirectory() && 
isTrashRetentionTimeoutExceeded(Long
-  .parseLong(timestampFolder.getName( {
-FileFactory.deleteAllCarbonFilesOfDir(timestampFolder);
-LOGGER.info("Timestamp subfolder from the Trash folder deleted: " 
+ timestampFolder
+  if 
(isTrashRetentionTimeoutExceeded(Long.parseLong(timestampFolder.getName( {
+if (timestampFolder.isDirectory()) {
+  // only calculate size in case of dry run or in case clean files 
is with show stats
+  if (isDryRun || showStats) {
+sizeFreed += 
FileFactory.getDirectorySize(timestampFolder.getAbsolutePath());
+  }
+  filesToDelete.add(timestampFolder);
+}
+  }
+}
+if (!isDryRun) {
+  for (CarbonFile carbonFile : filesToDelete) {
+LOGGER.info("Timestamp subfolder from the Trash folder deleted: " 
+ carbonFile
 .getAbsolutePath());
+FileFactory.deleteAllCarbonFilesOfDir(carbonFile);
   }
 }
   }
 } catch (IOException e) {
   LOGGER.error("Error during deleting expired timestamp folder from the 
trash folder", e);
 }
+return new long[] {sizeFreed, trashFolderSize - sizeFreed};
   }
 
   /**
* The below method deletes all the files and folders in the trash folder of 
a carbon table.
+   * Returns an array in which the first element contains the size freed in 
case of clean files
+   * operation or size that can be freed in case of dry run and the second 
element contains the
+   * remaining size.
*/
-  public static void emptyTrash(String tablePath) {
+  public static long[] emptyTrash(String tablePath, Boolean isDryRun, Boolean 
showStats) {
 CarbonFile trashFolder = FileFactory.getCarbonFile(CarbonTablePath
 .getTrashFolderPath(tablePath));
 // if the trash folder exists delete the contents of the trash folder
+long sizeFreed = 0;
+long[] sizeStatistics = new long[]{0, 0};
 try {
   if (trashFolder.isFileExist()) {
 CarbonFile[] carbonFileList = trashFolder.listFiles();
+List filesToDelete = new ArrayList<>();
 for (CarbonFile carbonFile : carbonFileList) {
-  FileFactory.deleteAllCarbonFilesOfDir(carbonFile);
+  //Only calculate size when it is dry run operation or when show 
statistics is
+  // true with actual operation
+  if (isDryRun || showStats) {
+sizeFreed += 
FileFactory.getDirectorySize(carbonFile.getAbsolutePath());
+  }
+  filesToDelete.add(carbonFile);
+}
+sizeStatistics[0] = sizeFreed;
+if (!isDryRun) {
+  for (CarbonFile carbonFile : filesToDelete) {
+FileFactory.deleteAllCarbonFilesOfDir(carbonFile);
+  }
+  LOGGER.info("Trash Folder has been emptied for table: " + tablePath);
+  if (showStats) {
+sizeStatistics[1] = 
FileFactory.getDirectorySize(trashFolder.getAbsolutePath());
+  }
+} else {
+  sizeStatistics[1] = 
FileFactory.getDirectorySize(trashFolder.getAbsolutePath()) -

Review comment:
   when dryRun is true, showStats is not taken into account





[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-19 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r579107479



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -1133,17 +1133,23 @@ public static void deleteSegment(String tablePath, 
Segment segment,
 List indexOrMergeFiles = 
fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
 FileFactory.getConfiguration());
 Map> indexFilesMap = fileStore.getIndexFilesMap();
+StringBuilder deletedFiles = new StringBuilder();

Review comment:
   used string.join with delimiter

##
File path: 
core/src/main/java/org/apache/carbondata/core/util/CleanFilesUtil.java
##
@@ -74,13 +74,19 @@ public static void cleanStaleSegments(CarbonTable 
carbonTable)
   // delete the segment file as well
   
FileFactory.deleteFile(CarbonTablePath.getSegmentFilePath(carbonTable.getTablePath(),
   staleSegmentFile));
+  StringBuilder deletedFiles = new StringBuilder();

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-19 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r579106425



##
File path: 
core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##
@@ -1895,6 +1895,11 @@ private CarbonCommonConstants() {
*/
   public static final String COMMA = ",";
 
+  /**
+   * SINGLE SPACE
+   */
+  public static final String SPACE = " ";

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-19 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r579083355



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##
@@ -87,13 +104,48 @@ object DataTrashManager {
 }
   }
 
-  private def checkAndCleanTrashFolder(carbonTable: CarbonTable, 
isForceDelete: Boolean): Unit = {
+  /**
+   * Checks the size of the segment files as well as datafiles, this method is 
used before and after
+   * clean files operation to check how much space is actually freed, during 
the operation.
+   */
+  def getSizeScreenshot(carbonTable: CarbonTable): Long = {
+val metadataDetails = 
SegmentStatusManager.readLoadMetadata(carbonTable.getMetadataPath)
+var size: Long = 
FileFactory.getDirectorySize(CarbonTablePath.getSegmentFilesLocation(

Review comment:
   changed logic





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-19 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r579082607



##
File path: docs/clean-files.md
##
@@ -64,4 +64,41 @@ The stale_inprogress option with force option will delete 
Marked for delete, Com
 
   ```
   CLEAN FILES FOR TABLE TABLE_NAME options('stale_inprogress'='true', 
'force'='true')
-  ```
\ No newline at end of file
+  ```
+### DRY RUN OPTION
+Clean files also support a dry run option which will let the user know how 
much space fill we freed 
+during the actual clean files operation. The dry run operation will not delete 
any data but will just give
+size based statistics on the data which will be cleaned in clean files. Dry 
run operation will return two columns where the first will 
+show how much space will be freed by that clean files operation and the second 
column will show the 
+remaining stale data(data which can be deleted but has not yet expired as per 
the ```max.query.execution.time``` and ``` carbon.trash.retention.days``` values
+).  By default the value of ```dryrun``` option is ```false```.
+
+Dry Run Operation is supported with four types of commands:
+  ```
+  CLEAN FILES FOR TABLE TABLE_NAME options('dryrun'='true')
+  ```
+  ```
+  CLEAN FILES FOR TABLE TABLE_NAME options('force'='true', 'dryrun'='true')
+  ```
+  ```
+  CLEAN FILES FOR TABLE TABLE_NAME 
options('stale_inprogress'='true','dryrun'='true')
+  ```
+
+  ```
+  CLEAN FILES FOR TABLE TABLE_NAME options('stale_inprogress'='true', 
'force'='true','dryrun'='true')
+  ```
+
+**NOTE**:
+  * Since the dry run operation will calculate size and will access File level 
API's, the operation can
+  be a costly and a time consuming operation in case of tables with large 
number of segments.
+  * When dry run is true, the statistics option will not matter.
+  
+### SHOW STATISTICS
+Clean files operation tells how much size is freed during that operation to 
the user.  By default, the clean files operation
+will show the size freed statistics. Since calculating and showing statistics 
can be a costly operation and reduce the performance of the
+clean files operation, the user can disable that option by using ```statistics 
= false``` in the clean files options.
+  
+   ```
+   CLEAN FILES FOR TABLE TABLE_NAME options('statistics`='false')

Review comment:
   done

##
File path: docs/clean-files.md
##
@@ -64,4 +64,41 @@ The stale_inprogress option with force option will delete 
Marked for delete, Com
 
   ```
   CLEAN FILES FOR TABLE TABLE_NAME options('stale_inprogress'='true', 
'force'='true')
-  ```
\ No newline at end of file
+  ```
+### DRY RUN OPTION
+Clean files also support a dry run option which will let the user know how 
much space fill we freed 

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-19 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r579070718



##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
##
@@ -41,6 +42,26 @@ case class CarbonCleanFilesCommand(
   extends DataCommand {
 
   val LOGGER: Logger = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+  val isDryRun: Boolean = options.getOrElse("dryrun", "false").toBoolean
+  var showStats: Boolean = options.getOrElse("statistics", "true").toBoolean
+  if (isInternalCleanCall) {
+showStats = false
+  }
+
+  override def output: Seq[AttributeReference] = {
+if (isDryRun) {
+  // dry run operation
+  Seq(
+AttributeReference("Size Freed", LongType, nullable = false)(),
+AttributeReference("Trash Data Remaining", LongType, nullable = 
false)())

Review comment:
   done, using, Using ByteUtil.convertByteToReadable, it will convert the 
result to string and add necessary unit to it, either KB or MR





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-19 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r578993995



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##
@@ -72,11 +78,26 @@ object DataTrashManager {
   carbonDeleteSegmentLock = CarbonLockUtil.getLockObject(carbonTable
 .getAbsoluteTableIdentifier, LockUsage.DELETE_SEGMENT_LOCK, 
deleteSegmentErrorMsg)
   // step 1: check and clean trash folder
-  checkAndCleanTrashFolder(carbonTable, isForceDelete)
+  // trashFolderSizeStats(0) contains the size that is freed/or can be 
freed and
+  // trashFolderSizeStats(1) contains the size of remaining data in the 
trash folder
+  val trashFolderSizeStats = checkAndCleanTrashFolder(carbonTable, 
isForceDelete,
+  isDryRun = false)
   // step 2: move stale segments which are not exists in metadata into 
.Trash
   moveStaleSegmentsToTrash(carbonTable)

Review comment:
   No it won't matter, because we need to show how much space can be 
cleared from the trash, when moving from segment folder to the trash folder, we 
will be cleaning it in the next clean files command only, so we don't need to 
put these stats





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-18 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r578990010



##
File path: 
core/src/main/java/org/apache/carbondata/core/util/CleanFilesUtil.java
##
@@ -71,14 +71,19 @@ public static void cleanStaleSegments(CarbonTable 
carbonTable)
 // Deleting the stale Segment folders and the segment file.
 try {
   CarbonUtil.deleteFoldersAndFiles(segmentPath);
+  LOGGER.info("Deleted the segment folder :" + 
segmentPath.getAbsolutePath() + " after"
+  + " moving it to the trash folder");
   // delete the segment file as well
   
FileFactory.deleteFile(CarbonTablePath.getSegmentFilePath(carbonTable.getTablePath(),
   staleSegmentFile));
+  LOGGER.info("Deleted stale segment file after moving it to the 
trash folder :"
+  + staleSegmentFile);

Review comment:
   done, actually the segment file will not be moved to trash, it will just 
be delete straightaway, Handled the same

##
File path: 
core/src/main/java/org/apache/carbondata/core/util/CleanFilesUtil.java
##
@@ -71,14 +71,19 @@ public static void cleanStaleSegments(CarbonTable 
carbonTable)
 // Deleting the stale Segment folders and the segment file.
 try {
   CarbonUtil.deleteFoldersAndFiles(segmentPath);
+  LOGGER.info("Deleted the segment folder :" + 
segmentPath.getAbsolutePath() + " after"
+  + " moving it to the trash folder");
   // delete the segment file as well
   
FileFactory.deleteFile(CarbonTablePath.getSegmentFilePath(carbonTable.getTablePath(),
   staleSegmentFile));
+  LOGGER.info("Deleted stale segment file after moving it to the 
trash folder :"
+  + staleSegmentFile);
   for (String duplicateStaleSegmentFile : redundantSegmentFile) {
 if 
(DataFileUtil.getSegmentNoFromSegmentFile(duplicateStaleSegmentFile)
 .equals(segmentNumber)) {
   
FileFactory.deleteFile(CarbonTablePath.getSegmentFilePath(carbonTable
   .getTablePath(), duplicateStaleSegmentFile));
+  LOGGER.info("Deleted redundant segment file :" + 
duplicateStaleSegmentFile);

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-18 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r578987875



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -1135,13 +1135,16 @@ public static void deleteSegment(String tablePath, 
Segment segment,
 Map> indexFilesMap = fileStore.getIndexFilesMap();
 for (Map.Entry> entry : indexFilesMap.entrySet()) {
   FileFactory.deleteFile(entry.getKey());
+  LOGGER.info("File deleted after clean files operation: " + 
entry.getKey());
   for (String file : entry.getValue()) {
 String[] deltaFilePaths =
 updateStatusManager.getDeleteDeltaFilePath(file, 
segment.getSegmentNo());
 for (String deltaFilePath : deltaFilePaths) {
   FileFactory.deleteFile(deltaFilePath);
+  LOGGER.info("File deleted after clean files operation: " + 
deltaFilePath);

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-18 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r578982799



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -1135,13 +1135,16 @@ public static void deleteSegment(String tablePath, 
Segment segment,
 Map> indexFilesMap = fileStore.getIndexFilesMap();
 for (Map.Entry> entry : indexFilesMap.entrySet()) {
   FileFactory.deleteFile(entry.getKey());
+  LOGGER.info("File deleted after clean files operation: " + 
entry.getKey());

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-18 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r578982527



##
File path: 
core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##
@@ -983,7 +983,28 @@ public static boolean 
isLoadInProgress(AbsoluteTableIdentifier absoluteTableIden
 }
   }
 
-  private static boolean isLoadDeletionRequired(LoadMetadataDetails[] details) 
{
+  public static boolean isExpiredSegment(LoadMetadataDetails oneLoad, 
AbsoluteTableIdentifier
+  absoluteTableIdentifier) {
+boolean result = false;

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-18 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r578982092



##
File path: 
core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##
@@ -983,7 +983,28 @@ public static boolean 
isLoadInProgress(AbsoluteTableIdentifier absoluteTableIden
 }
   }
 
-  private static boolean isLoadDeletionRequired(LoadMetadataDetails[] details) 
{
+  public static boolean isExpiredSegment(LoadMetadataDetails oneLoad, 
AbsoluteTableIdentifier
+  absoluteTableIdentifier) {
+boolean result = false;
+if (oneLoad.getSegmentStatus() == SegmentStatus.COMPACTED || 
oneLoad.getSegmentStatus() ==
+SegmentStatus.MARKED_FOR_DELETE) {
+  return true;

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-18 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r578981989



##
File path: 
core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##
@@ -983,7 +983,28 @@ public static boolean 
isLoadInProgress(AbsoluteTableIdentifier absoluteTableIden
 }
   }
 
-  private static boolean isLoadDeletionRequired(LoadMetadataDetails[] details) 
{
+  public static boolean isExpiredSegment(LoadMetadataDetails oneLoad, 
AbsoluteTableIdentifier
+  absoluteTableIdentifier) {
+boolean result = false;
+if (oneLoad.getSegmentStatus() == SegmentStatus.COMPACTED || 
oneLoad.getSegmentStatus() ==
+SegmentStatus.MARKED_FOR_DELETE) {
+  return true;
+} else if (oneLoad.getSegmentStatus() == SegmentStatus.INSERT_IN_PROGRESS 
|| oneLoad
+.getSegmentStatus() == SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS) {
+  // check if lock can be acquired
+  ICarbonLock segmentLock = 
CarbonLockFactory.getCarbonLockObj(absoluteTableIdentifier,
+  CarbonTablePath.addSegmentPrefix(oneLoad.getLoadName()) + 
LockUsage.LOCK);
+  if (segmentLock.lockWithRetries()) {
+result = true;
+segmentLock.unlock();

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-18 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r578974314



##
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##
@@ -152,46 +153,77 @@ public static void copyFilesToTrash(List 
filesToCopy,
* The below method deletes timestamp subdirectories in the trash folder 
which have expired as
* per the user defined retention time
*/
-  public static void deleteExpiredDataFromTrash(String tablePath) {
+  public static long[] deleteExpiredDataFromTrash(String tablePath, Boolean 
isDryRun) {
 CarbonFile trashFolder = FileFactory.getCarbonFile(CarbonTablePath
 .getTrashFolderPath(tablePath));
+long sizeFreed = 0;
+long trashFolderSize = 0;
 // Deleting the timestamp based subdirectories in the trashfolder by the 
given timestamp.
 try {
   if (trashFolder.isFileExist()) {
+trashFolderSize = 
FileFactory.getDirectorySize(trashFolder.getAbsolutePath());
 CarbonFile[] timestampFolderList = trashFolder.listFiles();
+List filesToDelete = new ArrayList<>();
 for (CarbonFile timestampFolder : timestampFolderList) {
   // If the timeStamp at which the timeStamp subdirectory has expired 
as per the user
   // defined value, delete the complete timeStamp subdirectory
-  if (timestampFolder.isDirectory() && 
isTrashRetentionTimeoutExceeded(Long
-  .parseLong(timestampFolder.getName( {
-FileFactory.deleteAllCarbonFilesOfDir(timestampFolder);
-LOGGER.info("Timestamp subfolder from the Trash folder deleted: " 
+ timestampFolder
+  if 
(isTrashRetentionTimeoutExceeded(Long.parseLong(timestampFolder.getName( {
+if (timestampFolder.isDirectory()) {
+  sizeFreed += 
FileFactory.getDirectorySize(timestampFolder.getAbsolutePath());
+  filesToDelete.add(timestampFolder);
+}
+  }
+}
+if (!isDryRun) {
+  for (CarbonFile carbonFile : filesToDelete) {
+LOGGER.info("Timestamp subfolder from the Trash folder deleted: " 
+ carbonFile
 .getAbsolutePath());
+FileFactory.deleteAllCarbonFilesOfDir(carbonFile);
   }
 }
   }
 } catch (IOException e) {
   LOGGER.error("Error during deleting expired timestamp folder from the 
trash folder", e);
 }
+return new long[] {sizeFreed, trashFolderSize - sizeFreed};
   }
 
   /**
* The below method deletes all the files and folders in the trash folder of 
a carbon table.
+   * Returns an array in which the first element contains the size freed in 
case of clean files
+   * operation or size that can be freed in case of dry run and the second 
element contains the
+   * remaining size.
*/
-  public static void emptyTrash(String tablePath) {
+  public static long[] emptyTrash(String tablePath, Boolean isDryRun) {
 CarbonFile trashFolder = FileFactory.getCarbonFile(CarbonTablePath
 .getTrashFolderPath(tablePath));
 // if the trash folder exists delete the contents of the trash folder
+long sizeFreed = 0;
+long[] sizeStatistics = new long[]{0, 0};
 try {
   if (trashFolder.isFileExist()) {
 CarbonFile[] carbonFileList = trashFolder.listFiles();
+List filesToDelete = new ArrayList<>();
 for (CarbonFile carbonFile : carbonFileList) {
-  FileFactory.deleteAllCarbonFilesOfDir(carbonFile);
+  sizeFreed += 
FileFactory.getDirectorySize(carbonFile.getAbsolutePath());

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-18 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r578967973



##
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##
@@ -152,46 +153,77 @@ public static void copyFilesToTrash(List 
filesToCopy,
* The below method deletes timestamp subdirectories in the trash folder 
which have expired as
* per the user defined retention time
*/
-  public static void deleteExpiredDataFromTrash(String tablePath) {
+  public static long[] deleteExpiredDataFromTrash(String tablePath, Boolean 
isDryRun) {

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-18 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r578967247



##
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##
@@ -152,46 +153,77 @@ public static void copyFilesToTrash(List 
filesToCopy,
* The below method deletes timestamp subdirectories in the trash folder 
which have expired as
* per the user defined retention time
*/
-  public static void deleteExpiredDataFromTrash(String tablePath) {
+  public static long[] deleteExpiredDataFromTrash(String tablePath, Boolean 
isDryRun) {
 CarbonFile trashFolder = FileFactory.getCarbonFile(CarbonTablePath
 .getTrashFolderPath(tablePath));
+long sizeFreed = 0;
+long trashFolderSize = 0;
 // Deleting the timestamp based subdirectories in the trashfolder by the 
given timestamp.
 try {
   if (trashFolder.isFileExist()) {
+trashFolderSize = 
FileFactory.getDirectorySize(trashFolder.getAbsolutePath());

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-18 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r578963097



##
File path: docs/clean-files.md
##
@@ -64,4 +64,40 @@ The stale_inprogress option with force option will delete 
Marked for delete, Com
 
   ```
   CLEAN FILES FOR TABLE TABLE_NAME options('stale_inprogress'='true', 
'force'='true')
-  ```
\ No newline at end of file
+  ```
+### DRY RUN OPTION
+Clean files also support a dry run option which will let the user know how 
much space fill we freed 
+during the actual clean files operation. The dry run operation will not delete 
any data but will just give
+size bases statistics to the data. Dry run operation will return two columns 
where the first will 
+show how much space will be freed by that clean files operation and the second 
column will show the 
+remaining stale data(data which can be deleted but has not yet expired as per 
the ```max.query.execution.time``` and ``` carbon.trash.retention.days``` values
+).  By default the value of ```dryrun``` option is ```false```.
+
+Dry Run Operation is supported with four types of commands:
+  ```
+  CLEAN FILES FOR TABLE TABLE_NAME options('dryrun'='true')
+  ```
+  ```
+  CLEAN FILES FOR TABLE TABLE_NAME options('force'='true', 'dryrun'='true')
+  ```
+  ```
+  CLEAN FILES FOR TABLE TABLE_NAME 
options('stale_inprogress'='true','dryrun'='true')
+  ```
+
+  ```
+  CLEAN FILES FOR TABLE TABLE_NAME options('stale_inprogress'='true', 
'force'='true','dryrun'='true')
+  ```
+
+**NOTE**:
+  * Since the dry run operation will calculate size and will access File level 
API's, the operation can
+  be a costly and a time consuming operation in case of tables with large 
number of segments.

Review comment:
   done

##
File path: docs/clean-files.md
##
@@ -64,4 +64,40 @@ The stale_inprogress option with force option will delete 
Marked for delete, Com
 
   ```
   CLEAN FILES FOR TABLE TABLE_NAME options('stale_inprogress'='true', 
'force'='true')
-  ```
\ No newline at end of file
+  ```
+### DRY RUN OPTION
+Clean files also support a dry run option which will let the user know how 
much space fill we freed 
+during the actual clean files operation. The dry run operation will not delete 
any data but will just give
+size bases statistics to the data. Dry run operation will return two columns 
where the first will 

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-18 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r578962180



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##
@@ -87,13 +108,70 @@ object DataTrashManager {
 }
   }
 
-  private def checkAndCleanTrashFolder(carbonTable: CarbonTable, 
isForceDelete: Boolean): Unit = {
+  /**
+   * Checks the size of the segment files as well as datafiles, this method is 
used before and after
+   * clean files operation to check how much space is actually freed, during 
the operation.
+   */
+  def getSizeScreenshot(carbonTable: CarbonTable): Long = {
+val segmentPath = 
CarbonTablePath.getSegmentFilesLocation(carbonTable.getTablePath)
+var size : Long = 0
+if (!carbonTable.isHivePartitionTable) {
+  if (carbonTable.getTableInfo.getFactTable.getTableProperties.containsKey(
+  CarbonCommonConstants.FLAT_FOLDER)) {
+// the size is table size + segment folder size - (metadata folder 
size + lockFiles size)
+(FileFactory.getDirectorySize(carbonTable.getTablePath) + 
FileFactory.getDirectorySize(

Review comment:
   done

##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/CleanFilesPostEventListener.scala
##
@@ -56,7 +56,6 @@ class CleanFilesPostEventListener extends 
OperationEventListener with Logging {
   cleanFilesPostEvent.carbonTable,
   cleanFilesPostEvent.options.getOrElse("force", "false").toBoolean,
   cleanFilesPostEvent.options.getOrElse("stale_inprogress", 
"false").toBoolean)
-

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-15 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r576601866



##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
##
@@ -37,11 +38,24 @@ case class CarbonCleanFilesCommand(
 databaseNameOp: Option[String],
 tableName: String,
 options: Map[String, String] = Map.empty,
+dryRun: Boolean,
 isInternalCleanCall: Boolean = false)
   extends DataCommand {
 
   val LOGGER: Logger = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
 
+  override def output: Seq[AttributeReference] = {
+if (dryRun) {
+  Seq(
+AttributeReference("Size that will be Freed", LongType, nullable = 
false)(),

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-15 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r576574599



##
File path: 
core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##
@@ -1297,4 +1356,37 @@ public static TableStatusReturnTuple 
separateVisibleAndInvisibleSegments(
   return new HashMap<>(0);
 }
   }
+
+  public static long partitionTableSegmentSize(CarbonTable carbonTable, 
LoadMetadataDetails
+  oneLoad, LoadMetadataDetails[] loadMetadataDetails, List
+  partitionSpecs) throws Exception {
+long size = 0;
+SegmentFileStore fileStore = new 
SegmentFileStore(carbonTable.getTablePath(), oneLoad
+.getSegmentFile());
+List indexOrMergeFiles = 
fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
+FileFactory.getConfiguration());
+Map> indexFilesMap = fileStore.getIndexFilesMap();

Review comment:
   This is just in the case of partition table, in non partition flow, we 
can just directly calculate the segment folder size





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-15 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r576569850



##
File path: 
core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##
@@ -1072,7 +1097,22 @@ public static void 
deleteLoadsAndUpdateMetadata(CarbonTable carbonTable, boolean
 isUpdateRequired(isForceDeletion, carbonTable,
 identifier, details, cleanStaleInprogress);
 if (!tuple2.isUpdateRequired) {
-  return;
+  try {
+for (LoadMetadataDetails oneLoad : details) {
+  if (isExpiredSegment(oneLoad, 
carbonTable.getAbsoluteTableIdentifier())) {
+if (!carbonTable.isHivePartitionTable()) {
+  trashSizeRemaining += 
FileFactory.getDirectorySize(CarbonTablePath
+.getSegmentPath(carbonTable.getTablePath(), 
oneLoad.getLoadName()));
+} else {
+  trashSizeRemaining += 
partitionTableSegmentSize(carbonTable, oneLoad,
+details, partitionSpecs);
+}
+  }
+}
+  } catch (Exception e) {
+LOG.error("Unable to calculate size of garbage data", e);
+  }
+  return new long[]{sizeFreed, trashSizeRemaining};

Review comment:
   we will just print the exception and carry on with the clean files 
command, not failing the whole thing, in case there is some error while 
calculating  the size, i guess it will return 0 then, or in case when nothing 
is deleted from clean files operation, then also we sill return the size freed 
as 0. Can have a discussion on this and change accrodingly.





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-15 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r576568703



##
File path: 
core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##
@@ -1072,7 +1097,22 @@ public static void 
deleteLoadsAndUpdateMetadata(CarbonTable carbonTable, boolean
 isUpdateRequired(isForceDeletion, carbonTable,
 identifier, details, cleanStaleInprogress);
 if (!tuple2.isUpdateRequired) {
-  return;
+  try {
+for (LoadMetadataDetails oneLoad : details) {
+  if (isExpiredSegment(oneLoad, 
carbonTable.getAbsoluteTableIdentifier())) {
+if (!carbonTable.isHivePartitionTable()) {

Review comment:
   done





This is an automated message from the Apache Git Service.
To respond to the message, please 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-15 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r576568152



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -1126,25 +1130,36 @@ public static void deleteSegmentFile(String tablePath, 
Segment segment) throws E
* @param partitionSpecs
* @throws IOException
*/
-  public static void deleteSegment(String tablePath, Segment segment,
+  public static long deleteSegment(String tablePath, Segment segment,
   List partitionSpecs,
   SegmentUpdateStatusManager updateStatusManager) throws Exception {
 SegmentFileStore fileStore = new SegmentFileStore(tablePath, 
segment.getSegmentFileName());
 List indexOrMergeFiles = 
fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
 FileFactory.getConfiguration());
+long sizeFreed = 0;
 Map> indexFilesMap = fileStore.getIndexFilesMap();
 for (Map.Entry> entry : indexFilesMap.entrySet()) {
-  FileFactory.deleteFile(entry.getKey());
+  CarbonFile entryCarbonFile = FileFactory.getCarbonFile(entry.getKey());
+  sizeFreed += entryCarbonFile.getSize();

Review comment:
   done

##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -1126,25 +1130,36 @@ public static void deleteSegmentFile(String tablePath, 
Segment segment) throws E
* @param partitionSpecs
* @throws IOException
*/
-  public static void deleteSegment(String tablePath, Segment segment,
+  public static long deleteSegment(String tablePath, Segment segment,
   List partitionSpecs,
   SegmentUpdateStatusManager updateStatusManager) throws Exception {
 SegmentFileStore fileStore = new SegmentFileStore(tablePath, 
segment.getSegmentFileName());
 List indexOrMergeFiles = 
fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
 FileFactory.getConfiguration());
+long sizeFreed = 0;
 Map> indexFilesMap = fileStore.getIndexFilesMap();
 for (Map.Entry> entry : indexFilesMap.entrySet()) {
-  FileFactory.deleteFile(entry.getKey());
+  CarbonFile entryCarbonFile = FileFactory.getCarbonFile(entry.getKey());
+  sizeFreed += entryCarbonFile.getSize();
+  FileFactory.deleteFile(entryCarbonFile);
+  LOGGER.info("File deleted after clean files operation: " + 
entry.getKey());
   for (String file : entry.getValue()) {
 String[] deltaFilePaths =
 updateStatusManager.getDeleteDeltaFilePath(file, 
segment.getSegmentNo());
 for (String deltaFilePath : deltaFilePaths) {
-  FileFactory.deleteFile(deltaFilePath);
+  CarbonFile deltaCarbonFile = 
FileFactory.getCarbonFile(deltaFilePath);
+  sizeFreed += deltaCarbonFile.getSize();
+  FileFactory.deleteFile(deltaCarbonFile);
+  LOGGER.info("File deleted after clean files operation: " + 
deltaFilePath);
 }
-FileFactory.deleteFile(file);
+CarbonFile deleteCarbonFile = FileFactory.getCarbonFile(file);
+sizeFreed += deleteCarbonFile.getSize();
+FileFactory.deleteFile(deleteCarbonFile);

Review comment:
   yes, good idea. Changed 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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

2021-02-15 Thread GitBox


vikramahuja1001 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r576564856



##
File path: 
core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
##
@@ -297,6 +297,10 @@ public static boolean deleteFile(String filePath) throws 
IOException {
 return getCarbonFile(filePath).deleteFile();
   }
 
+  public static boolean deleteFile(CarbonFile carbonFile) throws IOException {

Review comment:
   Yes, it is there but it tries to get the carbonFile first, in case there 
is carbonFile already present in the parent method, it will again get it, which 
can be avoided and is more optimised. There is even a JIRA opened by Ajantha to 
do the exact same thing, so that we reduce unnecessary calls to getCarbonFIle.





This is an automated message from the Apache Git Service.
To respond to the message, please 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