Author: gdusbabek
Date: Mon May 24 16:01:48 2010
New Revision: 947691

URL: http://svn.apache.org/viewvc?rev=947691&view=rev
Log:
revert changes for CASSANDRA-956

Modified:
    
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamIn.java
    
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamInManager.java
    
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamInitiateVerbHandler.java
    
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamOut.java

Modified: 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamIn.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamIn.java?rev=947691&r1=947690&r2=947691&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamIn.java
 (original)
+++ 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamIn.java
 Mon May 24 16:01:48 2010
@@ -44,7 +44,6 @@ public class StreamIn
     {
         if (logger.isDebugEnabled())
             logger.debug("Requesting from " + source + " ranges " + 
StringUtils.join(ranges, ", "));
-        StreamInManager.waitingForAnticompaction.put(source, tableName);
         StreamRequestMetadata streamRequestMetadata = new 
StreamRequestMetadata(FBUtilities.getLocalAddress(), ranges, tableName);
         Message message = StreamRequestMessage.makeStreamRequestMessage(new 
StreamRequestMessage(streamRequestMetadata));
         MessagingService.instance.sendOneWay(message, source);

Modified: 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamInManager.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamInManager.java?rev=947691&r1=947690&r2=947691&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamInManager.java
 (original)
+++ 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamInManager.java
 Mon May 24 16:01:48 2010
@@ -40,9 +40,6 @@ class StreamInManager
     public static final Map<InetAddress, IStreamComplete> 
streamNotificationHandlers_ = new HashMap<InetAddress, IStreamComplete>();
 
     public static final Multimap<InetAddress, PendingFile> activeStreams = 
Multimaps.synchronizedMultimap(HashMultimap.<InetAddress, PendingFile>create());
-    
-    /** keep track of which hosts this node has sent requests for ranges to. */
-    public static final Multimap<InetAddress, String> waitingForAnticompaction 
= Multimaps.synchronizedMultimap(HashMultimap.<InetAddress, String>create());
 
     public synchronized static PendingFile getStreamContext(InetAddress key)
     {        
@@ -72,7 +69,6 @@ class StreamInManager
         HashSet<InetAddress> set = new HashSet<InetAddress>();
         set.addAll(ctxBag_.keySet());
         set.addAll(activeStreams.keySet());
-        set.addAll(waitingForAnticompaction.keySet());
         return set;
     }
 

Modified: 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamInitiateVerbHandler.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamInitiateVerbHandler.java?rev=947691&r1=947690&r2=947691&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamInitiateVerbHandler.java
 (original)
+++ 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamInitiateVerbHandler.java
 Mon May 24 16:01:48 2010
@@ -70,7 +70,6 @@ public class StreamInitiateVerbHandler i
                     logger.debug("no data needed from " + message.getFrom());
                 if (StorageService.instance.isBootstrapMode())
                     
StorageService.instance.removeBootstrapSource(message.getFrom(), new 
String(message.getHeader(StreamOut.TABLE_NAME)));
-                
StreamInManager.waitingForAnticompaction.removeAll(message.getFrom());
                 return;
             }
 
@@ -92,7 +91,6 @@ public class StreamInitiateVerbHandler i
                   logger.debug("Received Data from  : " + message.getFrom() + 
" " + pendingFile.getTargetFile() + " " + file);
                 pendingFile.setTargetFile(file);
                 addStreamContext(message.getFrom(), pendingFile, streamStatus);
-                
StreamInManager.waitingForAnticompaction.remove(message.getFrom(), 
pendingFile.getTable());
             }
 
             StreamInManager.registerStreamCompletionHandler(message.getFrom(), 
new StreamCompletionHandler());

Modified: 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamOut.java
URL: 
http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamOut.java?rev=947691&r1=947690&r2=947691&view=diff
==============================================================================
--- 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamOut.java
 (original)
+++ 
cassandra/branches/cassandra-0.6/src/java/org/apache/cassandra/streaming/StreamOut.java
 Mon May 24 16:01:48 2010
@@ -65,9 +65,6 @@ public class StreamOut
     public static void transferRanges(InetAddress target, String tableName, 
Collection<Range> ranges, Runnable callback)
     {
         assert ranges.size() > 0;
-        
-        // this is so that this target shows up as a destination while 
anticompaction is happening.
-        StreamOutManager.pendingDestinations.add(target);        
 
         logger.debug("Beginning transfer process to " + target + " for ranges 
" + StringUtils.join(ranges, ", "));
 


Reply via email to