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 2012/03/23 16:02:58 UTC

[7/12] git commit: only deserialize keys we need in the sample patch by yukim; reviewed by jbellis for CASSANDRA-4023

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/cassandra-1.1
Commit: 1e18538c9868e92622e9e0acf87b2e88b9bd6182
Parents: fbdf7b0
Author: Jonathan Ellis <jb...@apache.org>
Authored: Wed Mar 21 16:33:02 2012 -0500
Committer: Jonathan Ellis <jb...@apache.org>
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());