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 2009/12/08 19:45:55 UTC

svn commit: r888512 - in /incubator/cassandra/trunk: CHANGES.txt NEWS.txt src/java/org/apache/cassandra/db/ColumnFamilyStore.java src/java/org/apache/cassandra/io/CompactionIterator.java test/unit/org/apache/cassandra/db/CompactionsTest.java

Author: jbellis
Date: Tue Dec  8 18:45:55 2009
New Revision: 888512

URL: http://svn.apache.org/viewvc?rev=888512&view=rev
Log:
only GC when compacting all sstables, to avoid situations where the data a tombstone is intended to supress is in an sstable that is not part of the compaction set.
patch by jbellis; reviewed by Ramzi Rabah for CASSANDRA-604

Modified:
    incubator/cassandra/trunk/CHANGES.txt
    incubator/cassandra/trunk/NEWS.txt
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/io/CompactionIterator.java
    incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java

Modified: incubator/cassandra/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/CHANGES.txt?rev=888512&r1=888511&r2=888512&view=diff
==============================================================================
--- incubator/cassandra/trunk/CHANGES.txt (original)
+++ incubator/cassandra/trunk/CHANGES.txt Tue Dec  8 18:45:55 2009
@@ -1,5 +1,6 @@
 0.5.0 dev
  * Bootstrap improvements (several tickets)
+ * add nodeprobe repair anti-entropy feature (CASSANDRA-193, CASSANDRA-520)
  * fix possibility of partition when many nodes restart at once
    in clusters with multiple seeds (CASSANDRA-150)
  * fix NPE in get_range_slice when no data is found (CASSANDRA-578)
@@ -14,6 +15,7 @@
    (see contrib/client_only for an example of use)
  * make consistency_level functional in get_range_slice (CASSANDRA-568)
  * optimize key deserialization for RandomPartitioner (CASSANDRA-581)
+ * avoid GCing tombstones except on major compaction (CASSANDRA-604)
 
 
 0.5.0 beta

Modified: incubator/cassandra/trunk/NEWS.txt
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/NEWS.txt?rev=888512&r1=888511&r2=888512&view=diff
==============================================================================
--- incubator/cassandra/trunk/NEWS.txt (original)
+++ incubator/cassandra/trunk/NEWS.txt Tue Dec  8 18:45:55 2009
@@ -7,7 +7,7 @@
    (If more writes come in after the flush command, starting 0.5 will error
    out; if that happens, just go back to 0.4 and flush again.)
 
-1. Bootstrap, move, and load balancing have been added.
+1. Bootstrap, move, load balancing, and active repair have been added.
    See http://wiki.apache.org/cassandra/Operations
 
 2. Performance improvements across the board, especially on the write

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=888512&r1=888511&r2=888512&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Tue Dec  8 18:45:55 2009
@@ -815,7 +815,7 @@
           logger_.debug("Expected bloom filter size : " + expectedBloomFilterSize);
 
         SSTableWriter writer = null;
-        CompactionIterator ci = new CompactionIterator(sstables, getDefaultGCBefore());
+        CompactionIterator ci = new CompactionIterator(sstables, getDefaultGCBefore(), sstables.size() == ssTables_.size());
         Iterator nni = new FilterIterator(ci, PredicateUtils.notNullPredicate());
 
         try
@@ -905,7 +905,7 @@
           logger_.debug("Expected bloom filter size : " + expectedBloomFilterSize);
 
         SSTableWriter writer;
-        CompactionIterator ci = new CompactionIterator(sstables, gcBefore); // retain a handle so we can call close()
+        CompactionIterator ci = new CompactionIterator(sstables, gcBefore, major); // retain a handle so we can call close()
         Iterator nni = new FilterIterator(ci, PredicateUtils.notNullPredicate());
 
         try
