Author: jbellis
Date: Wed May 11 16:03:12 2011
New Revision: 1101933
URL: http://svn.apache.org/viewvc?rev=1101933&view=rev
Log:
avoid using cached position of a keywhen GTis requested
patch by Richard Low and slebresne; reviewed by jbellis for CASSANDRA-2633
Modified:
cassandra/branches/cassandra-0.7/CHANGES.txt
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
Modified: cassandra/branches/cassandra-0.7/CHANGES.txt
URL:
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/CHANGES.txt?rev=1101933&r1=1101932&r2=1101933&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/CHANGES.txt (original)
+++ cassandra/branches/cassandra-0.7/CHANGES.txt Wed May 11 16:03:12 2011
@@ -19,6 +19,7 @@
(CASSANDRA-2625)
* Allow removing LocationInfo sstables (CASSANDRA-2632)
* avoid attempting to replay mutations from dropped keyspaces (CASSANDRA-2631)
+ * avoid using cached position of a key when GT is requested (CASSANDRA-2633)
0.7.5
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=1101933&r1=1101932&r2=1101933&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
Wed May 11 16:03:12 2011
@@ -451,10 +451,13 @@ public class SSTableReader extends SSTab
}
// next, the key cache
- Pair<Descriptor, DecoratedKey> unifiedKey = new Pair<Descriptor,
DecoratedKey>(descriptor, decoratedKey);
- Long cachedPosition = getCachedPosition(unifiedKey);
- if (cachedPosition != null)
- return cachedPosition;
+ if (op == Operator.EQ || op == Operator.GE)
+ {
+ Pair<Descriptor, DecoratedKey> unifiedKey = new Pair<Descriptor,
DecoratedKey>(descriptor, decoratedKey);
+ Long cachedPosition = getCachedPosition(unifiedKey);
+ if (cachedPosition != null)
+ return cachedPosition;
+ }
// next, see if the sampled index says it's impossible for the key to
be present
IndexSummary.KeyPosition sampledPosition =
getIndexScanPosition(decoratedKey);
Modified:
cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
URL:
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java?rev=1101933&r1=1101932&r2=1101933&view=diff
==============================================================================
---
cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
(original)
+++
cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
Wed May 11 16:03:12 2011
@@ -25,6 +25,7 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.concurrent.ExecutionException;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.List;
import org.junit.Test;
@@ -148,4 +149,48 @@ public class SSTableReaderTest extends C
store.forceBlockingFlush();
assert store.getMaxRowSize() != 0;
}
+
+ @Test
+ public void testGetPositionsForRangesWithKeyCache() throws IOException,
ExecutionException, InterruptedException
+ {
+ Table table = Table.open("Keyspace1");
+ ColumnFamilyStore store = table.getColumnFamilyStore("Standard2");
+ store.getKeyCache().setCapacity(100);
+
+ // insert data and compact to a single sstable
+ CompactionManager.instance.disableAutoCompaction();
+ for (int j = 0; j < 10; j++)
+ {
+ ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
+ RowMutation rm = new RowMutation("Keyspace1", key);
+ rm.add(new QueryPath("Standard2", null,
ByteBufferUtil.bytes("0")), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
+ rm.apply();
+ }
+ store.forceBlockingFlush();
+ CompactionManager.instance.performMajor(store);
+
+ SSTableReader sstable = store.getSSTables().iterator().next();
+ long p2 = sstable.getPosition(k(2), SSTableReader.Operator.EQ);
+ long p3 = sstable.getPosition(k(3), SSTableReader.Operator.EQ);
+ long p6 = sstable.getPosition(k(6), SSTableReader.Operator.EQ);
+ long p7 = sstable.getPosition(k(7), SSTableReader.Operator.EQ);
+
+ Pair<Long, Long> p = sstable.getPositionsForRanges(makeRanges(t(2),
t(6))).iterator().next();
+
+ // range are start exclusive so we should start at 3
+ assert p.left == p3;
+
+ // to capture 6 we have to stop at the start of 7
+ assert p.right == p7;
+ }
+
+ private List<Range> makeRanges(Token left, Token right)
+ {
+ return Arrays.asList(new Range[]{ new Range(left, right) });
+ }
+
+ private DecoratedKey k(int i)
+ {
+ return new DecoratedKey(t(i), ByteBufferUtil.bytes(String.valueOf(i)));
+ }
}