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 2009/06/23 19:48:19 UTC

svn commit: r787762 - in /incubator/cassandra/trunk: src/java/org/apache/cassandra/db/ src/java/org/apache/cassandra/io/ src/java/org/apache/cassandra/loader/ src/java/org/apache/cassandra/utils/ test/unit/org/apache/cassandra/db/ test/unit/org/apache/...

Author: jbellis
Date: Tue Jun 23 17:48:18 2009
New Revision: 787762

URL: http://svn.apache.org/viewvc?rev=787762&view=rev
Log:
make static methods of SSTable that take a filename and then use the cache to grab an sstable object, simple non-static methods instead
patch by jbellis; reviewed by Eric Evans for CASSANDRA-224

Added:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/FileStruct.java
      - copied, changed from r787761, incubator/cassandra/trunk/src/java/org/apache/cassandra/db/FileStruct.java
Removed:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/FileStruct.java
Modified:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/DataFileVerbHandler.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/FileStructComparator.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTable.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/loader/Loader.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/loader/PreLoad.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/utils/FileUtils.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableTest.java

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=787762&r1=787761&r2=787762&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 Tue Jun 23 17:48:18 2009
@@ -34,11 +34,7 @@
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.IPartitioner;
-import org.apache.cassandra.io.DataInputBuffer;
-import org.apache.cassandra.io.DataOutputBuffer;
-import org.apache.cassandra.io.IndexHelper;
-import org.apache.cassandra.io.SSTable;
-import org.apache.cassandra.io.SequenceFile;
+import org.apache.cassandra.io.*;
 import org.apache.cassandra.net.EndPoint;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.*;
