Fix repair hang when snapshot failed

patch by yukim reviewed by marcuse for CASSANDRA-10057


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

Branch: refs/heads/trunk
Commit: 0a4ce4449f137a959b02fee14e1fdd427ccd520e
Parents: c2b6dd4
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue Sep 1 08:58:34 2015 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Tue Sep 1 09:20:06 2015 -0500

----------------------------------------------------------------------
 CHANGES.txt                                                 | 1 +
 .../apache/cassandra/repair/RepairMessageVerbHandler.java   | 9 +++++++--
 2 files changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a4ce444/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index d9f8a50..ffff102 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.2.2
+ * Fix repair hang when snapshot failed (CASSANDRA-10057)
  * Fall back to 1/4 commitlog volume for commitlog_total_space on small disks
    (CASSANDRA-10199)
 Merged from 2.1:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a4ce444/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java 
b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
index 796f135..41d79aa 100644
--- a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
+++ b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
@@ -88,7 +88,7 @@ public class RepairMessageVerbHandler implements 
IVerbHandler<RepairMessage>
 
                 case SNAPSHOT:
                     logger.debug("Snapshotting {}", desc);
-                    ColumnFamilyStore cfs = 
Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily);
+                    final ColumnFamilyStore cfs = 
Keyspace.open(desc.keyspace).getColumnFamilyStore(desc.columnFamily);
                     final Range<Token> repairingRange = desc.range;
                     Set<SSTableReader> snapshottedSSSTables = 
cfs.snapshot(desc.sessionId.toString(), new Predicate<SSTableReader>()
                     {
@@ -103,8 +103,13 @@ public class RepairMessageVerbHandler implements 
IVerbHandler<RepairMessage>
                     Set<SSTableReader> currentlyRepairing = 
ActiveRepairService.instance.currentlyRepairing(cfs.metadata.cfId, 
desc.parentSessionId);
                     if (!Sets.intersection(currentlyRepairing, 
snapshottedSSSTables).isEmpty())
                     {
+                        // clear snapshot that we just created
+                        cfs.clearSnapshot(desc.sessionId.toString());
                         logger.error("Cannot start multiple repair sessions 
over the same sstables");
-                        throw new RuntimeException("Cannot start multiple 
repair sessions over the same sstables");
+                        MessageOut reply = new 
MessageOut(MessagingService.Verb.INTERNAL_RESPONSE)
+                                               
.withParameter(MessagingService.FAILURE_RESPONSE_PARAM, 
MessagingService.ONE_BYTE);
+                        MessagingService.instance().sendReply(reply, id, 
message.from);
+                        return;
                     }
                     
ActiveRepairService.instance.getParentRepairSession(desc.parentSessionId).addSSTables(cfs.metadata.cfId,
 snapshottedSSSTables);
                     logger.debug("Enqueuing response to snapshot request {} to 
{}", desc.sessionId, message.from);

Reply via email to