Repository: cassandra
Updated Branches:
  refs/heads/trunk 2d8be34a2 -> 25580e3a7


Avoid synchronized on prepareForRepair in ActiveRepairService

Patch by marcuse; reviewed by Blake Eggleston for CASSANDRA-9292


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

Branch: refs/heads/trunk
Commit: 25580e3a7f4bc01aa914a713ada45c50a45d0993
Parents: 2d8be34
Author: Marcus Eriksson <[email protected]>
Authored: Tue Feb 21 06:37:42 2017 +0100
Committer: Marcus Eriksson <[email protected]>
Committed: Fri Feb 24 17:08:01 2017 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                    | 1 +
 src/java/org/apache/cassandra/service/ActiveRepairService.java | 6 +++---
 2 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/25580e3a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f541c9a..a79315b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Avoid synchronized on prepareForRepair in ActiveRepairService 
(CASSANDRA-9292)
  * Adds the ability to use uncompressed chunks in compressed files 
(CASSANDRA-10520)
  * Don't flush sstables when streaming for incremental repair (CASSANDRA-13226)
  * Remove unused method (CASSANDRA-13227)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/25580e3a/src/java/org/apache/cassandra/service/ActiveRepairService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java 
b/src/java/org/apache/cassandra/service/ActiveRepairService.java
index 518cd8e..5e33913 100644
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@ -327,7 +327,7 @@ public class ActiveRepairService implements 
IEndpointStateChangeSubscriber, IFai
         return neighbors;
     }
 
-    public synchronized UUID prepareForRepair(UUID parentRepairSession, 
InetAddress coordinator, Set<InetAddress> endpoints, RepairOption options, 
List<ColumnFamilyStore> columnFamilyStores)
+    public UUID prepareForRepair(UUID parentRepairSession, InetAddress 
coordinator, Set<InetAddress> endpoints, RepairOption options, 
List<ColumnFamilyStore> columnFamilyStores)
     {
         // we only want repairedAt for incremental repairs, for non 
incremental repairs, UNREPAIRED_SSTABLE will preserve repairedAt on streamed 
sstables
         long repairedAt = options.isIncremental() ? 
Clock.instance.currentTimeMillis() : ActiveRepairService.UNREPAIRED_SSTABLE;
@@ -365,7 +365,7 @@ public class ActiveRepairService implements 
IEndpointStateChangeSubscriber, IFai
             {
                 PrepareMessage message = new 
PrepareMessage(parentRepairSession, tableIds, options.getRanges(), 
options.isIncremental(), repairedAt, options.isGlobal());
                 MessageOut<RepairMessage> msg = message.createMessage();
-                MessagingService.instance().sendRR(msg, neighbour, callback, 
TimeUnit.HOURS.toMillis(1), true);
+                MessagingService.instance().sendRR(msg, neighbour, callback, 
DatabaseDescriptor.getRpcTimeout(), true);
             }
             else
             {
@@ -376,7 +376,7 @@ public class ActiveRepairService implements 
IEndpointStateChangeSubscriber, IFai
         }
         try
         {
-            prepareLatch.await(1, TimeUnit.HOURS);
+            prepareLatch.await(DatabaseDescriptor.getRpcTimeout(), 
TimeUnit.MILLISECONDS);
         }
         catch (InterruptedException e)
         {

Reply via email to