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 2010/07/15 05:00:39 UTC

svn commit: r964288 - in /cassandra/trunk/src/java/org/apache/cassandra/io/sstable: SSTableReader.java SSTableWriter.java

Author: jbellis
Date: Thu Jul 15 03:00:38 2010
New Revision: 964288

URL: http://svn.apache.org/viewvc?rev=964288&view=rev
Log:
always use raw byte key in bloom filter.  patch by Stu Hood; reviewed by jbellis for CASSANDRA-1249

Modified:
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java

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=964288&r1=964287&r2=964288&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 Thu Jul 15 03:00:38 2010
@@ -380,7 +380,7 @@ public class SSTableReader extends SSTab
     public long getPosition(DecoratedKey decoratedKey, Operator op)
     {
         // first, check bloom filter
-        if (op == Operator.EQ && !bf.isPresent(partitioner.convertToDiskFormat(decoratedKey)))
+        if (op == Operator.EQ && !bf.isPresent(decoratedKey.key))
             return -1;
 
         // next, the key cache

Modified: cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java?rev=964288&r1=964287&r2=964288&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java (original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java Thu Jul 15 03:00:38 2010
@@ -299,10 +299,9 @@ public class SSTableWriter extends SSTab
 
         public void afterAppend(DecoratedKey key, long dataPosition) throws IOException
         {
-            byte[] diskKey = partitioner.convertToDiskFormat(key);
-            bf.add(diskKey);
+            bf.add(key.key);
             long indexPosition = indexFile.getFilePointer();
-            FBUtilities.writeShortByteArray(diskKey, indexFile);
+            FBUtilities.writeShortByteArray(partitioner.convertToDiskFormat(key), indexFile);
             indexFile.writeLong(dataPosition);
             if (logger.isTraceEnabled())
                 logger.trace("wrote index of " + key + " at " + indexPosition);