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

[GitHub] [carbondata] vikramahuja1001 opened a new pull request #4035: [WIP]: CleanFiles Behaviour Change

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


    ### Why is this PR needed?
    Change the behaviour change for clean files operation
   Old behaviour: Clean files command is by default force option and ignores query timeout.
    
    ### What changes were proposed in this PR?
   New Behaviour:
   
   default clean files behaviour(clean files on table t1): clean MFD and Compacted segments after query timeout(1 hr)
   clean files on table t1 options('force'='true'): clean MFD and Compacted segments immediately
   clean files on table t1 options('clean_inprgress'='true') : clean stale inprogress segments after 7 days(default behaviour)
   clean files on table t1 options('clean_inprgress'='true', 'force'='true') : clean MFD, Compacted and stale inprogress segments immediately.
       
    ### Does this PR introduce any user interface change?
    - Yes. (please explain the change and update document)
   
    ### Is any new testcase added?
    - No (previous test cases 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] ajantha-bhat commented on pull request #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


   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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +1039,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean

Review comment:
       @QiangCai @akashrn5 , can you clarify if we should delete the expired MFD, compacted in point 3?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +1039,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean

Review comment:
       i have put it back

##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -173,40 +176,68 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
+      boolean cleanStaleInProgress, boolean cleanCompactedAndMFD) {
+    if (oneLoad.getVisibility().equalsIgnoreCase("true")) {

Review comment:
       it's handeled in checkIfLoadCanBeDeletedPhysically method, it does not check visibility before deleting

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
##########
@@ -117,9 +119,9 @@ case class CarbonCleanFilesCommand(
         CleanFilesUtil.cleanStaleSegments(carbonTable)
       }
       if (forceTableClean) {

Review comment:
       it is removed in the PR4013 by David

##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +1039,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean

Review comment:
       updated as per the above discussion

##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +1039,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable 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] asfgit closed pull request #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [WIP]: CleanFiles Behaviour Change

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


   @vikramahuja1001 i dont see any test case with` clean_inprgress = true`, please add the missing test cases


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/events/CleanFilesPostEventListener.scala
##########
@@ -53,13 +51,15 @@ class CleanFilesPostEventListener extends OperationEventListener with Logging {
         val carbonTable = cleanFilesPostEvent.carbonTable
         val indexTables = CarbonIndexUtil
           .getIndexCarbonTables(carbonTable, cleanFilesPostEvent.sparkSession)
+        val force = cleanFilesPostEvent.ifForceDeletion

Review comment:
       ```suggestion
           val isForceDelete = cleanFilesPostEvent.ifForceDeletion
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFileCommand.scala
##########
@@ -346,4 +346,10 @@ class TestCleanFileCommand extends QueryTest with BeforeAndAfterAll {
     sql(s"""INSERT INTO CLEANTEST SELECT "abc", 1, "name"""")
   }
 
+  def removeSegmentEntryFromTableStatusFile(carbonTable: CarbonTable, segmentNo: String) : Unit = {

Review comment:
       @vikramahuja1001 : yes, instead of defining in each file, move it to some test util.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +1039,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean
+      cleanCompactedAndMFD, CarbonTable carbonTable, AbsoluteTableIdentifier
+      absoluteTableIdentifier, LoadMetadataDetails[] details) {
     // Delete marked loads
     boolean isUpdateRequired = DeleteLoadFolders
-        .deleteLoadFoldersFromFileSystem(absoluteTableIdentifier, isForceDeletion, details,
-            carbonTable.getMetadataPath());
+        .deleteLoadFoldersFromFileSystem(absoluteTableIdentifier, cleanStaleInProgress,
+            cleanCompactedAndMFD, details, carbonTable.getMetadataPath());
     return new ReturnTuple(details, isUpdateRequired);
   }
 
-  public static void deleteLoadsAndUpdateMetadata(CarbonTable carbonTable, boolean isForceDeletion,
-      List<PartitionSpec> partitionSpecs) throws IOException {
+  public static void deleteLoadsAndUpdateMetadata(CarbonTable carbonTable,
+      List<PartitionSpec> partitionSpecs, Boolean cleanStaleInprogress,
+                                                  Boolean cleanCompactedAndMFD) throws IOException {

Review comment:
       please correct the format here

##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +1039,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean
+      cleanCompactedAndMFD, CarbonTable carbonTable, AbsoluteTableIdentifier
+      absoluteTableIdentifier, LoadMetadataDetails[] details) {
     // Delete marked loads
     boolean isUpdateRequired = DeleteLoadFolders
-        .deleteLoadFoldersFromFileSystem(absoluteTableIdentifier, isForceDeletion, details,
-            carbonTable.getMetadataPath());
+        .deleteLoadFoldersFromFileSystem(absoluteTableIdentifier, cleanStaleInProgress,
+            cleanCompactedAndMFD, details, carbonTable.getMetadataPath());
     return new ReturnTuple(details, isUpdateRequired);
   }
 
-  public static void deleteLoadsAndUpdateMetadata(CarbonTable carbonTable, boolean isForceDeletion,
-      List<PartitionSpec> partitionSpecs) throws IOException {
+  public static void deleteLoadsAndUpdateMetadata(CarbonTable carbonTable,
+      List<PartitionSpec> partitionSpecs, Boolean cleanStaleInprogress,
+                                                  Boolean cleanCompactedAndMFD) throws IOException {

Review comment:
       wherever you are calling this methods, please name the Boolean variable

##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -173,40 +176,68 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
+      boolean cleanStaleInProgress, boolean cleanCompactedAndMFD) {
+    if (oneLoad.getVisibility().equalsIgnoreCase("true")) {

Review comment:
       here checking visibility to true, but, when i do delete segment by id or date, that time we will set the visibility to false, how its taken care that time? is it already handed?

##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1095,7 +1099,8 @@ public static void deleteLoadsAndUpdateMetadata(CarbonTable carbonTable, boolean
             // if execute command 'clean files' or the number of invisible segment info
             // exceeds the value of 'carbon.invisible.segments.preserve.count',
             // it need to append the invisible segment list to 'tablestatus.history' file.
-            if (isForceDeletion || (invisibleSegmentCnt > invisibleSegmentPreserveCnt)) {
+            if (cleanStaleInprogress || cleanCompactedAndMFD || (invisibleSegmentCnt >

Review comment:
       `cleanStaleInprogress || cleanCompactedAndMFD` is this condition really required here? as functionality is just moving to history file

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/events/CleanFilesEvents.scala
##########
@@ -32,7 +32,10 @@ case class CleanFilesPreEvent(carbonTable: CarbonTable, sparkSession: SparkSessi
 /**
  *
  * @param carbonTable
+ * @param cleanStaleInProgress
+ * @param cleanCompactedAndMFD

Review comment:
       ifnot adding any description remove these, already variable name depicts whats it for

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
##########
@@ -58,8 +58,10 @@ case class CarbonCleanFilesCommand(
   var carbonTable: CarbonTable = _
   var cleanFileCommands: List[CarbonCleanFilesCommand] = List.empty
   val optionsMap = options.getOrElse(List.empty[(String, String)]).toMap
-  // forceClean will empty trash
+  // forceClean will clean the MFD and Compacted segments immediately
   val forceClean = optionsMap.getOrElse("force", "false").toBoolean
+  // stale_inprogress willclean the In Progress segments immediately

Review comment:
       comment is wrong right? because when `stale_inprogress  = true`, you will not clean immediately, you will wait for retention time, when both are true, only then you will delete immediately, please update it accordingly

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
##########
@@ -571,7 +571,7 @@ object CommonUtil {
                 try {
                   val carbonTable = CarbonMetadata.getInstance
                     .getCarbonTable(tableUniqueName)
-                  SegmentStatusManager.deleteLoadsAndUpdateMetadata(carbonTable, true, null)
+                  SegmentStatusManager.deleteLoadsAndUpdateMetadata(carbonTable, null, true, false)

Review comment:
       please name boolean variables, as the first comment

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
##########
@@ -117,9 +119,9 @@ case class CarbonCleanFilesCommand(
         CleanFilesUtil.cleanStaleSegments(carbonTable)
       }
       if (forceTableClean) {

Review comment:
       can we remove this `forceTableClean`? i dont see anyone using variable or passing this as true, and its always false

##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +1039,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,

Review comment:
       there is method `writeLoadMetadata ` which take identifier as input, its unused method please check and remove that code

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
##########
@@ -125,7 +125,7 @@ case class CarbonLoadDataCommand(databaseNameOp: Option[String],
           updateModel = None,
           operationContext = operationContext)
       // Clean up the old invalid segment data before creating a new entry for new load.
-      SegmentStatusManager.deleteLoadsAndUpdateMetadata(table, false, currPartitions)
+      SegmentStatusManager.deleteLoadsAndUpdateMetadata(table, currPartitions, false, false)

Review comment:
       same as above

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/SecondaryIndexCreator.scala
##########
@@ -462,7 +462,7 @@ object SecondaryIndexCreator {
       try {
         if (!isCompactionCall) {
           SegmentStatusManager
-            .deleteLoadsAndUpdateMetadata(indexCarbonTable, false, null)
+            .deleteLoadsAndUpdateMetadata(indexCarbonTable, null, false, false)

Review comment:
       name boolean

##########
File path: integration/spark/src/main/scala/org/apache/spark/util/CleanFiles.scala
##########
@@ -39,7 +39,8 @@ object CleanFiles {
    *                        drop table from hive metastore so should be very careful to use it.
    */
   def cleanFiles(spark: SparkSession, dbName: String, tableName: String,
-     forceTableClean: Boolean = false): Unit = {
+     forceTableClean: Boolean = false, cleanCompactedAndMFD: Boolean = false,
+                 cleanStaleInProgress: Boolean = false ): Unit = {

Review comment:
       please reformat the code

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFileCommand.scala
##########
@@ -346,4 +346,10 @@ class TestCleanFileCommand extends QueryTest with BeforeAndAfterAll {
     sql(s"""INSERT INTO CLEANTEST SELECT "abc", 1, "name"""")
   }
 
+  def removeSegmentEntryFromTableStatusFile(carbonTable: CarbonTable, segmentNo: String) : Unit = {

Review comment:
       i see that this method is used in multiple places, please move to utility method and use everywhere

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFileCommand.scala
##########
@@ -212,14 +211,16 @@ class TestCleanFileCommand extends QueryTest with BeforeAndAfterAll {
 
   test("test trash folder with 2 segments with same segment number") {
     createTable()
-    sql(s"""INSERT INTO CLEANTEST SELECT "1", 2, "name"""")
+    loadData()
 
     val path = CarbonEnv.getCarbonTable(Some("default"), "cleantest")(sqlContext.sparkSession)
       .getTablePath
     val trashFolderPath = CarbonTablePath.getTrashFolderPath(path)
     assert(!FileFactory.isFileExist(trashFolderPath))
     // All 4  segments are made as stale segments, they should be moved to the trash folder
-    deleteTableStatusFile(path)
+    // deleteTableStatusFile(path)

Review comment:
       please check all places

##########
File path: core/src/main/java/org/apache/carbondata/core/util/CleanFilesUtil.java
##########
@@ -147,17 +147,28 @@ public static void cleanStaleSegmentsForPartitionTable(CarbonTable carbonTable)
    * stale segment. Only comparing from tablestatus file, not checking tablestatus.history file
    */
   private static void getStaleSegmentFiles(CarbonTable carbonTable, List<String> staleSegmentFiles,
-      List<String> redundantSegmentFile) {
+      List<String> redundantSegmentFile) throws IOException {
     String segmentFilesLocation =
         CarbonTablePath.getSegmentFilesLocation(carbonTable.getTablePath());
     List<String> segmentFiles = Arrays.stream(FileFactory.getCarbonFile(segmentFilesLocation)
         .listFiles()).map(CarbonFile::getName).collect(Collectors.toList());
     // there are no segments present in the Metadata folder. Can return here
-    if (segmentFiles.size() == 0) {
-      return;
+    // if table status file does not exist return
+    try {
+      if (segmentFiles.size() == 0 || !FileFactory.isFileExist(CarbonTablePath

Review comment:
       agree with @QiangCai , in the below line already it will take care

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonInsertIntoWithDf.scala
##########
@@ -108,7 +108,7 @@ case class CarbonInsertIntoWithDf(databaseNameOp: Option[String],
           operationContext = operationContext)
 
       // Clean up the old invalid segment data before creating a new entry for new load.
-      SegmentStatusManager.deleteLoadsAndUpdateMetadata(table, false, currPartitions)
+      SegmentStatusManager.deleteLoadsAndUpdateMetadata(table, currPartitions, false, false)

Review comment:
       name boolean

##########
File path: core/src/main/java/org/apache/carbondata/core/util/CleanFilesUtil.java
##########
@@ -147,17 +147,28 @@ public static void cleanStaleSegmentsForPartitionTable(CarbonTable carbonTable)
    * stale segment. Only comparing from tablestatus file, not checking tablestatus.history file
    */
   private static void getStaleSegmentFiles(CarbonTable carbonTable, List<String> staleSegmentFiles,
-      List<String> redundantSegmentFile) {
+      List<String> redundantSegmentFile) throws IOException {
     String segmentFilesLocation =
         CarbonTablePath.getSegmentFilesLocation(carbonTable.getTablePath());
     List<String> segmentFiles = Arrays.stream(FileFactory.getCarbonFile(segmentFilesLocation)
         .listFiles()).map(CarbonFile::getName).collect(Collectors.toList());
     // there are no segments present in the Metadata folder. Can return here
-    if (segmentFiles.size() == 0) {
-      return;
+    // if table status file does not exist return
+    try {
+      if (segmentFiles.size() == 0 || !FileFactory.isFileExist(CarbonTablePath
+          .getTableStatusFilePath(carbonTable.getTablePath()))) {
+        return;
+      }
+    } catch (IOException e) {
+      LOGGER.error("Unable to Check if tablestatus file exists while getting stale segments", e);
+      throw e;
     }
     LoadMetadataDetails[] details = SegmentStatusManager.readLoadMetadata(carbonTable
         .getMetadataPath());
+    // return if table status file is empty

Review comment:
       change the comment, there wont be any scenario where the table status file is empty

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFileCommand.scala
##########
@@ -90,25 +100,21 @@ class TestCleanFileCommand extends QueryTest with BeforeAndAfterAll {
     val trashFolderPath = CarbonTablePath.getTrashFolderPath(path)
     assert(!FileFactory.isFileExist(trashFolderPath))
     // All 4 segments are made as stale segments and should be moved to trash
-    deleteTableStatusFile(path)
+    // deleteTableStatusFile(path)

Review comment:
       if not required remove , instead of commenting

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFileCommand.scala
##########
@@ -151,34 +157,31 @@ class TestCleanFileCommand extends QueryTest with BeforeAndAfterAll {
 
     val mainTablePath = CarbonEnv.getCarbonTable(Some("default"), "cleantest")(sqlContext
       .sparkSession).getTablePath
-    deleteTableStatusFile(mainTablePath)
+    // deleteTableStatusFile(mainTablePath)

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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +1039,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean

Review comment:
       If we don't clean expired MFD, expired compacted and expired trash along with expired stale_inprogress. we have to call clean files at least two times to clean all expired data. I feel it is not good. 
   
   @QiangCai @akashrn5 : 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonCleanFilesCommand.scala
##########
@@ -58,8 +58,11 @@ case class CarbonCleanFilesCommand(
   var carbonTable: CarbonTable = _
   var cleanFileCommands: List[CarbonCleanFilesCommand] = List.empty
   val optionsMap = options.getOrElse(List.empty[(String, String)]).toMap
-  // forceClean will empty trash
+  // forceClean will clean the MFD and Compacted segments immediately and also empty the trash
+  // folder
   val forceClean = optionsMap.getOrElse("force", "false").toBoolean
+  // stale_inprogress will clean the In Progress segments

Review comment:
       update comment saying, it will clean based on retention time and it will clean immediately when force is true




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

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


   @vikramahuja1001 please change title to describe what you change


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -173,40 +176,66 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
+      boolean isForceDelete, boolean cleanStaleInProgress) {
+    if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+      return canDeleteThisLoad(oneLoad, isForceDelete, cleanStaleInProgress);
     }
-
     return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
+      boolean isForceDelete, boolean cleanStaleInProgress) {
     // Check if the segment is added externally and path is set then do not delete it
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-        || SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && (oneLoad.getPath() == null
-        || oneLoad.getPath().equalsIgnoreCase("NA"))) {
-      if (isForceDelete) {
+    if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) {
+      return canDeleteThisLoad(oneLoad,  isForceDelete, cleanStaleInProgress);
+    }
+    return false;
+  }
+
+  private static Boolean canDeleteThisLoad(LoadMetadataDetails oneLoad,
+      boolean isForceDelete, boolean cleanStaleInProgress) {
+    /*
+     * if cleanStaleInProgress == false and  force == false, clean MFD and Compacted
+     *  segments will depend on query timeout(1 hr) and trashRetentionTimeout(7 days, default).
+     *  For example:
+     *  If trashRetentionTimeout is 7 days and query timeout is 1 hr--> Delete after 7 days
+     *  If trashRetentionTimeout is 0 days and query timeout is 1 hr--> Delete after 1 hr
+     *
+     * if cleanStaleInProgress == false and  force == true, clean MFD and Compacted
+     *  segments immediately(Do not check for any timeout)
+     *
+     * if cleanStaleInProgress == true and  force == false, clean Stale Inprogress, MFD and
+     *  compacted segments after 7 days(taking carbon.trash.retention.time value)
+     *
+     * if cleanStaleInProgress == true and  force == true, clean MFD, Compacted and
+     *  stale inprogress segments immediately.(Do not check for any timeout)
+     */
+    if (isForceDelete) {
+      // immediately delete compacted and MFD
+      if (SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || SegmentStatus
+          .MARKED_FOR_DELETE == oneLoad.getSegmentStatus()) {
         return true;
       }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-
+      // immediately delete inprogress segments if cleanstaleinprogress is true
+      return cleanStaleInProgress && (SegmentStatus.INSERT_IN_PROGRESS == oneLoad
+          .getSegmentStatus() || SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad
+          .getSegmentStatus());
+    }
+    long deletionTime = oneLoad.getModificationOrDeletionTimestamp();

Review comment:
       The deletion time is used later also in line 238, can not combine 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -173,40 +176,66 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
+      boolean isForceDelete, boolean cleanStaleInProgress) {
+    if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+      return canDeleteThisLoad(oneLoad, isForceDelete, cleanStaleInProgress);
     }
-
     return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
+      boolean isForceDelete, boolean cleanStaleInProgress) {
     // Check if the segment is added externally and path is set then do not delete it
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-        || SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && (oneLoad.getPath() == null
-        || oneLoad.getPath().equalsIgnoreCase("NA"))) {
-      if (isForceDelete) {
+    if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) {
+      return canDeleteThisLoad(oneLoad,  isForceDelete, cleanStaleInProgress);
+    }
+    return false;
+  }
+
+  private static Boolean canDeleteThisLoad(LoadMetadataDetails oneLoad,
+      boolean isForceDelete, boolean cleanStaleInProgress) {
+    /*
+     * if cleanStaleInProgress == false and  force == false, clean MFD and Compacted
+     *  segments will depend on query timeout(1 hr) and trashRetentionTimeout(7 days, default).
+     *  For example:
+     *  If trashRetentionTimeout is 7 days and query timeout is 1 hr--> Delete after 7 days
+     *  If trashRetentionTimeout is 0 days and query timeout is 1 hr--> Delete after 1 hr
+     *
+     * if cleanStaleInProgress == false and  force == true, clean MFD and Compacted
+     *  segments immediately(Do not check for any timeout)
+     *
+     * if cleanStaleInProgress == true and  force == false, clean Stale Inprogress, MFD and
+     *  compacted segments after 7 days(taking carbon.trash.retention.time value)
+     *
+     * if cleanStaleInProgress == true and  force == true, clean MFD, Compacted and
+     *  stale inprogress segments immediately.(Do not check for any timeout)
+     */
+    if (isForceDelete) {
+      // immediately delete compacted and MFD
+      if (SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || SegmentStatus
+          .MARKED_FOR_DELETE == oneLoad.getSegmentStatus()) {
         return true;
       }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-
+      // immediately delete inprogress segments if cleanstaleinprogress is true
+      return cleanStaleInProgress && (SegmentStatus.INSERT_IN_PROGRESS == oneLoad
+          .getSegmentStatus() || SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad
+          .getSegmentStatus());
+    }
+    long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+    // in case there is no deletion or modification timestamp, take the load start time as
+    // deleteTime
+    if (deletionTime == 0) {
+      deletionTime = oneLoad.getLoadStartTime();
+    }
+    if (SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || SegmentStatus
+        .MARKED_FOR_DELETE == oneLoad.getSegmentStatus()) {
+      // delete MFD, compacted segments after checking trash timeout and query timeout
       return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
-
+        .isMaxQueryTimeoutExceeded(deletionTime);
     }
-
-    return false;
+    return (SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() || SegmentStatus

Review comment:
       CarbonCLeanFiles Command line 116




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +999,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean
+      isForceDeletion, CarbonTable carbonTable, AbsoluteTableIdentifier
+      absoluteTableIdentifier, LoadMetadataDetails[] details) {
     // Delete marked loads
     boolean isUpdateRequired = DeleteLoadFolders
-        .deleteLoadFoldersFromFileSystem(absoluteTableIdentifier, isForceDeletion, details,
-            carbonTable.getMetadataPath());
+        .deleteLoadFoldersFromFileSystem(absoluteTableIdentifier, cleanStaleInProgress,

Review comment:
       keep the original order of arguments (identifier, isForceDeletion, cleanStaleInProgress, deatils, path), to avoid sending wrong arguments values from callers




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: docs/clean-files.md
##########
@@ -37,10 +38,23 @@ Clean files command is used to remove the Compacted, Marked For Delete ,In Progr
    ``` 
   Once the timestamp subdirectory is expired as per the configured expiration day value, that subdirectory is deleted from the trash folder in the subsequent clean files command.
 
-### FORCE DELETE TRASH
-The force option with clean files command deletes all the files and folders from the trash folder.
+### FORCE OPTION
+The force option with clean files command deletes all the files and folders from the trash folder and delete the Marked for Delete and Compacted segments immediately.
 
   ```
   CLEAN FILES FOR TABLE TABLE_NAME options('force'='true')
   ```
-Since Clean Files operation with force option will delete data that can never be recovered, the force option by default is disabled. Clean files with force option is only allowed when the carbon property ```carbon.clean.file.force.allowed``` is set to true. The default value of this property is false.
\ No newline at end of file
+Since Clean Files operation with force option will delete data that can never be recovered, the force option by default is disabled. Clean files with force option is only allowed when the carbon property ```carbon.clean.file.force.allowed``` is set to true. The default value of this property is false.

Review comment:
       move this section, above force as they have to enable this before trying force




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1095,7 +1099,8 @@ public static void deleteLoadsAndUpdateMetadata(CarbonTable carbonTable, boolean
             // if execute command 'clean files' or the number of invisible segment info
             // exceeds the value of 'carbon.invisible.segments.preserve.count',
             // it need to append the invisible segment list to 'tablestatus.history' file.
-            if (isForceDeletion || (invisibleSegmentCnt > invisibleSegmentPreserveCnt)) {
+            if (cleanStaleInprogress || cleanCompactedAndMFD || (invisibleSegmentCnt >

Review comment:
       changes it to isCleanFilesOperation, if we do not keep this, we will change the behaviour of show history segments, as that file would never be updated, since previously clean files was always force option, it would always go in that if condition for clean files




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -173,42 +176,73 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
+      boolean cleanStaleInProgress, boolean isForceDeletion) {
+    if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+      return checkLoadDeletionLogic(oneLoad, isForceDeletion, cleanStaleInProgress);
     }
-
     return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
+      boolean isForceDeletion, boolean cleanStaleInProgress) {
     // Check if the segment is added externally and path is set then do not delete it
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-        || SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && (oneLoad.getPath() == null
-        || oneLoad.getPath().equalsIgnoreCase("NA"))) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
-
+    if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) {
+      return checkLoadDeletionLogic(oneLoad,  isForceDeletion, cleanStaleInProgress);
     }
-
     return false;
   }
 
+  private static Boolean checkLoadDeletionLogic(LoadMetadataDetails oneLoad,
+      boolean isForceDeletion, boolean cleanStaleInProgress) {
+    /*
+     * if cleanStaleInProgress == false and  force == false, clean MFD and Compacted
+     *  segments will depend on query timeout(1 hr) and trashRetentionTimeout(7 days, default).
+     *  For example:
+     *  If trashRetentionTimeout is 7 days and query timeout is 1 hr--> Delete after 7 days
+     *  If trashRetentionTimeout is 0 days and query timeout is 1 hr--> Delete after 1 hr
+     *
+     * if cleanStaleInProgress == false and  force == true, clean MFD and Compacted
+     *  segments immediately(Do not check for any timeout)
+     *
+     * if cleanStaleInProgress == true and  force == false, clean Stale Inprogress, MFD and
+     *  compacted segments after 7 days(taking carbon.trash.retention.time value)
+     *
+     * if cleanStaleInProgress == true and  force == true, clean MFD, Compacted and
+     *  stale inprogress segments immediately.(Do not check for any timeout)
+     */
+    if (isForceDeletion) {
+      // immediately delete compacted and MFD
+      if (cleanStaleInProgress) {
+        // immediately delete inprogress segments too
+        return SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() || SegmentStatus
+            .INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus() || SegmentStatus
+            .COMPACTED == oneLoad.getSegmentStatus() || SegmentStatus.MARKED_FOR_DELETE ==
+            oneLoad.getSegmentStatus();
+      }
+      return SegmentStatus.COMPACTED ==
+          oneLoad.getSegmentStatus() || SegmentStatus.MARKED_FOR_DELETE == oneLoad
+          .getSegmentStatus();
+    }
+    long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+    // in case there is no deletion or modification timestamp, take the load start time as
+    // deleteTime
+    if (deletionTime == 0) {
+      deletionTime = oneLoad.getLoadStartTime();
+    }
+    if (cleanStaleInProgress) {
+      // delete MFD, compacted and Inprogress segments after trash timeout
+      return (SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() || SegmentStatus

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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +1039,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean

Review comment:
       Instead of just a force option deleting all MFD, Compacted and stale Insert In Progress segments, dividing them into 2 parameters, cleanMFDandCompacted and cleanStaleInProgress. CleanMFDAndCompacted parameter will decide if MFD and Compacted segments can be deleted and cleanStaleInProgress will decide if stale InProgress segments can be deleted. After giving these 2 parameters, force can be removed. if the user wants to do force deletion, they can set both cleanMFDandCompacted and cleanStaleInProgress to true.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +1039,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean

Review comment:
       Instead of just a force option deleting all MFD, Compacted and stale Insert In Progress segments, dividing them into 2 parameters, cleanMFDandCompacted and cleanStaleInProgress. CleanMFDAndCompacted parameter will decide if MFD and Compacted segments can be deleted and cleanStaleInProgress will decide if stale InProgress segments can be deleted. After giving these 2 parameters, force can be removed. if the user wants to do force deletion, they can set both cleanMFDandCompacted and cleanStaleInProgress to true.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 a change in pull request #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1095,7 +1099,8 @@ public static void deleteLoadsAndUpdateMetadata(CarbonTable carbonTable, boolean
             // if execute command 'clean files' or the number of invisible segment info
             // exceeds the value of 'carbon.invisible.segments.preserve.count',
             // it need to append the invisible segment list to 'tablestatus.history' file.
-            if (isForceDeletion || (invisibleSegmentCnt > invisibleSegmentPreserveCnt)) {
+            if (cleanStaleInprogress || cleanCompactedAndMFD || (invisibleSegmentCnt >

Review comment:
       do clean files move load metadata to history always




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -173,40 +176,66 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
+      boolean isForceDelete, boolean cleanStaleInProgress) {
+    if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+      return canDeleteThisLoad(oneLoad, isForceDelete, cleanStaleInProgress);
     }
-
     return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
+      boolean isForceDelete, boolean cleanStaleInProgress) {
     // Check if the segment is added externally and path is set then do not delete it
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-        || SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && (oneLoad.getPath() == null
-        || oneLoad.getPath().equalsIgnoreCase("NA"))) {
-      if (isForceDelete) {
+    if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) {
+      return canDeleteThisLoad(oneLoad,  isForceDelete, cleanStaleInProgress);
+    }
+    return false;
+  }
+
+  private static Boolean canDeleteThisLoad(LoadMetadataDetails oneLoad,
+      boolean isForceDelete, boolean cleanStaleInProgress) {
+    /*
+     * if cleanStaleInProgress == false and  force == false, clean MFD and Compacted
+     *  segments will depend on query timeout(1 hr) and trashRetentionTimeout(7 days, default).
+     *  For example:
+     *  If trashRetentionTimeout is 7 days and query timeout is 1 hr--> Delete after 7 days
+     *  If trashRetentionTimeout is 0 days and query timeout is 1 hr--> Delete after 1 hr
+     *
+     * if cleanStaleInProgress == false and  force == true, clean MFD and Compacted
+     *  segments immediately(Do not check for any timeout)
+     *
+     * if cleanStaleInProgress == true and  force == false, clean Stale Inprogress, MFD and
+     *  compacted segments after 7 days(taking carbon.trash.retention.time value)
+     *
+     * if cleanStaleInProgress == true and  force == true, clean MFD, Compacted and
+     *  stale inprogress segments immediately.(Do not check for any timeout)
+     */
+    if (isForceDelete) {
+      // immediately delete compacted and MFD
+      if (SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || SegmentStatus
+          .MARKED_FOR_DELETE == oneLoad.getSegmentStatus()) {
         return true;
       }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-
+      // immediately delete inprogress segments if cleanstaleinprogress is true
+      return cleanStaleInProgress && (SegmentStatus.INSERT_IN_PROGRESS == oneLoad
+          .getSegmentStatus() || SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad
+          .getSegmentStatus());
+    }
+    long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+    // in case there is no deletion or modification timestamp, take the load start time as
+    // deleteTime
+    if (deletionTime == 0) {
+      deletionTime = oneLoad.getLoadStartTime();
+    }
+    if (SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || SegmentStatus
+        .MARKED_FOR_DELETE == oneLoad.getSegmentStatus()) {
+      // delete MFD, compacted segments after checking trash timeout and query timeout
       return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
-
+        .isMaxQueryTimeoutExceeded(deletionTime);
     }
-
-    return false;
+    return (SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() || SegmentStatus

Review comment:
       that is happening in another class(CarbonCleanFilesCommand).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 a change in pull request #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1095,7 +1099,8 @@ public static void deleteLoadsAndUpdateMetadata(CarbonTable carbonTable, boolean
             // if execute command 'clean files' or the number of invisible segment info
             // exceeds the value of 'carbon.invisible.segments.preserve.count',
             // it need to append the invisible segment list to 'tablestatus.history' file.
-            if (isForceDeletion || (invisibleSegmentCnt > invisibleSegmentPreserveCnt)) {
+            if (cleanStaleInprogress || cleanCompactedAndMFD || (invisibleSegmentCnt >

Review comment:
       no need move load metadata to history always

##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +1039,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean

Review comment:
       why remove isForceDeletion?

##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -173,40 +176,62 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
+      boolean cleanStaleInProgress, boolean cleanCompactedAndMFD) {
+    if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+      return checkLoadDeletionLogic(oneLoad, cleanCompactedAndMFD, cleanStaleInProgress);
     }
-
     return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
+      boolean cleanCompactedAndMFD, boolean cleanStaleInProgress) {
     // Check if the segment is added externally and path is set then do not delete it
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-        || SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && (oneLoad.getPath() == null
-        || oneLoad.getPath().equalsIgnoreCase("NA"))) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+    if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) {
+      return checkLoadDeletionLogic(oneLoad, cleanCompactedAndMFD, cleanStaleInProgress);
+    }
+    return false;
+  }
 
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
+  private static Boolean checkLoadDeletionLogic(LoadMetadataDetails oneLoad,
+      boolean cleanCompactedAndMFD, boolean cleanStaleInProgress) {
+    /*
+     * if cleanStaleInProgress == false and  cleanCompactedAndMFD == false, clean MFD and Compacted
+     *  segments after trashtimeout(7days) && query timeout(1 hr)
+     * if cleanStaleInProgress == false and  cleanCompactedAndMFD == true, clean MFD and Compacted
+     *  segments immediately
+     * if cleanStaleInProgress == true and  cleanCompactedAndMFD == false, clean Stale Inprogress
+     *  segments after 7 days(taking carbon.trash.retention.time value)
+     * if cleanStaleInProgress == true and  cleanCompactedAndMFD == true, clean MFD, Compacted and
+     *  stale inprogress segments immediately.
+     */
+    if (!cleanCompactedAndMFD && !cleanStaleInProgress) {
+      if (SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || SegmentStatus
+          .MARKED_FOR_DELETE == oneLoad.getSegmentStatus()) {
+        long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+        return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
           .isMaxQueryTimeoutExceeded(deletionTime);
-
+      }
+      return false;
+    } else if (cleanCompactedAndMFD && !cleanStaleInProgress) {
+      return SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || SegmentStatus
+        .MARKED_FOR_DELETE == oneLoad.getSegmentStatus();
+    } else if (!cleanCompactedAndMFD) {

Review comment:
       It is hard to understand each if condition.
   please improve the whole if code.

##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -173,40 +176,62 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
+      boolean cleanStaleInProgress, boolean cleanCompactedAndMFD) {
+    if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+      return checkLoadDeletionLogic(oneLoad, cleanCompactedAndMFD, cleanStaleInProgress);
     }
-
     return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
+      boolean cleanCompactedAndMFD, boolean cleanStaleInProgress) {
     // Check if the segment is added externally and path is set then do not delete it
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-        || SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && (oneLoad.getPath() == null
-        || oneLoad.getPath().equalsIgnoreCase("NA"))) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+    if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) {
+      return checkLoadDeletionLogic(oneLoad, cleanCompactedAndMFD, cleanStaleInProgress);
+    }
+    return false;
+  }
 
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
+  private static Boolean checkLoadDeletionLogic(LoadMetadataDetails oneLoad,
+      boolean cleanCompactedAndMFD, boolean cleanStaleInProgress) {
+    /*
+     * if cleanStaleInProgress == false and  cleanCompactedAndMFD == false, clean MFD and Compacted
+     *  segments after trashtimeout(7days) && query timeout(1 hr)
+     * if cleanStaleInProgress == false and  cleanCompactedAndMFD == true, clean MFD and Compacted
+     *  segments immediately
+     * if cleanStaleInProgress == true and  cleanCompactedAndMFD == false, clean Stale Inprogress
+     *  segments after 7 days(taking carbon.trash.retention.time value)
+     * if cleanStaleInProgress == true and  cleanCompactedAndMFD == true, clean MFD, Compacted and
+     *  stale inprogress segments immediately.
+     */
+    if (!cleanCompactedAndMFD && !cleanStaleInProgress) {
+      if (SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || SegmentStatus
+          .MARKED_FOR_DELETE == oneLoad.getSegmentStatus()) {
+        long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+        return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
           .isMaxQueryTimeoutExceeded(deletionTime);
-
+      }
+      return false;
+    } else if (cleanCompactedAndMFD && !cleanStaleInProgress) {

Review comment:
       cleanCompactedAndMFD means forceDelete ?

##########
File path: core/src/main/java/org/apache/carbondata/core/util/CleanFilesUtil.java
##########
@@ -147,17 +147,28 @@ public static void cleanStaleSegmentsForPartitionTable(CarbonTable carbonTable)
    * stale segment. Only comparing from tablestatus file, not checking tablestatus.history file
    */
   private static void getStaleSegmentFiles(CarbonTable carbonTable, List<String> staleSegmentFiles,
-      List<String> redundantSegmentFile) {
+      List<String> redundantSegmentFile) throws IOException {
     String segmentFilesLocation =
         CarbonTablePath.getSegmentFilesLocation(carbonTable.getTablePath());
     List<String> segmentFiles = Arrays.stream(FileFactory.getCarbonFile(segmentFilesLocation)
         .listFiles()).map(CarbonFile::getName).collect(Collectors.toList());
     // there are no segments present in the Metadata folder. Can return here
-    if (segmentFiles.size() == 0) {
-      return;
+    // if table status file does not exist return
+    try {
+      if (segmentFiles.size() == 0 || !FileFactory.isFileExist(CarbonTablePath

Review comment:
       maybe not require to check tablestatus




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -90,13 +93,13 @@ public static void physicalFactAndMeasureMetadataDeletion(CarbonTable carbonTabl
    * Delete the invalid data physically from table.
    * @param carbonTable table
    * @param loadDetails Load details which need clean up
-   * @param isForceDelete is Force delete requested by user
+   * @param force Force delete Compacted and MFD segments

Review comment:
       force --> isForceDelete
   Also it will force clean the trash. add that also in header description 

##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -68,19 +68,22 @@ private static String getSegmentPath(AbsoluteTableIdentifier identifier,
 
   public static void physicalFactAndMeasureMetadataDeletion(CarbonTable carbonTable,
       LoadMetadataDetails[] newAddedLoadHistoryList,
-      boolean isForceDelete,
+      boolean isForceDeletion,

Review comment:
       keep it same as previous `isForceDelete`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -173,42 +176,73 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
+      boolean cleanStaleInProgress, boolean isForceDeletion) {
+    if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+      return checkLoadDeletionLogic(oneLoad, isForceDeletion, cleanStaleInProgress);
     }
-
     return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
+      boolean isForceDeletion, boolean cleanStaleInProgress) {
     // Check if the segment is added externally and path is set then do not delete it
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-        || SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && (oneLoad.getPath() == null
-        || oneLoad.getPath().equalsIgnoreCase("NA"))) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
-
+    if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) {
+      return checkLoadDeletionLogic(oneLoad,  isForceDeletion, cleanStaleInProgress);

Review comment:
       ```suggestion
         return CanDeleteThisLoad(oneLoad,  isForceDeletion, cleanStaleInProgress);
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -173,40 +176,66 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
+      boolean isForceDelete, boolean cleanStaleInProgress) {
+    if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+      return canDeleteThisLoad(oneLoad, isForceDelete, cleanStaleInProgress);
     }
-
     return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
+      boolean isForceDelete, boolean cleanStaleInProgress) {
     // Check if the segment is added externally and path is set then do not delete it
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-        || SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && (oneLoad.getPath() == null
-        || oneLoad.getPath().equalsIgnoreCase("NA"))) {
-      if (isForceDelete) {
+    if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) {
+      return canDeleteThisLoad(oneLoad,  isForceDelete, cleanStaleInProgress);
+    }
+    return false;
+  }
+
+  private static Boolean canDeleteThisLoad(LoadMetadataDetails oneLoad,
+      boolean isForceDelete, boolean cleanStaleInProgress) {
+    /*
+     * if cleanStaleInProgress == false and  force == false, clean MFD and Compacted

Review comment:
       ```suggestion
        * if cleanStaleInProgress == false and  isForceDelete== false, clean MFD and Compacted
   ```
   
   please handle in other comments

##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -173,40 +176,66 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
+      boolean isForceDelete, boolean cleanStaleInProgress) {
+    if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+      return canDeleteThisLoad(oneLoad, isForceDelete, cleanStaleInProgress);
     }
-
     return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
+      boolean isForceDelete, boolean cleanStaleInProgress) {
     // Check if the segment is added externally and path is set then do not delete it
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-        || SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && (oneLoad.getPath() == null
-        || oneLoad.getPath().equalsIgnoreCase("NA"))) {
-      if (isForceDelete) {
+    if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) {
+      return canDeleteThisLoad(oneLoad,  isForceDelete, cleanStaleInProgress);
+    }
+    return false;
+  }
+
+  private static Boolean canDeleteThisLoad(LoadMetadataDetails oneLoad,
+      boolean isForceDelete, boolean cleanStaleInProgress) {
+    /*
+     * if cleanStaleInProgress == false and  force == false, clean MFD and Compacted
+     *  segments will depend on query timeout(1 hr) and trashRetentionTimeout(7 days, default).
+     *  For example:
+     *  If trashRetentionTimeout is 7 days and query timeout is 1 hr--> Delete after 7 days
+     *  If trashRetentionTimeout is 0 days and query timeout is 1 hr--> Delete after 1 hr
+     *
+     * if cleanStaleInProgress == false and  force == true, clean MFD and Compacted
+     *  segments immediately(Do not check for any timeout)
+     *
+     * if cleanStaleInProgress == true and  force == false, clean Stale Inprogress, MFD and
+     *  compacted segments after 7 days(taking carbon.trash.retention.time value)
+     *
+     * if cleanStaleInProgress == true and  force == true, clean MFD, Compacted and
+     *  stale inprogress segments immediately.(Do not check for any timeout)
+     */
+    if (isForceDelete) {
+      // immediately delete compacted and MFD
+      if (SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || SegmentStatus
+          .MARKED_FOR_DELETE == oneLoad.getSegmentStatus()) {
         return true;
       }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-
+      // immediately delete inprogress segments if cleanstaleinprogress is true
+      return cleanStaleInProgress && (SegmentStatus.INSERT_IN_PROGRESS == oneLoad
+          .getSegmentStatus() || SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad
+          .getSegmentStatus());
+    }
+    long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+    // in case there is no deletion or modification timestamp, take the load start time as
+    // deleteTime
+    if (deletionTime == 0) {
+      deletionTime = oneLoad.getLoadStartTime();
+    }
+    if (SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || SegmentStatus
+        .MARKED_FOR_DELETE == oneLoad.getSegmentStatus()) {
+      // delete MFD, compacted segments after checking trash timeout and query timeout
       return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
-
+        .isMaxQueryTimeoutExceeded(deletionTime);
     }
-
-    return false;
+    return (SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() || SegmentStatus

Review comment:
       where is the handling of the discussion, like, when any clean files called, we should check the trash folder and based on timeout we should delete it?

##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -173,40 +176,66 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
+      boolean isForceDelete, boolean cleanStaleInProgress) {
+    if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+      return canDeleteThisLoad(oneLoad, isForceDelete, cleanStaleInProgress);
     }
-
     return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
+      boolean isForceDelete, boolean cleanStaleInProgress) {
     // Check if the segment is added externally and path is set then do not delete it
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-        || SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && (oneLoad.getPath() == null
-        || oneLoad.getPath().equalsIgnoreCase("NA"))) {
-      if (isForceDelete) {
+    if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) {
+      return canDeleteThisLoad(oneLoad,  isForceDelete, cleanStaleInProgress);
+    }
+    return false;
+  }
+
+  private static Boolean canDeleteThisLoad(LoadMetadataDetails oneLoad,
+      boolean isForceDelete, boolean cleanStaleInProgress) {
+    /*
+     * if cleanStaleInProgress == false and  force == false, clean MFD and Compacted
+     *  segments will depend on query timeout(1 hr) and trashRetentionTimeout(7 days, default).
+     *  For example:
+     *  If trashRetentionTimeout is 7 days and query timeout is 1 hr--> Delete after 7 days
+     *  If trashRetentionTimeout is 0 days and query timeout is 1 hr--> Delete after 1 hr
+     *
+     * if cleanStaleInProgress == false and  force == true, clean MFD and Compacted
+     *  segments immediately(Do not check for any timeout)
+     *
+     * if cleanStaleInProgress == true and  force == false, clean Stale Inprogress, MFD and
+     *  compacted segments after 7 days(taking carbon.trash.retention.time value)
+     *
+     * if cleanStaleInProgress == true and  force == true, clean MFD, Compacted and
+     *  stale inprogress segments immediately.(Do not check for any timeout)
+     */
+    if (isForceDelete) {
+      // immediately delete compacted and MFD
+      if (SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || SegmentStatus
+          .MARKED_FOR_DELETE == oneLoad.getSegmentStatus()) {
         return true;
       }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-
+      // immediately delete inprogress segments if cleanstaleinprogress is true
+      return cleanStaleInProgress && (SegmentStatus.INSERT_IN_PROGRESS == oneLoad
+          .getSegmentStatus() || SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad
+          .getSegmentStatus());
+    }
+    long deletionTime = oneLoad.getModificationOrDeletionTimestamp();

Review comment:
       you can combine from 224 to 229 to conditional statement




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +1039,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean
+      cleanCompactedAndMFD, CarbonTable carbonTable, AbsoluteTableIdentifier
+      absoluteTableIdentifier, LoadMetadataDetails[] details) {
     // Delete marked loads
     boolean isUpdateRequired = DeleteLoadFolders
-        .deleteLoadFoldersFromFileSystem(absoluteTableIdentifier, isForceDeletion, details,
-            carbonTable.getMetadataPath());
+        .deleteLoadFoldersFromFileSystem(absoluteTableIdentifier, cleanStaleInProgress,
+            cleanCompactedAndMFD, details, carbonTable.getMetadataPath());

Review comment:
       clean `cleanCompactedAndMFD` flag is no need as everytime we need to clean it, after you fix your point 3




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


   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 pull request #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


   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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: docs/clean-files.md
##########
@@ -24,6 +24,7 @@ Clean files command is used to remove the Compacted, Marked For Delete ,In Progr
    ```
    CLEAN FILES FOR TABLE TABLE_NAME
    ```
+The above clean files command will clean Marked For Delete and Compacted segments depending on ```MAX_QUERY_EXECUTION_TIME``` (default 1 hr) and ``` carbon.trash.retention.days``` (default 7 days). It will also delete the timestamp subdirectories from the trash folder after expiration day(default 7 day, can be configured)

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 #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

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



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -173,40 +176,62 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
+      boolean cleanStaleInProgress, boolean cleanCompactedAndMFD) {
+    if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+      return checkLoadDeletionLogic(oneLoad, cleanCompactedAndMFD, cleanStaleInProgress);
     }
-
     return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
+      boolean cleanCompactedAndMFD, boolean cleanStaleInProgress) {
     // Check if the segment is added externally and path is set then do not delete it
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-        || SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && (oneLoad.getPath() == null
-        || oneLoad.getPath().equalsIgnoreCase("NA"))) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+    if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) {
+      return checkLoadDeletionLogic(oneLoad, cleanCompactedAndMFD, cleanStaleInProgress);
+    }
+    return false;
+  }
 
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
+  private static Boolean checkLoadDeletionLogic(LoadMetadataDetails oneLoad,
+      boolean cleanCompactedAndMFD, boolean cleanStaleInProgress) {
+    /*
+     * if cleanStaleInProgress == false and  cleanCompactedAndMFD == false, clean MFD and Compacted
+     *  segments after trashtimeout(7days) && query timeout(1 hr)
+     * if cleanStaleInProgress == false and  cleanCompactedAndMFD == true, clean MFD and Compacted
+     *  segments immediately
+     * if cleanStaleInProgress == true and  cleanCompactedAndMFD == false, clean Stale Inprogress
+     *  segments after 7 days(taking carbon.trash.retention.time value)
+     * if cleanStaleInProgress == true and  cleanCompactedAndMFD == true, clean MFD, Compacted and
+     *  stale inprogress segments immediately.
+     */
+    if (!cleanCompactedAndMFD && !cleanStaleInProgress) {
+      if (SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || SegmentStatus
+          .MARKED_FOR_DELETE == oneLoad.getSegmentStatus()) {
+        long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+        return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
           .isMaxQueryTimeoutExceeded(deletionTime);
-
+      }
+      return false;
+    } else if (cleanCompactedAndMFD && !cleanStaleInProgress) {

Review comment:
       previously forceDelete would delete MFD, Compacted and stale INprogress segments immediately. cleanCompactedAndMFD means only delete MFD, Compacted(immediately)  and cleanStaleInProgress means only delete staleInprogress segments.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +1039,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean

Review comment:
       agree with @ajantha-bhat , the clean files with options, let it do its specific functionality, along with this, if others expired we should do that also, to save time for users.It should be like (1+2), (1+3) and (1+4). I think this will be 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


   LGTM. merging 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 #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +1039,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean

Review comment:
       1. Trash behaviour remains unchanged. Trash will be deleted automatically when option force = true, it's already in the document, If force = true is not given, then it will delete trash according to timestamp retention property.
   2. in_progress = true, should only delete in_progress segments. Expired trash will be deleted anyways. 
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [WIP]: CleanFiles Behaviour Change

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 a change in pull request #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1095,7 +1099,8 @@ public static void deleteLoadsAndUpdateMetadata(CarbonTable carbonTable, boolean
             // if execute command 'clean files' or the number of invisible segment info
             // exceeds the value of 'carbon.invisible.segments.preserve.count',
             // it need to append the invisible segment list to 'tablestatus.history' file.
-            if (isForceDeletion || (invisibleSegmentCnt > invisibleSegmentPreserveCnt)) {
+            if (cleanStaleInprogress || cleanCompactedAndMFD || (invisibleSegmentCnt >

Review comment:
       does clean files move load metadata to history always




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -173,42 +176,73 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
+      boolean cleanStaleInProgress, boolean isForceDeletion) {
+    if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+      return checkLoadDeletionLogic(oneLoad, isForceDeletion, cleanStaleInProgress);
     }
-
     return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
+      boolean isForceDeletion, boolean cleanStaleInProgress) {
     // Check if the segment is added externally and path is set then do not delete it
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-        || SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && (oneLoad.getPath() == null
-        || oneLoad.getPath().equalsIgnoreCase("NA"))) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
-
+    if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) {
+      return checkLoadDeletionLogic(oneLoad,  isForceDeletion, cleanStaleInProgress);
     }
-
     return false;
   }
 
+  private static Boolean checkLoadDeletionLogic(LoadMetadataDetails oneLoad,
+      boolean isForceDeletion, boolean cleanStaleInProgress) {
+    /*
+     * if cleanStaleInProgress == false and  force == false, clean MFD and Compacted
+     *  segments will depend on query timeout(1 hr) and trashRetentionTimeout(7 days, default).
+     *  For example:
+     *  If trashRetentionTimeout is 7 days and query timeout is 1 hr--> Delete after 7 days
+     *  If trashRetentionTimeout is 0 days and query timeout is 1 hr--> Delete after 1 hr
+     *
+     * if cleanStaleInProgress == false and  force == true, clean MFD and Compacted
+     *  segments immediately(Do not check for any timeout)
+     *
+     * if cleanStaleInProgress == true and  force == false, clean Stale Inprogress, MFD and
+     *  compacted segments after 7 days(taking carbon.trash.retention.time value)
+     *
+     * if cleanStaleInProgress == true and  force == true, clean MFD, Compacted and
+     *  stale inprogress segments immediately.(Do not check for any timeout)
+     */
+    if (isForceDeletion) {
+      // immediately delete compacted and MFD
+      if (cleanStaleInProgress) {
+        // immediately delete inprogress segments too
+        return SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() || SegmentStatus
+            .INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus() || SegmentStatus
+            .COMPACTED == oneLoad.getSegmentStatus() || SegmentStatus.MARKED_FOR_DELETE ==

Review comment:
       move COMPACTED and MARKED_FOR_DELETE above and return if true.
   check only INSERT_OVERWRITE_IN_PROGRESS, INSERT_IN_PROGRESS in INSERT_IN_PROGRESS




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: docs/clean-files.md
##########
@@ -24,6 +24,7 @@ Clean files command is used to remove the Compacted, Marked For Delete ,In Progr
    ```
    CLEAN FILES FOR TABLE TABLE_NAME
    ```
+The above clean files command will clean Marked For Delete and Compacted segments depending on ```MAX_QUERY_EXECUTION_TIME``` (default 1 hr) and ``` carbon.trash.retention.days``` (default 7 days). It will also delete the timestamp subdirectories from the trash folder after expiration day(default 7 day, can be configured)

Review comment:
       ```suggestion
   The above clean files command will clean Marked For Delete and Compacted segments depending on ```max.query.execution.time``` (default 1 hr) and ``` carbon.trash.retention.days``` (default 7 days). It will also delete the timestamp subdirectories from the trash folder after expiration day(default 7 day, can be configured)
   ```

##########
File path: integration/spark/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
##########
@@ -117,7 +117,7 @@ class CarbonCommandSuite extends QueryTest with BeforeAndAfterAll {
     val table = "carbon_table3"
     createAndLoadTestTable(table, "csv_table")
     DeleteSegmentById.main(Array(s"${location}", table, "0"))
-    CleanFiles.main(Array(s"${location}", table))
+    CleanFiles.main(Array(s"${location}", table, "false", "true", "true"))

Review comment:
       How it is passing without setting force allow carbon property 

##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +999,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, boolean

Review comment:
       please keep new argument 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] ajantha-bhat commented on a change in pull request #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +1039,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean

Review comment:
       in point 3), If we don't clean expired MFD, expired compacted and expired trash along with expired stale_inprogress. we have to call clean files at least two times to clean all expired data. I feel it is not good. 
   
   @QiangCai @akashrn5 : 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: integration/spark/src/test/scala/org/apache/spark/util/CarbonCommandSuite.scala
##########
@@ -117,7 +117,7 @@ class CarbonCommandSuite extends QueryTest with BeforeAndAfterAll {
     val table = "carbon_table3"
     createAndLoadTestTable(table, "csv_table")
     DeleteSegmentById.main(Array(s"${location}", table, "0"))
-    CleanFiles.main(Array(s"${location}", table))
+    CleanFiles.main(Array(s"${location}", table, "false", "true", "true"))

Review comment:
       added code to fail in the cleanFiles API also

##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +999,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, boolean

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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -173,40 +176,66 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
+      boolean isForceDelete, boolean cleanStaleInProgress) {
+    if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+      return canDeleteThisLoad(oneLoad, isForceDelete, cleanStaleInProgress);
     }
-
     return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
+      boolean isForceDelete, boolean cleanStaleInProgress) {
     // Check if the segment is added externally and path is set then do not delete it
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-        || SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && (oneLoad.getPath() == null
-        || oneLoad.getPath().equalsIgnoreCase("NA"))) {
-      if (isForceDelete) {
+    if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) {
+      return canDeleteThisLoad(oneLoad,  isForceDelete, cleanStaleInProgress);
+    }
+    return false;
+  }
+
+  private static Boolean canDeleteThisLoad(LoadMetadataDetails oneLoad,
+      boolean isForceDelete, boolean cleanStaleInProgress) {
+    /*
+     * if cleanStaleInProgress == false and  force == false, clean MFD and Compacted

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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +999,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean
+      isForceDeletion, CarbonTable carbonTable, AbsoluteTableIdentifier
+      absoluteTableIdentifier, LoadMetadataDetails[] details) {
     // Delete marked loads
     boolean isUpdateRequired = DeleteLoadFolders
-        .deleteLoadFoldersFromFileSystem(absoluteTableIdentifier, isForceDeletion, details,
-            carbonTable.getMetadataPath());
+        .deleteLoadFoldersFromFileSystem(absoluteTableIdentifier, cleanStaleInProgress,

Review comment:
       check for all the places and handle




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: docs/clean-files.md
##########
@@ -24,6 +24,7 @@ Clean files command is used to remove the Compacted, Marked For Delete ,In Progr
    ```
    CLEAN FILES FOR TABLE TABLE_NAME
    ```
+The above clean files command will clean Marked For Delete and Compacted segments depending on query time (default 1 hr) and trash retention timeout (default 7 days). It will also delete the timestamp subdirectories from the trash folder after expiration day(default 7 day, can be configured)

Review comment:
       query time,  trash retention timeout  => use the original carbon property names 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

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



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##########
@@ -173,40 +176,62 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-        SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-        SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == oneLoad.getSegmentStatus())
-        && oneLoad.getVisibility().equalsIgnoreCase("true")) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
-          .isMaxQueryTimeoutExceeded(deletionTime);
+      boolean cleanStaleInProgress, boolean cleanCompactedAndMFD) {
+    if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+      return checkLoadDeletionLogic(oneLoad, cleanCompactedAndMFD, cleanStaleInProgress);
     }
-
     return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails oneLoad,
-      boolean isForceDelete) {
+      boolean cleanCompactedAndMFD, boolean cleanStaleInProgress) {
     // Check if the segment is added externally and path is set then do not delete it
-    if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-        || SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && (oneLoad.getPath() == null
-        || oneLoad.getPath().equalsIgnoreCase("NA"))) {
-      if (isForceDelete) {
-        return true;
-      }
-      long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+    if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) {
+      return checkLoadDeletionLogic(oneLoad, cleanCompactedAndMFD, cleanStaleInProgress);
+    }
+    return false;
+  }
 
-      return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
+  private static Boolean checkLoadDeletionLogic(LoadMetadataDetails oneLoad,
+      boolean cleanCompactedAndMFD, boolean cleanStaleInProgress) {
+    /*
+     * if cleanStaleInProgress == false and  cleanCompactedAndMFD == false, clean MFD and Compacted
+     *  segments after trashtimeout(7days) && query timeout(1 hr)
+     * if cleanStaleInProgress == false and  cleanCompactedAndMFD == true, clean MFD and Compacted
+     *  segments immediately
+     * if cleanStaleInProgress == true and  cleanCompactedAndMFD == false, clean Stale Inprogress
+     *  segments after 7 days(taking carbon.trash.retention.time value)
+     * if cleanStaleInProgress == true and  cleanCompactedAndMFD == true, clean MFD, Compacted and
+     *  stale inprogress segments immediately.
+     */
+    if (!cleanCompactedAndMFD && !cleanStaleInProgress) {
+      if (SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || SegmentStatus
+          .MARKED_FOR_DELETE == oneLoad.getSegmentStatus()) {
+        long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+        return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && CarbonUpdateUtil
           .isMaxQueryTimeoutExceeded(deletionTime);
-
+      }
+      return false;
+    } else if (cleanCompactedAndMFD && !cleanStaleInProgress) {
+      return SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || SegmentStatus
+        .MARKED_FOR_DELETE == oneLoad.getSegmentStatus();
+    } else if (!cleanCompactedAndMFD) {

Review comment:
       I have added comments in the code to tell what exactly is happening and also added in the PR description. Please refer it once




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub 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 #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +999,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean
+      isForceDeletion, CarbonTable carbonTable, AbsoluteTableIdentifier
+      absoluteTableIdentifier, LoadMetadataDetails[] details) {
     // Delete marked loads
     boolean isUpdateRequired = DeleteLoadFolders
-        .deleteLoadFoldersFromFileSystem(absoluteTableIdentifier, isForceDeletion, details,
-            carbonTable.getMetadataPath());
+        .deleteLoadFoldersFromFileSystem(absoluteTableIdentifier, cleanStaleInProgress,

Review comment:
       it is always advisable to add a new argument at 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] ajantha-bhat commented on a change in pull request #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress segments can be deleted

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



##########
File path: core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##########
@@ -1039,17 +1039,19 @@ private static void writeLoadMetadata(AbsoluteTableIdentifier identifier,
     }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, CarbonTable carbonTable,
-      AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, boolean

Review comment:
       @vikramahuja1001 : please update the document of clean files.
   
   > 1 ) default clean files behaviour(clean files on table t1): clean MFD and Compacted segments will depend on query timeout(1 hr) and trashRetentionTimeout(7 days, default). 
   2) clean files on table t1 options('force'='true'): clean MFD and Compacted segments immediately(Do not check for any timeout)
   3) clean files on table t1 options('clean_inprgress'='true') : clean stale inprogress segments depends on trashRetentionTimeout, after 7 days(default behaviour)
   4) clean files on table t1 options('clean_inprgress'='true', 'force'='true') : clean MFD, Compacted and stale inprogress segments immediately.(Do not check for any timeout)
   
   **my comments for description** 
   1) you have not mentioned about trash in any of this. mention what happens to trash in each. 
   2)  And in your point 3) we should delete the expired MFD, compacted and expired trash folders also.




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

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