Repository: cassandra Updated Branches: refs/heads/trunk 002d86b6e -> 0a6fa11b3
Fix coverity errors with SSTableDeletingTask and SnapshotDeletingtask Patch by jmckenzie; reviewed by stefania for CASSANDRA-10222 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/7b083a4e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/7b083a4e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/7b083a4e Branch: refs/heads/trunk Commit: 7b083a4e083442cb6e6bf34735753c6103d88654 Parents: 2a2394f Author: Joshua McKenzie <jmcken...@apache.org> Authored: Tue Sep 8 12:20:10 2015 -0400 Committer: Joshua McKenzie <jmcken...@apache.org> Committed: Tue Sep 8 12:20:10 2015 -0400 ---------------------------------------------------------------------- .../io/sstable/SSTableDeletingTask.java | 19 ++++++------------- .../io/sstable/SnapshotDeletingTask.java | 12 +++++------- 2 files changed, 11 insertions(+), 20 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b083a4e/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 34d0347..ef16b5c 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; @@ -30,7 +31,6 @@ import org.slf4j.LoggerFactory; import com.codahale.metrics.Counter; import org.apache.cassandra.concurrent.ScheduledExecutors; -import org.apache.cassandra.db.lifecycle.Tracker; import org.apache.cassandra.utils.FBUtilities; import org.apache.cassandra.utils.concurrent.Blocker; @@ -42,7 +42,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<Runnable> failedTasks = new ConcurrentLinkedQueue<>(); private static final Blocker blocker = new Blocker(); private final Descriptor desc; @@ -73,11 +73,6 @@ public class SSTableDeletingTask implements Runnable } } - public void schedule() - { - ScheduledExecutors.nonPeriodicTasks.submit(this); - } - public void run() { blocker.ask(); @@ -101,11 +96,9 @@ public class SSTableDeletingTask implements Runnable */ public static void rescheduleFailedTasks() { - for (SSTableDeletingTask task : failedTasks) - { - failedTasks.remove(task); - task.schedule(); - } + Runnable task; + while ( null != (task = failedTasks.poll())) + ScheduledExecutors.nonPeriodicTasks.submit(task); // On Windows, snapshots cannot be deleted so long as a segment of the root element is memory-mapped in NTFS. SnapshotDeletingTask.rescheduleFailedTasks(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/7b083a4e/src/java/org/apache/cassandra/io/sstable/SnapshotDeletingTask.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/io/sstable/SnapshotDeletingTask.java b/src/java/org/apache/cassandra/io/sstable/SnapshotDeletingTask.java index 7ab1f11..d23c488 100644 --- a/src/java/org/apache/cassandra/io/sstable/SnapshotDeletingTask.java +++ b/src/java/org/apache/cassandra/io/sstable/SnapshotDeletingTask.java @@ -19,8 +19,8 @@ package org.apache.cassandra.io.sstable; import java.io.File; -import java.util.Set; -import java.util.concurrent.CopyOnWriteArraySet; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; @@ -36,7 +36,7 @@ public class SnapshotDeletingTask implements Runnable private static final Logger logger = LoggerFactory.getLogger(SnapshotDeletingTask.class); public final File path; - private static final Set<SnapshotDeletingTask> failedTasks = new CopyOnWriteArraySet<>(); + private static final Queue<Runnable> failedTasks = new ConcurrentLinkedQueue<>(); public static void addFailedSnapshot(File path) { @@ -68,11 +68,9 @@ public class SnapshotDeletingTask implements Runnable */ public static void rescheduleFailedTasks() { - for (SnapshotDeletingTask task : failedTasks) - { - failedTasks.remove(task); + Runnable task; + while ( null != (task = failedTasks.poll())) ScheduledExecutors.nonPeriodicTasks.submit(task); - } } @VisibleForTesting