Author: jbellis
Date: Wed Mar 17 02:31:48 2010
New Revision: 924108

URL: http://svn.apache.org/viewvc?rev=924108&view=rev
Log:
make CFS.getColumnFamily throw IOError instead of IOException.  patch by 
Rodrigo Peinado; reviewed by jbellis for CASSANDRA-756

Modified:
    cassandra/trunk/src/java/org/apache/cassandra/db/BinaryMemtable.java
    cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
    cassandra/trunk/src/java/org/apache/cassandra/db/SystemTable.java
    cassandra/trunk/src/java/org/apache/cassandra/db/Table.java
    cassandra/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
    
cassandra/trunk/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
    cassandra/trunk/src/java/org/apache/cassandra/db/filter/QueryFilter.java
    
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableNamesIterator.java
    
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableSliceIterator.java
    
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
    
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/RowIndexedReader.java
    
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/RowIndexedScanner.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableTracker.java

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/BinaryMemtable.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/BinaryMemtable.java?rev=924108&r1=924107&r2=924108&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/BinaryMemtable.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/BinaryMemtable.java Wed 
Mar 17 02:31:48 2010
@@ -68,7 +68,7 @@ public class BinaryMemtable implements I
      * the memtable. This version will respect the threshold and flush
      * the memtable to disk when the size exceeds the threshold.
     */
