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 2011/02/10 19:40:52 UTC
svn commit: r1069517 - in
/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra:
db/ColumnFamilyStore.java io/sstable/SSTableReader.java
Author: jbellis
Date: Thu Feb 10 18:40:52 2011
New Revision: 1069517
URL: http://svn.apache.org/viewvc?rev=1069517&view=rev
Log:
fix NPE from #2102
Modified:
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=1069517&r1=1069516&r2=1069517&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Thu Feb 10 18:40:52 2011
@@ -1149,7 +1149,7 @@ public class ColumnFamilyStore implement
}
}
- // TFFT resuses the underlying buffer for the key
+ // avoid keeping a permanent reference to the original key buffer
ssTables.getRowCache().put(new DecoratedKey(key.token, ByteBufferUtil.clone(key.key)), cached);
}
return cached;
Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java?rev=1069517&r1=1069516&r2=1069517&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java (original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java Thu Feb 10 18:40:52 2011
@@ -417,8 +417,9 @@ public class SSTableReader extends SSTab
public void cacheKey(DecoratedKey key, Long info)
{
- //TFFT reuses the underlying buffer for the key
- keyCache.put(new Pair<Descriptor, DecoratedKey>(descriptor, new DecoratedKey(key.token, ByteBufferUtil.clone(key.key))), info);
+ // avoid keeping a permanent reference to the original key buffer
+ DecoratedKey copiedKey = new DecoratedKey(key.token, key.key == null ? null : ByteBufferUtil.clone(key.key));
+ keyCache.put(new Pair<Descriptor, DecoratedKey>(descriptor, copiedKey), info);
}
public Long getCachedPosition(DecoratedKey key)
@@ -441,8 +442,12 @@ public class SSTableReader extends SSTab
public long getPosition(DecoratedKey decoratedKey, Operator op)
{
// first, check bloom filter
- if (op == Operator.EQ && !bf.isPresent(decoratedKey.key))
- return -1;
+ if (op == Operator.EQ)
+ {
+ assert decoratedKey.key != null; // null is ok for GE scans
+ if (!bf.isPresent(decoratedKey.key))
+ return -1;
+ }
// next, the key cache
Pair<Descriptor, DecoratedKey> unifiedKey = new Pair<Descriptor, DecoratedKey>(descriptor, decoratedKey);
@@ -482,7 +487,7 @@ public class SSTableReader extends SSTab
if (op == Operator.EQ)
bloomFilterTracker.addTruePositive();
// store exact match for the key
- cacheKey(unifiedKey.right, dataPosition);
+ cacheKey(decoratedKey, dataPosition);
}
return dataPosition;
}