Updated Branches:
  refs/heads/cassandra-1.1 a4fab90af -> f74ed12a7
  refs/heads/trunk 8aaaacd09 -> 7398e9363


Merge branch 'cassandra-1.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/7398e936
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/7398e936
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/7398e936

Branch: refs/heads/trunk
Commit: 7398e9363ae99220d195d30eea3f3479485618f9
Parents: 8aaaacd f74ed12
Author: Jonathan Ellis <[email protected]>
Authored: Fri Jun 15 12:27:40 2012 -0500
Committer: Jonathan Ellis <[email protected]>
Committed: Fri Jun 15 12:27:40 2012 -0500

----------------------------------------------------------------------
 CHANGES.txt                                        |    1 +
 .../cassandra/db/compaction/LeveledManifest.java   |   60 ++++++--
 .../cassandra/db/compaction/CompactionsTest.java   |  114 +++++++-------
 3 files changed, 103 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7398e936/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 2eb1b57,693b03b..7064eb0
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,28 -1,5 +1,29 @@@
 +1.2-dev
 + * update MS protocol with a version handshake + broadcast address id
 +   (CASSANDRA-4311)
 + * multithreaded hint replay (CASSANDRA-4189)
 + * add inter-node message compression (CASSANDRA-3127)
 + * enforce 1m min keycache for auto (CASSANDRA-4306)
 + * remove COPP (CASSANDRA-2479)
 + * Track tombstone expiration and compact when tombstone content is
 +   higher than a configurable threshold, default 20% (CASSANDRA-3442)
 + * update MurmurHash to version 3 (CASSANDRA-2975)
 + * (CLI) track elapsed time for `delete' operation (CASSANDRA-4060)
 + * (CLI) jline version is bumped to 1.0 to properly  support
 +   'delete' key function (CASSANDRA-4132)
 + * Save IndexSummary into new SSTable 'Summary' component (CASSANDRA-2392)
 + * Add support for range tombstones (CASSANDRA-3708)
 + * Improve MessagingService efficiency (CASSANDRA-3617)
 + * Avoid ID conflicts from concurrent schema changes (CASSANDRA-3794)
 + * Set thrift HSHA server thread limit to unlimet by default (CASSANDRA-4277)
 + * Avoids double serialization of CF id in RowMutation messages
 +   (CASSANDRA-4293)
 + * fix Summary component and caches to use correct partitioner 
(CASSANDRA-4289)
 + * stream compressed sstables directly with java nio (CASSANDRA-4297)
 +
 +
  1.1.2
+  * fix bug in sstable blacklisting with LCS (CASSANDRA-4343)
   * LCS no longer promotes tiny sstables out of L0 (CASSANDRA-4341)
   * skip tombstones during hint replay (CASSANDRA-4320)
   * fix NPE in compactionstats (CASSANDRA-4318)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7398e936/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/7398e936/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
index 1476b4a,4f87c86..2b134d1
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
@@@ -85,69 -87,20 +85,69 @@@ public class CompactionsTest extends Sc
                  rm.apply();
                  inserted.add(key);
              }
-             store.forceBlockingFlush();
-             assertMaxTimestamp(store, maxTimestampExpected);
-             assertEquals(inserted.toString(), inserted.size(), 
Util.getRangeSlice(store).size());
+             cfs.forceBlockingFlush();
+             assertMaxTimestamp(cfs, maxTimestampExpected);
+             assertEquals(inserted.toString(), inserted.size(), 
Util.getRangeSlice(cfs).size());
          }
  
-         forceCompactions(store);
+         forceCompactions(cfs);
  
-         assertEquals(inserted.size(), Util.getRangeSlice(store).size());
+         assertEquals(inserted.size(), Util.getRangeSlice(cfs).size());
  
          // make sure max timestamp of compacted sstables is recorded properly 
after compaction.
-         assertMaxTimestamp(store, maxTimestampExpected);
-         store.truncate();
+         assertMaxTimestamp(cfs, maxTimestampExpected);
+         cfs.truncate();
      }
  
 +    /**
 +     * Test to see if sstable has enough expired columns, it is compacted 
itself.
 +     */
 +    @Test
 +    public void testSingleSSTableCompaction() throws Exception
 +    {
 +        Table table = Table.open(TABLE1);
 +        ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
 +        store.clearUnsafe();
 +        store.metadata.gcGraceSeconds(1);
 +        
store.setCompactionStrategyClass(SizeTieredCompactionStrategy.class.getCanonicalName());
 +
 +        // disable compaction while flushing
 +        store.disableAutoCompaction();
 +
 +        long timestamp = System.currentTimeMillis();
 +        for (int i = 0; i < 10; i++)
 +        {
 +            DecoratedKey key = Util.dk(Integer.toString(i));
 +            RowMutation rm = new RowMutation(TABLE1, key.key);
 +            for (int j = 0; j < 10; j++)
 +                rm.add(new QueryPath("Standard1", null, 
ByteBufferUtil.bytes(Integer.toString(j))),
 +                       ByteBufferUtil.EMPTY_BYTE_BUFFER,
 +                       timestamp,
 +                       j > 0 ? 3 : 0); // let first column never expire, 
since deleting all columns does not produce sstable
 +            rm.apply();
 +        }
 +        store.forceBlockingFlush();
 +        assertEquals(1, store.getSSTables().size());
 +        long originalSize = 
store.getSSTables().iterator().next().uncompressedLength();
 +
 +        // wait enough to force single compaction
 +        TimeUnit.SECONDS.sleep(5);
 +
 +        // enable compaction, submit background and wait for it to complete
 +        store.setMinimumCompactionThreshold(2);
 +        store.setMaximumCompactionThreshold(4);
 +        
FBUtilities.waitOnFuture(CompactionManager.instance.submitBackground(store));
 +        while (CompactionManager.instance.getPendingTasks() > 0 || 
CompactionManager.instance.getActiveCompactions() > 0)
 +            TimeUnit.SECONDS.sleep(1);
 +
 +        // and sstable with ttl should be compacted
 +        assertEquals(1, store.getSSTables().size());
 +        long size = 
store.getSSTables().iterator().next().uncompressedLength();
 +        assertTrue("should be less than " + originalSize + ", but was " + 
size, size < originalSize);
 +
 +        // make sure max timestamp of compacted sstables is recorded properly 
after compaction.
 +        assertMaxTimestamp(store, timestamp);
 +    }
  
      @Test
      public void testSuperColumnCompactions() throws IOException, 
ExecutionException, InterruptedException

Reply via email to