cassandra git commit: Handle abort() properly in SSTableRewriter

2014-11-26 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 3faff8b15 - b10629291


Handle abort() properly in SSTableRewriter

Patch by marcuse; reviewed by jmckenzie for CASSANDRA-8320


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

Branch: refs/heads/cassandra-2.1
Commit: b1062929185690567e4567e0e657b361c5105482
Parents: 3faff8b
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Nov 18 07:07:30 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Wed Nov 26 16:00:51 2014 +0100

--
 CHANGES.txt |   1 +
 .../cassandra/io/sstable/SSTableReader.java |  22 +++
 .../cassandra/io/sstable/SSTableRewriter.java   |  74 ++--
 .../io/sstable/SSTableRewriterTest.java | 180 +++
 4 files changed, 226 insertions(+), 51 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b1062929/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index f022b19..e5f7c28 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.3
+ * Handle abort() in SSTableRewriter properly (CASSANDRA-8320)
  * Fix high size calculations for prepared statements (CASSANDRA-8231)
  * Centralize shared executors (CASSANDRA-8055)
  * Fix filtering for CONTAINS (KEY) relations on frozen collection

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b1062929/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
--
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java 
b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
index a3e3cf5..1fe4330 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
@@ -202,6 +202,7 @@ public class SSTableReader extends SSTable
 private Object replaceLock = new Object();
 private SSTableReader replacedBy;
 private SSTableReader replaces;
+private SSTableReader sharesBfWith;
 private SSTableDeletingTask deletingTask;
 private Runnable runOnClose;
 
@@ -594,6 +595,14 @@ public class SSTableReader extends SSTable
 deleteFiles = !dfile.path.equals(replaces.dfile.path);
 }
 
+if (sharesBfWith != null)
+{
+closeBf = sharesBfWith.bf != bf;
+closeSummary = sharesBfWith.indexSummary != indexSummary;
+closeFiles = sharesBfWith.dfile != dfile;
+deleteFiles = !dfile.path.equals(sharesBfWith.dfile.path);
+}
+
 boolean deleteAll = false;
 if (release  isCompacted.get())
 {
@@ -928,6 +937,19 @@ public class SSTableReader extends SSTable
 }
 }
 
+/**
+ * this is used to avoid closing the bloom filter multiple times when 
finishing an SSTableRewriter
+ *
+ * note that the reason we don't use replacedBy is that we are not yet 
actually replaced
+ *
+ * @param newReader
+ */
+public void sharesBfWith(SSTableReader newReader)
+{
+assert openReason.equals(OpenReason.EARLY);
+this.sharesBfWith = newReader;
+}
+
 public SSTableReader cloneWithNewStart(DecoratedKey newStart, final 
Runnable runOnClose)
 {
 synchronized (replaceLock)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b1062929/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
--
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java 
b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
index 4d5a06f..d187e9d 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.io.sstable;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -75,6 +76,7 @@ public class SSTableRewriter
 private final ColumnFamilyStore cfs;
 
 private final long maxAge;
+private final ListSSTableReader finished = new ArrayList();
 private final SetSSTableReader rewriting; // the readers we are 
rewriting (updated as they are replaced)
 private final MapDescriptor, DecoratedKey originalStarts = new 
HashMap(); // the start key for each reader we are rewriting
 private final MapDescriptor, Integer fileDescriptors = new 

[1/2] cassandra git commit: Handle abort() properly in SSTableRewriter

2014-11-26 Thread marcuse
Repository: cassandra
Updated Branches:
  refs/heads/trunk 999ce832d - 32b0a4e95


Handle abort() properly in SSTableRewriter

Patch by marcuse; reviewed by jmckenzie for CASSANDRA-8320


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

Branch: refs/heads/trunk
Commit: b1062929185690567e4567e0e657b361c5105482
Parents: 3faff8b
Author: Marcus Eriksson marc...@apache.org
Authored: Tue Nov 18 07:07:30 2014 +0100
Committer: Marcus Eriksson marc...@apache.org
Committed: Wed Nov 26 16:00:51 2014 +0100

--
 CHANGES.txt |   1 +
 .../cassandra/io/sstable/SSTableReader.java |  22 +++
 .../cassandra/io/sstable/SSTableRewriter.java   |  74 ++--
 .../io/sstable/SSTableRewriterTest.java | 180 +++
 4 files changed, 226 insertions(+), 51 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b1062929/CHANGES.txt
--
diff --git a/CHANGES.txt b/CHANGES.txt
index f022b19..e5f7c28 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.3
+ * Handle abort() in SSTableRewriter properly (CASSANDRA-8320)
  * Fix high size calculations for prepared statements (CASSANDRA-8231)
  * Centralize shared executors (CASSANDRA-8055)
  * Fix filtering for CONTAINS (KEY) relations on frozen collection

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b1062929/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
--
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java 
b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
index a3e3cf5..1fe4330 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
@@ -202,6 +202,7 @@ public class SSTableReader extends SSTable
 private Object replaceLock = new Object();
 private SSTableReader replacedBy;
 private SSTableReader replaces;
+private SSTableReader sharesBfWith;
 private SSTableDeletingTask deletingTask;
 private Runnable runOnClose;
 
@@ -594,6 +595,14 @@ public class SSTableReader extends SSTable
 deleteFiles = !dfile.path.equals(replaces.dfile.path);
 }
 
+if (sharesBfWith != null)
+{
+closeBf = sharesBfWith.bf != bf;
+closeSummary = sharesBfWith.indexSummary != indexSummary;
+closeFiles = sharesBfWith.dfile != dfile;
+deleteFiles = !dfile.path.equals(sharesBfWith.dfile.path);
+}
+
 boolean deleteAll = false;
 if (release  isCompacted.get())
 {
@@ -928,6 +937,19 @@ public class SSTableReader extends SSTable
 }
 }
 
+/**
+ * this is used to avoid closing the bloom filter multiple times when 
finishing an SSTableRewriter
+ *
+ * note that the reason we don't use replacedBy is that we are not yet 
actually replaced
+ *
+ * @param newReader
+ */
+public void sharesBfWith(SSTableReader newReader)
+{
+assert openReason.equals(OpenReason.EARLY);
+this.sharesBfWith = newReader;
+}
+
 public SSTableReader cloneWithNewStart(DecoratedKey newStart, final 
Runnable runOnClose)
 {
 synchronized (replaceLock)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b1062929/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
--
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java 
b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
index 4d5a06f..d187e9d 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableRewriter.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.io.sstable;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -75,6 +76,7 @@ public class SSTableRewriter
 private final ColumnFamilyStore cfs;
 
 private final long maxAge;
+private final ListSSTableReader finished = new ArrayList();
 private final SetSSTableReader rewriting; // the readers we are 
rewriting (updated as they are replaced)
 private final MapDescriptor, DecoratedKey originalStarts = new 
HashMap(); // the start key for each reader we are rewriting
 private final MapDescriptor, Integer fileDescriptors = new HashMap(); 
// the