Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 9141f7ba7 -> 90477c310


Fix NPE in StreamTransferTask.createMessageForRetry()

patch by Josh McKenzie; reviewed by Aleksey Yeschenko for CASSANDRA-7323


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

Branch: refs/heads/cassandra-2.1
Commit: 886d9fce46b369807071c5fc87204bb552db906d
Parents: 8480fa7
Author: Josh McKenzie <josh.mcken...@datastax.com>
Authored: Fri May 30 23:49:09 2014 +0300
Committer: Aleksey Yeschenko <alek...@apache.org>
Committed: Fri May 30 23:49:09 2014 +0300

----------------------------------------------------------------------
 CHANGES.txt                                                     | 1 +
 src/java/org/apache/cassandra/streaming/StreamTransferTask.java | 5 +++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/886d9fce/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 14b43e5..d1d1030 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.0.9
+ * Fix NPE in StreamTransferTask.createMessageForRetry() (CASSANDRA-7323)
  * Add conditional CREATE/DROP USER support (CASSANDRA-7264)
  * Swap local and global default read repair chances (CASSANDRA-7320)
  * Add missing iso8601 patterns for date strings (CASSANDRA-6973)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/886d9fce/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamTransferTask.java 
b/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
index 7d0cc87..a543d01 100644
--- a/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
@@ -99,11 +99,12 @@ public class StreamTransferTask extends StreamTask
         return new ArrayList<>(files.values());
     }
 
-    public OutgoingFileMessage createMessageForRetry(int sequenceNumber)
+    public synchronized OutgoingFileMessage createMessageForRetry(int 
sequenceNumber)
     {
         // remove previous time out task to be rescheduled later
         ScheduledFuture future = timeoutTasks.get(sequenceNumber);
-        future.cancel(false);
+        if (future != null)
+            future.cancel(false);
         return files.get(sequenceNumber);
     }
 

Reply via email to