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 2011/07/14 16:32:17 UTC
svn commit: r1146732 -
/cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
Author: jbellis
Date: Thu Jul 14 14:32:16 2011
New Revision: 1146732
URL: http://svn.apache.org/viewvc?rev=1146732&view=rev
Log:
add test for including supercolumn tombstone time in max timestamp computation
patch by Daniel Doubleday; reviewed by jbellis for CASSANDRA-2753
Modified:
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
Modified: cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java
URL: http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java?rev=1146732&r1=1146731&r2=1146732&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java (original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTest.java Thu Jul 14 14:32:16 2011
@@ -21,16 +21,15 @@ package org.apache.cassandra.io.sstable;
*/
+import static org.apache.cassandra.Util.addMutation;
import static org.junit.Assert.*;
import java.io.IOException;
import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
import java.util.concurrent.ExecutionException;
+import org.apache.cassandra.Util;
import org.junit.Test;
import org.apache.cassandra.CleanupHelper;
@@ -137,4 +136,38 @@ public class SSTableWriterTest extends C
// ensure max timestamp is captured during rebuild
assert sstr.getMaxTimestamp() == 4321L;
}
+
+ @Test
+ public void testSuperColumnMaxTimestamp() throws IOException, ExecutionException, InterruptedException
+ {
+ ColumnFamilyStore store = Table.open("Keyspace1").getColumnFamilyStore("Super1");
+ RowMutation rm;
+ DecoratedKey dk = Util.dk("key1");
+
+ // add data
+ rm = new RowMutation("Keyspace1", dk.key);
+ addMutation(rm, "Super1", "SC1", 1, "val1", 0);
+ rm.apply();
+ store.forceBlockingFlush();
+
+ validateMinTimeStamp(store.getSSTables(), 0);
+
+ // remove
+ rm = new RowMutation("Keyspace1", dk.key);
+ rm.delete(new QueryPath("Super1", ByteBufferUtil.bytes("SC1")), 1);
+ rm.apply();
+ store.forceBlockingFlush();
+
+ validateMinTimeStamp(store.getSSTables(), 0);
+
+ CompactionManager.instance.performMaximal(store);
+ assertEquals(1, store.getSSTables().size());
+ validateMinTimeStamp(store.getSSTables(), 1);
+ }
+
+ private void validateMinTimeStamp(Collection<SSTableReader> ssTables, int timestamp)
+ {
+ for (SSTableReader ssTable : ssTables)
+ assertTrue(ssTable.getMaxTimestamp() >= timestamp);
+ }
}