You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2014/12/24 14:09:37 UTC

[1/2] cassandra git commit: Increase bloom filter true positive count on key cache hit

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 5cff788ef -> b53fefa73


Increase bloom filter true positive count on key cache hit

Patch by marcuse; reviewed by thobbs for CASSANDRA-8525


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f981bd59
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f981bd59
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f981bd59

Branch: refs/heads/cassandra-2.1
Commit: f981bd5969ad677e3ae6cda832075d94245acd4c
Parents: 5f54285
Author: Marcus Eriksson <ma...@apache.org>
Authored: Mon Dec 22 09:00:04 2014 +0100
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Wed Dec 24 13:57:00 2014 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/io/sstable/SSTableReader.java     |  3 ++
 .../cassandra/io/sstable/SSTableReaderTest.java | 36 +++++++++++++++++++-
 3 files changed, 39 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/f981bd59/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 57ab5b4..c8cf1d4 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.0.12:
+ * Increase bf true positive count on key cache hit (CASSANDRA-8525)
  * Move MeteredFlusher to its own thread (CASSANDRA-8485)
  * Fix non-distinct results in DISTNCT queries on static columns when
    paging is enabled (CASSANDRA-8087)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f981bd59/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 f0e9e65..803c7fa 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
@@ -887,7 +887,10 @@ public class SSTableReader extends SSTable implements Closeable
                 RowIndexEntry cachedEntry = keyCache.get(unifiedKey);
                 keyCacheRequest.incrementAndGet();
                 if (cachedEntry != null)
+                {
                     keyCacheHit.incrementAndGet();
+                    bloomFilterTracker.addTruePositive();
+                }
                 return cachedEntry;
             }
             else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/f981bd59/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 427d089..e49e564 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -56,6 +56,7 @@ import org.apache.cassandra.thrift.IndexExpression;
 import org.apache.cassandra.thrift.IndexOperator;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.Pair;
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
@@ -204,6 +205,39 @@ public class SSTableReaderTest extends SchemaLoader
     }
 
     @Test
