Repository: cassandra
Updated Branches:
  refs/heads/trunk 5023e620d -> e87008bb9


Make CompactionAwareWriterTest work with compression

Patch by marcuse; reviewed by pcmanus for CASSANDRA-9225


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

Branch: refs/heads/trunk
Commit: e87008bb9cd37315759623bfba0e3f8ba1eddca1
Parents: 5023e62
Author: Marcus Eriksson <marc...@apache.org>
Authored: Fri Apr 24 10:38:02 2015 +0200
Committer: Marcus Eriksson <marc...@apache.org>
Committed: Fri Apr 24 10:38:02 2015 +0200

----------------------------------------------------------------------
 .../SplittingSizeTieredCompactionWriter.java    |  2 +-
 .../compaction/CompactionAwareWriterTest.java   | 59 ++++++++++++++------
 2 files changed, 42 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e87008bb/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
----------------------------------------------------------------------
diff --git 
a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
 
b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
index 18c58ea..c97270c 100644
--- 
a/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
+++ 
b/src/java/org/apache/cassandra/db/compaction/writers/SplittingSizeTieredCompactionWriter.java
@@ -46,7 +46,7 @@ public class SplittingSizeTieredCompactionWriter extends 
CompactionAwareWriter
 {
     private static final Logger logger = 
LoggerFactory.getLogger(SplittingSizeTieredCompactionWriter.class);
 
-    private static final long DEFAULT_SMALLEST_SSTABLE_BYTES = 50_000_000;
+    public static final long DEFAULT_SMALLEST_SSTABLE_BYTES = 50_000_000;
     private final double[] ratios;
     private final SSTableRewriter sstableWriter;
     private final long totalSize;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e87008bb/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java
----------------------------------------------------------------------
diff --git 
a/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java 
b/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java
index ac12491..88074af 100644
--- 
a/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java
+++ 
b/test/unit/org/apache/cassandra/db/compaction/CompactionAwareWriterTest.java
@@ -17,17 +17,12 @@
  */
 package org.apache.cassandra.db.compaction;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
+import java.nio.ByteBuffer;
+import java.util.*;
+import java.util.concurrent.ExecutionException;
 
 import com.google.common.primitives.Longs;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -68,6 +63,16 @@ public class CompactionAwareWriterTest
                                     SchemaLoader.standardCFMD(KEYSPACE1, CF));
 
     }
+
+    @Before
+    public void clear()
+    {
+        // avoid one test affecting the next one
+        Keyspace ks = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = ks.getColumnFamilyStore(CF);
+        cfs.clearUnsafe();
+    }
+
     @Test
     public void testDefaultCompactionWriter()
     {
@@ -97,8 +102,8 @@ public class CompactionAwareWriterTest
         populate(cfs, rowCount);
         Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
         long beforeSize = sstables.iterator().next().onDiskLength();
-        int sstableCount = (int)beforeSize/10;
-        CompactionAwareWriter writer = new MaxSSTableSizeWriter(cfs, sstables, 
sstables, sstableCount, 0, false, OperationType.COMPACTION);
+        int sstableSize = (int)beforeSize/10;
+        CompactionAwareWriter writer = new MaxSSTableSizeWriter(cfs, sstables, 
sstables, sstableSize, 0, false, OperationType.COMPACTION);
         int rows = compact(cfs, sstables, writer);
         assertEquals(10, cfs.getSSTables().size());
         assertEquals(rowCount, rows);
@@ -130,7 +135,9 @@ public class CompactionAwareWriterTest
                                 });
         for (SSTableReader sstable : sortedSSTables)
         {
-            assertEquals(expectedSize, sstable.onDiskLength(), 10000);
+            // we dont create smaller files than this, everything will be in 
the last file
+            if (expectedSize > 
SplittingSizeTieredCompactionWriter.DEFAULT_SMALLEST_SSTABLE_BYTES)
+                assertEquals(expectedSize, sstable.onDiskLength(), 
expectedSize / 100); // allow 1% diff in estimated vs actual size
             expectedSize /= 2;
         }
         assertEquals(rowCount, rows);
@@ -144,14 +151,15 @@ public class CompactionAwareWriterTest
         Keyspace ks = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = ks.getColumnFamilyStore(CF);
         cfs.disableAutoCompaction();
-        int rowCount = 10000;
+        int rowCount = 20000;
+        int targetSSTableCount = 50;
         populate(cfs, rowCount);
         Set<SSTableReader> sstables = new HashSet<>(cfs.getSSTables());
         long beforeSize = sstables.iterator().next().onDiskLength();
-        int sstableCount = (int)beforeSize/100;
-        CompactionAwareWriter writer = new MajorLeveledCompactionWriter(cfs, 
sstables, sstables, sstableCount, false, OperationType.COMPACTION);
+        int sstableSize = (int)beforeSize/targetSSTableCount;
+        CompactionAwareWriter writer = new MajorLeveledCompactionWriter(cfs, 
sstables, sstables, sstableSize, false, OperationType.COMPACTION);
         int rows = compact(cfs, sstables, writer);
-        assertEquals(100, cfs.getSSTables().size());
+        assertEquals(targetSSTableCount, cfs.getSSTables().size());
         int [] levelCounts = new int[5];
         assertEquals(rowCount, rows);
         for (SSTableReader sstable : cfs.getSSTables())
@@ -160,7 +168,7 @@ public class CompactionAwareWriterTest
         }
         assertEquals(0, levelCounts[0]);
         assertEquals(10, levelCounts[1]);
-        assertEquals(90, levelCounts[2]);
+        assertEquals(targetSSTableCount - 10, levelCounts[2]); // note that if 
we want more levels, fix this
         for (int i = 3; i < levelCounts.length; i++)
             assertEquals(0, levelCounts[i]);
         validateData(cfs, rowCount);
@@ -190,17 +198,32 @@ public class CompactionAwareWriterTest
     private void populate(ColumnFamilyStore cfs, int count)
     {
         long timestamp = System.currentTimeMillis();
+        byte [] payload = new byte[1000];
+        new Random().nextBytes(payload);
+        ByteBuffer b = ByteBuffer.wrap(payload);
         for (int i = 0; i < count; i++)
         {
             DecoratedKey key = Util.dk(Integer.toString(i));
             Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int j = 0; j < 10; j++)
                 rm.add(CF,  Util.cellname(Integer.toString(j)),
-                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                        b,
                         timestamp);
             rm.applyUnsafe();
         }
         cfs.forceBlockingFlush();
+        if (cfs.getSSTables().size() > 1)
+        {
+            // we want just one big sstable to avoid doing actual compaction 
in compact() above
+            try
+            {
+                cfs.forceMajorCompaction();
+            }
+            catch (Throwable t)
+            {
+                throw new RuntimeException(t);
+            }
+        }
         assert cfs.getSSTables().size() == 1 : cfs.getSSTables();
     }
     private void validateData(ColumnFamilyStore cfs, int rowCount)

Reply via email to