@@ -766,7 +762,7 @@
             {
                 try
                 {
-                    fs = new FileStruct(SequenceFile.bufferedReader(file, bufferSize), StorageService.getPartitioner());
+                    fs = SSTable.get(file).getFileStruct();
                     fs.advance();
                     if (fs.isExhausted())
                     {
@@ -1503,9 +1499,9 @@
     }
 
     /** not threadsafe.  caller must have lock_ acquired. */
-    public SortedSet<String> getSSTableFilenames()
+    public Collection<SSTable> getSSTables()
     {
-        return Collections.unmodifiableSortedSet((SortedSet<String>)ssTables_.keySet());
+        return Collections.unmodifiableCollection(ssTables_.values());
     }
 
     public ReentrantReadWriteLock.ReadLock getReadLock()

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/DataFileVerbHandler.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/DataFileVerbHandler.java?rev=787762&r1=787761&r2=787762&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/DataFileVerbHandler.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/DataFileVerbHandler.java Tue Jun 23 17:48:18 2009
@@ -29,6 +29,8 @@
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.LogUtil;
+import org.apache.cassandra.io.SSTable;
+
 import org.apache.log4j.Logger;
 
 
@@ -44,13 +46,13 @@
         
         try
         {
-            List<String> allFiles = Table.open(table).getAllSSTablesOnDisk();
+            List<SSTable> ssTables = Table.open(table).getAllSSTablesOnDisk();
             ByteArrayOutputStream bos = new ByteArrayOutputStream();
             DataOutputStream dos = new DataOutputStream(bos);
-            dos.writeInt(allFiles.size());
-            for ( String file : allFiles )
+            dos.writeInt(ssTables.size());
+            for (SSTable sstable : ssTables)
             {
-                dos.writeUTF(file);
+                dos.writeUTF(sstable.getFilename());
             }
             Message response = message.getReply( StorageService.getLocalStorageEndPoint(), bos.toByteArray());
             MessagingService.getMessagingInstance().sendOneWay(response, message.getFrom());

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/FileStructComparator.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/FileStructComparator.java?rev=787762&r1=787761&r2=787762&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/FileStructComparator.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/FileStructComparator.java Tue Jun 23 17:48:18 2009
@@ -1,25 +1,27 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*    http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing,
-* software distributed under the License is distributed on an
-* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-* KIND, either express or implied.  See the License for the
-* specific language governing permissions and limitations
-* under the License.
-*/
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*    http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*/
 package org.apache.cassandra.db;
 
 import java.util.Comparator;
 
+import org.apache.cassandra.io.FileStruct;
+
 class FileStructComparator implements Comparator<FileStruct>
 {
     public int compare(FileStruct f, FileStruct f2)

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java?rev=787762&r1=787761&r2=787762&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java Tue Jun 23 17:48:18 2009
@@ -35,6 +35,7 @@
 import org.apache.cassandra.io.DataInputBuffer;
 import org.apache.cassandra.io.SSTable;
 import org.apache.cassandra.io.SequenceFile;
+import org.apache.cassandra.io.FileStruct;
 import org.apache.cassandra.net.EndPoint;
 import org.apache.cassandra.net.IVerbHandler;
 import org.apache.cassandra.net.Message;
@@ -453,15 +454,15 @@
     /*
      * Get the list of all SSTables on disk.  Not safe unless you aquire the CFS readlocks!
     */
-    public List<String> getAllSSTablesOnDisk()
+    public List<SSTable> getAllSSTablesOnDisk()
     {
-        List<String> list = new ArrayList<String>();
+        List<SSTable> list = new ArrayList<SSTable>();
         Set<String> columnFamilies = tableMetadata_.getColumnFamilies();
         for ( String columnFamily : columnFamilies )
         {
             ColumnFamilyStore cfStore = columnFamilyStores_.get( columnFamily );
             if ( cfStore != null )
-                list.addAll(cfStore.getSSTableFilenames());
+                list.addAll(cfStore.getSSTables());
         }
         return list;
     }
@@ -829,9 +830,9 @@
             }
 
             // sstables
-            for (String filename : cfs.getSSTableFilenames())
+            for (SSTable sstable : cfs.getSSTables())
             {
-                FileStruct fs = new FileStruct(SequenceFile.reader(filename), StorageService.getPartitioner());
+                FileStruct fs = sstable.getFileStruct();
                 fs.seekTo(startWith);
                 iterators.add(fs);
             }

Copied: incubator/cassandra/trunk/src/java/org/apache/cassandra/io/FileStruct.java (from r787761, incubator/cassandra/trunk/src/java/org/apache/cassandra/db/FileStruct.java)
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/FileStruct.java?p2=incubator/cassandra/trunk/src/java/org/apache/cassandra/io/FileStruct.java&p1=incubator/cassandra/trunk/src/java/org/apache/cassandra/db/FileStruct.java&r1=787761&r2=787762&rev=787762&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/FileStruct.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/io/FileStruct.java Tue Jun 23 17:48:18 2009
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-package org.apache.cassandra.db;
+package org.apache.cassandra.io;
 
 import java.io.IOException;
 import java.util.Iterator;
@@ -25,7 +25,6 @@
 import org.apache.cassandra.io.DataOutputBuffer;
 import org.apache.cassandra.io.IFileReader;
 import org.apache.cassandra.io.SSTable;
-import org.apache.cassandra.io.Coordinate;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.log4j.Logger;
 import com.google.common.collect.AbstractIterator;
@@ -40,13 +39,13 @@
     private IFileReader reader;
     private DataInputBuffer bufIn;
     private DataOutputBuffer bufOut;
-    private IPartitioner partitioner;
+    private SSTable sstable;
     private FileStructIterator iterator;
 
-    public FileStruct(IFileReader reader, IPartitioner partitioner)
+    FileStruct(SSTable sstable) throws IOException
     {
-        this.reader = reader;
-        this.partitioner = partitioner;
+        this.reader = SequenceFile.bufferedReader(sstable.getFilename(), 1024 * 1024);
+        this.sstable = sstable;
         bufIn = new DataInputBuffer();
         bufOut = new DataOutputBuffer();
     }
@@ -78,14 +77,14 @@
 
     public int compareTo(FileStruct f)
     {
-        return partitioner.getDecoratedKeyComparator().compare(key, f.key);
+        return sstable.getPartitioner().getDecoratedKeyComparator().compare(key, f.key);
     }    
 
     public void seekTo(String seekKey)
     {
         try
         {
-            long position = SSTable.getNearestPosition(seekKey, reader, partitioner);
+            long position = sstable.getNearestPosition(seekKey);
             if (position < 0)
             {
                 exhausted = true;

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTable.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTable.java?rev=787762&r1=787761&r2=787762&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTable.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTable.java Tue Jun 23 17:48:18 2009
@@ -27,10 +27,9 @@
 import org.apache.cassandra.db.RowMutation;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.io.SequenceFile.ColumnGroupReader;
-import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.BloomFilter;
 import org.apache.cassandra.utils.FileUtils;
-import org.apache.cassandra.utils.LogUtil;
+import org.cliffc.high_scale_lib.NonBlockingHashMap;
 
 /**
  * This class is built on top of the SequenceFile. It stores
@@ -48,7 +47,7 @@
 public class SSTable
 {
     private static Logger logger_ = Logger.getLogger(SSTable.class);
-    /* Every 128th key is an index. */
+    /* Every 128th index entry is loaded into memory so we know where to start looking for the actual key w/o seeking */
     private static final int indexInterval_ = 128;
     /* Required extension for temporary files created during compactions. */
     public static final String temporaryFile_ = "tmp";
@@ -61,15 +60,6 @@
         return indexInterval_;
     }
 
-    private static void deleteWithConfirm(File file) throws IOException
-    {
-        assert file.exists() : "attempted to delete non-existing file " + file.getName();
-        if (!file.delete())
-        {
-            throw new IOException("Failed to delete " + file.getName());
-        }
-    }
-
     // todo can we refactor to take list of sstables?
     public static int getApproximateKeyCount(List<String> dataFiles)
     {
@@ -195,7 +185,7 @@
             input.readLong();
             if (i++ % indexInterval_ == 0)
             {
-                indexPositions_.add(new KeyPosition(decoratedKey, indexPosition, partitioner_));
+                indexPositions_.add(new KeyPosition(decoratedKey, indexPosition));
             }
         }
     }
@@ -259,7 +249,7 @@
         {
             indexPositions_ = new ArrayList<KeyPosition>();
         }
-        indexPositions_.add(new KeyPosition(decoratedKey, indexPosition, partitioner_));
+        indexPositions_.add(new KeyPosition(decoratedKey, indexPosition));
         logger_.trace("wrote index of " + decoratedKey + " at " + indexPosition);
     }
 
@@ -279,11 +269,10 @@
     }
 
     /** get the position in the index file to start scanning to find the given key (at most indexInterval keys away) */
-    private static long getIndexScanPosition(String decoratedKey, IFileReader dataReader, IPartitioner partitioner)
+    private long getIndexScanPosition(String decoratedKey, IPartitioner partitioner)
     {
-        List<KeyPosition> positions = openedFiles.get(dataReader.getFileName()).getIndexPositions();
-        assert positions != null && positions.size() > 0;
-        int index = Collections.binarySearch(positions, new KeyPosition(decoratedKey, -1, partitioner));
+        assert indexPositions_ != null && indexPositions_.size() > 0;
+        int index = Collections.binarySearch(indexPositions_, new KeyPosition(decoratedKey, -1));
         if (index < 0)
         {
             // binary search gives us the first index _greater_ than the key searched for,
@@ -291,27 +280,26 @@
             int greaterThan = (index + 1) * -1;
             if (greaterThan == 0)
                 return -1;
-            return positions.get(greaterThan - 1).position;
+            return indexPositions_.get(greaterThan - 1).position;
         }
         else
         {
-            return positions.get(index).position;
+            return indexPositions_.get(index).position;
         }
     }
 
     /**
      * returns the position in the data file to find the given key, or -1 if the key is not present
      */
-    /* TODO having this static means we have to keep re-opening the index file, which sucks.  Need to move towards
-       greater encapsulation. */
-    public static long getPosition(String decoratedKey, IFileReader dataReader, IPartitioner partitioner) throws IOException
+    public long getPosition(String decoratedKey, IPartitioner partitioner) throws IOException
     {
-        long start = getIndexScanPosition(decoratedKey, dataReader, partitioner);
+        long start = getIndexScanPosition(decoratedKey, partitioner);
         if (start < 0)
         {
             return -1;
         }
-        BufferedRandomAccessFile input = new BufferedRandomAccessFile(indexFilename(dataReader.getFileName()), "r");
+        // TODO mmap the index file?
+        BufferedRandomAccessFile input = new BufferedRandomAccessFile(indexFilename(dataFile_), "r");
         input.seek(start);
         int i = 0;
         try
@@ -343,14 +331,14 @@
     }
 
     /** like getPosition, but if key is not found will return the location of the first key _greater_ than the desired one, or -1 if no such key exists. */
-    public static long getNearestPosition(String decoratedKey, IFileReader dataReader, IPartitioner partitioner) throws IOException
+    public long getNearestPosition(String decoratedKey) throws IOException
     {
-        long start = getIndexScanPosition(decoratedKey, dataReader, partitioner);
+        long start = getIndexScanPosition(decoratedKey, partitioner_);
         if (start < 0)
         {
             return 0;
         }
-        BufferedRandomAccessFile input = new BufferedRandomAccessFile(indexFilename(dataReader.getFileName()), "r");
+        BufferedRandomAccessFile input = new BufferedRandomAccessFile(indexFilename(dataFile_), "r");
         input.seek(start);
         try
         {
@@ -366,7 +354,7 @@
                     return -1;
                 }
                 long position = input.readLong();
-                int v = partitioner.getDecoratedKeyComparator().compare(indexDecoratedKey, decoratedKey);
+                int v = partitioner_.getDecoratedKeyComparator().compare(indexDecoratedKey, decoratedKey);
                 if (v >= 0)
                     return position;
             }
@@ -389,7 +377,7 @@
         {
             dataReader = SequenceFile.reader(dataFile_);
             String decoratedKey = partitioner_.decorateKey(clientKey);
-            long position = getPosition(decoratedKey, dataReader, partitioner_);
+            long position = getPosition(decoratedKey, partitioner_);
 
             DataOutputBuffer bufOut = new DataOutputBuffer();
             DataInputBuffer bufIn = new DataInputBuffer();
@@ -468,7 +456,7 @@
         {
             /* Morph key into actual key based on the partition type. */
             String decoratedKey = partitioner_.decorateKey(key);
-            long position = getPosition(decoratedKey, dataReader, partitioner_);
+            long position = getPosition(decoratedKey, partitioner_);
             return new ColumnGroupReader(dataFile_, decoratedKey, cfName, startColumn, isAscending, position);
         }
         finally
@@ -479,9 +467,10 @@
 
     public void delete() throws IOException
     {
-        deleteWithConfirm(new File(dataFile_));
-        deleteWithConfirm(new File(indexFilename(dataFile_)));
-        deleteWithConfirm(new File(filterFilename(dataFile_)));
+        FileUtils.deleteWithConfirm(new File(dataFile_));
+        FileUtils.deleteWithConfirm(new File(indexFilename(dataFile_)));
+        FileUtils.deleteWithConfirm(new File(filterFilename(dataFile_)));
+        openedFiles.remove(dataFile_);
     }
 
     /** obviously only for testing */
@@ -504,42 +493,59 @@
     {
         return bf.isPresent(partitioner_.decorateKey(clientKey));
     }
-}
 
-/**
- * This is a simple container for the index Key and its corresponding position
- * in the data file. Binary search is performed on a list of these objects
- * to lookup keys within the SSTable data file.
- *
- * All keys are decorated.
- */
-class KeyPosition implements Comparable<KeyPosition>
-{
-    public final String key; // decorated
-    public final long position;
-    private final IPartitioner partitioner; // TODO rip out the static uses of KP so we can just use the parent SSTable's partitioner, when necessary
+    IPartitioner getPartitioner()
+    {
+        return partitioner_;
+    }
 
-    public KeyPosition(String key, long position, IPartitioner partitioner)
+    public FileStruct getFileStruct() throws IOException
     {
-        this.key = key;
-        this.position = position;
-        this.partitioner = partitioner;
+        return new FileStruct(this);
     }
 
-    public int compareTo(KeyPosition kp)
+    public static void deleteAll() throws IOException
     {
-        return partitioner.getDecoratedKeyComparator().compare(key, kp.key);
+        for (SSTable sstable : openedFiles.values())
+        {
+            sstable.delete();
+        }
     }
 
-    public String toString()
+    /**
+     * This is a simple container for the index Key and its corresponding position
+     * in the data file. Binary search is performed on a list of these objects
+     * to lookup keys within the SSTable data file.
+     *
+     * All keys are decorated.
+     */
+    class KeyPosition implements Comparable<KeyPosition>
     {
-        return key + ":" + position;
+        public final String key; // decorated
+        public final long position;
+
+        public KeyPosition(String key, long position)
+        {
+            this.key = key;
+            this.position = position;
+        }
+
+        public int compareTo(KeyPosition kp)
+        {
+            return partitioner_.getDecoratedKeyComparator().compare(key, kp.key);
+        }
+
+        public String toString()
+        {
+            return key + ":" + position;
+        }
     }
+
 }
 
 class FileSSTableMap
 {
-    private final HashMap<String, SSTable> map = new HashMap<String, SSTable>();
+    private final Map<String, SSTable> map = new NonBlockingHashMap<String, SSTable>();
 
     public SSTable get(String filename)
     {
@@ -574,4 +580,9 @@
     {
         map.clear();
     }
+
+    public void remove(String filename) throws IOException
+    {
+        map.remove(new File(filename).getCanonicalPath());
+    }
 }
\ No newline at end of file

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/loader/Loader.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/loader/Loader.java?rev=787762&r1=787761&r2=787762&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/loader/Loader.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/loader/Loader.java Tue Jun 23 17:48:18 2009
@@ -310,13 +310,6 @@
             logger_.info("Taking a nap after forcing a compaction ...");
             Thread.sleep(Loader.siesta_);
 
-            /* Figure out the keys in the index file to relocate the node */
-            List<String> ssTables = Table.open(table).getAllSSTablesOnDisk();
-            /* Load the indexes into memory */
-            for ( String df : ssTables )
-            {
-                SSTable.open(df, StorageService.getPartitioner());
-            }
             /* We should have only one file since we just compacted. */
             List<String> indexedKeys = SSTable.getIndexedKeys();
             storageService_.relocate(indexedKeys.toArray( new String[0]) );
@@ -334,7 +327,7 @@
             */
 
             // TODO Hmm need to double check here
-            SSTable.get(ssTables.get(0)).delete();
+            SSTable.deleteAll();
             logger_.info("Finished all the requisite clean up ...");
         }
     }

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/loader/PreLoad.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/loader/PreLoad.java?rev=787762&r1=787761&r2=787762&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/loader/PreLoad.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/loader/PreLoad.java Tue Jun 23 17:48:18 2009
@@ -95,12 +95,6 @@
             Thread.sleep(PreLoad.siesta_);
 
             /* Figure out the keys in the index file to relocate the node */
-            List<String> ssTables = Table.open(table).getAllSSTablesOnDisk();
-            /* Load the indexes into memory */
-            for ( String df : ssTables )
-            {
-                SSTable.open(df, StorageService.getPartitioner());
-            }
             /* We should have only one file since we just compacted. */
             List<String> indexedKeys = SSTable.getIndexedKeys();
             storageService_.relocate(indexedKeys.toArray( new String[0]) );
@@ -116,7 +110,7 @@
              * Do the cleanup necessary. Delete all commit logs and
              * the SSTables and reset the load state in the StorageService.
             */
-            SSTable.get(ssTables.get(0)).delete();
+            SSTable.deleteAll();
         }
         logger_.info("Finished all the requisite clean up ...");
     }

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/utils/FileUtils.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/utils/FileUtils.java?rev=787762&r1=787761&r2=787762&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/utils/FileUtils.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/utils/FileUtils.java Tue Jun 23 17:48:18 2009
@@ -52,6 +52,15 @@
     	deleter_.shutdownNow();
     }
 
