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 2013/06/20 19:07:13 UTC

[1/5] Streaming 2.0

Updated Branches:
  refs/heads/trunk 40b6c5d9c -> 515116972


http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/test/unit/org/apache/cassandra/streaming/SerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/SerializationsTest.java b/test/unit/org/apache/cassandra/streaming/SerializationsTest.java
deleted file mode 100644
index 6db5b15..0000000
--- a/test/unit/org/apache/cassandra/streaming/SerializationsTest.java
+++ /dev/null
@@ -1,220 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.cassandra.streaming;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.IOException;
-import java.util.*;
-
-import org.junit.Test;
-
-import org.apache.cassandra.AbstractSerializationsTester;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.RowMutation;
-import org.apache.cassandra.db.Table;
-import org.apache.cassandra.dht.BytesToken;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTable;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.UUIDGen;
-
-public class SerializationsTest extends AbstractSerializationsTester
-{
-    private void testPendingFileWrite() throws IOException
-    {
-        // make sure to test serializing null and a pf with no sstable.
-        PendingFile normal = makePendingFile(true, 100, OperationType.BOOTSTRAP);
-        PendingFile noSections = makePendingFile(true, 0, OperationType.AES);
-        PendingFile noSST = makePendingFile(false, 100, OperationType.RESTORE_REPLICA_COUNT);
-
-        DataOutputStream out = getOutput("streaming.PendingFile.bin");
-        PendingFile.serializer.serialize(normal, out, getVersion());
-        PendingFile.serializer.serialize(noSections, out, getVersion());
-        PendingFile.serializer.serialize(noSST, out, getVersion());
-        PendingFile.serializer.serialize(null, out, getVersion());
-        out.close();
-
-        // test serializedSize
-        testSerializedSize(normal, PendingFile.serializer);
-        testSerializedSize(noSections, PendingFile.serializer);
-        testSerializedSize(noSST, PendingFile.serializer);
-        testSerializedSize(null, PendingFile.serializer);
-    }
-
-    @Test
-    public void testPendingFileRead() throws IOException
-    {
-        if (EXECUTE_WRITES)
-            testPendingFileWrite();
-
-        DataInputStream in = getInput("streaming.PendingFile.bin");
-        assert PendingFile.serializer.deserialize(in, getVersion()) != null;
-        assert PendingFile.serializer.deserialize(in, getVersion()) != null;
-        assert PendingFile.serializer.deserialize(in, getVersion()) != null;
-        assert PendingFile.serializer.deserialize(in, getVersion()) == null;
-        in.close();
-    }
-
-    private void testStreamHeaderWrite() throws IOException
-    {
-        UUID sessionId = UUIDGen.getTimeUUID();
-        StreamHeader sh0 = new StreamHeader("Keyspace1", sessionId, makePendingFile(true, 100, OperationType.BOOTSTRAP));
-        StreamHeader sh1 = new StreamHeader("Keyspace1", sessionId, makePendingFile(false, 100, OperationType.BOOTSTRAP));
-        Collection<PendingFile> files = new ArrayList<PendingFile>();
-        for (int i = 0; i < 50; i++)
-            files.add(makePendingFile(i % 2 == 0, 100, OperationType.BOOTSTRAP));
-        StreamHeader sh2 = new StreamHeader("Keyspace1", sessionId, makePendingFile(true, 100, OperationType.BOOTSTRAP), files);
-        StreamHeader sh3 = new StreamHeader("Keyspace1", sessionId, null, files);
-        StreamHeader sh4 = new StreamHeader("Keyspace1", sessionId, makePendingFile(true, 100, OperationType.BOOTSTRAP), new ArrayList<PendingFile>());
-
-        DataOutputStream out = getOutput("streaming.StreamHeader.bin");
-        StreamHeader.serializer.serialize(sh0, out, getVersion());
-        StreamHeader.serializer.serialize(sh1, out, getVersion());
-        StreamHeader.serializer.serialize(sh2, out, getVersion());
-        StreamHeader.serializer.serialize(sh3, out, getVersion());
-        StreamHeader.serializer.serialize(sh4, out, getVersion());
-        out.close();
-
-        // test serializedSize
-        testSerializedSize(sh0, StreamHeader.serializer);
-        testSerializedSize(sh1, StreamHeader.serializer);
-        testSerializedSize(sh2, StreamHeader.serializer);
-        testSerializedSize(sh3, StreamHeader.serializer);
-        testSerializedSize(sh4, StreamHeader.serializer);
-    }
-
-    @Test
-    public void testStreamHeaderRead() throws IOException
-    {
-        if (EXECUTE_WRITES)
-            testStreamHeaderWrite();
-
-        DataInputStream in = getInput("streaming.StreamHeader.bin");
-        assert StreamHeader.serializer.deserialize(in, getVersion()) != null;
-        assert StreamHeader.serializer.deserialize(in, getVersion()) != null;
-        assert StreamHeader.serializer.deserialize(in, getVersion()) != null;
-        assert StreamHeader.serializer.deserialize(in, getVersion()) != null;
-        assert StreamHeader.serializer.deserialize(in, getVersion()) != null;
-        in.close();
-    }
-
-    private void testStreamReplyWrite() throws IOException
-    {
-        UUID sessionId = UUIDGen.getTimeUUID();
-        StreamReply rep = new StreamReply("this is a file", sessionId, StreamReply.Status.FILE_FINISHED);
-        DataOutputStream out = getOutput("streaming.StreamReply.bin");
-        StreamReply.serializer.serialize(rep, out, getVersion());
-        rep.createMessage().serialize(out, getVersion());
-        out.close();
-
-        // test serializedSize
-        testSerializedSize(rep, StreamReply.serializer);
-    }
-
-    @Test
-    public void testStreamReplyRead() throws IOException
-    {
-        if (EXECUTE_WRITES)
-            testStreamReplyWrite();
-
-        DataInputStream in = getInput("streaming.StreamReply.bin");
-        assert StreamReply.serializer.deserialize(in, getVersion()) != null;
-        assert MessageIn.read(in, getVersion(), -1) != null;
-        in.close();
-    }
-
-    private static PendingFile makePendingFile(boolean sst, int numSecs, OperationType op)
-    {
-        Descriptor desc = new Descriptor("z", new File("path/doesn't/matter"), "Keyspace1", "Standard1", 23, false);
-        List<Pair<Long, Long>> sections = new ArrayList<Pair<Long, Long>>();
-        for (int i = 0; i < numSecs; i++)
-            sections.add(Pair.create(new Long(i), new Long(i * i)));
-        return new PendingFile(sst ? makeSSTable() : null, desc, SSTable.COMPONENT_DATA, sections, op);
-    }
-
-    private void testStreamRequestMessageWrite() throws IOException
-    {
-        UUID sessionId = UUIDGen.getTimeUUID();
-        Collection<Range<Token>> ranges = new ArrayList<Range<Token>>();
-        for (int i = 0; i < 5; i++)
-            ranges.add(new Range<Token>(new BytesToken(ByteBufferUtil.bytes(Integer.toString(10*i))), new BytesToken(ByteBufferUtil.bytes(Integer.toString(10*i+5)))));
-        List<ColumnFamilyStore> stores = Collections.singletonList(Table.open("Keyspace1").getColumnFamilyStore("Standard1"));
-        StreamRequest msg0 = new StreamRequest(FBUtilities.getBroadcastAddress(), ranges, "Keyspace1", stores, sessionId, OperationType.RESTORE_REPLICA_COUNT);
-        StreamRequest msg1 = new StreamRequest(FBUtilities.getBroadcastAddress(), makePendingFile(true, 100, OperationType.BOOTSTRAP), sessionId);
-        StreamRequest msg2 = new StreamRequest(FBUtilities.getBroadcastAddress(), makePendingFile(false, 100, OperationType.BOOTSTRAP), sessionId);
-
-        DataOutputStream out = getOutput("streaming.StreamRequestMessage.bin");
-        StreamRequest.serializer.serialize(msg0, out, getVersion());
-        StreamRequest.serializer.serialize(msg1, out, getVersion());
-        StreamRequest.serializer.serialize(msg2, out, getVersion());
-        msg0.createMessage().serialize(out, getVersion());
-        msg1.createMessage().serialize(out, getVersion());
-        msg2.createMessage().serialize(out, getVersion());
-        out.close();
-
-        // test serializedSize
-        testSerializedSize(msg0, StreamRequest.serializer);
-        testSerializedSize(msg1, StreamRequest.serializer);
-        testSerializedSize(msg2, StreamRequest.serializer);
-    }
-
-    @Test
-    public void testStreamRequestMessageRead() throws IOException
-    {
-        if (EXECUTE_WRITES)
-            testStreamRequestMessageWrite();
-
-        DataInputStream in = getInput("streaming.StreamRequestMessage.bin");
-        assert StreamRequest.serializer.deserialize(in, getVersion()) != null;
-        assert StreamRequest.serializer.deserialize(in, getVersion()) != null;
-        assert StreamRequest.serializer.deserialize(in, getVersion()) != null;
-        assert MessageIn.read(in, getVersion(), -1) != null;
-        assert MessageIn.read(in, getVersion(), -1) != null;
-        assert MessageIn.read(in, getVersion(), -1) != null;
-        in.close();
-    }
-
-    private static SSTableReader makeSSTable()
-    {
-        Table t = Table.open("Keyspace1");
-        for (int i = 0; i < 100; i++)
-        {
-            RowMutation rm = new RowMutation(t.getName(), ByteBufferUtil.bytes(Long.toString(System.nanoTime())));
-            rm.add("Standard1", ByteBufferUtil.bytes("cola"), ByteBufferUtil.bytes("value"), 0);
-            rm.apply();
-        }
-        try
-        {
-            t.getColumnFamilyStore("Standard1").forceBlockingFlush();
-            return t.getColumnFamilyStore("Standard1").getSSTables().iterator().next();
-        }
-        catch (Exception any)
-        {
-            throw new RuntimeException(any);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java b/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java
new file mode 100644
index 0000000..60fbf40
--- /dev/null
+++ b/test/unit/org/apache/cassandra/streaming/SessionInfoTest.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
+
+import org.junit.Test;
+
+import org.apache.cassandra.utils.FBUtilities;
+
+public class SessionInfoTest
+{
+    /**
+     * Test if total numbers are collect
+     */
+    @Test
+    public void testTotals()
+    {
+        UUID cfId = UUID.randomUUID();
+        InetAddress local = FBUtilities.getLocalAddress();
+
+        Collection<StreamSummary> summaries = new ArrayList<>();
+        for (int i = 0; i < 10; i++)
+        {
+            StreamSummary summary = new StreamSummary(cfId, i, (i + 1) * 10);
+            summaries.add(summary);
+        }
+
+        StreamSummary sending = new StreamSummary(cfId, 10, 100);
+        SessionInfo info = new SessionInfo(local, summaries, Collections.singleton(sending), StreamSession.State.PREPARING);
+
+        assert info.getTotalFilesToReceive() == 45;
+        assert info.getTotalFilesToSend() == 10;
+        assert info.getTotalSizeToReceive() == 550;
+        assert info.getTotalSizeToSend() == 100;
+        // still, no files received or sent
+        assert info.getTotalFilesReceived() == 0;
+        assert info.getTotalFilesSent() == 0;
+
+        // receive in progress
+        info.updateProgress(new ProgressInfo(local, "test.txt", ProgressInfo.Direction.IN, 50, 100));
+        // still in progress, but not completed yet
+        assert info.getTotalSizeReceived() == 50;
+        assert info.getTotalSizeSent() == 0;
+        assert info.getTotalFilesReceived() == 0;
+        assert info.getTotalFilesSent() == 0;
+        info.updateProgress(new ProgressInfo(local, "test.txt", ProgressInfo.Direction.IN, 100, 100));
+        // 1 file should be completed
+        assert info.getTotalSizeReceived() == 100;
+        assert info.getTotalSizeSent() == 0;
+        assert info.getTotalFilesReceived() == 1;
+        assert info.getTotalFilesSent() == 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/test/unit/org/apache/cassandra/streaming/StreamUtil.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamUtil.java b/test/unit/org/apache/cassandra/streaming/StreamUtil.java
deleted file mode 100644
index 4987923..0000000
--- a/test/unit/org/apache/cassandra/streaming/StreamUtil.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*    http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing,
-* software distributed under the License is distributed on an
-* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-* KIND, either express or implied.  See the License for the
-* specific language governing permissions and limitations
-* under the License.
-*/
-
-package org.apache.cassandra.streaming;
-
-import java.io.IOError;
-import java.io.IOException;
-import java.net.InetAddress;
-
-import org.apache.cassandra.net.MessageIn;
-
-public class StreamUtil
-{
-    /**
-     * Takes an stream request message and creates an empty status response. Exists here because StreamRequestMessage
-     * is package protected.
-     */
-    static public void finishStreamRequest(MessageIn<StreamRequest> msg, InetAddress to)
-    {
-        StreamInSession session = StreamInSession.get(to, msg.payload.sessionId);
-        try
-        {
-            session.closeIfFinished();
-        }
-        catch (IOException e)
-        {
-            throw new IOError(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
index a86330c..3e9eeb1 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -1,35 +1,40 @@
-package org.apache.cassandra.streaming;
-
 /*
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*    http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing,
-* software distributed under the License is distributed on an
-* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-* KIND, either express or implied.  See the License for the
-* specific language governing permissions and limitations
-* under the License.
-*/
-
-import static junit.framework.Assert.assertEquals;
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
-import org.apache.cassandra.Util;
-import static org.apache.cassandra.Util.column;
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
 
 import java.net.InetAddress;
-import java.sql.Date;
 import java.nio.ByteBuffer;
+import java.sql.Date;
 import java.util.*;
+import java.util.concurrent.TimeUnit;
 
+import com.google.common.collect.Iterables;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
@@ -39,21 +44,18 @@ import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.sstable.SSTableUtils;
 import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.SSTableUtils;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.thrift.IndexExpression;
 import org.apache.cassandra.thrift.IndexOperator;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CounterId;
 import org.apache.cassandra.utils.FBUtilities;
 
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.utils.ByteBufferUtil;
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.fail;
+import static org.apache.cassandra.Util.column;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
 public class StreamingTransferTest extends SchemaLoader
@@ -69,10 +71,64 @@ public class StreamingTransferTest extends SchemaLoader
     }
 
     /**
+     * Test if empty {@link StreamPlan} returns success with empty result.
+     */
+    @Test
+    public void testEmptyStreamPlan() throws Exception
+    {
+        StreamResultFuture futureResult = new StreamPlan("StreamingTransferTest").execute();
+        final UUID planId = futureResult.planId;
+        Futures.addCallback(futureResult, new FutureCallback<StreamState>()
+        {
+            public void onSuccess(StreamState result)
+            {
+                assert planId.equals(result.planId);
+                assert result.description.equals("StreamingTransferTest");
+                assert result.sessions.isEmpty();
+            }
+
+            public void onFailure(Throwable t)
+            {
+                fail();
+            }
+        });
+        // should be complete immediately
+        futureResult.get(100, TimeUnit.MILLISECONDS);
+    }
+
+    @Test
+    public void testRequestEmpty() throws Exception
+    {
+        // requesting empty data should succeed
+        IPartitioner p = StorageService.getPartitioner();
+        List<Range<Token>> ranges = new ArrayList<>();
+        ranges.add(new Range<>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("key1"))));
+        ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("key2")), p.getMinimumToken()));
+
+        StreamResultFuture futureResult = new StreamPlan("StreamingTransferTest")
+                                                  .requestRanges(LOCAL, "Keyspace2", ranges)
+                                                  .execute();
+
+        UUID planId = futureResult.planId;
+        StreamState result = futureResult.get();
+        assert planId.equals(result.planId);
+        assert result.description.equals("StreamingTransferTest");
+
+        // we should have completed session with empty transfer
+        assert result.sessions.size() == 1;
+        SessionInfo session = Iterables.get(result.sessions, 0);
+        assert session.peer.equals(LOCAL);
+        assert session.getTotalFilesReceived() == 0;
+        assert session.getTotalFilesSent() == 0;
+        assert session.getTotalSizeReceived() == 0;
+        assert session.getTotalSizeSent() == 0;
+    }
+
+    /**
      * Create and transfer a single sstable, and return the keys that should have been transferred.
      * The Mutator must create the given column, but it may also create any other columns it pleases.
      */
-    private List<String> createAndTransfer(Table table, ColumnFamilyStore cfs, Mutator mutator) throws Exception
+    private List<String> createAndTransfer(ColumnFamilyStore cfs, Mutator mutator) throws Exception
     {
         // write a temporary SSTable, and unregister it
         logger.debug("Mutating " + cfs.name);
@@ -87,7 +143,7 @@ public class StreamingTransferTest extends SchemaLoader
 
         // transfer the first and last key
         logger.debug("Transferring " + cfs.name);
-        transfer(table, sstable);
+        transfer(sstable);
 
         // confirm that a single SSTable was transferred and registered
         assertEquals(1, cfs.getSSTables().size());
@@ -108,7 +164,7 @@ public class StreamingTransferTest extends SchemaLoader
         // and that the max timestamp for the file was rediscovered
         assertEquals(timestamp, cfs.getSSTables().iterator().next().getMaxTimestamp());
 
-        List<String> keys = new ArrayList<String>();
+        List<String> keys = new ArrayList<>();
         for (int off : offs)
             keys.add("key" + off);
 
@@ -116,20 +172,18 @@ public class StreamingTransferTest extends SchemaLoader
         return keys;
     }
 
-    private void transfer(Table table, SSTableReader sstable) throws Exception
+    private void transfer(SSTableReader sstable) throws Exception
     {
         IPartitioner p = StorageService.getPartitioner();
-        List<Range<Token>> ranges = new ArrayList<Range<Token>>();
-        ranges.add(new Range<Token>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("key1"))));
-        ranges.add(new Range<Token>(p.getToken(ByteBufferUtil.bytes("key2")), p.getMinimumToken()));
-        transfer(table, sstable, ranges);
+        List<Range<Token>> ranges = new ArrayList<>();
+        ranges.add(new Range<>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("key1"))));
+        ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("key2")), p.getMinimumToken()));
+        transfer(sstable, ranges);
     }
 
-    private void transfer(Table table, SSTableReader sstable, List<Range<Token>> ranges) throws Exception
+    private void transfer(SSTableReader sstable, List<Range<Token>> ranges) throws Exception
     {
-        StreamOutSession session = StreamOutSession.create(table.getName(), LOCAL, (IStreamCallback)null);
-        StreamOut.transferSSTables(session, Arrays.asList(sstable), ranges, OperationType.BOOTSTRAP);
-        session.await();
+        new StreamPlan("StreamingTransferTest").transferFiles(LOCAL, ranges, Collections.singleton(sstable)).execute().get();
     }
 
     /**
@@ -157,7 +211,7 @@ public class StreamingTransferTest extends SchemaLoader
 
         SSTableReader sstable = cfs.getSSTables().iterator().next();
         cfs.clearUnsafe();
-        transfer(table, sstable);
+        transfer(sstable);
 
         // confirm that a single SSTable was transferred and registered
         assertEquals(1, cfs.getSSTables().size());
@@ -172,7 +226,7 @@ public class StreamingTransferTest extends SchemaLoader
         final Table table = Table.open("Keyspace1");
         final ColumnFamilyStore cfs = table.getColumnFamilyStore("Indexed1");
 
-        List<String> keys = createAndTransfer(table, cfs, new Mutator()
+        List<String> keys = createAndTransfer(cfs, new Mutator()
         {
             public void mutate(String key, String col, long timestamp) throws Exception
             {
@@ -190,7 +244,6 @@ public class StreamingTransferTest extends SchemaLoader
         for (String key : keys)
         {
             long val = key.hashCode();
-            IPartitioner p = StorageService.getPartitioner();
             IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"),
                                                        IndexOperator.EQ,
                                                        ByteBufferUtil.bytes(val));
@@ -210,14 +263,14 @@ public class StreamingTransferTest extends SchemaLoader
         final ColumnFamilyStore cfs = table.getColumnFamilyStore("Counter1");
         final CounterContext cc = new CounterContext();
 
-        final Map<String, ColumnFamily> cleanedEntries = new HashMap<String, ColumnFamily>();
+        final Map<String, ColumnFamily> cleanedEntries = new HashMap<>();
 
-        List<String> keys = createAndTransfer(table, cfs, new Mutator()
+        List<String> keys = createAndTransfer(cfs, new Mutator()
         {
             /** Creates a new SSTable per key: all will be merged before streaming. */
             public void mutate(String key, String col, long timestamp) throws Exception
             {
-                Map<String, ColumnFamily> entries = new HashMap<String, ColumnFamily>();
+                Map<String, ColumnFamily> entries = new HashMap<>();
                 ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(cfs.metadata);
                 ColumnFamily cfCleaned = TreeMapBackedSortedColumns.factory.create(cfs.metadata);
                 CounterContext.ContextState state = CounterContext.ContextState.allocate(4, 1);
@@ -254,7 +307,7 @@ public class StreamingTransferTest extends SchemaLoader
 
         // Retransfer the file, making sure it is now idempotent (see CASSANDRA-3481)
         cfs.clearUnsafe();
-        transfer(table, streamed);
+        transfer(streamed);
         SSTableReader restreamed = cfs.getSSTables().iterator().next();
         SSTableUtils.assertContentEquals(streamed, restreamed);
     }
@@ -263,7 +316,7 @@ public class StreamingTransferTest extends SchemaLoader
     public void testTransferTableMultiple() throws Exception
     {
         // write temporary SSTables, but don't register them
-        Set<String> content = new HashSet<String>();
+        Set<String> content = new HashSet<>();
         content.add("test");
         content.add("test2");
         content.add("test3");
@@ -271,7 +324,7 @@ public class StreamingTransferTest extends SchemaLoader
         String tablename = sstable.getTableName();
         String cfname = sstable.getColumnFamilyName();
 
-        content = new HashSet<String>();
+        content = new HashSet<>();
         content.add("transfer1");
         content.add("transfer2");
         content.add("transfer3");
@@ -279,15 +332,13 @@ public class StreamingTransferTest extends SchemaLoader
 
         // transfer the first and last key
         IPartitioner p = StorageService.getPartitioner();
-        List<Range<Token>> ranges = new ArrayList<Range<Token>>();
-        ranges.add(new Range<Token>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("test"))));
-        ranges.add(new Range<Token>(p.getToken(ByteBufferUtil.bytes("transfer2")), p.getMinimumToken()));
+        List<Range<Token>> ranges = new ArrayList<>();
+        ranges.add(new Range<>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("test"))));
+        ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("transfer2")), p.getMinimumToken()));
         // Acquiring references, transferSSTables needs it
         sstable.acquireReference();
         sstable2.acquireReference();
-        StreamOutSession session = StreamOutSession.create(tablename, LOCAL, (IStreamCallback) null);
-        StreamOut.transferSSTables(session, Arrays.asList(sstable, sstable2), ranges, OperationType.BOOTSTRAP);
-        session.await();
+        new StreamPlan("StreamingTransferTest").transferFiles(LOCAL, ranges, Arrays.asList(sstable, sstable2)).execute().get();
 
         // confirm that the sstables were transferred and registered and that 2 keys arrived
         ColumnFamilyStore cfstore = Table.open(tablename).getColumnFamilyStore(cfname);
@@ -311,12 +362,12 @@ public class StreamingTransferTest extends SchemaLoader
         String keyspace = "KeyCacheSpace";
         IPartitioner p = StorageService.getPartitioner();
         String[] columnFamilies = new String[] { "Standard1", "Standard2", "Standard3" };
-        List<SSTableReader> ssTableReaders = new ArrayList<SSTableReader>();
+        List<SSTableReader> ssTableReaders = new ArrayList<>();
 
-        NavigableMap<DecoratedKey,String> keys = new TreeMap<DecoratedKey,String>();
+        NavigableMap<DecoratedKey,String> keys = new TreeMap<>();
         for (String cf : columnFamilies)
         {
-            Set<String> content = new HashSet<String>();
+            Set<String> content = new HashSet<>();
             content.add("data-" + cf + "-1");
             content.add("data-" + cf + "-2");
             content.add("data-" + cf + "-3");
@@ -332,19 +383,16 @@ public class StreamingTransferTest extends SchemaLoader
         Map.Entry<DecoratedKey,String> first = keys.firstEntry();
         Map.Entry<DecoratedKey,String> last = keys.lastEntry();
         Map.Entry<DecoratedKey,String> secondtolast = keys.lowerEntry(last.getKey());
-        List<Range<Token>> ranges = new ArrayList<Range<Token>>();
-        ranges.add(new Range<Token>(p.getMinimumToken(), first.getKey().token));
+        List<Range<Token>> ranges = new ArrayList<>();
+        ranges.add(new Range<>(p.getMinimumToken(), first.getKey().token));
         // the left hand side of the range is exclusive, so we transfer from the second-to-last token
-        ranges.add(new Range<Token>(secondtolast.getKey().token, p.getMinimumToken()));
+        ranges.add(new Range<>(secondtolast.getKey().token, p.getMinimumToken()));
 
         // Acquiring references, transferSSTables needs it
         if (!SSTableReader.acquireReferences(ssTableReaders))
             throw new AssertionError();
 
-        StreamOutSession session = StreamOutSession.create(keyspace, LOCAL, (IStreamCallback)null);
-        StreamOut.transferSSTables(session, ssTableReaders, ranges, OperationType.BOOTSTRAP);
-
-        session.await();
+        new StreamPlan("StreamingTransferTest").transferFiles(LOCAL, ranges, ssTableReaders).execute().get();
 
         // check that only two keys were transferred
         for (Map.Entry<DecoratedKey,String> entry : Arrays.asList(first, last))
@@ -382,11 +430,11 @@ public class StreamingTransferTest extends SchemaLoader
         cfs.clearUnsafe();
 
         IPartitioner p = StorageService.getPartitioner();
-        List<Range<Token>> ranges = new ArrayList<Range<Token>>();
-        ranges.add(new Range<Token>(p.getToken(ByteBufferUtil.bytes("key1")), p.getToken(ByteBufferUtil.bytes("key1000"))));
-        ranges.add(new Range<Token>(p.getToken(ByteBufferUtil.bytes("key5")), p.getToken(ByteBufferUtil.bytes("key500"))));
-        ranges.add(new Range<Token>(p.getToken(ByteBufferUtil.bytes("key9")), p.getToken(ByteBufferUtil.bytes("key900"))));
-        transfer(table, sstable, ranges);
+        List<Range<Token>> ranges = new ArrayList<>();
+        ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("key1")), p.getToken(ByteBufferUtil.bytes("key1000"))));
+        ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("key5")), p.getToken(ByteBufferUtil.bytes("key500"))));
+        ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("key9")), p.getToken(ByteBufferUtil.bytes("key900"))));
+        transfer(sstable, ranges);
         assertEquals(1, cfs.getSSTables().size());
         assertEquals(7, Util.getRangeSlice(cfs).size());
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java b/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
index 95297b1..ab311e6 100644
--- a/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
+++ b/test/unit/org/apache/cassandra/streaming/compress/CompressedInputStreamTest.java
@@ -26,7 +26,10 @@ import java.util.*;
 import org.junit.Test;
 
 import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.io.compress.*;
+import org.apache.cassandra.io.compress.CompressedSequentialWriter;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.compress.CompressionParameters;
+import org.apache.cassandra.io.compress.SnappyCompressor;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableMetadata;


