[GitHub] [carbondata] QiangCai opened a new pull request #4013: [WIP] Remove automatically clean data

classic Classic list List threaded Threaded
64 messages Options
1234
Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox

QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534983455



##########
File path: core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
##########
@@ -482,176 +482,6 @@ public boolean accept(CarbonFile file) {
 
   }
 
-  /**
-   * Handling of the clean up of old carbondata files, index files , delete delta,
-   * update status files.
-   * @param table clean up will be handled on this table.
-   * @param forceDelete if true then max query execution timeout will not be considered.
-   */
-  public static void cleanUpDeltaFiles(CarbonTable table, boolean forceDelete) throws IOException {

Review comment:
       only keep cleanStaleDeltaFiles to handle exceptions.
   
   clean files should work with the concurrent update, so remove it.




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r535212465



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/events/CleanFilesEvents.scala
##########
@@ -34,5 +34,6 @@ case class CleanFilesPreEvent(carbonTable: CarbonTable, sparkSession: SparkSessi
  * @param carbonTable
  * @param sparkSession
  */
-case class CleanFilesPostEvent(carbonTable: CarbonTable, sparkSession: SparkSession)
-  extends Event with CleanFilesEventInfo
+case class CleanFilesPostEvent(carbonTable: CarbonTable,

Review comment:
       ok




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r535216366



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
##########
@@ -90,7 +90,6 @@ class CarbonTableCompactor(carbonLoadModel: CarbonLoadModel,
 
     while (loadsToMerge.size() > 1 || needSortSingleSegment(loadsToMerge)) {
       val lastSegment = sortedSegments.get(sortedSegments.size() - 1)
-      deletePartialLoadsInCompaction()

Review comment:
       In this case, clean files have no chance to clean data.
   keep the stale data in the folder, it will not impact compaction.




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r535218031



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/DropIndexCommand.scala
##########
@@ -210,12 +210,6 @@ private[sql] case class DropIndexCommand(
           logError("Table metadata unlocking is unsuccessful, index table may be in stale state")
         }
       }
-      // in case if the the physical folders still exists for the index table
-      // but the carbon and hive info for the index table is removed,
-      // DROP INDEX IF EXISTS should clean up those physical directories
-      if (ifExistsSet && carbonTable.isEmpty) {

Review comment:
       create table t1...
   create table t2...
   drop index t2 on t1
   
   this drop index sql will remove table folder of t2 by mistake.

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##########
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.trash
+
+import scala.collection.JavaConverters._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.indexstore.PartitionSpec
+import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.SegmentFileStore
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
+import org.apache.carbondata.core.util.{CarbonProperties, CleanFilesUtil, TrashUtil}
+
+/**
+ * This object will manage the following data.
+ * 1. .Trash folder
+ * 2. stale segments without metadata
+ * 3. expired segments (MARKED_FOR_DELETE, Compacted, In Progress)
+ */
+object DataTrashManager {
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  /**
+   * clean garbage data
+   *  1. clean .Trash folder
+   *  2. clean stale segments without metadata
+   *  3. clean expired segments (MARKED_FOR_DELETE, Compacted, In Progress)
+   *
+   * @param carbonTable     : CarbonTable Object
+   * @param partitionSpecs : Hive Partitions  details

Review comment:
       ok




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r535224083



##########
File path: core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
##########
@@ -482,176 +482,6 @@ public boolean accept(CarbonFile file) {
 
   }
 
-  /**
-   * Handling of the clean up of old carbondata files, index files , delete delta,
-   * update status files.
-   * @param table clean up will be handled on this table.
-   * @param forceDelete if true then max query execution timeout will not be considered.
-   */
-  public static void cleanUpDeltaFiles(CarbonTable table, boolean forceDelete) throws IOException {
-
-    SegmentStatusManager ssm = new SegmentStatusManager(table.getAbsoluteTableIdentifier());
-
-    LoadMetadataDetails[] details =
-        SegmentStatusManager.readLoadMetadata(table.getMetadataPath());
-
-    SegmentUpdateStatusManager updateStatusManager = new SegmentUpdateStatusManager(table);
-    SegmentUpdateDetails[] segmentUpdateDetails = updateStatusManager.getUpdateStatusDetails();
-    // hold all the segments updated so that wen can check the delta files in them, ne need to
-    // check the others.
-    Set<String> updatedSegments = new HashSet<>();
-    for (SegmentUpdateDetails updateDetails : segmentUpdateDetails) {
-      updatedSegments.add(updateDetails.getSegmentName());
-    }
-
-    String validUpdateStatusFile = "";
-
-    boolean isAbortedFile = true;
-
-    boolean isInvalidFile = false;
-
-    // take the update status file name from 0th segment.
-    validUpdateStatusFile = ssm.getUpdateStatusFileName(details);
-    // scan through each segment.
-    for (LoadMetadataDetails segment : details) {
-      // if this segment is valid then only we will go for delta file deletion.
-      // if the segment is mark for delete or compacted then any way it will get deleted.
-      if (segment.getSegmentStatus() == SegmentStatus.SUCCESS
-              || segment.getSegmentStatus() == SegmentStatus.LOAD_PARTIAL_SUCCESS) {
-        // when there is no update operations done on table, then no need to go ahead. So
-        // just check the update delta start timestamp and proceed if not empty
-        if (!segment.getUpdateDeltaStartTimestamp().isEmpty()
-                || updatedSegments.contains(segment.getLoadName())) {
-          // take the list of files from this segment.
-          String segmentPath = CarbonTablePath.getSegmentPath(
-              table.getAbsoluteTableIdentifier().getTablePath(), segment.getLoadName());
-          CarbonFile segDir =
-              FileFactory.getCarbonFile(segmentPath);
-          CarbonFile[] allSegmentFiles = segDir.listFiles();
-
-          // now handle all the delete delta files which needs to be deleted.
-          // there are 2 cases here .
-          // 1. if the block is marked as compacted then the corresponding delta files
-          //    can be deleted if query exec timeout is done.
-          // 2. if the block is in success state then also there can be delete
-          //    delta compaction happened and old files can be deleted.
-
-          SegmentUpdateDetails[] updateDetails = updateStatusManager.readLoadMetadata();
-          for (SegmentUpdateDetails block : updateDetails) {
-            CarbonFile[] completeListOfDeleteDeltaFiles;
-            CarbonFile[] invalidDeleteDeltaFiles;
-
-            if (!block.getSegmentName().equalsIgnoreCase(segment.getLoadName())) {
-              continue;
-            }
-
-            // aborted scenario.
-            invalidDeleteDeltaFiles = updateStatusManager
-                .getDeleteDeltaInvalidFilesList(block, false,
-                    allSegmentFiles, isAbortedFile);
-            for (CarbonFile invalidFile : invalidDeleteDeltaFiles) {
-              boolean doForceDelete = true;
-              compareTimestampsAndDelete(invalidFile, doForceDelete, false);
-            }
-
-            // case 1
-            if (CarbonUpdateUtil.isBlockInvalid(block.getSegmentStatus())) {
-              completeListOfDeleteDeltaFiles = updateStatusManager
-                  .getDeleteDeltaInvalidFilesList(block, true,
-                      allSegmentFiles, isInvalidFile);
-              for (CarbonFile invalidFile : completeListOfDeleteDeltaFiles) {
-                compareTimestampsAndDelete(invalidFile, forceDelete, false);
-              }
-
-            } else {
-              invalidDeleteDeltaFiles = updateStatusManager
-                  .getDeleteDeltaInvalidFilesList(block, false,
-                      allSegmentFiles, isInvalidFile);
-              for (CarbonFile invalidFile : invalidDeleteDeltaFiles) {
-                compareTimestampsAndDelete(invalidFile, forceDelete, false);
-              }
-            }
-          }
-        }
-        // handle cleanup of merge index files and data files after small files merge happened for
-        // SI table
-        cleanUpDataFilesAfterSmallFilesMergeForSI(table, segment);
-      }
-    }
-
-    // delete the update table status files which are old.
-    if (null != validUpdateStatusFile && !validUpdateStatusFile.isEmpty()) {
-
-      final String updateStatusTimestamp = validUpdateStatusFile
-          .substring(validUpdateStatusFile.lastIndexOf(CarbonCommonConstants.HYPHEN) + 1);
-
-      String tablePath = table.getAbsoluteTableIdentifier().getTablePath();
-      CarbonFile metaFolder = FileFactory.getCarbonFile(
-          CarbonTablePath.getMetadataPath(tablePath));
-
-      CarbonFile[] invalidUpdateStatusFiles = metaFolder.listFiles(new CarbonFileFilter() {
-        @Override
-        public boolean accept(CarbonFile file) {
-          if (file.getName().startsWith(CarbonCommonConstants.TABLEUPDATESTATUS_FILENAME)) {
-            // CHECK if this is valid or not.
-            // we only send invalid ones to delete.
-            return !file.getName().endsWith(updateStatusTimestamp);
-          }
-          return false;
-        }
-      });
-
-      for (CarbonFile invalidFile : invalidUpdateStatusFiles) {
-        compareTimestampsAndDelete(invalidFile, forceDelete, true);
-      }
-    }
-  }
-
-  /**
-   * this is the clean up added specifically for SI table, because after we merge the data files
-   * inside the secondary index table, we need to delete the stale carbondata files.
-   * refer org.apache.spark.sql.secondaryindex.rdd.CarbonSIRebuildRDD
-   */
-  private static void cleanUpDataFilesAfterSmallFilesMergeForSI(CarbonTable table,

Review comment:
       clean files on the main table will trigger clean files of the index tables




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r535224194



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/events/CleanFilesEvents.scala
##########
@@ -34,5 +34,6 @@ case class CleanFilesPreEvent(carbonTable: CarbonTable, sparkSession: SparkSessi
  * @param carbonTable
  * @param sparkSession
  */
-case class CleanFilesPostEvent(carbonTable: CarbonTable, sparkSession: SparkSession)
-  extends Event with CleanFilesEventInfo
+case class CleanFilesPostEvent(carbonTable: CarbonTable,

Review comment:
       ok




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r535228252



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##########
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.trash
+
+import scala.collection.JavaConverters._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.indexstore.PartitionSpec
+import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.SegmentFileStore
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
+import org.apache.carbondata.core.util.{CarbonProperties, CleanFilesUtil, TrashUtil}
+
+/**
+ * This object will manage the following data.
+ * 1. .Trash folder
+ * 2. stale segments without metadata
+ * 3. expired segments (MARKED_FOR_DELETE, Compacted, In Progress)
+ */
+object DataTrashManager {
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  /**
+   * clean garbage data
+   *  1. clean .Trash folder
+   *  2. clean stale segments without metadata
+   *  3. clean expired segments (MARKED_FOR_DELETE, Compacted, In Progress)
+   *
+   * @param carbonTable     : CarbonTable Object
+   * @param partitionSpecs : Hive Partitions  details
+   */
+  def cleanGarbageData(
+      carbonTable: CarbonTable,
+      force: Boolean = false,
+      partitionSpecs: Option[Seq[PartitionSpec]] = None): Unit = {
+    var carbonCleanFilesLock: ICarbonLock = null
+    val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+    try {
+      val errorMsg = "Clean files request is failed for " +
+        s"${ carbonTable.getQualifiedName }" +
+        ". Not able to acquire the clean files lock due to another clean files " +
+        "operation is running in the background."
+      carbonCleanFilesLock = CarbonLockUtil.getLockObject(absoluteTableIdentifier,
+        LockUsage.CLEAN_FILES_LOCK, errorMsg)
+      // step 1: clean trash folder
+      cleanTrashFolder(carbonTable, force)

Review comment:
       ok




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r535228453



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##########
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.carbondata.trash
+
+import scala.collection.JavaConverters._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.indexstore.PartitionSpec
+import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, LockUsage}
+import org.apache.carbondata.core.metadata.SegmentFileStore
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
+import org.apache.carbondata.core.util.{CarbonProperties, CleanFilesUtil, TrashUtil}
+
+/**
+ * This object will manage the following data.
+ * 1. .Trash folder
+ * 2. stale segments without metadata
+ * 3. expired segments (MARKED_FOR_DELETE, Compacted, In Progress)
+ */
+object DataTrashManager {
+  private val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  /**
+   * clean garbage data
+   *  1. clean .Trash folder
+   *  2. clean stale segments without metadata
+   *  3. clean expired segments (MARKED_FOR_DELETE, Compacted, In Progress)
+   *
+   * @param carbonTable     : CarbonTable Object
+   * @param partitionSpecs : Hive Partitions  details
+   */
+  def cleanGarbageData(
+      carbonTable: CarbonTable,
+      force: Boolean = false,
+      partitionSpecs: Option[Seq[PartitionSpec]] = None): Unit = {
+    var carbonCleanFilesLock: ICarbonLock = null
+    val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+    try {
+      val errorMsg = "Clean files request is failed for " +
+        s"${ carbonTable.getQualifiedName }" +
+        ". Not able to acquire the clean files lock due to another clean files " +
+        "operation is running in the background."
+      carbonCleanFilesLock = CarbonLockUtil.getLockObject(absoluteTableIdentifier,
+        LockUsage.CLEAN_FILES_LOCK, errorMsg)
+      // step 1: clean trash folder
+      cleanTrashFolder(carbonTable, force)
+      // step 2: clean stale segments which are not exists in metadata
+      cleanStaleSegments(carbonTable)

Review comment:
       ok




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r535224194



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/events/CleanFilesEvents.scala
##########
@@ -34,5 +34,6 @@ case class CleanFilesPreEvent(carbonTable: CarbonTable, sparkSession: SparkSessi
  * @param carbonTable
  * @param sparkSession
  */
-case class CleanFilesPostEvent(carbonTable: CarbonTable, sparkSession: SparkSession)
-  extends Event with CleanFilesEventInfo
+case class CleanFilesPostEvent(carbonTable: CarbonTable,

Review comment:
       ok




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] akashrn5 commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

akashrn5 commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r535332892



##########
File path: core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
##########
@@ -482,176 +482,6 @@ public boolean accept(CarbonFile file) {
 
   }
 
-  /**
-   * Handling of the clean up of old carbondata files, index files , delete delta,
-   * update status files.
-   * @param table clean up will be handled on this table.
-   * @param forceDelete if true then max query execution timeout will not be considered.
-   */
-  public static void cleanUpDeltaFiles(CarbonTable table, boolean forceDelete) throws IOException {

Review comment:
       cleanStaleDeltaFiles will be called only in case of exception here, please see below points
   1. in Projectfordeletecommand, its called two times both in exception  case and finally block, thats wrong, it should be only in finally block
   2. its called multiple times in `DeleteExecution ` class which needs to be checked and avoid it.
   3. since its handled only in case of exception case, how its taken care in case of application crash or shutdown scenario?
   4. cleanStaleDeltaFiles  had cases of handling the update and delete aborted case, how its handled now, as the complete method is removed?




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] akashrn5 commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

akashrn5 commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r535339696



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
##########
@@ -90,7 +90,6 @@ class CarbonTableCompactor(carbonLoadModel: CarbonLoadModel,
 
     while (loadsToMerge.size() > 1 || needSortSingleSegment(loadsToMerge)) {
       val lastSegment = sortedSegments.get(sortedSegments.size() - 1)
-      deletePartialLoadsInCompaction()

Review comment:
       but the stale data will always be present inside segment folder right? by chance assume if the segment file is corrupted or deleted, still carbon should pass the query, which it does by listing, in that case we will get the wrong data or query will always fail. I think we need to have some way to clean them, what you think @QiangCai @ajantha-bhat




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] akashrn5 commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

akashrn5 commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r535345986



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/DropIndexCommand.scala
##########
@@ -210,12 +210,6 @@ private[sql] case class DropIndexCommand(
           logError("Table metadata unlocking is unsuccessful, index table may be in stale state")
         }
       }
-      // in case if the the physical folders still exists for the index table
-      // but the carbon and hive info for the index table is removed,
-      // DROP INDEX IF EXISTS should clean up those physical directories
-      if (ifExistsSet && carbonTable.isEmpty) {

Review comment:
       are you saying that, even though the t2 is not an index, and i call drop index on t2, it will consider as a index and drop t2 table folder? If so, we should handle there and keep this code here. This was handled specially for SI i think in negative scenarios




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#issuecomment-738118531


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


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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#issuecomment-738119523


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


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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r535391315



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/DropIndexCommand.scala
##########
@@ -210,12 +210,6 @@ private[sql] case class DropIndexCommand(
           logError("Table metadata unlocking is unsuccessful, index table may be in stale state")
         }
       }
-      // in case if the the physical folders still exists for the index table
-      // but the carbon and hive info for the index table is removed,
-      // DROP INDEX IF EXISTS should clean up those physical directories
-      if (ifExistsSet && carbonTable.isEmpty) {

Review comment:
       I agree with @akashrn5 : If code is buggy, we need to fix it. If we remove it. It might bring back the original issues that caused this code 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:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

ajantha-bhat commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r535392789



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
##########
@@ -90,7 +90,6 @@ class CarbonTableCompactor(carbonLoadModel: CarbonLoadModel,
 
     while (loadsToMerge.size() > 1 || needSortSingleSegment(loadsToMerge)) {
       val lastSegment = sortedSegments.get(sortedSegments.size() - 1)
-      deletePartialLoadsInCompaction()

Review comment:
       I think this will go as a stale segment in trash folder when clean files is called right ?




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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#issuecomment-738625327


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


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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#issuecomment-738628213


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


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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#issuecomment-738921551


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


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

For queries about this service, please contact Infrastructure at:
[hidden email]


Reply | Threaded
Open this post in threaded view
|

[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

GitBox
In reply to this post by GitBox

CarbonDataQA2 commented on pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#issuecomment-738930560


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


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

For queries about this service, please contact Infrastructure at:
[hidden email]


1234