You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by GitBox <gi...@apache.org> on 2021/01/07 12:18:22 UTC

[GitHub] [carbondata] vikramahuja1001 opened a new pull request #4072: [WIP] Clean files phase2

vikramahuja1001 opened a new pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072


    ### Why is this PR needed?
    Dry run support
    
    ### What changes were proposed in this PR?
   
       
    ### Does this PR introduce any user interface change?
    - No
    - Yes. (please explain the change and update document)
   
    ### Is any new testcase added?
    - No
    - Yes
   
       
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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<String> 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<CarbonFile> 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<CarbonFile> 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




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-758545921


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3540/
   


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-758540797


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5300/
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-796670718


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5553/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
ydvpankaj99 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-783135423


   retest this please


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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-794008649


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3364/
   


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

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



[GitHub] [carbondata] vikramahuja1001 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-766562961


   retest this please


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

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



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

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r579263541



##########
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##########
@@ -150,48 +151,92 @@ public static void copyFilesToTrash(List<String> 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<CarbonFile> 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:
       would `if (timestampFolder.isDirectory()) ` ever be false ?

##########
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##########
@@ -150,48 +151,92 @@ public static void copyFilesToTrash(List<String> 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<CarbonFile> 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<CarbonFile> 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:
       need sizeStatistics[1] when isDryRun is true and showStats is false ?

##########
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:
       probably space missed before `!=`. same at line 223.

##########
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:
       Returning array of 2(freed size, remaining size). We return the tuple is more meaningful and readable in the scala?

##########
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:
       same as above comment regd. return type

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

Review comment:
       This could be index or merge index? If so, how about we add `LOGGER.info("Deleting files:)` before loop and inside loop just have `LOGGER.info(location.toString());`

##########
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:
       !isDryRun would always evaluates to true.

##########
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:
       you would want to use !StringUtils.isEmpty(oneLoad.getDataSize) ? can check similar at other place

##########
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:
       showStats can still be immutable. You might want to do something like this - 
     `val showStats: Boolean = if (isInternalCleanCall) {
       false
     } else {
       options.getOrElse("statistics", "true").toBoolean
     }`




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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r580013774



##########
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:
       can you add one validation by doing getSegmentSize() and compare it with clean files output ? If some problem in statistics current validations may not catch it




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

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



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

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-768131508


   retest this please


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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-780826470


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3726/
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r577527003



##########
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:
       yes, better not mix the logic of dry run size calculation and actual clean files, keep it separate, so that user will know for sure that when he/she runs the dry run it might take some time as it will do calculation of 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:
users@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

Posted by GitBox <gi...@apache.org>.
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<String> 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<CarbonFile> 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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r579089632



##########
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:
       same as above

##########
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<String> indexOrMergeFiles = fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
         FileFactory.getConfiguration());
     Map<String, List<String>> indexFilesMap = fileStore.getIndexFilesMap();
+    StringBuilder deletedFiles = new StringBuilder();

Review comment:
       do not use string builder, use java functional APIs like below
   
   `String firstThenLast2 = files
   	.stream()
   	.map(p -> file.getPath)
   	.collect(Collectors.toStringJoiner(","))
           .toString();`
   
   try this

##########
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:
       revert this




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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-783109607


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3734/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-768938250


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3609/
   


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

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



[GitHub] [carbondata] QiangCai commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
QiangCai commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-758345002


   please perfect the PR title


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

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



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

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r579243982



##########
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<String> indexOrMergeFiles = fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
         FileFactory.getConfiguration());
     Map<String, List<String>> indexFilesMap = fileStore.getIndexFilesMap();
+    List<String> deletedFiles = new ArrayList<>();

Review comment:
       Instead of making this array list and join them irrespective of logging(info) is enabled or not. Probably deferring this string manipulations till the point logging is enabled is better?




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-766551855


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5339/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-772241927


   @QiangCai @ajantha-bhat @akashrn5 please review


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-768188868


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5360/
   


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-766610144


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3582/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
ydvpankaj99 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-783105607


   retest this please


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-766553622


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3294/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-782053003


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3336/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r580013774



##########
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:
       can you add one validation by doing getSegmentSize() and compare it with clean files output. If some problem in statistics current validations may not catch it




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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-795231224


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3296/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r577387770



