Author: slebresne Date: Tue Oct 11 14:59:06 2011 New Revision: 1181820 URL: http://svn.apache.org/viewvc?rev=1181820&view=rev Log: patch by slebresne; reviewed by Brandon Williams for CASSANDRA-3345
Modified: cassandra/branches/cassandra-1.0.0/CHANGES.txt cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java Modified: cassandra/branches/cassandra-1.0.0/CHANGES.txt URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0.0/CHANGES.txt?rev=1181820&r1=1181819&r2=1181820&view=diff ============================================================================== --- cassandra/branches/cassandra-1.0.0/CHANGES.txt (original) +++ cassandra/branches/cassandra-1.0.0/CHANGES.txt Tue Oct 11 14:59:06 2011 @@ -14,6 +14,8 @@ * add bin\daemon to binary tarball for Windows service (CASSANDRA-3331) * Fix places where uncompressed size of sstables was use in place of the compressed one (CASSANDRA-3338) + * Fix hsha thrift server (CASSANDRA-3346) + * Make sure repair only stream needed sstables (CASSANDRA-3345) Fixes merged from 0.8 below: * Fix tool .bat files when CASSANDRA_HOME contains spaces (CASSANDRA-3258) * Force flush of status table when removing/updating token (CASSANDRA-3243) Modified: cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java?rev=1181820&r1=1181819&r2=1181820&view=diff ============================================================================== --- cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java (original) +++ cassandra/branches/cassandra-1.0.0/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java Tue Oct 11 14:59:06 2011 @@ -121,7 +121,7 @@ public class StreamingRepairTask impleme StreamOutSession outsession = StreamOutSession.create(tableName, dst, callback); StreamOut.transferSSTables(outsession, sstables, ranges, OperationType.AES); // request ranges from the remote node - StreamIn.requestRanges(dst, tableName, ranges, callback, OperationType.AES); + StreamIn.requestRanges(dst, tableName, Collections.singleton(cfstore), ranges, callback, OperationType.AES); } catch(Exception e) {