You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2015/12/30 09:01:31 UTC

cassandra git commit: Correctly count received sstables during streaming

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 36608cefa -> 3d78939a8


Correctly count received sstables during streaming

Patch by marcuse; reviewed by yukim for CASSANDRA-10949


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

Branch: refs/heads/cassandra-3.0
Commit: 3d78939a831eae5c5ae72bc977a9bb0107c24bab
Parents: 36608ce
Author: Marcus Eriksson <ma...@apache.org>
Authored: Mon Dec 28 16:34:03 2015 +0100
Committer: Marcus Eriksson <ma...@apache.org>
Committed: Wed Dec 30 08:57:13 2015 +0100

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


http://git-wip-us.apache.org/repos/asf/cassandra/blob/3d78939a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 837a592..dd517f5 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.3
+ * Fix counting of received sstables in streaming (CASSANDRA-10949)
  * Implement hints compression (CASSANDRA-9428)
  * Fix potential assertion error when reading static columns (CASSANDRA-10903)
  * Avoid NoSuchElementException when executing empty batch (CASSANDRA-10711)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3d78939a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
index 92a14d1..6280f3a 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -74,6 +74,8 @@ public class StreamReceiveTask extends StreamTask
     //  holds references to SSTables received
     protected Collection<SSTableReader> sstables;
 
+    private int remoteSSTablesReceived = 0;
+
     public StreamReceiveTask(StreamSession session, UUID cfId, int totalFiles, long totalSize)
     {
         super(session, cfId);
@@ -94,14 +96,14 @@ public class StreamReceiveTask extends StreamTask
     {
         if (done)
             return;
-
+        remoteSSTablesReceived++;
         assert cfId.equals(sstable.getCfId());
 
         Collection<SSTableReader> finished = sstable.finish(true);
         txn.update(finished, false);
         sstables.addAll(finished);
 
-        if (sstables.size() == totalFiles)
+        if (remoteSSTablesReceived == totalFiles)
         {
             done = true;
             executor.submit(new OnCompletionRunnable(this));