##########
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:
       Please remove `deleteFile(CarbonFile carbonFile)`, because there is no extra function is happening here. directly call carbonFile.deleteFile() in the usages.
   
   If there was some extra functionality, then it makes sense to have to API for carbonFile also to avoid getting CarbonFile again. for this case it is not needed 




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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-795103375


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5531/
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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<String> indexOrMergeFiles = fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
         FileFactory.getConfiguration());
     Map<String, List<String>> 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:
users@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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-781959981


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3330/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-793825289


   LGTM


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-783213957


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3736/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-788052788


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3762/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-782214356


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3732/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-783125434


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5500/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-781209057


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5492/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-792734488


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3357/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-779699134


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5472/
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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<String, List<String>> indexFilesMap = fileStore.getIndexFilesMap();
     for (Map.Entry<String, List<String>> 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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-777598712


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5075/
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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 message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-799180625


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5566/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r578921513



##########
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<String, List<String>> indexFilesMap = fileStore.getIndexFilesMap();
     for (Map.Entry<String, List<String>> 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:
       instead of logging each file name for every delete which will increase these logs when many delta files are there, once the loop completes, you can log once for all files in line 1147, along with actual block path or else you can say delete the the block file(print block file name) and the corresponding delta files as filestamp will same i guess. Just check once and add

##########
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<String, List<String>> indexFilesMap = fileStore.getIndexFilesMap();
     for (Map.Entry<String, List<String>> entry : indexFilesMap.entrySet()) {
       FileFactory.deleteFile(entry.getKey());
+      LOGGER.info("File deleted after clean files operation: " + entry.getKey());

Review comment:
       ```suggestion
         LOGGER.info("Deleted  file: " + entry.getKey() +  ", on clean files" );
   ```
   
   can do same for all 

##########
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##########
@@ -152,46 +153,77 @@ public static void copyFilesToTrash(List<String> 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:
       update the method comment based on method signature changes and return values, follow the same for other also if changed

##########
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:
       here better to add point of when dry run is true, other options doesnt matter except force = true, and i hope you have handled this in code

##########
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:
       rename to `isExpiredSegment`

##########
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:
       use `CarbonLockUtil.fileUnlock` method as it logs also, please follow other places if taking lock and better to have it in finally block

##########
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:
       ```suggestion
                     deleted stale segment file <segment_file_name> after moving it to the trash folder
   ```

##########
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##########
@@ -152,46 +153,77 @@ public static void copyFilesToTrash(List<String> 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:
       i think you should not calculate size when the statistics is false, so calculate only when it is true else it will affect clean files time, please take care not to impact time of clean files

##########
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:
       here also assign to the boolean variable as its just checking for one segment

##########
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:
       can log together after loop, all the deleted files and also please check all places try to follow the same, this is just to make logs cleaner 

##########
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##########
@@ -152,46 +153,77 @@ public static void copyFilesToTrash(List<String> 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<CarbonFile> 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<CarbonFile> filesToDelete = new ArrayList<>();
         for (CarbonFile carbonFile : carbonFileList) {
-          FileFactory.deleteAllCarbonFilesOfDir(carbonFile);
+          sizeFreed += FileFactory.getDirectorySize(carbonFile.getAbsolutePath());

Review comment:
       same as above

##########
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:
       ```suggestion
   size based statistics on the data which will be cleaned in clean files. Dry run operation will return two columns where the first will 
   ```

##########
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:
       if you are calculating the trash size before calling `moveStaleSegmentsToTrash`, it gives wrong info right? because after size calculation, some folders are getting into trash folder again?

##########
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:
       here do not calculate the fact folder size directly , because when there 1000s of segments it will be slow. We are already storing the data and index size in metadata, please make use of it and it will faster

##########
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:
       revert this file

##########
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:
       i agree with @ajantha-bhat , may be use loadmetadata details data and index size only and for update files you can use some logic, please check




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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-768191257


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3600/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-796765741


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5556/
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-782112467


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5096/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-782063137


   LGTM for test, need couple more reviews to merge


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-766552055


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5052/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r577388524



##########
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:
       As per above comment, you can directly call carbonSegmentFile.delete()




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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-793543867


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3361/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-799096244


   retest this please


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-795238904


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5054/
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-792594384


   LGTM


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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r579051553



##########
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:
       ```suggestion
      CLEAN FILES FOR TABLE TABLE_NAME options('statistics'='false')
   ```

##########
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:
       ```suggestion
       var segmentFilesLocationSize: Long = FileFactory.getDirectorySize(CarbonTablePath.getSegmentFilesLocation(
   ```

##########
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:
       please return the return values of size in readable format, if giving in MB, have UNIT in column header as (MB)

##########
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:
       ```suggestion
   Clean files also support a dry run option which will let the user know how much space will we freed 
   ```




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-766631872


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3585/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-795431956


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5543/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-796591581


   retest this please


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

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



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

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-787906686


   @akashrn5 @VenuReddy2103 @ajantha-bhat , can this PR be merged?


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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-781208770


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3729/
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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<PartitionSpec> 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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
ydvpankaj99 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-768880622


   retest this please


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

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



[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

Posted by GitBox <gi...@apache.org>.
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<PartitionSpec>
+      partitionSpecs) throws Exception {
+    long size = 0;
+    SegmentFileStore fileStore = new SegmentFileStore(carbonTable.getTablePath(), oneLoad
+        .getSegmentFile());
+    List<String> indexOrMergeFiles = fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
+        FileFactory.getConfiguration());
+    Map<String, List<String>> 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:
users@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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r580151473



##########
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:
       For dry run, 
   a) we don't free up the space, so change it to `Size to be freed' 
   b) And trash data remaining is only data inside trash right ?  change it to 'Trash folder size' 
   
   for clean files, why are we not printing trash size after cleaning ? @vikramahuja1001 , @akashrn5 
   
    




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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-788049572


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5527/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-781561697


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3327/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-780379984


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5477/
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590016222



##########
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:
       give a proper variable name here




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

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



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r577458113



##########
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:
       I am thinking now all the clean file operations will become slow because of these size calculation code, which need to interact with the file system.
   
   so, can we can some option as `summary = false`, which won't do any new size calculation operation and clean the files faster ?? @akashrn5 , @QiangCai what you think ?
   




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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-788656644


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5103/
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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<String> 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<CarbonFile> 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<CarbonFile> 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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r577452054



##########
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:
       when nothing is freed by clean files, returning 0 is ok. But when some exception happens in dry run may be better to throw exception




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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-795439800


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3778/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 removed a comment on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-787906686


   @akashrn5 @VenuReddy2103 @ajantha-bhat , can this PR be merged?


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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-756129486


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3529/
   


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

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



[GitHub] [carbondata] vikramahuja1001 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-758446477


   It's not yet finished, i will change the title and add jira after completing the code. Please review after that


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-768075845


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5357/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r577387770



##########
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:
       Please remove `deleteFile(CarbonFile carbonFile)`, because there is no extra function is happening here. directly call carbonFile.deleteFile() in the usages.
   
   If there was some extra functionality, then it makes sense to have to APIfor carbonFile also to avoid getting CarbonFile again. for this case it is not needed 




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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-781562499


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5084/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-796673654


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3787/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-782051474


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5095/
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-782214759


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5496/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-780388512


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3713/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-783127326


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3735/
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [carbondata] vikramahuja1001 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-766581185


   retest this please


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-756130401


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5289/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-788599749


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3343/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-795102114


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3766/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-793536218


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5120/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-796760313


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3790/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-796679499


   retest this please


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-783110117


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5499/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r570107627



##########
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:
       already delete API is there, please use the same

##########
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<PartitionSpec> partitionSpecs,
       SegmentUpdateStatusManager updateStatusManager) throws Exception {
     SegmentFileStore fileStore = new SegmentFileStore(tablePath, segment.getSegmentFileName());
     List<String> indexOrMergeFiles = fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
         FileFactory.getConfiguration());
+    long sizeFreed = 0;
     Map<String, List<String>> indexFilesMap = fileStore.getIndexFilesMap();
     for (Map.Entry<String, List<String>> 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:
       i can see lot of same code of lines here, may be you can refactor to a method like `getSizeAndDelete()`

##########
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:
       .show is a waste call in FTs, please have a proper asserts for all

##########
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<PartitionSpec>
+      partitionSpecs) throws Exception {
+    long size = 0;
+    SegmentFileStore fileStore = new SegmentFileStore(carbonTable.getTablePath(), oneLoad
+        .getSegmentFile());
+    List<String> indexOrMergeFiles = fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
+        FileFactory.getConfiguration());
+    Map<String, List<String>> indexFilesMap = fileStore.getIndexFilesMap();

Review comment:
       can't we calculate size together for all segments?, because here everytime its calling readindex files, and calling File APIs to calculate sizes, so in case of OBS it will be very slow, better to make it optimized to calculate one time i feel

##########
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<PartitionSpec> partitionSpecs,
       SegmentUpdateStatusManager updateStatusManager) throws Exception {
     SegmentFileStore fileStore = new SegmentFileStore(tablePath, segment.getSegmentFileName());
     List<String> indexOrMergeFiles = fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
         FileFactory.getConfiguration());
+    long sizeFreed = 0;
     Map<String, List<String>> indexFilesMap = fileStore.getIndexFilesMap();
     for (Map.Entry<String, List<String>> entry : indexFilesMap.entrySet()) {
-      FileFactory.deleteFile(entry.getKey());
+      CarbonFile entryCarbonFile = FileFactory.getCarbonFile(entry.getKey());
+      sizeFreed += entryCarbonFile.getSize();

Review comment:
       `entryCarbonFile ` please give a  meaningful name like indexfile, datafile etc

##########
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:
       dry run is meant to give the size, do you need to give empty size or fail it? if you give empty its purpose itself not completed.

##########
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:
       why putting negation and getting confusion, just remove negation, swap if and else block code, looks simple right :)

##########
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:
       same as above, just my opinion, what you guys think @ajantha-bhat @QiangCai 

##########
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:
       better to give a more simple and meaningful name




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-758477317


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5299/
   


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-766563818


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3308/
   


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-766633326


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5346/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-793427100


   retest this please


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

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



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

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r589969086



##########
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:
       agree with @ajantha-bhat , change the titles accordingly and @ajantha-bhat for clean files just total cleaned size is fine right? anyways we have separated size in dry run, what do you think?




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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-782114580


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3337/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-781284062


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3730/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r580012643



##########
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:
       better to mention is it in MB/GB




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

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



[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

Posted by GitBox <gi...@apache.org>.
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<String> 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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-799185057


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3800/
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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<PartitionSpec> partitionSpecs,
       SegmentUpdateStatusManager updateStatusManager) throws Exception {
     SegmentFileStore fileStore = new SegmentFileStore(tablePath, segment.getSegmentFileName());
     List<String> indexOrMergeFiles = fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
         FileFactory.getConfiguration());
+    long sizeFreed = 0;
     Map<String, List<String>> indexFilesMap = fileStore.getIndexFilesMap();
     for (Map.Entry<String, List<String>> 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<PartitionSpec> partitionSpecs,
       SegmentUpdateStatusManager updateStatusManager) throws Exception {
     SegmentFileStore fileStore = new SegmentFileStore(tablePath, segment.getSegmentFileName());
     List<String> indexOrMergeFiles = fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
         FileFactory.getConfiguration());
+    long sizeFreed = 0;
     Map<String, List<String>> indexFilesMap = fileStore.getIndexFilesMap();
     for (Map.Entry<String, List<String>> 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:
users@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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-793734908


   >  i agree with @ajantha-bhat , may be use loadmetadata details data and index size only and for update files you can use some logic, please check
    
   @akashrn5 , the tablestatus file is used to check the data size and the index size of each segment and in the case of update/delete checking the getUpdateDeltaStartTimestamp and getUpdateDeltaEndTimestamp using segmentUpdateStatusManager to getDeleteDeltaFIlesList and getting their sizes


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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@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

Posted by GitBox <gi...@apache.org>.
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:
users@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

Posted by GitBox <gi...@apache.org>.
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<String> indexOrMergeFiles = fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
         FileFactory.getConfiguration());
     Map<String, List<String>> indexFilesMap = fileStore.getIndexFilesMap();
+    List<String> 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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-794010924


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5124/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-792688228


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5116/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-783296467


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5502/
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@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

Posted by GitBox <gi...@apache.org>.
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:
users@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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r577388524



##########
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:
       As per above comment, you can directly call segmentFilePath.delete()




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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-768933560


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5370/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-781286159


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5494/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
ydvpankaj99 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-795235448


   retest this please


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

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



[GitHub] [carbondata] asfgit closed pull request #4072: [CARBONDATA-4110] Support clean files dry run operation and show statistics after clean files operation

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072


   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-779698201


   Build Success with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3708/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
VenuReddy2103 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-783290929


   LGTM


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-766551160






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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-783297309


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3737/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r590016320



##########
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:
       same as above




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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-768075252


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3597/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-781958308


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5089/
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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<String, List<String>> indexFilesMap = fileStore.getIndexFilesMap();
     for (Map.Entry<String, List<String>> 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:
users@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

Posted by GitBox <gi...@apache.org>.
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:
users@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

Posted by GitBox <gi...@apache.org>.
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:
users@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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r580012643



##########
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:
       better to mention is it in MB/GB like
   Size Freed in MB : 512
   




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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-779300995


   @QiangCai @ajantha-bhat please review this


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-766609232


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5343/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r577500588



##########
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:
       why not sending this in options itself ?

##########
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:
       I see that size calculation code is duplicate in dryrun flow and in clean up flow, can we extract a common method and use it ?

##########
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:
       I see that size calculation code is duplicate in dryrun flow and in clean up flow, can we extract a common method and use it ?

##########
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:
       please verify (at least manully) that dry run size and without dry run, it shows the correct size estimation (if somewhere we are adding from two places, it might give incorrect results)
   
   Also verify the summary is same as actual size cleaned up with big table in backend for both partition and non-partition table.

##########
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:
       In the current clean files parser itself would have added the option of dry run (similar to force delete option), why adding a new syntax in the end ?




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

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



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

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-792624555


   retest this please


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-777597046


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3317/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r580086955



##########
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:
       @vikramahuja1001 : can you paste here one output of clean files and dry run now ? 




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-766551160


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3579/
   


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-758476848


   Build Failed  with Spark 2.4.5, Please check CI http://121.244.95.60:12444/job/ApacheCarbon_PR_Builder_2.4.5/3539/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-780823505


   Build Failed  with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5489/
   


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

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



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r577450283



##########
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:
       when nothing is freed by clean files, returning 0 is ok. But when some exception happens in dry run may be better to throw exception 




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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r577533482



##########
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:
       if else both blocks are same? i think better to give these rows only in case of dry run

##########
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:
       i think we are mixing the dry run option also along with forcedelete, and making this complex with code and combination handling, what i think is, when user say dry run, it should be clear that i dont take any other options and i just tell user in return how much and what i am going to clean, thats all, we will not delete or clear any files when dry run. So it will be easy to code and cleaner, may be new class or a new method in clean files command class. What you guys think @ajantha-bhat @QiangCai 




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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@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

Posted by GitBox <gi...@apache.org>.
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<PartitionSpec>
+      partitionSpecs) throws Exception {
+    long size = 0;
+    SegmentFileStore fileStore = new SegmentFileStore(carbonTable.getTablePath(), oneLoad
+        .getSegmentFile());
+    List<String> indexOrMergeFiles = fileStore.readIndexFiles(SegmentStatus.SUCCESS, true,
+        FileFactory.getConfiguration());
+    Map<String, List<String>> 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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-783213702


   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12444/job/ApacheCarbonPRBuilder2.3/5501/
   


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

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



[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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org



[GitHub] [carbondata] vikramahuja1001 commented on pull request #4072: [WIP] Clean files phase2

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#issuecomment-766562961






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

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



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r577458113



##########
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:
       I am thinking now all the clean file operations will become slow because of these size calculation code, which need to interact with the file system.
   
   Default we can have this size calculation. but if user wants clean files to be faster. Can we have some option as `summary = false`, which won't do any new size calculation operation and clean the files faster ?? @akashrn5 , @QiangCai what you think ?
   




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

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



[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

Posted by GitBox <gi...@apache.org>.
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<String> 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:
users@infra.apache.org



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r580011816



##########
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:
       *snapshot
   screenshot --> snapshot




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

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



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

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r580013092



##########
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:
       is it combining outside trash and inside trash ? 




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

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



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

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #4072:
URL: https://github.com/apache/carbondata/pull/4072#discussion_r577523404



##########
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:
       yes, agree with @ajantha-bhat , i meant 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:
users@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

Posted by GitBox <gi...@apache.org>.
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:
users@infra.apache.org