cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From slebre...@apache.org
Subject svn commit: r1129151 - in /cassandra/branches/cassandra-0.7: src/java/org/apache/cassandra/db/CompactionManager.java test/unit/org/apache/cassandra/db/CompactionsTest.java
Date Mon, 30 May 2011 13:02:40 GMT
Author: slebresne
Date: Mon May 30 13:02:40 2011
New Revision: 1129151

URL: http://svn.apache.org/viewvc?rev=1129151&view=rev
Log:
Reset SSTII in EchoedRow iterator (see CASSANDRA-2653)

Modified:
    cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/CompactionManager.java
    cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/db/CompactionsTest.java

Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/CompactionManager.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/CompactionManager.java?rev=1129151&r1=1129150&r2=1129151&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/CompactionManager.java
(original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/CompactionManager.java
Mon May 30 13:02:40 2011
@@ -1144,6 +1144,8 @@ public class CompactionManager implement
         {
             super(row.getKey());
             this.row = row;
+            // Reset SSTableIdentityIterator because we have not guarantee the filePointer
hasn't moved since the Iterator was built
+            row.reset();
         }
 
         public void write(DataOutput out) throws IOException

Modified: cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/db/CompactionsTest.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/db/CompactionsTest.java?rev=1129151&r1=1129150&r2=1129151&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/db/CompactionsTest.java
(original)
+++ cassandra/branches/cassandra-0.7/test/unit/org/apache/cassandra/db/CompactionsTest.java
Mon May 30 13:02:40 2011
@@ -137,4 +137,34 @@ public class CompactionsTest extends Cle
         assertEquals(1, buckets.size());
     }
 
+    @Test
+    public void testEchoedRow() throws IOException, ExecutionException, InterruptedException
+    {
+        // This test check that EchoedRow doesn't skipp rows: see CASSANDRA-2653
+
+        Table table = Table.open(TABLE1);
+        ColumnFamilyStore store = table.getColumnFamilyStore("Standard2");
+
+        // disable compaction while flushing
+        store.disableAutoCompaction();
+
+        // Insert 4 keys in two sstables. We need the sstables to have 2 rows
+        // at least to trigger what was causing CASSANDRA-2653
+        for (int i=1; i < 5; i++)
+        {
+            DecoratedKey key = Util.dk(String.valueOf(i));
+            RowMutation rm = new RowMutation(TABLE1, key.key);
+            rm.add(new QueryPath("Standard2", null, ByteBufferUtil.bytes(String.valueOf(i))),
ByteBufferUtil.EMPTY_BYTE_BUFFER, i);
+            rm.apply();
+
+            if (i % 2 == 0)
+                store.forceBlockingFlush();
+        }
+
+        // Force compaction. Since each row is in only one sstable, we will be using EchoedRow.
+        CompactionManager.instance.performMajor(store);
+
+        // Now assert we do have the two keys
+        assertEquals(4, Util.getRangeSlice(store).size());
+    }
 }



Mime
View raw message