+    public void testGetPositionsKeyCacheStats() throws IOException, ExecutionException, InterruptedException
+    {
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
+        CacheService.instance.keyCache.setCapacity(1000);
+
+        // 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("Standard2", ByteBufferUtil.bytes("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
+            rm.apply();
+        }
+        store.forceBlockingFlush();
+        CompactionManager.instance.performMaximal(store);
+
+        SSTableReader sstable = store.getSSTables().iterator().next();
+        sstable.getPosition(k(2), SSTableReader.Operator.EQ);
+        assertEquals(0, sstable.getKeyCacheHit());
+        assertEquals(1, sstable.getBloomFilterTruePositiveCount());
+        sstable.getPosition(k(2), SSTableReader.Operator.EQ);
+        assertEquals(1, sstable.getKeyCacheHit());
+        assertEquals(2, sstable.getBloomFilterTruePositiveCount());
+        sstable.getPosition(k(15), SSTableReader.Operator.EQ);
+        assertEquals(1, sstable.getKeyCacheHit());
+        assertEquals(2, sstable.getBloomFilterTruePositiveCount());
+
+    }
+
+
+    @Test
     public void testPersistentStatisticsWithSecondaryIndex() throws IOException, ExecutionException, InterruptedException
     {
         // Create secondary index and flush to disk
@@ -254,7 +288,7 @@ public class SSTableReaderTest extends SchemaLoader
 
         // test to see if sstable can be opened as expected
         SSTableReader target = SSTableReader.open(desc);
-        Assert.assertEquals(target.getKeySampleSize(), 1);
+        assertEquals(target.getKeySampleSize(), 1);
         Assert.assertArrayEquals(ByteBufferUtil.getArray(firstKey.key), target.getKeySample(0));
         assert target.first.equals(firstKey);
         assert target.last.equals(lastKey);


[2/2] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by ma...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1

Conflicts:
	CHANGES.txt
	test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b53fefa7
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b53fefa7
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b53fefa7

Branch: refs/heads/cassandra-2.1
Commit: b53fefa731e326391573e7d24af4420c2a79c303
Parents: 5cff788 f981bd5
Author: Marcus Eriksson <ma...@apache.org>
Authored: Wed Dec 24 14:00:05 2014 +0100
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Wed Dec 24 14:00:05 2014 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../cassandra/io/sstable/SSTableReader.java     |  3 ++
 .../cassandra/io/sstable/SSTableReaderTest.java | 32 +++++++++++++++++++-
 3 files changed, 35 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b53fefa7/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 64bdb62,c8cf1d4..ec64aa9
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,47 -1,5 +1,48 @@@
 -2.0.12:
 +2.1.3
 + * Stress - user defined writes should populate sequentally (CASSANDRA-8524)
 + * Fix regression in SSTableRewriter causing some rows to become unreadable 
 +   during compaction (CASSANDRA-8429)
 + * Run major compactions for repaired/unrepaired in parallel (CASSANDRA-8510)
 + * (cqlsh) Fix compression options in DESCRIBE TABLE output when compression
 +   is disabled (CASSANDRA-8288)
 + * (cqlsh) Fix DESCRIBE output after keyspaces are altered (CASSANDRA-7623)
 + * Make sure we set lastCompactedKey correctly (CASSANDRA-8463)
 + * (cqlsh) Fix output of CONSISTENCY command (CASSANDRA-8507)
 + * (cqlsh) Fixed the handling of LIST statements (CASSANDRA-8370)
 + * Make sstablescrub check leveled manifest again (CASSANDRA-8432)
 + * Check first/last keys in sstable when giving out positions (CASSANDRA-8458)
 + * Disable mmap on Windows (CASSANDRA-6993)
 + * Add missing ConsistencyLevels to cassandra-stress (CASSANDRA-8253)
 + * Add auth support to cassandra-stress (CASSANDRA-7985)
 + * Fix ArrayIndexOutOfBoundsException when generating error message
 +   for some CQL syntax errors (CASSANDRA-8455)
 + * Scale memtable slab allocation logarithmically (CASSANDRA-7882)
 + * cassandra-stress simultaneous inserts over same seed (CASSANDRA-7964)
 + * Reduce cassandra-stress sampling memory requirements (CASSANDRA-7926)
 + * Ensure memtable flush cannot expire commit log entries from its future (CASSANDRA-8383)
 + * Make read "defrag" async to reclaim memtables (CASSANDRA-8459)
 + * Remove tmplink files for offline compactions (CASSANDRA-8321)
 + * Reduce maxHintsInProgress (CASSANDRA-8415)
 + * BTree updates may call provided update function twice (CASSANDRA-8018)
 + * Release sstable references after anticompaction (CASSANDRA-8386)
 + * Handle abort() in SSTableRewriter properly (CASSANDRA-8320)
 + * Fix high size calculations for prepared statements (CASSANDRA-8231)
 + * Centralize shared executors (CASSANDRA-8055)
 + * Fix filtering for CONTAINS (KEY) relations on frozen collection
 +   clustering columns when the query is restricted to a single
 +   partition (CASSANDRA-8203)
 + * Do more aggressive entire-sstable TTL expiry checks (CASSANDRA-8243)
 + * Add more log info if readMeter is null (CASSANDRA-8238)
 + * add check of the system wall clock time at startup (CASSANDRA-8305)
 + * Support for frozen collections (CASSANDRA-7859)
 + * Fix overflow on histogram computation (CASSANDRA-8028)
 + * Have paxos reuse the timestamp generation of normal queries (CASSANDRA-7801)
 + * Fix incremental repair not remove parent session on remote (CASSANDRA-8291)
 + * Improve JBOD disk utilization (CASSANDRA-7386)
 + * Log failed host when preparing incremental repair (CASSANDRA-8228)
 + * Force config client mode in CQLSSTableWriter (CASSANDRA-8281)
 +Merged from 2.0:
+  * Increase bf true positive count on key cache hit (CASSANDRA-8525)
   * Move MeteredFlusher to its own thread (CASSANDRA-8485)
   * Fix non-distinct results in DISTNCT queries on static columns when
     paging is enabled (CASSANDRA-8087)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b53fefa7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b53fefa7/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index 03b5553,e49e564..51588f2
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@@ -64,14 -50,13 +64,13 @@@ import org.apache.cassandra.dht.Token
  import org.apache.cassandra.io.util.FileDataInput;
  import org.apache.cassandra.io.util.MmappedSegmentedFile;
  import org.apache.cassandra.io.util.SegmentedFile;
 +import org.apache.cassandra.service.CacheService;
  import org.apache.cassandra.service.StorageService;
 -import org.apache.cassandra.thrift.IndexExpression;
 -import org.apache.cassandra.thrift.IndexOperator;
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.Pair;
- 
 +import static org.apache.cassandra.Util.cellname;
  import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertFalse;
  import static org.junit.Assert.assertTrue;
  
  @RunWith(OrderedJUnit4ClassRunner.class)
@@@ -250,22 -205,55 +249,53 @@@ public class SSTableReaderTest extends 
      }
  
      @Test
 +    public void testPersistentStatisticsWithSecondaryIndex()
 +    {
 +        // Create secondary index and flush to disk
 +        Keyspace keyspace = Keyspace.open("Keyspace1");
 +        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Indexed1");
 +        ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k1"));
 +        Mutation rm = new Mutation("Keyspace1", key);
 +        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), System.currentTimeMillis());
 +        rm.apply();
 +        store.forceBlockingFlush();
 +
 +        // check if opening and querying works
 +        assertIndexQueryWorks(store);
 +    }
