Created
January 3, 2014 19:39
-
-
Save yukim/8245030 to your computer and use it in GitHub Desktop.
Fix for possible endless lock in CLSM
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java | |
index 4275362..815616b 100644 | |
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java | |
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java | |
@@ -288,10 +288,9 @@ public class CommitLogSegmentManager | |
advanceAllocatingFrom(last); | |
// flush and wait for all CFs that are dirty in segments up-to and including 'last' | |
- Future<?> future = flushDataFrom(segmentsToRecycle); | |
try | |
{ | |
- future.get(); | |
+ FBUtilities.waitOnFutures(flushDataFrom(segmentsToRecycle)); | |
// now recycle segments that are unused, as we may not have triggered a discardCompletedSegments() | |
// if the previous active segment was the only one to recycle (since an active segment isn't | |
@@ -301,7 +300,7 @@ public class CommitLogSegmentManager | |
recycleSegment(segment); | |
CommitLogSegment first; | |
- assert (first = activeSegments.peek()) == null || first.id > last.id; | |
+ assert (first = activeSegments.peek()) == null || first.id > last.id : "first " + first.id + " <= last " + last.id; | |
} | |
catch (Throwable t) | |
{ | |
@@ -443,7 +442,7 @@ public class CommitLogSegmentManager | |
* | |
* @return a Future that will finish when all the flushes are complete. | |
*/ | |
- private Future<?> flushDataFrom(Collection<CommitLogSegment> segments) | |
+ private Collection<Future<?>> flushDataFrom(Collection<CommitLogSegment> segments) | |
{ | |
// a map of CfId -> forceFlush() to ensure we only queue one flush per cf | |
final Map<UUID, Future<?>> flushes = new LinkedHashMap<>(); | |
@@ -479,14 +478,7 @@ public class CommitLogSegmentManager | |
} | |
} | |
- return new FutureTask<>(new Callable<Object>() | |
- { | |
- public Object call() | |
- { | |
- FBUtilities.waitOnFutures(flushes.values()); | |
- return null; | |
- } | |
- }); | |
+ return flushes.values(); | |
} | |
/** |
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment