Switch SSTableDeletingTask.failingTasks to a ConcurrentLinkedQueue

patch by benedict; reviewed by aleksey for CASSANDRA-9447


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/304eae3b
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/304eae3b
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/304eae3b

Branch: refs/heads/trunk
Commit: 304eae3b799759d334fcc50cbc3439608e1a2647
Parents: 6caade0
Author: Benedict Elliott Smith <[email protected]>
Authored: Wed Jun 3 13:08:28 2015 +0100
Committer: Benedict Elliott Smith <[email protected]>
Committed: Wed Jun 3 13:08:28 2015 +0100

----------------------------------------------------------------------
 .../apache/cassandra/io/sstable/SSTableDeletingTask.java | 11 +++++------
 1 file changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/304eae3b/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java 
b/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
index cc837ba..13bfd6d 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableDeletingTask.java
@@ -19,8 +19,9 @@ package org.apache.cassandra.io.sstable;
 
 import java.io.File;
 import java.util.Collections;
+import java.util.Queue;
 import java.util.Set;
-import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -42,7 +43,7 @@ public class SSTableDeletingTask implements Runnable
     // and delete will fail (on Windows) until it is (we only force the 
unmapping on SUN VMs).
     // Additionally, we need to make sure to delete the data file first, so on 
restart the others
     // will be recognized as GCable.
-    private static final Set<SSTableDeletingTask> failedTasks = new 
CopyOnWriteArraySet<>();
+    private static final Queue<SSTableDeletingTask> failedTasks = new 
ConcurrentLinkedQueue<>();
     private static final Blocker blocker = new Blocker();
 
     private final SSTableReader referent;
@@ -119,11 +120,9 @@ public class SSTableDeletingTask implements Runnable
      */
     public static void rescheduleFailedTasks()
     {
-        for (SSTableDeletingTask task : failedTasks)
-        {
-            failedTasks.remove(task);
+        SSTableDeletingTask task;
+        while ( null != (task = failedTasks.poll()))
             task.schedule();
-        }
     }
 
     /** for tests */

Reply via email to