+    public static void deleteWithConfirm(File file) throws IOException
+    {
+        assert file.exists() : "attempted to delete non-existing file " + file.getName();
+        if (!file.delete())
+        {
+            throw new IOException("Failed to delete " + file.getName());
+        }
+    }
+
     public static class Deleter implements Runnable
     {
     	File file_ = null;

Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java?rev=787762&r1=787761&r2=787762&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java Tue Jun 23 17:48:18 2009
@@ -114,9 +114,9 @@
         rm.apply();
         store.forceBlockingFlush();
 
-        List<String> ssTables = table.getAllSSTablesOnDisk();
+        List<SSTable> ssTables = table.getAllSSTablesOnDisk();
         assertEquals(1, ssTables.size());
-        SSTable.get(ssTables.get(0)).forceBloomFilterFailures();
+        ssTables.get(0).forceBloomFilterFailures();
         ColumnFamily cf = store.getColumnFamily("key2", "Standard1:Column1", new IdentityFilter());
         assertNull(cf);
     }

Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java?rev=787762&r1=787761&r2=787762&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java Tue Jun 23 17:48:18 2009
@@ -58,9 +58,9 @@
             if (ft.get() == 0)
                 break;
         }
-        if (store.getSSTableFilenames().size() > 1)
+        if (store.getSSTables().size() > 1)
         {
-            store.doCompaction(store.getSSTableFilenames().size());
+            store.doCompaction(store.getSSTables().size());
         }
         assertEquals(table.getKeyRange(Arrays.asList("Standard1"), "", "", 10000).size(), inserted.size());
     }

Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java?rev=787762&r1=787761&r2=787762&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java Tue Jun 23 17:48:18 2009
@@ -48,7 +48,7 @@
         }
         Future<Integer> ft = MinorCompactionManager.instance().submit(store, 2);
         ft.get();
