You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2010/01/23 00:02:26 UTC

svn commit: r902304 - /incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java

Author: jbellis
Date: Fri Jan 22 23:02:26 2010
New Revision: 902304

URL: http://svn.apache.org/viewvc?rev=902304&view=rev
Log:
move sstable deletions into a Timer to reduce concurrency
patch by jbellis; reviewed by gdusbabek for CASSANDRA-667

Modified:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java?rev=902304&r1=902303&r2=902304&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java Fri Jan 22 23:02:26 2010
@@ -45,7 +45,6 @@
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
-import com.reardencommerce.kernel.collections.shared.evictable.ConcurrentLinkedHashMap;
 
 /**
  * SSTableReaders are open()ed by Table.onStart; after that they are created by SSTableWriter.renameAndOpen.
@@ -56,7 +55,7 @@
     private static final Logger logger = Logger.getLogger(SSTableReader.class);
 
     private static final FileSSTableMap openedFiles = new FileSSTableMap();
-    
+
     // `finalizers` is required to keep the PhantomReferences alive after the enclosing SSTR is itself
     // unreferenced.  otherwise they will never get enqueued.
     private static final Set<Reference<SSTableReader>> finalizers = new HashSet<Reference<SSTableReader>>();
@@ -642,6 +641,9 @@
 
 class FileDeletingReference extends PhantomReference<SSTableReader>
 {
+    private static final Timer timer = new Timer("SSTABLE-CLEANUP-TIMER");
+    private static final int RETRY_DELAY = 10000;
+
     public final String path;
     private boolean deleteOnCleanup;
 
@@ -661,35 +663,37 @@
         if (deleteOnCleanup)
         {
             // this is tricky because the mmapping might not have been finalized yet,
-            // and delete will until it is.  additionally, we need to make sure to
+            // and delete will fail until it is.  additionally, we need to make sure to
             // delete the data file first, so on restart the others will be recognized as GCable
-            // even if the compaction file deletion occurs next.
-            new Thread(new Runnable()
+            // even if the compaction marker gets deleted next.
+            timer.schedule(new CleanupTask(), RETRY_DELAY);
+        }
+    }
+
+    private class CleanupTask extends TimerTask
+    {
+        int attempts = 0;
+
+        @Override
+        public void run()
+        {
+            File datafile = new File(path);
+            if (!datafile.delete())
             {
-                public void run()
+                if (attempts++ < DeletionService.MAX_RETRIES)
                 {
-                    File datafile = new File(path);
-                    for (int i = 0; i < DeletionService.MAX_RETRIES; i++)
-                    {
-                        if (datafile.delete())
-                            break;
-                        try
-                        {
-                            Thread.sleep(10000);
-                        }
-                        catch (InterruptedException e)
-                        {
-                            throw new AssertionError(e);
-                        }
-                    }
-                    if (datafile.exists())
-                        throw new RuntimeException("Unable to delete " + path);
-                    SSTable.logger.info("Deleted " + path);
-                    DeletionService.submitDeleteWithRetry(SSTable.indexFilename(path));
-                    DeletionService.submitDeleteWithRetry(SSTable.filterFilename(path));
-                    DeletionService.submitDeleteWithRetry(SSTable.compactedFilename(path));
+                    timer.schedule(this, 10000);
+                    return;
+                }
+                else
+                {
+                    throw new RuntimeException("Unable to delete " + path);
                 }
-            }, "Cleanup " + path).start();
+            }
+            SSTable.logger.info("Deleted " + path);
+            DeletionService.submitDeleteWithRetry(SSTable.indexFilename(path));
+            DeletionService.submitDeleteWithRetry(SSTable.filterFilename(path));
+            DeletionService.submitDeleteWithRetry(SSTable.compactedFilename(path));
         }
     }
 }