Author: jbellis
Date: Fri Nov 13 21:00:39 2009
New Revision: 835986

URL: http://svn.apache.org/viewvc?rev=835986&view=rev
Log:
clean up temporary for-streaming files when done
patch by jbellis; tested by Ray Slakinski for CASSANDRA-550

Modified:
    
incubator/cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
    
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/Streaming.java

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java?rev=835986&r1=835985&r2=835986&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
 (original)
+++ 
incubator/cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
 Fri Nov 13 21:00:39 2009
@@ -41,6 +41,7 @@
 public class DatabaseDescriptor
 {
     private static Logger logger_ = Logger.getLogger(DatabaseDescriptor.class);
+    public static final String STREAMING_SUBDIR = "stream";
 
     // don't capitalize these; we need them to match what's in the config file 
for CLS.valueOf to parse
     public static enum CommitLogSync {
@@ -599,7 +600,11 @@
             FileUtils.createDirectory(dataFile + File.separator + 
Table.SYSTEM_TABLE);
             for (String table : tables_)
             {
-                FileUtils.createDirectory(dataFile + File.separator + table);
+                String oneDir = dataFile + File.separator + table;
+                FileUtils.createDirectory(oneDir);
+                File streamingDir = new File(oneDir, STREAMING_SUBDIR);
+                if (streamingDir.exists())
+                    FileUtils.deleteDir(streamingDir);
             }
         }
     }

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=835986&r1=835985&r2=835986&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
 (original)
+++ 
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
 Fri Nov 13 21:00:39 2009
@@ -822,7 +822,7 @@
                     {
                         if (target != null)
                         {
-                            compactionFileLocation = compactionFileLocation + 
File.separator + "stream";
+                            compactionFileLocation = compactionFileLocation + 
File.separator + DatabaseDescriptor.STREAMING_SUBDIR;
                         }
                         FileUtils.createDirectory(compactionFileLocation);
                         String newFilename = new File(compactionFileLocation, 
getTempSSTableFileName()).getAbsolutePath();

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/Streaming.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/Streaming.java?rev=835986&r1=835985&r2=835986&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/io/Streaming.java 
(original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/io/Streaming.java 
Fri Nov 13 21:00:39 2009
@@ -94,9 +94,12 @@
             if (logger.isDebugEnabled())
               logger.debug("Waiting for transfer to " + target + " to 
complete");
             StreamManager.instance(target).waitForStreamCompletion();
-            // reference sstables one more time to make sure it doesn't get 
GC'd early (causing delete of its files)
+            for (SSTableReader sstable : sstables)
+            {
+                sstable.markCompacted();
+            }
             if (logger.isDebugEnabled())
-                logger.debug("Done with transfer to " + target + " of " + 
StringUtils.join(sstables, ", "));
+                logger.debug("Done with transfer to " + target);
         }
     }
 


Reply via email to