-        assertEquals(1, store.getSSTableFilenames().size());
+        assertEquals(1, store.getSSTables().size());
         assertEquals(table.getKeyRange(Arrays.asList(columnFamilyName), "", "", 10000).size(), inserted.size());
     }
 

Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java?rev=787762&r1=787761&r2=787762&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/TableTest.java Tue Jun 23 17:48:18 2009
@@ -18,11 +18,7 @@
 
 package org.apache.cassandra.db;
 
-import java.util.SortedSet;
-import java.util.Arrays;
-import java.util.List;
-import java.util.ArrayList;
-import java.util.concurrent.ExecutionException;
+import java.util.*;
 import java.io.IOException;
 
 import org.apache.commons.lang.StringUtils;
@@ -264,9 +260,9 @@
         table.getColumnFamilyStore("Standard2").forceBlockingFlush();
         validateGetSliceNoMatch(table);
 
-        SortedSet<String> ssTables = table.getColumnFamilyStore("Standard2").getSSTableFilenames();
+        Collection<SSTable> ssTables = table.getColumnFamilyStore("Standard2").getSSTables();
         assertEquals(1, ssTables.size());
-        SSTable.get(ssTables.iterator().next()).forceBloomFilterFailures();
+        ssTables.iterator().next().forceBloomFilterFailures();
         validateGetSliceNoMatch(table);
     }
 

Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableTest.java?rev=787762&r1=787761&r2=787762&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableTest.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/SSTableTest.java Tue Jun 23 17:48:18 2009
@@ -25,7 +25,7 @@
 import org.junit.Test;
 
 import org.apache.cassandra.CleanupHelper;
