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

[GitHub] [carbondata] Karan980 opened a new pull request #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

Karan980 opened a new pull request #4093:
URL: https://github.com/apache/carbondata/pull/4093


    ### Why is this PR needed?
   Concurrent compaction was failing when run in parallel with load. During load we acquire SegmentLock for a particular segment, and when this same lock we try to acquire during compaction, we were not able to acquire this lock and compaction fails.
     
    ### What changes were proposed in this PR?
   Skipped compaction for segments for which we are not able to acquire the SegmentLock instead of throwing the exception.
   
       
    ### Does this PR introduce any user interface change?
    - No
   
    ### Is any new testcase added?
    - No
   
       
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please 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 #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please 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] kunal642 commented on a change in pull request #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
##########
@@ -92,19 +92,22 @@ class CarbonTableCompactor(
       val lastSegment = sortedSegments.get(sortedSegments.size() - 1)
       val compactedLoad = CarbonDataMergerUtil.getMergedLoadName(loadsToMerge)
       var segmentLocks: ListBuffer[ICarbonLock] = ListBuffer.empty
+      val validSegments = new java.util.ArrayList[LoadMetadataDetails]
       loadsToMerge.asScala.foreach { segmentId =>
         val segmentLock = CarbonLockFactory
           .getCarbonLockObj(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
             .getAbsoluteTableIdentifier,
             CarbonTablePath.addSegmentPrefix(segmentId.getLoadName) + LockUsage.LOCK)
-        if (!segmentLock.lockWithRetries()) {
-          throw new Exception(s"Failed to acquire lock on segment ${segmentId.getLoadName}," +
-            s" during compaction of table ${compactionModel.carbonTable.getQualifiedName}")
+        if (segmentLock.lockWithRetries()) {
+          validSegments.add(segmentId)
+          segmentLocks += segmentLock
+        } else {
+          LOGGER.warn(s"Failed to acquire lock on segment ${segmentId.getLoadName}, " +
+                      s"during compaction of table ${compactionModel.carbonTable.getQualifiedName}")
         }
-        segmentLocks += segmentLock
       }
       try {
-        scanSegmentsAndSubmitJob(loadsToMerge, compactedSegments, compactedLoad)
+        scanSegmentsAndSubmitJob(validSegments, compactedSegments, compactedLoad)

Review comment:
       log the valid segments identified for 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:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please 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 #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please 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] kunal642 commented on a change in pull request #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
##########
@@ -92,19 +92,22 @@ class CarbonTableCompactor(
       val lastSegment = sortedSegments.get(sortedSegments.size() - 1)
       val compactedLoad = CarbonDataMergerUtil.getMergedLoadName(loadsToMerge)
       var segmentLocks: ListBuffer[ICarbonLock] = ListBuffer.empty
+      val validSegments = new java.util.ArrayList[LoadMetadataDetails]
       loadsToMerge.asScala.foreach { segmentId =>
         val segmentLock = CarbonLockFactory
           .getCarbonLockObj(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
             .getAbsoluteTableIdentifier,
             CarbonTablePath.addSegmentPrefix(segmentId.getLoadName) + LockUsage.LOCK)
-        if (!segmentLock.lockWithRetries()) {
-          throw new Exception(s"Failed to acquire lock on segment ${segmentId.getLoadName}," +
-            s" during compaction of table ${compactionModel.carbonTable.getQualifiedName}")
+        if (segmentLock.lockWithRetries()) {
+          validSegments.add(segmentId)
+          segmentLocks += segmentLock
+        } else {
+          LOGGER.warn(s"Failed to acquire lock on segment ${segmentId.getLoadName}, " +
+                      s"during compaction of table ${compactionModel.carbonTable.getQualifiedName}")
         }
-        segmentLocks += segmentLock
       }
       try {
-        scanSegmentsAndSubmitJob(loadsToMerge, compactedSegments, compactedLoad)
+        scanSegmentsAndSubmitJob(validSegments, compactedSegments, compactedLoad)

Review comment:
       please add a test case for this scenario




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

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



[GitHub] [carbondata] ydvpankaj99 commented on pull request #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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


   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] Karan980 commented on a change in pull request #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
##########
@@ -92,19 +92,22 @@ class CarbonTableCompactor(
       val lastSegment = sortedSegments.get(sortedSegments.size() - 1)
       val compactedLoad = CarbonDataMergerUtil.getMergedLoadName(loadsToMerge)
       var segmentLocks: ListBuffer[ICarbonLock] = ListBuffer.empty
+      val validSegments = new java.util.ArrayList[LoadMetadataDetails]
       loadsToMerge.asScala.foreach { segmentId =>
         val segmentLock = CarbonLockFactory
           .getCarbonLockObj(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
             .getAbsoluteTableIdentifier,
             CarbonTablePath.addSegmentPrefix(segmentId.getLoadName) + LockUsage.LOCK)
-        if (!segmentLock.lockWithRetries()) {
-          throw new Exception(s"Failed to acquire lock on segment ${segmentId.getLoadName}," +
-            s" during compaction of table ${compactionModel.carbonTable.getQualifiedName}")
+        if (segmentLock.lockWithRetries()) {
+          validSegments.add(segmentId)
+          segmentLocks += segmentLock
+        } else {
+          LOGGER.warn(s"Failed to acquire lock on segment ${segmentId.getLoadName}, " +
+                      s"during compaction of table ${compactionModel.carbonTable.getQualifiedName}")
         }
-        segmentLocks += segmentLock
       }
       try {
-        scanSegmentsAndSubmitJob(loadsToMerge, compactedSegments, compactedLoad)
+        scanSegmentsAndSubmitJob(validSegments, compactedSegments, compactedLoad)

Review comment:
       Log statement is already present inside scanSegmentAndSubmitJob api.




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please 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] kunal642 removed a comment on pull request #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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


   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 #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please 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 #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please 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 #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please 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] kunal642 commented on pull request #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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


   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 #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please 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] kunal642 commented on a change in pull request #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
##########
@@ -92,19 +92,22 @@ class CarbonTableCompactor(
       val lastSegment = sortedSegments.get(sortedSegments.size() - 1)
       val compactedLoad = CarbonDataMergerUtil.getMergedLoadName(loadsToMerge)
       var segmentLocks: ListBuffer[ICarbonLock] = ListBuffer.empty
+      val validSegments = new java.util.ArrayList[LoadMetadataDetails]
       loadsToMerge.asScala.foreach { segmentId =>
         val segmentLock = CarbonLockFactory
           .getCarbonLockObj(carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
             .getAbsoluteTableIdentifier,
             CarbonTablePath.addSegmentPrefix(segmentId.getLoadName) + LockUsage.LOCK)
-        if (!segmentLock.lockWithRetries()) {
-          throw new Exception(s"Failed to acquire lock on segment ${segmentId.getLoadName}," +
-            s" during compaction of table ${compactionModel.carbonTable.getQualifiedName}")
+        if (segmentLock.lockWithRetries()) {
+          validSegments.add(segmentId)
+          segmentLocks += segmentLock
+        } else {
+          LOGGER.warn(s"Failed to acquire lock on segment ${segmentId.getLoadName}, " +
+                      s"during compaction of table ${compactionModel.carbonTable.getQualifiedName}")
         }
-        segmentLocks += segmentLock
       }
       try {
-        scanSegmentsAndSubmitJob(loadsToMerge, compactedSegments, compactedLoad)
+        scanSegmentsAndSubmitJob(validSegments, compactedSegments, compactedLoad)

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:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please 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 #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please 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] kunal642 commented on pull request #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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


   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 #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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


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


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please 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] kunal642 commented on pull request #4093: [CARBONDATA-4126] Concurrent compaction failed with load on table.

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


   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