Author: jbellis
Date: Tue Sep 14 20:34:53 2010
New Revision: 997082
URL: http://svn.apache.org/viewvc?rev=997082&view=rev
Log:
add back tablename parameter in preparation for merging initiated/non code
paths. patch by jbellis
Modified:
cassandra/trunk/src/java/org/apache/cassandra/service/AntiEntropyService.java
cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java
cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
Modified:
cassandra/trunk/src/java/org/apache/cassandra/service/AntiEntropyService.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/service/AntiEntropyService.java?rev=997082&r1=997081&r2=997082&view=diff
==============================================================================
---
cassandra/trunk/src/java/org/apache/cassandra/service/AntiEntropyService.java
(original)
+++
cassandra/trunk/src/java/org/apache/cassandra/service/AntiEntropyService.java
Tue Sep 14 20:34:53 2010
@@ -539,7 +539,7 @@ public class AntiEntropyService
protected void runMayThrow() throws Exception
{
StreamContext context = new
StreamContext(request.endpoint);
- StreamOut.transferSSTables(context, sstables, ranges);
+ StreamOut.transferSSTables(context, request.cf.left,
sstables, ranges);
StreamOutManager.remove(context);
}
});
Modified: cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java?rev=997082&r1=997081&r2=997082&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/streaming/StreamOut.java Tue
Sep 14 20:34:53 2010
@@ -76,7 +76,7 @@ public class StreamOut
{
Table table = flushSSTable(tableName);
// send the matching portion of every sstable in the keyspace
- transferSSTables(context, table.getAllSSTables(), ranges);
+ transferSSTables(context, tableName, table.getAllSSTables(),
ranges);
}
catch (IOException e)
{
@@ -144,7 +144,7 @@ public class StreamOut
/**
* Transfers matching portions of a group of sstables from a single table
to the target endpoint.
*/
- public static void transferSSTables(StreamContext context,
Collection<SSTableReader> sstables, Collection<Range> ranges) throws IOException
+ public static void transferSSTables(StreamContext context, String table,
Collection<SSTableReader> sstables, Collection<Range> ranges) throws IOException
{
List<PendingFile> pending = createPendingFiles(sstables, ranges);
Modified:
cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
URL:
http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java?rev=997082&r1=997081&r2=997082&view=diff
==============================================================================
---
cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
(original)
+++
cassandra/trunk/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
Tue Sep 14 20:34:53 2010
@@ -34,6 +34,8 @@ import org.apache.cassandra.dht.Range;
import org.apache.cassandra.io.sstable.SSTableUtils;
import org.apache.cassandra.io.sstable.SSTableReader;
import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.streaming.StreamContext;
+import org.apache.cassandra.streaming.StreamOut;
import org.apache.cassandra.utils.FBUtilities;
import org.junit.BeforeClass;
@@ -66,7 +68,7 @@ public class StreamingTransferTest exten
List<Range> ranges = new ArrayList<Range>();
ranges.add(new Range(p.getMinimumToken(),
p.getToken("key".getBytes())));
ranges.add(new Range(p.getToken("key2".getBytes()),
p.getMinimumToken()));
- StreamOut.transferSSTables(new StreamContext(LOCAL),
Arrays.asList(sstable), ranges);
+ StreamOut.transferSSTables(new StreamContext(LOCAL), tablename,
Arrays.asList(sstable), ranges);
// confirm that the SSTable was transferred and registered
ColumnFamilyStore cfstore =
Table.open(tablename).getColumnFamilyStore(cfname);
@@ -106,7 +108,7 @@ public class StreamingTransferTest exten
List<Range> ranges = new ArrayList<Range>();
ranges.add(new Range(p.getMinimumToken(),
p.getToken("transfer1".getBytes())));
ranges.add(new Range(p.getToken("test2".getBytes()),
p.getMinimumToken()));
- StreamOut.transferSSTables(new StreamContext(LOCAL),
Arrays.asList(sstable, sstable2), ranges);
+ StreamOut.transferSSTables(new StreamContext(LOCAL), tablename,
Arrays.asList(sstable, sstable2), ranges);
// confirm that the SSTable was transferred and registered
ColumnFamilyStore cfstore =
Table.open(tablename).getColumnFamilyStore(cfname);