-import org.apache.cassandra.db.FileStruct;
+import org.apache.cassandra.io.FileStruct;
 import org.apache.cassandra.dht.OrderPreservingPartitioner;
 
 public class SSTableTest extends CleanupHelper
@@ -45,15 +45,14 @@
         ssTable.close();
 
         // verify
-        verifySingle(ssTable.dataFile_, bytes, key);
+        verifySingle(ssTable, bytes, key);
         SSTable.reopenUnsafe(); // force reloading the index
-        verifySingle(ssTable.dataFile_, bytes, key);
+        verifySingle(ssTable, bytes, key);
     }
 
-    private void verifySingle(String filename, byte[] bytes, String key) throws IOException
+    private void verifySingle(SSTable sstable, byte[] bytes, String key) throws IOException
     {
-        SSTable ssTable = SSTable.open(filename, new OrderPreservingPartitioner());
-        FileStruct fs = new FileStruct(SequenceFile.bufferedReader(ssTable.dataFile_, 128 * 1024), new OrderPreservingPartitioner());
+        FileStruct fs = sstable.getFileStruct();
         fs.seekTo(key);
         int size = fs.getBufIn().readInt();
         byte[] bytes2 = new byte[size];
@@ -80,17 +79,16 @@
         ssTable.close();
 
         // verify
-        verifyMany(ssTable.dataFile_, map);
+        verifyMany(ssTable, map);
         SSTable.reopenUnsafe(); // force reloading the index
-        verifyMany(ssTable.dataFile_, map);
+        verifyMany(ssTable, map);
     }
 
-    private void verifyMany(String filename, TreeMap<String, byte[]> map) throws IOException
+    private void verifyMany(SSTable sstable, TreeMap<String, byte[]> map) throws IOException
     {
         List<String> keys = new ArrayList(map.keySet());
         Collections.shuffle(keys);
-        SSTable ssTable = SSTable.open(filename, new OrderPreservingPartitioner());
-        FileStruct fs = new FileStruct(SequenceFile.bufferedReader(ssTable.dataFile_, 128 * 1024), new OrderPreservingPartitioner());
+        FileStruct fs = sstable.getFileStruct();
         for (String key : keys)
         {
             fs.seekTo(key);