Repository: cassandra
Updated Branches:
  refs/heads/trunk afd6cf2cb -> d19a6af66


Don't initialize writer before checking if iter is empty

Patch by marcuse; reviewed by thobbs for CASSANDRA-9117


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

Branch: refs/heads/trunk
Commit: d19a6af6674c883721cbc966757027edda26bcc3
Parents: afd6cf2
Author: Marcus Eriksson <[email protected]>
Authored: Fri Apr 10 14:30:55 2015 +0200
Committer: Marcus Eriksson <[email protected]>
Committed: Mon Apr 13 07:21:14 2015 +0200

----------------------------------------------------------------------
 CHANGES.txt                                              |  1 +
 .../apache/cassandra/db/compaction/CompactionTask.java   | 11 +++++++----
 2 files changed, 8 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d19a6af6/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 859a86b..d6e8f57 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0
+ * Don't initialize compaction writer before checking if iter is empty 
(CASSANDRA-9117)
  * Remove line number generation from default logback.xml
  * Don't execute any functions at prepare-time (CASSANDRA-9037)
  * Share file handles between all instances of a SegmentedFile (CASSANDRA-8893)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d19a6af6/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java 
b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index 4233ca6..f472711 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -147,10 +147,10 @@ public class CompactionTask extends AbstractCompactionTask
 
         long totalKeysWritten = 0;
 
+        long estimatedKeys = 0;
         try (CompactionController controller = 
getCompactionController(sstables))
         {
             Set<SSTableReader> actuallyCompact = Sets.difference(sstables, 
controller.getFullyExpiredSSTables());
-            CompactionAwareWriter writer = getCompactionAwareWriter(cfs, 
sstables, actuallyCompact);
 
             SSTableFormat.Type sstableFormat = getFormatType(sstables);
 
@@ -167,6 +167,7 @@ public class CompactionTask extends AbstractCompactionTask
                 if (collector != null)
                     collector.beginCompaction(ci);
                 long lastCheckObsoletion = start;
+                CompactionAwareWriter writer = null;
                 try
                 {
                     if (!controller.cfs.getCompactionStrategy().isActive)
@@ -179,7 +180,8 @@ public class CompactionTask extends AbstractCompactionTask
                         cfs.markObsolete(sstables, compactionType);
                         return;
                     }
-
+                    writer = getCompactionAwareWriter(cfs, sstables, 
actuallyCompact);
+                    estimatedKeys = writer.estimatedKeys();
                     while (iter.hasNext())
                     {
                         if (ci.isStopRequested())
@@ -203,7 +205,8 @@ public class CompactionTask extends AbstractCompactionTask
                 {
                     try
                     {
-                        writer.abort();
+                        if (writer != null)
+                            writer.abort();
                     }
                     catch (Throwable t2)
                     {
@@ -243,7 +246,7 @@ public class CompactionTask extends AbstractCompactionTask
             logger.info(String.format("Compacted (%s) %d sstables to [%s] to 
level=%d.  %,d bytes to %,d (~%d%% of original) in %,dms = %fMB/s.  %,d total 
partitions merged to %,d.  Partition merge counts were {%s}",
                                       taskIdLoggerMsg, oldSStables.size(), 
newSSTableNames.toString(), getLevel(), startsize, endsize, (int) (ratio * 
100), dTime, mbps, totalSourceRows, totalKeysWritten, mergeSummary));
             logger.debug(String.format("CF Total Bytes Compacted: %,d", 
CompactionTask.addToTotalBytesCompacted(endsize)));
-            logger.debug("Actual #keys: {}, Estimated #keys:{}, Err%: {}", 
totalKeysWritten, writer.estimatedKeys(), ((double)(totalKeysWritten - 
writer.estimatedKeys())/totalKeysWritten));
+            logger.debug("Actual #keys: {}, Estimated #keys:{}, Err%: {}", 
totalKeysWritten, estimatedKeys, ((double)(totalKeysWritten - 
estimatedKeys)/totalKeysWritten));
 
             if (offline)
             {

Reply via email to