[2/5] Streaming 2.0

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamingService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamingService.java b/src/java/org/apache/cassandra/streaming/StreamingService.java
deleted file mode 100644
index ea22f82..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamingService.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.streaming;
-
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
-
-
-public class StreamingService implements StreamingServiceMBean
-{
-    public static final String MBEAN_OBJECT_NAME = "org.apache.cassandra.net:type=StreamingService";
-    public static final StreamingService instance = new StreamingService();
-
-    private StreamingService()
-    {
-        MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
-        try
-        {
-            mbs.registerMBean(this, new ObjectName(MBEAN_OBJECT_NAME));
-        }
-        catch (Exception e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
-    public String getStatus()
-    {
-        StringBuilder sb = new StringBuilder();
-        sb.append("Receiving from:\n");
-        for (InetAddress source : StreamInSession.getSources())
-        {
-            sb.append(String.format(" %s:%n", source.getHostAddress()));
-            for (PendingFile pf : StreamInSession.getIncomingFiles(source))
-            {
-                sb.append(String.format("  %s%n", pf.toString()));
-            }
-        }
-        sb.append("Sending to:%n");
-        for (InetAddress dest : StreamOutSession.getDestinations())
-        {
-            sb.append(String.format(" %s:%n", dest.getHostAddress()));
-            for (PendingFile pf : StreamOutSession.getOutgoingFiles(dest))
-            {
-                sb.append(String.format("  %s%n", pf.toString()));
-            }
-        }
-        return sb.toString();
-    }
-
-    /** hosts receiving outgoing streams. */
-    public Set<InetAddress> getStreamDestinations()
-    {
-        return StreamOutSession.getDestinations();
-    }
-
-    /** outgoing streams */
-    public List<String> getOutgoingFiles(String host) throws IOException
-    {
-        List<String> files = new ArrayList<String>();
-        // first, verify that host is a destination. calling StreamOutManager.get will put it in the collection
-        // leading to false positives in the future.
-        Set<InetAddress> existingDestinations = getStreamDestinations();
-        InetAddress dest = InetAddress.getByName(host);
-        if (!existingDestinations.contains(dest))
-            return files;
-
-        for (PendingFile f : StreamOutSession.getOutgoingFiles(dest))
-            files.add(String.format("%s", f.toString()));
-        return files;
-    }
-
-    /** hosts sending incoming streams */
-    public Set<InetAddress> getStreamSources()
-    {
-        return StreamInSession.getSources();
-    }
-
-    /** details about incoming streams. */
-    public List<String> getIncomingFiles(String host) throws IOException
-    {
-        List<String> files = new ArrayList<String>();
-        for (PendingFile pf : StreamInSession.getIncomingFiles(InetAddress.getByName(host)))
-        {
-            files.add(String.format("%s: %s", pf.desc.ksname, pf.toString()));
-        }
-        return files;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamingServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamingServiceMBean.java b/src/java/org/apache/cassandra/streaming/StreamingServiceMBean.java
deleted file mode 100644
index 8d748f3..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamingServiceMBean.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.streaming;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.util.List;
-import java.util.Set;
-
-public interface StreamingServiceMBean
-{
-    /** hosts recieving outgoing streams */
-    public Set<InetAddress> getStreamDestinations();
-
-    /** outgoing streams */
-    public List<String> getOutgoingFiles(String host) throws IOException;
-
-    /** hosts sending incoming streams. */
-    public Set<InetAddress> getStreamSources();
-
-    /** details about incoming streams */
-    public List<String> getIncomingFiles(String host) throws IOException;
-
-    /** What's currently happening wrt streaming. */
-    public String getStatus();
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/compress/CompressedFileStreamTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedFileStreamTask.java b/src/java/org/apache/cassandra/streaming/compress/CompressedFileStreamTask.java
deleted file mode 100644
index 07fb765..0000000
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedFileStreamTask.java
+++ /dev/null
@@ -1,160 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.streaming.compress;
-
-import java.io.*;
-import java.net.InetAddress;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.nio.channels.SocketChannel;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.io.compress.CompressionMetadata;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.metrics.StreamingMetrics;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.streaming.FileStreamTask;
-import org.apache.cassandra.streaming.StreamHeader;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.Pair;
-
-/**
- * FileStreamTask for compressed SSTable.
- *
- * This class sends relevant part of compressed file directly using nio if available.
- */
-public class CompressedFileStreamTask extends FileStreamTask
-{
-    private static final Logger logger = LoggerFactory.getLogger(CompressedFileStreamTask.class);
-    // 10MB chunks
-    public static final int CHUNK_SIZE = 10*1024*1024;
-
-    public CompressedFileStreamTask(StreamHeader header, InetAddress to)
-    {
-        super(header, to);
-    }
-
-    protected void stream() throws IOException
-    {
-        assert header.file.compressionInfo != null;
-
-        SocketChannel sc = socket.getChannel();
-        byte[] transferBuffer = null;
-
-        // write header
-        ByteBuffer headerBuffer = MessagingService.instance().constructStreamHeader(header, false, MessagingService.instance().getVersion(to));
-        socket.getOutputStream().write(ByteBufferUtil.getArray(headerBuffer));
-
-        RandomAccessReader file = RandomAccessReader.open(new File(header.file.getFilename()));
-        FileChannel fc = file.getChannel();
-
-        StreamingMetrics.activeStreamsOutbound.inc();
-        // calculate chunks to transfer. we want to send continuous chunks altogether.
-        List<Pair<Long, Long>> sections = getTransferSections(header.file.compressionInfo.chunks);
-        try
-        {
-            long totalBytesTransferred = 0;
-            // stream each of the required sections of the file
-            for (Pair<Long, Long> section : sections)
-            {
-                // seek to the beginning of the section when socket channel is not available
-                if (sc == null)
-                    file.seek(section.left);
-                // length of the section to stream
-                long length = section.right - section.left;
-                // tracks write progress
-                long bytesTransferred = 0;
-                while (bytesTransferred < length)
-                {
-                    int toTransfer = (int) Math.min(CHUNK_SIZE, length - bytesTransferred);
-                    long lastWrite;
-                    if (sc != null)
-                    {
-                        lastWrite = fc.transferTo(section.left + bytesTransferred, toTransfer, sc);
-                        throttle.throttleDelta(lastWrite);
-                    }
-                    else
-                    {
-                        // NIO is not available. Fall back to normal streaming.
-                        // This happens when inter-node encryption is turned on.
-                        if (transferBuffer == null)
-                            transferBuffer = new byte[CHUNK_SIZE];
-                        file.readFully(transferBuffer, 0, toTransfer);
-                        socket.getOutputStream().write(transferBuffer, 0, toTransfer);
-                        throttle.throttleDelta(toTransfer);
-                        lastWrite = toTransfer;
-                    }
-                    totalBytesTransferred += lastWrite;
-                    bytesTransferred += lastWrite;
-                    header.file.progress += lastWrite;
-                }
-
-                if (sc == null)
-                    socket.getOutputStream().flush();
-
-                logger.debug("Bytes transferred " + bytesTransferred + "/" + header.file.size);
-            }
-            StreamingMetrics.totalOutgoingBytes.inc(totalBytesTransferred);
-            metrics.outgoingBytes.inc(totalBytesTransferred);
-            // receive reply confirmation
-            receiveReply();
-        }
-        finally
-        {
-            StreamingMetrics.activeStreamsOutbound.dec();
-
-            // no matter what happens close file
-            FileUtils.closeQuietly(file);
-        }
-    }
-
-    // chunks are assumed to be sorted by offset
-    private List<Pair<Long, Long>> getTransferSections(CompressionMetadata.Chunk[] chunks)
-    {
-        List<Pair<Long, Long>> transferSections = new ArrayList<Pair<Long, Long>>();
-        Pair<Long, Long> lastSection = null;
-        for (CompressionMetadata.Chunk chunk : chunks)
-        {
-            if (lastSection != null)
-            {
-                if (chunk.offset == lastSection.right)
-                {
-                    // extend previous section to end of this chunk
-                    lastSection = Pair.create(lastSection.left, chunk.offset + chunk.length + 4); // 4 bytes for CRC
-                }
-                else
-                {
-                    transferSections.add(lastSection);
-                    lastSection = Pair.create(chunk.offset, chunk.offset + chunk.length + 4);
-                }
-            }
-            else
-            {
-                lastSection = Pair.create(chunk.offset, chunk.offset + chunk.length + 4);
-            }
-        }
-        if (lastSection != null)
-            transferSections.add(lastSection);
-        return transferSections;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/compress/CompressedInputStream.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedInputStream.java b/src/java/org/apache/cassandra/streaming/compress/CompressedInputStream.java
index 6fe1793..3305f50 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedInputStream.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedInputStream.java
@@ -17,9 +17,12 @@
  */
 package org.apache.cassandra.streaming.compress;
 
-import java.io.*;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
 import java.util.Iterator;
-import java.util.concurrent.*;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
 import java.util.zip.CRC32;
 import java.util.zip.Checksum;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
new file mode 100644
index 0000000..f3e511b
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming.compress;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+
+import com.google.common.base.Throwables;
+
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.Table;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.SSTableWriter;
+import org.apache.cassandra.streaming.ProgressInfo;
+import org.apache.cassandra.streaming.StreamReader;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.messages.FileMessageHeader;
+import org.apache.cassandra.utils.BytesReadTracker;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * StreamReader that reads from streamed compressed SSTable
+ */
+public class CompressedStreamReader extends StreamReader
+{
+    protected final CompressionInfo compressionInfo;
+
+    public CompressedStreamReader(FileMessageHeader header, StreamSession session)
+    {
+        super(header, session);
+        this.compressionInfo = header.compressionInfo;
+    }
+
+    /**
+     * @return SSTable transferred
+     * @throws java.io.IOException if reading the remote sstable fails. Will throw an RTE if local write fails.
+     */
+    @Override
+    public SSTableReader read(ReadableByteChannel channel) throws IOException
+    {
+        long totalSize = totalSize();
+        CompressedInputStream cis = new CompressedInputStream(Channels.newInputStream(channel), compressionInfo);
+
+        Pair<String, String> kscf = Schema.instance.getCF(cfId);
+        ColumnFamilyStore cfs = Table.open(kscf.left).getColumnFamilyStore(kscf.right);
+        Directories.DataDirectory localDir = cfs.directories.getLocationCapableOfSize(totalSize);
+        if (localDir == null)
+            throw new IOException("Insufficient disk space to store " + totalSize + " bytes");
+        desc = Descriptor.fromFilename(cfs.getTempSSTablePath(cfs.directories.getLocationForDisk(localDir)));
+
+        SSTableWriter writer = new SSTableWriter(desc.filenameFor(Component.DATA), estimatedKeys);
+        try
+        {
+            BytesReadTracker in = new BytesReadTracker(new DataInputStream(cis));
+
+            for (Pair<Long, Long> section : sections)
+            {
+                long length = section.right - section.left;
+                // skip to beginning of section inside chunk
+                cis.position(section.left);
+                in.reset(0);
+                while (in.getBytesRead() < length)
+                {
+                    writeRow(writer, in, cfs);
+                    // when compressed, report total bytes of compressed chunks read since remoteFile.size is the sum of chunks transferred
+                    session.progress(desc, ProgressInfo.Direction.IN, cis.getTotalCompressedBytesRead(), totalSize);
+                }
+            }
+            return writer.closeAndOpenReader();
+        }
+        catch (Throwable e)
+        {
+            writer.abort();
+            if (e instanceof IOException)
+                throw (IOException) e;
+            else
+                throw Throwables.propagate(e);
+        }
+    }
+
+    @Override
+    protected long totalSize()
+    {
+        long size = 0;
+        // calculate total length of transferring chunks
+        for (CompressionMetadata.Chunk chunk : compressionInfo.chunks)
+            size += chunk.length + 4; // 4 bytes for CRC
+        return size;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java
new file mode 100644
index 0000000..80fcef5
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamWriter.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming.compress;
+
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.channels.WritableByteChannel;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.streaming.ProgressInfo;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamWriter;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * StreamWriter for compressed SSTable.
+ */
+public class CompressedStreamWriter extends StreamWriter
+{
+    public static final int CHUNK_SIZE = 10 * 1024 * 1024;
+
+    private final CompressionInfo compressionInfo;
+
+    public CompressedStreamWriter(SSTableReader sstable, Collection<Pair<Long, Long>> sections, CompressionInfo compressionInfo, StreamSession session)
+    {
+        super(sstable, sections, session);
+        this.compressionInfo = compressionInfo;
+    }
+
+    @Override
+    public void write(WritableByteChannel channel) throws IOException
+    {
+        long totalSize = totalSize();
+        RandomAccessReader file = sstable.openDataReader();
+        FileChannel fc = file.getChannel();
+
+        long progress = 0L;
+        // calculate chunks to transfer. we want to send continuous chunks altogether.
+        List<Pair<Long, Long>> sections = getTransferSections(compressionInfo.chunks);
+        try
+        {
+            // stream each of the required sections of the file
+            for (Pair<Long, Long> section : sections)
+            {
+                // length of the section to stream
+                long length = section.right - section.left;
+                // tracks write progress
+                long bytesTransferred = 0;
+                while (bytesTransferred < length)
+                {
+                    int toTransfer = (int) Math.min(CHUNK_SIZE, length - bytesTransferred);
+                    limiter.acquire(toTransfer);
+                    long lastWrite = fc.transferTo(section.left + bytesTransferred, toTransfer, channel);
+                    bytesTransferred += lastWrite;
+                    progress += lastWrite;
+                    session.progress(sstable.descriptor, ProgressInfo.Direction.OUT, progress, totalSize);
+                }
+            }
+        }
+        finally
+        {
+            // no matter what happens close file
+            FileUtils.closeQuietly(file);
+        }
+
+        sstable.releaseReference();
+    }
+
+    @Override
+    protected long totalSize()
+    {
+        long size = 0;
+        // calculate total length of transferring chunks
+        for (CompressionMetadata.Chunk chunk : compressionInfo.chunks)
+            size += chunk.length + 4; // 4 bytes for CRC
+        return size;
+    }
+
+    // chunks are assumed to be sorted by offset
+    private List<Pair<Long, Long>> getTransferSections(CompressionMetadata.Chunk[] chunks)
+    {
+        List<Pair<Long, Long>> transferSections = new ArrayList<>();
+        Pair<Long, Long> lastSection = null;
+        for (CompressionMetadata.Chunk chunk : chunks)
+        {
+            if (lastSection != null)
+            {
+                if (chunk.offset == lastSection.right)
+                {
+                    // extend previous section to end of this chunk
+                    lastSection = Pair.create(lastSection.left, chunk.offset + chunk.length + 4); // 4 bytes for CRC
+                }
+                else
+                {
+                    transferSections.add(lastSection);
+                    lastSection = Pair.create(chunk.offset, chunk.offset + chunk.length + 4);
+                }
+            }
+            else
+            {
+                lastSection = Pair.create(chunk.offset, chunk.offset + chunk.length + 4);
+            }
+        }
+        if (lastSection != null)
+            transferSections.add(lastSection);
+        return transferSections;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java b/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java
new file mode 100644
index 0000000..b561abc
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/CompleteMessage.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming.messages;
+
+import java.io.IOException;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+
+import org.apache.cassandra.streaming.StreamSession;
+
+public class CompleteMessage extends StreamMessage
+{
+    public static Serializer<CompleteMessage> serializer = new Serializer<CompleteMessage>()
+    {
+        public CompleteMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
+        {
+            return new CompleteMessage();
+        }
+
+        public void serialize(CompleteMessage message, WritableByteChannel out, int version, StreamSession session) throws IOException {}
+    };
+
+    public CompleteMessage()
+    {
+        super(Type.COMPLETE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/messages/FileMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/FileMessage.java b/src/java/org/apache/cassandra/streaming/messages/FileMessage.java
new file mode 100644
index 0000000..fe05eac
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/FileMessage.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming.messages;
+
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.util.List;
+
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.streaming.StreamReader;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamWriter;
+import org.apache.cassandra.streaming.compress.CompressedStreamReader;
+import org.apache.cassandra.streaming.compress.CompressedStreamWriter;
+import org.apache.cassandra.streaming.compress.CompressionInfo;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * FileMessage is used to transfer/receive the part(or whole) of a SSTable data file.
+ */
+public class FileMessage extends StreamMessage
+{
+    public static Serializer<FileMessage> serializer = new Serializer<FileMessage>()
+    {
+        public FileMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
+        {
+            DataInputStream input = new DataInputStream(Channels.newInputStream(in));
+            FileMessageHeader header = FileMessageHeader.serializer.deserialize(input, version);
+            StreamReader reader = header.compressionInfo == null ? new StreamReader(header, session)
+                                          : new CompressedStreamReader(header, session);
+
+            try
+            {
+                return new FileMessage(reader.read(in), header);
+            }
+            catch (Throwable e)
+            {
+                session.doRetry(header, e);
+                return null;
+            }
+        }
+
+        public void serialize(FileMessage message, WritableByteChannel out, int version, StreamSession session) throws IOException
+        {
+            DataOutput output = new DataOutputStream(Channels.newOutputStream(out));
+            FileMessageHeader.serializer.serialize(message.header, output, version);
+            StreamWriter writer = message.header.compressionInfo == null ?
+                                          new StreamWriter(message.sstable, message.header.sections, session) :
+                                          new CompressedStreamWriter(message.sstable,
+                                                                     message.header.sections,
+                                                                     message.header.compressionInfo, session);
+            writer.write(out);
+            session.fileSent(message.header);
+        }
+    };
+
+    public final FileMessageHeader header;
+    public final SSTableReader sstable;
+
+    public FileMessage(SSTableReader sstable, FileMessageHeader header)
+    {
+        super(Type.FILE);
+        this.header = header;
+        this.sstable = sstable;
+    }
+
+    public FileMessage(SSTableReader sstable, int sequenceNumber, long estimatedKeys, List<Pair<Long, Long>> sections)
+    {
+        super(Type.FILE);
+        this.sstable = sstable;
+
+        CompressionInfo compressionInfo = null;
+        if (sstable.compression)
+        {
+            CompressionMetadata meta = sstable.getCompressionMetadata();
+            compressionInfo = new CompressionInfo(meta.getChunksForSections(sections), meta.parameters);
+        }
+        this.header = new FileMessageHeader(sstable.metadata.cfId,
+                                            sequenceNumber,
+                                            sstable.descriptor.version.toString(),
+                                            estimatedKeys,
+                                            sections,
+                                            compressionInfo);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
new file mode 100644
index 0000000..761e086
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming.messages;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.compress.CompressionMetadata;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.streaming.compress.CompressionInfo;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.UUIDSerializer;
+
+/**
+ * StreamingFileHeader is appended before sending actual data to describe what it's sending.
+ */
+public class FileMessageHeader
+{
+    public static IVersionedSerializer<FileMessageHeader> serializer = new FileMessageHeaderSerializer();
+
+    public final UUID cfId;
+    public final int sequenceNumber;
+    /** SSTable version */
+    public final String version;
+    public final long estimatedKeys;
+    public final List<Pair<Long, Long>> sections;
+    public final CompressionInfo compressionInfo;
+
+    public FileMessageHeader(UUID cfId,
+                             int sequenceNumber,
+                             String version,
+                             long estimatedKeys,
+                             List<Pair<Long, Long>> sections,
+                             CompressionInfo compressionInfo)
+    {
+        this.cfId = cfId;
+        this.sequenceNumber = sequenceNumber;
+        this.version = version;
+        this.estimatedKeys = estimatedKeys;
+        this.sections = sections;
+        this.compressionInfo = compressionInfo;
+    }
+
+    /**
+     * @return total file size to transfer in bytes
+     */
+    public long size()
+    {
+        long size = 0;
+        if (compressionInfo != null)
+        {
+            // calculate total length of transferring chunks
+            for (CompressionMetadata.Chunk chunk : compressionInfo.chunks)
+                size += chunk.length + 4; // 4 bytes for CRC
+        }
+        else
+        {
+            for (Pair<Long, Long> section : sections)
+                size += section.right - section.left;
+        }
+        return size;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        FileMessageHeader that = (FileMessageHeader) o;
+        return sequenceNumber == that.sequenceNumber && cfId.equals(that.cfId);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        int result = cfId.hashCode();
+        result = 31 * result + sequenceNumber;
+        return result;
+    }
+
+    static class FileMessageHeaderSerializer implements IVersionedSerializer<FileMessageHeader>
+    {
+        public void serialize(FileMessageHeader header, DataOutput out, int version) throws IOException
+        {
+            UUIDSerializer.serializer.serialize(header.cfId, out, version);
+            out.writeInt(header.sequenceNumber);
+            out.writeUTF(header.version);
+            out.writeLong(header.estimatedKeys);
+
+            out.writeInt(header.sections.size());
+            for (Pair<Long, Long> section : header.sections)
+            {
+                out.writeLong(section.left);
+                out.writeLong(section.right);
+            }
+            CompressionInfo.serializer.serialize(header.compressionInfo, out, version);
+        }
+
+        public FileMessageHeader deserialize(DataInput in, int version) throws IOException
+        {
+            UUID cfId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
+            int sequenceNumber = in.readInt();
+            String sstableVersion = in.readUTF();
+            long estimatedKeys = in.readLong();
+            int count = in.readInt();
+            List<Pair<Long, Long>> sections = new ArrayList<>(count);
+            for (int k = 0; k < count; k++)
+                sections.add(Pair.create(in.readLong(), in.readLong()));
+            CompressionInfo compressionInfo = CompressionInfo.serializer.deserialize(in, MessagingService.current_version);
+            return new FileMessageHeader(cfId, sequenceNumber, sstableVersion, estimatedKeys, sections, compressionInfo);
+        }
+
+        public long serializedSize(FileMessageHeader header, int version)
+        {
+            long size = UUIDSerializer.serializer.serializedSize(header.cfId, version);
+            size += TypeSizes.NATIVE.sizeof(header.sequenceNumber);
+            size += TypeSizes.NATIVE.sizeof(header.version);
+            size += TypeSizes.NATIVE.sizeof(header.estimatedKeys);
+
+            size += TypeSizes.NATIVE.sizeof(header.sections.size());
+            for (Pair<Long, Long> section : header.sections)
+            {
+                size += TypeSizes.NATIVE.sizeof(section.left);
+                size += TypeSizes.NATIVE.sizeof(section.right);
+            }
+            size += CompressionInfo.serializer.serializedSize(header.compressionInfo, version);
+            return size;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/messages/PrepareMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/PrepareMessage.java b/src/java/org/apache/cassandra/streaming/messages/PrepareMessage.java
new file mode 100644
index 0000000..ba94592
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/PrepareMessage.java
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming.messages;
+
+import java.io.*;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.util.ArrayList;
+import java.util.Collection;
+
+import org.apache.cassandra.streaming.StreamRequest;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.StreamSummary;
+
+public class PrepareMessage extends StreamMessage
+{
+    public static Serializer<PrepareMessage> serializer = new Serializer<PrepareMessage>()
+    {
+        public PrepareMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
+        {
+            DataInput input = new DataInputStream(Channels.newInputStream(in));
+            PrepareMessage message = new PrepareMessage();
+            // requests
+            int numRequests = input.readInt();
+            for (int i = 0; i < numRequests; i++)
+                message.requests.add(StreamRequest.serializer.deserialize(input, version));
+            // summaries
+            int numSummaries = input.readInt();
+            for (int i = 0; i < numSummaries; i++)
+                message.summaries.add(StreamSummary.serializer.deserialize(input, version));
+            return message;
+        }
+
+        public void serialize(PrepareMessage message, WritableByteChannel out, int version, StreamSession session) throws IOException
+        {
+            DataOutput output = new DataOutputStream(Channels.newOutputStream(out));
+            // requests
+            output.writeInt(message.requests.size());
+            for (StreamRequest request : message.requests)
+                StreamRequest.serializer.serialize(request, output, version);
+            // summaries
+            output.writeInt(message.summaries.size());
+            for (StreamSummary summary : message.summaries)
+                StreamSummary.serializer.serialize(summary, output, version);
+        }
+    };
+
+    /**
+     * Streaming requests
+     */
+    public final Collection<StreamRequest> requests = new ArrayList<>();
+
+    /**
+     * Summaries of streaming out
+     */
+    public final Collection<StreamSummary> summaries = new ArrayList<>();
+
+    public PrepareMessage()
+    {
+        super(Type.PREPARE);
+    }
+
+    @Override
+    public String toString()
+    {
+        final StringBuilder sb = new StringBuilder("PrepareMessage{");
+        sb.append(requests.size()).append(" requests, ");
+        int totalFile = 0;
+        for (StreamSummary summary : summaries)
+            totalFile += summary.files;
+        sb.append(totalFile).append(" files receiving");
+        sb.append('}');
+        return sb.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/messages/RetryMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/RetryMessage.java b/src/java/org/apache/cassandra/streaming/messages/RetryMessage.java
new file mode 100644
index 0000000..67b4dee
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/RetryMessage.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming.messages;
+
+import java.io.*;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+import java.util.UUID;
+
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.utils.UUIDSerializer;
+
+public class RetryMessage extends StreamMessage
+{
+    public static Serializer<RetryMessage> serializer = new Serializer<RetryMessage>()
+    {
+        public RetryMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
+        {
+            DataInput input = new DataInputStream(Channels.newInputStream(in));
+            return new RetryMessage(UUIDSerializer.serializer.deserialize(input, MessagingService.current_version), input.readInt());
+        }
+
+        public void serialize(RetryMessage message, WritableByteChannel out, int version, StreamSession session) throws IOException
+        {
+            DataOutput output = new DataOutputStream(Channels.newOutputStream(out));
+            UUIDSerializer.serializer.serialize(message.cfId, output, MessagingService.current_version);
+            output.writeInt(message.sequenceNumber);
+        }
+    };
+
+    public final UUID cfId;
+    public final int sequenceNumber;
+
+    public RetryMessage(UUID cfId, int sequenceNumber)
+    {
+        super(Type.RETRY);
+        this.cfId = cfId;
+        this.sequenceNumber = sequenceNumber;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java b/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java
new file mode 100644
index 0000000..450f67d
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/SessionFailedMessage.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming.messages;
+
+import java.io.IOException;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+
+import org.apache.cassandra.streaming.StreamSession;
+
+public class SessionFailedMessage extends StreamMessage
+{
+    public static Serializer<SessionFailedMessage> serializer = new Serializer<SessionFailedMessage>()
+    {
+        public SessionFailedMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
+        {
+            return new SessionFailedMessage();
+        }
+
+        public void serialize(SessionFailedMessage message, WritableByteChannel out, int version, StreamSession session) throws IOException {}
+    };
+
+    public SessionFailedMessage()
+    {
+        super(Type.SESSION_FAILED);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java b/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java
new file mode 100644
index 0000000..d1e797c
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/StreamInitMessage.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming.messages;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.UUID;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.utils.UUIDSerializer;
+
+/**
+ * StreamInitMessage is first sent from the node where {@link org.apache.cassandra.streaming.StreamSession} is started,
+ * to initiate corresponding {@link org.apache.cassandra.streaming.StreamSession} on the other side.
+ */
+public class StreamInitMessage
+{
+    public static IVersionedSerializer<StreamInitMessage> serializer = new StreamInitMessageSerializer();
+
+    public final UUID planId;
+    public final String description;
+
+    public StreamInitMessage(UUID planId, String description)
+    {
+        this.planId = planId;
+        this.description = description;
+    }
+
+    /**
+     * Create serialized message.
+     *
+     * @param compress true if message is compressed
+     * @param version Streaming protocol version
+     * @return serialized message in ByteBuffer format
+     */
+    public ByteBuffer createMessage(boolean compress, int version)
+    {
+        int header = 0;
+        // set compression bit.
+        if (compress)
+            header |= 4;
+        // set streaming bit
+        header |= 8;
+        // Setting up the version bit
+        header |= (version << 8);
+
+        /* Adding the StreamHeader which contains the session Id along
+         * with the pendingfile info for the stream.
+         * | Session Id | Pending File Size | Pending File | Bool more files |
+         * | No. of Pending files | Pending Files ... |
+         */
+        byte[] bytes;
+        try
+        {
+            int size = (int)StreamInitMessage.serializer.serializedSize(this, version);
+            DataOutputBuffer buffer = new DataOutputBuffer(size);
+            StreamInitMessage.serializer.serialize(this, buffer, version);
+            bytes = buffer.getData();
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+        assert bytes.length > 0;
+
+        ByteBuffer buffer = ByteBuffer.allocate(4 + 4 + bytes.length);
+        buffer.putInt(MessagingService.PROTOCOL_MAGIC);
+        buffer.putInt(header);
+        buffer.put(bytes);
+        buffer.flip();
+        return buffer;
+    }
+
+    private static class StreamInitMessageSerializer implements IVersionedSerializer<StreamInitMessage>
+    {
+        public void serialize(StreamInitMessage message, DataOutput out, int version) throws IOException
+        {
+            UUIDSerializer.serializer.serialize(message.planId, out, MessagingService.current_version);
+            out.writeUTF(message.description);
+        }
+
+        public StreamInitMessage deserialize(DataInput in, int version) throws IOException
+        {
+            UUID planId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
+            return new StreamInitMessage(planId, in.readUTF());
+        }
+
+        public long serializedSize(StreamInitMessage message, int version)
+        {
+            long size = UUIDSerializer.serializer.serializedSize(message.planId, MessagingService.current_version);
+            size += TypeSizes.NATIVE.sizeof(message.description);
+            return size;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java
new file mode 100644
index 0000000..f737675
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming.messages;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.WritableByteChannel;
+
+import org.apache.cassandra.streaming.StreamSession;
+
+/**
+ * StreamMessage is an abstract base class that every messages in streaming protocol inherit.
+ *
+ * Every message carries message type({@link Type}) and streaming protocol version byte.
+ */
+public abstract class StreamMessage
+{
+    /** Streaming protocol version */
+    public static final int CURRENT_VERSION = 1;
+
+    public static void serialize(StreamMessage message, WritableByteChannel out, int version, StreamSession session) throws IOException
+    {
+        ByteBuffer buff = ByteBuffer.allocate(1);
+        // message type
+        buff.put(message.type.type);
+        buff.flip();
+        out.write(buff);
+        message.type.serializer.serialize(message, out, version, session);
+    }
+
+    public static StreamMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException
+    {
+        ByteBuffer buff = ByteBuffer.allocate(1);
+        in.read(buff);
+        buff.flip();
+        Type type = Type.get(buff.get());
+        return type.serializer.deserialize(in, version, session);
+    }
+
+    /** StreamMessage serializer */
+    public static interface Serializer<V extends StreamMessage>
+    {
+        V deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException;
+        void serialize(V message, WritableByteChannel out, int version, StreamSession session) throws IOException;
+    }
+
+    /** StreamMessage types */
+    public static enum Type
+    {
+        PREPARE(1, 5, PrepareMessage.serializer),
+        FILE(2, 0, FileMessage.serializer),
+        RETRY(3, 1, RetryMessage.serializer),
+        COMPLETE(4, 4, CompleteMessage.serializer),
+        SESSION_FAILED(5, 5, SessionFailedMessage.serializer);
+
+        public static Type get(byte type)
+        {
+            for (Type t : Type.values())
+            {
+                if (t.type == type)
+                    return t;
+            }
+            throw new IllegalArgumentException("Unknown type " + type);
+        }
+
+        private final byte type;
+        public final int priority;
+        public final Serializer<StreamMessage> serializer;
+
+        @SuppressWarnings("unchecked")
+        private Type(int type, int priority, Serializer serializer)
+        {
+            this.type = (byte) type;
+            this.priority = priority;
+            this.serializer = serializer;
+        }
+    }
+
+    public final Type type;
+
+    protected StreamMessage(Type type)
+    {
+        this.type = type;
+    }
+
+    /**
+     * @return priority of this message. higher value, higher priority.
+     */
+    public int getPriority()
+    {
+        return type.priority;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/tools/BulkLoader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/BulkLoader.java b/src/java/org/apache/cassandra/tools/BulkLoader.java
index d2f3c4d..12650a2 100644
--- a/src/java/org/apache/cassandra/tools/BulkLoader.java
+++ b/src/java/org/apache/cassandra/tools/BulkLoader.java
@@ -24,6 +24,11 @@ import java.util.*;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.cli.*;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.protocol.TProtocol;
+import org.apache.thrift.transport.TFramedTransport;
+import org.apache.thrift.transport.TSocket;
+import org.apache.thrift.transport.TTransport;
 
 import org.apache.cassandra.auth.IAuthenticator;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -32,17 +37,10 @@ import org.apache.cassandra.db.Table;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.SSTableLoader;
-import org.apache.cassandra.streaming.PendingFile;
+import org.apache.cassandra.streaming.*;
 import org.apache.cassandra.thrift.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.OutputHandler;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.protocol.TProtocol;
-import org.apache.thrift.transport.TFramedTransport;
-import org.apache.thrift.transport.TSocket;
-import org.apache.thrift.transport.TTransport;
-
-import com.google.common.util.concurrent.Uninterruptibles;
 
 public class BulkLoader
 {
@@ -61,52 +59,20 @@ public class BulkLoader
     public static void main(String args[])
     {
         LoaderOptions options = LoaderOptions.parseArgs(args);
+        OutputHandler handler = new OutputHandler.SystemOutput(options.verbose, options.debug);
+        SSTableLoader loader = new SSTableLoader(options.directory, new ExternalClient(options.hosts, options.rpcPort, options.user, options.passwd), handler);
+        DatabaseDescriptor.setStreamThroughputOutboundMegabitsPerSec(options.throttle);
+        StreamResultFuture future = loader.stream(options.ignores);
+        future.addEventListener(new ProgressIndicator(handler));
         try
         {
-            OutputHandler handler = new OutputHandler.SystemOutput(options.verbose, options.debug);
-            SSTableLoader loader = new SSTableLoader(options.directory, new ExternalClient(handler, options.hosts, options.rpcPort, options.user, options.passwd), handler);
-            DatabaseDescriptor.setStreamThroughputOutboundMegabitsPerSec(options.throttle);
-            SSTableLoader.LoaderFuture future = loader.stream(options.ignores);
-
-            if (options.noProgress)
-            {
-                future.get();
-            }
-            else
-            {
-                ProgressIndicator indicator = new ProgressIndicator(future.getPendingFiles());
-                indicator.start();
-                System.out.println("");
-                boolean printEnd = false;
-                while (!future.isDone())
-                {
-                    if (indicator.printProgress())
-                    {
-                        // We're done with streaming
-                        System.out.println("\nWaiting for targets to rebuild indexes ...");
-                        printEnd = true;
-                        future.get();
-                        assert future.isDone();
-                    }
-                    else
-                    {
-                        Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-                    }
-                }
-                if (!printEnd)
-                    indicator.printProgress();
-                if (future.hadFailures())
-                {
-                    System.err.println("Streaming to the following hosts failed:");
-                    System.err.println(future.getFailedHosts());
-                    System.exit(1);
-                }
-            }
-
+            future.get();
             System.exit(0); // We need that to stop non daemonized threads
         }
         catch (Exception e)
         {
+            System.err.println("Streaming to the following hosts failed:");
+            System.err.println(loader.getFailedHosts());
             System.err.println(e.getMessage());
             if (options.debug)
                 e.printStackTrace(System.err);
@@ -115,62 +81,63 @@ public class BulkLoader
     }
 
     // Return true when everything is at 100%
-    static class ProgressIndicator
+    static class ProgressIndicator implements StreamEventHandler
     {
-        private final Map<InetAddress, Collection<PendingFile>> filesByHost;
+        private final Set<ProgressInfo> progresses = new HashSet<>();
+        private final OutputHandler handler;
+
         private long start;
         private long lastProgress;
         private long lastTime;
 
-        public ProgressIndicator(Map<InetAddress, Collection<PendingFile>> filesByHost)
-        {
-            this.filesByHost = new HashMap<InetAddress, Collection<PendingFile>>(filesByHost);
-        }
-
-        public void start()
+        public ProgressIndicator(OutputHandler handler)
         {
+            this.handler = handler;
             start = lastTime = System.nanoTime();
         }
 
-        public boolean printProgress()
+        public void onSuccess(StreamState finalState) {}
+        public void onFailure(Throwable t) {}
+
+        public void handleStreamEvent(StreamEvent event)
         {
-            boolean done = true;
-            StringBuilder sb = new StringBuilder();
-            sb.append("\rprogress: ");
-            long totalProgress = 0;
-            long totalSize = 0;
-            for (Map.Entry<InetAddress, Collection<PendingFile>> entry : filesByHost.entrySet())
+            if (event.eventType == StreamEvent.Type.FILE_PROGRESS)
             {
-                long progress = 0;
-                long size = 0;
-                int completed = 0;
-                Collection<PendingFile> pendings = entry.getValue();
-                for (PendingFile f : pendings)
+                ProgressInfo progressInfo = ((StreamEvent.ProgressEvent) event).progress;
+
+                // update progress
+                if (progresses.contains(progressInfo))
+                    progresses.remove(progressInfo);
+                progresses.add(progressInfo);
+
+                StringBuilder sb = new StringBuilder();
+                sb.append("\rprogress: ");
+
+                long totalProgress = 0;
+                long totalSize = 0;
+                long completed = 0;
+                for (ProgressInfo entry : progresses)
                 {
-                    progress += f.progress;
-                    size += f.size;
-                    if (f.progress == f.size)
+                    if (entry.currentBytes == entry.totalBytes)
                         completed++;
+                    totalProgress += entry.currentBytes;
+                    totalSize += entry.totalBytes;
+                    sb.append("[").append(entry.peer);
+                    sb.append(" ").append(completed).append("/").append(progresses.size());
+                    sb.append(" (").append(entry.totalBytes == 0 ? 100L : entry.currentBytes * 100L / entry.totalBytes).append(")] ");
                 }
-                totalProgress += progress;
-                totalSize += size;
-                if (completed != pendings.size())
-                    done = false;
-                sb.append("[").append(entry.getKey());
-                sb.append(" ").append(completed).append("/").append(pendings.size());
-                sb.append(" (").append(size == 0 ? 100L : progress * 100L / size).append(")] ");
+                long time = System.nanoTime();
+                long deltaTime = TimeUnit.NANOSECONDS.toMillis(time - lastTime);
+                lastTime = time;
+                long deltaProgress = totalProgress - lastProgress;
+                lastProgress = totalProgress;
+
+                sb.append("[total: ").append(totalSize == 0 ? 100L : totalProgress * 100L / totalSize).append(" - ");
+                sb.append(mbPerSec(deltaProgress, deltaTime)).append("MB/s");
+                sb.append(" (avg: ").append(mbPerSec(totalProgress, TimeUnit.NANOSECONDS.toMillis(time - start))).append("MB/s)]");
+
+                handler.output(sb.toString());
             }
-            long time = System.nanoTime();
-            long deltaTime = TimeUnit.NANOSECONDS.toMillis(time - lastTime);
-            lastTime = time;
-            long deltaProgress = totalProgress - lastProgress;
-            lastProgress = totalProgress;
-
-            sb.append("[total: ").append(totalSize == 0 ? 100L : totalProgress * 100L / totalSize).append(" - ");
-            sb.append(mbPerSec(deltaProgress, deltaTime)).append("MB/s");
-            sb.append(" (avg: ").append(mbPerSec(totalProgress, TimeUnit.NANOSECONDS.toMillis(time - start))).append("MB/s)]");
-            System.out.print(sb.toString());
-            return done;
         }
 
         private int mbPerSec(long bytes, long timeInMs)
@@ -182,13 +149,13 @@ public class BulkLoader
 
     static class ExternalClient extends SSTableLoader.Client
     {
-        private final Set<String> knownCfs = new HashSet<String>();
+        private final Set<String> knownCfs = new HashSet<>();
         private final Set<InetAddress> hosts;
         private final int rpcPort;
         private final String user;
         private final String passwd;
 
-        public ExternalClient(OutputHandler outputHandler, Set<InetAddress> hosts, int port, String user, String passwd)
+        public ExternalClient(Set<InetAddress> hosts, int port, String user, String passwd)
         {
             super();
             this.hosts = hosts;
@@ -213,7 +180,7 @@ public class BulkLoader
 
                     for (TokenRange tr : client.describe_ring(keyspace))
                     {
-                        Range<Token> range = new Range<Token>(tkFactory.fromString(tr.start_token), tkFactory.fromString(tr.end_token));
+                        Range<Token> range = new Range<>(tkFactory.fromString(tr.start_token), tkFactory.fromString(tr.end_token));
                         for (String ep : tr.endpoints)
                         {
                             addRangeForEndpoint(range, InetAddress.getByName(ep));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/tools/NodeCmd.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeCmd.java b/src/java/org/apache/cassandra/tools/NodeCmd.java
index b0b9e23..48ae998 100644
--- a/src/java/org/apache/cassandra/tools/NodeCmd.java
+++ b/src/java/org/apache/cassandra/tools/NodeCmd.java
@@ -45,6 +45,9 @@ import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
 import org.apache.cassandra.net.MessagingServiceMBean;
 import org.apache.cassandra.service.CacheServiceMBean;
 import org.apache.cassandra.service.StorageProxyMBean;
+import org.apache.cassandra.streaming.StreamState;
+import org.apache.cassandra.streaming.ProgressInfo;
+import org.apache.cassandra.streaming.SessionInfo;
 import org.apache.cassandra.utils.EstimatedHistogram;
 import org.apache.cassandra.utils.Pair;
 
@@ -614,56 +617,34 @@ public class NodeCmd
     public void printNetworkStats(final InetAddress addr, PrintStream outs)
     {
         outs.printf("Mode: %s%n", probe.getOperationMode());
-        Set<InetAddress> hosts = addr == null ? probe.getStreamDestinations() : new HashSet<InetAddress>(){{add(addr);}};
-        if (hosts.size() == 0)
+        Set<StreamState> statuses = probe.getStreamStatus();
+        if (statuses.isEmpty())
             outs.println("Not sending any streams.");
-        for (InetAddress host : hosts)
+        for (StreamState status : statuses)
         {
-            try
+            outs.printf("%s %s%n", status.description, status.planId.toString());
+            for (SessionInfo info : status.sessions)
             {
-                List<String> files = probe.getFilesDestinedFor(host);
-                if (files.size() > 0)
+                outs.printf("    %s%n", info.peer.toString());
+                if (!info.receivingSummaries.isEmpty())
                 {
-                    outs.printf("Streaming to: %s%n", host);
-                    for (String file : files)
-                        outs.printf("   %s%n", file);
+                    outs.printf("        Receiving %d files, %d bytes total%n", info.getTotalFilesToReceive(), info.getTotalSizeToReceive());
+                    for (ProgressInfo progress : info.getReceivingFiles())
+                    {
+                        outs.printf("            %s%n", progress.toString());
+                    }
                 }
-                else
+                if (!info.sendingSummaries.isEmpty())
                 {
-                    outs.printf(" Nothing streaming to %s%n", host);
+                    outs.printf("        Sending %d files, %d bytes total%n", info.getTotalFilesToSend(), info.getTotalSizeToSend());
+                    for (ProgressInfo progress : info.getSendingFiles())
+                    {
+                        outs.printf("            %s%n", progress.toString());
+                    }
                 }
             }
-            catch (IOException ex)
-            {
-                outs.printf("   Error retrieving file data for %s%n", host);
-            }
         }
 
-        hosts = addr == null ? probe.getStreamSources() : new HashSet<InetAddress>(){{add(addr); }};
-        if (hosts.size() == 0)
-            outs.println("Not receiving any streams.");
-        for (InetAddress host : hosts)
-        {
-            try
-            {
-                List<String> files = probe.getIncomingFiles(host);
-                if (files.size() > 0)
-                {
-                    outs.printf("Streaming from: %s%n", host);
-                    for (String file : files)
-                        outs.printf("   %s%n", file);
-                }
-                else
-                {
-                    outs.printf(" Nothing streaming from %s%n", host);
-                }
-            }
-            catch (IOException ex)
-            {
-                outs.printf("   Error retrieving file data for %s%n", host);
-            }
-        }
-        
         outs.printf("Read Repair Statistics:%nAttempted: %d%nMismatch (Blocking): %d%nMismatch (Background): %d%n", probe.getReadRepairAttempted(), probe.getReadRepairRepairedBlocking(), probe.getReadRepairRepairedBackground());
 
         MessagingServiceMBean ms = probe.msProxy;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index af94d14..452e865 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -50,8 +50,8 @@ import org.apache.cassandra.locator.EndpointSnitchInfoMBean;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.net.MessagingServiceMBean;
 import org.apache.cassandra.service.*;
-import org.apache.cassandra.streaming.StreamingService;
-import org.apache.cassandra.streaming.StreamingServiceMBean;
+import org.apache.cassandra.streaming.StreamState;
+import org.apache.cassandra.streaming.StreamManagerMBean;
 import org.apache.cassandra.utils.SimpleCondition;
 
 /**
@@ -73,7 +73,7 @@ public class NodeProbe
     private StorageServiceMBean ssProxy;
     private MemoryMXBean memProxy;
     private RuntimeMXBean runtimeProxy;
-    private StreamingServiceMBean streamProxy;
+    private StreamManagerMBean streamProxy;
     public MessagingServiceMBean msProxy;
     private FailureDetectorMBean fdProxy;
     private CacheServiceMBean cacheService;
@@ -150,8 +150,8 @@ public class NodeProbe
             ssProxy = JMX.newMBeanProxy(mbeanServerConn, name, StorageServiceMBean.class);
             name = new ObjectName(MessagingService.MBEAN_NAME);
             msProxy = JMX.newMBeanProxy(mbeanServerConn, name, MessagingServiceMBean.class);
-            name = new ObjectName(StreamingService.MBEAN_OBJECT_NAME);
-            streamProxy = JMX.newMBeanProxy(mbeanServerConn, name, StreamingServiceMBean.class);
+            name = new ObjectName(StreamManagerMBean.OBJECT_NAME);
+            streamProxy = JMX.newMBeanProxy(mbeanServerConn, name, StreamManagerMBean.class);
             name = new ObjectName(CompactionManager.MBEAN_OBJECT_NAME);
             compactionProxy = JMX.newMBeanProxy(mbeanServerConn, name, CompactionManagerMBean.class);
             name = new ObjectName(FailureDetector.MBEAN_NAME);
@@ -545,24 +545,9 @@ public class NodeProbe
         return cfsProxy.getSSTablesForKey(key);
     }
 
-    public Set<InetAddress> getStreamDestinations()
+    public Set<StreamState> getStreamStatus()
     {
-        return streamProxy.getStreamDestinations();
-    }
-
-    public List<String> getFilesDestinedFor(InetAddress host) throws IOException
-    {
-        return streamProxy.getOutgoingFiles(host.getHostAddress());
-    }
-
-    public Set<InetAddress> getStreamSources()
-    {
-        return streamProxy.getStreamSources();
-    }
-
-    public List<String> getIncomingFiles(InetAddress host) throws IOException
-    {
-        return streamProxy.getIncomingFiles(host.getHostAddress());
+        return streamProxy.getCurrentStreams();
     }
 
     public String getOperationMode()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
index 7bbe72d..17fa3fd 100644
--- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
+++ b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
@@ -36,7 +36,6 @@ import org.apache.cassandra.gms.IFailureDetectionEventListener;
 import org.apache.cassandra.gms.IFailureDetector;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.streaming.OperationType;
 
 import static org.junit.Assert.*;
 
@@ -66,7 +65,7 @@ public class BootStrapperTest extends SchemaLoader
 
         TokenMetadata tmd = ss.getTokenMetadata();
         assertEquals(numOldNodes, tmd.sortedTokens().size());
-        RangeStreamer s = new RangeStreamer(tmd, myEndpoint, OperationType.BOOTSTRAP);
+        RangeStreamer s = new RangeStreamer(tmd, myEndpoint, "Bootstrap");
         IFailureDetector mockFailureDetector = new IFailureDetector()
         {
             public boolean isAlive(InetAddress ep)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/test/unit/org/apache/cassandra/service/RemoveTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/RemoveTest.java b/test/unit/org/apache/cassandra/service/RemoveTest.java
index 4f3217c..1b3eb48 100644
--- a/test/unit/org/apache/cassandra/service/RemoveTest.java
+++ b/test/unit/org/apache/cassandra/service/RemoveTest.java
@@ -27,29 +27,23 @@ import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
+import org.junit.*;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.RandomPartitioner;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.locator.TokenMetadata;
-import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.net.sink.IMessageSink;
 import org.apache.cassandra.net.sink.SinkManager;
-import org.apache.cassandra.streaming.StreamUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 public class RemoveTest
 {
@@ -119,9 +113,6 @@ public class RemoveTest
     @Test
     public void testRemoveHostId() throws InterruptedException
     {
-        ReplicationSink rSink = new ReplicationSink();
-        SinkManager.add(rSink);
-
         // start removal in background and send replication confirmations
         final AtomicBoolean success = new AtomicBoolean(false);
         Thread remover = new Thread()
@@ -159,25 +150,4 @@ public class RemoveTest
         assertTrue(success.get());
         assertTrue(tmd.getLeavingEndpoints().isEmpty());
     }
-
-    /**
-     * sink that captures STREAM_REQUEST messages and calls finishStreamRequest on it
-     */
-    class ReplicationSink implements IMessageSink
-    {
-        public MessageIn handleMessage(MessageIn msg, int id, InetAddress to)
-        {
-            if (!msg.verb.equals(MessagingService.Verb.STREAM_REQUEST))
-                return msg;
-
-            StreamUtil.finishStreamRequest(msg, to);
-
-            return null;
-        }
-
-        public MessageOut handleMessage(MessageOut msg, int id, InetAddress to)
-        {
-            return msg;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/test/unit/org/apache/cassandra/streaming/BootstrapTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/BootstrapTest.java b/test/unit/org/apache/cassandra/streaming/BootstrapTest.java
deleted file mode 100644
index 6578dde..0000000
--- a/test/unit/org/apache/cassandra/streaming/BootstrapTest.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*    http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing,
-* software distributed under the License is distributed on an
-* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-* KIND, either express or implied.  See the License for the
-* specific language governing permissions and limitations
-* under the License.
-*/
-package org.apache.cassandra.streaming;
-
-import static junit.framework.Assert.assertEquals;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.utils.Pair;
-
-import java.util.Arrays;
-
-import org.junit.Test;
-
-public class BootstrapTest extends SchemaLoader
-{
-    @Test
-    public void testGetNewNames() throws IOException
-    {
-        String ver = Descriptor.Version.current_version;
-        Descriptor desc = Descriptor.fromFilename(new File("Keyspace1", "Keyspace1-Standard1-" + ver +"-500-Data.db").toString());
-        // assert !desc.isLatestVersion; // minimum compatible version -- for now it is the latest as well
-        PendingFile inContext = new PendingFile(null, desc, "Data.db", Arrays.asList(Pair.create(0L, 1L)), OperationType.BOOTSTRAP);
-
-        PendingFile outContext = StreamIn.getContextMapping(inContext);
-        // filename and generation are expected to have changed
-        assert !inContext.getFilename().equals(outContext.getFilename());
-
-        // nothing else should
-        assertEquals(inContext.component, outContext.component);
-        assertEquals(desc.ksname, outContext.desc.ksname);
-        assertEquals(desc.cfname, outContext.desc.cfname);
-        assertEquals(desc.version, outContext.desc.version);
-    }
-}


[3/5] Streaming 2.0

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamPlan.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamPlan.java b/src/java/org/apache/cassandra/streaming/StreamPlan.java
new file mode 100644
index 0000000..78d50ad
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamPlan.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.net.InetAddress;
+import java.net.Socket;
+import java.util.*;
+
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.utils.UUIDGen;
+
+/**
+ * {@link StreamPlan} is a helper class that builds StreamOperation of given configuration.
+ *
+ * This is the class you want to use for building streaming plan and starting streaming.
+ */
+public class StreamPlan
+{
+    private final UUID planId = UUIDGen.getTimeUUID();
+    private final String description;
+
+    // sessions per InetAddress of the other end.
+    private final Map<InetAddress, StreamSession> sessions = new HashMap<>();
+
+    private boolean flushBeforeTransfer = true;
+
+    /**
+     * Start building stream plan.
+     *
+     * @param description Stream type that describes this StreamPlan
+     */
+    public StreamPlan(String description)
+    {
+        this.description = description;
+    }
+
+    /**
+     * Request data in {@code keyspace} and {@code ranges} from specific node.
+     *
+     * @param from endpoint address to fetch data from.
+     * @param keyspace name of keyspace
+     * @param ranges ranges to fetch
+     * @return this object for chaining
+     */
+    public StreamPlan requestRanges(InetAddress from, String keyspace, Collection<Range<Token>> ranges)
+    {
+        return requestRanges(from, keyspace, ranges, new String[0]);
+    }
+
+    /**
+     * Request data in {@code columnFamilies} under {@code keyspace} and {@code ranges} from specific node.
+     *
+     * @param from endpoint address to fetch data from.
+     * @param keyspace name of keyspace
+     * @param ranges ranges to fetch
+     * @param columnFamilies specific column families
+     * @return this object for chaining
+     */
+    public StreamPlan requestRanges(InetAddress from, String keyspace, Collection<Range<Token>> ranges, String... columnFamilies)
+    {
+        StreamSession session = getOrCreateSession(from);
+        session.addStreamRequest(keyspace, ranges, Arrays.asList(columnFamilies));
+        return this;
+    }
+
+    /**
+     * Add transfer task to send data of specific keyspace and ranges.
+     *
+     * @param to endpoint address of receiver
+     * @param keyspace name of keyspace
+     * @param ranges ranges to send
+     * @return this object for chaining
+     */
+    public StreamPlan transferRanges(InetAddress to, String keyspace, Collection<Range<Token>> ranges)
+    {
+        return transferRanges(to, keyspace, ranges, new String[0]);
+    }
+
+    /**
+     * Add transfer task to send data of specific {@code columnFamilies} under {@code keyspace} and {@code ranges}.
+     *
+     * @param to endpoint address of receiver
+     * @param keyspace name of keyspace
+     * @param ranges ranges to send
+     * @param columnFamilies specific column families
+     * @return this object for chaining
+     */
+    public StreamPlan transferRanges(InetAddress to, String keyspace, Collection<Range<Token>> ranges, String... columnFamilies)
+    {
+        StreamSession session = getOrCreateSession(to);
+        session.addTransferRanges(keyspace, ranges, Arrays.asList(columnFamilies), flushBeforeTransfer);
+        return this;
+    }
+
+    /**
+     * Add transfer task to send given SSTable files.
+     *
+     * @param to endpoint address of receiver
+     * @param ranges ranges to send
+     * @param sstables files to send
+     * @return this object for chaining
+     */
+    public StreamPlan transferFiles(InetAddress to, Collection<Range<Token>> ranges, Collection<SSTableReader> sstables)
+    {
+        StreamSession session = getOrCreateSession(to);
+        session.addTransferFiles(ranges, sstables);
+        return this;
+    }
+
+    /**
+     * @return true if this plan has no plan to execute
+     */
+    public boolean isEmpty()
+    {
+        return sessions.isEmpty();
+    }
+
+    /**
+     * Execute this {@link StreamPlan} asynchronously.
+     *
+     * @return Future {@link StreamState} that you can use to listen on progress of streaming.
+     */
+    public StreamResultFuture execute()
+    {
+        return StreamResultFuture.startStreamingAsync(planId, description, sessions.values());
+    }
+
+    /**
+     * Set flushBeforeTransfer option.
+     * When it's true, will flush before streaming ranges. (Default: true)
+     *
+     * @param flushBeforeTransfer set to true when the node should flush before transfer
+     * @return this object for chaining
+     */
+    public StreamPlan flushBeforeTransfer(boolean flushBeforeTransfer)
+    {
+        this.flushBeforeTransfer = flushBeforeTransfer;
+        return this;
+    }
+
+    private StreamSession getOrCreateSession(InetAddress peer)
+    {
+        StreamSession session = sessions.get(peer);
+        if (session == null)
+        {
+            session = new StreamSession(peer);
+            sessions.put(peer, session);
+        }
+        return session;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReader.java b/src/java/org/apache/cassandra/streaming/StreamReader.java
new file mode 100644
index 0000000..a01de3c
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.util.Collection;
+import java.util.UUID;
+
+import com.google.common.base.Throwables;
+import com.ning.compress.lzf.LZFInputStream;
+
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.Table;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.sstable.SSTableWriter;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.streaming.messages.FileMessageHeader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.BytesReadTracker;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * StreamReader reads from stream and writes to SSTable.
+ */
+public class StreamReader
+{
+    protected final UUID cfId;
+    protected final long estimatedKeys;
+    protected final Collection<Pair<Long, Long>> sections;
+    protected final StreamSession session;
+
+    protected Descriptor desc;
+
+    public StreamReader(FileMessageHeader header, StreamSession session)
+    {
+        this.session = session;
+        this.cfId = header.cfId;
+        this.estimatedKeys = header.estimatedKeys;
+        this.sections = header.sections;
+    }
+
+    /**
+     * @param channel where this reads data from
+     * @return SSTable transferred
+     * @throws IOException if reading the remote sstable fails. Will throw an RTE if local write fails.
+     */
+    public SSTableReader read(ReadableByteChannel channel) throws IOException
+    {
+        long totalSize = totalSize();
+
+        Pair<String, String> kscf = Schema.instance.getCF(cfId);
+        ColumnFamilyStore cfs = Table.open(kscf.left).getColumnFamilyStore(kscf.right);
+        Directories.DataDirectory localDir = cfs.directories.getLocationCapableOfSize(totalSize);
+        if (localDir == null)
+            throw new IOException("Insufficient disk space to store " + totalSize + " bytes");
+        desc = Descriptor.fromFilename(cfs.getTempSSTablePath(cfs.directories.getLocationForDisk(localDir)));
+
+        SSTableWriter writer = new SSTableWriter(desc.filenameFor(Component.DATA), estimatedKeys);
+        try
+        {
+            DataInputStream dis = new DataInputStream(new LZFInputStream(Channels.newInputStream(channel)));
+            BytesReadTracker in = new BytesReadTracker(dis);
+
+            while (in.getBytesRead() < totalSize)
+            {
+                writeRow(writer, in, cfs);
+                // TODO move this to BytesReadTracker
+                session.progress(desc, ProgressInfo.Direction.IN, in.getBytesRead(), totalSize);
+            }
+            return writer.closeAndOpenReader();
+        }
+        catch (Throwable e)
+        {
+            writer.abort();
+            if (e instanceof IOException)
+                throw (IOException) e;
+            else
+                throw Throwables.propagate(e);
+        }
+    }
+
+    protected long totalSize()
+    {
+        long size = 0;
+        for (Pair<Long, Long> section : sections)
+            size += section.right - section.left;
+        return size;
+    }
+
+    protected void writeRow(SSTableWriter writer, DataInput in, ColumnFamilyStore cfs) throws IOException
+    {
+        DecoratedKey key = StorageService.getPartitioner().decorateKey(ByteBufferUtil.readWithShortLength(in));
+        writer.appendFromStream(key, cfs.metadata, in);
+        cfs.invalidateCachedRow(key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/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
new file mode 100644
index 0000000..af8c138
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.UUID;
+
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Table;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * Task that manages receiving files for the session for certain ColumnFamily.
+ */
+public class StreamReceiveTask extends StreamTask
+{
+    // number of files to receive
+    private final int totalFiles;
+    // total size of files to receive
+    private final long totalSize;
+
+    //  holds references to SSTables received
+    protected Collection<SSTableReader> sstables;
+
+    public StreamReceiveTask(StreamSession session, UUID cfId, int totalFiles, long totalSize)
+    {
+        super(session, cfId);
+        this.totalFiles = totalFiles;
+        this.totalSize = totalSize;
+        this.sstables =  new ArrayList<>(totalFiles);
+    }
+
+    /**
+     * Process received file.
+     *
+     * @param sstable SSTable file received.
+     */
+    public void receive(SSTableReader sstable)
+    {
+        assert cfId.equals(sstable.metadata.cfId);
+
+        sstables.add(sstable);
+        if (sstables.size() == totalFiles)
+            complete();
+    }
+
+    public int getTotalNumberOfFiles()
+    {
+        return totalFiles;
+    }
+
+    public long getTotalSize()
+    {
+        return totalSize;
+    }
+
+    // TODO should be run in background so that this does not block streaming
+    private void complete()
+    {
+        if (!SSTableReader.acquireReferences(sstables))
+            throw new AssertionError("We shouldn't fail acquiring a reference on a sstable that has just been transferred");
+        try
+        {
+            Pair<String, String> kscf = Schema.instance.getCF(cfId);
+            ColumnFamilyStore cfs = Table.open(kscf.left).getColumnFamilyStore(kscf.right);
+            // add sstables and build secondary indexes
+            cfs.addSSTables(sstables);
+            cfs.indexManager.maybeBuildSecondaryIndexes(sstables, cfs.indexManager.allIndexesNames());
+        }
+        finally
+        {
+            SSTableReader.releaseReferences(sstables);
+        }
+
+        session.taskCompleted(this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamReply.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReply.java b/src/java/org/apache/cassandra/streaming/StreamReply.java
deleted file mode 100644
index d54e9cc..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamReply.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.streaming;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.UUID;
-
-import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.UUIDSerializer;
-
-public class StreamReply
-{
-    static enum Status
-    {
-        FILE_FINISHED,
-        FILE_RETRY,
-        SESSION_FINISHED,
-        SESSION_FAILURE,
-    }
-
-    public static final IVersionedSerializer<StreamReply> serializer = new FileStatusSerializer();
-
-    public final UUID sessionId;
-    public final String file;
-    public final Status action;
-
-    public StreamReply(String file, UUID sessionId, Status action)
-    {
-        this.file = file;
-        this.action = action;
-        this.sessionId = sessionId;
-    }
-
-    public MessageOut<StreamReply> createMessage()
-    {
-        return new MessageOut<StreamReply>(MessagingService.Verb.STREAM_REPLY, this, serializer);
-    }
-
-    @Override
-    public String toString()
-    {
-        return "StreamReply(" +
-               "sessionId=" + sessionId +
-               ", file='" + file + '\'' +
-               ", action=" + action +
-               ')';
-    }
-
-    private static class FileStatusSerializer implements IVersionedSerializer<StreamReply>
-    {
-        public void serialize(StreamReply reply, DataOutput out, int version) throws IOException
-        {
-            UUIDSerializer.serializer.serialize(reply.sessionId, out, MessagingService.current_version);
-            out.writeUTF(reply.file);
-            out.writeInt(reply.action.ordinal());
-        }
-
-        public StreamReply deserialize(DataInput in, int version) throws IOException
-        {
-            UUID sessionId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
-            String targetFile = in.readUTF();
-            Status action = Status.values()[in.readInt()];
-            return new StreamReply(targetFile, sessionId, action);
-        }
-
-        public long serializedSize(StreamReply reply, int version)
-        {
-            return TypeSizes.NATIVE.sizeof(reply.sessionId) + TypeSizes.NATIVE.sizeof(reply.file) + TypeSizes.NATIVE.sizeof(reply.action.ordinal());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamReplyVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReplyVerbHandler.java b/src/java/org/apache/cassandra/streaming/StreamReplyVerbHandler.java
deleted file mode 100644
index b69b6d0..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamReplyVerbHandler.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.streaming;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
-
-public class StreamReplyVerbHandler implements IVerbHandler<StreamReply>
-{
-    private static final Logger logger = LoggerFactory.getLogger(StreamReplyVerbHandler.class);
-
-    public void doVerb(MessageIn<StreamReply> message, int id)
-    {
-        StreamReply reply = message.payload;
-        logger.debug("Received StreamReply {}", reply);
-        StreamOutSession session = StreamOutSession.get(reply.sessionId);
-        if (session == null)
-        {
-            logger.debug("Received stream action " + reply.action + " for an unknown session from " + message.from);
-            return;
-        }
-
-        switch (reply.action)
-        {
-            case FILE_FINISHED:
-                logger.info("Successfully sent {} to {}", reply.file, message.from);
-                session.validateCurrentFile(reply.file);
-                session.startNext();
-                break;
-            case FILE_RETRY:
-                session.validateCurrentFile(reply.file);
-                logger.info("Need to re-stream file {} to {}", reply.file, message.from);
-                session.retry();
-                break;
-            case SESSION_FINISHED:
-                session.close(true);
-                break;
-            case SESSION_FAILURE:
-                session.close(false);
-                break;
-            default:
-                throw new RuntimeException("Cannot handle FileStatus.Action: " + reply.action);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamRequest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamRequest.java b/src/java/org/apache/cassandra/streaming/StreamRequest.java
index 465d8bf..9d3fdb2 100644
--- a/src/java/org/apache/cassandra/streaming/StreamRequest.java
+++ b/src/java/org/apache/cassandra/streaming/StreamRequest.java
@@ -20,173 +20,77 @@ package org.apache.cassandra.streaming;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.HashSet;
 import java.util.List;
-import java.util.UUID;
 
-import com.google.common.collect.Iterables;
-
-import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.db.Table;
-import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.net.CompactEndpointSerializationHelper;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.UUIDSerializer;
 
-/**
-* This class encapsulates the message that needs to be sent to nodes
-* that handoff data. The message contains information about ranges
-* that need to be transferred and the target node.
-*
-* If a file is specified, ranges and table will not. vice-versa should hold as well.
-*/
 public class StreamRequest
 {
     public static final IVersionedSerializer<StreamRequest> serializer = new StreamRequestSerializer();
 
-    protected final UUID sessionId;
-    protected final InetAddress target;
-
-    // if this is specified, ranges and table should not be.
-    protected final PendingFile file;
+    public final String keyspace;
+    public final Collection<Range<Token>> ranges;
+    public final Collection<String> columnFamilies = new HashSet<>();
 
-    // if these are specified, file should not be.
-    protected final Collection<Range<Token>> ranges;
-    protected final String table;
-    protected final Iterable<ColumnFamilyStore> columnFamilies;
-    protected final OperationType type;
-
-    StreamRequest(InetAddress target, Collection<Range<Token>> ranges, String table, Iterable<ColumnFamilyStore> columnFamilies, UUID sessionId, OperationType type)
+    public StreamRequest(String keyspace, Collection<Range<Token>> ranges, Collection<String> columnFamilies)
     {
-        this.target = target;
+        this.keyspace = keyspace;
         this.ranges = ranges;
-        this.table = table;
-        this.columnFamilies = columnFamilies;
-        this.sessionId = sessionId;
-        this.type = type;
-        file = null;
-    }
-
-    StreamRequest(InetAddress target, PendingFile file, UUID sessionId)
-    {
-        this.target = target;
-        this.file = file;
-        this.sessionId = sessionId;
-        this.type = file.type;
-        ranges = null;
-        table = null;
-        columnFamilies = null;
-    }
-
-    public MessageOut<StreamRequest> createMessage()
-    {
-        return new MessageOut<StreamRequest>(MessagingService.Verb.STREAM_REQUEST, this, serializer);
+        this.columnFamilies.addAll(columnFamilies);
     }
 
-    public String toString()
+    public static class StreamRequestSerializer implements IVersionedSerializer<StreamRequest>
     {
-        StringBuilder sb = new StringBuilder("");
-        if (file == null)
+        public void serialize(StreamRequest request, DataOutput out, int version) throws IOException
         {
-            sb.append(table);
-            sb.append("@");
-            sb.append(columnFamilies.toString());
-            sb.append("@");
-            sb.append(target);
-            sb.append("------->");
-            for (Range<Token> range : ranges)
+            out.writeUTF(request.keyspace);
+            out.writeInt(request.ranges.size());
+            for (Range<Token> range : request.ranges)
             {
-                sb.append(range);
-                sb.append(" ");
+                Token.serializer.serialize(range.left, out);
+                Token.serializer.serialize(range.right, out);
             }
-            sb.append(type);
-        }
-        else
-        {
-            sb.append(file.toString());
+            out.writeInt(request.columnFamilies.size());
+            for (String cf : request.columnFamilies)
+                out.writeUTF(cf);
         }
-        return sb.toString();
-    }
 
-    private static class StreamRequestSerializer implements IVersionedSerializer<StreamRequest>
-    {
-        public void serialize(StreamRequest srm, DataOutput out, int version) throws IOException
+        public StreamRequest deserialize(DataInput in, int version) throws IOException
         {
-            UUIDSerializer.serializer.serialize(srm.sessionId, out, MessagingService.current_version);
-            CompactEndpointSerializationHelper.serialize(srm.target, out);
-            if (srm.file != null)
+            String keyspace = in.readUTF();
+            int rangeCount = in.readInt();
+            List<Range<Token>> ranges = new ArrayList<>(rangeCount);
+            for (int i = 0; i < rangeCount; i++)
             {
-                out.writeBoolean(true);
-                PendingFile.serializer.serialize(srm.file, out, version);
-            }
-            else
-            {
-                out.writeBoolean(false);
-                out.writeUTF(srm.table);
-                out.writeInt(srm.ranges.size());
-                for (Range<Token> range : srm.ranges)
-                    AbstractBounds.serializer.serialize(range, out, version);
-
-                out.writeUTF(srm.type.name());
-
-                out.writeInt(Iterables.size(srm.columnFamilies));
-                for (ColumnFamilyStore cfs : srm.columnFamilies)
-                    ColumnFamily.serializer.serializeCfId(cfs.metadata.cfId, out, version);
+                Token left = Token.serializer.deserialize(in);
+                Token right = Token.serializer.deserialize(in);
+                ranges.add(new Range<>(left, right));
             }
+            int cfCount = in.readInt();
+            List<String> columnFamilies = new ArrayList<>(cfCount);
+            for (int i = 0; i < cfCount; i++)
+                columnFamilies.add(in.readUTF());
+            return new StreamRequest(keyspace, ranges, columnFamilies);
         }
 
-        public StreamRequest deserialize(DataInput in, int version) throws IOException
+        public long serializedSize(StreamRequest request, int version)
         {
-            UUID sessionId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
-            InetAddress target = CompactEndpointSerializationHelper.deserialize(in);
-            boolean singleFile = in.readBoolean();
-            if (singleFile)
+            int size = TypeSizes.NATIVE.sizeof(request.keyspace);
+            size += TypeSizes.NATIVE.sizeof(request.ranges.size());
+            for (Range<Token> range : request.ranges)
             {
-                PendingFile file = PendingFile.serializer.deserialize(in, version);
-                return new StreamRequest(target, file, sessionId);
+                size += Token.serializer.serializedSize(range.left, TypeSizes.NATIVE);
+                size += Token.serializer.serializedSize(range.right, TypeSizes.NATIVE);
             }
-            else
-            {
-                String table = in.readUTF();
-                int size = in.readInt();
-                List<Range<Token>> ranges = (size == 0) ? null : new ArrayList<Range<Token>>(size);
-                for (int i = 0; i < size; ++i)
-                    ranges.add((Range<Token>) AbstractBounds.serializer.deserialize(in, version).toTokenBounds());
-                OperationType type = OperationType.valueOf(in.readUTF());
-
-                List<ColumnFamilyStore> stores = new ArrayList<ColumnFamilyStore>();
-                int cfsSize = in.readInt();
-                for (int i = 0; i < cfsSize; ++i)
-                    stores.add(Table.open(table).getColumnFamilyStore(ColumnFamily.serializer.deserializeCfId(in, version)));
-
-                return new StreamRequest(target, ranges, table, stores, sessionId, type);
-            }
-        }
-
-        public long serializedSize(StreamRequest sr, int version)
-        {
-            long size = TypeSizes.NATIVE.sizeof(sr.sessionId);
-            size += CompactEndpointSerializationHelper.serializedSize(sr.target);
-            size += TypeSizes.NATIVE.sizeof(true);
-            if (sr.file != null)
-                return size + PendingFile.serializer.serializedSize(sr.file, version);
-
-            size += TypeSizes.NATIVE.sizeof(sr.table);
-            size += TypeSizes.NATIVE.sizeof(sr.ranges.size());
-            for (Range<Token> range : sr.ranges)
-                size += AbstractBounds.serializer.serializedSize(range, version);
-            size += TypeSizes.NATIVE.sizeof(sr.type.name());
-            size += TypeSizes.NATIVE.sizeof(Iterables.size(sr.columnFamilies));
-            for (ColumnFamilyStore cfs : sr.columnFamilies)
-                size += ColumnFamily.serializer.cfIdSerializedSize(cfs.metadata.cfId, TypeSizes.NATIVE, version);
+            size += TypeSizes.NATIVE.sizeof(request.columnFamilies.size());
+            for (String cf : request.columnFamilies)
+                size += TypeSizes.NATIVE.sizeof(cf);
             return size;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamRequestVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamRequestVerbHandler.java b/src/java/org/apache/cassandra/streaming/StreamRequestVerbHandler.java
deleted file mode 100644
index bbed34d..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamRequestVerbHandler.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
- package org.apache.cassandra.streaming;
-
- import org.slf4j.Logger;
- import org.slf4j.LoggerFactory;
-
- import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
-
-/**
- * This verb handler handles the StreamRequestMessage that is sent by
- * the node requesting range transfer.
-*/
-public class StreamRequestVerbHandler implements IVerbHandler<StreamRequest>
-{
-    private static final Logger logger = LoggerFactory.getLogger(StreamRequestVerbHandler.class);
-
-    public void doVerb(MessageIn<StreamRequest> message, int id)
-    {
-        if (logger.isDebugEnabled())
-            logger.debug("Received a StreamRequestMessage from {}", message.from);
-
-        StreamRequest srm = message.payload;
-        if (logger.isDebugEnabled())
-            logger.debug(srm.toString());
-
-        StreamOutSession session = StreamOutSession.create(srm.table, message.from, srm.sessionId);
-        StreamOut.transferRanges(session, srm.columnFamilies, srm.ranges, srm.type);
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamResultFuture.java b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
new file mode 100644
index 0000000..84332bd
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamResultFuture.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.net.InetAddress;
+import java.util.*;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.AbstractFuture;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.cliffc.high_scale_lib.NonBlockingHashMap;
+
+import org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor;
+import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.gms.FailureDetector;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.utils.FBUtilities;
+
+/**
+ * StreamResultFuture asynchronously returns the final {@link StreamState} of execution of {@link StreamPlan}.
+ * <p>
+ * You can attach {@link StreamEventHandler} to this object to listen on {@link StreamEvent}s to track progress of the streaming.
+ */
+public final class StreamResultFuture extends AbstractFuture<StreamState>
+{
+    // Executor that establish the streaming connection. Once we're connected to the other end, the rest of the streaming
+    // is directly handled by the ConnectionHandler incoming and outgoing threads.
+    private static final DebuggableThreadPoolExecutor streamExecutor = DebuggableThreadPoolExecutor.createWithFixedPoolSize("StreamConnectionEstablisher",
+                                                                                                                            FBUtilities.getAvailableProcessors());
+
+    public final UUID planId;
+    public final String description;
+    private final List<StreamEventHandler> eventListeners = Collections.synchronizedList(new ArrayList<StreamEventHandler>());
+    private final Set<UUID> ongoingSessions;
+    private final Map<InetAddress, SessionInfo> sessionStates = new NonBlockingHashMap<>();
+
+    /**
+     * Create new StreamResult of given {@code planId} and type.
+     *
+     * Constructor is package private. You need to use {@link StreamPlan#execute()} to get the instance.
+     *
+     * @param planId Stream plan ID
+     * @param description Stream description
+     * @param numberOfSessions number of sessions to wait for complete
+     */
+    private StreamResultFuture(UUID planId, String description, Set<UUID> sessions)
+    {
+        this.planId = planId;
+        this.description = description;
+        this.ongoingSessions = sessions;
+
+        // if there is no session to listen to, we immediately set result for returning
+        if (sessions.isEmpty())
+            set(getCurrentState());
+    }
+
+    static StreamResultFuture startStreamingAsync(UUID planId, String description, Collection<StreamSession> sessions)
+    {
+        Set<UUID> sessionsIds = new HashSet<>(sessions.size());
+        for (StreamSession session : sessions)
+            sessionsIds.add(session.id);
+
+        StreamResultFuture future = new StreamResultFuture(planId, description, sessionsIds);
+
+        StreamManager.instance.register(future);
+
+        // start sessions
+        for (StreamSession session : sessions)
+        {
+            session.register(future);
+            // register to gossiper/FD to fail on node failure
+            Gossiper.instance.register(session);
+            FailureDetector.instance.registerFailureDetectionEventListener(session);
+            streamExecutor.submit(session);
+        }
+        return future;
+    }
+
+    public void addEventListener(StreamEventHandler listener)
+    {
+        Futures.addCallback(this, listener);
+        eventListeners.add(listener);
+    }
+
+    /**
+     * @return Current snapshot of streaming progress.
+     */
+    public StreamState getCurrentState()
+    {
+        return new StreamState(planId, description, ImmutableSet.copyOf(sessionStates.values()));
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        StreamResultFuture that = (StreamResultFuture) o;
+        return planId.equals(that.planId);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return planId.hashCode();
+    }
+
+    void handleSessionPrepared(StreamSession session)
+    {
+        SessionInfo sessionInfo = session.getSessionInfo();
+        StreamEvent.SessionPreparedEvent event = new StreamEvent.SessionPreparedEvent(planId, sessionInfo);
+        sessionStates.put(sessionInfo.peer, sessionInfo);
+        fireStreamEvent(event);
+    }
+
+    void handleSessionComplete(StreamSession session)
+    {
+        Gossiper.instance.unregister(session);
+        FailureDetector.instance.unregisterFailureDetectionEventListener(session);
+
+        SessionInfo sessionInfo = session.getSessionInfo();
+        sessionStates.put(sessionInfo.peer, sessionInfo);
+        fireStreamEvent(new StreamEvent.SessionCompleteEvent(session));
+        maybeComplete(session.id);
+    }
+
+    public void handleProgress(ProgressInfo progress)
+    {
+        sessionStates.get(progress.peer).updateProgress(progress);
+        fireStreamEvent(new StreamEvent.ProgressEvent(planId, progress));
+    }
+
+    void fireStreamEvent(StreamEvent event)
+    {
+        // delegate to listener
+        for (StreamEventHandler listener : eventListeners)
+            listener.handleStreamEvent(event);
+    }
+
+    private synchronized void maybeComplete(UUID sessionId)
+    {
+        ongoingSessions.remove(sessionId);
+        if (ongoingSessions.isEmpty())
+        {
+            StreamState finalState = getCurrentState();
+            if (finalState.hasFailedSession())
+                setException(new StreamException(finalState, "Stream failed"));
+            else
+                set(finalState);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamSession.java b/src/java/org/apache/cassandra/streaming/StreamSession.java
new file mode 100644
index 0000000..7d96f43
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamSession.java
@@ -0,0 +1,591 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.util.*;
+import java.util.concurrent.Future;
+
+import com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.RowPosition;
+import org.apache.cassandra.db.Table;
+import org.apache.cassandra.dht.AbstractBounds;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.gms.*;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.metrics.StreamingMetrics;
+import org.apache.cassandra.streaming.messages.*;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.UUIDGen;
+
+/**
+ * StreamSession is the center of Cassandra Streaming API.
+ *
+ * StreamSession on the both endpoints exchange messages and files until complete.
+ *
+ * It is created through {@link StreamPlan} on the initiator node,
+ * and also is created directly from connected socket on the other end when received init message.
+ *
+ * <p>
+ * StreamSession goes through several stages:
+ * <ol>
+ *  <li>
+ *    Init
+ *    <p>StreamSession in one end send init message to the other end.</p>
+ *  </li>
+ *  <li>
+ *    Prepare
+ *    <p>StreamSession in both endpoints are created, so in this phase, they exchange
+ *    request and summary messages to prepare receiving/streaming files in next phase.</p>
+ *  </li>
+ *  <li>
+ *    Stream
+ *    <p>StreamSessions in both ends stream and receive files.</p>
+ *  </li>
+ *  <li>
+ *    Complete
+ *    <p>Session completes if both endpoints completed by exchanging complete message.</p>
+ *  </li>
+ * </ol>
+ */
+public class StreamSession implements Runnable, IEndpointStateChangeSubscriber, IFailureDetectionEventListener
+{
+    private static final Logger logger = LoggerFactory.getLogger(StreamSession.class);
+
+    public final UUID id = UUIDGen.getTimeUUID();
+    public final InetAddress peer;
+
+    // should not be null when session is started
+    private StreamResultFuture streamResult;
+
+    // stream requests to send to the peer
+    private final List<StreamRequest> requests = new ArrayList<>();
+    // streaming tasks are created and managed per ColumnFamily ID
+    private final Map<UUID, StreamTransferTask> transfers = new HashMap<>();
+    // data receivers, filled after receiving prepare message
+    private final Map<UUID, StreamReceiveTask> receivers = new HashMap<>();
+    private final StreamingMetrics metrics;
+
+    public final ConnectionHandler handler;
+
+    private int retries;
+
+    public static enum State
+    {
+        INITIALIZING,
+        PREPARING,
+        STREAMING,
+        WAIT_COMPLETE,
+        COMPLETE,
+        FAILED,
+    }
+
+    private volatile State state = State.INITIALIZING;
+
+    /**
+     * Create new streaming session with the peer.
+     *
+     * @param peer Address of streaming peer
+     */
+    public StreamSession(InetAddress peer)
+    {
+        this.peer = peer;
+        this.handler = new ConnectionHandler(this);
+        this.metrics = StreamingMetrics.get(peer);
+    }
+
+    /**
+     * Create streaming session from established connection.
+     *
+     * @param socket established connection
+     * @param protocolVersion Streaming protocol verison
+     */
+    public StreamSession(Socket socket, int protocolVersion)
+    {
+        this.peer = socket.getInetAddress();
+        this.handler = new ConnectionHandler(this, socket, protocolVersion);
+        this.metrics = StreamingMetrics.get(peer);
+    }
+
+    public UUID planId()
+    {
+        return streamResult == null ? null : streamResult.planId;
+    }
+
+    public String description()
+    {
+        return streamResult == null ? null : streamResult.description;
+    }
+
+    public static StreamSession startReceivingStreamAsync(UUID planId, String description, Socket socket, int version)
+    {
+        StreamSession session = new StreamSession(socket, version);
+        StreamResultFuture.startStreamingAsync(planId, description, Collections.singleton(session));
+        return session;
+    }
+
+    /**
+     * Bind this session to report to specific {@link StreamResultFuture}.
+     *
+     * @param streamResult result to report to
+     * @return this object for chaining
+     */
+    public StreamSession register(StreamResultFuture streamResult)
+    {
+        this.streamResult = streamResult;
+        return this;
+    }
+
+    /**
+     * Request data fetch task to this session.
+     *
+     * @param keyspace Requesting keyspace
+     * @param ranges Ranges to retrieve data
+     * @param columnFamilies ColumnFamily names. Can be empty if requesting all CF under the keyspace.
+     */
+    public void addStreamRequest(String keyspace, Collection<Range<Token>> ranges, Collection<String> columnFamilies)
+    {
+        requests.add(new StreamRequest(keyspace, ranges, columnFamilies));
+    }
+
+    /**
+     * Set up transfer for specific keyspace/ranges/CFs
+     *
+     * @param keyspace Transfer keyspace
+     * @param ranges Transfer ranges
+     * @param columnFamilies Transfer ColumnFamilies
+     */
+    public void addTransferRanges(String keyspace, Collection<Range<Token>> ranges, Collection<String> columnFamilies, boolean flushTables)
+    {
+        Collection<ColumnFamilyStore> stores = new HashSet<>();
+        // if columnfamilies are not specified, we add all cf under the keyspace
+        if (columnFamilies.isEmpty())
+        {
+            stores.addAll(Table.open(keyspace).getColumnFamilyStores());
+        }
+        else
+        {
+            for (String cf : columnFamilies)
+                stores.add(Table.open(keyspace).getColumnFamilyStore(cf));
+        }
+
+        if (flushTables)
+            flushSSTables(stores);
+
+        List<SSTableReader> sstables = Lists.newLinkedList();
+        for (ColumnFamilyStore cfStore : stores)
+        {
+            List<AbstractBounds<RowPosition>> rowBoundsList = Lists.newLinkedList();
+            for (Range<Token> range : ranges)
+                rowBoundsList.add(range.toRowBounds());
+            ColumnFamilyStore.ViewFragment view = cfStore.markReferenced(rowBoundsList);
+            sstables.addAll(view.sstables);
+        }
+        addTransferFiles(ranges, sstables);
+    }
+
+    /**
+     * Set up transfer of the specific SSTables.
+     * {@code sstables} must be marked as referenced so that not get deleted until transfer completes.
+     *
+     * @param ranges Transfer ranges
+     * @param sstables Transfer files
+     */
+    public void addTransferFiles(Collection<Range<Token>> ranges, Collection<SSTableReader> sstables)
+    {
+        for (SSTableReader sstable : sstables)
+        {
+            List<Pair<Long, Long>> sections = sstable.getPositionsForRanges(ranges);
+            if (sections.isEmpty())
+            {
+                // A reference was acquired on the sstable and we won't stream it
+                sstable.releaseReference();
+                continue;
+            }
+            long estimatedKeys = sstable.estimatedKeysForRanges(ranges);
+            UUID cfId = sstable.metadata.cfId;
+            StreamTransferTask task = transfers.get(cfId);
+            if (task == null)
+            {
+                task = new StreamTransferTask(this, cfId);
+                transfers.put(cfId, task);
+            }
+            task.addTransferFile(sstable, estimatedKeys, sections);
+        }
+    }
+
+    /**
+     * Start this stream session.
+     */
+    public void run()
+    {
+        assert streamResult != null : "No result is associated with this session";
+
+        try
+        {
+            if (handler.isConnected())
+            {
+                // if this session is created from remote...
+                handler.start();
+            }
+            else
+            {
+                if (requests.isEmpty() && transfers.isEmpty())
+                {
+                    logger.debug("Session does not have any tasks.");
+                    state(State.COMPLETE);
+                    streamResult.handleSessionComplete(this);
+                }
+                else
+                {
+                    handler.connect();
+                }
+            }
+        }
+        catch (IOException e)
+        {
+            onError(e);
+        }
+    }
+
+    /**
+     * Set current state to {@code newState}.
+     *
+     * @param newState new state to set
+     */
+    public void state(State newState)
+    {
+        state = newState;
+    }
+
+    /**
+     * @return current state
+     */
+    public State state()
+    {
+        return state;
+    }
+
+    /**
+     * Return if this session completed successfully.
+     *
+     * @return true if session completed successfully.
+     */
+    public boolean isSuccess()
+    {
+        return state == State.COMPLETE;
+    }
+
+    public void messageReceived(StreamMessage message)
+    {
+        switch (message.type)
+        {
+            case PREPARE:
+                PrepareMessage msg = (PrepareMessage) message;
+                prepare(msg.requests, msg.summaries);
+                break;
+
+            case FILE:
+                receive((FileMessage) message);
+                break;
+
+            case RETRY:
+                RetryMessage retry = (RetryMessage) message;
+                retry(retry.cfId, retry.sequenceNumber);
+                break;
+
+            case COMPLETE:
+                complete();
+                break;
+
+            case SESSION_FAILED:
+                sessionFailed();
+                break;
+        }
+    }
+
+    /**
+     * Call back for connection success.
+     *
+     * When connected, session moves to preparing phase and sends prepare message.
+     */
+    public void onConnect()
+    {
+        logger.debug("Connected. Sending prepare...");
+
+        // send prepare message
+        state(State.PREPARING);
+        PrepareMessage prepare = new PrepareMessage();
+        prepare.requests.addAll(requests);
+        for (StreamTransferTask task : transfers.values())
+            prepare.summaries.add(task.getSummary());
+        handler.sendMessage(prepare);
+
+        // if we don't need to prepare for receiving stream, start sending files immediately
+        if (requests.isEmpty())
+        {
+            logger.debug("Prepare complete. Start streaming files.");
+            startStreamingFiles();
+        }
+    }
+
+    /**
+     * Call back for handling exception during streaming.
+     *
+     * @param e thrown exception
+     */
+    public void onError(Throwable e)
+    {
+        state(State.FAILED);
+
+        logger.error("Streaming error occurred", e);
+        // send session failure message
+        handler.sendMessage(new SessionFailedMessage());
+        // fail session
+        streamResult.handleSessionComplete(this);
+    }
+
+    /**
+     * Prepare this session for sending/receiving files.
+     */
+    public void prepare(Collection<StreamRequest> requests, Collection<StreamSummary> summaries)
+    {
+        logger.debug("Start preparing this session (" + requests.size() + " requests, " + summaries.size() + " columnfamilies receiving)");
+        // prepare tasks
+        state(State.PREPARING);
+        for (StreamRequest request : requests)
+            addTransferRanges(request.keyspace, request.ranges, request.columnFamilies, true); // always flush on stream request
+        for (StreamSummary summary : summaries)
+            prepareReceiving(summary);
+
+        // send back prepare message if prepare message contains stream request
+        if (!requests.isEmpty())
+        {
+            PrepareMessage prepare = new PrepareMessage();
+            for (StreamTransferTask task : transfers.values())
+                prepare.summaries.add(task.getSummary());
+            handler.sendMessage(prepare);
+        }
+
+        // if there are files to stream
+        if (!maybeCompleted())
+        {
+            logger.debug("Prepare complete. Start streaming files.");
+            startStreamingFiles();
+        }
+    }
+
+    /**
+     * Call back after sending FileMessageHeader.
+     *
+     * @param header sent header
+     */
+    public void fileSent(FileMessageHeader header)
+    {
+        StreamingMetrics.totalOutgoingBytes.inc(header.size());
+        metrics.outgoingBytes.inc(header.size());
+        transfers.get(header.cfId).complete(header.sequenceNumber);
+    }
+
+    /**
+     * Call back after receiving FileMessageHeader.
+     *
+     * @param message received file
+     */
+    public void receive(FileMessage message)
+    {
+        StreamingMetrics.totalIncomingBytes.inc(message.header.size());
+        metrics.incomingBytes.inc(message.header.size());
+        receivers.get(message.header.cfId).receive(message.sstable);
+    }
+
+    public void progress(Descriptor desc, ProgressInfo.Direction direction, long bytes, long total)
+    {
+        ProgressInfo progress = new ProgressInfo(peer, desc.filenameFor(Component.DATA), direction, bytes, total);
+        streamResult.handleProgress(progress);
+    }
+
+    /**
+     * Call back on receiving {@code StreamMessage.Type.RETRY} message.
+     *
+     * @param cfId ColumnFamily ID
+     * @param sequenceNumber Sequence number to indicate which file to stream again
+     */
+    public void retry(UUID cfId, int sequenceNumber)
+    {
+        FileMessage message = transfers.get(cfId).createMessageForRetry(sequenceNumber);
+        handler.sendMessage(message);
+    }
+
+    /**
+     * Check if session is completed on receiving {@code StreamMessage.Type.COMPLETE} message.
+     */
+    public synchronized void complete()
+    {
+        if (state == State.WAIT_COMPLETE)
+        {
+            state(State.COMPLETE);
+            handler.close();
+            streamResult.handleSessionComplete(this);
+        }
+        else
+        {
+            state(State.WAIT_COMPLETE);
+        }
+    }
+
+    /**
+     * Call back on receiving {@code StreamMessage.Type.SESSION_FAILED} message.
+     */
+    public synchronized void sessionFailed()
+    {
+        handler.close();
+        streamResult.handleSessionComplete(this);
+    }
+
+    public void doRetry(FileMessageHeader header, Throwable e)
+    {
+        // retry
+        retries++;
+        if (retries > DatabaseDescriptor.getMaxStreamingRetries())
+            onError(new IOException("Too many retries for " + header, e));
+        else
+            handler.sendMessage(new RetryMessage(header.cfId, header.sequenceNumber));
+    }
+
+    /**
+     * @return Current snapshot of this session info.
+     */
+    public SessionInfo getSessionInfo()
+    {
+        List<StreamSummary> receivingSummaries = Lists.newArrayList();
+        for (StreamTask receiver : receivers.values())
+            receivingSummaries.add(receiver.getSummary());
+        List<StreamSummary> transferSummaries = Lists.newArrayList();
+        for (StreamTask transfer : transfers.values())
+            transferSummaries.add(transfer.getSummary());
+        return new SessionInfo(peer, receivingSummaries, transferSummaries, state);
+    }
+
+    public synchronized void taskCompleted(StreamReceiveTask completedTask)
+    {
+        receivers.remove(completedTask.cfId);
+        maybeCompleted();
+    }
+
+    public synchronized void taskCompleted(StreamTransferTask completedTask)
+    {
+        transfers.remove(completedTask.cfId);
+        maybeCompleted();
+    }
+
+    public void onJoin(InetAddress endpoint, EndpointState epState) {}
+    public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value) {}
+    public void onAlive(InetAddress endpoint, EndpointState state) {}
+    public void onDead(InetAddress endpoint, EndpointState state) {}
+
+    public void onRemove(InetAddress endpoint)
+    {
+        convict(endpoint, Double.MAX_VALUE);
+    }
+
+    public void onRestart(InetAddress endpoint, EndpointState epState)
+    {
+        convict(endpoint, Double.MAX_VALUE);
+    }
+
+    public void convict(InetAddress endpoint, double phi)
+    {
+        if (!endpoint.equals(peer))
+            return;
+
+        // We want a higher confidence in the failure detection than usual because failing a streaming wrongly has a high cost.
+        if (phi < 2 * DatabaseDescriptor.getPhiConvictThreshold())
+            return;
+
+        state(State.FAILED);
+        streamResult.handleSessionComplete(this);
+    }
+
+    private boolean maybeCompleted()
+    {
+        boolean completed = receivers.isEmpty() && transfers.isEmpty();
+        if (completed)
+        {
+            if (state == State.WAIT_COMPLETE)
+            {
+                state(State.COMPLETE);
+                handler.close();
+                streamResult.handleSessionComplete(this);
+            }
+            else
+            {
+                // notify peer that this session is completed
+                handler.sendMessage(new CompleteMessage());
+                state(State.WAIT_COMPLETE);
+            }
+        }
+        return completed;
+    }
+
+
+    /**
+     * Flushes matching column families from the given keyspace, or all columnFamilies
+     * if the cf list is empty.
+     */
+    private void flushSSTables(Iterable<ColumnFamilyStore> stores)
+    {
+        logger.info("Flushing memtables for {}...", stores);
+        List<Future<?>> flushes = new ArrayList<>();
+        for (ColumnFamilyStore cfs : stores)
+            flushes.add(cfs.forceFlush());
+        FBUtilities.waitOnFutures(flushes);
+    }
+
+    private void prepareReceiving(StreamSummary summary)
+    {
+        logger.debug("prepare receiving " + summary);
+        if (summary.files > 0)
+            receivers.put(summary.cfId, new StreamReceiveTask(this, summary.cfId, summary.files, summary.totalSize));
+    }
+
+    private void startStreamingFiles()
+    {
+        streamResult.handleSessionPrepared(this);
+
+        state(State.STREAMING);
+        for (StreamTransferTask task : transfers.values())
+        {
+            if (task.getFileMessages().size() > 0)
+                handler.sendMessages(task.getFileMessages());
+            else
+                taskCompleted(task); // there is no file to send
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamState.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamState.java b/src/java/org/apache/cassandra/streaming/StreamState.java
new file mode 100644
index 0000000..db50c2a
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamState.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.io.Serializable;
+import java.util.Set;
+import java.util.UUID;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
+/**
+ * Current snapshot of streaming progress.
+ */
+public class StreamState implements Serializable
+{
+    public final UUID planId;
+    public final String description;
+    public final Set<SessionInfo> sessions;
+
+    public StreamState(UUID planId, String description, Set<SessionInfo> sessions)
+    {
+        this.planId = planId;
+        this.description = description;
+        this.sessions = sessions;
+    }
+
+    public boolean hasFailedSession()
+    {
+        return Iterables.any(sessions, new Predicate<SessionInfo>()
+        {
+            public boolean apply(SessionInfo session)
+            {
+                return session.isFailed();
+            }
+        });
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamSummary.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamSummary.java b/src/java/org/apache/cassandra/streaming/StreamSummary.java
new file mode 100644
index 0000000..a31e333
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamSummary.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.UUID;
+
+import com.google.common.base.Objects;
+
+import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.utils.UUIDSerializer;
+
+/**
+ * Summary of streaming.
+ */
+public class StreamSummary implements Serializable
+{
+    public static final IVersionedSerializer<StreamSummary> serializer = new StreamSummarySerializer();
+
+    public final UUID cfId;
+
+    /**
+     * Number of files to transfer. Can be 0 if nothing to transfer for some streaming request.
+     */
+    public final int files;
+    public final long totalSize;
+
+    public StreamSummary(UUID cfId, int files, long totalSize)
+    {
+        this.cfId = cfId;
+        this.files = files;
+        this.totalSize = totalSize;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        StreamSummary summary = (StreamSummary) o;
+        return files == summary.files && totalSize == summary.totalSize && cfId.equals(summary.cfId);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hashCode(cfId, files, totalSize);
+    }
+
+    @Override
+    public String toString()
+    {
+        final StringBuilder sb = new StringBuilder("StreamSummary{");
+        sb.append("path=").append(cfId);
+        sb.append(", files=").append(files);
+        sb.append(", totalSize=").append(totalSize);
+        sb.append('}');
+        return sb.toString();
+    }
+
+    public static class StreamSummarySerializer implements IVersionedSerializer<StreamSummary>
+    {
+        // arbitrary version is fine for UUIDSerializer for now...
+        public void serialize(StreamSummary summary, DataOutput out, int version) throws IOException
+        {
+            UUIDSerializer.serializer.serialize(summary.cfId, out, MessagingService.current_version);
+            out.writeInt(summary.files);
+            out.writeLong(summary.totalSize);
+        }
+
+        public StreamSummary deserialize(DataInput in, int version) throws IOException
+        {
+            UUID cfId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
+            int files = in.readInt();
+            long totalSize = in.readLong();
+            return new StreamSummary(cfId, files, totalSize);
+        }
+
+        public long serializedSize(StreamSummary summary, int version)
+        {
+            long size = UUIDSerializer.serializer.serializedSize(summary.cfId, MessagingService.current_version);
+            size += TypeSizes.NATIVE.sizeof(summary.files);
+            size += TypeSizes.NATIVE.sizeof(summary.totalSize);
+            return size;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamTask.java b/src/java/org/apache/cassandra/streaming/StreamTask.java
new file mode 100644
index 0000000..9e9e06f
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamTask.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.util.UUID;
+
+/**
+ * StreamTask is an abstraction of the streaming task performed over specific ColumnFamily.
+ */
+public abstract class StreamTask
+{
+    /** StreamSession that this task belongs */
+    protected final StreamSession session;
+
+    protected final UUID cfId;
+
+    protected StreamTask(StreamSession session, UUID cfId)
+    {
+        this.session = session;
+        this.cfId = cfId;
+    }
+
+    /**
+     * @return total number of files this task receives/streams.
+     */
+    public abstract int getTotalNumberOfFiles();
+
+    /**
+     * @return total bytes expected to receive
+     */
+    public abstract long getTotalSize();
+
+    /**
+     * @return StreamSummary that describes this task
+     */
+    public StreamSummary getSummary()
+    {
+        return new StreamSummary(cfId, getTotalNumberOfFiles(), getTotalSize());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamTransferTask.java b/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
new file mode 100644
index 0000000..ba2df03
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamTransferTask.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.util.*;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.streaming.messages.FileMessage;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * StreamTransferTask sends sections of SSTable files in certain ColumnFamily.
+ */
+public class StreamTransferTask extends StreamTask
+{
+    private final AtomicInteger sequenceNumber = new AtomicInteger(0);
+
+    private final Map<Integer, FileMessage> files = new HashMap<>();
+
+    private long totalSize;
+
+    public StreamTransferTask(StreamSession session, UUID cfId)
+    {
+        super(session, cfId);
+    }
+
+    public void addTransferFile(SSTableReader sstable, long estimatedKeys, List<Pair<Long, Long>> sections)
+    {
+        assert sstable != null && cfId.equals(sstable.metadata.cfId);
+        FileMessage message = new FileMessage(sstable, sequenceNumber.getAndIncrement(), estimatedKeys, sections);
+        files.put(message.header.sequenceNumber, message);
+        totalSize += message.header.size();
+    }
+
+    /**
+     * Complete sending file.
+     *
+     * @param sequenceNumber sequence number of completed file transfer
+     */
+    public void complete(int sequenceNumber)
+    {
+        files.remove(sequenceNumber);
+        // all file sent, notify session this task is complete.
+        if (files.isEmpty())
+            session.taskCompleted(this);
+    }
+
+    public int getTotalNumberOfFiles()
+    {
+        return files.size();
+    }
+
+    public long getTotalSize()
+    {
+        return totalSize;
+    }
+
+    public Collection<FileMessage> getFileMessages()
+    {
+        return files.values();
+    }
+
+    public FileMessage createMessageForRetry(int sequenceNumber)
+    {
+        return files.get(sequenceNumber);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamWriter.java b/src/java/org/apache/cassandra/streaming/StreamWriter.java
new file mode 100644
index 0000000..04301ba
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamWriter.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+
+import com.google.common.util.concurrent.RateLimiter;
+import com.ning.compress.lzf.LZFOutputStream;
+
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.io.util.DataIntegrityMetadata;
+import org.apache.cassandra.io.util.DataIntegrityMetadata.ChecksumValidator;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.io.util.RandomAccessReader;
+import org.apache.cassandra.utils.Pair;
+
+/**
+ * StreamWriter writes given section of the SSTable to given channel.
+ */
+public class StreamWriter
+{
+    private static final int DEFAULT_CHUNK_SIZE = 64 * 1024;
+
+    protected final SSTableReader sstable;
+    protected final Collection<Pair<Long, Long>> sections;
+    protected final RateLimiter limiter = StreamManager.getRateLimiter();
+    protected final StreamSession session;
+
+    private OutputStream compressedOutput;
+
+    // allocate buffer to use for transfers only once
+    private byte[] transferBuffer;
+
+    public StreamWriter(SSTableReader sstable, Collection<Pair<Long, Long>> sections, StreamSession session)
+    {
+        this.session = session;
+        this.sstable = sstable;
+        this.sections = sections;
+    }
+
+    /**
+     * Stream file of specified sections to given channel.
+     *
+     * StreamWriter uses LZF compression on wire to decrease size to transfer.
+     *
+     * @param channel where this writes data to
+     * @throws IOException on any I/O error
+     */
+    public void write(WritableByteChannel channel) throws IOException
+    {
+        long totalSize = totalSize();
+        RandomAccessReader file = sstable.openDataReader();
+        ChecksumValidator validator = null;
+        if (new File(sstable.descriptor.filenameFor(Component.CRC)).exists())
+            validator = DataIntegrityMetadata.checksumValidator(sstable.descriptor);
+
+        transferBuffer = validator == null ? new byte[DEFAULT_CHUNK_SIZE] : new byte[validator.chunkSize];
+
+        // setting up data compression stream
+        compressedOutput = new LZFOutputStream(Channels.newOutputStream(channel));
+        long progress = 0L;
+
+        try
+        {
+            // stream each of the required sections of the file
+            for (Pair<Long, Long> section : sections)
+            {
+                long start = validator == null ? section.left : validator.chunkStart(section.left);
+                int skipBytes = (int) (section.left - start);
+                // seek to the beginning of the section
+                file.seek(start);
+                if (validator != null)
+                    validator.seek(start);
+
+                // length of the section to read
+                long length = section.right - start;
+                // tracks write progress
+                long bytesTransferred = 0;
+                while (bytesTransferred < length)
+                {
+                    long lastWrite = write(file, validator, skipBytes, length, bytesTransferred);
+                    bytesTransferred += lastWrite;
+                    progress += lastWrite;
+                    session.progress(sstable.descriptor, ProgressInfo.Direction.OUT, progress, totalSize);
+                    skipBytes = 0;
+                }
+
+                // make sure that current section is send
+                compressedOutput.flush();
+            }
+        }
+        finally
+        {
+            // no matter what happens close file
+            FileUtils.closeQuietly(file);
+        }
+
+        // release reference only when completed successfully
+        sstable.releaseReference();
+    }
+
+    protected long totalSize()
+    {
+        long size = 0;
+        for (Pair<Long, Long> section : sections)
+            size += section.right - section.left;
+        return size;
+    }
+
+    /**
+     * Sequentially read bytes from the file and write them to the output stream
+     *
+     * @param reader The file reader to read from
+     * @param validator validator to verify data integrity
+     * @param start number of bytes to skip transfer, but include for validation.
+     * @param length The full length that should be transferred
+     * @param bytesTransferred Number of bytes remaining to transfer
+     *
+     * @return Number of bytes transferred
+     *
+     * @throws java.io.IOException on any I/O error
+     */
+    protected long write(RandomAccessReader reader, ChecksumValidator validator, int start, long length, long bytesTransferred) throws IOException
+    {
+        int toTransfer = (int) Math.min(transferBuffer.length, length - bytesTransferred);
+        int minReadable = (int) Math.min(transferBuffer.length, reader.length() - reader.getFilePointer());
+
+        reader.readFully(transferBuffer, 0, minReadable);
+        if (validator != null)
+            validator.validate(transferBuffer, 0, minReadable);
+
+        limiter.acquire(toTransfer);
+        compressedOutput.write(transferBuffer, start, (toTransfer - start));
+
+        return toTransfer;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java b/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
index 8c2a7bc..3730b0e 100644
--- a/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamingRepairTask.java
@@ -23,33 +23,24 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicInteger;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Table;
 import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.net.CompactEndpointSerializationHelper;
-import org.apache.cassandra.net.IVerbHandler;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.net.*;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.UUIDGen;
 import org.apache.cassandra.utils.UUIDSerializer;
 
-
 /**
  * Task that make two nodes exchange (stream) some ranges (for a given table/cf).
  * This handle the case where the local node is neither of the two nodes that
@@ -72,9 +63,9 @@ public class StreamingRepairTask implements Runnable
     private final String tableName;
     private final String cfName;
     private final Collection<Range<Token>> ranges;
-    private final IStreamCallback callback;
+    private final StreamEventHandler callback;
 
-    private StreamingRepairTask(UUID id, InetAddress owner, InetAddress src, InetAddress dst, String tableName, String cfName, Collection<Range<Token>> ranges, IStreamCallback callback)
+    private StreamingRepairTask(UUID id, InetAddress owner, InetAddress src, InetAddress dst, String tableName, String cfName, Collection<Range<Token>> ranges, StreamEventHandler callback)
     {
         this.id = id;
         this.owner = owner;
@@ -121,21 +112,15 @@ public class StreamingRepairTask implements Runnable
 
     private void initiateStreaming()
     {
-        ColumnFamilyStore cfstore = Table.open(tableName).getColumnFamilyStore(cfName);
-        try
-        {
-            logger.info(String.format("[streaming task #%s] Performing streaming repair of %d ranges with %s", id, ranges.size(), dst));
-            Collection<ColumnFamilyStore> cfses = Collections.singleton(cfstore);
-            // send ranges to the remote node
-            StreamOutSession outsession = StreamOutSession.create(tableName, dst, callback);
-            StreamOut.transferRanges(outsession, cfses, ranges, OperationType.AES, false);
-            // request ranges from the remote node
-            StreamIn.requestRanges(dst, tableName, cfses, ranges, callback, OperationType.AES);
-        }
-        catch(Exception e)
-        {
-            throw new RuntimeException("Streaming repair failed", e);
-        }
+        logger.info(String.format("[streaming task #%s] Performing streaming repair of %d ranges with %s", id, ranges.size(), dst));
+        StreamResultFuture op = new StreamPlan("Repair")
+                                    .flushBeforeTransfer(true)
+                                    // request ranges from the remote node
+                                    .requestRanges(dst, tableName, ranges, cfName)
+                                    // send ranges to the remote node
+                                    .transferRanges(dst, tableName, ranges, cfName)
+                                    .execute();
+        op.addEventListener(callback);
     }
 
     private void forwardToSource()
@@ -147,45 +132,34 @@ public class StreamingRepairTask implements Runnable
         MessagingService.instance().sendOneWay(msg, src);
     }
 
-    private static IStreamCallback makeReplyingCallback(final InetAddress taskOwner, final UUID taskId)
+    private static StreamEventHandler makeReplyingCallback(final InetAddress taskOwner, final UUID taskId)
     {
-        return new IStreamCallback()
+        return new StreamEventHandler()
         {
-            // we expect one callback for the receive, and one for the send
-            private final AtomicInteger outstanding = new AtomicInteger(2);
-
-            public void onSuccess()
+            public void onSuccess(StreamState finalState)
             {
-                if (outstanding.decrementAndGet() > 0)
-                    return; // waiting on more calls
-
                 StreamingRepairResponse.reply(taskOwner, taskId);
             }
 
-            public void onFailure() {}
+            public void onFailure(Throwable t) {}
+            public void handleStreamEvent(StreamEvent event) {}
         };
     }
 
     // wrap a given callback so as to unregister the streaming repair task on completion
-    private static IStreamCallback wrapCallback(final Runnable callback, final UUID taskid, final boolean isLocalTask)
+    private static StreamEventHandler wrapCallback(final Runnable callback, final UUID taskid, final boolean isLocalTask)
     {
-        return new IStreamCallback()
+        return new StreamEventHandler()
         {
-            // we expect one callback for the receive, and one for the send
-            private final AtomicInteger outstanding = new AtomicInteger(isLocalTask ? 2 : 1);
-
-            public void onSuccess()
+            public void onSuccess(StreamState finalState)
             {
-                if (outstanding.decrementAndGet() > 0)
-                    // waiting on more calls
-                    return;
-
                 tasks.remove(taskid);
                 if (callback != null)
                     callback.run();
             }
 
-            public void onFailure() {}
+            public void onFailure(Throwable t) {}
+            public void handleStreamEvent(StreamEvent event) {}
         };
     }
 
@@ -220,7 +194,10 @@ public class StreamingRepairTask implements Runnable
 
             logger.info(String.format("[streaming task #%s] task succeeded", task.id));
             if (task.callback != null)
-                task.callback.onSuccess();
+            {
+                // TODO null
+                task.callback.onSuccess(null);
+            }
         }
 
         private static void reply(InetAddress remote, UUID taskid)


[4/5] Streaming 2.0

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/FileStreamTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/FileStreamTask.java b/src/java/org/apache/cassandra/streaming/FileStreamTask.java
deleted file mode 100644
index 04890ba..0000000
--- a/src/java/org/apache/cassandra/streaming/FileStreamTask.java
+++ /dev/null
@@ -1,291 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.streaming;
-
-import java.io.*;
-import java.net.InetAddress;
-import java.net.Socket;
-import java.nio.ByteBuffer;
-import java.util.concurrent.TimeUnit;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.util.concurrent.Uninterruptibles;
-import com.ning.compress.lzf.LZFOutputStream;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.io.sstable.Component;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.util.DataIntegrityMetadata;
-import org.apache.cassandra.io.util.DataIntegrityMetadata.ChecksumValidator;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.metrics.StreamingMetrics;
-import org.apache.cassandra.net.MessageIn;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.Throttle;
-import org.apache.cassandra.utils.WrappedRunnable;
-
-public class FileStreamTask extends WrappedRunnable
-{
-    private static final Logger logger = LoggerFactory.getLogger(FileStreamTask.class);
-
-    private static final int DEFAULT_CHUNK_SIZE = 64 * 1024;
-    public static final int MAX_CONNECT_ATTEMPTS = 4;
-
-    protected final StreamHeader header;
-    protected final InetAddress to;
-
-    // communication socket
-    protected Socket socket;
-    // socket's output/input stream
-    private OutputStream output;
-    private OutputStream compressedoutput;
-    private DataInputStream input;
-    // allocate buffer to use for transfers only once
-    private byte[] transferBuffer;
-    // outbound global throughput limiter
-    protected final Throttle throttle;
-    private final StreamReplyVerbHandler handler = new StreamReplyVerbHandler();
-    protected final StreamingMetrics metrics;
-
-    public FileStreamTask(StreamHeader header, InetAddress to)
-    {
-        this.header = header;
-        this.to = to;
-        this.throttle = new Throttle(toString(), new Throttle.ThroughputFunction()
-        {
-            /** @return Instantaneous throughput target in bytes per millisecond. */
-            public int targetThroughput()
-            {
-                if (DatabaseDescriptor.getStreamThroughputOutboundMegabitsPerSec() < 1)
-                    // throttling disabled
-                    return 0;
-                // total throughput
-                int totalBytesPerMS = DatabaseDescriptor.getStreamThroughputOutboundMegabitsPerSec() * 1024 * 1024 / 8 / 1000;
-                // per stream throughput (target bytes per MS)
-                return totalBytesPerMS / Math.max(1, (int)StreamingMetrics.activeStreamsOutbound.count());
-            }
-        });
-        metrics = StreamingMetrics.get(to);
-    }
-
-    public void runMayThrow() throws IOException
-    {
-        try
-        {
-            connectAttempt();
-            // successfully connected: stream.
-            // (at this point, if we fail, it is the receiver's job to re-request)
-            stream();
-
-            StreamOutSession session = StreamOutSession.get(header.sessionId);
-            if (session == null)
-            {
-                logger.info("Found no stream out session at end of file stream task - this is expected if the receiver went down");
-            }
-            else if (session.getFiles().size() == 0)
-            {
-                // we are the last of our kind, receive the final confirmation before closing
-                receiveReply();
-                logger.info("Finished streaming session to {}", to);
-            }
-        }
-        catch (IOException e)
-        {
-            StreamOutSession session = StreamOutSession.get(header.sessionId);
-            if (session != null)
-                session.close(false);
-            throw e;
-        }
-        finally
-        {
-            try
-            {
-                close();
-            }
-            catch (IOException e)
-            {
-                if (logger.isDebugEnabled())
-                    logger.debug("error closing socket", e);
-            }
-        }
-        if (logger.isDebugEnabled())
-            logger.debug("Done streaming " + header.file);
-    }
-
-    /**
-     * Stream file by it's sections specified by this.header
-     * @throws IOException on any I/O error
-     */
-    protected void stream() throws IOException
-    {
-        ByteBuffer headerBuffer = MessagingService.instance().constructStreamHeader(header, false, MessagingService.instance().getVersion(to));
-        // write header (this should not be compressed for compatibility with other messages)
-        output.write(ByteBufferUtil.getArray(headerBuffer));
-
-        if (header.file == null)
-            return;
-
-        // try to skip kernel page cache if possible
-        RandomAccessReader file = RandomAccessReader.open(new File(header.file.getFilename()));
-        Descriptor desc = Descriptor.fromFilename(header.file.getFilename());
-        ChecksumValidator metadata = null;
-        if (new File(desc.filenameFor(Component.CRC)).exists())
-            metadata = DataIntegrityMetadata.checksumValidator(desc);
-        transferBuffer = metadata == null ? new byte[DEFAULT_CHUNK_SIZE] : new byte[metadata.chunkSize];
-
-        // setting up data compression stream
-        compressedoutput = new LZFOutputStream(output);
-
-        StreamingMetrics.activeStreamsOutbound.inc();
-        try
-        {
-            long totalBytesTransferred = 0;
-            // stream each of the required sections of the file
-            for (Pair<Long, Long> section : header.file.sections)
-            {
-                long start = metadata == null ? section.left : metadata.chunkStart(section.left);
-                int skipBytes = (int) (section.left - start);
-                // seek to the beginning of the section
-                file.seek(start);
-                if (metadata != null)
-                    metadata.seek(start);
-
-                // length of the section to read
-                long length = section.right - start;
-                // tracks write progress
-                long bytesTransferred = 0;
-
-                while (bytesTransferred < length)
-                {
-                    long lastWrite = write(file, metadata, skipBytes, length, bytesTransferred);
-                    bytesTransferred += lastWrite;
-                    totalBytesTransferred += lastWrite;
-                    // store streaming progress
-                    header.file.progress += lastWrite;
-                    skipBytes = 0;
-                }
-
-                // make sure that current section is send
-                compressedoutput.flush();
-
-                if (logger.isDebugEnabled())
-                    logger.debug("Bytes transferred " + bytesTransferred + "/" + header.file.size);
-            }
-            StreamingMetrics.totalOutgoingBytes.inc(totalBytesTransferred);
-            metrics.outgoingBytes.inc(totalBytesTransferred);
-            // receive reply confirmation
-            receiveReply();
-        }
-        finally
-        {
-            StreamingMetrics.activeStreamsOutbound.dec();
-
-            // no matter what happens close file
-            FileUtils.closeQuietly(file);
-        }
-    }
-
-    public static void sendReply(MessageOut message, DataOutputStream out) throws IOException
-    {
-        out.writeInt(MessagingService.PROTOCOL_MAGIC);
-        message.serialize(out, MessagingService.current_version);
-    }
-
-    protected void receiveReply() throws IOException
-    {
-        MessagingService.validateMagic(input.readInt());
-        // since we reject streaming with different version, using current_version here is fine
-        MessageIn message = MessageIn.read(input, MessagingService.current_version, -1);
-        assert message.verb == MessagingService.Verb.STREAM_REPLY : "Non-reply message received on stream socket";
-        handler.doVerb(message, -1);
-    }
-
-    /**
-     * Sequentially read bytes from the file and write them to the output stream
-     *
-     * @param reader The file reader to read from
-     * @param validator validator to verify data integrity
-     * @param start number of bytes to skip transfer, but include for validation.
-     * @param length The full length that should be transferred
-     * @param bytesTransferred Number of bytes remaining to transfer
-     *
-     * @return Number of bytes transferred
-     *
-     * @throws IOException on any I/O error
-     */
-    protected long write(RandomAccessReader reader, ChecksumValidator validator, int start, long length, long bytesTransferred) throws IOException
-    {
-        int toTransfer = (int) Math.min(transferBuffer.length, length - bytesTransferred);
-        int minReadable = (int) Math.min(transferBuffer.length, reader.length() - reader.getFilePointer());
-
-        reader.readFully(transferBuffer, 0, minReadable);
-        if (validator != null)
-            validator.validate(transferBuffer, 0, minReadable);
-
-        compressedoutput.write(transferBuffer, start, (toTransfer - start));
-        throttle.throttleDelta(toTransfer);
-
-        return toTransfer;
-    }
-
-    /**
-     * Connects to the destination, with backoff for failed attempts.
-     * TODO: all nodes on a cluster must currently use the same storage port
-     * @throws IOException If all attempts fail.
-     */
-    private void connectAttempt() throws IOException
-    {
-        int attempts = 0;
-        while (true)
-        {
-            try
-            {
-                socket = MessagingService.instance().getConnectionPool(to).newSocket();
-                socket.setSoTimeout(DatabaseDescriptor.getStreamingSocketTimeout());
-                output = socket.getOutputStream();
-                input = new DataInputStream(socket.getInputStream());
-                break;
-            }
-            catch (IOException e)
-            {
-                if (++attempts >= MAX_CONNECT_ATTEMPTS)
-                    throw e;
-
-                long waitms = DatabaseDescriptor.getRpcTimeout() * (long)Math.pow(2, attempts);
-                logger.warn("Failed attempt " + attempts + " to connect to " + to + " to stream " + header.file + ". Retrying in " + waitms + " ms. (" + e + ")");
-                Uninterruptibles.sleepUninterruptibly(waitms, TimeUnit.MILLISECONDS);
-            }
-        }
-    }
-
-    protected void close() throws IOException
-    {
-        if (output != null)
-            output.close();
-    }
-
-    public String toString()
-    {
-        return String.format("FileStreamTask(session=%s, to=%s)", header.sessionId, to);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/IStreamCallback.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/IStreamCallback.java b/src/java/org/apache/cassandra/streaming/IStreamCallback.java
deleted file mode 100644
index f0d7754..0000000
--- a/src/java/org/apache/cassandra/streaming/IStreamCallback.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.cassandra.streaming;
-
-/**
- * Callback interface for streaming session success/failure.
- */
-public interface IStreamCallback
-{
-    /**
-     * called when stream session is finished successfully.
-     */
-    public void onSuccess();
-
-    /**
-     * called when streaming somehow got in trouble.
-     */
-    public void onFailure();
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java b/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java
deleted file mode 100644
index 92f5c7f..0000000
--- a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java
+++ /dev/null
@@ -1,200 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.streaming;
-
-import java.io.*;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.util.Collections;
-
-import com.google.common.base.Throwables;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.ning.compress.lzf.LZFInputStream;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.Table;
-import org.apache.cassandra.db.compaction.CompactionController;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.io.sstable.SSTableWriter;
-import org.apache.cassandra.io.util.FileUtils;
-import org.apache.cassandra.metrics.StreamingMetrics;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.streaming.compress.CompressedInputStream;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.BytesReadTracker;
-import org.apache.cassandra.utils.Pair;
-
-public class IncomingStreamReader
-{
-    private static final Logger logger = LoggerFactory.getLogger(IncomingStreamReader.class);
-
-    protected final PendingFile localFile;
-    protected final PendingFile remoteFile;
-    protected final StreamInSession session;
-    private final InputStream underliningStream;
-    private final StreamingMetrics metrics;
-
-    public IncomingStreamReader(StreamHeader header, Socket socket) throws IOException
-    {
-        socket.setSoTimeout(DatabaseDescriptor.getStreamingSocketTimeout());
-        InetAddress host = ((InetSocketAddress)socket.getRemoteSocketAddress()).getAddress();
-        if (header.pendingFiles.isEmpty() && header.file != null)
-        {
-            // StreamInSession should be created already when receiving 2nd and after files
-            if (!StreamInSession.hasSession(header.sessionId))
-            {
-                StreamReply reply = new StreamReply("", header.sessionId, StreamReply.Status.SESSION_FAILURE);
-                FileStreamTask.sendReply(reply.createMessage(), new DataOutputStream(socket.getOutputStream()));
-                throw new IOException("Session " + header.sessionId + " already closed.");
-            }
-        }
-        session = StreamInSession.get(host, header.sessionId);
-        session.setSocket(socket);
-
-        session.addFiles(header.pendingFiles);
-        // set the current file we are streaming so progress shows up in jmx
-        session.setCurrentFile(header.file);
-        session.setTable(header.table);
-        // pendingFile gets the new context for the local node.
-        remoteFile = header.file;
-        localFile = remoteFile != null ? StreamIn.getContextMapping(remoteFile) : null;
-
-        if (remoteFile != null)
-        {
-            if (remoteFile.compressionInfo == null)
-                underliningStream = new LZFInputStream(socket.getInputStream());
-            else
-                underliningStream = new CompressedInputStream(socket.getInputStream(), remoteFile.compressionInfo);
-        }
-        else
-        {
-            underliningStream = null;
-        }
-        metrics = StreamingMetrics.get(host);
-    }
-
-    /**
-     * @throws IOException if reading the remote sstable fails. Will throw an RTE if local write fails.
-     */
-    public void read() throws IOException
-    {
-        if (remoteFile != null)
-        {
-            if (logger.isDebugEnabled())
-            {
-                logger.debug("Receiving stream");
-                logger.debug("Creating file for {} with {} estimated keys",
-                             localFile.getFilename(),
-                             remoteFile.estimatedKeys);
-            }
-
-            assert remoteFile.estimatedKeys > 0;
-            DataInput in = new DataInputStream(underliningStream);
-            try
-            {
-                SSTableReader reader = streamIn(in, localFile, remoteFile);
-                session.finished(remoteFile, reader);
-            }
-            catch (IOException ex)
-            {
-                retry();
-                throw ex;
-            }
-            catch (RuntimeException e)
-            {
-                // if we encountered unexpected exception, fail this session
-                session.close(false);
-                throw e;
-            }
-        }
-
-        session.closeIfFinished();
-    }
-
-    /**
-     * @throws IOException if reading the remote sstable fails. Will throw an RTE if local write fails.
-     */
-    private SSTableReader streamIn(DataInput input, PendingFile localFile, PendingFile remoteFile) throws IOException
-    {
-        ColumnFamilyStore cfs = Table.open(localFile.desc.ksname).getColumnFamilyStore(localFile.desc.cfname);
-        DecoratedKey key;
-        SSTableWriter writer = new SSTableWriter(localFile.getFilename(), remoteFile.estimatedKeys);
-        CompactionController controller = new CompactionController(cfs, Collections.<SSTableReader>emptySet(), Integer.MIN_VALUE);
-
-        try
-        {
-            BytesReadTracker in = new BytesReadTracker(input);
-            long totalBytesRead = 0;
-
-            for (Pair<Long, Long> section : localFile.sections)
-            {
-                long length = section.right - section.left;
-                // skip to beginning of section inside chunk
-                if (remoteFile.compressionInfo != null)
-                    ((CompressedInputStream) underliningStream).position(section.left);
-                long bytesRead = 0;
-                while (bytesRead < length)
-                {
-                    in.reset(0);
-
-                    key = StorageService.getPartitioner().decorateKey(ByteBufferUtil.readWithShortLength(in));
-                    writer.appendFromStream(key, cfs.metadata, in);
-
-                    cfs.invalidateCachedRow(key);
-
-                    bytesRead += in.getBytesRead();
-                    // when compressed, report total bytes of compressed chunks read since remoteFile.size is the sum of chunks transferred
-                    if (remoteFile.compressionInfo != null)
-                        remoteFile.progress = ((CompressedInputStream) underliningStream).getTotalCompressedBytesRead();
-                    else
-                        remoteFile.progress += in.getBytesRead();
-                    totalBytesRead += in.getBytesRead();
-                }
-            }
-            StreamingMetrics.totalIncomingBytes.inc(totalBytesRead);
-            metrics.incomingBytes.inc(totalBytesRead);
-            return writer.closeAndOpenReader();
-        }
-        catch (Throwable e)
-        {
-            writer.abort();
-            if (e instanceof IOException)
-                throw (IOException) e;
-            else
-                throw Throwables.propagate(e);
-        }
-        finally
-        {
-            controller.close();
-        }
-    }
-
-    private void retry()
-    {
-        /* Ask the source node to re-stream this file. */
-        session.retry(remoteFile);
-
-        /* Delete the orphaned file. */
-        if (new File(localFile.getFilename()).isFile())
-            FileUtils.deleteWithConfirm(new File(localFile.getFilename()));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/OperationType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/OperationType.java b/src/java/org/apache/cassandra/streaming/OperationType.java
deleted file mode 100644
index da6e2f6..0000000
--- a/src/java/org/apache/cassandra/streaming/OperationType.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.streaming;
-
-/**
- * Streaming operation type.
- */
-public enum OperationType
-{
-    AES,
-    BOOTSTRAP,
-    UNBOOTSTRAP,
-    RESTORE_REPLICA_COUNT,
-    BULK_LOAD,
-    REBUILD
-}
-

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/PendingFile.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/PendingFile.java b/src/java/org/apache/cassandra/streaming/PendingFile.java
deleted file mode 100644
index 0d9ec35..0000000
--- a/src/java/org/apache/cassandra/streaming/PendingFile.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.streaming;
-
-import java.io.*;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.compress.CompressionMetadata;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.streaming.compress.CompressionInfo;
-import org.apache.cassandra.utils.Pair;
-
-/**
- * Represents portions of a file to be streamed between nodes.
- */
-public class PendingFile
-{
-    public static final PendingFileSerializer serializer = new PendingFileSerializer();
-
-    // NB: this reference is used to be able to release the acquired reference upon completion
-    public final SSTableReader sstable;
-
-    public final Descriptor desc;
-    public final String component;
-    public final List<Pair<Long, Long>> sections;
-    public final OperationType type;
-    /** total length of data to transfer */
-    public final long size;
-    /** estimated number of keys to transfer */
-    public final long estimatedKeys;
-    /** compression information. null if data is not compressed */
-    public final CompressionInfo compressionInfo;
-    public long progress;
-
-    public PendingFile(Descriptor desc, PendingFile pf)
-    {
-        this(null, desc, pf.component, pf.sections, pf.type, pf.estimatedKeys, pf.compressionInfo);
-    }
-
-    public PendingFile(SSTableReader sstable, Descriptor desc, String component, List<Pair<Long,Long>> sections, OperationType type)
-    {
-        this(sstable, desc, component, sections, type, 0, null);
-    }
-
-    public PendingFile(SSTableReader sstable,
-                       Descriptor desc,
-                       String component,
-                       List<Pair<Long,Long>> sections,
-                       OperationType type,
-                       long estimatedKeys,
-                       CompressionInfo compressionInfo)
-    {
-        this.sstable = sstable;
-        this.desc = desc;
-        this.component = component;
-        this.sections = sections;
-        this.type = type;
-
-        long tempSize = 0;
-        if (compressionInfo == null)
-        {
-            for (Pair<Long, Long> section : sections)
-                tempSize += section.right - section.left;
-        }
-        else
-        {
-            // calculate total length of transferring chunks
-            for (CompressionMetadata.Chunk chunk : compressionInfo.chunks)
-                tempSize += chunk.length + 4; // 4 bytes for CRC
-        }
-        size = tempSize;
-
-        this.estimatedKeys = estimatedKeys;
-        this.compressionInfo = compressionInfo;
-    }
-
-    public String getFilename()
-    {
-        return desc.filenameFor(component);
-    }
-
-    public boolean equals(Object o)
-    {
-        if (!(o instanceof PendingFile))
-            return false;
-
-        PendingFile rhs = (PendingFile)o;
-        return getFilename().equals(rhs.getFilename());
-    }
-
-    public int hashCode()
-    {
-        return getFilename().hashCode();
-    }
-
-    public String toString()
-    {
-        return getFilename() + " sections=" + sections.size() + " progress=" + progress + "/" + size + " - " + progress*100/size + "%";
-    }
-
-    public static class PendingFileSerializer implements IVersionedSerializer<PendingFile>
-    {
-        public void serialize(PendingFile sc, DataOutput out, int version) throws IOException
-        {
-            if (sc == null)
-            {
-                out.writeUTF("");
-                return;
-            }
-
-            out.writeUTF(sc.desc.filenameFor(sc.component));
-            out.writeUTF(sc.component);
-            out.writeInt(sc.sections.size());
-            for (Pair<Long,Long> section : sc.sections)
-            {
-                out.writeLong(section.left);
-                out.writeLong(section.right);
-            }
-            out.writeUTF(sc.type.name());
-            out.writeLong(sc.estimatedKeys);
-            CompressionInfo.serializer.serialize(sc.compressionInfo, out, version);
-        }
-
-        public PendingFile deserialize(DataInput in, int version) throws IOException
-        {
-            String filename = in.readUTF();
-            if (filename.isEmpty())
-                return null;
-
-            Descriptor desc = Descriptor.fromFilename(filename);
-            String component = in.readUTF();
-            int count = in.readInt();
-            List<Pair<Long,Long>> sections = new ArrayList<Pair<Long,Long>>(count);
-            for (int i = 0; i < count; i++)
-                sections.add(Pair.create(in.readLong(), in.readLong()));
-            // this controls the way indexes are rebuilt when streaming in.
-            OperationType type = OperationType.RESTORE_REPLICA_COUNT;
-            type = OperationType.valueOf(in.readUTF());
-            long estimatedKeys = in.readLong();
-            CompressionInfo info = null;
-            info = CompressionInfo.serializer.deserialize(in, version);
-            return new PendingFile(null, desc, component, sections, type, estimatedKeys, info);
-        }
-
-        public long serializedSize(PendingFile pf, int version)
-        {
-            if (pf == null)
-                return TypeSizes.NATIVE.sizeof("");
-
-            long size = TypeSizes.NATIVE.sizeof(pf.desc.filenameFor(pf.component));
-            size += TypeSizes.NATIVE.sizeof(pf.component);
-            size += TypeSizes.NATIVE.sizeof(pf.sections.size());
-            for (Pair<Long,Long> section : pf.sections)
-                size += TypeSizes.NATIVE.sizeof(section.left) + TypeSizes.NATIVE.sizeof(section.right);
-            size += TypeSizes.NATIVE.sizeof(pf.type.name());
-            size += TypeSizes.NATIVE.sizeof(pf.estimatedKeys);
-            size += CompressionInfo.serializer.serializedSize(pf.compressionInfo, version);
-            return size;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/ProgressInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/ProgressInfo.java b/src/java/org/apache/cassandra/streaming/ProgressInfo.java
new file mode 100644
index 0000000..d308ed0
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/ProgressInfo.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.io.Serializable;
+import java.net.InetAddress;
+
+import com.google.common.base.Objects;
+
+/**
+ * ProgressInfo contains file transfer progress.
+ */
+public class ProgressInfo implements Serializable
+{
+    /**
+     * Direction of the stream.
+     */
+    public static enum Direction
+    {
+        OUT(0),
+        IN(1);
+
+        public final byte code;
+
+        private Direction(int code)
+        {
+            this.code = (byte) code;
+        }
+
+        public static Direction fromByte(byte direction)
+        {
+            return direction == 0 ? OUT : IN;
+        }
+    }
+
+    public final InetAddress peer;
+    public final String fileName;
+    public final Direction direction;
+    public final long currentBytes;
+    public final long totalBytes;
+
+    public ProgressInfo(InetAddress peer, String fileName, Direction direction, long currentBytes, long totalBytes)
+    {
+        assert totalBytes > 0;
+
+        this.peer = peer;
+        this.fileName = fileName;
+        this.direction = direction;
+        this.currentBytes = currentBytes;
+        this.totalBytes = totalBytes;
+    }
+
+    /**
+     * @return true if file transfer is completed
+     */
+    public boolean isCompleted()
+    {
+        return currentBytes == totalBytes;
+    }
+
+    /**
+     * ProgressInfo is considered to be equal only when all attributes except currentBytes are equal.
+     */
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+
+        ProgressInfo that = (ProgressInfo) o;
+
+        if (totalBytes != that.totalBytes) return false;
+        if (direction != that.direction) return false;
+        if (!fileName.equals(that.fileName)) return false;
+        return peer.equals(that.peer);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return Objects.hashCode(peer, fileName, direction, totalBytes);
+    }
+
+    @Override
+    public String toString()
+    {
+        StringBuilder sb = new StringBuilder(fileName);
+        sb.append(" ").append(currentBytes);
+        sb.append("/").append(totalBytes).append(" bytes");
+        sb.append("(").append(currentBytes*100/totalBytes).append("%) ");
+        sb.append(direction == Direction.OUT ? "sent to " : "received from ");
+        sb.append(peer);
+        return sb.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/SessionInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/SessionInfo.java b/src/java/org/apache/cassandra/streaming/SessionInfo.java
new file mode 100644
index 0000000..9532041
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/SessionInfo.java
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.io.Serializable;
+import java.net.InetAddress;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+
+/**
+ * Stream session info.
+ */
+public final class SessionInfo implements Serializable
+{
+    public final InetAddress peer;
+    /** Immutable collection of receiving summaries */
+    public final Collection<StreamSummary> receivingSummaries;
+    /** Immutable collection of sending summaries*/
+    public final Collection<StreamSummary> sendingSummaries;
+    /** Current session state */
+    public final StreamSession.State state;
+
+    private final Map<String, ProgressInfo> receivingFiles;
+    private final Map<String, ProgressInfo> sendingFiles;
+
+    public SessionInfo(InetAddress peer,
+                       Collection<StreamSummary> receivingSummaries,
+                       Collection<StreamSummary> sendingSummaries,
+                       StreamSession.State state)
+    {
+        this.peer = peer;
+        this.receivingSummaries = ImmutableSet.copyOf(receivingSummaries);
+        this.sendingSummaries = ImmutableSet.copyOf(sendingSummaries);
+        this.receivingFiles = new HashMap<>();
+        this.sendingFiles = new HashMap<>();
+        this.state = state;
+    }
+
+    public boolean isFailed()
+    {
+        return state == StreamSession.State.FAILED;
+    }
+
+    /**
+     * Update progress of receiving/sending file.
+     *
+     * @param newProgress new progress info
+     */
+    public synchronized void updateProgress(ProgressInfo newProgress)
+    {
+        assert peer.equals(newProgress.peer);
+
+        Map<String, ProgressInfo> currentFiles = newProgress.direction == ProgressInfo.Direction.IN
+                                                    ? receivingFiles : sendingFiles;
+        currentFiles.put(newProgress.fileName, newProgress);
+    }
+
+    public Collection<ProgressInfo> getReceivingFiles()
+    {
+        return receivingFiles.values();
+    }
+
+    public Collection<ProgressInfo> getSendingFiles()
+    {
+        return sendingFiles.values();
+    }
+
+    /**
+     * @return total number of files already received.
+     */
+    public long getTotalFilesReceived()
+    {
+        return getTotalFilesCompleted(receivingFiles.values());
+    }
+
+    /**
+     * @return total number of files already sent.
+     */
+    public long getTotalFilesSent()
+    {
+        return getTotalFilesCompleted(sendingFiles.values());
+    }
+
+    /**
+     * @return total size(in bytes) already received.
+     */
+    public long getTotalSizeReceived()
+    {
+        return getTotalSizeInProgress(receivingFiles.values());
+    }
+
+    /**
+     * @return total size(in bytes) already sent.
+     */
+    public long getTotalSizeSent()
+    {
+        return getTotalSizeInProgress(sendingFiles.values());
+    }
+
+    /**
+     * @return total number of files to receive in the session
+     */
+    public long getTotalFilesToReceive()
+    {
+        return getTotalFiles(receivingSummaries);
+    }
+
+    /**
+     * @return total number of files to send in the session
+     */
+    public long getTotalFilesToSend()
+    {
+        return getTotalFiles(sendingSummaries);
+    }
+
+    /**
+     * @return total size(in bytes) to receive in the session
+     */
+    public long getTotalSizeToReceive()
+    {
+        return getTotalSizes(receivingSummaries);
+    }
+
+    /**
+     * @return total size(in bytes) to send in the session
+     */
+    public long getTotalSizeToSend()
+    {
+        return getTotalSizes(sendingSummaries);
+    }
+
+    private long getTotalSizeInProgress(Collection<ProgressInfo> files)
+    {
+        long total = 0;
+        for (ProgressInfo file : files)
+            total += file.currentBytes;
+        return total;
+    }
+
+    private long getTotalFiles(Collection<StreamSummary> summaries)
+    {
+        long total = 0;
+        for (StreamSummary summary : summaries)
+            total += summary.files;
+        return total;
+    }
+
+    private long getTotalSizes(Collection<StreamSummary> summaries)
+    {
+        long total = 0;
+        for (StreamSummary summary : summaries)
+            total += summary.totalSize;
+        return total;
+    }
+
+    private long getTotalFilesCompleted(Collection<ProgressInfo> files)
+    {
+        Iterable<ProgressInfo> completed = Iterables.filter(files, new Predicate<ProgressInfo>()
+        {
+            public boolean apply(ProgressInfo input)
+            {
+                return input.isCompleted();
+            }
+        });
+        return Iterables.size(completed);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamEvent.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamEvent.java b/src/java/org/apache/cassandra/streaming/StreamEvent.java
new file mode 100644
index 0000000..9af1fbd
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamEvent.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.net.InetAddress;
+import java.util.UUID;
+
+public abstract class StreamEvent
+{
+    public static enum Type
+    {
+        STREAM_PREPARED,
+        STREAM_COMPLETE,
+        FILE_PROGRESS,
+    }
+
+    public final Type eventType;
+    public final UUID planId;
+
+    protected StreamEvent(Type eventType, UUID planId)
+    {
+        this.eventType = eventType;
+        this.planId = planId;
+    }
+
+    public static class SessionCompleteEvent extends StreamEvent
+    {
+        public final InetAddress peer;
+        public final boolean success;
+
+        public SessionCompleteEvent(StreamSession session)
+        {
+            super(Type.STREAM_COMPLETE, session.planId());
+            this.peer = session.peer;
+            this.success = session.isSuccess();
+        }
+    }
+
+    public static class ProgressEvent extends StreamEvent
+    {
+        public final ProgressInfo progress;
+
+        public ProgressEvent(UUID planId, ProgressInfo progress)
+        {
+            super(Type.FILE_PROGRESS, planId);
+            this.progress = progress;
+        }
+
+        @Override
+        public String toString()
+        {
+            return "<ProgressEvent " + progress.toString() + ">";
+        }
+    }
+
+    public static class SessionPreparedEvent extends StreamEvent
+    {
+        public final SessionInfo session;
+
+        public SessionPreparedEvent(UUID planId, SessionInfo session)
+        {
+            super(Type.STREAM_PREPARED, planId);
+            this.session = session;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamEventHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamEventHandler.java b/src/java/org/apache/cassandra/streaming/StreamEventHandler.java
new file mode 100644
index 0000000..e2e84bb
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamEventHandler.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import com.google.common.util.concurrent.FutureCallback;
+
+public interface StreamEventHandler extends FutureCallback<StreamState>
+{
+    /**
+     * Callback for various streaming events.
+     *
+     * @see StreamEvent.Type
+     * @param event Stream event.
+     */
+    void handleStreamEvent(StreamEvent event);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamException.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamException.java b/src/java/org/apache/cassandra/streaming/StreamException.java
new file mode 100644
index 0000000..6e22db2
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamException.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+public class StreamException extends Throwable
+{
+    public final StreamState finalState;
+
+    public StreamException(StreamState finalState, String message)
+    {
+        super(message);
+        this.finalState = finalState;
+    }
+
+    public StreamException(StreamState finalState, String message, Throwable cause)
+    {
+        super(message, cause);
+        this.finalState = finalState;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamHeader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamHeader.java b/src/java/org/apache/cassandra/streaming/StreamHeader.java
deleted file mode 100644
index a0bf832..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamHeader.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.streaming;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.*;
-
-import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.UUIDSerializer;
-
-public class StreamHeader
-{
-    public static final IVersionedSerializer<StreamHeader> serializer = new StreamHeaderSerializer();
-
-    public final String table;
-
-    /** file being sent on initial stream */
-    public final PendingFile file;
-
-    /** session ID */
-    public final UUID sessionId;
-
-    /** files to add to the session */
-    public final Collection<PendingFile> pendingFiles;
-
-    public StreamHeader(String table, UUID sessionId, PendingFile file)
-    {
-        this(table, sessionId, file, Collections.<PendingFile>emptyList());
-    }
-
-    public StreamHeader(String table, UUID sessionId, PendingFile first, Collection<PendingFile> pendingFiles)
-    {
-        this.table = table;
-        this.sessionId  = sessionId;
-        this.file = first;
-        this.pendingFiles = pendingFiles;
-    }
-
-    private static class StreamHeaderSerializer implements IVersionedSerializer<StreamHeader>
-    {
-        public void serialize(StreamHeader sh, DataOutput out, int version) throws IOException
-        {
-            out.writeUTF(sh.table);
-            UUIDSerializer.serializer.serialize(sh.sessionId, out, MessagingService.current_version);
-            PendingFile.serializer.serialize(sh.file, out, version);
-            out.writeInt(sh.pendingFiles.size());
-            for (PendingFile file : sh.pendingFiles)
-                PendingFile.serializer.serialize(file, out, version);
-        }
-
-        public StreamHeader deserialize(DataInput in, int version) throws IOException
-        {
-            String table = in.readUTF();
-            UUID sessionId = UUIDSerializer.serializer.deserialize(in, MessagingService.current_version);
-            PendingFile file = PendingFile.serializer.deserialize(in, version);
-            int size = in.readInt();
-
-            List<PendingFile> pendingFiles = new ArrayList<PendingFile>(size);
-            for (int i = 0; i < size; i++)
-                pendingFiles.add(PendingFile.serializer.deserialize(in, version));
-            return new StreamHeader(table, sessionId, file, pendingFiles);
-        }
-
-        public long serializedSize(StreamHeader sh, int version)
-        {
-            long size = TypeSizes.NATIVE.sizeof(sh.table);
-            size += TypeSizes.NATIVE.sizeof(sh.sessionId);
-            size += PendingFile.serializer.serializedSize(sh.file, version);
-            size += TypeSizes.NATIVE.sizeof(sh.pendingFiles.size());
-            for (PendingFile file : sh.pendingFiles)
-                size += PendingFile.serializer.serializedSize(file, version);
-            return size;
-       }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamIn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamIn.java b/src/java/org/apache/cassandra/streaming/StreamIn.java
deleted file mode 100644
index 588728c..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamIn.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.streaming;
-
-import java.net.InetAddress;
-import java.util.Collection;
-
-import org.apache.commons.lang.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Directories;
-import org.apache.cassandra.db.Table;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.FBUtilities;
-
-/**
- * for streaming data from other nodes in to this one.
- * Sends a STREAM_REQUEST Message to the source node(s), after which StreamOut on that side takes over.
- * See StreamOut for details.
- */
-public class StreamIn
-{
-    private static final Logger logger = LoggerFactory.getLogger(StreamIn.class);
-
-    /** Request ranges for all column families in the given keyspace. */
-    public static void requestRanges(InetAddress source, String tableName, Collection<Range<Token>> ranges, IStreamCallback callback, OperationType type)
-    {
-        requestRanges(source, tableName, Table.open(tableName).getColumnFamilyStores(), ranges, callback, type);
-    }
-
-    /**
-     * Request ranges to be transferred from specific CFs
-     */
-    public static void requestRanges(InetAddress source, String tableName, Collection<ColumnFamilyStore> columnFamilies, Collection<Range<Token>> ranges, IStreamCallback callback, OperationType type)
-    {
-        assert ranges.size() > 0;
-
-        if (logger.isDebugEnabled())
-            logger.debug("Requesting from {} ranges {}", source, StringUtils.join(ranges, ", "));
-        StreamInSession session = StreamInSession.create(source, callback);
-        StreamRequest srm = new StreamRequest(FBUtilities.getBroadcastAddress(),
-                                                            ranges,
-                                                            tableName,
-                                                            columnFamilies,
-                                                            session.getSessionId(),
-                                                            type);
-        MessagingService.instance().sendOneWay(srm.createMessage(), source);
-    }
-
-    /** Translates remote files to local files by creating a local sstable per remote sstable. */
-    public static PendingFile getContextMapping(PendingFile remote)
-    {
-        /* Create a local sstable for each remote sstable */
-        Descriptor remotedesc = remote.desc;
-        if (!remotedesc.isStreamCompatible())
-            throw new UnsupportedOperationException(String.format("SSTable %s is not compatible with current version %s",
-                                                                  remote.getFilename(), Descriptor.Version.CURRENT));
-
-        // new local sstable
-        Table table = Table.open(remotedesc.ksname);
-        ColumnFamilyStore cfStore = table.getColumnFamilyStore(remotedesc.cfname);
-        Directories.DataDirectory localDir = cfStore.directories.getLocationCapableOfSize(remote.size);
-        if (localDir == null)
-            throw new RuntimeException("Insufficient disk space to store " + remote.size + " bytes");
-        Descriptor localdesc = Descriptor.fromFilename(cfStore.getTempSSTablePath(cfStore.directories.getLocationForDisk(localDir)));
-
-        return new PendingFile(localdesc, remote);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamInSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamInSession.java b/src/java/org/apache/cassandra/streaming/StreamInSession.java
deleted file mode 100644
index 370183f..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamInSession.java
+++ /dev/null
@@ -1,251 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.streaming;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.Socket;
-import java.util.*;
-import java.util.concurrent.ConcurrentMap;
-
-import org.cliffc.high_scale_lib.NonBlockingHashMap;
-import org.cliffc.high_scale_lib.NonBlockingHashSet;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Table;
-import org.apache.cassandra.gms.FailureDetector;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.net.MessageOut;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.UUIDGen;
-
-/** each context gets its own StreamInSession. So there may be >1 Session per host */
-public class StreamInSession extends AbstractStreamSession
-{
-    private static final Logger logger = LoggerFactory.getLogger(StreamInSession.class);
-
-    private static final ConcurrentMap<UUID, StreamInSession> sessions = new NonBlockingHashMap<UUID, StreamInSession>();
-
-    private final Set<PendingFile> files = new NonBlockingHashSet<PendingFile>();
-    private final List<SSTableReader> readers = new ArrayList<SSTableReader>();
-    private PendingFile current;
-    private Socket socket;
-    private volatile int retries;
-
-    private StreamInSession(InetAddress host, UUID sessionId, IStreamCallback callback)
-    {
-        super(null, host, sessionId, callback);
-    }
-
-    public static StreamInSession create(InetAddress host, IStreamCallback callback)
-    {
-        StreamInSession session = new StreamInSession(host, UUIDGen.getTimeUUID(), callback);
-        sessions.put(session.getSessionId(), session);
-        return session;
-    }
-
-    public static StreamInSession get(InetAddress host, UUID sessionId)
-    {
-        StreamInSession session = sessions.get(sessionId);
-        if (session == null)
-        {
-            StreamInSession possibleNew = new StreamInSession(host, sessionId, null);
-            if ((session = sessions.putIfAbsent(sessionId, possibleNew)) == null)
-                session = possibleNew;
-        }
-        return session;
-    }
-
-    public static boolean hasSession(UUID sessionId)
-    {
-        return sessions.get(sessionId) != null;
-    }
-
-    public void setCurrentFile(PendingFile file)
-    {
-        this.current = file;
-    }
-
-    public void setTable(String table)
-    {
-        this.table = table;
-    }
-
-    public void setSocket(Socket socket)
-    {
-        this.socket = socket;
-    }
-
-    public void addFiles(Collection<PendingFile> files)
-    {
-        for (PendingFile file : files)
-        {
-            if(logger.isDebugEnabled())
-                logger.debug("Adding file {} to Stream Request queue", file.getFilename());
-            this.files.add(file);
-        }
-    }
-
-    public void finished(PendingFile remoteFile, SSTableReader reader) throws IOException
-    {
-        if (logger.isDebugEnabled())
-            logger.debug("Finished {} (from {}). Sending ack to {}", new Object[] {remoteFile, getHost(), this});
-
-        assert reader != null;
-        readers.add(reader);
-        files.remove(remoteFile);
-        if (remoteFile.equals(current))
-            current = null;
-        StreamReply reply = new StreamReply(remoteFile.getFilename(), getSessionId(), StreamReply.Status.FILE_FINISHED);
-        // send a StreamStatus message telling the source node it can delete this file
-        sendMessage(reply.createMessage());
-        logger.debug("ack {} sent for {}", reply, remoteFile);
-    }
-
-    public void retry(PendingFile remoteFile)
-    {
-        retries++;
-        if (retries > DatabaseDescriptor.getMaxStreamingRetries())
-        {
-            logger.error(String.format("Failed streaming session %s from %s while receiving %s", getSessionId(), getHost().toString(), current),
-                         new IllegalStateException("Too many retries for " + remoteFile));
-            close(false);
-            return;
-        }
-        StreamReply reply = new StreamReply(remoteFile.getFilename(), getSessionId(), StreamReply.Status.FILE_RETRY);
-        logger.info("Streaming of file {} for {} failed: requesting a retry.", remoteFile, this);
-        try
-        {
-            sendMessage(reply.createMessage());
-        }
-        catch (IOException e)
-        {
-            logger.error("Sending retry message failed, closing session.", e);
-            close(false);
-        }
-    }
-
-    public void sendMessage(MessageOut<StreamReply> message) throws IOException
-    {
-        DataOutputStream out = new DataOutputStream(socket.getOutputStream());
-        FileStreamTask.sendReply(message,
-                                 out);
-        out.flush();
-    }
-
-    public void closeIfFinished() throws IOException
-    {
-        if (files.isEmpty())
-        {
-            HashMap <ColumnFamilyStore, List<SSTableReader>> cfstores = new HashMap<ColumnFamilyStore, List<SSTableReader>>();
-            try
-            {
-                for (SSTableReader sstable : readers)
-                {
-                    assert sstable.getTableName().equals(table);
-
-                    // Acquire the reference (for secondary index building) before submitting the index build,
-                    // so it can't get compacted out of existence in between
-                    if (!sstable.acquireReference())
-                        throw new AssertionError("We shouldn't fail acquiring a reference on a sstable that has just been transferred");
-
-                    ColumnFamilyStore cfs = Table.open(sstable.getTableName()).getColumnFamilyStore(sstable.getColumnFamilyName());
-                    if (!cfstores.containsKey(cfs))
-                        cfstores.put(cfs, new ArrayList<SSTableReader>());
-                    cfstores.get(cfs).add(sstable);
-                }
-
-                // add sstables and build secondary indexes
-                for (Map.Entry<ColumnFamilyStore, List<SSTableReader>> entry : cfstores.entrySet())
-                {
-                    if (entry.getKey() != null)
-                    {
-                        entry.getKey().addSSTables(entry.getValue());
-                        entry.getKey().indexManager.maybeBuildSecondaryIndexes(entry.getValue(), entry.getKey().indexManager.allIndexesNames());
-                    }
-                }
-            }
-            finally
-            {
-                for (List<SSTableReader> referenced : cfstores.values())
-                    SSTableReader.releaseReferences(referenced);
-            }
-
-            // send reply to source that we're done
-            StreamReply reply = new StreamReply("", getSessionId(), StreamReply.Status.SESSION_FINISHED);
-            logger.info("Finished streaming session {} from {}", getSessionId(), getHost());
-            try
-            {
-                if (socket != null)
-                    FileStreamTask.sendReply(reply.createMessage(),
-                                             new DataOutputStream(socket.getOutputStream()));
-                else
-                    logger.debug("No socket to reply to {} with!", getHost());
-            }
-            finally
-            {
-                if (socket != null)
-                    socket.close();
-            }
-
-            close(true);
-        }
-    }
-
-    protected void closeInternal(boolean success)
-    {
-        sessions.remove(sessionId);
-        if (!success && FailureDetector.instance.isAlive(getHost()))
-        {
-            StreamReply reply = new StreamReply("", getSessionId(), StreamReply.Status.SESSION_FAILURE);
-            MessagingService.instance().sendOneWay(reply.createMessage(), getHost());
-        }
-    }
-
-    /** query method to determine which hosts are streaming to this node. */
-    public static Set<InetAddress> getSources()
-    {
-        HashSet<InetAddress> set = new HashSet<InetAddress>();
-        for (StreamInSession session : sessions.values())
-        {
-            set.add(session.getHost());
-        }
-        return set;
-    }
-
-    /** query the status of incoming files. */
-    public static Set<PendingFile> getIncomingFiles(InetAddress host)
-    {
-        Set<PendingFile> set = new HashSet<PendingFile>();
-        for (Map.Entry<UUID, StreamInSession> entry : sessions.entrySet())
-        {
-            StreamInSession session = entry.getValue();
-            if (session.getHost().equals(host))
-            {
-                if (session.current != null)
-                    set.add(session.current);
-                set.addAll(session.files);
-            }
-        }
-        return set;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamManager.java b/src/java/org/apache/cassandra/streaming/StreamManager.java
new file mode 100644
index 0000000..d64cc65
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamManager.java
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.RateLimiter;
+import com.google.common.util.concurrent.MoreExecutors;
+import org.cliffc.high_scale_lib.NonBlockingHashMap;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+
+/**
+ * StreamManager manages currently running {@link StreamResultFuture}s and provides status of all operation invoked.
+ *
+ * All stream operation should be created through this class to track streaming status and progress.
+ */
+public class StreamManager implements StreamManagerMBean
+{
+    public static final StreamManager instance = new StreamManager();
+
+    private static final RateLimiter limiter = RateLimiter.create(Double.MAX_VALUE);
+
+    /**
+     * Gets streaming rate limiter.
+     * When stream_throughput_outbound_megabits_per_sec is 0, this returns rate limiter
+     * with the rate of Double.MAX_VALUE bytes per second.
+     * Rate unit is bytes per sec.
+     *
+     * @return RateLimiter with rate limit set
+     */
+    public static RateLimiter getRateLimiter()
+    {
+        double currentThroughput = DatabaseDescriptor.getStreamThroughputOutboundMegabitsPerSec() * 1024 * 1024 / 8 / 1000;
+        // if throughput is set to 0, throttling is disabled
+        if (currentThroughput == 0)
+            currentThroughput = Double.MAX_VALUE;
+        if (limiter.getRate() != currentThroughput)
+            limiter.setRate(currentThroughput);
+        return limiter;
+    }
+
+    /** Currently running stream plans. Removed after completion/failure. */
+    private final Map<UUID, StreamResultFuture> currentStreams = new NonBlockingHashMap<>();
+
+    public Set<StreamState> getCurrentStreams()
+    {
+        return Sets.newHashSet(Iterables.transform(currentStreams.values(), new Function<StreamResultFuture, StreamState>()
+        {
+            public StreamState apply(StreamResultFuture input)
+            {
+                return input.getCurrentState();
+            }
+        }));
+    }
+
+    public void register(final StreamResultFuture result)
+    {
+        // Make sure we remove the stream on completion (whether successful or not)
+        result.addListener(new Runnable()
+        {
+            public void run()
+            {
+                currentStreams.remove(result.planId);
+            }
+        }, MoreExecutors.sameThreadExecutor());
+
+        currentStreams.put(result.planId, result);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamManagerMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamManagerMBean.java b/src/java/org/apache/cassandra/streaming/StreamManagerMBean.java
new file mode 100644
index 0000000..eb6f6ae
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/StreamManagerMBean.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+public interface StreamManagerMBean
+{
+    public static final String OBJECT_NAME = "org.apache.cassandra.net:type=StreamManager";
+
+    /**
+     * Returns the current state of all ongoing streams.
+     */
+    Set<StreamState> getCurrentStreams();
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamOut.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamOut.java b/src/java/org/apache/cassandra/streaming/StreamOut.java
deleted file mode 100644
index 709741b..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamOut.java
+++ /dev/null
@@ -1,187 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.streaming;
-
-import java.net.InetAddress;
-import java.util.*;
-import java.util.concurrent.Future;
-
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import org.apache.commons.lang.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Table;
-import org.apache.cassandra.db.RowPosition;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.dht.AbstractBounds;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.sstable.SSTable;
-import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.streaming.compress.CompressionInfo;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.Pair;
-
-/**
- * This class handles streaming data from one node to another.
- *
- * The source node [the Out side] is always in charge of the streaming session.  Streams may
- * be initiated either directly by the source via the methods in this class,
- * or on demand from the target (via StreamRequest).
- *
- * Files to stream are grouped into sessions, which can have callbacks associated
- * with them so that (for instance) we can mark a new node a full member of the
- * cluster after all the data it needs has been streamed.
- *
- * The source begins a session by sending
- * a Message with the stream bit flag in the Header turned on.  Part of that Message
- * will include a StreamHeader that includes the files that will be streamed as part
- * of that session, as well as the first file-to-be-streamed. (Combining session list
- * and first file like this is inconvenient, but not as inconvenient as the old
- * three-part send-file-list, wait-for-ack, start-first-file dance.)
- *
- * This is done over a separate TCP connection to avoid blocking ordinary intra-node
- * traffic during the stream.  So there is no Handler for the main stream of data --
- * when a connection sets the Stream bit, IncomingTcpConnection knows what to expect
- * without any further Messages.
- *
- * After each file, the target node [the In side] will send a StreamReply indicating success
- * (FILE_FINISHED) or failure (FILE_RETRY).
- *
- * When all files have been successfully transferred and integrated the target will
- * send an additional SESSION_FINISHED reply and the session is complete.
- *
- * For Stream requests (for bootstrap), one subtlety is that we always have to
- * create at least one stream reply, even if the list of files is empty, otherwise the
- * target has no way to know that it can stop waiting for an answer.
- *
- */
-public class StreamOut
-{
-    private static final Logger logger = LoggerFactory.getLogger(StreamOut.class);
-
-    /**
-     * Stream the given ranges to the target endpoint from each CF in the given keyspace.
-    */
-    public static void transferRanges(InetAddress target, Table table, Collection<Range<Token>> ranges, IStreamCallback callback, OperationType type)
-    {
-        transferRanges(target, table, table.getColumnFamilyStores(), ranges, callback, type);
-    }
-
-    /**
-     * Stream the given ranges to the target endpoint for provided CFs in the given keyspace.
-     */
-    public static void transferRanges(InetAddress target, Table table, Iterable<ColumnFamilyStore> cfses, Collection<Range<Token>> ranges, IStreamCallback callback, OperationType type)
-    {
-        StreamOutSession session = StreamOutSession.create(table.getName(), target, callback);
-        transferRanges(session, cfses, ranges, type);
-    }
-
-    /**
-     * Flushes matching column families from the given keyspace, or all columnFamilies
-     * if the cf list is empty.
-     */
-    private static void flushSSTables(Iterable<ColumnFamilyStore> stores)
-    {
-        logger.info("Flushing memtables for {}...", stores);
-        List<Future<?>> flushes = new ArrayList<Future<?>>();
-        for (ColumnFamilyStore cfstore : stores)
-            flushes.add(cfstore.forceFlush());
-        FBUtilities.waitOnFutures(flushes);
-    }
-
-    /**
-     * Stream the given ranges to the target endpoint from each of the given CFs.
-    */
-    public static void transferRanges(StreamOutSession session, Iterable<ColumnFamilyStore> cfses, Collection<Range<Token>> ranges, OperationType type)
-    {
-        transferRanges(session, cfses, ranges, type, true);
-    }
-
-    /**
-     * Stream the given ranges to the target endpoint from each of the given CFs.
-    */
-    public static void transferRanges(StreamOutSession session,
-                                      Iterable<ColumnFamilyStore> cfses,
-                                      Collection<Range<Token>> ranges,
-                                      OperationType type,
-                                      boolean flushTables)
-    {
-        assert ranges.size() > 0;
-        logger.info("Beginning transfer to {}", session.getHost());
-        logger.debug("Ranges are {}", StringUtils.join(ranges, ","));
-
-        if (flushTables)
-            flushSSTables(cfses);
-
-        List<SSTableReader> sstables = Lists.newLinkedList();
-        for (ColumnFamilyStore cfStore : cfses)
-        {
-            List<AbstractBounds<RowPosition>> rowBoundsList = Lists.newLinkedList();
-            for (Range<Token> range : ranges)
-                rowBoundsList.add(range.toRowBounds());
-            ColumnFamilyStore.ViewFragment view = cfStore.markReferenced(rowBoundsList);
-            sstables.addAll(view.sstables);
-        }
-
-        transferSSTables(session, sstables, ranges, type);
-    }
-
-    /**
-     * Low-level transfer of matching portions of a group of sstables from a single table to the target endpoint.
-     * You should probably call transferRanges instead. This moreover assumes that references have been acquired on the sstables.
-     */
-    public static void transferSSTables(StreamOutSession session, Iterable<SSTableReader> sstables, Collection<Range<Token>> ranges, OperationType type)
-    {
-        List<PendingFile> pending = createPendingFiles(sstables, ranges, type);
-
-        // Even if the list of pending files is empty, we need to initiate the transfer otherwise
-        // the remote end will hang in cases where this was a requested transfer.
-        session.addFilesToStream(pending);
-        session.begin();
-    }
-
-    // called prior to sending anything.
-    private static List<PendingFile> createPendingFiles(Iterable<SSTableReader> sstables, Collection<Range<Token>> ranges, OperationType type)
-    {
-        List<PendingFile> pending = new ArrayList<PendingFile>();
-        for (SSTableReader sstable : sstables)
-        {
-            Descriptor desc = sstable.descriptor;
-            List<Pair<Long,Long>> sections = sstable.getPositionsForRanges(ranges);
-            if (sections.isEmpty())
-            {
-                // A reference was acquired on the sstable and we won't stream it
-                sstable.releaseReference();
-                continue;
-            }
-            CompressionInfo compression = null;
-            if (sstable.compression)
-            {
-                compression = new CompressionInfo(sstable.getCompressionMetadata().getChunksForSections(sections),
-                                                  sstable.getCompressionMetadata().parameters);
-            }
-            pending.add(new PendingFile(sstable, desc, SSTable.COMPONENT_DATA, sections, type, sstable.estimatedKeysForRanges(ranges), compression));
-        }
-        logger.info("Stream context metadata {}, {} sstables.", pending, Iterables.size(sstables));
-        return pending;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/StreamOutSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamOutSession.java b/src/java/org/apache/cassandra/streaming/StreamOutSession.java
deleted file mode 100644
index edc07ca..0000000
--- a/src/java/org/apache/cassandra/streaming/StreamOutSession.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.streaming;
-
-import java.net.InetAddress;
-import java.util.*;
-import java.util.concurrent.ConcurrentMap;
-
-import org.apache.commons.lang.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.UUIDGen;
-
-import org.cliffc.high_scale_lib.NonBlockingHashMap;
-
-/**
- * This class manages the streaming of multiple files one after the other.
- */
-public class StreamOutSession extends AbstractStreamSession
-{
-    private static final Logger logger = LoggerFactory.getLogger(StreamOutSession.class);
-
-    // one host may have multiple stream sessions.
-    private static final ConcurrentMap<UUID, StreamOutSession> streams = new NonBlockingHashMap<UUID, StreamOutSession>();
-
-    public static StreamOutSession create(String table, InetAddress host, IStreamCallback callback)
-    {
-        return create(table, host, UUIDGen.getTimeUUID(), callback);
-    }
-
-    public static StreamOutSession create(String table, InetAddress host, UUID sessionId)
-    {
-        return create(table, host, sessionId, null);
-    }
-
-    public static StreamOutSession create(String table, InetAddress host, UUID sessionId, IStreamCallback callback)
-    {
-        StreamOutSession session = new StreamOutSession(table, host, sessionId, callback);
-        streams.put(sessionId, session);
-        return session;
-    }
-
-    public static StreamOutSession get(UUID sessionId)
-    {
-        return streams.get(sessionId);
-    }
-
-    private final Map<String, PendingFile> files = new NonBlockingHashMap<String, PendingFile>();
-
-    private volatile String currentFile;
-
-    private StreamOutSession(String table, InetAddress host, UUID sessionId, IStreamCallback callback)
-    {
-        super(table, host, sessionId, callback);
-    }
-
-    public void addFilesToStream(List<PendingFile> pendingFiles)
-    {
-        for (PendingFile pendingFile : pendingFiles)
-        {
-            if (logger.isDebugEnabled())
-                logger.debug("Adding file {} to be streamed.", pendingFile.getFilename());
-            files.put(pendingFile.getFilename(), pendingFile);
-        }
-    }
-
-    public void retry()
-    {
-        streamFile(files.get(currentFile));
-    }
-
-    private void streamFile(PendingFile pf)
-    {
-        if (logger.isDebugEnabled())
-            logger.debug("Streaming {} ...", pf);
-        currentFile = pf.getFilename();
-        MessagingService.instance().stream(new StreamHeader(table, getSessionId(), pf), getHost());
-    }
-
-    public void startNext()
-    {
-        assert files.containsKey(currentFile);
-        files.get(currentFile).sstable.releaseReference();
-        files.remove(currentFile);
-        Iterator<PendingFile> iter = files.values().iterator();
-        if (iter.hasNext())
-            streamFile(iter.next());
-    }
-
-    protected void closeInternal(boolean success)
-    {
-        // Release reference on last file (or any uncompleted ones)
-        for (PendingFile file : files.values())
-            file.sstable.releaseReference();
-        streams.remove(sessionId);
-    }
-
-    /** convenience method for use when testing */
-    void await() throws InterruptedException
-    {
-        while (streams.containsKey(sessionId))
-            Thread.sleep(10);
-    }
-
-    public Collection<PendingFile> getFiles()
-    {
-        return files.values();
-    }
-
-    public static Set<InetAddress> getDestinations()
-    {
-        Set<InetAddress> hosts = new HashSet<InetAddress>();
-        for (StreamOutSession session : streams.values())
-        {
-            hosts.add(session.getHost());
-        }
-        return hosts;
-    }
-
-    public static List<PendingFile> getOutgoingFiles(InetAddress host)
-    {
-        List<PendingFile> list = new ArrayList<PendingFile>();
-        for (Map.Entry<UUID, StreamOutSession> entry : streams.entrySet())
-        {
-            StreamOutSession session = entry.getValue();
-            if (session.getHost().equals(host))
-                list.addAll(session.getFiles());
-        }
-        return list;
-    }
-
-    public void validateCurrentFile(String file)
-    {
-        if (!file.equals(currentFile))
-            throw new IllegalStateException(String.format("target reports current file is %s but is %s", file, currentFile));
-    }
-
-    public void begin()
-    {
-        PendingFile first = files.isEmpty() ? null : files.values().iterator().next();
-        currentFile = first == null ? null : first.getFilename();
-        StreamHeader header = new StreamHeader(table, getSessionId(), first, files.values());
-        logger.info("Streaming to {}", getHost());
-        logger.debug("Files are {}", StringUtils.join(files.values(), ","));
-        MessagingService.instance().stream(header, getHost());
-    }
-}


[5/5] git commit: Streaming 2.0

Posted by sl...@apache.org.
Streaming 2.0

patch by Yukim; reviewed by slebresne for CASSANDRA-5286


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

Branch: refs/heads/trunk
Commit: 51511697254615b570f4162bbcd2baee23a234e9
Parents: 40b6c5d
Author: Yuki Morishita <yu...@apache.org>
Authored: Tue May 28 20:28:13 2013 -0500
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Thu Jun 20 19:05:24 2013 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../org/apache/cassandra/dht/BootStrapper.java  |  19 +-
 .../org/apache/cassandra/dht/RangeStreamer.java |  69 +--
 .../cassandra/hadoop/BulkRecordWriter.java      |  15 +-
 .../cassandra/io/sstable/SSTableLoader.java     | 145 ++---
 .../net/IncomingStreamingConnection.java        |  76 +++
 .../cassandra/net/IncomingTcpConnection.java    |  54 +-
 .../apache/cassandra/net/MessagingService.java  | 102 +---
 .../cassandra/service/StorageService.java       | 273 +++------
 .../streaming/AbstractStreamSession.java        | 114 ----
 .../cassandra/streaming/ConnectionHandler.java  | 309 ++++++++++
 .../cassandra/streaming/FileStreamTask.java     | 291 ---------
 .../cassandra/streaming/IStreamCallback.java    |  36 --
 .../streaming/IncomingStreamReader.java         | 200 -------
 .../cassandra/streaming/OperationType.java      |  32 -
 .../apache/cassandra/streaming/PendingFile.java | 180 ------
 .../cassandra/streaming/ProgressInfo.java       | 110 ++++
 .../apache/cassandra/streaming/SessionInfo.java | 187 ++++++
 .../apache/cassandra/streaming/StreamEvent.java |  81 +++
 .../cassandra/streaming/StreamEventHandler.java |  31 +
 .../cassandra/streaming/StreamException.java    |  35 ++
 .../cassandra/streaming/StreamHeader.java       |  94 ---
 .../apache/cassandra/streaming/StreamIn.java    |  89 ---
 .../cassandra/streaming/StreamInSession.java    | 251 --------
 .../cassandra/streaming/StreamManager.java      |  94 +++
 .../cassandra/streaming/StreamManagerMBean.java |  32 +
 .../apache/cassandra/streaming/StreamOut.java   | 187 ------
 .../cassandra/streaming/StreamOutSession.java   | 164 -----
 .../apache/cassandra/streaming/StreamPlan.java  | 169 ++++++
 .../cassandra/streaming/StreamReader.java       | 120 ++++
 .../cassandra/streaming/StreamReceiveTask.java  |  95 +++
 .../apache/cassandra/streaming/StreamReply.java |  91 ---
 .../streaming/StreamReplyVerbHandler.java       |  63 --
 .../cassandra/streaming/StreamRequest.java      | 174 ++----
 .../streaming/StreamRequestVerbHandler.java     |  47 --
 .../cassandra/streaming/StreamResultFuture.java | 172 ++++++
 .../cassandra/streaming/StreamSession.java      | 591 +++++++++++++++++++
 .../apache/cassandra/streaming/StreamState.java |  53 ++
 .../cassandra/streaming/StreamSummary.java      | 107 ++++
 .../apache/cassandra/streaming/StreamTask.java  |  55 ++
 .../cassandra/streaming/StreamTransferTask.java |  83 +++
 .../cassandra/streaming/StreamWriter.java       | 158 +++++
 .../streaming/StreamingRepairTask.java          |  75 +--
 .../cassandra/streaming/StreamingService.java   | 110 ----
 .../streaming/StreamingServiceMBean.java        |  41 --
 .../compress/CompressedFileStreamTask.java      | 160 -----
 .../compress/CompressedInputStream.java         |   7 +-
 .../compress/CompressedStreamReader.java        | 112 ++++
 .../compress/CompressedStreamWriter.java        | 129 ++++
 .../streaming/messages/CompleteMessage.java     |  42 ++
 .../streaming/messages/FileMessage.java         | 106 ++++
 .../streaming/messages/FileMessageHeader.java   | 151 +++++
 .../streaming/messages/PrepareMessage.java      |  91 +++
 .../streaming/messages/RetryMessage.java        |  57 ++
 .../messages/SessionFailedMessage.java          |  42 ++
 .../streaming/messages/StreamInitMessage.java   | 115 ++++
 .../streaming/messages/StreamMessage.java       | 109 ++++
 .../org/apache/cassandra/tools/BulkLoader.java  | 151 ++---
 .../org/apache/cassandra/tools/NodeCmd.java     |  61 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |  29 +-
 .../apache/cassandra/dht/BootStrapperTest.java  |   3 +-
 .../apache/cassandra/service/RemoveTest.java    |  38 +-
 .../cassandra/streaming/BootstrapTest.java      |  54 --
 .../cassandra/streaming/SerializationsTest.java | 220 -------
 .../cassandra/streaming/SessionInfoTest.java    |  73 +++
 .../apache/cassandra/streaming/StreamUtil.java  |  46 --
 .../streaming/StreamingTransferTest.java        | 198 ++++---
 .../compress/CompressedInputStreamTest.java     |   5 +-
 68 files changed, 4068 insertions(+), 3406 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 7e1bf77..2b21d05 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -60,6 +60,7 @@
  * cqlsh: Add row count to SELECT output (CASSANDRA-5636)
  * Include a timestamp with all read commands to determine column expiration
    (CASSANDRA-5149)
+ * Streaming 2.0 (CASSANDRA-5286)
 
 1.2.6
  * Fix cross-DC mutation forwarding (CASSANDRA-5632)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/dht/BootStrapper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/BootStrapper.java b/src/java/org/apache/cassandra/dht/BootStrapper.java
index 28615cf..55d82e1 100644
--- a/src/java/org/apache/cassandra/dht/BootStrapper.java
+++ b/src/java/org/apache/cassandra/dht/BootStrapper.java
@@ -22,6 +22,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.util.*;
+import java.util.concurrent.ExecutionException;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -36,7 +37,6 @@ import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.streaming.OperationType;
 
 public class BootStrapper
 {
@@ -63,7 +63,7 @@ public class BootStrapper
         if (logger.isDebugEnabled())
             logger.debug("Beginning bootstrap process");
 
-        RangeStreamer streamer = new RangeStreamer(tokenMetadata, address, OperationType.BOOTSTRAP);
+        RangeStreamer streamer = new RangeStreamer(tokenMetadata, address, "Bootstrap");
         streamer.addSourceFilter(new RangeStreamer.FailureDetectorSourceFilter(FailureDetector.instance));
 
         for (String table : Schema.instance.getNonSystemTables())
@@ -72,8 +72,19 @@ public class BootStrapper
             streamer.addRanges(table, strategy.getPendingAddressRanges(tokenMetadata, tokens, address));
         }
 
-        streamer.fetch();
-        StorageService.instance.finishBootstrapping();
+        try
+        {
+            streamer.fetchAsync().get();
+            StorageService.instance.finishBootstrapping();
+        }
+        catch (InterruptedException e)
+        {
+            throw new RuntimeException("Interrupted while waiting on boostrap to complete. Bootstrap will have to be restarted.");
+        }
+        catch (ExecutionException e)
+        {
+            throw new RuntimeException("Error during boostrap: " + e.getCause().getMessage(), e.getCause());
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/dht/RangeStreamer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/RangeStreamer.java b/src/java/org/apache/cassandra/dht/RangeStreamer.java
index a0e1a93..2196f4d 100644
--- a/src/java/org/apache/cassandra/dht/RangeStreamer.java
+++ b/src/java/org/apache/cassandra/dht/RangeStreamer.java
@@ -19,14 +19,10 @@ package org.apache.cassandra.dht;
 
 import java.net.InetAddress;
 import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
 
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Multimap;
-import org.apache.cassandra.streaming.IStreamCallback;
-import org.apache.cassandra.utils.FBUtilities;
 import org.apache.commons.lang.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -38,8 +34,9 @@ import org.apache.cassandra.gms.IFailureDetector;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.locator.IEndpointSnitch;
 import org.apache.cassandra.locator.TokenMetadata;
-import org.apache.cassandra.streaming.OperationType;
-import org.apache.cassandra.streaming.StreamIn;
+import org.apache.cassandra.streaming.StreamPlan;
+import org.apache.cassandra.streaming.StreamResultFuture;
+import org.apache.cassandra.utils.FBUtilities;
 
 /**
  * Assists in streaming ranges to a node.
@@ -50,12 +47,10 @@ public class RangeStreamer
 
     private final TokenMetadata metadata;
     private final InetAddress address;
-    private final OperationType opType;
+    private final String description;
     private final Multimap<String, Map.Entry<InetAddress, Collection<Range<Token>>>> toFetch = HashMultimap.create();
     private final Set<ISourceFilter> sourceFilters = new HashSet<ISourceFilter>();
-    // protected for testing.
-    protected CountDownLatch latch;
-    private Set<Range<Token>> completed = Collections.newSetFromMap(new ConcurrentHashMap<Range<Token>, Boolean>());
+    private final StreamPlan streamPlan;
 
     /**
      * A filter applied to sources to stream from when constructing a fetch map.
@@ -104,11 +99,12 @@ public class RangeStreamer
         }
     }
 
-    public RangeStreamer(TokenMetadata metadata, InetAddress address, OperationType opType)
+    public RangeStreamer(TokenMetadata metadata, InetAddress address, String description)
     {
         this.metadata = metadata;
         this.address = address;
-        this.opType = opType;
+        this.description = description;
+        this.streamPlan = new StreamPlan(description);
     }
 
     public void addSourceFilter(ISourceFilter filter)
@@ -123,7 +119,7 @@ public class RangeStreamer
         if (logger.isDebugEnabled())
         {
             for (Map.Entry<Range<Token>, InetAddress> entry: rangesForTable.entries())
-                logger.debug(String.format("%s: range %s exists on %s", opType, entry.getKey(), entry.getValue()));
+                logger.debug(String.format("%s: range %s exists on %s", description, entry.getKey(), entry.getValue()));
         }
 
         for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : getRangeFetchMap(rangesForTable, sourceFilters).asMap().entrySet())
@@ -131,7 +127,7 @@ public class RangeStreamer
             if (logger.isDebugEnabled())
             {
                 for (Range r : entry.getValue())
-                    logger.debug(String.format("%s: range %s from source %s for table %s", opType, r, entry.getKey(), table));
+                    logger.debug(String.format("%s: range %s from source %s for table %s", description, r, entry.getKey(), table));
             }
             toFetch.put(table, entry);
         }
@@ -219,50 +215,19 @@ public class RangeStreamer
         return toFetch;
     }
 
-    public void fetch()
+    public StreamResultFuture fetchAsync()
     {
-        latch = new CountDownLatch(toFetch.entries().size());
-
         for (Map.Entry<String, Map.Entry<InetAddress, Collection<Range<Token>>>> entry : toFetch.entries())
         {
-            final String table = entry.getKey();
-            final InetAddress source = entry.getValue().getKey();
-            final Collection<Range<Token>> ranges = entry.getValue().getValue();
+            String keyspace = entry.getKey();
+            InetAddress source = entry.getValue().getKey();
+            Collection<Range<Token>> ranges = entry.getValue().getValue();
             /* Send messages to respective folks to stream data over to me */
-            IStreamCallback callback = new IStreamCallback()
-            {
-                public void onSuccess()
-                {
-                    completed.addAll(ranges);
-                    latch.countDown();
-                    if (logger.isDebugEnabled())
-                        logger.debug(String.format("Removed %s/%s as a %s source; remaining is %s",
-                                     source, table, opType, latch.getCount()));
-                }
-
-                public void onFailure()
-                {
-                    latch.countDown();
-                    logger.warn("Streaming from " + source + " failed");
-                }
-            };
             if (logger.isDebugEnabled())
-                logger.debug("" + opType + "ing from " + source + " ranges " + StringUtils.join(ranges, ", "));
-            StreamIn.requestRanges(source, table, ranges, callback, opType);
+                logger.debug("" + description + "ing from " + source + " ranges " + StringUtils.join(ranges, ", "));
+            streamPlan.requestRanges(source, keyspace, ranges);
         }
 
-        try
-        {
-            latch.await();
-            for (Map.Entry<String, Map.Entry<InetAddress, Collection<Range<Token>>>> entry : toFetch.entries())
-            {
-                if (!completed.containsAll(entry.getValue().getValue()))
-                    throw new RuntimeException(String.format("Unable to fetch range %s for keyspace %s from any hosts", entry.getValue().getValue(), entry.getKey()));
-            }
-        }
-        catch (InterruptedException e)
-        {
-            throw new AssertionError(e);
-        }
+        return streamPlan.execute();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/hadoop/BulkRecordWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/BulkRecordWriter.java b/src/java/org/apache/cassandra/hadoop/BulkRecordWriter.java
index de6d2e0..704d19f 100644
--- a/src/java/org/apache/cassandra/hadoop/BulkRecordWriter.java
+++ b/src/java/org/apache/cassandra/hadoop/BulkRecordWriter.java
@@ -23,6 +23,8 @@ import java.net.InetAddress;
 import java.nio.ByteBuffer;
 import java.net.UnknownHostException;
 import java.util.*;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
@@ -38,6 +40,7 @@ import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.SSTableLoader;
 import org.apache.cassandra.io.sstable.SSTableSimpleUnsortedWriter;
+import org.apache.cassandra.streaming.StreamState;
 import org.apache.cassandra.thrift.*;
 import org.apache.cassandra.utils.OutputHandler;
 import org.apache.hadoop.conf.Configuration;
@@ -221,7 +224,7 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
         if (writer != null)
         {
             writer.close();
-            SSTableLoader.LoaderFuture future = loader.stream();
+            Future<StreamState> future = loader.stream();
             while (true)
             {
                 try
@@ -229,7 +232,7 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
                     future.get(1000, TimeUnit.MILLISECONDS);
                     break;
                 }
-                catch (TimeoutException te)
+                catch (ExecutionException | TimeoutException te)
                 {
                     progress.progress();
                 }
@@ -238,12 +241,12 @@ implements org.apache.hadoop.mapred.RecordWriter<ByteBuffer,List<Mutation>>
                     throw new IOException(e);
                 }
             }
-            if (future.hadFailures())
+            if (loader.getFailedHosts().size() > 0)
             {
-                if (future.getFailedHosts().size() > maxFailures)
-                    throw new IOException("Too many hosts failed: " + future.getFailedHosts());
+                if (loader.getFailedHosts().size() > maxFailures)
+                    throw new IOException("Too many hosts failed: " + loader.getFailedHosts());
                 else
-                    logger.warn("Some hosts failed: " + future.getFailedHosts());
+                    logger.warn("Some hosts failed: " + loader.getFailedHosts());
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
index 56bb1d4..4d54cd9 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
@@ -22,30 +22,29 @@ import java.io.FilenameFilter;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.util.*;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 
 import org.apache.cassandra.config.Config;
-import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.streaming.*;
-import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.Pair;
 
 /**
  * Cassandra SSTable bulk loader.
  * Load an externally created sstable into a cluster.
  */
-public class SSTableLoader
+public class SSTableLoader implements StreamEventHandler
 {
     private final File directory;
     private final String keyspace;
     private final Client client;
     private final OutputHandler outputHandler;
+    private final Set<InetAddress> failedHosts = new HashSet<>();
 
     static
     {
@@ -109,110 +108,61 @@ public class SSTableLoader
         return sstables;
     }
 
-    public LoaderFuture stream()
+    public StreamResultFuture stream()
     {
         return stream(Collections.<InetAddress>emptySet());
     }
 
-    public LoaderFuture stream(Set<InetAddress> toIgnore)
+    public StreamResultFuture stream(Set<InetAddress> toIgnore)
     {
         client.init(keyspace);
 
+        StreamPlan plan = new StreamPlan("Bulk Load");
         Collection<SSTableReader> sstables = openSSTables();
         if (sstables.isEmpty())
         {
-            outputHandler.output("No sstables to stream");
-            return new LoaderFuture(0);
+            // return empty result
+            return plan.execute();
         }
-
         Map<InetAddress, Collection<Range<Token>>> endpointToRanges = client.getEndpointToRangesMap();
-        outputHandler.output(String.format("Streaming revelant part of %s to %s", names(sstables), endpointToRanges.keySet()));
+        outputHandler.output(String.format("Streaming relevant part of %sto %s", names(sstables), endpointToRanges.keySet()));
 
-        // There will be one streaming session by endpoint
-        LoaderFuture future = new LoaderFuture(endpointToRanges.size());
         for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : endpointToRanges.entrySet())
         {
             InetAddress remote = entry.getKey();
             if (toIgnore.contains(remote))
-            {
-                future.latch.countDown();
                 continue;
-            }
             Collection<Range<Token>> ranges = entry.getValue();
-            StreamOutSession session = StreamOutSession.create(keyspace, remote, new CountDownCallback(future, remote));
             // transferSSTables assumes references have been acquired
             SSTableReader.acquireReferences(sstables);
-            StreamOut.transferSSTables(session, sstables, ranges, OperationType.BULK_LOAD);
-            future.setPendings(remote, session.getFiles());
+            plan.transferFiles(remote, ranges, sstables);
         }
-        return future;
+        StreamResultFuture bulkResult = plan.execute();
+        bulkResult.addEventListener(this);
+        return bulkResult;
     }
 
-    public static class LoaderFuture implements Future<Void>
-    {
-        final CountDownLatch latch;
-        final Map<InetAddress, Collection<PendingFile>> pendingFiles;
-        private List<InetAddress> failedHosts = new ArrayList<InetAddress>();
-
-        private LoaderFuture(int request)
-        {
-            latch = new CountDownLatch(request);
-            pendingFiles = new HashMap<InetAddress, Collection<PendingFile>>();
-        }
-
-        private void setPendings(InetAddress remote, Collection<PendingFile> files)
-        {
-            pendingFiles.put(remote, new ArrayList(files));
-        }
+    public void onSuccess(StreamState finalState) {}
+    public void onFailure(Throwable t) {}
 
-        private void setFailed(InetAddress addr)
-        {
-            failedHosts.add(addr);
-        }
-
-        public List<InetAddress> getFailedHosts()
-        {
-            return failedHosts;
-        }
-
-        public boolean cancel(boolean mayInterruptIfRunning)
-        {
-            throw new UnsupportedOperationException("Cancellation is not yet supported");
-        }
-
-        public Void get() throws InterruptedException
-        {
-            latch.await();
-            return null;
-        }
-
-        public Void get(long timeout, TimeUnit unit) throws InterruptedException, TimeoutException
-        {
-            if (latch.await(timeout, unit))
-                return null;
-            else
-                throw new TimeoutException();
-        }
-
-        public boolean isCancelled()
-        {
-            // For now, cancellation is not supported, maybe one day...
-            return false;
-        }
-
-        public boolean isDone()
-        {
-            return latch.getCount() == 0;
-        }
-
-        public boolean hadFailures()
+    public void handleStreamEvent(StreamEvent event)
+    {
+        if (event.eventType == StreamEvent.Type.FILE_PROGRESS)
         {
-            return failedHosts.size() > 0;
+            ProgressInfo progress = ((StreamEvent.ProgressEvent) event).progress;
+            StringBuilder sb = new StringBuilder("\r");
+            sb.append(progress.fileName);
+            sb.append(": ");
+            sb.append(progress.currentBytes).append("/").append(progress.totalBytes);
+            System.out.print(sb.toString());
+            if (progress.currentBytes == progress.totalBytes)
+                System.out.println();
         }
-
-        public Map<InetAddress, Collection<PendingFile>> getPendingFiles()
+        else if (event.eventType == StreamEvent.Type.STREAM_COMPLETE)
         {
-            return pendingFiles;
+            StreamEvent.SessionCompleteEvent se = (StreamEvent.SessionCompleteEvent) event;
+            if (!se.success)
+                failedHosts.add(se.peer);
         }
     }
 
@@ -224,34 +174,9 @@ public class SSTableLoader
         return builder.toString();
     }
 
-    private class CountDownCallback implements IStreamCallback
+    public Set<InetAddress> getFailedHosts()
     {
-        private final InetAddress endpoint;
-        private final LoaderFuture future;
-
-        CountDownCallback(LoaderFuture future, InetAddress endpoint)
-        {
-            this.future = future;
-            this.endpoint = endpoint;
-        }
-
-        public void onSuccess()
-        {
-            future.latch.countDown();
-            outputHandler.debug(String.format("Streaming session to %s completed (waiting on %d outstanding sessions)", endpoint, future.latch.getCount()));
-
-            // There could be race with stop being called twice but it should be ok
-            if (future.latch.getCount() == 0)
-                client.stop();
-        }
-
-        public void onFailure()
-        {
-            outputHandler.output(String.format("Streaming session to %s failed", endpoint));
-            future.setFailed(endpoint);
-            future.latch.countDown();
-            client.stop();
-        }
+        return failedHosts;
     }
 
     public static abstract class Client

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java b/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
new file mode 100644
index 0000000..dd5b7b4
--- /dev/null
+++ b/src/java/org/apache/cassandra/net/IncomingStreamingConnection.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.net;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.net.Socket;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.streaming.StreamSession;
+import org.apache.cassandra.streaming.messages.StreamInitMessage;
+import org.apache.cassandra.streaming.messages.StreamMessage;
+
+/**
+ * Thread to consume stream init messages.
+ */
+public class IncomingStreamingConnection extends Thread
+{
+    private static final Logger logger = LoggerFactory.getLogger(IncomingStreamingConnection.class);
+
+    private final int version;
+    private final Socket socket;
+
+    public IncomingStreamingConnection(int version, Socket socket)
+    {
+        super("stream-init " + socket.getRemoteSocketAddress());
+        this.version = version;
+        this.socket = socket;
+    }
+
+    @Override
+    public void run()
+    {
+        try
+        {
+            // streaming connections are per-session and have a fixed version.  we can't do anything with a wrong-version stream connection, so drop it.
+            if (version != StreamMessage.CURRENT_VERSION)
+                throw new IOException(String.format("Received stream using protocol version %d (my version %d). Terminating connection", version, MessagingService.current_version));
+
+            DataInput input = new DataInputStream(socket.getInputStream());
+            StreamInitMessage init = StreamInitMessage.serializer.deserialize(input, version);
+
+            StreamSession.startReceivingStreamAsync(init.planId, init.description, socket, version);
+        }
+        catch (IOException e)
+        {
+            logger.debug("IOException reading from socket; closing", e);
+            try
+            {
+                socket.close();
+            }
+            catch (IOException e2)
+            {
+                logger.debug("error closing socket", e2);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
index 1977086..4130006 100644
--- a/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
+++ b/src/java/org/apache/cassandra/net/IncomingTcpConnection.java
@@ -24,24 +24,25 @@ import java.net.SocketException;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.xerial.snappy.SnappyInputStream;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.gms.Gossiper;
-import org.apache.cassandra.io.util.FastByteArrayInputStream;
-import org.apache.cassandra.streaming.IncomingStreamReader;
-import org.apache.cassandra.streaming.StreamHeader;
-import org.xerial.snappy.SnappyInputStream;
 
 public class IncomingTcpConnection extends Thread
 {
     private static final Logger logger = LoggerFactory.getLogger(IncomingTcpConnection.class);
 
+    private final int version;
+    private final boolean compressed;
     private final Socket socket;
     public InetAddress from;
 
-    public IncomingTcpConnection(Socket socket)
+    public IncomingTcpConnection(int version, boolean compressed, Socket socket)
     {
         assert socket != null;
+        this.version = version;
+        this.compressed = compressed;
         this.socket = socket;
         if (DatabaseDescriptor.getInternodeRecvBufferSize() != null)
         {
@@ -66,20 +67,10 @@ public class IncomingTcpConnection extends Thread
     {
         try
         {
-            // determine the connection type to decide whether to buffer
-            DataInputStream in = new DataInputStream(socket.getInputStream());
-            MessagingService.validateMagic(in.readInt());
-            int header = in.readInt();
-            boolean isStream = MessagingService.getBits(header, 3, 1) == 1;
-            int version = MessagingService.getBits(header, 15, 8);
-            logger.debug("Connection version {} from {}", version, socket.getInetAddress());
-
-            if (isStream)
-                handleStream(in, version);
-            else if (version < MessagingService.VERSION_12)
-                handleLegacyVersion(version);
+            if (version < MessagingService.VERSION_12)
+                handleLegacyVersion();
             else
-                handleModernVersion(version, header);
+                handleModernVersion();
         }
         catch (EOFException e)
         {
@@ -96,7 +87,7 @@ public class IncomingTcpConnection extends Thread
         }
     }
 
-    private void handleModernVersion(int version, int header) throws IOException
+    private void handleModernVersion() throws IOException
     {
         DataOutputStream out = new DataOutputStream(socket.getOutputStream());
         out.writeInt(MessagingService.current_version);
@@ -105,7 +96,6 @@ public class IncomingTcpConnection extends Thread
         DataInputStream in = new DataInputStream(socket.getInputStream());
         int maxVersion = in.readInt();
         from = CompactEndpointSerializationHelper.deserialize(in);
-        boolean compressed = MessagingService.getBits(header, 2, 1) == 1;
 
         if (compressed)
         {
@@ -136,28 +126,11 @@ public class IncomingTcpConnection extends Thread
         }
     }
 
-    private void handleLegacyVersion(int version)
+    private void handleLegacyVersion()
     {
         throw new UnsupportedOperationException("Unable to read obsolete message version " + version + "; the earliest version supported is 1.2.0");
     }
 
-    private void handleStream(DataInputStream input, int version) throws IOException
-    {
-        if (version == MessagingService.current_version)
-        {
-            int size = input.readInt();
-            byte[] headerBytes = new byte[size];
-            input.readFully(headerBytes);
-            stream(StreamHeader.serializer.deserialize(new DataInputStream(new FastByteArrayInputStream(headerBytes)), version), input);
-        }
-        else
-        {
-            // streaming connections are per-session and have a fixed version.  we can't do anything with a wrong-version stream connection, so drop it.
-            logger.error("Received stream using protocol version {} (my version {}). Terminating connection",
-                         version, MessagingService.current_version);
-        }
-    }
-
     private InetAddress receiveMessage(DataInputStream input, int version) throws IOException
     {
         int id;
@@ -204,9 +177,4 @@ public class IncomingTcpConnection extends Thread
                 logger.debug("error closing socket", e);
         }
     }
-
-    private void stream(StreamHeader streamHeader, DataInputStream input) throws IOException
-    {
-        new IncomingStreamReader(streamHeader, socket).read();
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/net/MessagingService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/net/MessagingService.java b/src/java/org/apache/cassandra/net/MessagingService.java
index 1569faf..8e94f8e 100644
--- a/src/java/org/apache/cassandra/net/MessagingService.java
+++ b/src/java/org/apache/cassandra/net/MessagingService.java
@@ -17,13 +17,9 @@
  */
 package org.apache.cassandra.net;
 
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOError;
-import java.io.IOException;
+import java.io.*;
 import java.lang.management.ManagementFactory;
 import java.net.*;
-import java.nio.ByteBuffer;
 import java.nio.channels.AsynchronousCloseException;
 import java.nio.channels.ClosedChannelException;
 import java.nio.channels.ServerSocketChannel;
@@ -53,7 +49,6 @@ import org.apache.cassandra.gms.GossipDigestAck;
 import org.apache.cassandra.gms.GossipDigestAck2;
 import org.apache.cassandra.gms.GossipDigestSyn;
 import org.apache.cassandra.io.IVersionedSerializer;
-import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.locator.ILatencySubscriber;
 import org.apache.cassandra.metrics.ConnectionMetrics;
 import org.apache.cassandra.metrics.DroppedMessageMetrics;
@@ -63,7 +58,6 @@ import org.apache.cassandra.service.*;
 import org.apache.cassandra.service.paxos.Commit;
 import org.apache.cassandra.service.paxos.PrepareResponse;
 import org.apache.cassandra.streaming.*;
-import org.apache.cassandra.streaming.compress.CompressedFileStreamTask;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.*;
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
@@ -92,8 +86,8 @@ public final class MessagingService implements MessagingServiceMBean
         REQUEST_RESPONSE, // client-initiated reads and writes
         @Deprecated STREAM_INITIATE,
         @Deprecated STREAM_INITIATE_DONE,
-        STREAM_REPLY,
-        STREAM_REQUEST,
+        @Deprecated STREAM_REPLY,
+        @Deprecated STREAM_REQUEST,
         RANGE_SLICE,
         @Deprecated BOOTSTRAP_TOKEN,
         TREE_REQUEST,
@@ -192,8 +186,6 @@ public final class MessagingService implements MessagingServiceMBean
         put(Verb.MUTATION, RowMutation.serializer);
         put(Verb.READ_REPAIR, RowMutation.serializer);
         put(Verb.READ, ReadCommand.serializer);
-        put(Verb.STREAM_REPLY, StreamReply.serializer);
-        put(Verb.STREAM_REQUEST, StreamRequest.serializer);
         put(Verb.RANGE_SLICE, RangeSliceCommand.serializer);
         put(Verb.BOOTSTRAP_TOKEN, BootStrapper.StringSerializer.instance);
         put(Verb.TREE_REQUEST, ActiveRepairService.TreeRequest.serializer);
@@ -637,34 +629,6 @@ public final class MessagingService implements MessagingServiceMBean
         return iar;
     }
 
-    /**
-     * Stream a file from source to destination. This is highly optimized
-     * to not hold any of the contents of the file in memory.
-     *
-     * @param header Header contains file to stream and other metadata.
-     * @param to     endpoint to which we need to stream the file.
-     */
-
-    public void stream(StreamHeader header, InetAddress to)
-    {
-        DebuggableThreadPoolExecutor executor = streamExecutors.get(to);
-        if (executor == null)
-        {
-            // Using a core pool size of 0 is important. See documentation of streamExecutors.
-            executor = DebuggableThreadPoolExecutor.createWithMaximumPoolSize("Streaming to " + to, 1, 1, TimeUnit.SECONDS);
-            DebuggableThreadPoolExecutor old = streamExecutors.putIfAbsent(to, executor);
-            if (old != null)
-            {
-                executor.shutdown();
-                executor = old;
-            }
-        }
-
-        executor.execute(header.file == null || header.file.compressionInfo == null
-                         ? new FileStreamTask(header, to)
-                         : new CompressedFileStreamTask(header, to));
-    }
-
     public void register(ILatencySubscriber subcriber)
     {
         subscribers.add(subcriber);
@@ -763,44 +727,6 @@ public final class MessagingService implements MessagingServiceMBean
         return packed >>> (start + 1) - count & ~(-1 << count);
     }
 
-    public ByteBuffer constructStreamHeader(StreamHeader streamHeader, boolean compress, int version)
-    {
-        int header = 0;
-        // set compression bit.
-        if (compress)
-            header |= 4;
-        // set streaming bit
-        header |= 8;
-        // Setting up the version bit
-        header |= (version << 8);
-
-        /* Adding the StreamHeader which contains the session Id along
-         * with the pendingfile info for the stream.
-         * | Session Id | Pending File Size | Pending File | Bool more files |
-         * | No. of Pending files | Pending Files ... |
-         */
-        byte[] bytes;
-        try
-        {
-            DataOutputBuffer buffer = new DataOutputBuffer();
-            StreamHeader.serializer.serialize(streamHeader, buffer, version);
-            bytes = buffer.getData();
-        }
-        catch (IOException e)
-        {
-            throw new RuntimeException(e);
-        }
-        assert bytes.length > 0;
-
-        ByteBuffer buffer = ByteBuffer.allocate(4 + 4 + 4 + bytes.length);
-        buffer.putInt(PROTOCOL_MAGIC);
-        buffer.putInt(header);
-        buffer.putInt(bytes.length);
-        buffer.put(bytes);
-        buffer.flip();
-        return buffer;
-    }
-
     /**
      * @return the last version associated with address, or @param version if this is the first such version
      */
@@ -885,9 +811,29 @@ public final class MessagingService implements MessagingServiceMBean
                 {
                     Socket socket = server.accept();
                     if (authenticate(socket))
-                        new IncomingTcpConnection(socket).start();
+                    {
+                        // determine the connection type to decide whether to buffer
+                        DataInputStream in = new DataInputStream(socket.getInputStream());
+                        MessagingService.validateMagic(in.readInt());
+                        int header = in.readInt();
+                        boolean isStream = MessagingService.getBits(header, 3, 1) == 1;
+                        int version = MessagingService.getBits(header, 15, 8);
+                        logger.debug("Connection version {} from {}", version, socket.getInetAddress());
+
+                        if (isStream)
+                        {
+                            new IncomingStreamingConnection(version, socket).start();
+                        }
+                        else
+                        {
+                            boolean compressed = MessagingService.getBits(header, 2, 1) == 1;
+                            new IncomingTcpConnection(version, compressed, socket).start();
+                        }
+                    }
                     else
+                    {
                         socket.close();
+                    }
                 }
                 catch (AsynchronousCloseException e)
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index f6becd0..5e30207 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -37,6 +37,8 @@ import javax.management.ObjectName;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.*;
 import com.google.common.util.concurrent.AtomicDouble;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.Uninterruptibles;
 
 import org.apache.log4j.Level;
@@ -214,6 +216,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         {
             jmxObjectName = new ObjectName("org.apache.cassandra.db:type=StorageService");
             mbs.registerMBean(this, jmxObjectName);
+            mbs.registerMBean(StreamManager.instance, new ObjectName(StreamManager.OBJECT_NAME));
         }
         catch (Exception e)
         {
@@ -232,8 +235,6 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         MessagingService.instance().registerVerbHandlers(MessagingService.Verb.PAXOS_COMMIT, new CommitVerbHandler());
 
         // see BootStrapper for a summary of how the bootstrap verbs interact
-        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.STREAM_REQUEST, new StreamRequestVerbHandler());
-        MessagingService.instance().registerVerbHandlers(MessagingService.Verb.STREAM_REPLY, new StreamReplyVerbHandler());
         MessagingService.instance().registerVerbHandlers(MessagingService.Verb.REPLICATION_FINISHED, new ReplicationFinishedVerbHandler());
         MessagingService.instance().registerVerbHandlers(MessagingService.Verb.REQUEST_RESPONSE, new ResponseVerbHandler());
         MessagingService.instance().registerVerbHandlers(MessagingService.Verb.INTERNAL_RESPONSE, new ResponseVerbHandler());
@@ -253,10 +254,6 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
         MessagingService.instance().registerVerbHandlers(MessagingService.Verb.SNAPSHOT, new SnapshotVerbHandler());
         MessagingService.instance().registerVerbHandlers(MessagingService.Verb.ECHO, new EchoVerbHandler());
-
-        // spin up the streaming service so it is available for jmx tools.
-        if (StreamingService.instance == null)
-            throw new RuntimeException("Streaming service is unavailable.");
     }
 
     public void registerDaemon(CassandraDaemon daemon)
@@ -792,7 +789,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
     {
         logger.info("rebuild from dc: {}", sourceDc == null ? "(any dc)" : sourceDc);
 
-        RangeStreamer streamer = new RangeStreamer(tokenMetadata, FBUtilities.getBroadcastAddress(), OperationType.REBUILD);
+        RangeStreamer streamer = new RangeStreamer(tokenMetadata, FBUtilities.getBroadcastAddress(), "Rebuild");
         streamer.addSourceFilter(new RangeStreamer.FailureDetectorSourceFilter(FailureDetector.instance));
         if (sourceDc != null)
             streamer.addSourceFilter(new RangeStreamer.SingleDatacenterFilter(DatabaseDescriptor.getEndpointSnitch(), sourceDc));
@@ -800,7 +797,20 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         for (String table : Schema.instance.getNonSystemTables())
             streamer.addRanges(table, getLocalRanges(table));
 
-        streamer.fetch();
+        try
+        {
+            streamer.fetchAsync().get();
+        }
+        catch (InterruptedException e)
+        {
+            throw new RuntimeException("Interrupted while waiting on rebuild streaming");
+        }
+        catch (ExecutionException e)
+        {
+            // This is used exclusively through JMX, so log the full trace but only throw a simple RTE
+            logger.error("Error while rebuilding node", e.getCause());
+            throw new RuntimeException("Error while rebuilding node: " + e.getCause().getMessage());
+        }
     }
 
     public void setStreamThroughputMbPerSec(int value)
@@ -1775,7 +1785,6 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      */
     private void restoreReplicaCount(InetAddress endpoint, final InetAddress notifyEndpoint)
     {
-        final Multimap<InetAddress, String> fetchSources = HashMultimap.create();
         Multimap<String, Map.Entry<InetAddress, Collection<Range<Token>>>> rangesToFetch = HashMultimap.create();
 
         final InetAddress myAddress = FBUtilities.getBroadcastAddress();
@@ -1792,40 +1801,37 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             Multimap<InetAddress, Range<Token>> sourceRanges = getNewSourceRanges(table, myNewRanges);
             for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : sourceRanges.asMap().entrySet())
             {
-                fetchSources.put(entry.getKey(), table);
                 rangesToFetch.put(table, entry);
             }
         }
 
+        StreamPlan stream = new StreamPlan("Restore replica count");
         for (final String table : rangesToFetch.keySet())
         {
             for (Map.Entry<InetAddress, Collection<Range<Token>>> entry : rangesToFetch.get(table))
             {
                 final InetAddress source = entry.getKey();
                 Collection<Range<Token>> ranges = entry.getValue();
-                final IStreamCallback callback = new IStreamCallback()
-                {
-                    public void onSuccess()
-                    {
-                        synchronized (fetchSources)
-                        {
-                            fetchSources.remove(source, table);
-                            if (fetchSources.isEmpty())
-                                sendReplicationNotification(notifyEndpoint);
-                        }
-                    }
-
-                    public void onFailure()
-                    {
-                        logger.warn("Streaming from " + source + " failed");
-                        onSuccess(); // calling onSuccess to send notification
-                    }
-                };
                 if (logger.isDebugEnabled())
                     logger.debug("Requesting from " + source + " ranges " + StringUtils.join(ranges, ", "));
-                StreamIn.requestRanges(source, table, ranges, callback, OperationType.RESTORE_REPLICA_COUNT);
+                stream.requestRanges(source, table, ranges);
             }
         }
+        StreamResultFuture future = stream.execute();
+        Futures.addCallback(future, new FutureCallback<StreamState>()
+        {
+            public void onSuccess(StreamState finalState)
+            {
+                sendReplicationNotification(notifyEndpoint);
+            }
+
+            public void onFailure(Throwable t)
+            {
+                logger.warn("Streaming to restore replica count failed", t);
+                // We still want to send the notification
+                sendReplicationNotification(notifyEndpoint);
+            }
+        });
     }
 
     // needs to be modified to accept either a table or ARS.
@@ -2694,17 +2700,17 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
         setMode(Mode.LEAVING, "streaming data to other nodes", true);
 
-        CountDownLatch latch = streamRanges(rangesToStream);
-        CountDownLatch hintsLatch = streamHints();
+        Future<StreamState> streamSuccess = streamRanges(rangesToStream);
+        Future<StreamState> hintsSuccess = streamHints();
 
         // wait for the transfer runnables to signal the latch.
         logger.debug("waiting for stream aks.");
         try
         {
-            latch.await();
-            hintsLatch.await();
+            streamSuccess.get();
+            hintsSuccess.get();
         }
-        catch (InterruptedException e)
+        catch (ExecutionException | InterruptedException e)
         {
             throw new RuntimeException(e);
         }
@@ -2713,10 +2719,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         onFinish.run();
     }
 
-    private CountDownLatch streamHints()
+    private Future<StreamState> streamHints()
     {
         if (HintedHandOffManager.instance.listEndpointsPendingHints().size() == 0)
-            return new CountDownLatch(0);
+            return Futures.immediateFuture(null);
 
         // gather all live nodes in the cluster that aren't also leaving
         List<InetAddress> candidates = new ArrayList<InetAddress>(StorageService.instance.getTokenMetadata().cloneAfterAllLeft().getAllEndpoints());
@@ -2731,7 +2737,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         if (candidates.isEmpty())
         {
             logger.warn("Unable to stream hints since no live endpoints seen");
-            return new CountDownLatch(0);
+            return Futures.immediateFuture(null);
         }
         else
         {
@@ -2743,14 +2749,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             Token token = StorageService.getPartitioner().getMinimumToken();
             List<Range<Token>> ranges = Collections.singletonList(new Range<Token>(token, token));
 
-            CountDownLatch latch = new CountDownLatch(1);
-            StreamOut.transferRanges(hintsDestinationHost,
-                                     Table.open(Table.SYSTEM_KS),
-                                     Collections.singletonList(Table.open(Table.SYSTEM_KS).getColumnFamilyStore(SystemTable.HINTS_CF)),
-                                     ranges,
-                                     new CountingDownStreamCallback(latch, hintsDestinationHost),
-                                     OperationType.UNBOOTSTRAP);
-            return latch;
+            return new StreamPlan("Hints").transferRanges(hintsDestinationHost,
+                                                                      Table.SYSTEM_KS,
+                                                                      ranges,
+                                                                      SystemTable.HINTS_CF)
+                                                      .execute();
         }
     }
 
@@ -2812,25 +2815,19 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         if (relocator.streamsNeeded())
         {
             setMode(Mode.MOVING, "fetching new ranges and streaming old ranges", true);
-
-            relocator.logStreamsMap("[Move->STREAMING]");
-            CountDownLatch streamLatch = relocator.streams();
-
-            relocator.logRequestsMap("[Move->FETCHING]");
-            CountDownLatch fetchLatch = relocator.requests();
-
             try
             {
-                streamLatch.await();
-                fetchLatch.await();
+                relocator.stream().get();
             }
-            catch (InterruptedException e)
+            catch (ExecutionException | InterruptedException e)
             {
-                throw new RuntimeException("Interrupted latch while waiting for stream/fetch ranges to finish: " + e.getMessage());
+                throw new RuntimeException("Interrupted while waiting for stream/fetch ranges to finish: " + e.getMessage());
             }
         }
         else
+        {
             setMode(Mode.MOVING, "No ranges to fetch/stream", true);
+        }
 
         setTokens(Collections.singleton(newToken)); // setting new token as we have everything settled
 
@@ -2840,8 +2837,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     private class RangeRelocator
     {
-        private Map<String, Multimap<InetAddress, Range<Token>>> rangesToFetch = new HashMap<String, Multimap<InetAddress, Range<Token>>>();
-        private Map<String, Multimap<Range<Token>, InetAddress>> rangesToStreamByTable = new HashMap<String, Multimap<Range<Token>, InetAddress>>();
+        private StreamPlan streamPlan = new StreamPlan("Bootstrap");
 
         private RangeRelocator(Collection<Token> tokens, List<String> tables)
         {
@@ -2856,15 +2852,15 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             // clone to avoid concurrent modification in calculateNaturalEndpoints
             TokenMetadata tokenMetaClone = tokenMetadata.cloneOnlyTokenMap();
 
-            for (String table : tables)
+            for (String keyspace : tables)
             {
                 for (Token newToken : newTokens)
                 {
                     // replication strategy of the current keyspace (aka table)
-                    AbstractReplicationStrategy strategy = Table.open(table).getReplicationStrategy();
+                    AbstractReplicationStrategy strategy = Table.open(keyspace).getReplicationStrategy();
 
                     // getting collection of the currently used ranges by this keyspace
-                    Collection<Range<Token>> currentRanges = getRangesForEndpoint(table, localAddress);
+                    Collection<Range<Token>> currentRanges = getRangesForEndpoint(keyspace, localAddress);
                     // collection of ranges which this node will serve after move to the new token
                     Collection<Range<Token>> updatedRanges = strategy.getPendingAddressRanges(tokenMetadata, newToken, localAddress);
 
@@ -2895,51 +2891,39 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
                     // calculating endpoints to stream current ranges to if needed
                     // in some situations node will handle current ranges as part of the new ranges
-                    Multimap<Range<Token>, InetAddress> rangeWithEndpoints = HashMultimap.create();
-
+                    Multimap<InetAddress, Range<Token>> endpointRanges = HashMultimap.create();
                     for (Range<Token> toStream : rangesPerTable.left)
                     {
                         Set<InetAddress> currentEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(toStream.right, tokenMetaClone));
                         Set<InetAddress> newEndpoints = ImmutableSet.copyOf(strategy.calculateNaturalEndpoints(toStream.right, tokenMetaCloneAllSettled));
                         logger.debug("Range:" + toStream + "Current endpoints: " + currentEndpoints + " New endpoints: " + newEndpoints);
-                        rangeWithEndpoints.putAll(toStream, Sets.difference(newEndpoints, currentEndpoints));
+                        for (InetAddress address : Sets.difference(newEndpoints, currentEndpoints))
+                            endpointRanges.put(address, toStream);
                     }
 
-                    // associating table with range-to-endpoints map
-                    rangesToStreamByTable.put(table, rangeWithEndpoints);
+                    // stream ranges
+                    for (InetAddress address : endpointRanges.keySet())
+                        streamPlan.transferRanges(address, keyspace, endpointRanges.get(address));
 
+                    // stream requests
                     Multimap<InetAddress, Range<Token>> workMap = RangeStreamer.getWorkMap(rangesToFetchWithPreferredEndpoints);
-                    rangesToFetch.put(table, workMap);
+                    for (InetAddress address : workMap.keySet())
+                        streamPlan.requestRanges(address, keyspace, workMap.get(address));
 
                     if (logger.isDebugEnabled())
-                        logger.debug("Table {}: work map {}.", table, workMap);
+                        logger.debug("Table {}: work map {}.", keyspace, workMap);
                 }
             }
         }
 
-        private void logStreamsMap(String prefix)
-        {
-            logger.debug("{} Work map: {}", prefix, rangesToStreamByTable);
-        }
-
-        private void logRequestsMap(String prefix)
+        public Future<StreamState> stream()
         {
-            logger.debug("{} Work map: {}", prefix, rangesToFetch);
+            return streamPlan.execute();
         }
 
-        private boolean streamsNeeded()
+        public boolean streamsNeeded()
         {
-            return !rangesToStreamByTable.isEmpty() || !rangesToFetch.isEmpty();
-        }
-
-        private CountDownLatch streams()
-        {
-            return streamRanges(rangesToStreamByTable);
-        }
-
-        private CountDownLatch requests()
-        {
-            return requestRanges(rangesToFetch);
+            return !streamPlan.isEmpty();
         }
     }
 
@@ -2995,25 +2979,19 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         if (relocator.streamsNeeded())
         {
             setMode(Mode.RELOCATING, "fetching new ranges and streaming old ranges", true);
-
-            relocator.logStreamsMap("[Relocate->STREAMING]");
-            CountDownLatch streamLatch = relocator.streams();
-
-            relocator.logRequestsMap("[Relocate->FETCHING]");
-            CountDownLatch fetchLatch = relocator.requests();
-
             try
             {
-                streamLatch.await();
-                fetchLatch.await();
+                relocator.stream().get();
             }
-            catch (InterruptedException e)
+            catch (ExecutionException | InterruptedException e)
             {
                 throw new RuntimeException("Interrupted latch while waiting for stream/fetch ranges to finish: " + e.getMessage());
             }
         }
         else
+        {
             setMode(Mode.RELOCATING, "no new ranges to stream/fetch", true);
+        }
 
         Collection<Token> currentTokens = SystemTable.updateLocalTokens(tokens, Collections.<Token>emptyList());
         tokenMetadata.updateNormalTokens(currentTokens, FBUtilities.getBroadcastAddress());
@@ -3420,26 +3398,21 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
      * Seed data to the endpoints that will be responsible for it at the future
      *
      * @param rangesToStreamByTable tables and data ranges with endpoints included for each
-     * @return latch to count down
+     * @return async Future for whether stream was success
      */
-    private CountDownLatch streamRanges(final Map<String, Multimap<Range<Token>, InetAddress>> rangesToStreamByTable)
+    private Future<StreamState> streamRanges(final Map<String, Multimap<Range<Token>, InetAddress>> rangesToStreamByTable)
     {
         // First, we build a list of ranges to stream to each host, per table
         final Map<String, Map<InetAddress, List<Range<Token>>>> sessionsToStreamByTable = new HashMap<String, Map<InetAddress, List<Range<Token>>>>();
-        // The number of stream out sessions we need to start, to be built up as we build sessionsToStreamByTable
-        int sessionCount = 0;
-
         for (Map.Entry<String, Multimap<Range<Token>, InetAddress>> entry : rangesToStreamByTable.entrySet())
         {
+            String keyspace = entry.getKey();
             Multimap<Range<Token>, InetAddress> rangesWithEndpoints = entry.getValue();
 
             if (rangesWithEndpoints.isEmpty())
                 continue;
 
-            final String table = entry.getKey();
-
             Map<InetAddress, List<Range<Token>>> rangesPerEndpoint = new HashMap<InetAddress, List<Range<Token>>>();
-
             for (final Map.Entry<Range<Token>, InetAddress> endPointEntry : rangesWithEndpoints.entries())
             {
                 final Range<Token> range = endPointEntry.getKey();
@@ -3454,12 +3427,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 curRanges.add(range);
             }
 
-            sessionCount += rangesPerEndpoint.size();
-            sessionsToStreamByTable.put(table, rangesPerEndpoint);
+            sessionsToStreamByTable.put(keyspace, rangesPerEndpoint);
         }
 
-        final CountDownLatch latch = new CountDownLatch(sessionCount);
-
+        StreamPlan streamPlan = new StreamPlan("Unbootstrap");
         for (Map.Entry<String, Map<InetAddress, List<Range<Token>>>> entry : sessionsToStreamByTable.entrySet())
         {
             final String table = entry.getKey();
@@ -3471,90 +3442,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 final InetAddress newEndpoint = rangesEntry.getKey();
 
                 // TODO each call to transferRanges re-flushes, this is potentially a lot of waste
-                StreamOut.transferRanges(newEndpoint,
-                                         Table.open(table),
-                                         ranges,
-                                         new CountingDownStreamCallback(latch, newEndpoint),
-                                         OperationType.UNBOOTSTRAP);
-            }
-        }
-        return latch;
-    }
-
-    static class CountingDownStreamCallback implements IStreamCallback
-    {
-        private final CountDownLatch latch;
-        private final InetAddress targetAddr;
-
-        CountingDownStreamCallback(CountDownLatch latch, InetAddress targetAddr)
-        {
-            this.latch = latch;
-            this.targetAddr = targetAddr;
-        }
-
-        public void onSuccess()
-        {
-            latch.countDown();
-        }
-
-        public void onFailure()
-        {
-            logger.warn("Streaming to " + targetAddr + " failed");
-            onSuccess(); // calling onSuccess for latch countdown
-        }
-    }
-
-    /**
-     * Used to request ranges from endpoints in the ring (will block until all data is fetched and ready)
-     * @param ranges ranges to fetch as map of the preferred address and range collection
-     * @return latch to count down
-     */
-    private CountDownLatch requestRanges(final Map<String, Multimap<InetAddress, Range<Token>>> ranges)
-    {
-        final CountDownLatch latch = new CountDownLatch(ranges.keySet().size());
-        for (Map.Entry<String, Multimap<InetAddress, Range<Token>>> entry : ranges.entrySet())
-        {
-            Multimap<InetAddress, Range<Token>> endpointWithRanges = entry.getValue();
-
-            if (endpointWithRanges.isEmpty())
-            {
-                latch.countDown();
-                continue;
-            }
-
-            final String table = entry.getKey();
-            final Set<InetAddress> pending = new HashSet<InetAddress>(endpointWithRanges.keySet());
-
-            // Send messages to respective folks to stream data over to me
-            for (final InetAddress source: endpointWithRanges.keySet())
-            {
-                Collection<Range<Token>> toFetch = endpointWithRanges.get(source);
-
-                final IStreamCallback callback = new IStreamCallback()
-                {
-                    public void onSuccess()
-                    {
-                        pending.remove(source);
-
-                        if (pending.isEmpty())
-                            latch.countDown();
-                    }
-
-                    public void onFailure()
-                    {
-                        logger.warn("Streaming from " + source + " failed");
-                        onSuccess(); // calling onSuccess for latch countdown
-                    }
-                };
-
-                if (logger.isDebugEnabled())
-                    logger.debug("Requesting from " + source + " ranges " + StringUtils.join(toFetch, ", "));
-
-                // sending actual request
-                StreamIn.requestRanges(source, table, toFetch, callback, OperationType.BOOTSTRAP);
+                streamPlan.transferRanges(newEndpoint, table, ranges);
             }
         }
-        return latch;
+        return streamPlan.execute();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/AbstractStreamSession.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/AbstractStreamSession.java b/src/java/org/apache/cassandra/streaming/AbstractStreamSession.java
deleted file mode 100644
index 89fbf5f..0000000
--- a/src/java/org/apache/cassandra/streaming/AbstractStreamSession.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.cassandra.streaming;
-
-import java.net.InetAddress;
-import java.util.UUID;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.slf4j.LoggerFactory;
-import org.slf4j.Logger;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.gms.*;
-
-public abstract class AbstractStreamSession implements IEndpointStateChangeSubscriber, IFailureDetectionEventListener
-{
-    private static final Logger logger = LoggerFactory.getLogger(AbstractStreamSession.class);
-
-    protected final InetAddress host;
-    protected final UUID sessionId;
-    protected String table;
-    protected final IStreamCallback callback;
-    private final AtomicBoolean isClosed = new AtomicBoolean(false);
-
-    protected AbstractStreamSession(String table, InetAddress host, UUID sessionId, IStreamCallback callback)
-    {
-        this.host = host;
-        this.sessionId = sessionId;
-        this.table = table;
-        this.callback = callback;
-        Gossiper.instance.register(this);
-        FailureDetector.instance.registerFailureDetectionEventListener(this);
-    }
-
-    public UUID getSessionId()
-    {
-        return sessionId;
-    }
-
-    public InetAddress getHost()
-    {
-        return host;
-    }
-
-    public void close(boolean success)
-    {
-        if (!isClosed.compareAndSet(false, true))
-        {
-            logger.debug("Stream session {} already closed", getSessionId());
-            return;
-        }
-
-        closeInternal(success);
-
-        Gossiper.instance.unregister(this);
-        FailureDetector.instance.unregisterFailureDetectionEventListener(this);
-
-        logger.debug("closing with status " + success);
-        if (callback != null)
-        {
-            if (success)
-                callback.onSuccess();
-            else
-                callback.onFailure();
-        }
-    }
-
-    protected abstract void closeInternal(boolean success);
-
-    public void onJoin(InetAddress endpoint, EndpointState epState) {}
-    public void onChange(InetAddress endpoint, ApplicationState state, VersionedValue value) {}
-    public void onAlive(InetAddress endpoint, EndpointState state) {}
-    public void onDead(InetAddress endpoint, EndpointState state) {}
-
-    public void onRemove(InetAddress endpoint)
-    {
-        convict(endpoint, Double.MAX_VALUE);
-    }
-
-    public void onRestart(InetAddress endpoint, EndpointState epState)
-    {
-        convict(endpoint, Double.MAX_VALUE);
-    }
-
-    public void convict(InetAddress endpoint, double phi)
-    {
-        if (!endpoint.equals(getHost()))
-            return;
-
-        // We want a higher confidence in the failure detection than usual because failing a streaming wrongly has a high cost.
-        if (phi < 2 * DatabaseDescriptor.getPhiConvictThreshold())
-            return;
-
-        logger.error("Stream failed because {} died or was restarted/removed (streams may still be active "
-                      + "in background, but further streams won't be started)", endpoint);
-        close(false);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/51511697/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/ConnectionHandler.java b/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
new file mode 100644
index 0000000..27ea5af
--- /dev/null
+++ b/src/java/org/apache/cassandra/streaming/ConnectionHandler.java
@@ -0,0 +1,309 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.streaming;
+
+import java.io.IOException;
+import java.net.Socket;
+import java.net.SocketException;
+import java.nio.channels.Channels;
+import java.nio.channels.ReadableByteChannel;
+import java.nio.channels.SocketChannel;
+import java.nio.channels.WritableByteChannel;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.concurrent.PriorityBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.streaming.messages.StreamInitMessage;
+import org.apache.cassandra.streaming.messages.StreamMessage;
+
+/**
+ * ConnectionHandler manages incoming/outgoing message exchange for the {@link StreamSession}.
+ *
+ * <p>
+ * Internally, ConnectionHandler manages thread to receive incoming {@link StreamMessage} and thread to
+ * send outgoing message. Messages are encoded/decoded on those thread and handed to
+ * {@link StreamSession#messageReceived(org.apache.cassandra.streaming.messages.StreamMessage)}.
+ */
+public class ConnectionHandler
+{
+    private static final Logger logger = LoggerFactory.getLogger(ConnectionHandler.class);
+
+    private static final int MAX_CONNECT_ATTEMPTS = 3;
+
+    private final StreamSession session;
+    private final int protocolVersion;
+
+    private IncomingMessageHandler incoming;
+    private OutgoingMessageHandler outgoing;
+
+    private boolean connected = false;
+    private Socket socket;
+
+    ConnectionHandler(StreamSession session)
+    {
+        this.session = session;
+        this.protocolVersion = StreamMessage.CURRENT_VERSION;
+    }
+
+    ConnectionHandler(StreamSession session, Socket socket, int protocolVersion)
+    {
+        this.session = session;
+        this.socket = Preconditions.checkNotNull(socket);
+        this.connected = socket.isConnected();
+        this.protocolVersion = protocolVersion;
+    }
+
+    /**
+     * Connect to peer and start exchanging message.
+     * When connect attempt fails, this retries for maximum of MAX_CONNECT_ATTEMPTS times.
+     *
+     * @throws IOException when connection failed.
+     */
+    public void connect() throws IOException
+    {
+        int attempts = 0;
+        while (true)
+        {
+            try
+            {
+                socket = MessagingService.instance().getConnectionPool(session.peer).newSocket();
+                socket.setSoTimeout(DatabaseDescriptor.getStreamingSocketTimeout());
+                break;
+            }
+            catch (IOException e)
+            {
+                if (++attempts >= MAX_CONNECT_ATTEMPTS)
+                    throw e;
+
+                long waitms = DatabaseDescriptor.getRpcTimeout() * (long)Math.pow(2, attempts);
+                logger.warn("Failed attempt " + attempts + " to connect to " + session.peer + ". Retrying in " + waitms + " ms. (" + e + ")");
+                try
+                {
+                    Thread.sleep(waitms);
+                }
+                catch (InterruptedException wtf)
+                {
+                    throw new IOException("interrupted", wtf);
+                }
+            }
+        }
+        // send stream init message
+        SocketChannel channel = socket.getChannel();
+        WritableByteChannel out = channel;
+        // socket channel is null when encrypted(SSL)
+        if (channel == null)
+        {
+            out = Channels.newChannel(socket.getOutputStream());
+        }
+        logger.debug("Sending stream init...");
+        StreamInitMessage message = new StreamInitMessage(session.planId(), session.description());
+        out.write(message.createMessage(false, protocolVersion));
+
+        connected = true;
+
+        start();
+        session.onConnect();
+    }
+
+    public void close()
+    {
+        incoming.terminate();
+        outgoing.terminate();
+        if (socket != null && !isConnected())
+        {
+            try
+            {
+                socket.close();
+            }
+            catch (IOException ignore) {}
+        }
+    }
+
+    /**
+     * Start incoming/outgoing messaging threads.
+     */
+    public void start() throws IOException
+    {
+        SocketChannel channel = socket.getChannel();
+        ReadableByteChannel in = channel;
+        WritableByteChannel out = channel;
+        // socket channel is null when encrypted(SSL)
+        if (channel == null)
+        {
+            in = Channels.newChannel(socket.getInputStream());
+            out = Channels.newChannel(socket.getOutputStream());
+        }
+
+        incoming = new IncomingMessageHandler(session, protocolVersion, in);
+        outgoing = new OutgoingMessageHandler(session, protocolVersion, out);
+
+        // ready to send/receive files
+        new Thread(incoming, "STREAM-IN-" + session.peer).start();
+        new Thread(outgoing, "STREAM-OUT-" + session.peer).start();
+    }
+
+    public boolean isConnected()
+    {
+        return connected;
+    }
+
+    /**
+     * Enqueue messages to be sent.
+     *
+     * @param messages messages to send
+     */
+    public void sendMessages(Collection<? extends StreamMessage> messages)
+    {
+        for (StreamMessage message : messages)
+            sendMessage(message);
+    }
+
+    public void sendMessage(StreamMessage message)
+    {
+        assert isConnected();
+        outgoing.enqueue(message);
+    }
+
+    abstract static class MessageHandler implements Runnable
+    {
+        protected final StreamSession session;
+        protected final int protocolVersion;
+        private volatile boolean terminated;
+
+        protected MessageHandler(StreamSession session, int protocolVersion)
+        {
+            this.session = session;
+            this.protocolVersion = protocolVersion;
+        }
+
+        public void terminate()
+        {
+            terminated = true;
+        }
+
+        public boolean terminated()
+        {
+            return terminated;
+        }
+    }
+
+    /**
+     * Incoming streaming message handler
+     */
+    static class IncomingMessageHandler extends MessageHandler
+    {
+        private final ReadableByteChannel in;
+
+        IncomingMessageHandler(StreamSession session, int protocolVersion, ReadableByteChannel in)
+        {
+            super(session, protocolVersion);
+            this.in = in;
+        }
+
+        public void run()
+        {
+            while (!terminated())
+            {
+                try
+                {
+                    // receive message
+                    StreamMessage message = StreamMessage.deserialize(in, protocolVersion, session);
+                    assert message != null;
+                    session.messageReceived(message);
+                }
+                catch (SocketException e)
+                {
+                    // socket is closed
+                    terminate();
+                }
+                catch (Throwable e)
+                {
+                    session.onError(e);
+                }
+            }
+        }
+    }
+
+    /**
+     * Outgoing file transfer thread
+     */
+    static class OutgoingMessageHandler extends MessageHandler
+    {
+        /*
+         * All out going messages are queued up into messageQueue.
+         * The size will grow when received streaming request.
+         *
+         * Queue is also PriorityQueue so that prior messages can go out fast.
+         */
+        private final PriorityBlockingQueue<StreamMessage> messageQueue = new PriorityBlockingQueue<>(64, new Comparator<StreamMessage>()
+        {
+            public int compare(StreamMessage o1, StreamMessage o2)
+            {
+                return o2.getPriority() - o1.getPriority();
+            }
+        });
+
+        private final WritableByteChannel out;
+
+        OutgoingMessageHandler(StreamSession session, int protocolVersion, WritableByteChannel out)
+        {
+            super(session, protocolVersion);
+            this.out = out;
+        }
+
+        public void enqueue(StreamMessage message)
+        {
+            messageQueue.put(message);
+        }
+
+        public void run()
+        {
+            while (!terminated())
+            {
+                try
+                {
+                    StreamMessage next = messageQueue.poll(1, TimeUnit.SECONDS);
+                    if (next != null)
+                    {
+                        logger.debug("Sending " + next);
+                        StreamMessage.serialize(next, out, protocolVersion, session);
+                        if (next.type == StreamMessage.Type.SESSION_FAILED)
+                            terminate();
+                    }
+                }
+                catch (SocketException e)
+                {
+                    session.onError(e);
+                    terminate();
+                }
+                catch (InterruptedException | IOException e)
+                {
+                    session.onError(e);
+                }
+            }
+        }
+    }
+}