cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbel...@apache.org
Subject svn commit: r791589 - in /incubator/cassandra/trunk/src/java/org/apache/cassandra: db/ColumnFamilyStore.java io/SSTable.java
Date Mon, 06 Jul 2009 19:57:44 GMT
Author: jbellis
Date: Mon Jul  6 19:57:44 2009
New Revision: 791589

URL: http://svn.apache.org/viewvc?rev=791589&view=rev
Log:
encapsulate bloom filter access into sstable.getPosition
patch by jbellis; reviewed by Jun Rao for CASSANDRA-259

Modified:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTable.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=791589&r1=791588&r2=791589&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 Mon
Jul  6 19:57:44 2009
@@ -551,15 +551,6 @@
         {
             for (SSTable sstable : ssTables_.values())
             {
-                /*
-                 * Get the BloomFilter associated with this file. Check if the key
-                 * is present in the BloomFilter. If not continue to the next file.
-                */
-                boolean bVal = sstable.isKeyPossible(key);
-                if (!bVal)
-                {
-                    continue;
-                }
                 ColumnFamily columnFamily = null;
                 try
                 {
@@ -567,6 +558,7 @@
                 }
                 catch (IOException e)
                 {
+                    // annotate exception w/ more information about context
                     throw new IOException("Error fetching " + key + ":" + cf + " from " +
sstable, e);
                 }
                 if (columnFamily != null)
@@ -1571,9 +1563,6 @@
             List<SSTable> sstables = new ArrayList<SSTable>(ssTables_.values());
             for (SSTable sstable : sstables)
             {
-                // If the key is not present in the SSTable's BloomFilter, continue to the
next file
-                if (!sstable.isKeyPossible(key))
-                    continue;
                 iter = new SSTableColumnIterator(sstable.getFilename(), key, cfName, startColumn,
isAscending);
                 if (iter.hasNext())
                 {

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=791589&r1=791588&r2=791589&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 Mon Jul  6 19:57:44
2009
@@ -292,11 +292,14 @@
      */
     public long getPosition(String decoratedKey, IPartitioner partitioner) throws IOException
     {
+        if (!bf.isPresent(decoratedKey))
+            return -1;
         long start = getIndexScanPosition(decoratedKey, partitioner);
         if (start < 0)
         {
             return -1;
         }
+
         // TODO mmap the index file?
         BufferedRandomAccessFile input = new BufferedRandomAccessFile(indexFilename(dataFile_),
"r");
         input.seek(start);
@@ -488,11 +491,6 @@
         }
     }
 
-    public boolean isKeyPossible(String clientKey)
-    {
-        return bf.isPresent(partitioner_.decorateKey(clientKey));
-    }
-
     IPartitioner getPartitioner()
     {
         return partitioner_;



Mime
View raw message