-    void put(String key, byte[] buffer) throws IOException
+    void put(String key, byte[] buffer)
     {
         if (isThresholdViolated())
         {

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=924108&r1=924107&r2=924108&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Wed 
Mar 17 02:31:48 2010
@@ -341,7 +341,7 @@ public class ColumnFamilyStore implement
     }
 
     /** flush the given memtable and swap in a new one for its CFS, if it 
hasn't been frozen already.  threadsafe. */
-    Future<?> maybeSwitchMemtable(Memtable oldMemtable, final boolean 
writeCommitLog) throws IOException
+    Future<?> maybeSwitchMemtable(Memtable oldMemtable, final boolean 
writeCommitLog)
     {
         /**
          *  If we can get the writelock, that means no new updates can come in 
and 
@@ -389,19 +389,19 @@ public class ColumnFamilyStore implement
         }
     }
 
-    void switchBinaryMemtable(String key, byte[] buffer) throws IOException
+    void switchBinaryMemtable(String key, byte[] buffer)
     {
         binaryMemtable_.set(new BinaryMemtable(this));
         binaryMemtable_.get().put(key, buffer);
     }
 
-    public void forceFlushIfExpired() throws IOException
+    public void forceFlushIfExpired()
     {
         if (memtable_.isExpired())
             forceFlush();
     }
 
-    public Future<?> forceFlush() throws IOException
+    public Future<?> forceFlush()
     {
         if (memtable_.isClean())
             return null;
@@ -409,7 +409,7 @@ public class ColumnFamilyStore implement
         return maybeSwitchMemtable(memtable_, true);
     }
 
-    public void forceBlockingFlush() throws IOException, ExecutionException, 
InterruptedException
+    public void forceBlockingFlush() throws ExecutionException, 
InterruptedException
     {
         Future<?> future = forceFlush();
         if (future != null)
@@ -431,7 +431,7 @@ public class ColumnFamilyStore implement
      * param @ key - key for update/insert
      * param @ columnFamily - columnFamily changes
      */
-    Memtable apply(String key, ColumnFamily columnFamily) throws IOException
+    Memtable apply(String key, ColumnFamily columnFamily)
     {
         long start = System.nanoTime();
 
@@ -447,7 +447,7 @@ public class ColumnFamilyStore implement
      * needs to be used. param @ key - key for update/insert param @
      * columnFamily - columnFamily changes
      */
-    void applyBinary(String key, byte[] buffer) throws IOException
+    void applyBinary(String key, byte[] buffer)
     {
         long start = System.nanoTime();
         binaryMemtable_.get().put(key, buffer);
@@ -577,7 +577,7 @@ public class ColumnFamilyStore implement
         return Table.open(table_);
     }
 
-    void markCompacted(Collection<SSTableReader> sstables) throws IOException
+    void markCompacted(Collection<SSTableReader> sstables)
     {
         ssTables_.markCompacted(sstables);
     }
@@ -588,7 +588,6 @@ public class ColumnFamilyStore implement
     }
 
     void replaceCompactedSSTables(Collection<SSTableReader> sstables, 
Iterable<SSTableReader> replacements)
-            throws IOException
     {
         ssTables_.replace(sstables, replacements);
     }
@@ -707,22 +706,22 @@ public class ColumnFamilyStore implement
         return writeStats_.getRecentLatencyMicros();
     }
 
-    public ColumnFamily getColumnFamily(String key, QueryPath path, byte[] 
start, byte[] finish, List<byte[]> bitmasks, boolean reversed, int limit) 
throws IOException
+    public ColumnFamily getColumnFamily(String key, QueryPath path, byte[] 
start, byte[] finish, List<byte[]> bitmasks, boolean reversed, int limit)
     {
         return getColumnFamily(new SliceQueryFilter(key, path, start, finish, 
bitmasks, reversed, limit));
     }
 
-    public ColumnFamily getColumnFamily(String key, QueryPath path, byte[] 
start, byte[] finish, boolean reversed, int limit) throws IOException
+    public ColumnFamily getColumnFamily(String key, QueryPath path, byte[] 
start, byte[] finish, boolean reversed, int limit)
     {
         return getColumnFamily(new SliceQueryFilter(key, path, start, finish, 
reversed, limit));
     }
 
-    public ColumnFamily getColumnFamily(QueryFilter filter) throws IOException
+    public ColumnFamily getColumnFamily(QueryFilter filter)
     {
         return getColumnFamily(filter, CompactionManager.getDefaultGCBefore());
     }
 
-    private ColumnFamily cacheRow(String key) throws IOException
+    private ColumnFamily cacheRow(String key)
     {
         ColumnFamily cached;
         if ((cached = ssTables_.getRowCache().get(key)) == null)
@@ -740,7 +739,7 @@ public class ColumnFamilyStore implement
      * only the latest version of a column is returned.
      * @return null if there is no data and no tombstones; otherwise a 
ColumnFamily
      */
-    public ColumnFamily getColumnFamily(QueryFilter filter, int gcBefore) 
throws IOException
+    public ColumnFamily getColumnFamily(QueryFilter filter, int gcBefore)
     {
         assert columnFamily_.equals(filter.getColumnFamilyName());
 
@@ -794,7 +793,7 @@ public class ColumnFamilyStore implement
         }
     }
 
-    private ColumnFamily getTopLevelColumns(QueryFilter filter, int gcBefore) 
throws IOException
+    private ColumnFamily getTopLevelColumns(QueryFilter filter, int gcBefore)
     {
         // we are querying top-level columns, do a merging fetch with indexes.
         List<ColumnIterator> iterators = new ArrayList<ColumnIterator>();
@@ -873,7 +872,7 @@ public class ColumnFamilyStore implement
        would be better.
      */
     private boolean getKeyRange(List<String> keys, final AbstractBounds range, 
int maxResults)
-    throws IOException, ExecutionException, InterruptedException
+    throws ExecutionException, InterruptedException
     {
         final DecoratedKey startWith = new DecoratedKey(range.left, null);
         final DecoratedKey stopAt = new DecoratedKey(range.right, null);
@@ -973,7 +972,14 @@ public class ColumnFamilyStore implement
             {
                 if (iter instanceof Closeable)
                 {
-                    ((Closeable)iter).close();
+                    try
+                    {
+                        ((Closeable)iter).close();
+                    }
+                    catch (IOException e)
+                    {
+                        throw new IOError(e);
+                    }
                 }
             }
         }
@@ -986,12 +992,11 @@ public class ColumnFamilyStore implement
      * @param keyMax maximum number of keys to process, regardless of 
startKey/finishKey
      * @param sliceRange may be null if columnNames is specified. specifies 
contiguous columns to return in what order.
      * @param columnNames may be null if sliceRange is specified. specifies 
which columns to return in what order.      @return list of key->list<column> 
tuples.
-     * @throws IOException
      * @throws ExecutionException
      * @throws InterruptedException
      */
     public RangeSliceReply getRangeSlice(byte[] super_column, final 
AbstractBounds range, int keyMax, SliceRange sliceRange, List<byte[]> 
columnNames)
-    throws IOException, ExecutionException, InterruptedException
+    throws ExecutionException, InterruptedException
     {
         List<String> keys = new ArrayList<String>();
         boolean completed;
@@ -1035,7 +1040,7 @@ public class ColumnFamilyStore implement
      * 
      * @param snapshotName the name of the associated with the snapshot 
      */
-    public void snapshot(String snapshotName) throws IOException
+    public void snapshot(String snapshotName)
     {
         try
         {
@@ -1052,27 +1057,34 @@ public class ColumnFamilyStore implement
 
         for (SSTableReader ssTable : ssTables_)
         {
-            // mkdir
-            File sourceFile = new File(ssTable.getFilename());
-            File dataDirectory = sourceFile.getParentFile().getParentFile();
-            String snapshotDirectoryPath = 
Table.getSnapshotPath(dataDirectory.getAbsolutePath(), table_, snapshotName);
-            FileUtils.createDirectory(snapshotDirectoryPath);
-
-            // hard links
-            File targetLink = new File(snapshotDirectoryPath, 
sourceFile.getName());
-            FileUtils.createHardLink(sourceFile, targetLink);
-
-            sourceFile = new File(ssTable.indexFilename());
-            targetLink = new File(snapshotDirectoryPath, sourceFile.getName());
-            FileUtils.createHardLink(sourceFile, targetLink);
-
-            sourceFile = new File(ssTable.filterFilename());
-            targetLink = new File(snapshotDirectoryPath, sourceFile.getName());
-            FileUtils.createHardLink(sourceFile, targetLink);
-
-            if (logger_.isDebugEnabled())
-                logger_.debug("Snapshot for " + table_ + " table data file " + 
sourceFile.getAbsolutePath() +
-                    " created as " + targetLink.getAbsolutePath());
+            try
+            {
+                // mkdir
+                File sourceFile = new File(ssTable.getFilename());
+                File dataDirectory = 
sourceFile.getParentFile().getParentFile();
+                String snapshotDirectoryPath = 
Table.getSnapshotPath(dataDirectory.getAbsolutePath(), table_, snapshotName);
+                FileUtils.createDirectory(snapshotDirectoryPath);
+
+                // hard links
+                File targetLink = new File(snapshotDirectoryPath, 
sourceFile.getName());
+                FileUtils.createHardLink(sourceFile, targetLink);
+
+                sourceFile = new File(ssTable.indexFilename());
+                targetLink = new File(snapshotDirectoryPath, 
sourceFile.getName());
+                FileUtils.createHardLink(sourceFile, targetLink);
+
+                sourceFile = new File(ssTable.filterFilename());
+                targetLink = new File(snapshotDirectoryPath, 
sourceFile.getName());
+                FileUtils.createHardLink(sourceFile, targetLink);
+                if (logger_.isDebugEnabled())
+                    logger_.debug("Snapshot for " + table_ + " table data file 
" + sourceFile.getAbsolutePath() +
+                        " created as " + targetLink.getAbsolutePath());
+            }
+            catch (IOException e)
+            {
+                throw new IOError(e);
+            }
+
         }
     }
 

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/SystemTable.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/SystemTable.java?rev=924108&r1=924107&r2=924108&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/SystemTable.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/SystemTable.java Wed Mar 
17 02:31:48 2010
@@ -202,17 +202,10 @@ public class SystemTable
     public static boolean isBootstrapped()
     {
         Table table = null;
-        try
-        {
-            table = Table.open(Table.SYSTEM_TABLE);
-            QueryFilter filter = new NamesQueryFilter(BOOTSTRAP_KEY, new 
QueryPath(STATUS_CF), BOOTSTRAP);
-            ColumnFamily cf = 
table.getColumnFamilyStore(STATUS_CF).getColumnFamily(filter);
-            return cf != null && cf.getColumn(BOOTSTRAP).value()[0] == 1;
-        }
-        catch (IOException e)
-        {
-            throw new RuntimeException(e);
-        }
+        table = Table.open(Table.SYSTEM_TABLE);
+        QueryFilter filter = new NamesQueryFilter(BOOTSTRAP_KEY, new 
QueryPath(STATUS_CF), BOOTSTRAP);
+        ColumnFamily cf = 
table.getColumnFamilyStore(STATUS_CF).getColumnFamily(filter);
+        return cf != null && cf.getColumn(BOOTSTRAP).value()[0] == 1;
     }
 
     public static void setBootstrapped(boolean isBootstrapped)

Modified: cassandra/trunk/src/java/org/apache/cassandra/db/Table.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java?rev=924108&r1=924107&r2=924108&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/Table.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/Table.java Wed Mar 17 
02:31:48 2010
@@ -246,14 +246,7 @@ public class Table 
             {
                 for (ColumnFamilyStore cfs : columnFamilyStores.values())
                 {
-                    try
-                    {
-                        cfs.forceFlushIfExpired();
-                    }
-                    catch (IOException e)
-                    {
-                        throw new RuntimeException(e);
-                    }
+                    cfs.forceFlushIfExpired();
                 }
             }
         }, checkMs, checkMs);

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLog.java?rev=924108&r1=924107&r2=924108&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLog.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLog.java 
Wed Mar 17 02:31:48 2010
@@ -309,7 +309,7 @@ public class CommitLog
         return segments.getLast();
     }
     
-    public CommitLogSegment.CommitLogContext getContext() throws IOException
+    public CommitLogSegment.CommitLogContext getContext()
     {
         Callable<CommitLogSegment.CommitLogContext> task = new 
Callable<CommitLogSegment.CommitLogContext>()
         {

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java?rev=924108&r1=924107&r2=924108&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java 
Wed Mar 17 02:31:48 2010
@@ -64,7 +64,7 @@ public class NamesQueryFilter extends Qu
         return memtable.getNamesIterator(cf, this);
     }
 
-    public ColumnIterator getSSTableColumnIterator(SSTableReader sstable) 
throws IOException
+    public ColumnIterator getSSTableColumnIterator(SSTableReader sstable)
     {
         return new SSTableNamesIterator(sstable, key, columns);
     }

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/QueryFilter.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/filter/QueryFilter.java?rev=924108&r1=924107&r2=924108&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/db/filter/QueryFilter.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/db/filter/QueryFilter.java 
Wed Mar 17 02:31:48 2010
@@ -57,7 +57,7 @@ public abstract class QueryFilter
      * returns an iterator that returns columns from the given SSTable
      * matching the Filter criteria in sorted order.
      */
-    public abstract ColumnIterator getSSTableColumnIterator(SSTableReader 
sstable) throws IOException;
+    public abstract ColumnIterator getSSTableColumnIterator(SSTableReader 
sstable);
 
     /**
      * collects columns from reducedColumns into returnCF.  Termination is 
determined

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableNamesIterator.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableNamesIterator.java?rev=924108&r1=924107&r2=924108&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableNamesIterator.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableNamesIterator.java
 Wed Mar 17 02:31:48 2010
@@ -21,6 +21,7 @@ package org.apache.cassandra.db.filter;
  */
 
 
+import java.io.IOError;
 import java.io.IOException;
 import java.util.*;
 
@@ -33,26 +34,32 @@ import org.apache.cassandra.io.sstable.S
 import org.apache.cassandra.io.util.BufferedRandomAccessFile;
 import org.apache.cassandra.io.util.FileDataInput;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.BloomFilter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class SSTableNamesIterator extends SimpleAbstractColumnIterator
 {
+    private static Logger logger = 
LoggerFactory.getLogger(SSTableNamesIterator.class);
+
     private ColumnFamily cf;
     private Iterator<IColumn> iter;
     public final SortedSet<byte[]> columns;
 
-    public SSTableNamesIterator(SSTableReader ssTable, String key, 
SortedSet<byte[]> columnNames) throws IOException
+    public SSTableNamesIterator(SSTableReader ssTable, String key, 
SortedSet<byte[]> columnNames)
     {
         assert columnNames != null;
         this.columns = columnNames;
 
         DecoratedKey decoratedKey = ssTable.getPartitioner().decorateKey(key);
 
-        FileDataInput file = ssTable.getFileDataInput(decoratedKey, 
DatabaseDescriptor.getIndexedReadBufferSizeInKB() * 1024);
-        if (file == null)
-            return;
+        FileDataInput file = null;
         try
         {
+            file = ssTable.getFileDataInput(decoratedKey, 
DatabaseDescriptor.getIndexedReadBufferSizeInKB() * 1024);
+            if (file == null)
+                return;
             DecoratedKey keyInDisk = 
ssTable.getPartitioner().convertFromDiskFormat(file.readUTF());
             assert keyInDisk.equals(decoratedKey) : keyInDisk;
             file.readInt(); // data size
@@ -109,11 +116,22 @@ public class SSTableNamesIterator extend
                 }
             }
         }
+        catch (IOException e)
+        {
+           throw new IOError(e); 
+        }
         finally
         {
-            file.close();
+            try
+            {
+                if (file != null)
+                    file.close();
+            }
+            catch (IOException e)
+            {
+                logger.error("error closing file", e);
+            }
         }
-
         iter = cf.getSortedColumns().iterator();
     }
 

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableSliceIterator.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableSliceIterator.java?rev=924108&r1=924107&r2=924108&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableSliceIterator.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableSliceIterator.java
 Wed Mar 17 02:31:48 2010
@@ -22,6 +22,7 @@ package org.apache.cassandra.db.filter;
 
 
 import java.util.*;
+import java.io.IOError;
 import java.io.IOException;
 
 import org.apache.cassandra.db.DecoratedKey;
@@ -49,7 +50,6 @@ class SSTableSliceIterator extends Abstr
     private ColumnGroupReader reader;
 
     public SSTableSliceIterator(SSTableReader ssTable, String key, byte[] 
startColumn, byte[] finishColumn, Predicate<IColumn> predicate, boolean 
reversed)
-    throws IOException
     {
         this.reversed = reversed;
 
@@ -132,20 +132,25 @@ class SSTableSliceIterator extends Abstr
         private int curRangeIndex;
         private Deque<IColumn> blockColumns = new ArrayDeque<IColumn>();
 
-        public ColumnGroupReader(SSTableReader ssTable, DecoratedKey key, 
FileDataInput input) throws IOException
+        public ColumnGroupReader(SSTableReader ssTable, DecoratedKey key, 
FileDataInput input)
         {
             this.file = input;
-
-            DecoratedKey keyInDisk = 
ssTable.getPartitioner().convertFromDiskFormat(file.readUTF());
-            assert keyInDisk.equals(key);
-
-            file.readInt(); // row size
-            IndexHelper.skipBloomFilter(file);
-            indexes = IndexHelper.deserializeIndex(file);
-
-            emptyColumnFamily = 
ColumnFamily.serializer().deserializeFromSSTableNoColumns(ssTable.makeColumnFamily(),
 file);
-            file.readInt(); // column count
-
+            try
+            {
+                DecoratedKey keyInDisk = 
ssTable.getPartitioner().convertFromDiskFormat(file.readUTF());
+                assert keyInDisk.equals(key);
+    
+                file.readInt(); // row size
+                IndexHelper.skipBloomFilter(file);
+                indexes = IndexHelper.deserializeIndex(file);
+    
+                emptyColumnFamily = 
ColumnFamily.serializer().deserializeFromSSTableNoColumns(ssTable.makeColumnFamily(),
 file);
+                file.readInt(); // column count
+            }
+            catch (IOException e)
+            {
+                throw new IOError(e);
+            }
             file.mark();
             curRangeIndex = IndexHelper.indexFor(startColumn, indexes, 
comparator, reversed);
             if (reversed && curRangeIndex == indexes.size())

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java?rev=924108&r1=924107&r2=924108&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java 
Wed Mar 17 02:31:48 2010
@@ -68,7 +68,7 @@ public class SliceQueryFilter extends Qu
         return memtable.getSliceIterator(cf, this, comparator);
     }
 
-    public ColumnIterator getSSTableColumnIterator(SSTableReader sstable) 
throws IOException
+    public ColumnIterator getSSTableColumnIterator(SSTableReader sstable)
     {
         Predicate<IColumn> predicate = (bitmasks == null || bitmasks.isEmpty())
                                        ? Predicates.<IColumn>alwaysTrue()

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/RowIndexedReader.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/RowIndexedReader.java?rev=924108&r1=924107&r2=924108&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/RowIndexedReader.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/RowIndexedReader.java 
Wed Mar 17 02:31:48 2010
@@ -234,7 +234,7 @@ class RowIndexedReader extends SSTableRe
     /**
      * returns the position in the data file to find the given key, or -1 if 
the key is not present
      */
-    public PositionSize getPosition(DecoratedKey decoratedKey) throws 
IOException
+    public PositionSize getPosition(DecoratedKey decoratedKey)
     {
         // first, check bloom filter
         if (!bf.isPresent(partitioner.convertToDiskFormat(decoratedKey)))
@@ -268,18 +268,18 @@ class RowIndexedReader extends SSTableRe
 
         // scan the on-disk index, starting at the nearest sampled position
         long p = sampledPosition.position;
-        FileDataInput input;
-        if (indexBuffers == null)
-        {
-            input = new BufferedRandomAccessFile(indexFilename(), "r");
-            ((BufferedRandomAccessFile)input).seek(p);
-        }
-        else
-        {
-            input = new MappedFileDataInput(indexBuffers[bufferIndex(p)], 
indexFilename(), (int)(p % BUFFER_SIZE));
-        }
+        FileDataInput input = null;
         try
         {
+            if (indexBuffers == null)
+            {
+                input = new BufferedRandomAccessFile(indexFilename(), "r");
+                ((BufferedRandomAccessFile)input).seek(p);
+            }
+            else
+            {
+                input = new MappedFileDataInput(indexBuffers[bufferIndex(p)], 
indexFilename(), (int)(p % BUFFER_SIZE));
+            }
             int i = 0;
             do
             {
@@ -316,9 +316,21 @@ class RowIndexedReader extends SSTableRe
                     return null;
             } while  (++i < INDEX_INTERVAL);
         }
+        catch (IOException e)
+        {
+            throw new IOError(e);
+        }
         finally
         {
-            input.close();
+            try
+            {
+                if (input != null)
+                    input.close();
+            }
+            catch (IOException e)
+            {
+                logger.error("error closing file", e);
+            }
         }
         return null;
     }
@@ -375,12 +387,12 @@ class RowIndexedReader extends SSTableRe
         bf = BloomFilter.alwaysMatchingBloomFilter();
     }
 
-    public SSTableScanner getScanner(int bufferSize) throws IOException
+    public SSTableScanner getScanner(int bufferSize)
     {
         return new RowIndexedScanner(this, bufferSize);
     }
 
-    public FileDataInput getFileDataInput(DecoratedKey decoratedKey, int 
bufferSize) throws IOException
+    public FileDataInput getFileDataInput(DecoratedKey decoratedKey, int 
bufferSize)
     {
         PositionSize info = getPosition(decoratedKey);
         if (info == null)
@@ -388,9 +400,16 @@ class RowIndexedReader extends SSTableRe
 
         if (buffers == null || (bufferIndex(info.position) != 
bufferIndex(info.position + info.size)))
         {
-            BufferedRandomAccessFile file = new 
BufferedRandomAccessFile(getFilename(), "r", bufferSize);
-            file.seek(info.position);
-            return file;
+            try
+            {
+                BufferedRandomAccessFile file = new 
BufferedRandomAccessFile(getFilename(), "r", bufferSize);
+                file.seek(info.position);
+                return file;
+            }
+            catch (IOException e)
+            {
+                throw new IOError(e);
+            }
         }
         return new MappedFileDataInput(buffers[bufferIndex(info.position)], 
getFilename(), (int) (info.position % BUFFER_SIZE));
     }

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/RowIndexedScanner.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/RowIndexedScanner.java?rev=924108&r1=924107&r2=924108&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/RowIndexedScanner.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/RowIndexedScanner.java 
Wed Mar 17 02:31:48 2010
@@ -47,9 +47,16 @@ public class RowIndexedScanner extends S
     /**
      * @param sstable SSTable to scan.
      */
-    RowIndexedScanner(SSTableReader sstable, int bufferSize) throws IOException
+    RowIndexedScanner(SSTableReader sstable, int bufferSize)
     {
-        this.file = new BufferedRandomAccessFile(sstable.getFilename(), "r", 
bufferSize);
+        try
+        {
+            this.file = new BufferedRandomAccessFile(sstable.getFilename(), 
"r", bufferSize);
+        }
+        catch (IOException e)
+        {
+            throw new IOError(e);
+        }
         this.sstable = sstable;
     }
 

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java?rev=924108&r1=924107&r2=924108&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java 
Wed Mar 17 02:31:48 2010
@@ -226,13 +226,18 @@ public abstract class SSTableReader exte
      */
     public abstract long length();
 
-    public void markCompacted() throws IOException
+    public void markCompacted()
     {
         if (logger.isDebugEnabled())
             logger.debug("Marking " + getFilename() + " compacted");
-        if (!new File(compactedFilename()).createNewFile())
+        try
         {
-            throw new IOException("Unable to create compaction marker");
+            if (!new File(compactedFilename()).createNewFile())
+                throw new IOException("Unable to create compaction marker");
+        }
+        catch (IOException e)
+        {
+            throw new IOError(e);
         }
         phantomReference.deleteOnCleanup();
     }
@@ -241,13 +246,13 @@ public abstract class SSTableReader exte
      * @param bufferSize Buffer size in bytes for this Scanner.
      * @return A Scanner for seeking over the rows of the SSTable.
      */
-    public abstract SSTableScanner getScanner(int bufferSize) throws 
IOException;
+    public abstract SSTableScanner getScanner(int bufferSize);
 
     /**
      * FIXME: should not be public: use Scanner.
      */
     @Deprecated
-    public abstract FileDataInput getFileDataInput(DecoratedKey decoratedKey, 
int bufferSize) throws IOException;
+    public abstract FileDataInput getFileDataInput(DecoratedKey decoratedKey, 
int bufferSize);
 
     public AbstractType getColumnComparator()
     {

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableTracker.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableTracker.java?rev=924108&r1=924107&r2=924108&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableTracker.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableTracker.java 
Wed Mar 17 02:31:48 2010
@@ -55,7 +55,7 @@ public class SSTableTracker implements I
         rowCache = new JMXInstrumentedCache<String, ColumnFamily>(ksname, 
cfname + "RowCache", 0);
     }
 
-    public synchronized void replace(Collection<SSTableReader> oldSSTables, 
Iterable<SSTableReader> replacements) throws IOException
+    public synchronized void replace(Collection<SSTableReader> oldSSTables, 
Iterable<SSTableReader> replacements)
     {
         Set<SSTableReader> sstablesNew = new HashSet<SSTableReader>(sstables);
 
@@ -84,17 +84,10 @@ public class SSTableTracker implements I
     public synchronized void add(Iterable<SSTableReader> sstables)
     {
         assert sstables != null;
-        try
-        {
-            replace(Collections.<SSTableReader>emptyList(), sstables);
-        }
-        catch (IOException e)
-        {
-            throw new AssertionError(e);
-        }
+        replace(Collections.<SSTableReader>emptyList(), sstables);
     }
 
-    public synchronized void markCompacted(Collection<SSTableReader> 
compacted) throws IOException
+    public synchronized void markCompacted(Collection<SSTableReader> compacted)
     {
         replace(compacted, Collections.<SSTableReader>emptyList());
     }


Reply via email to