You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2011/06/21 21:47:37 UTC

svn commit: r1138148 - in /cassandra/branches/cassandra-0.7: CHANGES.txt src/java/org/apache/cassandra/streaming/StreamOutSession.java

Author: slebresne
Date: Tue Jun 21 19:47:36 2011
New Revision: 1138148

URL: http://svn.apache.org/viewvc?rev=1138148&view=rev
Log:
Improve thread safety in StreamOutSession
patch by slebresne; reviewed by jbellis for CASSANDRA-2792

Modified:
    cassandra/branches/cassandra-0.7/CHANGES.txt
    cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/streaming/StreamOutSession.java

Modified: cassandra/branches/cassandra-0.7/CHANGES.txt
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/CHANGES.txt?rev=1138148&r1=1138147&r2=1138148&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/CHANGES.txt (original)
+++ cassandra/branches/cassandra-0.7/CHANGES.txt Tue Jun 21 19:47:36 2011
@@ -18,7 +18,8 @@
  * fix nodetool ring use with Ec2Snitch (CASSANDRA-2733)
  * fix removing columns and subcolumns that are supressed by a row or
    supercolumn tombstone during replica resolution (CASSANDRA-2590)
- * use threadsafe collections for StreamInSession (CASSANDRA-2766)
+ * use threadsafe collections for StreamInSession and StreamOutSession
+   (CASSANDRA-2766, CASSANDRA-2792)
 
 
 0.7.6

Modified: cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/streaming/StreamOutSession.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/streaming/StreamOutSession.java?rev=1138148&r1=1138147&r2=1138148&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/streaming/StreamOutSession.java (original)
+++ cassandra/branches/cassandra-0.7/src/java/org/apache/cassandra/streaming/StreamOutSession.java Tue Jun 21 19:47:36 2011
@@ -64,12 +64,12 @@ public class StreamOutSession
         return streams.get(new Pair<InetAddress, Long>(host, sessionId));
     }
 
-    private final Map<String, PendingFile> files = new LinkedHashMap<String, PendingFile>();
+    private final Map<String, PendingFile> files = new NonBlockingHashMap<String, PendingFile>();
 
     public final String table;
     private final Pair<InetAddress, Long> context;
     private final Runnable callback;
-    private String currentFile;
+    private volatile String currentFile;
 
     private StreamOutSession(String table, Pair<InetAddress, Long> context, Runnable callback)
     {