Author: jbellis
Date: Wed Mar  9 14:36:59 2011
New Revision: 1079816

URL: http://svn.apache.org/viewvc?rev=1079816&view=rev
Log:
avoid writing empty rows when scrubbing tombstoned rows
patch by jbellis; reviewed by slebresne for CASSANDRA-2296

Modified:
    cassandra/branches/cassandra-0.7/CHANGES.txt
    
cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/db/CompactionManager.java

Modified: cassandra/branches/cassandra-0.7/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/CHANGES.txt?rev=1079816&r1=1079815&r2=1079816&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/CHANGES.txt (original)
+++ cassandra/branches/cassandra-0.7/CHANGES.txt Wed Mar  9 14:36:59 2011
@@ -13,6 +13,7 @@
    (CASSANDRA-2259)
  * move sample schema from cassandra.yaml to schema-sample.txt,
    a cli scripts (CASSANDRA-2007)
+ * avoid writing empty rows when scrubbing tombstoned rows (CASSANDRA-2296)
 
 
 0.7.3

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=1079816&r1=1079815&r2=1079816&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
 Wed Mar  9 14:36:59 2011
@@ -534,7 +534,7 @@ public class CompactionManager implement
 
             SSTableWriter writer = maybeCreateWriter(cfs, 
compactionFileLocation, expectedBloomFilterSize, null);
             executor.beginCompaction(cfs.columnFamily, new ScrubInfo(dataFile, 
sstable));
-            int goodRows = 0, badRows = 0;
+            int goodRows = 0, badRows = 0, emptyRows = 0;
 
             while (!dataFile.isEOF())
             {
@@ -588,8 +588,16 @@ public class CompactionManager implement
                     if (dataSize > dataFile.length())
                         throw new IOError(new IOException("Impossible row size 
" + dataSize));
                     SSTableIdentityIterator row = new 
SSTableIdentityIterator(sstable, dataFile, key, dataStart, dataSize, true);
-                    writer.append(getCompactedRow(row, cfs, 
sstable.descriptor, true));
-                    goodRows++;
+                    AbstractCompactedRow compactedRow = getCompactedRow(row, 
cfs, sstable.descriptor, true);
+                    if (compactedRow.isEmpty())
+                    {
+                        emptyRows++;
+                    }
+                    else
+                    {
+                        writer.append(compactedRow);
+                        goodRows++;
+                    }
                     if (!key.key.equals(currentIndexKey) || dataStart != 
dataStartFromIndex)
                         logger.warn("Row scrubbed successfully but index file 
contains a different key or row size; consider rebuilding the index as 
described in 
http://www.mail-archive.com/user@cassandra.apache.org/msg03325.html";);
                 }
@@ -608,8 +616,16 @@ public class CompactionManager implement
                         try
                         {
                             SSTableIdentityIterator row = new 
SSTableIdentityIterator(sstable, dataFile, key, dataStartFromIndex, 
dataSizeFromIndex, true);
-                            writer.append(getCompactedRow(row, cfs, 
sstable.descriptor, true));
-                            goodRows++;
+                            AbstractCompactedRow compactedRow = 
getCompactedRow(row, cfs, sstable.descriptor, true);
+                            if (compactedRow.isEmpty())
+                            {
+                                emptyRows++;
+                            }
+                            else
+                            {
+                                writer.append(compactedRow);
+                                goodRows++;
+                            }
                         }
                         catch (Throwable th2)
                         {
@@ -634,7 +650,7 @@ public class CompactionManager implement
             {
                 SSTableReader newSstable = 
writer.closeAndOpenReader(sstable.maxDataAge);
                 cfs.replaceCompactedSSTables(Arrays.asList(sstable), 
Arrays.asList(newSstable));
-                logger.info("Scrub of " + sstable + " complete: " + goodRows + 
" rows in new sstable");
+                logger.info("Scrub of " + sstable + " complete: " + goodRows + 
" rows in new sstable and " + emptyRows + " empty (tombstoned) rows dropped");
                 if (badRows > 0)
                     logger.warn("Unable to recover " + badRows + " rows that 
were skipped.  You can attempt manual recovery from the pre-scrub snapshot.  
You can also run nodetool repair to transfer the data from a healthy replica, 
if any");
             }


Reply via email to