cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From marc...@apache.org
Subject [3/3] git commit: Merge branch 'cassandra-2.1' into trunk
Date Tue, 27 May 2014 08:53:15 GMT
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 1b62f3d39b219e7678b849c22a9980fa6a8805a7
Parents: 8ffb9cc 86b6ec5
Author: Marcus Eriksson <marcuse@apache.org>
Authored: Tue May 27 10:45:47 2014 +0200
Committer: Marcus Eriksson <marcuse@apache.org>
Committed: Tue May 27 10:49:25 2014 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 doc/cql3/CQL.textile                            |  21 ++--
 pylib/cqlshlib/cql3handling.py                  |   2 +-
 .../compaction/AbstractCompactionStrategy.java  |  21 ++++
 .../db/compaction/CompactionsTest.java          | 104 ++++++++++++++++---
 5 files changed, 125 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b62f3d3/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1b62f3d3/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
index 6671457,8a24771..a720375
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
@@@ -18,19 -18,9 +18,20 @@@
  */
  package org.apache.cassandra.db.compaction;
  
 -import java.io.*;
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertFalse;
 +import static org.junit.Assert.assertTrue;
 +import static org.junit.Assert.assertNotNull;
 +
 +import java.io.File;
  import java.nio.ByteBuffer;
 -import java.util.*;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.Collections;
++import java.util.Iterator;
 +import java.util.Map;
 +import java.util.Set;
 +import java.util.UUID;
  import java.util.concurrent.ExecutionException;
  import java.util.concurrent.TimeUnit;
  
@@@ -169,10 -158,80 +177,80 @@@ public class CompactionsTest extends Sc
          SSTableScanner scanner = sstable.getScanner(DataRange.forKeyRange(keyRange));
          OnDiskAtomIterator iter = scanner.next();
          assertEquals(key, iter.getKey());
 -        assert iter.next() instanceof RangeTombstone;
 -        assert !iter.hasNext();
 +        assertTrue(iter.next() instanceof RangeTombstone);
 +        assertFalse(iter.hasNext());
      }
  
+     @Test
+     public void testUncheckedTombstoneSizeTieredCompaction() throws Exception
+     {
+         Keyspace keyspace = Keyspace.open(KEYSPACE1);
+         ColumnFamilyStore store = keyspace.getColumnFamilyStore(STANDARD1);
+         store.clearUnsafe();
+         store.metadata.gcGraceSeconds(1);
+         store.metadata.compactionStrategyOptions.put("tombstone_compaction_interval", "1");
+         store.metadata.compactionStrategyOptions.put("unchecked_tombstone_compaction", "false");
+         store.reload();
+         store.setCompactionStrategyClass(SizeTieredCompactionStrategy.class.getName());
+ 
+         // disable compaction while flushing
+         store.disableAutoCompaction();
+ 
+         //Populate sstable1 with with keys [0..9]
+         populate(KEYSPACE1, STANDARD1, 0, 9, 3); //ttl=3s
+         store.forceBlockingFlush();
+ 
+         //Populate sstable2 with with keys [10..19] (keys do not overlap with SSTable1)
+         long timestamp2 = populate(KEYSPACE1, STANDARD1, 10, 19, 3); //ttl=3s
+         store.forceBlockingFlush();
+ 
+         assertEquals(2, store.getSSTables().size());
+ 
+         Iterator<SSTableReader> it = store.getSSTables().iterator();
+         long originalSize1 = it.next().uncompressedLength();
+         long originalSize2 = it.next().uncompressedLength();
+ 
+         // wait enough to force single compaction
+         TimeUnit.SECONDS.sleep(5);
+ 
+         // enable compaction, submit background and wait for it to complete
+         store.enableAutoCompaction();
+         FBUtilities.waitOnFutures(CompactionManager.instance.submitBackground(store));
+         while (CompactionManager.instance.getPendingTasks() > 0 || CompactionManager.instance.getActiveCompactions()
> 0)
+             TimeUnit.SECONDS.sleep(1);
+ 
+         // even though both sstables were candidate for tombstone compaction
+         // it was not executed because they have an overlapping token range
+         assertEquals(2, store.getSSTables().size());
+         it = store.getSSTables().iterator();
+         long newSize1 = it.next().uncompressedLength();
+         long newSize2 = it.next().uncompressedLength();
+         assertEquals("candidate sstable should not be tombstone-compacted because its key
range overlap with other sstable",
+                       originalSize1, newSize1);
+         assertEquals("candidate sstable should not be tombstone-compacted because its key
range overlap with other sstable",
+                       originalSize2, newSize2);
+ 
+         // now let's enable the magic property
+         store.metadata.compactionStrategyOptions.put("unchecked_tombstone_compaction", "true");
+         store.reload();
+ 
+         //submit background task again and wait for it to complete
+         FBUtilities.waitOnFutures(CompactionManager.instance.submitBackground(store));
+         while (CompactionManager.instance.getPendingTasks() > 0 || CompactionManager.instance.getActiveCompactions()
> 0)
+             TimeUnit.SECONDS.sleep(1);
+ 
+         //we still have 2 sstables, since they were not compacted against each other
+         assertEquals(2, store.getSSTables().size());
+         it = store.getSSTables().iterator();
+         newSize1 = it.next().uncompressedLength();
+         newSize2 = it.next().uncompressedLength();
+         assertTrue("should be less than " + originalSize1 + ", but was " + newSize1, newSize1
< originalSize1);
+         assertTrue("should be less than " + originalSize2 + ", but was " + newSize2, newSize2
< originalSize2);
+ 
+         // make sure max timestamp of compacted sstables is recorded properly after compaction.
+         assertMaxTimestamp(store, timestamp2);
+     }
+ 
      public static void assertMaxTimestamp(ColumnFamilyStore cfs, long maxTimestampExpected)
      {
          long maxTimestampObserved = Long.MIN_VALUE;


Mime
View raw message