+     public void testGetPositionsKeyCacheStats() throws IOException, ExecutionException, InterruptedException
+     {
+         Keyspace keyspace = Keyspace.open("Keyspace1");
+         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
+         CacheService.instance.keyCache.setCapacity(1000);
+ 
+         // 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("Standard2", ByteBufferUtil.bytes("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
++            Mutation rm = new Mutation("Keyspace1", key);
++            rm.add("Standard2", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
+             rm.apply();
+         }
+         store.forceBlockingFlush();
+         CompactionManager.instance.performMaximal(store);
+ 
+         SSTableReader sstable = store.getSSTables().iterator().next();
+         sstable.getPosition(k(2), SSTableReader.Operator.EQ);
+         assertEquals(0, sstable.getKeyCacheHit());
+         assertEquals(1, sstable.getBloomFilterTruePositiveCount());
+         sstable.getPosition(k(2), SSTableReader.Operator.EQ);
+         assertEquals(1, sstable.getKeyCacheHit());
+         assertEquals(2, sstable.getBloomFilterTruePositiveCount());
+         sstable.getPosition(k(15), SSTableReader.Operator.EQ);
+         assertEquals(1, sstable.getKeyCacheHit());
+         assertEquals(2, sstable.getBloomFilterTruePositiveCount());
+ 
+     }
+ 
  
      @Test
 -    public void testPersistentStatisticsWithSecondaryIndex() throws IOException, ExecutionException, InterruptedException
 -    {
 -        // Create secondary index and flush to disk
 -        Keyspace keyspace = Keyspace.open("Keyspace1");
 -        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Indexed1");
 -        ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k1"));
 -        RowMutation rm = new RowMutation("Keyspace1", key);
 -        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), System.currentTimeMillis());
 -        rm.apply();
 -        store.forceBlockingFlush();
 -
 -        // check if opening and querying works
 -        assertIndexQueryWorks(store);
 -    }
 -
 -    @Test
      public void testOpeningSSTable() throws Exception
      {
          String ks = "Keyspace1";