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/27 00:14:11 UTC
svn commit: r903482 - in
/incubator/cassandra/trunk/src/java/org/apache/cassandra/io:
SSTableDeletingReference.java SSTableReader.java
Author: jbellis
Date: Tue Jan 26 23:14:10 2010
New Revision: 903482
URL: http://svn.apache.org/viewvc?rev=903482&view=rev
Log:
mv SSTableReader.FileDeletingReference -> SSTableDeletingReference. patch by jbellis
Added:
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableDeletingReference.java (with props)
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableReader.java
Added: incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableDeletingReference.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableDeletingReference.java?rev=903482&view=auto
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableDeletingReference.java (added)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableDeletingReference.java Tue Jan 26 23:14:10 2010
@@ -0,0 +1,71 @@
+package org.apache.cassandra.io;
+
+import java.io.File;
+import java.io.IOException;
+import java.lang.ref.PhantomReference;
+import java.lang.ref.ReferenceQueue;
+import java.util.Timer;
+import java.util.TimerTask;
+
+import org.apache.log4j.Logger;
+
+class SSTableDeletingReference extends PhantomReference<SSTableReader>
+{
+ private static final Logger logger = Logger.getLogger(SSTableDeletingReference.class);
+
+ private static final Timer timer = new Timer("SSTABLE-CLEANUP-TIMER");
+ private static final int RETRY_DELAY = 10000;
+
+ public final String path;
+ private boolean deleteOnCleanup;
+
+ SSTableDeletingReference(SSTableReader referent, ReferenceQueue<? super SSTableReader> q)
+ {
+ super(referent, q);
+ this.path = referent.path;
+ }
+
+ public void deleteOnCleanup()
+ {
+ deleteOnCleanup = true;
+ }
+
+ public void cleanup() throws IOException
+ {
+ if (deleteOnCleanup)
+ {
+ // this is tricky because the mmapping might not have been finalized yet,
+ // 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 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())
+ {
+ if (attempts++ < DeletionService.MAX_RETRIES)
+ {
+ timer.schedule(this, 10000);
+ return;
+ }
+ else
+ {
+ throw new RuntimeException("Unable to delete " + path);
+ }
+ }
+ logger.info("Deleted " + path);
+ DeletionService.submitDeleteWithRetry(SSTable.indexFilename(path));
+ DeletionService.submitDeleteWithRetry(SSTable.filterFilename(path));
+ DeletionService.submitDeleteWithRetry(SSTable.compactedFilename(path));
+ }
+ }
+}
Propchange: incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableDeletingReference.java
------------------------------------------------------------------------------
svn:eol-style = native
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=903482&r1=903481&r2=903482&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 Tue Jan 26 23:14:10 2010
@@ -21,7 +21,6 @@
import java.io.*;
import java.util.*;
import java.lang.ref.ReferenceQueue;
-import java.lang.ref.PhantomReference;
import java.lang.ref.Reference;
import java.nio.channels.FileChannel;
import java.nio.MappedByteBuffer;
@@ -67,10 +66,10 @@
{
while (true)
{
- FileDeletingReference r = null;
+ SSTableDeletingReference r = null;
try
{
- r = (FileDeletingReference) finalizerQueue.remove();
+ r = (SSTableDeletingReference) finalizerQueue.remove();
finalizers.remove(r);
}
catch (InterruptedException e)
@@ -198,7 +197,7 @@
return sstable;
}
- FileDeletingReference phantomReference;
+ SSTableDeletingReference phantomReference;
// jvm can only map up to 2GB at a time, so we split index/data into segments of that size when using mmap i/o
private final MappedByteBuffer[] indexBuffers;
private final MappedByteBuffer[] buffers;
@@ -259,7 +258,7 @@
this.indexPositions = indexPositions;
this.spannedIndexDataPositions = spannedIndexDataPositions;
this.bf = bloomFilter;
- phantomReference = new FileDeletingReference(this, finalizerQueue);
+ phantomReference = new SSTableDeletingReference(this, finalizerQueue);
finalizers.add(phantomReference);
openedFiles.put(filename, this);
this.keyCache = keyCache;
@@ -640,61 +639,3 @@
}
}
-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;
-
- FileDeletingReference(SSTableReader referent, ReferenceQueue<? super SSTableReader> q)
- {
- super(referent, q);
- this.path = referent.path;
- }
-
- public void deleteOnCleanup()
- {
- deleteOnCleanup = true;
- }
-
- public void cleanup() throws IOException
- {
- if (deleteOnCleanup)
- {
- // this is tricky because the mmapping might not have been finalized yet,
- // 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 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())
- {
- if (attempts++ < DeletionService.MAX_RETRIES)
- {
- timer.schedule(this, 10000);
- return;
- }
- else
- {
- 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));
- }
- }
-}