only deserialize keys we need in the sample patch by yukim; reviewed by jbellis for CASSANDRA-4023
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/1e18538c Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/1e18538c Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/1e18538c Branch: refs/heads/trunk Commit: 1e18538c9868e92622e9e0acf87b2e88b9bd6182 Parents: fbdf7b0 Author: Jonathan Ellis <[email protected]> Authored: Wed Mar 21 16:33:02 2012 -0500 Committer: Jonathan Ellis <[email protected]> Committed: Fri Mar 23 09:54:27 2012 -0500 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../apache/cassandra/io/sstable/SSTableReader.java | 25 +++++--- .../org/apache/cassandra/utils/ByteBufferUtil.java | 4 +- .../cassandra/io/sstable/SSTableReaderTest.java | 44 ++++++++++++++- 4 files changed, 59 insertions(+), 15 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e18538c/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index c1e1cfe..74522ef 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 1.0.9 + * improve index sampling performance (CASSANDRA-4023) * always compact away deleted hints immediately after handoff (CASSANDRA-3955) * delete hints from dropped ColumnFamilies on handoff instead of erroring out (CASSANDRA-3975) http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e18538c/src/java/org/apache/cassandra/io/sstable/SSTableReader.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java index 10f868b..f460725 100644 --- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java +++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java @@ -364,23 +364,28 @@ public class SSTableReader extends SSTable if (indexPosition == indexSize) break; - ByteBuffer key = null, skippedKey; - skippedKey = ByteBufferUtil.readWithShortLength(input); + DecoratedKey decoratedKey = null; + int len = ByteBufferUtil.readShortLength(input); + boolean firstKey = left == null; + boolean lastKey = indexPosition + DBConstants.shortSize + len + DBConstants.longSize == indexSize; boolean shouldAddEntry = indexSummary.shouldAddEntry(); - if (shouldAddEntry || cacheLoading || recreatebloom) + if (shouldAddEntry || cacheLoading || recreatebloom || firstKey || lastKey) { - key = skippedKey; + decoratedKey = decodeKey(partitioner, descriptor, ByteBufferUtil.read(input, len)); + if (firstKey) + left = decoratedKey; + if (lastKey) + right = decoratedKey; + } + else + { + FileUtils.skipBytesFully(input, len); } - - if(null == left) - left = decodeKey(partitioner, descriptor, skippedKey); - right = decodeKey(partitioner, descriptor, skippedKey); long dataPosition = input.readLong(); - if (key != null) + if (decoratedKey != null) { - DecoratedKey decoratedKey = decodeKey(partitioner, descriptor, key); if (recreatebloom) bf.add(decoratedKey.key); if (shouldAddEntry) http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e18538c/src/java/org/apache/cassandra/utils/ByteBufferUtil.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java index 8d33155..c584205 100644 --- a/src/java/org/apache/cassandra/utils/ByteBufferUtil.java +++ b/src/java/org/apache/cassandra/utils/ByteBufferUtil.java @@ -355,7 +355,7 @@ public class ByteBufferUtil } /* @return An unsigned short in an integer. */ - private static int readShortLength(DataInput in) throws IOException + public static int readShortLength(DataInput in) throws IOException { int length = (in.readByte() & 0xFF) << 8; return length | (in.readByte() & 0xFF); @@ -383,7 +383,7 @@ public class ByteBufferUtil return null; } - private static ByteBuffer read(DataInput in, int length) throws IOException + public static ByteBuffer read(DataInput in, int length) throws IOException { if (in instanceof FileDataInput) return ((FileDataInput) in).readBytes(length); http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e18538c/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java ---------------------------------------------------------------------- diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java index aff92cf..5acfd56 100644 --- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java +++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java @@ -24,10 +24,8 @@ package org.apache.cassandra.io.sstable; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.*; import java.util.concurrent.ExecutionException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; import org.junit.Test; @@ -243,6 +241,46 @@ public class SSTableReaderTest extends CleanupHelper assertIndexQueryWorks(store); } + @Test + public void testOpeningSSTable() throws Exception + { + String ks = "Keyspace1"; + String cf = "Standard1"; + + // clear and create just one sstable for this test + Table table = Table.open(ks); + ColumnFamilyStore store = table.getColumnFamilyStore(cf); + store.clearUnsafe(); + store.disableAutoCompaction(); + + DecoratedKey firstKey = null, lastKey = null; + long timestamp = System.currentTimeMillis(); + for (int i = 0; i < DatabaseDescriptor.getIndexInterval(); i++) { + DecoratedKey key = Util.dk(String.valueOf(i)); + if (firstKey == null) + firstKey = key; + if (lastKey == null) + lastKey = key; + if (store.metadata.getKeyValidator().compare(lastKey.key, key.key) < 0) + lastKey = key; + RowMutation rm = new RowMutation(ks, key.key); + rm.add(new QueryPath(cf, null, ByteBufferUtil.bytes("col")), + ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp); + rm.apply(); + } + store.forceBlockingFlush(); + + SSTableReader sstable = store.getSSTables().iterator().next(); + Descriptor desc = sstable.descriptor; + + // test to see if sstable can be opened as expected + SSTableReader target = SSTableReader.open(desc); + Collection<DecoratedKey> keySamples = target.getKeySamples(); + assert keySamples.size() == 1 && keySamples.iterator().next().equals(firstKey); + assert target.first.equals(firstKey); + assert target.last.equals(lastKey); + } + private void assertIndexQueryWorks(ColumnFamilyStore indexedCFS) { assert "Indexed1".equals(indexedCFS.getColumnFamilyName());
