Author: gdusbabek
Date: Mon Feb 22 15:38:05 2010
New Revision: 912620

URL: http://svn.apache.org/viewvc?rev=912620&view=rev
Log:
No longer use a streaming directory. Patch by Gary Dusbabek and Stu Hood, 
reviewed by same.  CASSANDRA-810.

Modified:
    
incubator/cassandra/trunk/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
    
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CompactionManager.java
    
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableWriter.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=912620&r1=912619&r2=912620&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
 Mon Feb 22 15:38:05 2010
@@ -47,7 +47,6 @@
 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 {
@@ -712,7 +711,9 @@
             {
                 String oneDir = dataFile + File.separator + table;
                 FileUtils.createDirectory(oneDir);
-                File streamingDir = new File(oneDir, STREAMING_SUBDIR);
+
+                // remove the deprecated streaming directory.
+                File streamingDir = new File(oneDir, "stream");
                 if (streamingDir.exists())
                     FileUtils.deleteDir(streamingDir);
             }

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CompactionManager.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CompactionManager.java?rev=912620&r1=912619&r2=912620&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CompactionManager.java
 (original)
+++ 
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/CompactionManager.java
 Mon Feb 22 15:38:05 2010
@@ -343,13 +343,8 @@
         {
             throw new UnsupportedOperationException("disk full");
         }
-        if (target != null)
-        {
-            // compacting for streaming: send to subdirectory
-            compactionFileLocation = compactionFileLocation + File.separator + 
DatabaseDescriptor.STREAMING_SUBDIR;
-        }
-        List<SSTableReader> results = new ArrayList<SSTableReader>();
 
+        List<SSTableReader> results = new ArrayList<SSTableReader>();
         long startTime = System.currentTimeMillis();
         long totalkeysWritten = 0;
 
@@ -389,7 +384,7 @@
 
         if (writer != null)
         {
-            
results.add(writer.closeAndOpenReader(DatabaseDescriptor.getKeysCachedFraction(table.name,
 cfs.getColumnFamilyName())));
+            
results.add(writer.closeAndOpenReader(DatabaseDescriptor.getKeysCachedFraction(table.name,
 cfs.getColumnFamilyName()), target != null));
             String format = "AntiCompacted to %s.  %d/%d bytes for %d keys.  
Time: %dms.";
             long dTime = System.currentTimeMillis() - startTime;
             logger.info(String.format(format, writer.getFilename(), 
SSTable.getTotalBytes(sstables), results.get(0).length(), totalkeysWritten, 
dTime));

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableWriter.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableWriter.java?rev=912620&r1=912619&r2=912620&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableWriter.java 
(original)
+++ 
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableWriter.java 
Mon Feb 22 15:38:05 2010
@@ -128,10 +128,15 @@
         afterAppend(decoratedKey, currentPosition, value.length);
     }
 
+    public SSTableReader closeAndOpenReader(double cacheFraction) throws 
IOException
+    {
+        return closeAndOpenReader(cacheFraction, false);
+    }
+
     /**
      * Renames temporary SSTable files to valid data, index, and bloom filter 
files
      */
-    public SSTableReader closeAndOpenReader(double cacheFraction) throws 
IOException
+    public SSTableReader closeAndOpenReader(double cacheFraction, boolean 
temporary) throws IOException
     {
         // bloom filter
         FileOutputStream fos = new FileOutputStream(filterFilename());
@@ -148,9 +153,12 @@
         // main data
         dataFile.close(); // calls force
 
-        rename(indexFilename());
-        rename(filterFilename());
-        path = rename(path); // important to do this last since index & filter 
file names are derived from it
+        if (!temporary)
+        {
+            rename(indexFilename());
+            rename(filterFilename());
+            path = rename(path); // important to do this last since index & 
filter file names are derived from it
+        }
 
         InstrumentedCache<DecoratedKey, PositionSize> keyCache = 
SSTableReader.createKeyCache((int)(cacheFraction * keysWritten));
         return new SSTableReader(path, partitioner, indexPositions, 
spannedIndexDataPositions, bf, keyCache);


Reply via email to