@@ -958,7 +958,7 @@
     void doReadonlyCompaction(InetAddress initiator) throws IOException
     {
         Collection<SSTableReader> sstables = ssTables_.getSSTables();
-        CompactionIterator ci = new CompactionIterator(sstables, getDefaultGCBefore());
+        CompactionIterator ci = new CompactionIterator(sstables, getDefaultGCBefore(), true);
         try
         {
             Iterator nni = new FilterIterator(ci, PredicateUtils.notNullPredicate());

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/io/CompactionIterator.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/CompactionIterator.java?rev=888512&r1=888511&r2=888512&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/io/CompactionIterator.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/io/CompactionIterator.java Tue Dec  8 18:45:55 2009
@@ -23,6 +23,7 @@
 
 import java.io.Closeable;
 import java.io.IOException;
+import java.io.IOError;
 import java.util.List;
 import java.util.ArrayList;
 import java.util.Comparator;
@@ -41,12 +42,14 @@
 
     private final List<IteratingRow> rows = new ArrayList<IteratingRow>();
     private final int gcBefore;
+    private boolean major;
 
     @SuppressWarnings("unchecked")
-    public CompactionIterator(Iterable<SSTableReader> sstables, int gcBefore) throws IOException
+    public CompactionIterator(Iterable<SSTableReader> sstables, int gcBefore, boolean major) throws IOException
     {
         super(getCollatingIterator(sstables));
         this.gcBefore = gcBefore;
+        this.major = major;
     }
 
     @SuppressWarnings("unchecked")
@@ -84,35 +87,54 @@
         DataOutputBuffer buffer = new DataOutputBuffer();
         DecoratedKey key = rows.get(0).getKey();
 
-        ColumnFamily cf = null;
-        for (IteratingRow row : rows)
+        try
         {
-            ColumnFamily thisCF;
-            try
+            if (rows.size() > 1 || major)
             {
-                thisCF = row.getColumnFamily();
-            }
-            catch (IOException e)
-            {
-                logger.error("Skipping row " + key + " in " + row.getPath(), e);
-                continue;
-            }
-            if (cf == null)
-            {
-                cf = thisCF;
+                ColumnFamily cf = null;
+                for (IteratingRow row : rows)
+                {
+                    ColumnFamily thisCF;
+                    try
+                    {
+                        thisCF = row.getColumnFamily();
+                    }
+                    catch (IOException e)
+                    {
+                        logger.error("Skipping row " + key + " in " + row.getPath(), e);
+                        continue;
+                    }
+                    if (cf == null)
+                    {
+                        cf = thisCF;
+                    }
+                    else
+                    {
+                        cf.addAll(thisCF);
+                    }
+                }
+                ColumnFamily cfPurged = major ? ColumnFamilyStore.removeDeleted(cf, gcBefore) : cf;
+                if (cfPurged == null)
+                    return null;
+                ColumnFamily.serializer().serializeWithIndexes(cfPurged, buffer);
             }
             else
             {
-                cf.addAll(thisCF);
+                assert rows.size() == 1;
+                try
+                {
+                    rows.get(0).echoData(buffer);
+                }
+                catch (IOException e)
+                {
+                    throw new IOError(e);
+                }
             }
         }
-        rows.clear();
-
-        ColumnFamily cfPurged = ColumnFamilyStore.removeDeleted(cf, gcBefore);
-        if (cfPurged == null)
-            return null;
-        ColumnFamily.serializer().serializeWithIndexes(cfPurged, buffer);
-
+        finally
+        {
+            rows.clear();
+        }
         return new CompactedRow(key, buffer);
     }
 

Modified: incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java?rev=888512&r1=888511&r2=888512&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java (original)
+++ incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java Tue Dec  8 18:45:55 2009
@@ -25,6 +25,7 @@
 import java.util.ArrayList;
 import java.util.Set;
 import java.util.HashSet;
+import java.util.Collection;
 
 import org.junit.Test;
 
@@ -105,15 +106,25 @@
         }
         store.forceBlockingFlush();
 
-        // resurrect one row
+        // resurrect one column
         rm = new RowMutation(TABLE1, key);
         rm.add(new QueryPath(cfName, null, String.valueOf(5).getBytes()), new byte[0], 2);
         rm.apply();
         store.forceBlockingFlush();
 
-        // compact and test that all columns but the resurrected one is completely gone
-        store.doFileCompaction(store.getSSTables(), Integer.MAX_VALUE);
+        // verify that non-major compaction does no GC to ensure correctness (see CASSANDRA-604)
+        Collection<SSTableReader> sstablesIncomplete = store.getSSTables();
+        rm = new RowMutation(TABLE1, key + "x");
+        rm.add(new QueryPath(cfName, null, "0".getBytes()), new byte[0], 0);
+        rm.apply();
+        store.forceBlockingFlush();
+        store.doFileCompaction(sstablesIncomplete, Integer.MAX_VALUE);
         ColumnFamily cf = table.getColumnFamilyStore(cfName).getColumnFamily(new IdentityQueryFilter(key, new QueryPath(cfName)));
+        assert cf.getColumnCount() == 10;
+
+        // major compact and test that all columns but the resurrected one is completely gone
+        store.doFileCompaction(store.getSSTables(), Integer.MAX_VALUE);
+        cf = table.getColumnFamilyStore(cfName).getColumnFamily(new IdentityQueryFilter(key, new QueryPath(cfName)));
         assert cf.getColumnCount() == 1;
         assert cf.getColumn(String.valueOf(5).getBytes()) != null;
     }